From fccb34a841be9487353991f707a7c734dd074cc3 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 4 Jan 2021 23:18:05 +0800 Subject: [PATCH 01/38] deduplicate relations --- .../sql/catalyst/analysis/Analyzer.scala | 204 ++++++----------- .../analysis/DeduplicateRelations.scala | 210 ++++++++++++++++++ .../spark/sql/catalyst/plans/QueryPlan.scala | 16 +- .../sql/catalyst/analysis/AnalysisTest.scala | 14 +- .../LeftSemiAntiJoinPushDownSuite.scala | 15 +- .../optimizer/NestedColumnAliasingSuite.scala | 2 +- .../RemoveRedundantAliasAndProjectSuite.scala | 4 +- .../joinReorder/JoinReorderSuite.scala | 8 + .../StatsEstimationTestBase.scala | 4 +- 9 files changed, 315 insertions(+), 162 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala 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 947aeb56f6ca2..924a296f82b89 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 @@ -253,6 +253,7 @@ class Analyzer(override val catalogManager: CatalogManager) ResolveTables :: ResolvePartitionSpec :: AddMetadataColumns :: + DeduplicateRelations :: ResolveReferences :: ResolveCreateNamedStruct :: ResolveDeserializer :: @@ -1431,123 +1432,78 @@ class Analyzer(override val catalogManager: CatalogManager) */ object ResolveReferences extends Rule[LogicalPlan] { /** - * Generate a new logical plan for the right child with different expression IDs - * for all conflicting attributes. + * Resolves the attribute and extract value expressions(s) by traversing the + * input expression in top down manner. The traversal is done in top-down manner as + * we need to skip over unbound lambda function expression. The lambda expressions are + * resolved in a different rule [[ResolveLambdaVariables]] + * + * Example : + * SELECT transform(array(1, 2, 3), (x, i) -> x + i)" + * + * In the case above, x and i are resolved as lambda variables in [[ResolveLambdaVariables]] + * + * Note : In this routine, the unresolved attributes are resolved from the input plan's + * children attributes. + * + * @param e The expression need to be resolved. + * @param q The LogicalPlan whose children are used to resolve expression's attribute. + * @param trimAlias When true, trim unnecessary alias of `GetStructField`. Note that, + * we cannot trim the alias of top-level `GetStructField`, as we should + * resolve `UnresolvedAttribute` to a named expression. The caller side + * can trim the alias of top-level `GetStructField` if it's safe to do so. + * @return resolved Expression. */ - private def dedupRight (left: LogicalPlan, right: LogicalPlan): LogicalPlan = { - val conflictingAttributes = left.outputSet.intersect(right.outputSet) - logDebug(s"Conflicting attributes ${conflictingAttributes.mkString(",")} " + - s"between $left and $right") - - /** - * For LogicalPlan likes MultiInstanceRelation, Project, Aggregate, etc, whose output doesn't - * inherit directly from its children, we could just stop collect on it. Because we could - * always replace all the lower conflict attributes with the new attributes from the new - * plan. Theoretically, we should do recursively collect for Generate and Window but we leave - * it to the next batch to reduce possible overhead because this should be a corner case. - */ - def collectConflictPlans(plan: LogicalPlan): Seq[(LogicalPlan, LogicalPlan)] = plan match { - // Handle base relations that might appear more than once. - case oldVersion: MultiInstanceRelation - if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => - val newVersion = oldVersion.newInstance() - newVersion.copyTagsFrom(oldVersion) - Seq((oldVersion, newVersion)) - - case oldVersion: SerializeFromObject - if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => - Seq((oldVersion, oldVersion.copy( - serializer = oldVersion.serializer.map(_.newInstance())))) - - // Handle projects that create conflicting aliases. - case oldVersion @ Project(projectList, _) - if findAliases(projectList).intersect(conflictingAttributes).nonEmpty => - Seq((oldVersion, oldVersion.copy(projectList = newAliases(projectList)))) - - // We don't need to search child plan recursively if the projectList of a Project - // is only composed of Alias and doesn't contain any conflicting attributes. - // Because, even if the child plan has some conflicting attributes, the attributes - // will be aliased to non-conflicting attributes by the Project at the end. - case _ @ Project(projectList, _) - if findAliases(projectList).size == projectList.size => - Nil - - case oldVersion @ Aggregate(_, aggregateExpressions, _) - if findAliases(aggregateExpressions).intersect(conflictingAttributes).nonEmpty => - Seq((oldVersion, oldVersion.copy( - aggregateExpressions = newAliases(aggregateExpressions)))) - - // We don't search the child plan recursively for the same reason as the above Project. - case _ @ Aggregate(_, aggregateExpressions, _) - if findAliases(aggregateExpressions).size == aggregateExpressions.size => - Nil - - case oldVersion @ FlatMapGroupsInPandas(_, _, output, _) - if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => - Seq((oldVersion, oldVersion.copy(output = output.map(_.newInstance())))) - - case oldVersion @ FlatMapCoGroupsInPandas(_, _, _, output, _, _) - if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => - Seq((oldVersion, oldVersion.copy(output = output.map(_.newInstance())))) - - case oldVersion @ MapInPandas(_, output, _) - if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => - Seq((oldVersion, oldVersion.copy(output = output.map(_.newInstance())))) - - case oldVersion: Generate - if oldVersion.producedAttributes.intersect(conflictingAttributes).nonEmpty => - val newOutput = oldVersion.generatorOutput.map(_.newInstance()) - Seq((oldVersion, oldVersion.copy(generatorOutput = newOutput))) - - case oldVersion: Expand - if oldVersion.producedAttributes.intersect(conflictingAttributes).nonEmpty => - val producedAttributes = oldVersion.producedAttributes - val newOutput = oldVersion.output.map { attr => - if (producedAttributes.contains(attr)) { - attr.newInstance() - } else { - attr + private def resolveExpressionTopDown( + e: Expression, + q: LogicalPlan, + trimAlias: Boolean = false): Expression = { + + def innerResolve(e: Expression, isTopLevel: Boolean): Expression = { + if (e.resolved) return e + e match { + case f: LambdaFunction if !f.bound => f + case u @ UnresolvedAttribute(nameParts) => + // Leave unchanged if resolution fails. Hopefully will be resolved next round. + val resolved = + withPosition(u) { + q.resolveChildren(nameParts, resolver) + .orElse(resolveLiteralFunction(nameParts, u, q)) + .getOrElse(u) + } + val result = resolved match { + // As the comment of method `resolveExpressionTopDown`'s param `trimAlias` said, + // when trimAlias = true, we will trim unnecessary alias of `GetStructField` and + // we won't trim the alias of top-level `GetStructField`. Since we will call + // CleanupAliases later in Analyzer, trim non top-level unnecessary alias of + // `GetStructField` here is safe. + case Alias(s: GetStructField, _) if trimAlias && !isTopLevel => s + case others => others } - } - Seq((oldVersion, oldVersion.copy(output = newOutput))) - - case oldVersion @ Window(windowExpressions, _, _, child) - if AttributeSet(windowExpressions.map(_.toAttribute)).intersect(conflictingAttributes) - .nonEmpty => - Seq((oldVersion, oldVersion.copy(windowExpressions = newAliases(windowExpressions)))) - - case oldVersion @ ScriptTransformation(_, _, output, _, _) - if AttributeSet(output).intersect(conflictingAttributes).nonEmpty => - Seq((oldVersion, oldVersion.copy(output = output.map(_.newInstance())))) - - case _ => plan.children.flatMap(collectConflictPlans) + logDebug(s"Resolving $u to $result") + result + case UnresolvedExtractValue(child, fieldExpr) if child.resolved => + ExtractValue(child, fieldExpr, resolver) + case _ => e.mapChildren(innerResolve(_, isTopLevel = false)) + } } - val conflictPlans = collectConflictPlans(right) + innerResolve(e, isTopLevel = true) + } - /* - * Note that it's possible `conflictPlans` can be empty which implies that there - * is a logical plan node that produces new references that this rule cannot handle. - * When that is the case, there must be another rule that resolves these conflicts. - * Otherwise, the analysis will fail. - */ - if (conflictPlans.isEmpty) { - right - } else { - val planMapping = conflictPlans.toMap - right.transformUpWithNewOutput { - case oldPlan => - val newPlanOpt = planMapping.get(oldPlan) - newPlanOpt.map { newPlan => - newPlan -> oldPlan.output.zip(newPlan.output) - }.getOrElse(oldPlan -> Nil) - } + private def hasConflictingAttrs(p: LogicalPlan): Boolean = { + p.children.length > 1 && { + val numDistinctAttrs = p.children.flatMap(_.output.map(_.exprId)).distinct.length + val numOutputAttrs = p.children.map(_.output.length).sum + numDistinctAttrs < numOutputAttrs } } def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { case p: LogicalPlan if !p.childrenResolved => p + // Wait for the rule `DeduplicateRelations` to resolve conflicting attrs first. + case p: LogicalPlan if hasConflictingAttrs(p) => p + // If the projection list contains Stars, expand it. case p: Project if containsStar(p.projectList) => p.copy(projectList = buildExpandedProjectList(p.projectList, p.child)) @@ -1569,37 +1525,12 @@ class Analyzer(override val catalogManager: CatalogManager) case g: Generate if containsStar(g.generator.children) => throw QueryCompilationErrors.invalidStarUsageError("explode/json_tuple/UDTF") - // To resolve duplicate expression IDs for Join and Intersect - case j @ Join(left, right, _, _, _) if !j.duplicateResolved => - j.copy(right = dedupRight(left, right)) case f @ FlatMapCoGroupsInPandas(leftAttributes, rightAttributes, _, _, left, right) => val leftRes = leftAttributes .map(x => resolveExpressionByPlanOutput(x, left).asInstanceOf[Attribute]) val rightRes = rightAttributes .map(x => resolveExpressionByPlanOutput(x, right).asInstanceOf[Attribute]) f.copy(leftAttributes = leftRes, rightAttributes = rightRes) - // intersect/except will be rewritten to join at the beginning of optimizer. Here we need to - // deduplicate the right side plan, so that we won't produce an invalid self-join later. - case i @ Intersect(left, right, _) if !i.duplicateResolved => - i.copy(right = dedupRight(left, right)) - case e @ Except(left, right, _) if !e.duplicateResolved => - e.copy(right = dedupRight(left, right)) - // Only after we finish by-name resolution for Union - case u: Union if !u.byName && !u.duplicateResolved => - // Use projection-based de-duplication for Union to avoid breaking the checkpoint sharing - // feature in streaming. - val newChildren = u.children.foldRight(Seq.empty[LogicalPlan]) { (head, tail) => - head +: tail.map { - case child if head.outputSet.intersect(child.outputSet).isEmpty => - child - case child => - val projectList = child.output.map { attr => - Alias(attr, attr.name)() - } - Project(projectList, child) - } - } - u.copy(children = newChildren) // When resolve `SortOrder`s in Sort based on child, don't report errors as // we still have chance to resolve it based on its descendants @@ -1756,17 +1687,6 @@ class Analyzer(override val catalogManager: CatalogManager) } } - def newAliases(expressions: Seq[NamedExpression]): Seq[NamedExpression] = { - expressions.map { - case a: Alias => Alias(a.child, a.name)() - case other => other - } - } - - def findAliases(projectList: Seq[NamedExpression]): AttributeSet = { - AttributeSet(projectList.collect { case a: Alias => a.toAttribute }) - } - // This method is used to trim groupByExpressions/selectedGroupByExpressions's top-level // GetStructField Alias. Since these expression are not NamedExpression originally, // we are safe to trim top-level GetStructField Alias. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala new file mode 100644 index 0000000000000..caaf20afb955c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -0,0 +1,210 @@ +/* + * 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.ArrayBuffer + +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, AttributeSet, NamedExpression} +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Except, Expand, FlatMapGroupsInPandas, Generate, Intersect, Join, LogicalPlan, Project, SerializeFromObject, Union, Window} +import org.apache.spark.sql.catalyst.rules.Rule + +object DeduplicateRelations extends Rule[LogicalPlan] { + + override def apply(plan: LogicalPlan): LogicalPlan = { + renewDuplicatedRelations(Nil, plan)._1.resolveOperatorsUp { + case p: LogicalPlan if !p.childrenResolved => p + // To resolve duplicate expression IDs for Join. + case j @ Join(left, right, _, _, _) if !j.duplicateResolved => + j.copy(right = dedupRight(left, right)) + // intersect/except will be rewritten to join at the beginning of optimizer. Here we need to + // deduplicate the right side plan, so that we won't produce an invalid self-join later. + case i @ Intersect(left, right, _) if !i.duplicateResolved => + i.copy(right = dedupRight(left, right)) + case e @ Except(left, right, _) if !e.duplicateResolved => + e.copy(right = dedupRight(left, right)) + // Only after we finish by-name resolution for Union + case u: Union if !u.byName && !u.duplicateResolved => + // Use projection-based de-duplication for Union to avoid breaking the checkpoint sharing + // feature in streaming. + val newChildren = u.children.foldRight(Seq.empty[LogicalPlan]) { (head, tail) => + head +: tail.map { + case child if head.outputSet.intersect(child.outputSet).isEmpty => + child + case child => + val projectList = child.output.map { attr => + Alias(attr, attr.name)() + } + Project(projectList, child) + } + } + u.copy(children = newChildren) + } + } + + private def renewDuplicatedRelations( + existingRelations: Seq[MultiInstanceRelation], + plan: LogicalPlan): (LogicalPlan, Seq[MultiInstanceRelation]) = plan match { + case m: MultiInstanceRelation => + if (isDuplicated(existingRelations, m)) { + (m.newInstance(), Nil) + } else { + (m, Seq(m)) + } + + case _ if plan.children.nonEmpty => + val newChildren = ArrayBuffer.empty[LogicalPlan] + val relations = ArrayBuffer.empty[MultiInstanceRelation] + for (c <- plan.children) { + val (renewed, collected) = renewDuplicatedRelations(existingRelations ++ relations, c) + newChildren += renewed + relations ++= collected + } + + if (plan.childrenResolved) { + val attrMap = AttributeMap(plan.children.flatMap(_.output).zip( + newChildren.flatMap(_.output)).filter { case (a1, a2) => a1.exprId != a2.exprId }) + val newPlan = plan.withNewChildren(newChildren).rewriteAttrs(attrMap) + (newPlan, relations) + } else { + (plan.withNewChildren(newChildren), relations) + } + + case _ => (plan, Nil) + } + + private def isDuplicated( + existingRelations: Seq[MultiInstanceRelation], + relation: MultiInstanceRelation): Boolean = { + existingRelations.exists { er => + er.asInstanceOf[LogicalPlan].outputSet + .intersect(relation.asInstanceOf[LogicalPlan].outputSet).nonEmpty + } + } + + /** + * Generate a new logical plan for the right child with different expression IDs + * for all conflicting attributes. + */ + private def dedupRight (left: LogicalPlan, right: LogicalPlan): LogicalPlan = { + val conflictingAttributes = left.outputSet.intersect(right.outputSet) + logDebug(s"Conflicting attributes ${conflictingAttributes.mkString(",")} " + + s"between $left and $right") + + /** + * For LogicalPlan likes MultiInstanceRelation, Project, Aggregate, etc, whose output doesn't + * inherit directly from its children, we could just stop collect on it. Because we could + * always replace all the lower conflict attributes with the new attributes from the new + * plan. Theoretically, we should do recursively collect for Generate and Window but we leave + * it to the next batch to reduce possible overhead because this should be a corner case. + */ + def collectConflictPlans(plan: LogicalPlan): Seq[(LogicalPlan, LogicalPlan)] = plan match { + // Handle base relations that might appear more than once. + case oldVersion: SerializeFromObject + if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => + Seq((oldVersion, oldVersion.copy( + serializer = oldVersion.serializer.map(_.newInstance())))) + + // Handle projects that create conflicting aliases. + case oldVersion @ Project(projectList, _) + if findAliases(projectList).intersect(conflictingAttributes).nonEmpty => + Seq((oldVersion, oldVersion.copy(projectList = newAliases(projectList)))) + + // We don't need to search child plan recursively if the projectList of a Project + // is only composed of Alias and doesn't contain any conflicting attributes. + // Because, even if the child plan has some conflicting attributes, the attributes + // will be aliased to non-conflicting attributes by the Project at the end. + case _ @ Project(projectList, _) + if findAliases(projectList).size == projectList.size => + Nil + + case oldVersion @ Aggregate(_, aggregateExpressions, _) + if findAliases(aggregateExpressions).intersect(conflictingAttributes).nonEmpty => + Seq((oldVersion, oldVersion.copy( + aggregateExpressions = newAliases(aggregateExpressions)))) + + // We don't search the child plan recursively for the same reason as the above Project. + case _ @ Aggregate(_, aggregateExpressions, _) + if findAliases(aggregateExpressions).size == aggregateExpressions.size => + Nil + + case oldVersion @ FlatMapGroupsInPandas(_, _, output, _) + if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => + Seq((oldVersion, oldVersion.copy(output = output.map(_.newInstance())))) + + case oldVersion: Generate + if oldVersion.producedAttributes.intersect(conflictingAttributes).nonEmpty => + val newOutput = oldVersion.generatorOutput.map(_.newInstance()) + Seq((oldVersion, oldVersion.copy(generatorOutput = newOutput))) + + case oldVersion: Expand + if oldVersion.producedAttributes.intersect(conflictingAttributes).nonEmpty => + val producedAttributes = oldVersion.producedAttributes + val newOutput = oldVersion.output.map { attr => + if (producedAttributes.contains(attr)) { + attr.newInstance() + } else { + attr + } + } + Seq((oldVersion, oldVersion.copy(output = newOutput))) + + case oldVersion @ Window(windowExpressions, _, _, child) + if AttributeSet(windowExpressions.map(_.toAttribute)).intersect(conflictingAttributes) + .nonEmpty => + Seq((oldVersion, oldVersion.copy(windowExpressions = newAliases(windowExpressions)))) + + case oldVersion @ ScriptTransformation(_, _, output, _, _) + if AttributeSet(output).intersect(conflictingAttributes).nonEmpty => + Seq((oldVersion, oldVersion.copy(output = output.map(_.newInstance())))) + + case _ => plan.children.flatMap(collectConflictPlans) + } + + val conflictPlans = collectConflictPlans(right) + + /* + * Note that it's possible `conflictPlans` can be empty which implies that there + * is a logical plan node that produces new references that this rule cannot handle. + * When that is the case, there must be another rule that resolves these conflicts. + * Otherwise, the analysis will fail. + */ + if (conflictPlans.isEmpty) { + right + } else { + val planMapping = conflictPlans.toMap + right.transformUpWithNewOutput { + case oldPlan => + val newPlanOpt = planMapping.get(oldPlan) + newPlanOpt.map { newPlan => + newPlan -> oldPlan.output.zip(newPlan.output) + }.getOrElse(oldPlan -> Nil) + } + } + } + + private def newAliases(expressions: Seq[NamedExpression]): Seq[NamedExpression] = { + expressions.map { + case a: Alias => Alias(a.child, a.name)() + case other => other + } + } + + private def findAliases(projectList: Seq[NamedExpression]): AttributeSet = { + AttributeSet(projectList.collect { case a: Alias => a.toAttribute }) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 89e8c191acc84..aa72c39e3cb67 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -209,12 +209,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] val attributeRewrites = AttributeMap(attrMappingForCurrentPlan.toSeq) // Using attrMapping from the children plans to rewrite their parent node. // Note that we shouldn't rewrite a node using attrMapping from its sibling nodes. - newPlan = newPlan.transformExpressions { - case a: AttributeReference => - updateAttr(a, attributeRewrites) - case pe: PlanExpression[PlanType] => - pe.withNewPlan(updateOuterReferencesInSubquery(pe.plan, attributeRewrites)) - } + newPlan = newPlan.rewriteAttrs(attributeRewrites) } val (planAfterRule, newAttrMapping) = CurrentOrigin.withOrigin(origin) { @@ -252,6 +247,15 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] rewrite(this)._1 } + def rewriteAttrs(attrMap: AttributeMap[Attribute]): PlanType = { + transformExpressions { + case a: AttributeReference => + updateAttr(a, attrMap) + case pe: PlanExpression[PlanType] => + pe.withNewPlan(updateOuterReferencesInSubquery(pe.plan, attrMap)) + }.asInstanceOf[PlanType] + } + private def updateAttr(attr: Attribute, attrMap: AttributeMap[Attribute]): Attribute = { val exprId = attrMap.getOrElse(attr, attr).exprId attr.withExprId(exprId) 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 8102d85fb01b2..e4c88df882d9d 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 @@ -74,13 +74,13 @@ trait AnalysisTest extends PlanTest { catalog.createDatabase( CatalogDatabase("default", "", new URI("loc"), Map.empty), ignoreIfExists = false) - createTempView(catalog, "TaBlE", TestRelations.testRelation, overrideIfExists = true) - createTempView(catalog, "TaBlE2", TestRelations.testRelation2, overrideIfExists = true) - createTempView(catalog, "TaBlE3", TestRelations.testRelation3, overrideIfExists = true) - createGlobalTempView(catalog, "TaBlE4", TestRelations.testRelation4, overrideIfExists = true) - createGlobalTempView(catalog, "TaBlE5", TestRelations.testRelation5, overrideIfExists = true) + catalog.createTempView("TaBlE", TestRelations.testRelation, overrideIfExists = true) + catalog.createTempView("TaBlE2", TestRelations.testRelation2, overrideIfExists = true) + catalog.createTempView("TaBlE3", TestRelations.testRelation3, overrideIfExists = true) + catalog.createGlobalTempView("TaBlE4", TestRelations.testRelation4, overrideIfExists = true) + catalog.createGlobalTempView("TaBlE5", TestRelations.testRelation5, overrideIfExists = true) new Analyzer(catalog) { - override val extendedResolutionRules = EliminateSubqueryAliases +: extendedAnalysisRules + override val extendedResolutionRules = extendedAnalysisRules } } @@ -91,7 +91,7 @@ trait AnalysisTest extends PlanTest { withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { val analyzer = getAnalyzer val actualPlan = analyzer.executeAndCheck(inputPlan, new QueryPlanningTracker) - comparePlans(actualPlan, expectedPlan) + comparePlans(EliminateSubqueryAliases(actualPlan), expectedPlan) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala index 1672c6d91660a..88c29c9274a68 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala @@ -240,14 +240,23 @@ class LeftSemiPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } - test("Union: LeftSemiAnti join no pushdown in self join scenario") { + test("Union: LeftSemiAnti join pushdown in self join scenario") { val testRelation2 = LocalRelation('x.int, 'y.int, 'z.int) + val attrX = testRelation2.output.head val originalQuery = Union(Seq(testRelation, testRelation2)) - .join(testRelation2, joinType = LeftSemi, condition = Some('a === 'x)) + .join(testRelation2, joinType = LeftSemi, condition = Some('a === attrX)) val optimized = Optimize.execute(originalQuery.analyze) - comparePlans(optimized, originalQuery.analyze) + + val correctAnswer = Union(Seq( + testRelation.join(testRelation2, joinType = LeftSemi, condition = Some('a === 'x)), + // We can't construct the actual query, as relations deduplication will create new attribute + // IDs. Here we use a fake join condition (always true) to verify the query plan shape. + testRelation2.join(testRelation2, joinType = LeftSemi, condition = Some(attrX === attrX)))) + .analyze + + comparePlans(optimized, correctAnswer) } test("Unary: LeftSemiAnti join pushdown") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala index c83ab375ee15a..0ae4d3f6e6801 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala @@ -161,7 +161,7 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { comparePlans(optimized, expected) } val expectedUnion = - contact.select('name).union(contact.select('name.as('name))) + contact.select('name).union(contact.select('name)) .select(GetStructField('name, 1, Some("middle"))).analyze comparePlans(optimizedUnion, expectedUnion) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala index 2e0ab7f64f4d6..4b02a847880f7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala @@ -83,12 +83,12 @@ class RemoveRedundantAliasAndProjectSuite extends PlanTest with PredicateHelper comparePlans(optimized, query) } - test("retain deduplicating alias in self-join") { + test("remove redundant project with self-join") { val relation = LocalRelation('a.int) val fragment = relation.select('a as 'a) val query = fragment.select('a as 'a).join(fragment.select('a as 'a)).analyze val optimized = Optimize.execute(query) - val expected = relation.join(relation.select('a as 'a)).analyze + val expected = relation.join(relation).analyze comparePlans(optimized, expected) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala index 2e1cf4a137e25..792f9643a3aa5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala @@ -378,4 +378,12 @@ class JoinReorderSuite extends JoinReorderPlanTestBase with StatsEstimationTestB assert(plan1.betterThan(plan2, conf)) assert(!plan2.betterThan(plan1, conf)) } + + test("with self-join") { + val query = t2.join(t1, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .select(nameToAttr("t1.v-1-10")) + .join(t2, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t2.k-1-5"))) + println(query.analyze.treeString) + println(Optimize.execute(query.analyze)) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala index 0a27e31b3c9f6..17d182ba172ba 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.statsEstimation import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.internal.SQLConf @@ -72,11 +73,12 @@ case class StatsTestPlan( outputList: Seq[Attribute], rowCount: BigInt, attributeStats: AttributeMap[ColumnStat], - size: Option[BigInt] = None) extends LeafNode { + size: Option[BigInt] = None) extends LeafNode with MultiInstanceRelation { override def output: Seq[Attribute] = outputList override def computeStats(): Statistics = Statistics( // If sizeInBytes is useless in testing, we just use a fake value sizeInBytes = size.getOrElse(Int.MaxValue), rowCount = Some(rowCount), attributeStats = attributeStats) + override def newInstance(): LogicalPlan = copy(outputList = outputList.map(_.newInstance())) } From 2ed60b060805b959e8f0f057b3ac30b976c5f7fa Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 4 Feb 2021 16:25:51 +0800 Subject: [PATCH 02/38] fix --- .../apache/spark/sql/catalyst/analysis/Analyzer.scala | 6 +++--- .../sql/catalyst/analysis/DeduplicateRelations.scala | 6 +++++- .../org/apache/spark/sql/catalyst/plans/QueryPlan.scala | 2 +- .../sql/catalyst/plans/logical/AnalysisHelper.scala | 9 ++++++++- .../optimizer/joinReorder/JoinReorderSuite.scala | 9 +++++---- 5 files changed, 22 insertions(+), 10 deletions(-) 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 924a296f82b89..75816ffa1c11c 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 @@ -1492,9 +1492,9 @@ class Analyzer(override val catalogManager: CatalogManager) private def hasConflictingAttrs(p: LogicalPlan): Boolean = { p.children.length > 1 && { - val numDistinctAttrs = p.children.flatMap(_.output.map(_.exprId)).distinct.length - val numOutputAttrs = p.children.map(_.output.length).sum - numDistinctAttrs < numOutputAttrs + p.children.tail.foldLeft(p.children.head.outputSet) { + case (conflictAttrs, child) => conflictAttrs.intersect(child.outputSet) + }.nonEmpty } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index caaf20afb955c..aaa8474969cb0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -59,9 +59,13 @@ object DeduplicateRelations extends Rule[LogicalPlan] { private def renewDuplicatedRelations( existingRelations: Seq[MultiInstanceRelation], plan: LogicalPlan): (LogicalPlan, Seq[MultiInstanceRelation]) = plan match { + case p: LogicalPlan if p.isStreaming => (plan, Nil) + case m: MultiInstanceRelation => if (isDuplicated(existingRelations, m)) { - (m.newInstance(), Nil) + val newNode = m.newInstance() + newNode.copyTagsFrom(m) + (newNode, Nil) } else { (m, Seq(m)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index aa72c39e3cb67..58d9be6a19483 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -265,7 +265,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] * The outer plan may have old references and the function below updates the * outer references to refer to the new attributes. */ - private def updateOuterReferencesInSubquery( + protected def updateOuterReferencesInSubquery( plan: PlanType, attrMap: AttributeMap[Attribute]): PlanType = { plan.transformDown { case currentFragment => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AnalysisHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AnalysisHelper.scala index 54b01416381c6..f19280e525465 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AnalysisHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AnalysisHelper.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.util.Utils @@ -142,6 +142,13 @@ trait AnalysisHelper extends QueryPlan[LogicalPlan] { self: LogicalPlan => } } + override def updateOuterReferencesInSubquery(plan: LogicalPlan, attrMap: AttributeMap[Attribute]) + : LogicalPlan = { + AnalysisHelper.allowInvokingTransformsInAnalyzer { + super.updateOuterReferencesInSubquery(plan, attrMap) + } + } + /** * Recursively transforms the expressions of a tree, skipping nodes that have already * been analyzed. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala index 792f9643a3aa5..8968fd86024a0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala @@ -379,11 +379,12 @@ class JoinReorderSuite extends JoinReorderPlanTestBase with StatsEstimationTestB assert(!plan2.betterThan(plan1, conf)) } - test("with self-join") { - val query = t2.join(t1, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + test("join reorder with self-join") { + val plan = t2.join(t1, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) .select(nameToAttr("t1.v-1-10")) .join(t2, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t2.k-1-5"))) - println(query.analyze.treeString) - println(Optimize.execute(query.analyze)) + + // this can fail before the fix + Optimize.execute(plan.analyze) } } From 06bb709aea902028c25ea9b64223c7fc8126e9f7 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 4 Feb 2021 21:30:26 +0800 Subject: [PATCH 03/38] fix SPARK-34319 --- .../sql/catalyst/analysis/DeduplicateRelations.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index aaa8474969cb0..f17941d9f7617 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, AttributeSet, NamedExpression} -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Except, Expand, FlatMapGroupsInPandas, Generate, Intersect, Join, LogicalPlan, Project, SerializeFromObject, Union, Window} +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Except, Expand, FlatMapCoGroupsInPandas, FlatMapGroupsInPandas, Generate, Intersect, Join, LogicalPlan, MapInPandas, Project, SerializeFromObject, Union, Window} import org.apache.spark.sql.catalyst.rules.Rule object DeduplicateRelations extends Rule[LogicalPlan] { @@ -150,6 +150,14 @@ object DeduplicateRelations extends Rule[LogicalPlan] { if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => Seq((oldVersion, oldVersion.copy(output = output.map(_.newInstance())))) + case oldVersion @ FlatMapCoGroupsInPandas(_, _, _, output, _, _) + if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => + Seq((oldVersion, oldVersion.copy(output = output.map(_.newInstance())))) + + case oldVersion @ MapInPandas(_, output, _) + if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => + Seq((oldVersion, oldVersion.copy(output = output.map(_.newInstance())))) + case oldVersion: Generate if oldVersion.producedAttributes.intersect(conflictingAttributes).nonEmpty => val newOutput = oldVersion.generatorOutput.map(_.newInstance()) From 2e440e98c5a68b40e0f150136ee9e3c1c2e008b1 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 8 Feb 2021 17:26:30 +0800 Subject: [PATCH 04/38] fix SPARK-25278 --- .../scala/org/apache/spark/sql/execution/PlannerSuite.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index e851722fa4ea6..a09eeb3466a7b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -728,8 +728,6 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { case r: Range => r } assert(ranges.length == 2) - // Ensure the two Range instances are equal according to their equal method - assert(ranges.head == ranges.last) val execRanges = df.queryExecution.sparkPlan.collect { case r: RangeExec => r } From fc05305c25e3f7eee7e3ae55624c92cb5afcfc19 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 8 Feb 2021 17:53:07 +0800 Subject: [PATCH 05/38] fix stream self join --- .../spark/sql/catalyst/analysis/DeduplicateRelations.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index f17941d9f7617..f479bd0436833 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -118,6 +118,12 @@ object DeduplicateRelations extends Rule[LogicalPlan] { */ def collectConflictPlans(plan: LogicalPlan): Seq[(LogicalPlan, LogicalPlan)] = plan match { // Handle base relations that might appear more than once. + case oldVersion: MultiInstanceRelation + if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => + val newVersion = oldVersion.newInstance() + newVersion.copyTagsFrom(oldVersion) + Seq((oldVersion, newVersion)) + case oldVersion: SerializeFromObject if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => Seq((oldVersion, oldVersion.copy( From 77c9b05c51e3536878061a063172fb7af38f6a97 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 8 Feb 2021 17:58:36 +0800 Subject: [PATCH 06/38] regen golden --- .../q14a.sf100/explain.txt | 708 ++++----- .../q14a.sf100/simplified.txt | 158 +- .../approved-plans-v1_4/q14a/explain.txt | 634 ++++---- .../approved-plans-v1_4/q14a/simplified.txt | 80 +- .../q14b.sf100/explain.txt | 638 ++++---- .../q14b.sf100/simplified.txt | 146 +- .../approved-plans-v1_4/q14b/explain.txt | 584 ++++---- .../approved-plans-v1_4/q14b/simplified.txt | 80 +- .../approved-plans-v1_4/q33.sf100/explain.txt | 287 ++-- .../q33.sf100/simplified.txt | 20 +- .../approved-plans-v1_4/q33/explain.txt | 287 ++-- .../approved-plans-v1_4/q33/simplified.txt | 20 +- .../approved-plans-v1_4/q56.sf100/explain.txt | 283 ++-- .../q56.sf100/simplified.txt | 20 +- .../approved-plans-v1_4/q56/explain.txt | 283 ++-- .../approved-plans-v1_4/q56/simplified.txt | 20 +- .../approved-plans-v1_4/q60.sf100/explain.txt | 283 ++-- .../q60.sf100/simplified.txt | 20 +- .../approved-plans-v1_4/q60/explain.txt | 283 ++-- .../approved-plans-v1_4/q60/simplified.txt | 20 +- .../approved-plans-v1_4/q83.sf100/explain.txt | 294 ++-- .../q83.sf100/simplified.txt | 25 +- .../approved-plans-v1_4/q83/explain.txt | 284 ++-- .../approved-plans-v1_4/q83/simplified.txt | 25 +- .../approved-plans-v2_7/q14.sf100/explain.txt | 638 ++++---- .../q14.sf100/simplified.txt | 146 +- .../approved-plans-v2_7/q14/explain.txt | 584 ++++---- .../approved-plans-v2_7/q14/simplified.txt | 80 +- .../q14a.sf100/explain.txt | 1310 ++++++++--------- .../q14a.sf100/simplified.txt | 210 ++- .../approved-plans-v2_7/q14a/explain.txt | 1114 +++++++------- .../approved-plans-v2_7/q14a/simplified.txt | 80 +- 32 files changed, 5090 insertions(+), 4554 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt index d4ab3f60494d9..6033ad52be4e9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt @@ -37,39 +37,39 @@ TakeOrderedAndProject (134) : : : : +- * HashAggregate (44) : : : : +- Exchange (43) : : : : +- * HashAggregate (42) - : : : : +- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) + : : : : +- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Project (18) + : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : :- * Filter (11) + : : : : : : +- * ColumnarToRow (10) + : : : : : : +- Scan parquet default.store_sales (9) + : : : : : +- BroadcastExchange (16) + : : : : : +- * Project (15) + : : : : : +- * Filter (14) + : : : : : +- * ColumnarToRow (13) + : : : : : +- Scan parquet default.date_dim (12) + : : : : +- BroadcastExchange (39) + : : : : +- SortMergeJoin LeftSemi (38) + : : : : :- * Sort (23) + : : : : : +- Exchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (37) + : : : : +- Exchange (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- Scan parquet default.catalog_sales (24) + : : : : : +- ReusedExchange (27) + : : : : +- BroadcastExchange (33) + : : : : +- * Filter (32) + : : : : +- * ColumnarToRow (31) + : : : : +- Scan parquet default.item (30) : : : +- * Sort (57) : : : +- Exchange (56) : : : +- * Project (55) @@ -165,10 +165,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 20] +(7) ColumnarToRow [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 20] +(8) Filter [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) @@ -180,10 +180,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 5] +(10) ColumnarToRow [codegen id : 10] Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -(11) Filter [codegen id : 5] +(11) Filter [codegen id : 10] Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -209,51 +209,38 @@ Input [2]: [d_date_sk#12, d_year#13] Input [1]: [d_date_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(17) BroadcastHashJoin [codegen id : 5] +(17) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#12] Join condition: None -(18) Project [codegen id : 5] +(18) Project [codegen id : 10] Output [1]: [ss_item_sk#1] Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] (19) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Condition : (((isnotnull(i_item_sk#15) AND isnotnull(i_brand_id#16)) AND isnotnull(i_class_id#17)) AND isnotnull(i_category_id#18)) -(22) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] +(22) Exchange +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: hashpartitioning(coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join condition: None - -(24) Project [codegen id : 5] -Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] -Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] - -(25) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] +(23) Sort [codegen id : 5] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: [coalesce(i_brand_id#16, 0) ASC NULLS FIRST, isnull(i_brand_id#16) ASC NULLS FIRST, coalesce(i_class_id#17, 0) ASC NULLS FIRST, isnull(i_class_id#17) ASC NULLS FIRST, coalesce(i_category_id#18, 0) ASC NULLS FIRST, isnull(i_category_id#18) ASC NULLS FIRST], false, 0 -(26) Sort [codegen id : 6] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 - -(27) Scan parquet default.catalog_sales +(24) Scan parquet default.catalog_sales Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] @@ -261,215 +248,228 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] +(25) ColumnarToRow [codegen id : 8] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -(29) Filter [codegen id : 9] +(26) Filter [codegen id : 8] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] Condition : isnotnull(cs_item_sk#20) -(30) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +(27) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#22] -(31) BroadcastHashJoin [codegen id : 9] +(28) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#22] Join condition: None -(32) Project [codegen id : 9] +(29) Project [codegen id : 8] Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] +Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#22] -(33) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(30) Scan parquet default.item +Output [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(31) ColumnarToRow [codegen id : 7] +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -(35) Filter [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +(32) Filter [codegen id : 7] +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Condition : isnotnull(i_item_sk#23) -(36) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +(33) BroadcastExchange +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] -(37) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#23] Join condition: None -(38) Project [codegen id : 9] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(35) Project [codegen id : 8] +Output [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Input [5]: [cs_item_sk#20, i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] + +(36) Exchange +Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: hashpartitioning(coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26), 5), ENSURE_REQUIREMENTS, [id=#28] -(39) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] +(37) Sort [codegen id : 9] +Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: [coalesce(i_brand_id#24, 0) ASC NULLS FIRST, isnull(i_brand_id#24) ASC NULLS FIRST, coalesce(i_class_id#25, 0) ASC NULLS FIRST, isnull(i_class_id#25) ASC NULLS FIRST, coalesce(i_category_id#26, 0) ASC NULLS FIRST, isnull(i_category_id#26) ASC NULLS FIRST], false, 0 -(40) Sort [codegen id : 10] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(38) SortMergeJoin +Left keys [6]: [coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18)] +Right keys [6]: [coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26)] +Join condition: None -(41) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +(39) BroadcastExchange +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] + +(40) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#15] Join condition: None -(42) HashAggregate [codegen id : 11] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(41) Project [codegen id : 10] +Output [3]: [i_brand_id#16 AS brand_id#30, i_class_id#17 AS class_id#31, i_category_id#18 AS category_id#32] +Input [5]: [ss_item_sk#1, i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] + +(42) HashAggregate [codegen id : 10] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#30, class_id#31, category_id#32] (43) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] -(44) HashAggregate [codegen id : 12] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(44) HashAggregate [codegen id : 11] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#30, class_id#31, category_id#32] (45) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32), 5), ENSURE_REQUIREMENTS, [id=#34] -(46) Sort [codegen id : 13] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 12] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: [coalesce(brand_id#30, 0) ASC NULLS FIRST, isnull(brand_id#30) ASC NULLS FIRST, coalesce(class_id#31, 0) ASC NULLS FIRST, isnull(class_id#31) ASC NULLS FIRST, coalesce(category_id#32, 0) ASC NULLS FIRST, isnull(category_id#32) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Output [2]: [ws_item_sk#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +(48) ColumnarToRow [codegen id : 15] +Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] -(49) Filter [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(49) Filter [codegen id : 15] +Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#35) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#37] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(51) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#37] Join condition: None -(52) Project [codegen id : 16] -Output [1]: [ws_item_sk#26] -Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] +(52) Project [codegen id : 15] +Output [1]: [ws_item_sk#35] +Input [3]: [ws_item_sk#35, ws_sold_date_sk#36, d_date_sk#37] -(53) ReusedExchange [Reuses operator id: 36] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(53) ReusedExchange [Reuses operator id: 33] +Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(54) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [i_item_sk#38] Join condition: None -(55) Project [codegen id : 16] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(55) Project [codegen id : 15] +Output [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Input [5]: [ws_item_sk#35, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] (56) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: hashpartitioning(coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41), 5), ENSURE_REQUIREMENTS, [id=#42] -(57) Sort [codegen id : 17] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 16] +Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: [coalesce(i_brand_id#39, 0) ASC NULLS FIRST, isnull(i_brand_id#39) ASC NULLS FIRST, coalesce(i_class_id#40, 0) ASC NULLS FIRST, isnull(i_class_id#40) ASC NULLS FIRST, coalesce(i_category_id#41, 0) ASC NULLS FIRST, isnull(i_category_id#41) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41)] Join condition: None -(59) HashAggregate [codegen id : 18] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(59) HashAggregate [codegen id : 17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#30, class_id#31, category_id#32] (60) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#43] -(61) HashAggregate [codegen id : 19] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(61) HashAggregate [codegen id : 18] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#30, class_id#31, category_id#32] (62) BroadcastExchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#44] -(63) BroadcastHashJoin [codegen id : 20] +(63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#16, class_id#17, category_id#18] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None -(64) Project [codegen id : 20] -Output [1]: [i_item_sk#7 AS ss_item_sk#31] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] +(64) Project [codegen id : 19] +Output [1]: [i_item_sk#7 AS ss_item_sk#45] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#30, class_id#31, category_id#32] (65) Exchange -Input [1]: [ss_item_sk#31] -Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [1]: [ss_item_sk#45] +Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] -(66) Sort [codegen id : 21] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 20] +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#45] Join condition: None (68) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#33] +Output [3]: [d_date_sk#12, d_year#13, d_moy#47] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 22] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] +(69) ColumnarToRow [codegen id : 21] +Input [3]: [d_date_sk#12, d_year#13, d_moy#47] -(70) Filter [codegen id : 22] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#33)) AND (d_year#13 = 2001)) AND (d_moy#33 = 11)) AND isnotnull(d_date_sk#12)) +(70) Filter [codegen id : 21] +Input [3]: [d_date_sk#12, d_year#13, d_moy#47] +Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#47)) AND (d_year#13 = 2001)) AND (d_moy#47 = 11)) AND isnotnull(d_date_sk#12)) -(71) Project [codegen id : 22] +(71) Project [codegen id : 21] Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] +Input [3]: [d_date_sk#12, d_year#13, d_moy#47] (72) BroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] -(73) BroadcastHashJoin [codegen id : 44] +(73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#12] Join condition: None -(74) Project [codegen id : 44] +(74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] @@ -480,273 +480,273 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 23] +(76) ColumnarToRow [codegen id : 22] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(77) Filter [codegen id : 23] +(77) Filter [codegen id : 22] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : isnotnull(i_item_sk#7) (78) Exchange Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#35] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#49] -(79) Sort [codegen id : 24] +(79) Sort [codegen id : 23] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#45] -(81) Sort [codegen id : 43] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 41] +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (82) SortMergeJoin Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#45] Join condition: None (83) BroadcastExchange Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] -(84) BroadcastHashJoin [codegen id : 44] +(84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#7] Join condition: None -(85) Project [codegen id : 44] +(85) Project [codegen id : 42] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(86) HashAggregate [codegen id : 44] +(86) HashAggregate [codegen id : 42] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#37, isEmpty#38, count#39] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] +Aggregate Attributes [3]: [sum#51, isEmpty#52, count#53] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#57] -(88) HashAggregate [codegen id : 45] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] +(88) HashAggregate [codegen id : 43] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44, count(1)#45] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sales#46, count(1)#45 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] -(89) Filter [codegen id : 45] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48 as decimal(32,6)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(89) Filter [codegen id : 43] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(90) Project [codegen id : 45] -Output [6]: [sales#46, number_sales#47, store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] +(90) Project [codegen id : 43] +Output [6]: [sales#60, number_sales#61, store AS channel#65, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] (91) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Output [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 46] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +(92) ColumnarToRow [codegen id : 44] +Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] -(93) Filter [codegen id : 46] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +(93) Filter [codegen id : 44] +Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] Condition : isnotnull(cs_item_sk#20) (94) Exchange -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#68] -(95) Sort [codegen id : 47] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +(95) Sort [codegen id : 45] +Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] Arguments: [cs_item_sk#20 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#45] -(97) Sort [codegen id : 66] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 63] +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (98) SortMergeJoin Left keys [1]: [cs_item_sk#20] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#45] Join condition: None (99) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#69] -(100) BroadcastHashJoin [codegen id : 89] +(100) BroadcastHashJoin [codegen id : 85] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#69] Join condition: None -(101) Project [codegen id : 89] -Output [3]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53] -Input [5]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] +(101) Project [codegen id : 85] +Output [3]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67] +Input [5]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21, d_date_sk#69] (102) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73] -(103) BroadcastHashJoin [codegen id : 89] +(103) BroadcastHashJoin [codegen id : 85] Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#70] Join condition: None -(104) Project [codegen id : 89] -Output [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(104) Project [codegen id : 85] +Output [5]: [cs_quantity#66, cs_list_price#67, i_brand_id#71, i_class_id#72, i_category_id#73] +Input [7]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73] -(105) HashAggregate [codegen id : 89] -Input [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] +(105) HashAggregate [codegen id : 85] +Input [5]: [cs_quantity#66, cs_list_price#67, i_brand_id#71, i_class_id#72, i_category_id#73] +Keys [3]: [i_brand_id#71, i_class_id#72, i_category_id#73] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#74, isEmpty#75, count#76] +Results [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] (106) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] +Arguments: hashpartitioning(i_brand_id#71, i_class_id#72, i_category_id#73, 5), ENSURE_REQUIREMENTS, [id=#80] -(107) HashAggregate [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] +(107) HashAggregate [codegen id : 86] +Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] +Keys [3]: [i_brand_id#71, i_class_id#72, i_category_id#73] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81, count(1)#82] +Results [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sales#83, count(1)#82 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] -(108) Filter [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(108) Filter [codegen id : 86] +Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(109) Project [codegen id : 90] -Output [6]: [sales#64, number_sales#65, catalog AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] +(109) Project [codegen id : 86] +Output [6]: [sales#83, number_sales#84, catalog AS channel#86, i_brand_id#71, i_class_id#72, i_category_id#73] +Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] (110) Scan parquet default.web_sales -Output [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Output [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 91] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +(111) ColumnarToRow [codegen id : 87] +Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] -(112) Filter [codegen id : 91] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(112) Filter [codegen id : 87] +Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#35) (113) Exchange -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_item_sk#26, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Arguments: hashpartitioning(ws_item_sk#35, 5), ENSURE_REQUIREMENTS, [id=#89] -(114) Sort [codegen id : 92] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Arguments: [ws_item_sk#26 ASC NULLS FIRST], false, 0 +(114) Sort [codegen id : 88] +Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Arguments: [ws_item_sk#35 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#45] -(116) Sort [codegen id : 111] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(116) Sort [codegen id : 106] +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (117) SortMergeJoin -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [ss_item_sk#45] Join condition: None (118) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#90] -(119) BroadcastHashJoin [codegen id : 134] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(119) BroadcastHashJoin [codegen id : 128] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#90] Join condition: None -(120) Project [codegen id : 134] -Output [3]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69] -Input [5]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] +(120) Project [codegen id : 128] +Output [3]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88] +Input [5]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36, d_date_sk#90] (121) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94] -(122) BroadcastHashJoin [codegen id : 134] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(122) BroadcastHashJoin [codegen id : 128] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [i_item_sk#91] Join condition: None -(123) Project [codegen id : 134] -Output [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(123) Project [codegen id : 128] +Output [5]: [ws_quantity#87, ws_list_price#88, i_brand_id#92, i_class_id#93, i_category_id#94] +Input [7]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94] -(124) HashAggregate [codegen id : 134] -Input [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#71, isEmpty#72, count#73] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] +(124) HashAggregate [codegen id : 128] +Input [5]: [ws_quantity#87, ws_list_price#88, i_brand_id#92, i_class_id#93, i_category_id#94] +Keys [3]: [i_brand_id#92, i_class_id#93, i_category_id#94] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#95, isEmpty#96, count#97] +Results [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] (125) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] +Arguments: hashpartitioning(i_brand_id#92, i_class_id#93, i_category_id#94, 5), ENSURE_REQUIREMENTS, [id=#101] -(126) HashAggregate [codegen id : 135] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78, count(1)#79] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sales#80, count(1)#79 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +(126) HashAggregate [codegen id : 129] +Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] +Keys [3]: [i_brand_id#92, i_class_id#93, i_category_id#94] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102, count(1)#103] +Results [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sales#104, count(1)#103 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] -(127) Filter [codegen id : 135] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(127) Filter [codegen id : 129] +Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(128) Project [codegen id : 135] -Output [6]: [sales#80, number_sales#81, web AS channel#83, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +(128) Project [codegen id : 129] +Output [6]: [sales#104, number_sales#105, web AS channel#107, i_brand_id#92, i_class_id#93, i_category_id#94] +Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] (129) Union -(130) Expand [codegen id : 136] -Input [6]: [sales#46, number_sales#47, channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [List(sales#46, number_sales#47, channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 0), List(sales#46, number_sales#47, channel#51, i_brand_id#8, i_class_id#9, null, 1), List(sales#46, number_sales#47, channel#51, i_brand_id#8, null, null, 3), List(sales#46, number_sales#47, channel#51, null, null, null, 7), List(sales#46, number_sales#47, null, null, null, null, 15)], [sales#46, number_sales#47, channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] +(130) Expand [codegen id : 130] +Input [6]: [sales#60, number_sales#61, channel#65, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [List(sales#60, number_sales#61, channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, 0), List(sales#60, number_sales#61, channel#65, i_brand_id#8, i_class_id#9, null, 1), List(sales#60, number_sales#61, channel#65, i_brand_id#8, null, null, 3), List(sales#60, number_sales#61, channel#65, null, null, null, 7), List(sales#60, number_sales#61, null, null, null, null, 15)], [sales#60, number_sales#61, channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112] -(131) HashAggregate [codegen id : 136] -Input [7]: [sales#46, number_sales#47, channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] -Keys [5]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#89, isEmpty#90, sum#91] -Results [8]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, sum#92, isEmpty#93, sum#94] +(131) HashAggregate [codegen id : 130] +Input [7]: [sales#60, number_sales#61, channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112] +Keys [5]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112] +Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] +Aggregate Attributes [3]: [sum#113, isEmpty#114, sum#115] +Results [8]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112, sum#116, isEmpty#117, sum#118] (132) Exchange -Input [8]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, sum#92, isEmpty#93, sum#94] -Arguments: hashpartitioning(channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, 5), ENSURE_REQUIREMENTS, [id=#95] +Input [8]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112, sum#116, isEmpty#117, sum#118] +Arguments: hashpartitioning(channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112, 5), ENSURE_REQUIREMENTS, [id=#119] -(133) HashAggregate [codegen id : 137] -Input [8]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, sum#92, isEmpty#93, sum#94] -Keys [5]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#96, sum(number_sales#47)#97] -Results [6]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, sum(sales#46)#96 AS sum(sales)#98, sum(number_sales#47)#97 AS sum(number_sales)#99] +(133) HashAggregate [codegen id : 131] +Input [8]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112, sum#116, isEmpty#117, sum#118] +Keys [5]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112] +Functions [2]: [sum(sales#60), sum(number_sales#61)] +Aggregate Attributes [2]: [sum(sales#60)#120, sum(number_sales#61)#121] +Results [6]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, sum(sales#60)#120 AS sum(sales)#122, sum(number_sales#61)#121 AS sum(number_sales)#123] (134) TakeOrderedAndProject -Input [6]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, sum(sales)#98, sum(number_sales)#99] -Arguments: 100, [channel#84 ASC NULLS FIRST, i_brand_id#85 ASC NULLS FIRST, i_class_id#86 ASC NULLS FIRST, i_category_id#87 ASC NULLS FIRST], [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, sum(sales)#98, sum(number_sales)#99] +Input [6]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, sum(sales)#122, sum(number_sales)#123] +Arguments: 100, [channel#108 ASC NULLS FIRST, i_brand_id#109 ASC NULLS FIRST, i_class_id#110 ASC NULLS FIRST, i_category_id#111 ASC NULLS FIRST], [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, sum(sales)#122, sum(number_sales)#123] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#49, [id=#50] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#63, [id=#64] * HashAggregate (157) +- Exchange (156) +- * HashAggregate (155) @@ -776,7 +776,7 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#100)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#124)] ReadSchema: struct (136) ColumnarToRow [codegen id : 2] @@ -802,7 +802,7 @@ Input [2]: [d_date_sk#12, d_year#13] (141) BroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#101] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#125] (142) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#4] @@ -810,83 +810,83 @@ Right keys [1]: [d_date_sk#12] Join condition: None (143) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#102, ss_list_price#3 AS list_price#103] +Output [2]: [ss_quantity#2 AS quantity#126, ss_list_price#3 AS list_price#127] Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (144) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Output [3]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#100)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#124)] ReadSchema: struct (145) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Input [3]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] (146) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#128] (147) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#128] Join condition: None (148) Project [codegen id : 4] -Output [2]: [cs_quantity#52 AS quantity#104, cs_list_price#53 AS list_price#105] -Input [4]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] +Output [2]: [cs_quantity#66 AS quantity#129, cs_list_price#67 AS list_price#130] +Input [4]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21, d_date_sk#128] (149) Scan parquet default.web_sales -Output [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#100)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#124)] ReadSchema: struct (150) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] (151) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#131] (152) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#131] Join condition: None (153) Project [codegen id : 6] -Output [2]: [ws_quantity#68 AS quantity#106, ws_list_price#69 AS list_price#107] -Input [4]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] +Output [2]: [ws_quantity#87 AS quantity#132, ws_list_price#88 AS list_price#133] +Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36, d_date_sk#131] (154) Union (155) HashAggregate [codegen id : 7] -Input [2]: [quantity#102, list_price#103] +Input [2]: [quantity#126, list_price#127] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#108, count#109] -Results [2]: [sum#110, count#111] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#126 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#127 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#134, count#135] +Results [2]: [sum#136, count#137] (156) Exchange -Input [2]: [sum#110, count#111] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#112] +Input [2]: [sum#136, count#137] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#138] (157) HashAggregate [codegen id : 8] -Input [2]: [sum#110, count#111] +Input [2]: [sum#136, count#137] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))#113] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))#113 AS average_sales#114] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#126 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#127 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#126 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#127 as decimal(12,2)))), DecimalType(18,2), true))#139] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#126 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#127 as decimal(12,2)))), DecimalType(18,2), true))#139 AS average_sales#140] -Subquery:2 Hosting operator id = 135 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#100 +Subquery:2 Hosting operator id = 135 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#124 ReusedExchange (158) (158) ReusedExchange [Reuses operator id: 141] Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 144 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#100 +Subquery:3 Hosting operator id = 144 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#124 -Subquery:4 Hosting operator id = 149 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#100 +Subquery:4 Hosting operator id = 149 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#124 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (159) @@ -902,16 +902,16 @@ ReusedExchange (160) (160) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#11 -Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt index 4ee1ff31e9beb..2134091c43b82 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (137) + WholeStageCodegen (131) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (136) + WholeStageCodegen (130) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] InputAdapter Union - WholeStageCodegen (45) + WholeStageCodegen (43) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #3 @@ -57,7 +57,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (44) + WholeStageCodegen (42) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -76,11 +76,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - WholeStageCodegen (21) + WholeStageCodegen (20) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #5 - WholeStageCodegen (20) + WholeStageCodegen (19) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -89,84 +89,82 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 - WholeStageCodegen (19) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (18) + WholeStageCodegen (17) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (13) + WholeStageCodegen (12) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (11) + WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #10 InputAdapter - Exchange [brand_id,class_id,category_id] #10 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] + BroadcastExchange #10 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #11 + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #10 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (7) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #13 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (17) + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (16) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (16) + WholeStageCodegen (15) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -177,12 +175,12 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #11 + ReusedExchange [d_date_sk] #10 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 InputAdapter BroadcastExchange #4 - WholeStageCodegen (22) + WholeStageCodegen (21) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow @@ -191,27 +189,27 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter BroadcastExchange #16 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #17 - WholeStageCodegen (23) + WholeStageCodegen (22) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 - WholeStageCodegen (90) + WholeStageCodegen (86) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (89) + WholeStageCodegen (85) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -219,17 +217,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [cs_item_sk,ss_item_sk] - WholeStageCodegen (47) + WholeStageCodegen (45) Sort [cs_item_sk] InputAdapter Exchange [cs_item_sk] #22 - WholeStageCodegen (46) + WholeStageCodegen (44) Filter [cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (66) + WholeStageCodegen (63) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 @@ -237,14 +235,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [d_date_sk] #4 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (135) + WholeStageCodegen (129) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #23 - WholeStageCodegen (134) + WholeStageCodegen (128) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] @@ -252,17 +250,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ws_item_sk,ss_item_sk] - WholeStageCodegen (92) + WholeStageCodegen (88) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #24 - WholeStageCodegen (91) + WholeStageCodegen (87) Filter [ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (111) + WholeStageCodegen (106) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 85a3a63413f92..34aa615cf910c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -31,36 +31,36 @@ TakeOrderedAndProject (115) : : : :- * HashAggregate (39) : : : : +- Exchange (38) : : : : +- * HashAggregate (37) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) + : : : : +- * Project (36) + : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : :- * Project (33) + : : : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet default.store_sales (7) + : : : : : +- BroadcastExchange (31) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Project (28) + : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet default.item (16) + : : : : : +- BroadcastExchange (26) + : : : : : +- * Project (25) + : : : : : +- * Filter (24) + : : : : : +- * ColumnarToRow (23) + : : : : : +- Scan parquet default.date_dim (22) + : : : : +- ReusedExchange (34) : : : +- BroadcastExchange (49) : : : +- * Project (48) : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -161,226 +161,226 @@ Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) (10) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(11) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -(12) Filter [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +(12) Filter [codegen id : 4] +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14)) -(13) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +(13) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join condition: None +(14) ColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] -(15) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(15) Filter [codegen id : 3] +Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Condition : isnotnull(cs_item_sk#15) -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +(16) Scan parquet default.item +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] 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 - -(17) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(18) Filter [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +(17) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -(19) Project [codegen id : 2] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +(18) Filter [codegen id : 1] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : isnotnull(i_item_sk#17) -(20) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +(19) BroadcastExchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [i_item_sk#17] Join condition: None -(22) Project [codegen id : 6] -Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] -Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -(23) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(22) Scan parquet default.date_dim +Output [2]: [d_date_sk#22, d_year#23] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +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 -(24) ColumnarToRow [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(23) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#22, d_year#23] -(25) Filter [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +(24) Filter [codegen id : 2] +Input [2]: [d_date_sk#22, d_year#23] +Condition : (((isnotnull(d_year#23) AND (d_year#23 >= 1999)) AND (d_year#23 <= 2001)) AND isnotnull(d_date_sk#22)) -(26) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +(25) Project [codegen id : 2] +Output [1]: [d_date_sk#22] +Input [2]: [d_date_sk#22, d_year#23] -(27) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(26) BroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(28) Filter [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +(27) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#22] +Join condition: None + +(28) Project [codegen id : 3] +Output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] +Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#22] (29) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] -(30) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +(30) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)] +Right keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] Join condition: None -(31) Project [codegen id : 5] -Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(32) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(31) BroadcastExchange +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +(32) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#11] Join condition: None -(34) Project [codegen id : 5] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(33) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14] +Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -(35) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] +(34) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#27] -(36) BroadcastHashJoin [codegen id : 6] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#27] Join condition: None +(36) Project [codegen id : 6] +Output [3]: [i_brand_id#12 AS brand_id#28, i_class_id#13 AS class_id#29, i_category_id#14 AS category_id#30] +Input [5]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#27] + (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#28, class_id#29, category_id#30] (38) Exchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [id=#31] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#28, class_id#29, category_id#30] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Output [2]: [ws_item_sk#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Condition : isnotnull(ws_item_sk#32) -(43) ReusedExchange [Reuses operator id: 29] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(43) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#32] +Right keys [1]: [i_item_sk#34] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37] +Input [6]: [ws_item_sk#32, ws_sold_date_sk#33, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -(46) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(46) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#38] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#38] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +Input [5]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] (49) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#39] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] +Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#28, class_id#29, category_id#30] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#28, class_id#29, category_id#30] (53) BroadcastExchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#40] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#15, class_id#16, category_id#17] +Right keys [3]: [brand_id#28, class_id#29, category_id#30] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#27] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] +Output [1]: [i_item_sk#6 AS ss_item_sk#41] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] (56) BroadcastExchange -Input [1]: [ss_item_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [ss_item_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#41] Join condition: None (58) Scan parquet default.item @@ -398,16 +398,16 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : isnotnull(i_item_sk#6) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#41] (62) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#41] Join condition: None (63) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] @@ -419,239 +419,239 @@ Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_ Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (66) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#30] +Output [3]: [d_date_sk#27, d_year#44, d_moy#45] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] +Input [3]: [d_date_sk#27, d_year#44, d_moy#45] (68) Filter [codegen id : 24] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#30)) AND (d_year#13 = 2001)) AND (d_moy#30 = 11)) AND isnotnull(d_date_sk#12)) +Input [3]: [d_date_sk#27, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#27)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] +Output [1]: [d_date_sk#27] +Input [3]: [d_date_sk#27, d_year#44, d_moy#45] (70) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#27] Join condition: None (72) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#27] (73) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#32, isEmpty#33, count#34] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] +Aggregate Attributes [3]: [sum#47, isEmpty#48, count#49] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#53] (75) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] (76) Filter [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [sales#41, number_sales#42, store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] +Output [6]: [sales#56, number_sales#57, store AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] (78) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Output [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Input [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] (80) Filter [codegen id : 51] -Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +Input [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Condition : isnotnull(cs_item_sk#15) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#41] (82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [ss_item_sk#41] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] (84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [i_item_sk#64] Join condition: None (85) Project [codegen id : 51] -Output [6]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_brand_id#65, i_class_id#66, i_category_id#67] +Input [8]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] (86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#68] (87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#68] Join condition: None (88) Project [codegen id : 51] -Output [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [cs_quantity#62, cs_list_price#63, i_brand_id#65, i_class_id#66, i_category_id#67] +Input [7]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_brand_id#65, i_class_id#66, i_category_id#67, d_date_sk#68] (89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#49, isEmpty#50, count#51] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] +Input [5]: [cs_quantity#62, cs_list_price#63, i_brand_id#65, i_class_id#66, i_category_id#67] +Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] +Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] (90) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] +Arguments: hashpartitioning(i_brand_id#65, i_class_id#66, i_category_id#67, 5), ENSURE_REQUIREMENTS, [id=#75] (91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56, count(1)#57] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sales#58, count(1)#57 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] +Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76, count(1)#77] +Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sales#78, count(1)#77 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] (92) Filter [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [sales#58, number_sales#59, catalog AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] +Output [6]: [sales#78, number_sales#79, catalog AS channel#81, i_brand_id#65, i_class_id#66, i_category_id#67] +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] (94) Scan parquet default.web_sales -Output [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Output [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Input [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] (96) Filter [codegen id : 77] -Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Condition : isnotnull(ws_item_sk#32) (97) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#41] (98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [ws_item_sk#32] +Right keys [1]: [ss_item_sk#41] Join condition: None (99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] (100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#32] +Right keys [1]: [i_item_sk#84] Join condition: None (101) Project [codegen id : 77] -Output [6]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_brand_id#85, i_class_id#86, i_category_id#87] +Input [8]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] (102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#88] (103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#88] Join condition: None (104) Project [codegen id : 77] -Output [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#85, i_class_id#86, i_category_id#87] +Input [7]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_brand_id#85, i_class_id#86, i_category_id#87, d_date_sk#88] (105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] +Input [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#85, i_class_id#86, i_category_id#87] +Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#89, isEmpty#90, count#91] +Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] (106) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, 5), ENSURE_REQUIREMENTS, [id=#95] (107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71, count(1)#72] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sales#73, count(1)#72 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96, count(1)#97] +Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sales#98, count(1)#97 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] (108) Filter [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [sales#73, number_sales#74, web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] +Output [6]: [sales#98, number_sales#99, web AS channel#101, i_brand_id#85, i_class_id#86, i_category_id#87] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] (110) Union (111) Expand [codegen id : 79] -Input [6]: [sales#41, number_sales#42, channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: [List(sales#41, number_sales#42, channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#41, number_sales#42, channel#46, i_brand_id#7, i_class_id#8, null, 1), List(sales#41, number_sales#42, channel#46, i_brand_id#7, null, null, 3), List(sales#41, number_sales#42, channel#46, null, null, null, 7), List(sales#41, number_sales#42, null, null, null, null, 15)], [sales#41, number_sales#42, channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] +Input [6]: [sales#56, number_sales#57, channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: [List(sales#56, number_sales#57, channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#56, number_sales#57, channel#61, i_brand_id#7, i_class_id#8, null, 1), List(sales#56, number_sales#57, channel#61, i_brand_id#7, null, null, 3), List(sales#56, number_sales#57, channel#61, null, null, null, 7), List(sales#56, number_sales#57, null, null, null, null, 15)], [sales#56, number_sales#57, channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] (112) HashAggregate [codegen id : 79] -Input [7]: [sales#41, number_sales#42, channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] -Keys [5]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#82, isEmpty#83, sum#84] -Results [8]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, sum#85, isEmpty#86, sum#87] +Input [7]: [sales#56, number_sales#57, channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] +Keys [5]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] +Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] +Aggregate Attributes [3]: [sum#107, isEmpty#108, sum#109] +Results [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] (113) Exchange -Input [8]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, sum#85, isEmpty#86, sum#87] -Arguments: hashpartitioning(channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] +Arguments: hashpartitioning(channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] (114) HashAggregate [codegen id : 80] -Input [8]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, sum#85, isEmpty#86, sum#87] -Keys [5]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#89, sum(number_sales#42)#90] -Results [6]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, sum(sales#41)#89 AS sum(sales)#91, sum(number_sales#42)#90 AS sum(number_sales)#92] +Input [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] +Keys [5]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] +Functions [2]: [sum(sales#56), sum(number_sales#57)] +Aggregate Attributes [2]: [sum(sales#56)#114, sum(number_sales#57)#115] +Results [6]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales#56)#114 AS sum(sales)#116, sum(number_sales#57)#115 AS sum(number_sales)#117] (115) TakeOrderedAndProject -Input [6]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, sum(sales)#91, sum(number_sales)#92] -Arguments: 100, [channel#77 ASC NULLS FIRST, i_brand_id#78 ASC NULLS FIRST, i_class_id#79 ASC NULLS FIRST, i_category_id#80 ASC NULLS FIRST], [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, sum(sales)#91, sum(number_sales)#92] +Input [6]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales)#116, sum(number_sales)#117] +Arguments: 100, [channel#102 ASC NULLS FIRST, i_brand_id#103 ASC NULLS FIRST, i_class_id#104 ASC NULLS FIRST, i_category_id#105 ASC NULLS FIRST], [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales)#116, sum(number_sales)#117] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#59, [id=#60] * HashAggregate (138) +- Exchange (137) +- * HashAggregate (136) @@ -681,142 +681,142 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#93)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#118)] ReadSchema: struct (117) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (118) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#27, d_year#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (119) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#27, d_year#44] (120) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#27, d_year#44] +Condition : (((isnotnull(d_year#44) AND (d_year#44 >= 1999)) AND (d_year#44 <= 2001)) AND isnotnull(d_date_sk#27)) (121) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#27] +Input [2]: [d_date_sk#27, d_year#44] (122) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#94] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#119] (123) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#27] Join condition: None (124) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#95, ss_list_price#3 AS list_price#96] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#2 AS quantity#120, ss_list_price#3 AS list_price#121] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#27] (125) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Output [3]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#93)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#118)] ReadSchema: struct (126) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Input [3]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] (127) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#122] (128) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#122] Join condition: None (129) Project [codegen id : 4] -Output [2]: [cs_quantity#47 AS quantity#97, cs_list_price#48 AS list_price#98] -Input [4]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, d_date_sk#12] +Output [2]: [cs_quantity#62 AS quantity#123, cs_list_price#63 AS list_price#124] +Input [4]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, d_date_sk#122] (130) Scan parquet default.web_sales -Output [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Output [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#93)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#118)] ReadSchema: struct (131) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Input [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] (132) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#125] (133) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#125] Join condition: None (134) Project [codegen id : 6] -Output [2]: [ws_quantity#62 AS quantity#99, ws_list_price#63 AS list_price#100] -Input [4]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, d_date_sk#12] +Output [2]: [ws_quantity#82 AS quantity#126, ws_list_price#83 AS list_price#127] +Input [4]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, d_date_sk#125] (135) Union (136) HashAggregate [codegen id : 7] -Input [2]: [quantity#95, list_price#96] +Input [2]: [quantity#120, list_price#121] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#101, count#102] -Results [2]: [sum#103, count#104] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#120 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#121 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#128, count#129] +Results [2]: [sum#130, count#131] (137) Exchange -Input [2]: [sum#103, count#104] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] +Input [2]: [sum#130, count#131] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#132] (138) HashAggregate [codegen id : 8] -Input [2]: [sum#103, count#104] +Input [2]: [sum#130, count#131] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))#106] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))#106 AS average_sales#107] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#120 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#121 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#120 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#121 as decimal(12,2)))), DecimalType(18,2), true))#133] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#120 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#121 as decimal(12,2)))), DecimalType(18,2), true))#133 AS average_sales#134] -Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#93 +Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#118 ReusedExchange (139) (139) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#27] -Subquery:3 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#93 +Subquery:3 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#118 -Subquery:4 Hosting operator id = 130 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#93 +Subquery:4 Hosting operator id = 130 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#118 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#27] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 ReusedExchange (141) -(141) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(141) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#27] -Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#10 -Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index f9b76560e363c..9ae1a0e30e90e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -90,53 +90,53 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Exchange [brand_id,class_id,category_id] #6 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #10 + BroadcastExchange #9 WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (1) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #7 + BroadcastExchange #7 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #11 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt index 76f5ac48f96cc..013c3339f6ec3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt @@ -33,39 +33,39 @@ TakeOrderedAndProject (116) : : : : +- * HashAggregate (44) : : : : +- Exchange (43) : : : : +- * HashAggregate (42) - : : : : +- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) + : : : : +- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Project (18) + : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : :- * Filter (11) + : : : : : : +- * ColumnarToRow (10) + : : : : : : +- Scan parquet default.store_sales (9) + : : : : : +- BroadcastExchange (16) + : : : : : +- * Project (15) + : : : : : +- * Filter (14) + : : : : : +- * ColumnarToRow (13) + : : : : : +- Scan parquet default.date_dim (12) + : : : : +- BroadcastExchange (39) + : : : : +- SortMergeJoin LeftSemi (38) + : : : : :- * Sort (23) + : : : : : +- Exchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (37) + : : : : +- Exchange (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- Scan parquet default.catalog_sales (24) + : : : : : +- ReusedExchange (27) + : : : : +- BroadcastExchange (33) + : : : : +- * Filter (32) + : : : : +- * ColumnarToRow (31) + : : : : +- Scan parquet default.item (30) : : : +- * Sort (57) : : : +- Exchange (56) : : : +- * Project (55) @@ -147,10 +147,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 20] +(7) ColumnarToRow [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 20] +(8) Filter [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) @@ -162,10 +162,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 5] +(10) ColumnarToRow [codegen id : 10] Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -(11) Filter [codegen id : 5] +(11) Filter [codegen id : 10] Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -191,51 +191,38 @@ Input [2]: [d_date_sk#12, d_year#13] Input [1]: [d_date_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(17) BroadcastHashJoin [codegen id : 5] +(17) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#12] Join condition: None -(18) Project [codegen id : 5] +(18) Project [codegen id : 10] Output [1]: [ss_item_sk#1] Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] (19) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Condition : (((isnotnull(i_item_sk#15) AND isnotnull(i_brand_id#16)) AND isnotnull(i_class_id#17)) AND isnotnull(i_category_id#18)) -(22) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] +(22) Exchange +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: hashpartitioning(coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join condition: None - -(24) Project [codegen id : 5] -Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] -Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] - -(25) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] +(23) Sort [codegen id : 5] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: [coalesce(i_brand_id#16, 0) ASC NULLS FIRST, isnull(i_brand_id#16) ASC NULLS FIRST, coalesce(i_class_id#17, 0) ASC NULLS FIRST, isnull(i_class_id#17) ASC NULLS FIRST, coalesce(i_category_id#18, 0) ASC NULLS FIRST, isnull(i_category_id#18) ASC NULLS FIRST], false, 0 -(26) Sort [codegen id : 6] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 - -(27) Scan parquet default.catalog_sales +(24) Scan parquet default.catalog_sales Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] @@ -243,215 +230,228 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] +(25) ColumnarToRow [codegen id : 8] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -(29) Filter [codegen id : 9] +(26) Filter [codegen id : 8] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] Condition : isnotnull(cs_item_sk#20) -(30) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +(27) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#22] -(31) BroadcastHashJoin [codegen id : 9] +(28) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#22] Join condition: None -(32) Project [codegen id : 9] +(29) Project [codegen id : 8] Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] +Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#22] -(33) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(30) Scan parquet default.item +Output [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(31) ColumnarToRow [codegen id : 7] +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -(35) Filter [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +(32) Filter [codegen id : 7] +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Condition : isnotnull(i_item_sk#23) -(36) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +(33) BroadcastExchange +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] -(37) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#23] Join condition: None -(38) Project [codegen id : 9] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(35) Project [codegen id : 8] +Output [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Input [5]: [cs_item_sk#20, i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -(39) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] +(36) Exchange +Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: hashpartitioning(coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26), 5), ENSURE_REQUIREMENTS, [id=#28] -(40) Sort [codegen id : 10] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(37) Sort [codegen id : 9] +Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: [coalesce(i_brand_id#24, 0) ASC NULLS FIRST, isnull(i_brand_id#24) ASC NULLS FIRST, coalesce(i_class_id#25, 0) ASC NULLS FIRST, isnull(i_class_id#25) ASC NULLS FIRST, coalesce(i_category_id#26, 0) ASC NULLS FIRST, isnull(i_category_id#26) ASC NULLS FIRST], false, 0 -(41) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +(38) SortMergeJoin +Left keys [6]: [coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18)] +Right keys [6]: [coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26)] Join condition: None -(42) HashAggregate [codegen id : 11] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(39) BroadcastExchange +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] + +(40) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#15] +Join condition: None + +(41) Project [codegen id : 10] +Output [3]: [i_brand_id#16 AS brand_id#30, i_class_id#17 AS class_id#31, i_category_id#18 AS category_id#32] +Input [5]: [ss_item_sk#1, i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] + +(42) HashAggregate [codegen id : 10] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#30, class_id#31, category_id#32] (43) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] -(44) HashAggregate [codegen id : 12] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(44) HashAggregate [codegen id : 11] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#30, class_id#31, category_id#32] (45) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32), 5), ENSURE_REQUIREMENTS, [id=#34] -(46) Sort [codegen id : 13] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 12] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: [coalesce(brand_id#30, 0) ASC NULLS FIRST, isnull(brand_id#30) ASC NULLS FIRST, coalesce(class_id#31, 0) ASC NULLS FIRST, isnull(class_id#31) ASC NULLS FIRST, coalesce(category_id#32, 0) ASC NULLS FIRST, isnull(category_id#32) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Output [2]: [ws_item_sk#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +(48) ColumnarToRow [codegen id : 15] +Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] -(49) Filter [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(49) Filter [codegen id : 15] +Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#35) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#37] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(51) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#37] Join condition: None -(52) Project [codegen id : 16] -Output [1]: [ws_item_sk#26] -Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] +(52) Project [codegen id : 15] +Output [1]: [ws_item_sk#35] +Input [3]: [ws_item_sk#35, ws_sold_date_sk#36, d_date_sk#37] -(53) ReusedExchange [Reuses operator id: 36] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(53) ReusedExchange [Reuses operator id: 33] +Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(54) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [i_item_sk#38] Join condition: None -(55) Project [codegen id : 16] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(55) Project [codegen id : 15] +Output [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Input [5]: [ws_item_sk#35, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] (56) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: hashpartitioning(coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41), 5), ENSURE_REQUIREMENTS, [id=#42] -(57) Sort [codegen id : 17] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 16] +Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: [coalesce(i_brand_id#39, 0) ASC NULLS FIRST, isnull(i_brand_id#39) ASC NULLS FIRST, coalesce(i_class_id#40, 0) ASC NULLS FIRST, isnull(i_class_id#40) ASC NULLS FIRST, coalesce(i_category_id#41, 0) ASC NULLS FIRST, isnull(i_category_id#41) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41)] Join condition: None -(59) HashAggregate [codegen id : 18] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(59) HashAggregate [codegen id : 17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#30, class_id#31, category_id#32] (60) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#43] -(61) HashAggregate [codegen id : 19] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(61) HashAggregate [codegen id : 18] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#30, class_id#31, category_id#32] (62) BroadcastExchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#44] -(63) BroadcastHashJoin [codegen id : 20] +(63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#16, class_id#17, category_id#18] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None -(64) Project [codegen id : 20] -Output [1]: [i_item_sk#7 AS ss_item_sk#31] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] +(64) Project [codegen id : 19] +Output [1]: [i_item_sk#7 AS ss_item_sk#45] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#30, class_id#31, category_id#32] (65) Exchange -Input [1]: [ss_item_sk#31] -Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [1]: [ss_item_sk#45] +Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] -(66) Sort [codegen id : 21] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 20] +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#45] Join condition: None (68) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#33] +Output [2]: [d_date_sk#12, d_week_seq#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 22] -Input [2]: [d_date_sk#12, d_week_seq#33] +(69) ColumnarToRow [codegen id : 21] +Input [2]: [d_date_sk#12, d_week_seq#47] -(70) Filter [codegen id : 22] -Input [2]: [d_date_sk#12, d_week_seq#33] -Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#34, [id=#35])) AND isnotnull(d_date_sk#12)) +(70) Filter [codegen id : 21] +Input [2]: [d_date_sk#12, d_week_seq#47] +Condition : ((isnotnull(d_week_seq#47) AND (d_week_seq#47 = Subquery scalar-subquery#48, [id=#49])) AND isnotnull(d_date_sk#12)) -(71) Project [codegen id : 22] +(71) Project [codegen id : 21] Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#33] +Input [2]: [d_date_sk#12, d_week_seq#47] (72) BroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] -(73) BroadcastHashJoin [codegen id : 44] +(73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#12] Join condition: None -(74) Project [codegen id : 44] +(74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] @@ -462,192 +462,192 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 23] +(76) ColumnarToRow [codegen id : 22] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(77) Filter [codegen id : 23] +(77) Filter [codegen id : 22] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (78) Exchange Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#37] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#51] -(79) Sort [codegen id : 24] +(79) Sort [codegen id : 23] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#45] -(81) Sort [codegen id : 43] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 41] +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (82) SortMergeJoin Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#45] Join condition: None (83) BroadcastExchange Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] -(84) BroadcastHashJoin [codegen id : 44] +(84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#7] Join condition: None -(85) Project [codegen id : 44] +(85) Project [codegen id : 42] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(86) HashAggregate [codegen id : 44] +(86) HashAggregate [codegen id : 42] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#39, isEmpty#40, count#41] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] +Aggregate Attributes [3]: [sum#53, isEmpty#54, count#55] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#59] -(88) HashAggregate [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] +(88) HashAggregate [codegen id : 86] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46, count(1)#47] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sales#48, count(1)#47 AS number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60, count(1)#61] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sales#62, count(1)#61 AS number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] -(89) Filter [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) +(89) Filter [codegen id : 86] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 as decimal(32,6)) > cast(Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) -(90) Project [codegen id : 90] -Output [6]: [store AS channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] +(90) Project [codegen id : 86] +Output [6]: [store AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] (91) Scan parquet default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#54)] +PartitionFilters: [isnotnull(ss_sold_date_sk#71), dynamicpruningexpression(ss_sold_date_sk#71 IN dynamicpruning#72)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 45] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +(92) ColumnarToRow [codegen id : 43] +Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] -(93) Filter [codegen id : 45] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +(93) Filter [codegen id : 43] +Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Condition : isnotnull(ss_item_sk#68) (94) Exchange -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Arguments: hashpartitioning(ss_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#73] -(95) Sort [codegen id : 46] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 44] +Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Arguments: [ss_item_sk#68 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#45] -(97) Sort [codegen id : 65] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 62] +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [ss_item_sk#68] +Right keys [1]: [ss_item_sk#45] Join condition: None (99) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#33] +Output [2]: [d_date_sk#74, d_week_seq#75] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 66] -Input [2]: [d_date_sk#12, d_week_seq#33] +(100) ColumnarToRow [codegen id : 63] +Input [2]: [d_date_sk#74, d_week_seq#75] -(101) Filter [codegen id : 66] -Input [2]: [d_date_sk#12, d_week_seq#33] -Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#56, [id=#57])) AND isnotnull(d_date_sk#12)) +(101) Filter [codegen id : 63] +Input [2]: [d_date_sk#74, d_week_seq#75] +Condition : ((isnotnull(d_week_seq#75) AND (d_week_seq#75 = Subquery scalar-subquery#76, [id=#77])) AND isnotnull(d_date_sk#74)) -(102) Project [codegen id : 66] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#33] +(102) Project [codegen id : 63] +Output [1]: [d_date_sk#74] +Input [2]: [d_date_sk#74, d_week_seq#75] (103) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] +Input [1]: [d_date_sk#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] -(104) BroadcastHashJoin [codegen id : 88] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(104) BroadcastHashJoin [codegen id : 84] +Left keys [1]: [ss_sold_date_sk#71] +Right keys [1]: [d_date_sk#74] Join condition: None -(105) Project [codegen id : 88] -Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +(105) Project [codegen id : 84] +Output [3]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70] +Input [5]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71, d_date_sk#74] (106) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +Output [4]: [i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] -(107) BroadcastHashJoin [codegen id : 88] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#59] +(107) BroadcastHashJoin [codegen id : 84] +Left keys [1]: [ss_item_sk#68] +Right keys [1]: [i_item_sk#79] Join condition: None -(108) Project [codegen id : 88] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +(108) Project [codegen id : 84] +Output [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#80, i_class_id#81, i_category_id#82] +Input [7]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] -(109) HashAggregate [codegen id : 88] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] +(109) HashAggregate [codegen id : 84] +Input [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#80, i_class_id#81, i_category_id#82] +Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] +Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] (110) Exchange -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] -Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] +Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [id=#89] -(111) HashAggregate [codegen id : 89] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70, count(1)#71] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sales#72, count(1)#71 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] +(111) HashAggregate [codegen id : 85] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] +Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90, count(1)#91] +Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sales#92, count(1)#91 AS number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] -(112) Filter [codegen id : 89] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) +(112) Filter [codegen id : 85] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) -(113) Project [codegen id : 89] -Output [6]: [store AS channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] +(113) Project [codegen id : 85] +Output [6]: [store AS channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] (114) BroadcastExchange -Input [6]: [channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#76] +Input [6]: [channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#96] -(115) BroadcastHashJoin [codegen id : 90] +(115) BroadcastHashJoin [codegen id : 86] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Right keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] Join condition: None (116) TakeOrderedAndProject -Input [12]: [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Input [12]: [channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#65, [id=#66] * HashAggregate (139) +- Exchange (138) +- * HashAggregate (137) @@ -677,7 +677,7 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#97)] ReadSchema: struct (118) ColumnarToRow [codegen id : 2] @@ -703,7 +703,7 @@ Input [2]: [d_date_sk#12, d_year#13] (123) BroadcastExchange Input [1]: [d_date_sk#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=#98] (124) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#4] @@ -711,83 +711,83 @@ Right keys [1]: [d_date_sk#12] Join condition: None (125) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#79, ss_list_price#3 AS list_price#80] +Output [2]: [ss_quantity#2 AS quantity#99, ss_list_price#3 AS list_price#100] Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (126) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] +Output [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#97)] ReadSchema: struct (127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] +Input [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21] (128) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#103] (129) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#103] Join condition: None (130) Project [codegen id : 4] -Output [2]: [cs_quantity#81 AS quantity#83, cs_list_price#82 AS list_price#84] -Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21, d_date_sk#12] +Output [2]: [cs_quantity#101 AS quantity#104, cs_list_price#102 AS list_price#105] +Input [4]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21, d_date_sk#103] (131) Scan parquet default.web_sales -Output [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] +Output [3]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#97)] ReadSchema: struct (132) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] +Input [3]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36] (133) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#108] (134) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#108] Join condition: None (135) Project [codegen id : 6] -Output [2]: [ws_quantity#85 AS quantity#87, ws_list_price#86 AS list_price#88] -Input [4]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27, d_date_sk#12] +Output [2]: [ws_quantity#106 AS quantity#109, ws_list_price#107 AS list_price#110] +Input [4]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36, d_date_sk#108] (136) Union (137) HashAggregate [codegen id : 7] -Input [2]: [quantity#79, list_price#80] +Input [2]: [quantity#99, list_price#100] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#89, count#90] -Results [2]: [sum#91, count#92] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#111, count#112] +Results [2]: [sum#113, count#114] (138) Exchange -Input [2]: [sum#91, count#92] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] +Input [2]: [sum#113, count#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] (139) HashAggregate [codegen id : 8] -Input [2]: [sum#91, count#92] +Input [2]: [sum#113, count#114] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94 AS average_sales#95] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#116] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#116 AS average_sales#117] -Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#77 +Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#97 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 123] Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#77 +Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#97 -Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#77 +Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#97 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (141) @@ -803,11 +803,11 @@ ReusedExchange (142) (142) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#11 -Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#34, [id=#35] +Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#48, [id=#49] * Project (146) +- * Filter (145) +- * ColumnarToRow (144) @@ -815,33 +815,33 @@ Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquer (143) Scan parquet default.date_dim -Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] (145) Filter [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 2000)) AND (d_moy#96 = 12)) AND (d_dom#97 = 11)) +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#13 = 2000)) AND (d_moy#118 = 12)) AND (d_dom#119 = 11)) (146) Project [codegen id : 1] -Output [1]: [d_week_seq#33] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [1]: [d_week_seq#47] +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] -Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] +Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#65, [id=#66] -Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#54 +Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#71 IN dynamicpruning#72 ReusedExchange (147) (147) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#74] -Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#56, [id=#57] +Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#76, [id=#77] * Project (151) +- * Filter (150) +- * ColumnarToRow (149) @@ -849,21 +849,21 @@ Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subqu (148) Scan parquet default.date_dim -Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (149) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] (150) Filter [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 1999)) AND (d_moy#96 = 12)) AND (d_dom#97 = 11)) +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#13 = 1999)) AND (d_moy#118 = 12)) AND (d_dom#119 = 11)) (151) Project [codegen id : 1] -Output [1]: [d_week_seq#33] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [1]: [d_week_seq#47] +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt index 1d5907e319658..1b41abbc97b06 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (90) + WholeStageCodegen (86) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] @@ -49,7 +49,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (44) + WholeStageCodegen (42) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -68,11 +68,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #3 - WholeStageCodegen (21) + WholeStageCodegen (20) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 - WholeStageCodegen (20) + WholeStageCodegen (19) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -81,84 +81,82 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (19) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (18) + WholeStageCodegen (17) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (13) + WholeStageCodegen (12) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (11) + WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #9 InputAdapter - Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] + BroadcastExchange #9 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #10 + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #9 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (7) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (17) + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (16) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (16) + WholeStageCodegen (15) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -169,12 +167,12 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #10 + ReusedExchange [d_date_sk] #9 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter BroadcastExchange #3 - WholeStageCodegen (22) + WholeStageCodegen (21) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #3 @@ -190,29 +188,29 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #15 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #16 - WholeStageCodegen (23) + WholeStageCodegen (22) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #20 - WholeStageCodegen (89) + WholeStageCodegen (85) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (88) + WholeStageCodegen (84) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -220,24 +218,24 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] - WholeStageCodegen (46) + WholeStageCodegen (44) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #22 - WholeStageCodegen (45) + WholeStageCodegen (43) Filter [ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #6 ReusedExchange [d_date_sk] #23 - WholeStageCodegen (65) + WholeStageCodegen (62) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #23 - WholeStageCodegen (66) + WholeStageCodegen (63) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 01bcb83622844..83604fd5cdff8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -27,36 +27,36 @@ TakeOrderedAndProject (100) : : : :- * HashAggregate (39) : : : : +- Exchange (38) : : : : +- * HashAggregate (37) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) + : : : : +- * Project (36) + : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : :- * Project (33) + : : : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet default.store_sales (7) + : : : : : +- BroadcastExchange (31) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Project (28) + : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet default.item (16) + : : : : : +- BroadcastExchange (26) + : : : : : +- * Project (25) + : : : : : +- * Filter (24) + : : : : : +- * ColumnarToRow (23) + : : : : : +- Scan parquet default.date_dim (22) + : : : : +- ReusedExchange (34) : : : +- BroadcastExchange (49) : : : +- * Project (48) : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -146,226 +146,226 @@ Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) (10) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(11) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -(12) Filter [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +(12) Filter [codegen id : 4] +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14)) -(13) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +(13) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join condition: None +(14) ColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] -(15) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(15) Filter [codegen id : 3] +Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Condition : isnotnull(cs_item_sk#15) -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +(16) Scan parquet default.item +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] 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 - -(17) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(18) Filter [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +(17) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -(19) Project [codegen id : 2] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +(18) Filter [codegen id : 1] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : isnotnull(i_item_sk#17) -(20) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +(19) BroadcastExchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [i_item_sk#17] Join condition: None -(22) Project [codegen id : 6] -Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] -Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -(23) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(22) Scan parquet default.date_dim +Output [2]: [d_date_sk#22, d_year#23] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +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 -(24) ColumnarToRow [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(23) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#22, d_year#23] -(25) Filter [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +(24) Filter [codegen id : 2] +Input [2]: [d_date_sk#22, d_year#23] +Condition : (((isnotnull(d_year#23) AND (d_year#23 >= 1999)) AND (d_year#23 <= 2001)) AND isnotnull(d_date_sk#22)) -(26) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +(25) Project [codegen id : 2] +Output [1]: [d_date_sk#22] +Input [2]: [d_date_sk#22, d_year#23] -(27) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(26) BroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(28) Filter [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +(27) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#22] +Join condition: None + +(28) Project [codegen id : 3] +Output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] +Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#22] (29) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] -(30) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +(30) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)] +Right keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] Join condition: None -(31) Project [codegen id : 5] -Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(31) BroadcastExchange +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] -(32) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] - -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +(32) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#11] Join condition: None -(34) Project [codegen id : 5] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(33) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14] +Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -(35) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] +(34) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#27] -(36) BroadcastHashJoin [codegen id : 6] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#27] Join condition: None +(36) Project [codegen id : 6] +Output [3]: [i_brand_id#12 AS brand_id#28, i_class_id#13 AS class_id#29, i_category_id#14 AS category_id#30] +Input [5]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#27] + (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#28, class_id#29, category_id#30] (38) Exchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [id=#31] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#28, class_id#29, category_id#30] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Output [2]: [ws_item_sk#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Condition : isnotnull(ws_item_sk#32) -(43) ReusedExchange [Reuses operator id: 29] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(43) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#32] +Right keys [1]: [i_item_sk#34] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37] +Input [6]: [ws_item_sk#32, ws_sold_date_sk#33, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -(46) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(46) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#38] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#38] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +Input [5]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] (49) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#39] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] +Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#28, class_id#29, category_id#30] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#28, class_id#29, category_id#30] (53) BroadcastExchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#40] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#15, class_id#16, category_id#17] +Right keys [3]: [brand_id#28, class_id#29, category_id#30] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#27] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] +Output [1]: [i_item_sk#6 AS ss_item_sk#41] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] (56) BroadcastExchange -Input [1]: [ss_item_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [ss_item_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#41] Join condition: None (58) Scan parquet default.item @@ -383,16 +383,16 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#41] (62) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#41] Join condition: None (63) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] @@ -404,170 +404,170 @@ Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_ Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (66) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#30] +Output [2]: [d_date_sk#27, d_week_seq#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [2]: [d_date_sk#12, d_week_seq#30] +Input [2]: [d_date_sk#27, d_week_seq#44] (68) Filter [codegen id : 24] -Input [2]: [d_date_sk#12, d_week_seq#30] -Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#31, [id=#32])) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#27, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#27)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#30] +Output [1]: [d_date_sk#27] +Input [2]: [d_date_sk#27, d_week_seq#44] (70) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#27] Join condition: None (72) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#27] (73) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#34, isEmpty#35, count#36] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] +Aggregate Attributes [3]: [sum#48, isEmpty#49, count#50] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#54] (75) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41, count(1)#42] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sales#43, count(1)#42 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55, count(1)#56] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sales#57, count(1)#56 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] (76) Filter [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45 as decimal(32,6)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59 as decimal(32,6)) > cast(Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) (77) Project [codegen id : 52] -Output [6]: [store AS channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] +Output [6]: [store AS channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] (78) Scan parquet default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#49)] +PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] (80) Filter [codegen id : 50] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] +Condition : isnotnull(ss_item_sk#63) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#41] (82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [ss_item_sk#63] +Right keys [1]: [ss_item_sk#41] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] +Output [4]: [i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] (84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#50] +Left keys [1]: [ss_item_sk#63] +Right keys [1]: [i_item_sk#68] Join condition: None (85) Project [codegen id : 50] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] +Output [6]: [ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_brand_id#69, i_class_id#70, i_category_id#71] +Input [8]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#30] +Output [2]: [d_date_sk#72, d_week_seq#73] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#12, d_week_seq#30] +Input [2]: [d_date_sk#72, d_week_seq#73] (88) Filter [codegen id : 49] -Input [2]: [d_date_sk#12, d_week_seq#30] -Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#54, [id=#55])) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#72, d_week_seq#73] +Condition : ((isnotnull(d_week_seq#73) AND (d_week_seq#73 = Subquery scalar-subquery#74, [id=#75])) AND isnotnull(d_date_sk#72)) (89) Project [codegen id : 49] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#30] +Output [1]: [d_date_sk#72] +Input [2]: [d_date_sk#72, d_week_seq#73] (90) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +Input [1]: [d_date_sk#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] (91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#66] +Right keys [1]: [d_date_sk#72] Join condition: None (92) Project [codegen id : 50] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53, d_date_sk#12] +Output [5]: [ss_quantity#64, ss_list_price#65, i_brand_id#69, i_class_id#70, i_category_id#71] +Input [7]: [ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_brand_id#69, i_class_id#70, i_category_id#71, d_date_sk#72] (93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] -Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#57, isEmpty#58, count#59] -Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] +Input [5]: [ss_quantity#64, ss_list_price#65, i_brand_id#69, i_class_id#70, i_category_id#71] +Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#77, isEmpty#78, count#79] +Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] (94) Exchange -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] -Arguments: hashpartitioning(i_brand_id#51, i_class_id#52, i_category_id#53, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] +Arguments: hashpartitioning(i_brand_id#69, i_class_id#70, i_category_id#71, 5), ENSURE_REQUIREMENTS, [id=#83] (95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] -Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] -Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] +Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84, count(1)#85] +Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sales#86, count(1)#85 AS number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] (96) Filter [codegen id : 51] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) (97) Project [codegen id : 51] -Output [6]: [store AS channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] +Output [6]: [store AS channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] (98) BroadcastExchange -Input [6]: [channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] +Input [6]: [channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#90] (99) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] +Right keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] Join condition: None (100) TakeOrderedAndProject -Input [12]: [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Input [12]: [channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] +Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#46, [id=#47] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#60, [id=#61] * HashAggregate (123) +- Exchange (122) +- * HashAggregate (121) @@ -597,137 +597,137 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#91)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (103) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#27, d_year#92] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#27, d_year#92] (105) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#27, d_year#92] +Condition : (((isnotnull(d_year#92) AND (d_year#92 >= 1999)) AND (d_year#92 <= 2001)) AND isnotnull(d_date_sk#27)) (106) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#27] +Input [2]: [d_date_sk#27, d_year#92] (107) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] (108) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#27] Join condition: None (109) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#73, ss_list_price#3 AS list_price#74] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#2 AS quantity#94, ss_list_price#3 AS list_price#95] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#27] (110) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] +Output [3]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#91)] ReadSchema: struct (111) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] +Input [3]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16] (112) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#98] (113) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#98] Join condition: None (114) Project [codegen id : 4] -Output [2]: [cs_quantity#75 AS quantity#77, cs_list_price#76 AS list_price#78] -Input [4]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19, d_date_sk#12] +Output [2]: [cs_quantity#96 AS quantity#99, cs_list_price#97 AS list_price#100] +Input [4]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16, d_date_sk#98] (115) Scan parquet default.web_sales -Output [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] +Output [3]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#91)] ReadSchema: struct (116) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] +Input [3]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33] (117) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#103] (118) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#103] Join condition: None (119) Project [codegen id : 6] -Output [2]: [ws_quantity#79 AS quantity#81, ws_list_price#80 AS list_price#82] -Input [4]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24, d_date_sk#12] +Output [2]: [ws_quantity#101 AS quantity#104, ws_list_price#102 AS list_price#105] +Input [4]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33, d_date_sk#103] (120) Union (121) HashAggregate [codegen id : 7] -Input [2]: [quantity#73, list_price#74] +Input [2]: [quantity#94, list_price#95] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#83, count#84] -Results [2]: [sum#85, count#86] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#106, count#107] +Results [2]: [sum#108, count#109] (122) Exchange -Input [2]: [sum#85, count#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#87] +Input [2]: [sum#108, count#109] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] (123) HashAggregate [codegen id : 8] -Input [2]: [sum#85, count#86] +Input [2]: [sum#108, count#109] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88 AS average_sales#89] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))#111] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))#111 AS average_sales#112] -Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#71 +Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#91 ReusedExchange (124) (124) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#27] -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#71 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#91 -Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#71 +Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#91 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#27] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 ReusedExchange (126) -(126) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(126) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#27] -Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#10 -Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#31, [id=#32] +Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#45, [id=#46] * Project (130) +- * Filter (129) +- * ColumnarToRow (128) @@ -735,33 +735,33 @@ Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer (127) Scan parquet default.date_dim -Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (128) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] (129) Filter [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 2000)) AND (d_moy#90 = 12)) AND (d_dom#91 = 11)) +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#92 = 2000)) AND (d_moy#113 = 12)) AND (d_dom#114 = 11)) (130) Project [codegen id : 1] -Output [1]: [d_week_seq#30] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [1]: [d_week_seq#44] +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] -Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#60, [id=#61] -Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#49 +Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67 ReusedExchange (131) (131) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#72] -Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#74, [id=#75] * Project (135) +- * Filter (134) +- * ColumnarToRow (133) @@ -769,21 +769,21 @@ Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subque (132) Scan parquet default.date_dim -Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (133) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] (134) Filter [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 1999)) AND (d_moy#90 = 12)) AND (d_dom#91 = 11)) +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#92 = 1999)) AND (d_moy#113 = 12)) AND (d_dom#114 = 11)) (135) Project [codegen id : 1] -Output [1]: [d_week_seq#30] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [1]: [d_week_seq#44] +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index 3507d76bcf9b9..cb9b410ea8568 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -82,53 +82,53 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Exchange [brand_id,class_id,category_id] #5 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #9 + BroadcastExchange #8 WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 + BroadcastExchange #6 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index cba099c2bb5b0..613e56b58c17b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- Union (63) +TakeOrderedAndProject (76) ++- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- Union (72) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (67) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) + :- * HashAggregate (56) + : +- Exchange (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,22 +50,31 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- ReusedExchange (42) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) + : +- BroadcastExchange (51) + : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : :- * Filter (44) + : : +- * ColumnarToRow (43) + : : +- Scan parquet default.item (42) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- * Filter (47) + : +- * ColumnarToRow (46) + : +- Scan parquet default.item (45) + +- * HashAggregate (71) + +- Exchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Filter (59) + : : : +- * ColumnarToRow (58) + : : : +- Scan parquet default.web_sales (57) + : : +- ReusedExchange (60) + : +- ReusedExchange (63) + +- ReusedExchange (66) (1) Scan parquet default.store_sales @@ -163,7 +172,7 @@ Condition : isnotnull(i_item_sk#13) Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +180,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics)) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] @@ -233,163 +242,205 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_manufact_id#14] +(42) Scan parquet default.item +Output [2]: [i_item_sk#30, i_manufact_id#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct + +(43) ColumnarToRow [codegen id : 10] +Input [2]: [i_item_sk#30, i_manufact_id#31] -(43) BroadcastHashJoin [codegen id : 11] +(44) Filter [codegen id : 10] +Input [2]: [i_item_sk#30, i_manufact_id#31] +Condition : isnotnull(i_item_sk#30) + +(45) Scan parquet default.item +Output [2]: [i_category#15, i_manufact_id#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] +ReadSchema: struct + +(46) ColumnarToRow [codegen id : 9] +Input [2]: [i_category#15, i_manufact_id#14] + +(47) Filter [codegen id : 9] +Input [2]: [i_category#15, i_manufact_id#14] +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) + +(48) Project [codegen id : 9] +Output [1]: [i_manufact_id#14] +Input [2]: [i_category#15, i_manufact_id#14] + +(49) BroadcastExchange +Input [1]: [i_manufact_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] + +(50) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [i_manufact_id#31] +Right keys [1]: [i_manufact_id#14] +Join condition: None + +(51) BroadcastExchange +Input [2]: [i_item_sk#30, i_manufact_id#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] + +(52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None -(44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_manufact_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_manufact_id#14] +(53) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#26, i_manufact_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_manufact_id#31] -(45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_manufact_id#14] -Keys [1]: [i_manufact_id#14] +(54) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#26, i_manufact_id#31] +Keys [1]: [i_manufact_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_manufact_id#14, sum#29] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_manufact_id#31, sum#35] -(46) Exchange -Input [2]: [i_manufact_id#14, sum#29] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +(55) Exchange +Input [2]: [i_manufact_id#31, sum#35] +Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] -(47) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#14, sum#29] -Keys [1]: [i_manufact_id#14] +(56) HashAggregate [codegen id : 12] +Input [2]: [i_manufact_id#31, sum#35] +Keys [1]: [i_manufact_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] +Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] -(48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(57) Scan parquet default.web_sales +Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(58) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -(50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +(59) Filter [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) -(51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +(60) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#43] -(52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +(61) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#42] +Right keys [1]: [d_date_sk#43] Join condition: None -(53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +(62) Project [codegen id : 17] +Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] +Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +(63) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#44] -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +(64) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#40] +Right keys [1]: [ca_address_sk#44] Join condition: None -(56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +(65) Project [codegen id : 17] +Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] -(57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_manufact_id#14] +(66) ReusedExchange [Reuses operator id: 51] +Output [2]: [i_item_sk#45, i_manufact_id#46] -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +(67) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#45] Join condition: None -(59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_manufact_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_manufact_id#14] +(68) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#41, i_manufact_id#46] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_manufact_id#46] -(60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_manufact_id#14] -Keys [1]: [i_manufact_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_manufact_id#14, sum#38] +(69) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#41, i_manufact_id#46] +Keys [1]: [i_manufact_id#46] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum#47] +Results [2]: [i_manufact_id#46, sum#48] -(61) Exchange -Input [2]: [i_manufact_id#14, sum#38] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +(70) Exchange +Input [2]: [i_manufact_id#46, sum#48] +Arguments: hashpartitioning(i_manufact_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] -(62) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#14, sum#38] -Keys [1]: [i_manufact_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +(71) HashAggregate [codegen id : 18] +Input [2]: [i_manufact_id#46, sum#48] +Keys [1]: [i_manufact_id#46] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] +Results [2]: [i_manufact_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] -(63) Union +(72) Union -(64) HashAggregate [codegen id : 19] +(73) HashAggregate [codegen id : 19] Input [2]: [i_manufact_id#14, total_sales#23] Keys [1]: [i_manufact_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_manufact_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#52, isEmpty#53] +Results [3]: [i_manufact_id#14, sum#54, isEmpty#55] -(65) Exchange -Input [3]: [i_manufact_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +(74) Exchange +Input [3]: [i_manufact_id#14, sum#54, isEmpty#55] +Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] -(66) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#14, sum#44, isEmpty#45] +(75) HashAggregate [codegen id : 20] +Input [3]: [i_manufact_id#14, sum#54, isEmpty#55] Keys [1]: [i_manufact_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_manufact_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#57] +Results [2]: [i_manufact_id#14, sum(total_sales#23)#57 AS total_sales#58] -(67) TakeOrderedAndProject -Input [2]: [i_manufact_id#14, total_sales#48] -Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_manufact_id#14, total_sales#48] +(76) TakeOrderedAndProject +Input [2]: [i_manufact_id#14, total_sales#58] +Arguments: 100, [total_sales#58 ASC NULLS FIRST], [i_manufact_id#14, total_sales#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (68) +ReusedExchange (77) -(68) ReusedExchange [Reuses operator id: 8] +(77) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt index d1a115fa528d5..ad5473e9da920 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt @@ -79,11 +79,25 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + BroadcastExchange #8 + WholeStageCodegen (10) + BroadcastHashJoin [i_manufact_id,i_manufact_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_manufact_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_category,i_manufact_id] WholeStageCodegen (18) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_manufact_id] #8 + Exchange [i_manufact_id] #10 WholeStageCodegen (17) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_manufact_id] @@ -102,4 +116,4 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + ReusedExchange [i_item_sk,i_manufact_id] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index cba099c2bb5b0..613e56b58c17b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- Union (63) +TakeOrderedAndProject (76) ++- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- Union (72) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (67) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) + :- * HashAggregate (56) + : +- Exchange (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,22 +50,31 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- ReusedExchange (42) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) + : +- BroadcastExchange (51) + : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : :- * Filter (44) + : : +- * ColumnarToRow (43) + : : +- Scan parquet default.item (42) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- * Filter (47) + : +- * ColumnarToRow (46) + : +- Scan parquet default.item (45) + +- * HashAggregate (71) + +- Exchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Filter (59) + : : : +- * ColumnarToRow (58) + : : : +- Scan parquet default.web_sales (57) + : : +- ReusedExchange (60) + : +- ReusedExchange (63) + +- ReusedExchange (66) (1) Scan parquet default.store_sales @@ -163,7 +172,7 @@ Condition : isnotnull(i_item_sk#13) Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +180,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics)) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] @@ -233,163 +242,205 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_manufact_id#14] +(42) Scan parquet default.item +Output [2]: [i_item_sk#30, i_manufact_id#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct + +(43) ColumnarToRow [codegen id : 10] +Input [2]: [i_item_sk#30, i_manufact_id#31] -(43) BroadcastHashJoin [codegen id : 11] +(44) Filter [codegen id : 10] +Input [2]: [i_item_sk#30, i_manufact_id#31] +Condition : isnotnull(i_item_sk#30) + +(45) Scan parquet default.item +Output [2]: [i_category#15, i_manufact_id#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] +ReadSchema: struct + +(46) ColumnarToRow [codegen id : 9] +Input [2]: [i_category#15, i_manufact_id#14] + +(47) Filter [codegen id : 9] +Input [2]: [i_category#15, i_manufact_id#14] +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) + +(48) Project [codegen id : 9] +Output [1]: [i_manufact_id#14] +Input [2]: [i_category#15, i_manufact_id#14] + +(49) BroadcastExchange +Input [1]: [i_manufact_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] + +(50) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [i_manufact_id#31] +Right keys [1]: [i_manufact_id#14] +Join condition: None + +(51) BroadcastExchange +Input [2]: [i_item_sk#30, i_manufact_id#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] + +(52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None -(44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_manufact_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_manufact_id#14] +(53) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#26, i_manufact_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_manufact_id#31] -(45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_manufact_id#14] -Keys [1]: [i_manufact_id#14] +(54) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#26, i_manufact_id#31] +Keys [1]: [i_manufact_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_manufact_id#14, sum#29] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_manufact_id#31, sum#35] -(46) Exchange -Input [2]: [i_manufact_id#14, sum#29] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +(55) Exchange +Input [2]: [i_manufact_id#31, sum#35] +Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] -(47) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#14, sum#29] -Keys [1]: [i_manufact_id#14] +(56) HashAggregate [codegen id : 12] +Input [2]: [i_manufact_id#31, sum#35] +Keys [1]: [i_manufact_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] +Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] -(48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(57) Scan parquet default.web_sales +Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(58) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -(50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +(59) Filter [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) -(51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +(60) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#43] -(52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +(61) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#42] +Right keys [1]: [d_date_sk#43] Join condition: None -(53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +(62) Project [codegen id : 17] +Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] +Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +(63) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#44] -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +(64) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#40] +Right keys [1]: [ca_address_sk#44] Join condition: None -(56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +(65) Project [codegen id : 17] +Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] -(57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_manufact_id#14] +(66) ReusedExchange [Reuses operator id: 51] +Output [2]: [i_item_sk#45, i_manufact_id#46] -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +(67) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#45] Join condition: None -(59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_manufact_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_manufact_id#14] +(68) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#41, i_manufact_id#46] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_manufact_id#46] -(60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_manufact_id#14] -Keys [1]: [i_manufact_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_manufact_id#14, sum#38] +(69) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#41, i_manufact_id#46] +Keys [1]: [i_manufact_id#46] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum#47] +Results [2]: [i_manufact_id#46, sum#48] -(61) Exchange -Input [2]: [i_manufact_id#14, sum#38] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +(70) Exchange +Input [2]: [i_manufact_id#46, sum#48] +Arguments: hashpartitioning(i_manufact_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] -(62) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#14, sum#38] -Keys [1]: [i_manufact_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +(71) HashAggregate [codegen id : 18] +Input [2]: [i_manufact_id#46, sum#48] +Keys [1]: [i_manufact_id#46] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] +Results [2]: [i_manufact_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] -(63) Union +(72) Union -(64) HashAggregate [codegen id : 19] +(73) HashAggregate [codegen id : 19] Input [2]: [i_manufact_id#14, total_sales#23] Keys [1]: [i_manufact_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_manufact_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#52, isEmpty#53] +Results [3]: [i_manufact_id#14, sum#54, isEmpty#55] -(65) Exchange -Input [3]: [i_manufact_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +(74) Exchange +Input [3]: [i_manufact_id#14, sum#54, isEmpty#55] +Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] -(66) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#14, sum#44, isEmpty#45] +(75) HashAggregate [codegen id : 20] +Input [3]: [i_manufact_id#14, sum#54, isEmpty#55] Keys [1]: [i_manufact_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_manufact_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#57] +Results [2]: [i_manufact_id#14, sum(total_sales#23)#57 AS total_sales#58] -(67) TakeOrderedAndProject -Input [2]: [i_manufact_id#14, total_sales#48] -Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_manufact_id#14, total_sales#48] +(76) TakeOrderedAndProject +Input [2]: [i_manufact_id#14, total_sales#58] +Arguments: 100, [total_sales#58 ASC NULLS FIRST], [i_manufact_id#14, total_sales#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (68) +ReusedExchange (77) -(68) ReusedExchange [Reuses operator id: 8] +(77) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index d1a115fa528d5..ad5473e9da920 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -79,11 +79,25 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + BroadcastExchange #8 + WholeStageCodegen (10) + BroadcastHashJoin [i_manufact_id,i_manufact_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_manufact_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_category,i_manufact_id] WholeStageCodegen (18) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_manufact_id] #8 + Exchange [i_manufact_id] #10 WholeStageCodegen (17) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_manufact_id] @@ -102,4 +116,4 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + ReusedExchange [i_item_sk,i_manufact_id] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt index 8e3bf5bf57bcf..3c6fe13107e04 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- Union (63) +TakeOrderedAndProject (76) ++- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- Union (72) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (67) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) + :- * HashAggregate (56) + : +- Exchange (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,22 +50,31 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- ReusedExchange (42) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) + : +- BroadcastExchange (51) + : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : :- * Filter (44) + : : +- * ColumnarToRow (43) + : : +- Scan parquet default.item (42) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- * Filter (47) + : +- * ColumnarToRow (46) + : +- Scan parquet default.item (45) + +- * HashAggregate (71) + +- Exchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Filter (59) + : : : +- * ColumnarToRow (58) + : : : +- Scan parquet default.web_sales (57) + : : +- ReusedExchange (60) + : +- ReusedExchange (63) + +- ReusedExchange (66) (1) Scan parquet default.store_sales @@ -233,163 +242,205 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +(42) Scan parquet default.item +Output [2]: [i_item_sk#30, i_item_id#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct + +(43) ColumnarToRow [codegen id : 10] +Input [2]: [i_item_sk#30, i_item_id#31] -(43) BroadcastHashJoin [codegen id : 11] +(44) Filter [codegen id : 10] +Input [2]: [i_item_sk#30, i_item_id#31] +Condition : isnotnull(i_item_sk#30) + +(45) Scan parquet default.item +Output [2]: [i_item_id#14, i_color#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] +ReadSchema: struct + +(46) ColumnarToRow [codegen id : 9] +Input [2]: [i_item_id#14, i_color#15] + +(47) Filter [codegen id : 9] +Input [2]: [i_item_id#14, i_color#15] +Condition : i_color#15 IN (slate ,blanched ,burnished ) + +(48) Project [codegen id : 9] +Output [1]: [i_item_id#14] +Input [2]: [i_item_id#14, i_color#15] + +(49) BroadcastExchange +Input [1]: [i_item_id#14] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] + +(50) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [i_item_id#31] +Right keys [1]: [i_item_id#14] +Join condition: None + +(51) BroadcastExchange +Input [2]: [i_item_sk#30, i_item_id#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] + +(52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None -(44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] +(53) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#26, i_item_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] -(45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#14] -Keys [1]: [i_item_id#14] +(54) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#26, i_item_id#31] +Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#14, sum#29] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#31, sum#35] -(46) Exchange -Input [2]: [i_item_id#14, sum#29] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +(55) Exchange +Input [2]: [i_item_id#31, sum#35] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] -(47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#14, sum#29] -Keys [1]: [i_item_id#14] +(56) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#31, sum#35] +Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] -(48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(57) Scan parquet default.web_sales +Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(58) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -(50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +(59) Filter [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) -(51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +(60) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#43] -(52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +(61) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#42] +Right keys [1]: [d_date_sk#43] Join condition: None -(53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +(62) Project [codegen id : 17] +Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] +Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +(63) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#44] -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +(64) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#40] +Right keys [1]: [ca_address_sk#44] Join condition: None -(56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +(65) Project [codegen id : 17] +Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] -(57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +(66) ReusedExchange [Reuses operator id: 51] +Output [2]: [i_item_sk#45, i_item_id#46] -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +(67) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#45] Join condition: None -(59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_item_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] +(68) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#41, i_item_id#46] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_item_id#46] -(60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_item_id#14, sum#38] +(69) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#41, i_item_id#46] +Keys [1]: [i_item_id#46] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum#47] +Results [2]: [i_item_id#46, sum#48] -(61) Exchange -Input [2]: [i_item_id#14, sum#38] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +(70) Exchange +Input [2]: [i_item_id#46, sum#48] +Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] -(62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#38] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +(71) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#46, sum#48] +Keys [1]: [i_item_id#46] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] +Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] -(63) Union +(72) Union -(64) HashAggregate [codegen id : 19] +(73) HashAggregate [codegen id : 19] Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_item_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#52, isEmpty#53] +Results [3]: [i_item_id#14, sum#54, isEmpty#55] -(65) Exchange -Input [3]: [i_item_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +(74) Exchange +Input [3]: [i_item_id#14, sum#54, isEmpty#55] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] -(66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#44, isEmpty#45] +(75) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#14, sum#54, isEmpty#55] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#57] +Results [2]: [i_item_id#14, sum(total_sales#23)#57 AS total_sales#58] -(67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#48] -Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] +(76) TakeOrderedAndProject +Input [2]: [i_item_id#14, total_sales#58] +Arguments: 100, [total_sales#58 ASC NULLS FIRST], [i_item_id#14, total_sales#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (68) +ReusedExchange (77) -(68) ReusedExchange [Reuses operator id: 8] +(77) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt index 2d21010c26b8d..34d320db2bb91 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt @@ -79,11 +79,25 @@ TakeOrderedAndProject [total_sales,i_item_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + BroadcastExchange #8 + WholeStageCodegen (10) + BroadcastHashJoin [i_item_id,i_item_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_item_id] + Filter [i_color] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_id,i_color] WholeStageCodegen (18) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #8 + Exchange [i_item_id] #10 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -102,4 +116,4 @@ TakeOrderedAndProject [total_sales,i_item_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [i_item_sk,i_item_id] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 8e3bf5bf57bcf..3c6fe13107e04 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- Union (63) +TakeOrderedAndProject (76) ++- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- Union (72) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (67) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) + :- * HashAggregate (56) + : +- Exchange (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,22 +50,31 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- ReusedExchange (42) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) + : +- BroadcastExchange (51) + : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : :- * Filter (44) + : : +- * ColumnarToRow (43) + : : +- Scan parquet default.item (42) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- * Filter (47) + : +- * ColumnarToRow (46) + : +- Scan parquet default.item (45) + +- * HashAggregate (71) + +- Exchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Filter (59) + : : : +- * ColumnarToRow (58) + : : : +- Scan parquet default.web_sales (57) + : : +- ReusedExchange (60) + : +- ReusedExchange (63) + +- ReusedExchange (66) (1) Scan parquet default.store_sales @@ -233,163 +242,205 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +(42) Scan parquet default.item +Output [2]: [i_item_sk#30, i_item_id#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct + +(43) ColumnarToRow [codegen id : 10] +Input [2]: [i_item_sk#30, i_item_id#31] -(43) BroadcastHashJoin [codegen id : 11] +(44) Filter [codegen id : 10] +Input [2]: [i_item_sk#30, i_item_id#31] +Condition : isnotnull(i_item_sk#30) + +(45) Scan parquet default.item +Output [2]: [i_item_id#14, i_color#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] +ReadSchema: struct + +(46) ColumnarToRow [codegen id : 9] +Input [2]: [i_item_id#14, i_color#15] + +(47) Filter [codegen id : 9] +Input [2]: [i_item_id#14, i_color#15] +Condition : i_color#15 IN (slate ,blanched ,burnished ) + +(48) Project [codegen id : 9] +Output [1]: [i_item_id#14] +Input [2]: [i_item_id#14, i_color#15] + +(49) BroadcastExchange +Input [1]: [i_item_id#14] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] + +(50) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [i_item_id#31] +Right keys [1]: [i_item_id#14] +Join condition: None + +(51) BroadcastExchange +Input [2]: [i_item_sk#30, i_item_id#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] + +(52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None -(44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] +(53) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#26, i_item_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] -(45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#14] -Keys [1]: [i_item_id#14] +(54) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#26, i_item_id#31] +Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#14, sum#29] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#31, sum#35] -(46) Exchange -Input [2]: [i_item_id#14, sum#29] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +(55) Exchange +Input [2]: [i_item_id#31, sum#35] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] -(47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#14, sum#29] -Keys [1]: [i_item_id#14] +(56) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#31, sum#35] +Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] -(48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(57) Scan parquet default.web_sales +Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(58) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -(50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +(59) Filter [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) -(51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +(60) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#43] -(52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +(61) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#42] +Right keys [1]: [d_date_sk#43] Join condition: None -(53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +(62) Project [codegen id : 17] +Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] +Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +(63) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#44] -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +(64) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#40] +Right keys [1]: [ca_address_sk#44] Join condition: None -(56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +(65) Project [codegen id : 17] +Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] -(57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +(66) ReusedExchange [Reuses operator id: 51] +Output [2]: [i_item_sk#45, i_item_id#46] -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +(67) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#45] Join condition: None -(59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_item_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] +(68) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#41, i_item_id#46] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_item_id#46] -(60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_item_id#14, sum#38] +(69) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#41, i_item_id#46] +Keys [1]: [i_item_id#46] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum#47] +Results [2]: [i_item_id#46, sum#48] -(61) Exchange -Input [2]: [i_item_id#14, sum#38] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +(70) Exchange +Input [2]: [i_item_id#46, sum#48] +Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] -(62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#38] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +(71) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#46, sum#48] +Keys [1]: [i_item_id#46] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] +Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] -(63) Union +(72) Union -(64) HashAggregate [codegen id : 19] +(73) HashAggregate [codegen id : 19] Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_item_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#52, isEmpty#53] +Results [3]: [i_item_id#14, sum#54, isEmpty#55] -(65) Exchange -Input [3]: [i_item_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +(74) Exchange +Input [3]: [i_item_id#14, sum#54, isEmpty#55] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] -(66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#44, isEmpty#45] +(75) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#14, sum#54, isEmpty#55] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#57] +Results [2]: [i_item_id#14, sum(total_sales#23)#57 AS total_sales#58] -(67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#48] -Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] +(76) TakeOrderedAndProject +Input [2]: [i_item_id#14, total_sales#58] +Arguments: 100, [total_sales#58 ASC NULLS FIRST], [i_item_id#14, total_sales#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (68) +ReusedExchange (77) -(68) ReusedExchange [Reuses operator id: 8] +(77) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 2d21010c26b8d..34d320db2bb91 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -79,11 +79,25 @@ TakeOrderedAndProject [total_sales,i_item_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + BroadcastExchange #8 + WholeStageCodegen (10) + BroadcastHashJoin [i_item_id,i_item_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_item_id] + Filter [i_color] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_id,i_color] WholeStageCodegen (18) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #8 + Exchange [i_item_id] #10 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -102,4 +116,4 @@ TakeOrderedAndProject [total_sales,i_item_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [i_item_sk,i_item_id] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt index 10b8d0d9e7f05..8774d6d342fb0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- Union (63) +TakeOrderedAndProject (76) ++- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- Union (72) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (67) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) + :- * HashAggregate (56) + : +- Exchange (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,22 +50,31 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- ReusedExchange (42) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) + : +- BroadcastExchange (51) + : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : :- * Filter (44) + : : +- * ColumnarToRow (43) + : : +- Scan parquet default.item (42) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- * Filter (47) + : +- * ColumnarToRow (46) + : +- Scan parquet default.item (45) + +- * HashAggregate (71) + +- Exchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Filter (59) + : : : +- * ColumnarToRow (58) + : : : +- Scan parquet default.web_sales (57) + : : +- ReusedExchange (60) + : +- ReusedExchange (63) + +- ReusedExchange (66) (1) Scan parquet default.store_sales @@ -233,163 +242,205 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +(42) Scan parquet default.item +Output [2]: [i_item_sk#30, i_item_id#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct + +(43) ColumnarToRow [codegen id : 10] +Input [2]: [i_item_sk#30, i_item_id#31] -(43) BroadcastHashJoin [codegen id : 11] +(44) Filter [codegen id : 10] +Input [2]: [i_item_sk#30, i_item_id#31] +Condition : isnotnull(i_item_sk#30) + +(45) Scan parquet default.item +Output [2]: [i_item_id#14, i_category#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] +ReadSchema: struct + +(46) ColumnarToRow [codegen id : 9] +Input [2]: [i_item_id#14, i_category#15] + +(47) Filter [codegen id : 9] +Input [2]: [i_item_id#14, i_category#15] +Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) + +(48) Project [codegen id : 9] +Output [1]: [i_item_id#14] +Input [2]: [i_item_id#14, i_category#15] + +(49) BroadcastExchange +Input [1]: [i_item_id#14] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] + +(50) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [i_item_id#31] +Right keys [1]: [i_item_id#14] +Join condition: None + +(51) BroadcastExchange +Input [2]: [i_item_sk#30, i_item_id#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] + +(52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None -(44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] +(53) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#26, i_item_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] -(45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#14] -Keys [1]: [i_item_id#14] +(54) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#26, i_item_id#31] +Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#14, sum#29] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#31, sum#35] -(46) Exchange -Input [2]: [i_item_id#14, sum#29] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +(55) Exchange +Input [2]: [i_item_id#31, sum#35] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] -(47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#14, sum#29] -Keys [1]: [i_item_id#14] +(56) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#31, sum#35] +Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] -(48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(57) Scan parquet default.web_sales +Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(58) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -(50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +(59) Filter [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) -(51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +(60) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#43] -(52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +(61) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#42] +Right keys [1]: [d_date_sk#43] Join condition: None -(53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +(62) Project [codegen id : 17] +Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] +Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +(63) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#44] -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +(64) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#40] +Right keys [1]: [ca_address_sk#44] Join condition: None -(56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +(65) Project [codegen id : 17] +Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] -(57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +(66) ReusedExchange [Reuses operator id: 51] +Output [2]: [i_item_sk#45, i_item_id#46] -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +(67) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#45] Join condition: None -(59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_item_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] +(68) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#41, i_item_id#46] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_item_id#46] -(60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_item_id#14, sum#38] +(69) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#41, i_item_id#46] +Keys [1]: [i_item_id#46] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum#47] +Results [2]: [i_item_id#46, sum#48] -(61) Exchange -Input [2]: [i_item_id#14, sum#38] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +(70) Exchange +Input [2]: [i_item_id#46, sum#48] +Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] -(62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#38] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +(71) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#46, sum#48] +Keys [1]: [i_item_id#46] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] +Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] -(63) Union +(72) Union -(64) HashAggregate [codegen id : 19] +(73) HashAggregate [codegen id : 19] Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_item_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#52, isEmpty#53] +Results [3]: [i_item_id#14, sum#54, isEmpty#55] -(65) Exchange -Input [3]: [i_item_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +(74) Exchange +Input [3]: [i_item_id#14, sum#54, isEmpty#55] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] -(66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#44, isEmpty#45] +(75) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#14, sum#54, isEmpty#55] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#57] +Results [2]: [i_item_id#14, sum(total_sales#23)#57 AS total_sales#58] -(67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#48] -Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] +(76) TakeOrderedAndProject +Input [2]: [i_item_id#14, total_sales#58] +Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#58 ASC NULLS FIRST], [i_item_id#14, total_sales#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (68) +ReusedExchange (77) -(68) ReusedExchange [Reuses operator id: 8] +(77) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt index 46775417cbb70..948e022cd11c8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt @@ -79,11 +79,25 @@ TakeOrderedAndProject [i_item_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + BroadcastExchange #8 + WholeStageCodegen (10) + BroadcastHashJoin [i_item_id,i_item_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_item_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_id,i_category] WholeStageCodegen (18) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #8 + Exchange [i_item_id] #10 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -102,4 +116,4 @@ TakeOrderedAndProject [i_item_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [i_item_sk,i_item_id] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 10b8d0d9e7f05..8774d6d342fb0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- Union (63) +TakeOrderedAndProject (76) ++- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- Union (72) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (67) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) + :- * HashAggregate (56) + : +- Exchange (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,22 +50,31 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- ReusedExchange (42) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) + : +- BroadcastExchange (51) + : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : :- * Filter (44) + : : +- * ColumnarToRow (43) + : : +- Scan parquet default.item (42) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- * Filter (47) + : +- * ColumnarToRow (46) + : +- Scan parquet default.item (45) + +- * HashAggregate (71) + +- Exchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Filter (59) + : : : +- * ColumnarToRow (58) + : : : +- Scan parquet default.web_sales (57) + : : +- ReusedExchange (60) + : +- ReusedExchange (63) + +- ReusedExchange (66) (1) Scan parquet default.store_sales @@ -233,163 +242,205 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +(42) Scan parquet default.item +Output [2]: [i_item_sk#30, i_item_id#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct + +(43) ColumnarToRow [codegen id : 10] +Input [2]: [i_item_sk#30, i_item_id#31] -(43) BroadcastHashJoin [codegen id : 11] +(44) Filter [codegen id : 10] +Input [2]: [i_item_sk#30, i_item_id#31] +Condition : isnotnull(i_item_sk#30) + +(45) Scan parquet default.item +Output [2]: [i_item_id#14, i_category#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] +ReadSchema: struct + +(46) ColumnarToRow [codegen id : 9] +Input [2]: [i_item_id#14, i_category#15] + +(47) Filter [codegen id : 9] +Input [2]: [i_item_id#14, i_category#15] +Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) + +(48) Project [codegen id : 9] +Output [1]: [i_item_id#14] +Input [2]: [i_item_id#14, i_category#15] + +(49) BroadcastExchange +Input [1]: [i_item_id#14] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] + +(50) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [i_item_id#31] +Right keys [1]: [i_item_id#14] +Join condition: None + +(51) BroadcastExchange +Input [2]: [i_item_sk#30, i_item_id#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] + +(52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None -(44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] +(53) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#26, i_item_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] -(45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#14] -Keys [1]: [i_item_id#14] +(54) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#26, i_item_id#31] +Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#14, sum#29] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#31, sum#35] -(46) Exchange -Input [2]: [i_item_id#14, sum#29] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +(55) Exchange +Input [2]: [i_item_id#31, sum#35] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] -(47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#14, sum#29] -Keys [1]: [i_item_id#14] +(56) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#31, sum#35] +Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] -(48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(57) Scan parquet default.web_sales +Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(58) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -(50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +(59) Filter [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) -(51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +(60) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#43] -(52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +(61) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#42] +Right keys [1]: [d_date_sk#43] Join condition: None -(53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +(62) Project [codegen id : 17] +Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] +Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +(63) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#44] -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +(64) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#40] +Right keys [1]: [ca_address_sk#44] Join condition: None -(56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +(65) Project [codegen id : 17] +Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] -(57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +(66) ReusedExchange [Reuses operator id: 51] +Output [2]: [i_item_sk#45, i_item_id#46] -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +(67) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#45] Join condition: None -(59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_item_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] +(68) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#41, i_item_id#46] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_item_id#46] -(60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_item_id#14, sum#38] +(69) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#41, i_item_id#46] +Keys [1]: [i_item_id#46] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum#47] +Results [2]: [i_item_id#46, sum#48] -(61) Exchange -Input [2]: [i_item_id#14, sum#38] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +(70) Exchange +Input [2]: [i_item_id#46, sum#48] +Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] -(62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#38] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +(71) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#46, sum#48] +Keys [1]: [i_item_id#46] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] +Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] -(63) Union +(72) Union -(64) HashAggregate [codegen id : 19] +(73) HashAggregate [codegen id : 19] Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_item_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#52, isEmpty#53] +Results [3]: [i_item_id#14, sum#54, isEmpty#55] -(65) Exchange -Input [3]: [i_item_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +(74) Exchange +Input [3]: [i_item_id#14, sum#54, isEmpty#55] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] -(66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#44, isEmpty#45] +(75) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#14, sum#54, isEmpty#55] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#57] +Results [2]: [i_item_id#14, sum(total_sales#23)#57 AS total_sales#58] -(67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#48] -Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] +(76) TakeOrderedAndProject +Input [2]: [i_item_id#14, total_sales#58] +Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#58 ASC NULLS FIRST], [i_item_id#14, total_sales#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (68) +ReusedExchange (77) -(68) ReusedExchange [Reuses operator id: 8] +(77) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index 46775417cbb70..948e022cd11c8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -79,11 +79,25 @@ TakeOrderedAndProject [i_item_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + BroadcastExchange #8 + WholeStageCodegen (10) + BroadcastHashJoin [i_item_id,i_item_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_item_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_id,i_category] WholeStageCodegen (18) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #8 + Exchange [i_item_id] #10 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -102,4 +116,4 @@ TakeOrderedAndProject [i_item_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [i_item_sk,i_item_id] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt index d006b61d20c33..ef2ac135f64eb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (61) -+- * Project (60) - +- * BroadcastHashJoin Inner BuildRight (59) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) +TakeOrderedAndProject (72) ++- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) : :- * HashAggregate (30) : : +- Exchange (29) : : +- * HashAggregate (28) @@ -34,32 +34,43 @@ TakeOrderedAndProject (61) : : +- * Filter (24) : : +- * ColumnarToRow (23) : : +- Scan parquet default.item (22) - : +- BroadcastExchange (43) - : +- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) + : +- BroadcastExchange (54) + : +- * HashAggregate (53) + : +- Exchange (52) + : +- * HashAggregate (51) + : +- * Project (50) + : +- * BroadcastHashJoin Inner BuildRight (49) + : :- * Project (47) + : : +- * BroadcastHashJoin Inner BuildRight (46) : : :- * Filter (33) : : : +- * ColumnarToRow (32) : : : +- Scan parquet default.catalog_returns (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- BroadcastExchange (58) - +- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- * Project (54) - +- * BroadcastHashJoin Inner BuildRight (53) - :- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet default.web_returns (46) - : +- ReusedExchange (49) - +- ReusedExchange (52) + : : +- BroadcastExchange (45) + : : +- * Project (44) + : : +- * BroadcastHashJoin LeftSemi BuildRight (43) + : : :- * Filter (36) + : : : +- * ColumnarToRow (35) + : : : +- Scan parquet default.date_dim (34) + : : +- BroadcastExchange (42) + : : +- * Project (41) + : : +- * BroadcastHashJoin LeftSemi BuildRight (40) + : : :- * ColumnarToRow (38) + : : : +- Scan parquet default.date_dim (37) + : : +- ReusedExchange (39) + : +- ReusedExchange (48) + +- BroadcastExchange (69) + +- * HashAggregate (68) + +- Exchange (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Filter (59) + : : +- * ColumnarToRow (58) + : : +- Scan parquet default.web_returns (57) + : +- ReusedExchange (60) + +- ReusedExchange (63) (1) Scan parquet default.store_returns @@ -216,132 +227,181 @@ Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] Condition : isnotnull(cr_item_sk#21) -(34) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#4] +(34) Scan parquet default.date_dim +Output [2]: [d_date_sk#24, d_date#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct + +(35) ColumnarToRow [codegen id : 8] +Input [2]: [d_date_sk#24, d_date#25] + +(36) Filter [codegen id : 8] +Input [2]: [d_date_sk#24, d_date#25] +Condition : isnotnull(d_date_sk#24) + +(37) Scan parquet default.date_dim +Output [2]: [d_date#5, d_week_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +ReadSchema: struct + +(38) ColumnarToRow [codegen id : 7] +Input [2]: [d_date#5, d_week_seq#6] + +(39) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_week_seq#6#26] + +(40) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [d_week_seq#6] +Right keys [1]: [d_week_seq#6#26] +Join condition: None + +(41) Project [codegen id : 7] +Output [1]: [d_date#5] +Input [2]: [d_date#5, d_week_seq#6] -(35) BroadcastHashJoin [codegen id : 10] +(42) BroadcastExchange +Input [1]: [d_date#5] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#27] + +(43) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [d_date#25] +Right keys [1]: [d_date#5] +Join condition: None + +(44) Project [codegen id : 8] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_date#25] + +(45) BroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] + +(46) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cr_returned_date_sk#23] -Right keys [1]: [d_date_sk#4] +Right keys [1]: [d_date_sk#24] Join condition: None -(36) Project [codegen id : 10] +(47) Project [codegen id : 10] Output [2]: [cr_item_sk#21, cr_return_quantity#22] -Input [4]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, d_date_sk#4] +Input [4]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, d_date_sk#24] -(37) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#12, i_item_id#13] +(48) ReusedExchange [Reuses operator id: 25] +Output [2]: [i_item_sk#29, i_item_id#30] -(38) BroadcastHashJoin [codegen id : 10] +(49) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cr_item_sk#21] -Right keys [1]: [i_item_sk#12] +Right keys [1]: [i_item_sk#29] Join condition: None -(39) Project [codegen id : 10] -Output [2]: [cr_return_quantity#22, i_item_id#13] -Input [4]: [cr_item_sk#21, cr_return_quantity#22, i_item_sk#12, i_item_id#13] +(50) Project [codegen id : 10] +Output [2]: [cr_return_quantity#22, i_item_id#30] +Input [4]: [cr_item_sk#21, cr_return_quantity#22, i_item_sk#29, i_item_id#30] -(40) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#22, i_item_id#13] -Keys [1]: [i_item_id#13] +(51) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#22, i_item_id#30] +Keys [1]: [i_item_id#30] Functions [1]: [partial_sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum#24] -Results [2]: [i_item_id#13, sum#25] +Aggregate Attributes [1]: [sum#31] +Results [2]: [i_item_id#30, sum#32] -(41) Exchange -Input [2]: [i_item_id#13, sum#25] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#26] +(52) Exchange +Input [2]: [i_item_id#30, sum#32] +Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [id=#33] -(42) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#13, sum#25] -Keys [1]: [i_item_id#13] +(53) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#30, sum#32] +Keys [1]: [i_item_id#30] Functions [1]: [sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum(cr_return_quantity#22)#27] -Results [2]: [i_item_id#13 AS item_id#28, sum(cr_return_quantity#22)#27 AS cr_item_qty#29] +Aggregate Attributes [1]: [sum(cr_return_quantity#22)#34] +Results [2]: [i_item_id#30 AS item_id#35, sum(cr_return_quantity#22)#34 AS cr_item_qty#36] -(43) BroadcastExchange -Input [2]: [item_id#28, cr_item_qty#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] +(54) BroadcastExchange +Input [2]: [item_id#35, cr_item_qty#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#37] -(44) BroadcastHashJoin [codegen id : 18] +(55) BroadcastHashJoin [codegen id : 18] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#28] +Right keys [1]: [item_id#35] Join condition: None -(45) Project [codegen id : 18] -Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] -Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] +(56) Project [codegen id : 18] +Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#36] +Input [4]: [item_id#19, sr_item_qty#20, item_id#35, cr_item_qty#36] -(46) Scan parquet default.web_returns -Output [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +(57) Scan parquet default.web_returns +Output [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#33), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(wr_returned_date_sk#40), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +(58) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] -(48) Filter [codegen id : 16] -Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] -Condition : isnotnull(wr_item_sk#31) +(59) Filter [codegen id : 16] +Input [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] +Condition : isnotnull(wr_item_sk#38) -(49) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#4] +(60) ReusedExchange [Reuses operator id: 45] +Output [1]: [d_date_sk#41] -(50) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#33] -Right keys [1]: [cast(d_date_sk#4 as bigint)] +(61) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#40] +Right keys [1]: [cast(d_date_sk#41 as bigint)] Join condition: None -(51) Project [codegen id : 16] -Output [2]: [wr_item_sk#31, wr_return_quantity#32] -Input [4]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33, d_date_sk#4] +(62) Project [codegen id : 16] +Output [2]: [wr_item_sk#38, wr_return_quantity#39] +Input [4]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40, d_date_sk#41] -(52) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#12, i_item_id#13] +(63) ReusedExchange [Reuses operator id: 25] +Output [2]: [i_item_sk#42, i_item_id#43] -(53) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#31] -Right keys [1]: [cast(i_item_sk#12 as bigint)] +(64) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#38] +Right keys [1]: [cast(i_item_sk#42 as bigint)] Join condition: None -(54) Project [codegen id : 16] -Output [2]: [wr_return_quantity#32, i_item_id#13] -Input [4]: [wr_item_sk#31, wr_return_quantity#32, i_item_sk#12, i_item_id#13] - -(55) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#32, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(wr_return_quantity#32)] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#13, sum#35] - -(56) Exchange -Input [2]: [i_item_id#13, sum#35] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#36] - -(57) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#13, sum#35] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(wr_return_quantity#32)] -Aggregate Attributes [1]: [sum(wr_return_quantity#32)#37] -Results [2]: [i_item_id#13 AS item_id#38, sum(wr_return_quantity#32)#37 AS wr_item_qty#39] - -(58) BroadcastExchange -Input [2]: [item_id#38, wr_item_qty#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] - -(59) BroadcastHashJoin [codegen id : 18] +(65) Project [codegen id : 16] +Output [2]: [wr_return_quantity#39, i_item_id#43] +Input [4]: [wr_item_sk#38, wr_return_quantity#39, i_item_sk#42, i_item_id#43] + +(66) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#39, i_item_id#43] +Keys [1]: [i_item_id#43] +Functions [1]: [partial_sum(wr_return_quantity#39)] +Aggregate Attributes [1]: [sum#44] +Results [2]: [i_item_id#43, sum#45] + +(67) Exchange +Input [2]: [i_item_id#43, sum#45] +Arguments: hashpartitioning(i_item_id#43, 5), ENSURE_REQUIREMENTS, [id=#46] + +(68) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#43, sum#45] +Keys [1]: [i_item_id#43] +Functions [1]: [sum(wr_return_quantity#39)] +Aggregate Attributes [1]: [sum(wr_return_quantity#39)#47] +Results [2]: [i_item_id#43 AS item_id#48, sum(wr_return_quantity#39)#47 AS wr_item_qty#49] + +(69) BroadcastExchange +Input [2]: [item_id#48, wr_item_qty#49] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] + +(70) BroadcastHashJoin [codegen id : 18] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#38] +Right keys [1]: [item_id#48] Join condition: None -(60) Project [codegen id : 18] -Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] -Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] +(71) Project [codegen id : 18] +Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS sr_dev#51, cr_item_qty#36, (((cast(cr_item_qty#36 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS cr_dev#52, wr_item_qty#49, (((cast(wr_item_qty#49 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS wr_dev#53, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#54] +Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#36, item_id#48, wr_item_qty#49] -(61) TakeOrderedAndProject -Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] -Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +(72) TakeOrderedAndProject +Input [8]: [item_id#19, sr_item_qty#20, sr_dev#51, cr_item_qty#36, cr_dev#52, wr_item_qty#49, wr_dev#53, average#54] +Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#51, cr_item_qty#36, cr_dev#52, wr_item_qty#49, wr_dev#53, average#54] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt index b46131afd5ec7..819d571695a83 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt @@ -66,15 +66,32 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (8) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + Filter [d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (7) + Project [d_date] + BroadcastHashJoin [d_week_seq,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] + InputAdapter + ReusedExchange [d_week_seq] #4 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 InputAdapter - BroadcastExchange #8 + BroadcastExchange #10 WholeStageCodegen (17) HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] InputAdapter - Exchange [i_item_id] #9 + Exchange [i_item_id] #11 WholeStageCodegen (16) HashAggregate [i_item_id,wr_return_quantity] [sum,sum] Project [wr_return_quantity,i_item_id] @@ -86,6 +103,6 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #8 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index c926ab7e3ccb6..8b14a282c2671 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (61) -+- * Project (60) - +- * BroadcastHashJoin Inner BuildRight (59) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) +TakeOrderedAndProject (72) ++- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) : :- * HashAggregate (30) : : +- Exchange (29) : : +- * HashAggregate (28) @@ -34,32 +34,43 @@ TakeOrderedAndProject (61) : : +- * Filter (17) : : +- * ColumnarToRow (16) : : +- Scan parquet default.date_dim (15) - : +- BroadcastExchange (43) - : +- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) + : +- BroadcastExchange (54) + : +- * HashAggregate (53) + : +- Exchange (52) + : +- * HashAggregate (51) + : +- * Project (50) + : +- * BroadcastHashJoin Inner BuildRight (49) : :- * Project (36) : : +- * BroadcastHashJoin Inner BuildRight (35) : : :- * Filter (33) : : : +- * ColumnarToRow (32) : : : +- Scan parquet default.catalog_returns (31) : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- BroadcastExchange (58) - +- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- * Project (54) - +- * BroadcastHashJoin Inner BuildRight (53) - :- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet default.web_returns (46) - : +- ReusedExchange (49) - +- ReusedExchange (52) + : +- BroadcastExchange (48) + : +- * Project (47) + : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : :- * Filter (39) + : : +- * ColumnarToRow (38) + : : +- Scan parquet default.date_dim (37) + : +- BroadcastExchange (45) + : +- * Project (44) + : +- * BroadcastHashJoin LeftSemi BuildRight (43) + : :- * ColumnarToRow (41) + : : +- Scan parquet default.date_dim (40) + : +- ReusedExchange (42) + +- BroadcastExchange (69) + +- * HashAggregate (68) + +- Exchange (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Filter (59) + : : +- * ColumnarToRow (58) + : : +- Scan parquet default.web_returns (57) + : +- ReusedExchange (60) + +- ReusedExchange (63) (1) Scan parquet default.store_returns @@ -217,131 +228,180 @@ Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] Condition : isnotnull(cr_item_sk#21) (34) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#24, i_item_id#25] (35) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cr_item_sk#21] -Right keys [1]: [i_item_sk#4] +Right keys [1]: [i_item_sk#24] Join condition: None (36) Project [codegen id : 10] -Output [3]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#5] -Input [5]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, i_item_sk#4, i_item_id#5] +Output [3]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#25] +Input [5]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, i_item_sk#24, i_item_id#25] -(37) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#7] +(37) Scan parquet default.date_dim +Output [2]: [d_date_sk#26, d_date#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct + +(38) ColumnarToRow [codegen id : 9] +Input [2]: [d_date_sk#26, d_date#27] + +(39) Filter [codegen id : 9] +Input [2]: [d_date_sk#26, d_date#27] +Condition : isnotnull(d_date_sk#26) + +(40) Scan parquet default.date_dim +Output [2]: [d_date#8, d_week_seq#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +ReadSchema: struct + +(41) ColumnarToRow [codegen id : 8] +Input [2]: [d_date#8, d_week_seq#9] -(38) BroadcastHashJoin [codegen id : 10] +(42) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_week_seq#9#28] + +(43) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [d_week_seq#9] +Right keys [1]: [d_week_seq#9#28] +Join condition: None + +(44) Project [codegen id : 8] +Output [1]: [d_date#8] +Input [2]: [d_date#8, d_week_seq#9] + +(45) BroadcastExchange +Input [1]: [d_date#8] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#29] + +(46) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [d_date#27] +Right keys [1]: [d_date#8] +Join condition: None + +(47) Project [codegen id : 9] +Output [1]: [d_date_sk#26] +Input [2]: [d_date_sk#26, d_date#27] + +(48) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] + +(49) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cr_returned_date_sk#23] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#26] Join condition: None -(39) Project [codegen id : 10] -Output [2]: [cr_return_quantity#22, i_item_id#5] -Input [4]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#5, d_date_sk#7] +(50) Project [codegen id : 10] +Output [2]: [cr_return_quantity#22, i_item_id#25] +Input [4]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#25, d_date_sk#26] -(40) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#22, i_item_id#5] -Keys [1]: [i_item_id#5] +(51) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#22, i_item_id#25] +Keys [1]: [i_item_id#25] Functions [1]: [partial_sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum#24] -Results [2]: [i_item_id#5, sum#25] +Aggregate Attributes [1]: [sum#31] +Results [2]: [i_item_id#25, sum#32] -(41) Exchange -Input [2]: [i_item_id#5, sum#25] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#26] +(52) Exchange +Input [2]: [i_item_id#25, sum#32] +Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [id=#33] -(42) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#5, sum#25] -Keys [1]: [i_item_id#5] +(53) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#25, sum#32] +Keys [1]: [i_item_id#25] Functions [1]: [sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum(cr_return_quantity#22)#27] -Results [2]: [i_item_id#5 AS item_id#28, sum(cr_return_quantity#22)#27 AS cr_item_qty#29] +Aggregate Attributes [1]: [sum(cr_return_quantity#22)#34] +Results [2]: [i_item_id#25 AS item_id#35, sum(cr_return_quantity#22)#34 AS cr_item_qty#36] -(43) BroadcastExchange -Input [2]: [item_id#28, cr_item_qty#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] +(54) BroadcastExchange +Input [2]: [item_id#35, cr_item_qty#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#37] -(44) BroadcastHashJoin [codegen id : 18] +(55) BroadcastHashJoin [codegen id : 18] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#28] +Right keys [1]: [item_id#35] Join condition: None -(45) Project [codegen id : 18] -Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] -Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] +(56) Project [codegen id : 18] +Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#36] +Input [4]: [item_id#19, sr_item_qty#20, item_id#35, cr_item_qty#36] -(46) Scan parquet default.web_returns -Output [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +(57) Scan parquet default.web_returns +Output [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#33), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(wr_returned_date_sk#40), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +(58) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] -(48) Filter [codegen id : 16] -Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] -Condition : isnotnull(wr_item_sk#31) +(59) Filter [codegen id : 16] +Input [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] +Condition : isnotnull(wr_item_sk#38) -(49) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] +(60) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#41, i_item_id#42] -(50) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#31] -Right keys [1]: [cast(i_item_sk#4 as bigint)] +(61) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#38] +Right keys [1]: [cast(i_item_sk#41 as bigint)] Join condition: None -(51) Project [codegen id : 16] -Output [3]: [wr_return_quantity#32, wr_returned_date_sk#33, i_item_id#5] -Input [5]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33, i_item_sk#4, i_item_id#5] +(62) Project [codegen id : 16] +Output [3]: [wr_return_quantity#39, wr_returned_date_sk#40, i_item_id#42] +Input [5]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40, i_item_sk#41, i_item_id#42] -(52) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#7] +(63) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#43] -(53) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#33] -Right keys [1]: [cast(d_date_sk#7 as bigint)] +(64) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#40] +Right keys [1]: [cast(d_date_sk#43 as bigint)] Join condition: None -(54) Project [codegen id : 16] -Output [2]: [wr_return_quantity#32, i_item_id#5] -Input [4]: [wr_return_quantity#32, wr_returned_date_sk#33, i_item_id#5, d_date_sk#7] - -(55) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#32, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(wr_return_quantity#32)] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#5, sum#35] - -(56) Exchange -Input [2]: [i_item_id#5, sum#35] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#36] - -(57) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#5, sum#35] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(wr_return_quantity#32)] -Aggregate Attributes [1]: [sum(wr_return_quantity#32)#37] -Results [2]: [i_item_id#5 AS item_id#38, sum(wr_return_quantity#32)#37 AS wr_item_qty#39] - -(58) BroadcastExchange -Input [2]: [item_id#38, wr_item_qty#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] - -(59) BroadcastHashJoin [codegen id : 18] +(65) Project [codegen id : 16] +Output [2]: [wr_return_quantity#39, i_item_id#42] +Input [4]: [wr_return_quantity#39, wr_returned_date_sk#40, i_item_id#42, d_date_sk#43] + +(66) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#39, i_item_id#42] +Keys [1]: [i_item_id#42] +Functions [1]: [partial_sum(wr_return_quantity#39)] +Aggregate Attributes [1]: [sum#44] +Results [2]: [i_item_id#42, sum#45] + +(67) Exchange +Input [2]: [i_item_id#42, sum#45] +Arguments: hashpartitioning(i_item_id#42, 5), ENSURE_REQUIREMENTS, [id=#46] + +(68) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#42, sum#45] +Keys [1]: [i_item_id#42] +Functions [1]: [sum(wr_return_quantity#39)] +Aggregate Attributes [1]: [sum(wr_return_quantity#39)#47] +Results [2]: [i_item_id#42 AS item_id#48, sum(wr_return_quantity#39)#47 AS wr_item_qty#49] + +(69) BroadcastExchange +Input [2]: [item_id#48, wr_item_qty#49] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] + +(70) BroadcastHashJoin [codegen id : 18] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#38] +Right keys [1]: [item_id#48] Join condition: None -(60) Project [codegen id : 18] -Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] -Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] +(71) Project [codegen id : 18] +Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS sr_dev#51, cr_item_qty#36, (((cast(cr_item_qty#36 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS cr_dev#52, wr_item_qty#49, (((cast(wr_item_qty#49 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS wr_dev#53, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#54] +Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#36, item_id#48, wr_item_qty#49] -(61) TakeOrderedAndProject -Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] -Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +(72) TakeOrderedAndProject +Input [8]: [item_id#19, sr_item_qty#20, sr_dev#51, cr_item_qty#36, cr_dev#52, wr_item_qty#49, wr_dev#53, average#54] +Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#51, cr_item_qty#36, cr_dev#52, wr_item_qty#49, wr_dev#53, average#54] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index 7fc930f2885fd..9632348854668 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -68,13 +68,30 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #8 + WholeStageCodegen (9) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + Filter [d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (8) + Project [d_date] + BroadcastHashJoin [d_week_seq,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] + InputAdapter + ReusedExchange [d_week_seq] #5 InputAdapter - BroadcastExchange #8 + BroadcastExchange #10 WholeStageCodegen (17) HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] InputAdapter - Exchange [i_item_id] #9 + Exchange [i_item_id] #11 WholeStageCodegen (16) HashAggregate [i_item_id,wr_return_quantity] [sum,sum] Project [wr_return_quantity,i_item_id] @@ -88,4 +105,4 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt index 312aba94bd9c6..b19d4f3aeae0f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt @@ -33,39 +33,39 @@ TakeOrderedAndProject (116) : : : : +- * HashAggregate (44) : : : : +- Exchange (43) : : : : +- * HashAggregate (42) - : : : : +- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) + : : : : +- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Project (18) + : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : :- * Filter (11) + : : : : : : +- * ColumnarToRow (10) + : : : : : : +- Scan parquet default.store_sales (9) + : : : : : +- BroadcastExchange (16) + : : : : : +- * Project (15) + : : : : : +- * Filter (14) + : : : : : +- * ColumnarToRow (13) + : : : : : +- Scan parquet default.date_dim (12) + : : : : +- BroadcastExchange (39) + : : : : +- SortMergeJoin LeftSemi (38) + : : : : :- * Sort (23) + : : : : : +- Exchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (37) + : : : : +- Exchange (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- Scan parquet default.catalog_sales (24) + : : : : : +- ReusedExchange (27) + : : : : +- BroadcastExchange (33) + : : : : +- * Filter (32) + : : : : +- * ColumnarToRow (31) + : : : : +- Scan parquet default.item (30) : : : +- * Sort (57) : : : +- Exchange (56) : : : +- * Project (55) @@ -147,10 +147,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 20] +(7) ColumnarToRow [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 20] +(8) Filter [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) @@ -162,10 +162,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 5] +(10) ColumnarToRow [codegen id : 10] Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -(11) Filter [codegen id : 5] +(11) Filter [codegen id : 10] Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -191,51 +191,38 @@ Input [2]: [d_date_sk#12, d_year#13] Input [1]: [d_date_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(17) BroadcastHashJoin [codegen id : 5] +(17) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#12] Join condition: None -(18) Project [codegen id : 5] +(18) Project [codegen id : 10] Output [1]: [ss_item_sk#1] Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] (19) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Condition : (((isnotnull(i_item_sk#15) AND isnotnull(i_brand_id#16)) AND isnotnull(i_class_id#17)) AND isnotnull(i_category_id#18)) -(22) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] +(22) Exchange +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: hashpartitioning(coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join condition: None - -(24) Project [codegen id : 5] -Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] -Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] - -(25) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] +(23) Sort [codegen id : 5] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: [coalesce(i_brand_id#16, 0) ASC NULLS FIRST, isnull(i_brand_id#16) ASC NULLS FIRST, coalesce(i_class_id#17, 0) ASC NULLS FIRST, isnull(i_class_id#17) ASC NULLS FIRST, coalesce(i_category_id#18, 0) ASC NULLS FIRST, isnull(i_category_id#18) ASC NULLS FIRST], false, 0 -(26) Sort [codegen id : 6] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 - -(27) Scan parquet default.catalog_sales +(24) Scan parquet default.catalog_sales Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] @@ -243,215 +230,228 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] +(25) ColumnarToRow [codegen id : 8] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -(29) Filter [codegen id : 9] +(26) Filter [codegen id : 8] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] Condition : isnotnull(cs_item_sk#20) -(30) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +(27) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#22] -(31) BroadcastHashJoin [codegen id : 9] +(28) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#22] Join condition: None -(32) Project [codegen id : 9] +(29) Project [codegen id : 8] Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] +Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#22] -(33) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(30) Scan parquet default.item +Output [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(31) ColumnarToRow [codegen id : 7] +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -(35) Filter [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +(32) Filter [codegen id : 7] +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Condition : isnotnull(i_item_sk#23) -(36) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +(33) BroadcastExchange +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] -(37) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#23] Join condition: None -(38) Project [codegen id : 9] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(35) Project [codegen id : 8] +Output [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Input [5]: [cs_item_sk#20, i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -(39) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] +(36) Exchange +Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: hashpartitioning(coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26), 5), ENSURE_REQUIREMENTS, [id=#28] -(40) Sort [codegen id : 10] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(37) Sort [codegen id : 9] +Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: [coalesce(i_brand_id#24, 0) ASC NULLS FIRST, isnull(i_brand_id#24) ASC NULLS FIRST, coalesce(i_class_id#25, 0) ASC NULLS FIRST, isnull(i_class_id#25) ASC NULLS FIRST, coalesce(i_category_id#26, 0) ASC NULLS FIRST, isnull(i_category_id#26) ASC NULLS FIRST], false, 0 -(41) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +(38) SortMergeJoin +Left keys [6]: [coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18)] +Right keys [6]: [coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26)] Join condition: None -(42) HashAggregate [codegen id : 11] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(39) BroadcastExchange +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] + +(40) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#15] +Join condition: None + +(41) Project [codegen id : 10] +Output [3]: [i_brand_id#16 AS brand_id#30, i_class_id#17 AS class_id#31, i_category_id#18 AS category_id#32] +Input [5]: [ss_item_sk#1, i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] + +(42) HashAggregate [codegen id : 10] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#30, class_id#31, category_id#32] (43) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] -(44) HashAggregate [codegen id : 12] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(44) HashAggregate [codegen id : 11] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#30, class_id#31, category_id#32] (45) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32), 5), ENSURE_REQUIREMENTS, [id=#34] -(46) Sort [codegen id : 13] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 12] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: [coalesce(brand_id#30, 0) ASC NULLS FIRST, isnull(brand_id#30) ASC NULLS FIRST, coalesce(class_id#31, 0) ASC NULLS FIRST, isnull(class_id#31) ASC NULLS FIRST, coalesce(category_id#32, 0) ASC NULLS FIRST, isnull(category_id#32) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Output [2]: [ws_item_sk#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +(48) ColumnarToRow [codegen id : 15] +Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] -(49) Filter [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(49) Filter [codegen id : 15] +Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#35) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#37] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(51) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#37] Join condition: None -(52) Project [codegen id : 16] -Output [1]: [ws_item_sk#26] -Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] +(52) Project [codegen id : 15] +Output [1]: [ws_item_sk#35] +Input [3]: [ws_item_sk#35, ws_sold_date_sk#36, d_date_sk#37] -(53) ReusedExchange [Reuses operator id: 36] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(53) ReusedExchange [Reuses operator id: 33] +Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(54) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [i_item_sk#38] Join condition: None -(55) Project [codegen id : 16] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(55) Project [codegen id : 15] +Output [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Input [5]: [ws_item_sk#35, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] (56) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: hashpartitioning(coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41), 5), ENSURE_REQUIREMENTS, [id=#42] -(57) Sort [codegen id : 17] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 16] +Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: [coalesce(i_brand_id#39, 0) ASC NULLS FIRST, isnull(i_brand_id#39) ASC NULLS FIRST, coalesce(i_class_id#40, 0) ASC NULLS FIRST, isnull(i_class_id#40) ASC NULLS FIRST, coalesce(i_category_id#41, 0) ASC NULLS FIRST, isnull(i_category_id#41) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41)] Join condition: None -(59) HashAggregate [codegen id : 18] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(59) HashAggregate [codegen id : 17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#30, class_id#31, category_id#32] (60) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#43] -(61) HashAggregate [codegen id : 19] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(61) HashAggregate [codegen id : 18] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#30, class_id#31, category_id#32] (62) BroadcastExchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#44] -(63) BroadcastHashJoin [codegen id : 20] +(63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#16, class_id#17, category_id#18] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None -(64) Project [codegen id : 20] -Output [1]: [i_item_sk#7 AS ss_item_sk#31] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] +(64) Project [codegen id : 19] +Output [1]: [i_item_sk#7 AS ss_item_sk#45] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#30, class_id#31, category_id#32] (65) Exchange -Input [1]: [ss_item_sk#31] -Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [1]: [ss_item_sk#45] +Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] -(66) Sort [codegen id : 21] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 20] +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#45] Join condition: None (68) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#33] +Output [2]: [d_date_sk#12, d_week_seq#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 22] -Input [2]: [d_date_sk#12, d_week_seq#33] +(69) ColumnarToRow [codegen id : 21] +Input [2]: [d_date_sk#12, d_week_seq#47] -(70) Filter [codegen id : 22] -Input [2]: [d_date_sk#12, d_week_seq#33] -Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#34, [id=#35])) AND isnotnull(d_date_sk#12)) +(70) Filter [codegen id : 21] +Input [2]: [d_date_sk#12, d_week_seq#47] +Condition : ((isnotnull(d_week_seq#47) AND (d_week_seq#47 = Subquery scalar-subquery#48, [id=#49])) AND isnotnull(d_date_sk#12)) -(71) Project [codegen id : 22] +(71) Project [codegen id : 21] Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#33] +Input [2]: [d_date_sk#12, d_week_seq#47] (72) BroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] -(73) BroadcastHashJoin [codegen id : 44] +(73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#12] Join condition: None -(74) Project [codegen id : 44] +(74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] @@ -462,192 +462,192 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 23] +(76) ColumnarToRow [codegen id : 22] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(77) Filter [codegen id : 23] +(77) Filter [codegen id : 22] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (78) Exchange Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#37] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#51] -(79) Sort [codegen id : 24] +(79) Sort [codegen id : 23] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#45] -(81) Sort [codegen id : 43] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 41] +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (82) SortMergeJoin Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#45] Join condition: None (83) BroadcastExchange Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] -(84) BroadcastHashJoin [codegen id : 44] +(84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#7] Join condition: None -(85) Project [codegen id : 44] +(85) Project [codegen id : 42] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(86) HashAggregate [codegen id : 44] +(86) HashAggregate [codegen id : 42] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#39, isEmpty#40, count#41] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] +Aggregate Attributes [3]: [sum#53, isEmpty#54, count#55] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#59] -(88) HashAggregate [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] +(88) HashAggregate [codegen id : 86] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46, count(1)#47] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sales#48, count(1)#47 AS number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60, count(1)#61] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sales#62, count(1)#61 AS number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] -(89) Filter [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) +(89) Filter [codegen id : 86] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 as decimal(32,6)) > cast(Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) -(90) Project [codegen id : 90] -Output [6]: [store AS channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] +(90) Project [codegen id : 86] +Output [6]: [store AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] (91) Scan parquet default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#54)] +PartitionFilters: [isnotnull(ss_sold_date_sk#71), dynamicpruningexpression(ss_sold_date_sk#71 IN dynamicpruning#72)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 45] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +(92) ColumnarToRow [codegen id : 43] +Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] -(93) Filter [codegen id : 45] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +(93) Filter [codegen id : 43] +Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Condition : isnotnull(ss_item_sk#68) (94) Exchange -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Arguments: hashpartitioning(ss_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#73] -(95) Sort [codegen id : 46] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 44] +Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Arguments: [ss_item_sk#68 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#45] -(97) Sort [codegen id : 65] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 62] +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [ss_item_sk#68] +Right keys [1]: [ss_item_sk#45] Join condition: None (99) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#33] +Output [2]: [d_date_sk#74, d_week_seq#75] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 66] -Input [2]: [d_date_sk#12, d_week_seq#33] +(100) ColumnarToRow [codegen id : 63] +Input [2]: [d_date_sk#74, d_week_seq#75] -(101) Filter [codegen id : 66] -Input [2]: [d_date_sk#12, d_week_seq#33] -Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#56, [id=#57])) AND isnotnull(d_date_sk#12)) +(101) Filter [codegen id : 63] +Input [2]: [d_date_sk#74, d_week_seq#75] +Condition : ((isnotnull(d_week_seq#75) AND (d_week_seq#75 = Subquery scalar-subquery#76, [id=#77])) AND isnotnull(d_date_sk#74)) -(102) Project [codegen id : 66] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#33] +(102) Project [codegen id : 63] +Output [1]: [d_date_sk#74] +Input [2]: [d_date_sk#74, d_week_seq#75] (103) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] +Input [1]: [d_date_sk#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] -(104) BroadcastHashJoin [codegen id : 88] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(104) BroadcastHashJoin [codegen id : 84] +Left keys [1]: [ss_sold_date_sk#71] +Right keys [1]: [d_date_sk#74] Join condition: None -(105) Project [codegen id : 88] -Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +(105) Project [codegen id : 84] +Output [3]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70] +Input [5]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71, d_date_sk#74] (106) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +Output [4]: [i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] -(107) BroadcastHashJoin [codegen id : 88] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#59] +(107) BroadcastHashJoin [codegen id : 84] +Left keys [1]: [ss_item_sk#68] +Right keys [1]: [i_item_sk#79] Join condition: None -(108) Project [codegen id : 88] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +(108) Project [codegen id : 84] +Output [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#80, i_class_id#81, i_category_id#82] +Input [7]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] -(109) HashAggregate [codegen id : 88] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] +(109) HashAggregate [codegen id : 84] +Input [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#80, i_class_id#81, i_category_id#82] +Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] +Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] (110) Exchange -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] -Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] +Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [id=#89] -(111) HashAggregate [codegen id : 89] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70, count(1)#71] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sales#72, count(1)#71 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] +(111) HashAggregate [codegen id : 85] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] +Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90, count(1)#91] +Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sales#92, count(1)#91 AS number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] -(112) Filter [codegen id : 89] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) +(112) Filter [codegen id : 85] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) -(113) Project [codegen id : 89] -Output [6]: [store AS channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] +(113) Project [codegen id : 85] +Output [6]: [store AS channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] (114) BroadcastExchange -Input [6]: [channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#76] +Input [6]: [channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#96] -(115) BroadcastHashJoin [codegen id : 90] +(115) BroadcastHashJoin [codegen id : 86] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Right keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] Join condition: None (116) TakeOrderedAndProject -Input [12]: [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Input [12]: [channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#65, [id=#66] * HashAggregate (139) +- Exchange (138) +- * HashAggregate (137) @@ -677,7 +677,7 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#97)] ReadSchema: struct (118) ColumnarToRow [codegen id : 2] @@ -703,7 +703,7 @@ Input [2]: [d_date_sk#12, d_year#13] (123) BroadcastExchange Input [1]: [d_date_sk#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=#98] (124) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#4] @@ -711,83 +711,83 @@ Right keys [1]: [d_date_sk#12] Join condition: None (125) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#79, ss_list_price#3 AS list_price#80] +Output [2]: [ss_quantity#2 AS quantity#99, ss_list_price#3 AS list_price#100] Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (126) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] +Output [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#97)] ReadSchema: struct (127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] +Input [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21] (128) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#103] (129) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#103] Join condition: None (130) Project [codegen id : 4] -Output [2]: [cs_quantity#81 AS quantity#83, cs_list_price#82 AS list_price#84] -Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21, d_date_sk#12] +Output [2]: [cs_quantity#101 AS quantity#104, cs_list_price#102 AS list_price#105] +Input [4]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21, d_date_sk#103] (131) Scan parquet default.web_sales -Output [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] +Output [3]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#97)] ReadSchema: struct (132) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] +Input [3]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36] (133) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#108] (134) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#108] Join condition: None (135) Project [codegen id : 6] -Output [2]: [ws_quantity#85 AS quantity#87, ws_list_price#86 AS list_price#88] -Input [4]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27, d_date_sk#12] +Output [2]: [ws_quantity#106 AS quantity#109, ws_list_price#107 AS list_price#110] +Input [4]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36, d_date_sk#108] (136) Union (137) HashAggregate [codegen id : 7] -Input [2]: [quantity#79, list_price#80] +Input [2]: [quantity#99, list_price#100] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#89, count#90] -Results [2]: [sum#91, count#92] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#111, count#112] +Results [2]: [sum#113, count#114] (138) Exchange -Input [2]: [sum#91, count#92] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] +Input [2]: [sum#113, count#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] (139) HashAggregate [codegen id : 8] -Input [2]: [sum#91, count#92] +Input [2]: [sum#113, count#114] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94 AS average_sales#95] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#116] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#116 AS average_sales#117] -Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#77 +Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#97 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 123] Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#77 +Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#97 -Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#77 +Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#97 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (141) @@ -803,11 +803,11 @@ ReusedExchange (142) (142) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#11 -Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#34, [id=#35] +Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#48, [id=#49] * Project (146) +- * Filter (145) +- * ColumnarToRow (144) @@ -815,33 +815,33 @@ Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquer (143) Scan parquet default.date_dim -Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] (145) Filter [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 1999)) AND (d_moy#96 = 12)) AND (d_dom#97 = 16)) +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#13 = 1999)) AND (d_moy#118 = 12)) AND (d_dom#119 = 16)) (146) Project [codegen id : 1] -Output [1]: [d_week_seq#33] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [1]: [d_week_seq#47] +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] -Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] +Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#65, [id=#66] -Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#54 +Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#71 IN dynamicpruning#72 ReusedExchange (147) (147) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#74] -Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#56, [id=#57] +Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#76, [id=#77] * Project (151) +- * Filter (150) +- * ColumnarToRow (149) @@ -849,21 +849,21 @@ Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subqu (148) Scan parquet default.date_dim -Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (149) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] (150) Filter [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 1998)) AND (d_moy#96 = 12)) AND (d_dom#97 = 16)) +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#13 = 1998)) AND (d_moy#118 = 12)) AND (d_dom#119 = 16)) (151) Project [codegen id : 1] -Output [1]: [d_week_seq#33] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [1]: [d_week_seq#47] +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt index 1d5907e319658..1b41abbc97b06 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (90) + WholeStageCodegen (86) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] @@ -49,7 +49,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (44) + WholeStageCodegen (42) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -68,11 +68,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #3 - WholeStageCodegen (21) + WholeStageCodegen (20) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 - WholeStageCodegen (20) + WholeStageCodegen (19) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -81,84 +81,82 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (19) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (18) + WholeStageCodegen (17) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (13) + WholeStageCodegen (12) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (11) + WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #9 InputAdapter - Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] + BroadcastExchange #9 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #10 + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #9 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (7) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (17) + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (16) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (16) + WholeStageCodegen (15) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -169,12 +167,12 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #10 + ReusedExchange [d_date_sk] #9 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter BroadcastExchange #3 - WholeStageCodegen (22) + WholeStageCodegen (21) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #3 @@ -190,29 +188,29 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #15 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #16 - WholeStageCodegen (23) + WholeStageCodegen (22) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #20 - WholeStageCodegen (89) + WholeStageCodegen (85) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (88) + WholeStageCodegen (84) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -220,24 +218,24 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] - WholeStageCodegen (46) + WholeStageCodegen (44) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #22 - WholeStageCodegen (45) + WholeStageCodegen (43) Filter [ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #6 ReusedExchange [d_date_sk] #23 - WholeStageCodegen (65) + WholeStageCodegen (62) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #23 - WholeStageCodegen (66) + WholeStageCodegen (63) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 258224e574cfb..4c62de561c263 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -27,36 +27,36 @@ TakeOrderedAndProject (100) : : : :- * HashAggregate (39) : : : : +- Exchange (38) : : : : +- * HashAggregate (37) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) + : : : : +- * Project (36) + : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : :- * Project (33) + : : : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet default.store_sales (7) + : : : : : +- BroadcastExchange (31) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Project (28) + : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet default.item (16) + : : : : : +- BroadcastExchange (26) + : : : : : +- * Project (25) + : : : : : +- * Filter (24) + : : : : : +- * ColumnarToRow (23) + : : : : : +- Scan parquet default.date_dim (22) + : : : : +- ReusedExchange (34) : : : +- BroadcastExchange (49) : : : +- * Project (48) : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -146,226 +146,226 @@ Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) (10) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(11) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -(12) Filter [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +(12) Filter [codegen id : 4] +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14)) -(13) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +(13) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join condition: None +(14) ColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] -(15) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(15) Filter [codegen id : 3] +Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Condition : isnotnull(cs_item_sk#15) -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +(16) Scan parquet default.item +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] 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 - -(17) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(18) Filter [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +(17) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -(19) Project [codegen id : 2] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +(18) Filter [codegen id : 1] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : isnotnull(i_item_sk#17) -(20) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +(19) BroadcastExchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [i_item_sk#17] Join condition: None -(22) Project [codegen id : 6] -Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] -Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -(23) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(22) Scan parquet default.date_dim +Output [2]: [d_date_sk#22, d_year#23] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +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 -(24) ColumnarToRow [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(23) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#22, d_year#23] -(25) Filter [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +(24) Filter [codegen id : 2] +Input [2]: [d_date_sk#22, d_year#23] +Condition : (((isnotnull(d_year#23) AND (d_year#23 >= 1998)) AND (d_year#23 <= 2000)) AND isnotnull(d_date_sk#22)) -(26) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +(25) Project [codegen id : 2] +Output [1]: [d_date_sk#22] +Input [2]: [d_date_sk#22, d_year#23] -(27) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(26) BroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(28) Filter [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +(27) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#22] +Join condition: None + +(28) Project [codegen id : 3] +Output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] +Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#22] (29) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] -(30) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +(30) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)] +Right keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] Join condition: None -(31) Project [codegen id : 5] -Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(31) BroadcastExchange +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] -(32) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] - -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +(32) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#11] Join condition: None -(34) Project [codegen id : 5] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(33) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14] +Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -(35) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] +(34) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#27] -(36) BroadcastHashJoin [codegen id : 6] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#27] Join condition: None +(36) Project [codegen id : 6] +Output [3]: [i_brand_id#12 AS brand_id#28, i_class_id#13 AS class_id#29, i_category_id#14 AS category_id#30] +Input [5]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#27] + (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#28, class_id#29, category_id#30] (38) Exchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [id=#31] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#28, class_id#29, category_id#30] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Output [2]: [ws_item_sk#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Condition : isnotnull(ws_item_sk#32) -(43) ReusedExchange [Reuses operator id: 29] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(43) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#32] +Right keys [1]: [i_item_sk#34] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37] +Input [6]: [ws_item_sk#32, ws_sold_date_sk#33, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -(46) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(46) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#38] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#38] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +Input [5]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] (49) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#39] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] +Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#28, class_id#29, category_id#30] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#28, class_id#29, category_id#30] (53) BroadcastExchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#40] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#15, class_id#16, category_id#17] +Right keys [3]: [brand_id#28, class_id#29, category_id#30] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#27] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] +Output [1]: [i_item_sk#6 AS ss_item_sk#41] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] (56) BroadcastExchange -Input [1]: [ss_item_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [ss_item_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#41] Join condition: None (58) Scan parquet default.item @@ -383,16 +383,16 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#41] (62) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#41] Join condition: None (63) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] @@ -404,170 +404,170 @@ Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_ Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (66) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#30] +Output [2]: [d_date_sk#27, d_week_seq#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [2]: [d_date_sk#12, d_week_seq#30] +Input [2]: [d_date_sk#27, d_week_seq#44] (68) Filter [codegen id : 24] -Input [2]: [d_date_sk#12, d_week_seq#30] -Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#31, [id=#32])) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#27, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#27)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#30] +Output [1]: [d_date_sk#27] +Input [2]: [d_date_sk#27, d_week_seq#44] (70) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#27] Join condition: None (72) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#27] (73) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#34, isEmpty#35, count#36] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] +Aggregate Attributes [3]: [sum#48, isEmpty#49, count#50] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#54] (75) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41, count(1)#42] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sales#43, count(1)#42 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55, count(1)#56] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sales#57, count(1)#56 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] (76) Filter [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45 as decimal(32,6)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59 as decimal(32,6)) > cast(Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) (77) Project [codegen id : 52] -Output [6]: [store AS channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] +Output [6]: [store AS channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] (78) Scan parquet default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#49)] +PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] (80) Filter [codegen id : 50] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] +Condition : isnotnull(ss_item_sk#63) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#41] (82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [ss_item_sk#63] +Right keys [1]: [ss_item_sk#41] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] +Output [4]: [i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] (84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#50] +Left keys [1]: [ss_item_sk#63] +Right keys [1]: [i_item_sk#68] Join condition: None (85) Project [codegen id : 50] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] +Output [6]: [ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_brand_id#69, i_class_id#70, i_category_id#71] +Input [8]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#30] +Output [2]: [d_date_sk#72, d_week_seq#73] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#12, d_week_seq#30] +Input [2]: [d_date_sk#72, d_week_seq#73] (88) Filter [codegen id : 49] -Input [2]: [d_date_sk#12, d_week_seq#30] -Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#54, [id=#55])) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#72, d_week_seq#73] +Condition : ((isnotnull(d_week_seq#73) AND (d_week_seq#73 = Subquery scalar-subquery#74, [id=#75])) AND isnotnull(d_date_sk#72)) (89) Project [codegen id : 49] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#30] +Output [1]: [d_date_sk#72] +Input [2]: [d_date_sk#72, d_week_seq#73] (90) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +Input [1]: [d_date_sk#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] (91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#66] +Right keys [1]: [d_date_sk#72] Join condition: None (92) Project [codegen id : 50] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53, d_date_sk#12] +Output [5]: [ss_quantity#64, ss_list_price#65, i_brand_id#69, i_class_id#70, i_category_id#71] +Input [7]: [ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_brand_id#69, i_class_id#70, i_category_id#71, d_date_sk#72] (93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] -Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#57, isEmpty#58, count#59] -Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] +Input [5]: [ss_quantity#64, ss_list_price#65, i_brand_id#69, i_class_id#70, i_category_id#71] +Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#77, isEmpty#78, count#79] +Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] (94) Exchange -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] -Arguments: hashpartitioning(i_brand_id#51, i_class_id#52, i_category_id#53, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] +Arguments: hashpartitioning(i_brand_id#69, i_class_id#70, i_category_id#71, 5), ENSURE_REQUIREMENTS, [id=#83] (95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] -Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] -Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] +Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84, count(1)#85] +Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sales#86, count(1)#85 AS number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] (96) Filter [codegen id : 51] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) (97) Project [codegen id : 51] -Output [6]: [store AS channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] +Output [6]: [store AS channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] (98) BroadcastExchange -Input [6]: [channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] +Input [6]: [channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#90] (99) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] +Right keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] Join condition: None (100) TakeOrderedAndProject -Input [12]: [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Input [12]: [channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] +Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#46, [id=#47] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#60, [id=#61] * HashAggregate (123) +- Exchange (122) +- * HashAggregate (121) @@ -597,137 +597,137 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#91)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (103) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#27, d_year#92] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#27, d_year#92] (105) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#27, d_year#92] +Condition : (((isnotnull(d_year#92) AND (d_year#92 >= 1998)) AND (d_year#92 <= 2000)) AND isnotnull(d_date_sk#27)) (106) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#27] +Input [2]: [d_date_sk#27, d_year#92] (107) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] (108) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#27] Join condition: None (109) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#73, ss_list_price#3 AS list_price#74] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#2 AS quantity#94, ss_list_price#3 AS list_price#95] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#27] (110) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] +Output [3]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#91)] ReadSchema: struct (111) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] +Input [3]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16] (112) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#98] (113) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#98] Join condition: None (114) Project [codegen id : 4] -Output [2]: [cs_quantity#75 AS quantity#77, cs_list_price#76 AS list_price#78] -Input [4]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19, d_date_sk#12] +Output [2]: [cs_quantity#96 AS quantity#99, cs_list_price#97 AS list_price#100] +Input [4]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16, d_date_sk#98] (115) Scan parquet default.web_sales -Output [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] +Output [3]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#91)] ReadSchema: struct (116) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] +Input [3]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33] (117) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#103] (118) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#103] Join condition: None (119) Project [codegen id : 6] -Output [2]: [ws_quantity#79 AS quantity#81, ws_list_price#80 AS list_price#82] -Input [4]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24, d_date_sk#12] +Output [2]: [ws_quantity#101 AS quantity#104, ws_list_price#102 AS list_price#105] +Input [4]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33, d_date_sk#103] (120) Union (121) HashAggregate [codegen id : 7] -Input [2]: [quantity#73, list_price#74] +Input [2]: [quantity#94, list_price#95] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#83, count#84] -Results [2]: [sum#85, count#86] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#106, count#107] +Results [2]: [sum#108, count#109] (122) Exchange -Input [2]: [sum#85, count#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#87] +Input [2]: [sum#108, count#109] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] (123) HashAggregate [codegen id : 8] -Input [2]: [sum#85, count#86] +Input [2]: [sum#108, count#109] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88 AS average_sales#89] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))#111] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))#111 AS average_sales#112] -Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#71 +Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#91 ReusedExchange (124) (124) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#27] -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#71 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#91 -Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#71 +Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#91 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#27] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 ReusedExchange (126) -(126) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(126) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#27] -Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#10 -Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#31, [id=#32] +Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#45, [id=#46] * Project (130) +- * Filter (129) +- * ColumnarToRow (128) @@ -735,33 +735,33 @@ Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer (127) Scan parquet default.date_dim -Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (128) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] (129) Filter [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 1999)) AND (d_moy#90 = 12)) AND (d_dom#91 = 16)) +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#92 = 1999)) AND (d_moy#113 = 12)) AND (d_dom#114 = 16)) (130) Project [codegen id : 1] -Output [1]: [d_week_seq#30] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [1]: [d_week_seq#44] +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] -Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#60, [id=#61] -Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#49 +Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67 ReusedExchange (131) (131) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#72] -Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#74, [id=#75] * Project (135) +- * Filter (134) +- * ColumnarToRow (133) @@ -769,21 +769,21 @@ Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subque (132) Scan parquet default.date_dim -Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (133) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] (134) Filter [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 1998)) AND (d_moy#90 = 12)) AND (d_dom#91 = 16)) +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#92 = 1998)) AND (d_moy#113 = 12)) AND (d_dom#114 = 16)) (135) Project [codegen id : 1] -Output [1]: [d_week_seq#30] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [1]: [d_week_seq#44] +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index 3507d76bcf9b9..cb9b410ea8568 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -82,53 +82,53 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Exchange [brand_id,class_id,category_id] #5 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #9 + BroadcastExchange #8 WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 + BroadcastExchange #6 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 WholeStageCodegen (9) 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 9b47400d3b780..26141bdb911ca 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 @@ -40,39 +40,39 @@ TakeOrderedAndProject (213) : : : : : +- * HashAggregate (44) : : : : : +- Exchange (43) : : : : : +- * HashAggregate (42) - : : : : : +- SortMergeJoin LeftSemi (41) - : : : : : :- * Sort (26) - : : : : : : +- Exchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (18) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : : :- * Filter (11) - : : : : : : : : +- * ColumnarToRow (10) - : : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : : +- BroadcastExchange (16) - : : : : : : : +- * Project (15) - : : : : : : : +- * Filter (14) - : : : : : : : +- * ColumnarToRow (13) - : : : : : : : +- Scan parquet default.date_dim (12) - : : : : : : +- BroadcastExchange (22) - : : : : : : +- * Filter (21) - : : : : : : +- * ColumnarToRow (20) - : : : : : : +- Scan parquet default.item (19) - : : : : : +- * Sort (40) - : : : : : +- Exchange (39) - : : : : : +- * Project (38) - : : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : : :- * Project (32) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : : :- * Filter (29) - : : : : : : : +- * ColumnarToRow (28) - : : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : : +- ReusedExchange (30) - : : : : : +- BroadcastExchange (36) - : : : : : +- * Filter (35) - : : : : : +- * ColumnarToRow (34) - : : : : : +- Scan parquet default.item (33) + : : : : : +- * Project (41) + : : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (39) + : : : : : +- SortMergeJoin LeftSemi (38) + : : : : : :- * Sort (23) + : : : : : : +- Exchange (22) + : : : : : : +- * Filter (21) + : : : : : : +- * ColumnarToRow (20) + : : : : : : +- Scan parquet default.item (19) + : : : : : +- * Sort (37) + : : : : : +- Exchange (36) + : : : : : +- * Project (35) + : : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : : :- * Project (29) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : : :- * Filter (26) + : : : : : : : +- * ColumnarToRow (25) + : : : : : : : +- Scan parquet default.catalog_sales (24) + : : : : : : +- ReusedExchange (27) + : : : : : +- BroadcastExchange (33) + : : : : : +- * Filter (32) + : : : : : +- * ColumnarToRow (31) + : : : : : +- Scan parquet default.item (30) : : : : +- * Sort (57) : : : : +- Exchange (56) : : : : +- * Project (55) @@ -244,10 +244,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 20] +(7) ColumnarToRow [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 20] +(8) Filter [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) @@ -259,10 +259,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 5] +(10) ColumnarToRow [codegen id : 10] Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -(11) Filter [codegen id : 5] +(11) Filter [codegen id : 10] Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -288,51 +288,38 @@ Input [2]: [d_date_sk#12, d_year#13] Input [1]: [d_date_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(17) BroadcastHashJoin [codegen id : 5] +(17) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#12] Join condition: None -(18) Project [codegen id : 5] +(18) Project [codegen id : 10] Output [1]: [ss_item_sk#1] Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] (19) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) - -(22) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] - -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join condition: None +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Condition : (((isnotnull(i_item_sk#15) AND isnotnull(i_brand_id#16)) AND isnotnull(i_class_id#17)) AND isnotnull(i_category_id#18)) -(24) Project [codegen id : 5] -Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] -Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(22) Exchange +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: hashpartitioning(coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] -(25) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] +(23) Sort [codegen id : 5] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: [coalesce(i_brand_id#16, 0) ASC NULLS FIRST, isnull(i_brand_id#16) ASC NULLS FIRST, coalesce(i_class_id#17, 0) ASC NULLS FIRST, isnull(i_class_id#17) ASC NULLS FIRST, coalesce(i_category_id#18, 0) ASC NULLS FIRST, isnull(i_category_id#18) ASC NULLS FIRST], false, 0 -(26) Sort [codegen id : 6] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 - -(27) Scan parquet default.catalog_sales +(24) Scan parquet default.catalog_sales Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] @@ -340,215 +327,228 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] +(25) ColumnarToRow [codegen id : 8] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -(29) Filter [codegen id : 9] +(26) Filter [codegen id : 8] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] Condition : isnotnull(cs_item_sk#20) -(30) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +(27) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#22] -(31) BroadcastHashJoin [codegen id : 9] +(28) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#22] Join condition: None -(32) Project [codegen id : 9] +(29) Project [codegen id : 8] Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] +Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#22] -(33) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(30) Scan parquet default.item +Output [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(31) ColumnarToRow [codegen id : 7] +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -(35) Filter [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +(32) Filter [codegen id : 7] +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Condition : isnotnull(i_item_sk#23) -(36) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +(33) BroadcastExchange +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] -(37) BroadcastHashJoin [codegen id : 9] +(34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#23] Join condition: None -(38) Project [codegen id : 9] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(35) Project [codegen id : 8] +Output [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Input [5]: [cs_item_sk#20, i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] + +(36) Exchange +Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: hashpartitioning(coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26), 5), ENSURE_REQUIREMENTS, [id=#28] -(39) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] +(37) Sort [codegen id : 9] +Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: [coalesce(i_brand_id#24, 0) ASC NULLS FIRST, isnull(i_brand_id#24) ASC NULLS FIRST, coalesce(i_class_id#25, 0) ASC NULLS FIRST, isnull(i_class_id#25) ASC NULLS FIRST, coalesce(i_category_id#26, 0) ASC NULLS FIRST, isnull(i_category_id#26) ASC NULLS FIRST], false, 0 + +(38) SortMergeJoin +Left keys [6]: [coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18)] +Right keys [6]: [coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26)] +Join condition: None -(40) Sort [codegen id : 10] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(39) BroadcastExchange +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] -(41) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +(40) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#15] Join condition: None -(42) HashAggregate [codegen id : 11] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(41) Project [codegen id : 10] +Output [3]: [i_brand_id#16 AS brand_id#30, i_class_id#17 AS class_id#31, i_category_id#18 AS category_id#32] +Input [5]: [ss_item_sk#1, i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] + +(42) HashAggregate [codegen id : 10] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#30, class_id#31, category_id#32] (43) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] -(44) HashAggregate [codegen id : 12] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(44) HashAggregate [codegen id : 11] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#30, class_id#31, category_id#32] (45) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32), 5), ENSURE_REQUIREMENTS, [id=#34] -(46) Sort [codegen id : 13] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 12] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: [coalesce(brand_id#30, 0) ASC NULLS FIRST, isnull(brand_id#30) ASC NULLS FIRST, coalesce(class_id#31, 0) ASC NULLS FIRST, isnull(class_id#31) ASC NULLS FIRST, coalesce(category_id#32, 0) ASC NULLS FIRST, isnull(category_id#32) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Output [2]: [ws_item_sk#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +(48) ColumnarToRow [codegen id : 15] +Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] -(49) Filter [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(49) Filter [codegen id : 15] +Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#35) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#37] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(51) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#37] Join condition: None -(52) Project [codegen id : 16] -Output [1]: [ws_item_sk#26] -Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] +(52) Project [codegen id : 15] +Output [1]: [ws_item_sk#35] +Input [3]: [ws_item_sk#35, ws_sold_date_sk#36, d_date_sk#37] -(53) ReusedExchange [Reuses operator id: 36] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(53) ReusedExchange [Reuses operator id: 33] +Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(54) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [i_item_sk#38] Join condition: None -(55) Project [codegen id : 16] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(55) Project [codegen id : 15] +Output [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Input [5]: [ws_item_sk#35, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] (56) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: hashpartitioning(coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41), 5), ENSURE_REQUIREMENTS, [id=#42] -(57) Sort [codegen id : 17] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 16] +Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: [coalesce(i_brand_id#39, 0) ASC NULLS FIRST, isnull(i_brand_id#39) ASC NULLS FIRST, coalesce(i_class_id#40, 0) ASC NULLS FIRST, isnull(i_class_id#40) ASC NULLS FIRST, coalesce(i_category_id#41, 0) ASC NULLS FIRST, isnull(i_category_id#41) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41)] Join condition: None -(59) HashAggregate [codegen id : 18] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(59) HashAggregate [codegen id : 17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#30, class_id#31, category_id#32] (60) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#43] -(61) HashAggregate [codegen id : 19] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(61) HashAggregate [codegen id : 18] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#30, class_id#31, category_id#32] (62) BroadcastExchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#44] -(63) BroadcastHashJoin [codegen id : 20] +(63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#16, class_id#17, category_id#18] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None -(64) Project [codegen id : 20] -Output [1]: [i_item_sk#7 AS ss_item_sk#31] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] +(64) Project [codegen id : 19] +Output [1]: [i_item_sk#7 AS ss_item_sk#45] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#30, class_id#31, category_id#32] (65) Exchange -Input [1]: [ss_item_sk#31] -Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [1]: [ss_item_sk#45] +Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] -(66) Sort [codegen id : 21] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 20] +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#45] Join condition: None (68) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#33] +Output [3]: [d_date_sk#12, d_year#13, d_moy#47] 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 -(69) ColumnarToRow [codegen id : 22] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] +(69) ColumnarToRow [codegen id : 21] +Input [3]: [d_date_sk#12, d_year#13, d_moy#47] -(70) Filter [codegen id : 22] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#33)) AND (d_year#13 = 2000)) AND (d_moy#33 = 11)) AND isnotnull(d_date_sk#12)) +(70) Filter [codegen id : 21] +Input [3]: [d_date_sk#12, d_year#13, d_moy#47] +Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#47)) AND (d_year#13 = 2000)) AND (d_moy#47 = 11)) AND isnotnull(d_date_sk#12)) -(71) Project [codegen id : 22] +(71) Project [codegen id : 21] Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] +Input [3]: [d_date_sk#12, d_year#13, d_moy#47] (72) BroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] -(73) BroadcastHashJoin [codegen id : 44] +(73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#12] Join condition: None -(74) Project [codegen id : 44] +(74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] @@ -559,657 +559,657 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 23] +(76) ColumnarToRow [codegen id : 22] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(77) Filter [codegen id : 23] +(77) Filter [codegen id : 22] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : isnotnull(i_item_sk#7) (78) Exchange Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#35] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#49] -(79) Sort [codegen id : 24] +(79) Sort [codegen id : 23] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#45] -(81) Sort [codegen id : 43] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 41] +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (82) SortMergeJoin Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#45] Join condition: None (83) BroadcastExchange Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] -(84) BroadcastHashJoin [codegen id : 44] +(84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#7] Join condition: None -(85) Project [codegen id : 44] +(85) Project [codegen id : 42] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(86) HashAggregate [codegen id : 44] +(86) HashAggregate [codegen id : 42] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#37, isEmpty#38, count#39] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] +Aggregate Attributes [3]: [sum#51, isEmpty#52, count#53] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#57] -(88) HashAggregate [codegen id : 45] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] +(88) HashAggregate [codegen id : 43] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44, count(1)#45] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sales#46, count(1)#45 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] -(89) Filter [codegen id : 45] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48 as decimal(32,6)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(89) Filter [codegen id : 43] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(90) Project [codegen id : 45] -Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] +(90) Project [codegen id : 43] +Output [6]: [store AS channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] (91) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Output [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 46] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +(92) ColumnarToRow [codegen id : 44] +Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] -(93) Filter [codegen id : 46] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +(93) Filter [codegen id : 44] +Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] Condition : isnotnull(cs_item_sk#20) (94) Exchange -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#68] -(95) Sort [codegen id : 47] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +(95) Sort [codegen id : 45] +Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] Arguments: [cs_item_sk#20 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#45] -(97) Sort [codegen id : 66] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 63] +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (98) SortMergeJoin Left keys [1]: [cs_item_sk#20] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#45] Join condition: None (99) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#69] -(100) BroadcastHashJoin [codegen id : 89] +(100) BroadcastHashJoin [codegen id : 85] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#69] Join condition: None -(101) Project [codegen id : 89] -Output [3]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53] -Input [5]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] +(101) Project [codegen id : 85] +Output [3]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67] +Input [5]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21, d_date_sk#69] (102) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73] -(103) BroadcastHashJoin [codegen id : 89] +(103) BroadcastHashJoin [codegen id : 85] Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#70] Join condition: None -(104) Project [codegen id : 89] -Output [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(104) Project [codegen id : 85] +Output [5]: [cs_quantity#66, cs_list_price#67, i_brand_id#71, i_class_id#72, i_category_id#73] +Input [7]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73] -(105) HashAggregate [codegen id : 89] -Input [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] +(105) HashAggregate [codegen id : 85] +Input [5]: [cs_quantity#66, cs_list_price#67, i_brand_id#71, i_class_id#72, i_category_id#73] +Keys [3]: [i_brand_id#71, i_class_id#72, i_category_id#73] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#74, isEmpty#75, count#76] +Results [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] (106) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] +Arguments: hashpartitioning(i_brand_id#71, i_class_id#72, i_category_id#73, 5), ENSURE_REQUIREMENTS, [id=#80] -(107) HashAggregate [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] +(107) HashAggregate [codegen id : 86] +Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] +Keys [3]: [i_brand_id#71, i_class_id#72, i_category_id#73] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81, count(1)#82] +Results [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sales#83, count(1)#82 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] -(108) Filter [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(108) Filter [codegen id : 86] +Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(109) Project [codegen id : 90] -Output [6]: [catalog AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] +(109) Project [codegen id : 86] +Output [6]: [catalog AS channel#86, i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84] +Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] (110) Scan parquet default.web_sales -Output [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Output [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 91] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +(111) ColumnarToRow [codegen id : 87] +Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] -(112) Filter [codegen id : 91] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(112) Filter [codegen id : 87] +Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#35) (113) Exchange -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_item_sk#26, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Arguments: hashpartitioning(ws_item_sk#35, 5), ENSURE_REQUIREMENTS, [id=#89] -(114) Sort [codegen id : 92] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Arguments: [ws_item_sk#26 ASC NULLS FIRST], false, 0 +(114) Sort [codegen id : 88] +Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Arguments: [ws_item_sk#35 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#45] -(116) Sort [codegen id : 111] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(116) Sort [codegen id : 106] +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (117) SortMergeJoin -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [ss_item_sk#45] Join condition: None (118) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#90] -(119) BroadcastHashJoin [codegen id : 134] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(119) BroadcastHashJoin [codegen id : 128] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#90] Join condition: None -(120) Project [codegen id : 134] -Output [3]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69] -Input [5]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] +(120) Project [codegen id : 128] +Output [3]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88] +Input [5]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36, d_date_sk#90] (121) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94] -(122) BroadcastHashJoin [codegen id : 134] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(122) BroadcastHashJoin [codegen id : 128] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [i_item_sk#91] Join condition: None -(123) Project [codegen id : 134] -Output [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(123) Project [codegen id : 128] +Output [5]: [ws_quantity#87, ws_list_price#88, i_brand_id#92, i_class_id#93, i_category_id#94] +Input [7]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94] -(124) HashAggregate [codegen id : 134] -Input [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#71, isEmpty#72, count#73] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] +(124) HashAggregate [codegen id : 128] +Input [5]: [ws_quantity#87, ws_list_price#88, i_brand_id#92, i_class_id#93, i_category_id#94] +Keys [3]: [i_brand_id#92, i_class_id#93, i_category_id#94] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#95, isEmpty#96, count#97] +Results [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] (125) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] +Arguments: hashpartitioning(i_brand_id#92, i_class_id#93, i_category_id#94, 5), ENSURE_REQUIREMENTS, [id=#101] -(126) HashAggregate [codegen id : 135] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78, count(1)#79] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sales#80, count(1)#79 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +(126) HashAggregate [codegen id : 129] +Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] +Keys [3]: [i_brand_id#92, i_class_id#93, i_category_id#94] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102, count(1)#103] +Results [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sales#104, count(1)#103 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] -(127) Filter [codegen id : 135] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(127) Filter [codegen id : 129] +Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(128) Project [codegen id : 135] -Output [6]: [web AS channel#83, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +(128) Project [codegen id : 129] +Output [6]: [web AS channel#107, i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105] +Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] (129) Union -(130) HashAggregate [codegen id : 136] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#84, isEmpty#85, sum#86] -Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#87, isEmpty#88, sum#89] +(130) HashAggregate [codegen id : 130] +Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61] +Keys [4]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] +Aggregate Attributes [3]: [sum#108, isEmpty#109, sum#110] +Results [7]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum#111, isEmpty#112, sum#113] (131) Exchange -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#87, isEmpty#88, sum#89] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [7]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum#111, isEmpty#112, sum#113] +Arguments: hashpartitioning(channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#114] -(132) HashAggregate [codegen id : 137] -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#87, isEmpty#88, sum#89] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#91, sum(number_sales#47)#92] -Results [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum(sales#46)#91 AS sum_sales#93, sum(number_sales#47)#92 AS number_sales#94] +(132) HashAggregate [codegen id : 131] +Input [7]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum#111, isEmpty#112, sum#113] +Keys [4]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(sales#60), sum(number_sales#61)] +Aggregate Attributes [2]: [sum(sales#60)#115, sum(number_sales#61)#116] +Results [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum(sales#60)#115 AS sum_sales#117, sum(number_sales#61)#116 AS number_sales#118] (133) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#95, isEmpty#96, count#97] +Output [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sum#122, isEmpty#123, count#124] -(134) HashAggregate [codegen id : 182] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#95, isEmpty#96, count#97] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#98, count(1)#99] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#98 AS sales#46, count(1)#99 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#98 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100] +(134) HashAggregate [codegen id : 174] +Input [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sum#122, isEmpty#123, count#124] +Keys [3]: [i_brand_id#119, i_class_id#120, i_category_id#121] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#127, count(1)#128] +Results [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#127 AS sales#60, count(1)#128 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#127 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129] -(135) Filter [codegen id : 182] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(135) Filter [codegen id : 174] +Input [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(136) Project [codegen id : 182] -Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100] +(136) Project [codegen id : 174] +Output [6]: [store AS channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sales#60, number_sales#61] +Input [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129] (137) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#101, isEmpty#102, count#103] +Output [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sum#133, isEmpty#134, count#135] -(138) HashAggregate [codegen id : 227] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#101, isEmpty#102, count#103] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#104, count(1)#105] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sales#64, count(1)#105 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106] +(138) HashAggregate [codegen id : 217] +Input [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sum#133, isEmpty#134, count#135] +Keys [3]: [i_brand_id#130, i_class_id#131, i_category_id#132] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#138, count(1)#139] +Results [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sales#83, count(1)#139 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140] -(139) Filter [codegen id : 227] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(139) Filter [codegen id : 217] +Input [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(140) Project [codegen id : 227] -Output [6]: [catalog AS channel#107, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106] +(140) Project [codegen id : 217] +Output [6]: [catalog AS channel#86, i_brand_id#130, i_class_id#131, i_category_id#132, sales#83, number_sales#84] +Input [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140] (141) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#108, isEmpty#109, count#110] +Output [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sum#144, isEmpty#145, count#146] -(142) HashAggregate [codegen id : 272] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#108, isEmpty#109, count#110] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#111, count(1)#112] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#111 AS sales#80, count(1)#112 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#111 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113] +(142) HashAggregate [codegen id : 260] +Input [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sum#144, isEmpty#145, count#146] +Keys [3]: [i_brand_id#141, i_class_id#142, i_category_id#143] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#149, count(1)#150] +Results [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#149 AS sales#104, count(1)#150 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151] -(143) Filter [codegen id : 272] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(143) Filter [codegen id : 260] +Input [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(144) Project [codegen id : 272] -Output [6]: [web AS channel#114, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113] +(144) Project [codegen id : 260] +Output [6]: [web AS channel#107, i_brand_id#141, i_class_id#142, i_category_id#143, sales#104, number_sales#105] +Input [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151] (145) Union -(146) HashAggregate [codegen id : 273] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] -Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#118, isEmpty#119, sum#120] +(146) HashAggregate [codegen id : 261] +Input [6]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sales#60, number_sales#61] +Keys [4]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121] +Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] +Aggregate Attributes [3]: [sum#152, isEmpty#153, sum#154] +Results [7]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sum#155, isEmpty#156, sum#157] (147) Exchange -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#118, isEmpty#119, sum#120] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#121] - -(148) HashAggregate [codegen id : 274] -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#118, isEmpty#119, sum#120] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#122, sum(number_sales#47)#123] -Results [5]: [channel#51, i_brand_id#8, i_class_id#9, sum(sales#46)#122 AS sum_sales#93, sum(number_sales#47)#123 AS number_sales#94] - -(149) HashAggregate [codegen id : 274] -Input [5]: [channel#51, i_brand_id#8, i_class_id#9, sum_sales#93, number_sales#94] -Keys [3]: [channel#51, i_brand_id#8, i_class_id#9] -Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] -Aggregate Attributes [3]: [sum#124, isEmpty#125, sum#126] -Results [6]: [channel#51, i_brand_id#8, i_class_id#9, sum#127, isEmpty#128, sum#129] +Input [7]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sum#155, isEmpty#156, sum#157] +Arguments: hashpartitioning(channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, 5), ENSURE_REQUIREMENTS, [id=#158] + +(148) HashAggregate [codegen id : 262] +Input [7]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sum#155, isEmpty#156, sum#157] +Keys [4]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121] +Functions [2]: [sum(sales#60), sum(number_sales#61)] +Aggregate Attributes [2]: [sum(sales#60)#159, sum(number_sales#61)#160] +Results [5]: [channel#65, i_brand_id#119, i_class_id#120, sum(sales#60)#159 AS sum_sales#117, sum(number_sales#61)#160 AS number_sales#118] + +(149) HashAggregate [codegen id : 262] +Input [5]: [channel#65, i_brand_id#119, i_class_id#120, sum_sales#117, number_sales#118] +Keys [3]: [channel#65, i_brand_id#119, i_class_id#120] +Functions [2]: [partial_sum(sum_sales#117), partial_sum(number_sales#118)] +Aggregate Attributes [3]: [sum#161, isEmpty#162, sum#163] +Results [6]: [channel#65, i_brand_id#119, i_class_id#120, sum#164, isEmpty#165, sum#166] (150) Exchange -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, sum#127, isEmpty#128, sum#129] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, 5), ENSURE_REQUIREMENTS, [id=#130] +Input [6]: [channel#65, i_brand_id#119, i_class_id#120, sum#164, isEmpty#165, sum#166] +Arguments: hashpartitioning(channel#65, i_brand_id#119, i_class_id#120, 5), ENSURE_REQUIREMENTS, [id=#167] -(151) HashAggregate [codegen id : 275] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, sum#127, isEmpty#128, sum#129] -Keys [3]: [channel#51, i_brand_id#8, i_class_id#9] -Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] -Aggregate Attributes [2]: [sum(sum_sales#93)#131, sum(number_sales#94)#132] -Results [6]: [channel#51, i_brand_id#8, i_class_id#9, null AS i_category_id#133, sum(sum_sales#93)#131 AS sum(sum_sales)#134, sum(number_sales#94)#132 AS sum(number_sales)#135] +(151) HashAggregate [codegen id : 263] +Input [6]: [channel#65, i_brand_id#119, i_class_id#120, sum#164, isEmpty#165, sum#166] +Keys [3]: [channel#65, i_brand_id#119, i_class_id#120] +Functions [2]: [sum(sum_sales#117), sum(number_sales#118)] +Aggregate Attributes [2]: [sum(sum_sales#117)#168, sum(number_sales#118)#169] +Results [6]: [channel#65, i_brand_id#119, i_class_id#120, null AS i_category_id#170, sum(sum_sales#117)#168 AS sum(sum_sales)#171, sum(number_sales#118)#169 AS sum(number_sales)#172] (152) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#136, isEmpty#137, count#138] +Output [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sum#176, isEmpty#177, count#178] -(153) HashAggregate [codegen id : 320] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#136, isEmpty#137, count#138] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#139, count(1)#140] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sales#46, count(1)#140 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141] +(153) HashAggregate [codegen id : 306] +Input [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sum#176, isEmpty#177, count#178] +Keys [3]: [i_brand_id#173, i_class_id#174, i_category_id#175] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#181, count(1)#182] +Results [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sales#60, count(1)#182 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183] -(154) Filter [codegen id : 320] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(154) Filter [codegen id : 306] +Input [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(155) Project [codegen id : 320] -Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141] +(155) Project [codegen id : 306] +Output [6]: [store AS channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sales#60, number_sales#61] +Input [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183] (156) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#142, isEmpty#143, count#144] +Output [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sum#187, isEmpty#188, count#189] -(157) HashAggregate [codegen id : 365] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#142, isEmpty#143, count#144] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#145, count(1)#146] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sales#64, count(1)#146 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147] +(157) HashAggregate [codegen id : 349] +Input [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sum#187, isEmpty#188, count#189] +Keys [3]: [i_brand_id#184, i_class_id#185, i_category_id#186] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#192, count(1)#193] +Results [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sales#83, count(1)#193 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194] -(158) Filter [codegen id : 365] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(158) Filter [codegen id : 349] +Input [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(159) Project [codegen id : 365] -Output [6]: [catalog AS channel#148, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147] +(159) Project [codegen id : 349] +Output [6]: [catalog AS channel#86, i_brand_id#184, i_class_id#185, i_category_id#186, sales#83, number_sales#84] +Input [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194] (160) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#149, isEmpty#150, count#151] +Output [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sum#198, isEmpty#199, count#200] -(161) HashAggregate [codegen id : 410] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#149, isEmpty#150, count#151] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#152, count(1)#153] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#152 AS sales#80, count(1)#153 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#152 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154] +(161) HashAggregate [codegen id : 392] +Input [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sum#198, isEmpty#199, count#200] +Keys [3]: [i_brand_id#195, i_class_id#196, i_category_id#197] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#203, count(1)#204] +Results [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#203 AS sales#104, count(1)#204 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#203 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205] -(162) Filter [codegen id : 410] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(162) Filter [codegen id : 392] +Input [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(163) Project [codegen id : 410] -Output [6]: [web AS channel#155, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154] +(163) Project [codegen id : 392] +Output [6]: [web AS channel#107, i_brand_id#195, i_class_id#196, i_category_id#197, sales#104, number_sales#105] +Input [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205] (164) Union -(165) HashAggregate [codegen id : 411] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#156, isEmpty#157, sum#158] -Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#159, isEmpty#160, sum#161] +(165) HashAggregate [codegen id : 393] +Input [6]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sales#60, number_sales#61] +Keys [4]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175] +Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] +Aggregate Attributes [3]: [sum#206, isEmpty#207, sum#208] +Results [7]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sum#209, isEmpty#210, sum#211] (166) Exchange -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#159, isEmpty#160, sum#161] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#162] - -(167) HashAggregate [codegen id : 412] -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#159, isEmpty#160, sum#161] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#163, sum(number_sales#47)#164] -Results [4]: [channel#51, i_brand_id#8, sum(sales#46)#163 AS sum_sales#93, sum(number_sales#47)#164 AS number_sales#94] - -(168) HashAggregate [codegen id : 412] -Input [4]: [channel#51, i_brand_id#8, sum_sales#93, number_sales#94] -Keys [2]: [channel#51, i_brand_id#8] -Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] -Aggregate Attributes [3]: [sum#165, isEmpty#166, sum#167] -Results [5]: [channel#51, i_brand_id#8, sum#168, isEmpty#169, sum#170] +Input [7]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sum#209, isEmpty#210, sum#211] +Arguments: hashpartitioning(channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, 5), ENSURE_REQUIREMENTS, [id=#212] + +(167) HashAggregate [codegen id : 394] +Input [7]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sum#209, isEmpty#210, sum#211] +Keys [4]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175] +Functions [2]: [sum(sales#60), sum(number_sales#61)] +Aggregate Attributes [2]: [sum(sales#60)#213, sum(number_sales#61)#214] +Results [4]: [channel#65, i_brand_id#173, sum(sales#60)#213 AS sum_sales#117, sum(number_sales#61)#214 AS number_sales#118] + +(168) HashAggregate [codegen id : 394] +Input [4]: [channel#65, i_brand_id#173, sum_sales#117, number_sales#118] +Keys [2]: [channel#65, i_brand_id#173] +Functions [2]: [partial_sum(sum_sales#117), partial_sum(number_sales#118)] +Aggregate Attributes [3]: [sum#215, isEmpty#216, sum#217] +Results [5]: [channel#65, i_brand_id#173, sum#218, isEmpty#219, sum#220] (169) Exchange -Input [5]: [channel#51, i_brand_id#8, sum#168, isEmpty#169, sum#170] -Arguments: hashpartitioning(channel#51, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [id=#171] +Input [5]: [channel#65, i_brand_id#173, sum#218, isEmpty#219, sum#220] +Arguments: hashpartitioning(channel#65, i_brand_id#173, 5), ENSURE_REQUIREMENTS, [id=#221] -(170) HashAggregate [codegen id : 413] -Input [5]: [channel#51, i_brand_id#8, sum#168, isEmpty#169, sum#170] -Keys [2]: [channel#51, i_brand_id#8] -Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] -Aggregate Attributes [2]: [sum(sum_sales#93)#172, sum(number_sales#94)#173] -Results [6]: [channel#51, i_brand_id#8, null AS i_class_id#174, null AS i_category_id#175, sum(sum_sales#93)#172 AS sum(sum_sales)#176, sum(number_sales#94)#173 AS sum(number_sales)#177] +(170) HashAggregate [codegen id : 395] +Input [5]: [channel#65, i_brand_id#173, sum#218, isEmpty#219, sum#220] +Keys [2]: [channel#65, i_brand_id#173] +Functions [2]: [sum(sum_sales#117), sum(number_sales#118)] +Aggregate Attributes [2]: [sum(sum_sales#117)#222, sum(number_sales#118)#223] +Results [6]: [channel#65, i_brand_id#173, null AS i_class_id#224, null AS i_category_id#225, sum(sum_sales#117)#222 AS sum(sum_sales)#226, sum(number_sales#118)#223 AS sum(number_sales)#227] (171) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#178, isEmpty#179, count#180] +Output [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sum#231, isEmpty#232, count#233] -(172) HashAggregate [codegen id : 458] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#178, isEmpty#179, count#180] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#181, count(1)#182] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sales#46, count(1)#182 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183] +(172) HashAggregate [codegen id : 438] +Input [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sum#231, isEmpty#232, count#233] +Keys [3]: [i_brand_id#228, i_class_id#229, i_category_id#230] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#236, count(1)#237] +Results [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#236 AS sales#60, count(1)#237 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#236 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238] -(173) Filter [codegen id : 458] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(173) Filter [codegen id : 438] +Input [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(174) Project [codegen id : 458] -Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183] +(174) Project [codegen id : 438] +Output [6]: [store AS channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sales#60, number_sales#61] +Input [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238] (175) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#184, isEmpty#185, count#186] +Output [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sum#242, isEmpty#243, count#244] -(176) HashAggregate [codegen id : 503] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#184, isEmpty#185, count#186] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#187, count(1)#188] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sales#64, count(1)#188 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189] +(176) HashAggregate [codegen id : 481] +Input [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sum#242, isEmpty#243, count#244] +Keys [3]: [i_brand_id#239, i_class_id#240, i_category_id#241] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#247, count(1)#248] +Results [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#247 AS sales#83, count(1)#248 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#247 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249] -(177) Filter [codegen id : 503] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(177) Filter [codegen id : 481] +Input [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(178) Project [codegen id : 503] -Output [6]: [catalog AS channel#190, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189] +(178) Project [codegen id : 481] +Output [6]: [catalog AS channel#86, i_brand_id#239, i_class_id#240, i_category_id#241, sales#83, number_sales#84] +Input [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249] (179) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#191, isEmpty#192, count#193] +Output [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sum#253, isEmpty#254, count#255] -(180) HashAggregate [codegen id : 548] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#191, isEmpty#192, count#193] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#194, count(1)#195] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#194 AS sales#80, count(1)#195 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#194 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196] +(180) HashAggregate [codegen id : 524] +Input [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sum#253, isEmpty#254, count#255] +Keys [3]: [i_brand_id#250, i_class_id#251, i_category_id#252] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#258, count(1)#259] +Results [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#258 AS sales#104, count(1)#259 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#258 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260] -(181) Filter [codegen id : 548] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(181) Filter [codegen id : 524] +Input [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(182) Project [codegen id : 548] -Output [6]: [web AS channel#197, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196] +(182) Project [codegen id : 524] +Output [6]: [web AS channel#107, i_brand_id#250, i_class_id#251, i_category_id#252, sales#104, number_sales#105] +Input [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260] (183) Union -(184) HashAggregate [codegen id : 549] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#198, isEmpty#199, sum#200] -Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#201, isEmpty#202, sum#203] +(184) HashAggregate [codegen id : 525] +Input [6]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sales#60, number_sales#61] +Keys [4]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230] +Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] +Aggregate Attributes [3]: [sum#261, isEmpty#262, sum#263] +Results [7]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sum#264, isEmpty#265, sum#266] (185) Exchange -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#201, isEmpty#202, sum#203] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#204] - -(186) HashAggregate [codegen id : 550] -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#201, isEmpty#202, sum#203] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#205, sum(number_sales#47)#206] -Results [3]: [channel#51, sum(sales#46)#205 AS sum_sales#93, sum(number_sales#47)#206 AS number_sales#94] - -(187) HashAggregate [codegen id : 550] -Input [3]: [channel#51, sum_sales#93, number_sales#94] -Keys [1]: [channel#51] -Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] -Aggregate Attributes [3]: [sum#207, isEmpty#208, sum#209] -Results [4]: [channel#51, sum#210, isEmpty#211, sum#212] +Input [7]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sum#264, isEmpty#265, sum#266] +Arguments: hashpartitioning(channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, 5), ENSURE_REQUIREMENTS, [id=#267] + +(186) HashAggregate [codegen id : 526] +Input [7]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sum#264, isEmpty#265, sum#266] +Keys [4]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230] +Functions [2]: [sum(sales#60), sum(number_sales#61)] +Aggregate Attributes [2]: [sum(sales#60)#268, sum(number_sales#61)#269] +Results [3]: [channel#65, sum(sales#60)#268 AS sum_sales#117, sum(number_sales#61)#269 AS number_sales#118] + +(187) HashAggregate [codegen id : 526] +Input [3]: [channel#65, sum_sales#117, number_sales#118] +Keys [1]: [channel#65] +Functions [2]: [partial_sum(sum_sales#117), partial_sum(number_sales#118)] +Aggregate Attributes [3]: [sum#270, isEmpty#271, sum#272] +Results [4]: [channel#65, sum#273, isEmpty#274, sum#275] (188) Exchange -Input [4]: [channel#51, sum#210, isEmpty#211, sum#212] -Arguments: hashpartitioning(channel#51, 5), ENSURE_REQUIREMENTS, [id=#213] +Input [4]: [channel#65, sum#273, isEmpty#274, sum#275] +Arguments: hashpartitioning(channel#65, 5), ENSURE_REQUIREMENTS, [id=#276] -(189) HashAggregate [codegen id : 551] -Input [4]: [channel#51, sum#210, isEmpty#211, sum#212] -Keys [1]: [channel#51] -Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] -Aggregate Attributes [2]: [sum(sum_sales#93)#214, sum(number_sales#94)#215] -Results [6]: [channel#51, null AS i_brand_id#216, null AS i_class_id#217, null AS i_category_id#218, sum(sum_sales#93)#214 AS sum(sum_sales)#219, sum(number_sales#94)#215 AS sum(number_sales)#220] +(189) HashAggregate [codegen id : 527] +Input [4]: [channel#65, sum#273, isEmpty#274, sum#275] +Keys [1]: [channel#65] +Functions [2]: [sum(sum_sales#117), sum(number_sales#118)] +Aggregate Attributes [2]: [sum(sum_sales#117)#277, sum(number_sales#118)#278] +Results [6]: [channel#65, null AS i_brand_id#279, null AS i_class_id#280, null AS i_category_id#281, sum(sum_sales#117)#277 AS sum(sum_sales)#282, sum(number_sales#118)#278 AS sum(number_sales)#283] (190) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#221, isEmpty#222, count#223] +Output [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sum#287, isEmpty#288, count#289] -(191) HashAggregate [codegen id : 596] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#221, isEmpty#222, count#223] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#224, count(1)#225] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sales#46, count(1)#225 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226] +(191) HashAggregate [codegen id : 570] +Input [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sum#287, isEmpty#288, count#289] +Keys [3]: [i_brand_id#284, i_class_id#285, i_category_id#286] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#292, count(1)#293] +Results [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#292 AS sales#60, count(1)#293 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#292 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294] -(192) Filter [codegen id : 596] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(192) Filter [codegen id : 570] +Input [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(193) Project [codegen id : 596] -Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226] +(193) Project [codegen id : 570] +Output [6]: [store AS channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sales#60, number_sales#61] +Input [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294] (194) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#227, isEmpty#228, count#229] +Output [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sum#298, isEmpty#299, count#300] -(195) HashAggregate [codegen id : 641] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#227, isEmpty#228, count#229] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#230, count(1)#231] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sales#64, count(1)#231 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232] +(195) HashAggregate [codegen id : 613] +Input [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sum#298, isEmpty#299, count#300] +Keys [3]: [i_brand_id#295, i_class_id#296, i_category_id#297] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#303, count(1)#304] +Results [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#303 AS sales#83, count(1)#304 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#303 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305] -(196) Filter [codegen id : 641] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(196) Filter [codegen id : 613] +Input [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(197) Project [codegen id : 641] -Output [6]: [catalog AS channel#233, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232] +(197) Project [codegen id : 613] +Output [6]: [catalog AS channel#86, i_brand_id#295, i_class_id#296, i_category_id#297, sales#83, number_sales#84] +Input [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305] (198) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#234, isEmpty#235, count#236] +Output [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sum#309, isEmpty#310, count#311] -(199) HashAggregate [codegen id : 686] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#234, isEmpty#235, count#236] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#237, count(1)#238] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#237 AS sales#80, count(1)#238 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#237 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239] +(199) HashAggregate [codegen id : 656] +Input [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sum#309, isEmpty#310, count#311] +Keys [3]: [i_brand_id#306, i_class_id#307, i_category_id#308] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#314, count(1)#315] +Results [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#314 AS sales#104, count(1)#315 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#314 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316] -(200) Filter [codegen id : 686] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(200) Filter [codegen id : 656] +Input [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) -(201) Project [codegen id : 686] -Output [6]: [web AS channel#240, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239] +(201) Project [codegen id : 656] +Output [6]: [web AS channel#107, i_brand_id#306, i_class_id#307, i_category_id#308, sales#104, number_sales#105] +Input [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316] (202) Union -(203) HashAggregate [codegen id : 687] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#241, isEmpty#242, sum#243] -Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#244, isEmpty#245, sum#246] +(203) HashAggregate [codegen id : 657] +Input [6]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sales#60, number_sales#61] +Keys [4]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286] +Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] +Aggregate Attributes [3]: [sum#317, isEmpty#318, sum#319] +Results [7]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sum#320, isEmpty#321, sum#322] (204) Exchange -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#244, isEmpty#245, sum#246] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#247] - -(205) HashAggregate [codegen id : 688] -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#244, isEmpty#245, sum#246] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#248, sum(number_sales#47)#249] -Results [2]: [sum(sales#46)#248 AS sum_sales#93, sum(number_sales#47)#249 AS number_sales#94] - -(206) HashAggregate [codegen id : 688] -Input [2]: [sum_sales#93, number_sales#94] +Input [7]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sum#320, isEmpty#321, sum#322] +Arguments: hashpartitioning(channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, 5), ENSURE_REQUIREMENTS, [id=#323] + +(205) HashAggregate [codegen id : 658] +Input [7]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sum#320, isEmpty#321, sum#322] +Keys [4]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286] +Functions [2]: [sum(sales#60), sum(number_sales#61)] +Aggregate Attributes [2]: [sum(sales#60)#324, sum(number_sales#61)#325] +Results [2]: [sum(sales#60)#324 AS sum_sales#117, sum(number_sales#61)#325 AS number_sales#118] + +(206) HashAggregate [codegen id : 658] +Input [2]: [sum_sales#117, number_sales#118] Keys: [] -Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] -Aggregate Attributes [3]: [sum#250, isEmpty#251, sum#252] -Results [3]: [sum#253, isEmpty#254, sum#255] +Functions [2]: [partial_sum(sum_sales#117), partial_sum(number_sales#118)] +Aggregate Attributes [3]: [sum#326, isEmpty#327, sum#328] +Results [3]: [sum#329, isEmpty#330, sum#331] (207) Exchange -Input [3]: [sum#253, isEmpty#254, sum#255] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#256] +Input [3]: [sum#329, isEmpty#330, sum#331] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#332] -(208) HashAggregate [codegen id : 689] -Input [3]: [sum#253, isEmpty#254, sum#255] +(208) HashAggregate [codegen id : 659] +Input [3]: [sum#329, isEmpty#330, sum#331] Keys: [] -Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] -Aggregate Attributes [2]: [sum(sum_sales#93)#257, sum(number_sales#94)#258] -Results [6]: [null AS channel#259, null AS i_brand_id#260, null AS i_class_id#261, null AS i_category_id#262, sum(sum_sales#93)#257 AS sum(sum_sales)#263, sum(number_sales#94)#258 AS sum(number_sales)#264] +Functions [2]: [sum(sum_sales#117), sum(number_sales#118)] +Aggregate Attributes [2]: [sum(sum_sales#117)#333, sum(number_sales#118)#334] +Results [6]: [null AS channel#335, null AS i_brand_id#336, null AS i_class_id#337, null AS i_category_id#338, sum(sum_sales#117)#333 AS sum(sum_sales)#339, sum(number_sales#118)#334 AS sum(number_sales)#340] (209) Union -(210) HashAggregate [codegen id : 690] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] -Keys [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +(210) HashAggregate [codegen id : 660] +Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +Keys [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Results [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] (211) Exchange -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94, 5), ENSURE_REQUIREMENTS, [id=#265] +Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +Arguments: hashpartitioning(channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118, 5), ENSURE_REQUIREMENTS, [id=#341] -(212) HashAggregate [codegen id : 691] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] -Keys [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +(212) HashAggregate [codegen id : 661] +Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +Keys [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Results [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] (213) TakeOrderedAndProject -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] -Arguments: 100, [channel#51 ASC NULLS FIRST, i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +Arguments: 100, [channel#65 ASC NULLS FIRST, i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#49, [id=#50] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#63, [id=#64] * HashAggregate (240) +- Exchange (239) +- * HashAggregate (238) @@ -1243,7 +1243,7 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#266)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#342)] ReadSchema: struct (215) ColumnarToRow [codegen id : 2] @@ -1269,7 +1269,7 @@ Input [2]: [d_date_sk#12, d_year#13] (220) BroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#267] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#343] (221) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#4] @@ -1277,107 +1277,107 @@ Right keys [1]: [d_date_sk#12] Join condition: None (222) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#268, ss_list_price#3 AS list_price#269] +Output [2]: [ss_quantity#2 AS quantity#344, ss_list_price#3 AS list_price#345] Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (223) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Output [3]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#270)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#346)] ReadSchema: struct (224) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Input [3]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] (225) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#347, d_year#348] 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 (226) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#347, d_year#348] (227) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#347, d_year#348] +Condition : (((isnotnull(d_year#348) AND (d_year#348 >= 1998)) AND (d_year#348 <= 2000)) AND isnotnull(d_date_sk#347)) (228) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#347] +Input [2]: [d_date_sk#347, d_year#348] (229) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#271] +Input [1]: [d_date_sk#347] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#349] (230) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#347] Join condition: None (231) Project [codegen id : 4] -Output [2]: [cs_quantity#52 AS quantity#272, cs_list_price#53 AS list_price#273] -Input [4]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] +Output [2]: [cs_quantity#66 AS quantity#350, cs_list_price#67 AS list_price#351] +Input [4]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21, d_date_sk#347] (232) Scan parquet default.web_sales -Output [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#270)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#346)] ReadSchema: struct (233) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] (234) ReusedExchange [Reuses operator id: 229] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#352] (235) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#352] Join condition: None (236) Project [codegen id : 6] -Output [2]: [ws_quantity#68 AS quantity#274, ws_list_price#69 AS list_price#275] -Input [4]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] +Output [2]: [ws_quantity#87 AS quantity#353, ws_list_price#88 AS list_price#354] +Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36, d_date_sk#352] (237) Union (238) HashAggregate [codegen id : 7] -Input [2]: [quantity#268, list_price#269] +Input [2]: [quantity#344, list_price#345] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#276, count#277] -Results [2]: [sum#278, count#279] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#344 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#345 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#355, count#356] +Results [2]: [sum#357, count#358] (239) Exchange -Input [2]: [sum#278, count#279] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#280] +Input [2]: [sum#357, count#358] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#359] (240) HashAggregate [codegen id : 8] -Input [2]: [sum#278, count#279] +Input [2]: [sum#357, count#358] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#281] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#281 AS average_sales#282] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#344 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#345 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#344 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#345 as decimal(12,2)))), DecimalType(18,2), true))#360] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#344 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#345 as decimal(12,2)))), DecimalType(18,2), true))#360 AS average_sales#361] -Subquery:2 Hosting operator id = 214 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#266 +Subquery:2 Hosting operator id = 214 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#342 ReusedExchange (241) (241) ReusedExchange [Reuses operator id: 220] Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 223 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#270 +Subquery:3 Hosting operator id = 223 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#346 ReusedExchange (242) (242) ReusedExchange [Reuses operator id: 229] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#347] -Subquery:4 Hosting operator id = 232 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#270 +Subquery:4 Hosting operator id = 232 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#346 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (243) @@ -1393,40 +1393,40 @@ ReusedExchange (244) (244) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#11 -Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 -Subquery:13 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:13 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:14 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:14 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:15 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:15 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:16 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:16 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:17 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:17 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:18 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:18 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:19 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:19 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:20 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:20 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:21 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:21 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:22 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:22 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:23 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:23 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:24 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:24 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt index d27eb5a32b387..cb186b0ddcd3b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt @@ -1,21 +1,21 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - WholeStageCodegen (691) + WholeStageCodegen (661) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (690) + WholeStageCodegen (660) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union - WholeStageCodegen (137) + WholeStageCodegen (131) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (136) + WholeStageCodegen (130) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (45) + WholeStageCodegen (43) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (44) + WholeStageCodegen (42) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -90,11 +90,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) + WholeStageCodegen (20) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #6 - WholeStageCodegen (20) + WholeStageCodegen (19) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -103,84 +103,82 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #7 - WholeStageCodegen (19) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (18) + WholeStageCodegen (17) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (13) + WholeStageCodegen (12) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #10 - WholeStageCodegen (11) + WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #11 InputAdapter - Exchange [brand_id,class_id,category_id] #11 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #12 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] + BroadcastExchange #11 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #12 + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #11 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (7) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #12 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (17) + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (16) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (16) + WholeStageCodegen (15) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -191,12 +189,12 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #12 + ReusedExchange [d_date_sk] #11 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 InputAdapter BroadcastExchange #5 - WholeStageCodegen (22) + WholeStageCodegen (21) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow @@ -205,27 +203,27 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter BroadcastExchange #17 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #18 - WholeStageCodegen (23) + WholeStageCodegen (22) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 - WholeStageCodegen (90) + WholeStageCodegen (86) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #23 - WholeStageCodegen (89) + WholeStageCodegen (85) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -233,17 +231,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [cs_item_sk,ss_item_sk] - WholeStageCodegen (47) + WholeStageCodegen (45) Sort [cs_item_sk] InputAdapter Exchange [cs_item_sk] #24 - WholeStageCodegen (46) + WholeStageCodegen (44) Filter [cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (66) + WholeStageCodegen (63) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 @@ -251,14 +249,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedExchange [d_date_sk] #5 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (135) + WholeStageCodegen (129) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #25 - WholeStageCodegen (134) + WholeStageCodegen (128) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] @@ -266,17 +264,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ws_item_sk,ss_item_sk] - WholeStageCodegen (92) + WholeStageCodegen (88) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #26 - WholeStageCodegen (91) + WholeStageCodegen (87) Filter [ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (111) + WholeStageCodegen (106) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 @@ -284,136 +282,136 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedExchange [d_date_sk] #5 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (275) + WholeStageCodegen (263) HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id] #27 - WholeStageCodegen (274) + WholeStageCodegen (262) HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #28 - WholeStageCodegen (273) + WholeStageCodegen (261) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (182) + WholeStageCodegen (174) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (227) + WholeStageCodegen (217) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (272) + WholeStageCodegen (260) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (413) + WholeStageCodegen (395) HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id] #32 - WholeStageCodegen (412) + WholeStageCodegen (394) HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #33 - WholeStageCodegen (411) + WholeStageCodegen (393) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (320) + WholeStageCodegen (306) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (365) + WholeStageCodegen (349) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (410) + WholeStageCodegen (392) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (551) + WholeStageCodegen (527) HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel] #34 - WholeStageCodegen (550) + WholeStageCodegen (526) HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #35 - WholeStageCodegen (549) + WholeStageCodegen (525) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (458) + WholeStageCodegen (438) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (503) + WholeStageCodegen (481) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (548) + WholeStageCodegen (524) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (689) + WholeStageCodegen (659) HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange #36 - WholeStageCodegen (688) + WholeStageCodegen (658) HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #37 - WholeStageCodegen (687) + WholeStageCodegen (657) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (596) + WholeStageCodegen (570) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (641) + WholeStageCodegen (613) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (686) + WholeStageCodegen (656) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 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 04fe14cab2c9b..4bd1e57d994ab 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 @@ -34,36 +34,36 @@ TakeOrderedAndProject (194) : : : : :- * HashAggregate (39) : : : : : +- Exchange (38) : : : : : +- * HashAggregate (37) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : : :- * Project (22) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : : :- * Project (15) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : :- * Filter (9) - : : : : : : : : +- * ColumnarToRow (8) - : : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : : +- BroadcastExchange (13) - : : : : : : : +- * Filter (12) - : : : : : : : +- * ColumnarToRow (11) - : : : : : : : +- Scan parquet default.item (10) - : : : : : : +- BroadcastExchange (20) - : : : : : : +- * Project (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet default.date_dim (16) - : : : : : +- BroadcastExchange (35) - : : : : : +- * Project (34) - : : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : : :- * Project (31) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : : :- * Filter (25) - : : : : : : : +- * ColumnarToRow (24) - : : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : : +- BroadcastExchange (29) - : : : : : : +- * Filter (28) - : : : : : : +- * ColumnarToRow (27) - : : : : : : +- Scan parquet default.item (26) - : : : : : +- ReusedExchange (32) + : : : : : +- * Project (36) + : : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : : :- * Project (33) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (31) + : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : : : : : :- * Filter (12) + : : : : : : : +- * ColumnarToRow (11) + : : : : : : : +- Scan parquet default.item (10) + : : : : : : +- BroadcastExchange (29) + : : : : : : +- * Project (28) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * Filter (15) + : : : : : : : : +- * ColumnarToRow (14) + : : : : : : : : +- Scan parquet default.catalog_sales (13) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * Filter (18) + : : : : : : : +- * ColumnarToRow (17) + : : : : : : : +- Scan parquet default.item (16) + : : : : : : +- BroadcastExchange (26) + : : : : : : +- * Project (25) + : : : : : : +- * Filter (24) + : : : : : : +- * ColumnarToRow (23) + : : : : : : +- Scan parquet default.date_dim (22) + : : : : : +- ReusedExchange (34) : : : : +- BroadcastExchange (49) : : : : +- * Project (48) : : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -240,226 +240,226 @@ Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) (10) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(11) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -(12) Filter [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +(12) Filter [codegen id : 4] +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14)) -(13) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +(13) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join condition: None +(14) ColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] -(15) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(15) Filter [codegen id : 3] +Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Condition : isnotnull(cs_item_sk#15) -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +(16) Scan parquet default.item +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] 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 - -(17) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(18) Filter [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +(17) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -(19) Project [codegen id : 2] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +(18) Filter [codegen id : 1] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : isnotnull(i_item_sk#17) -(20) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +(19) BroadcastExchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [i_item_sk#17] Join condition: None -(22) Project [codegen id : 6] -Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] -Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -(23) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(22) Scan parquet default.date_dim +Output [2]: [d_date_sk#22, d_year#23] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +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 -(24) ColumnarToRow [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(23) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#22, d_year#23] -(25) Filter [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +(24) Filter [codegen id : 2] +Input [2]: [d_date_sk#22, d_year#23] +Condition : (((isnotnull(d_year#23) AND (d_year#23 >= 1999)) AND (d_year#23 <= 2001)) AND isnotnull(d_date_sk#22)) -(26) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +(25) Project [codegen id : 2] +Output [1]: [d_date_sk#22] +Input [2]: [d_date_sk#22, d_year#23] -(27) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(26) BroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] -(28) Filter [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +(27) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#22] +Join condition: None + +(28) Project [codegen id : 3] +Output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] +Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#22] (29) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] -(30) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +(30) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)] +Right keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] Join condition: None -(31) Project [codegen id : 5] -Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(32) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(31) BroadcastExchange +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +(32) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#11] Join condition: None -(34) Project [codegen id : 5] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(33) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14] +Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -(35) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] +(34) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#27] -(36) BroadcastHashJoin [codegen id : 6] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#27] Join condition: None +(36) Project [codegen id : 6] +Output [3]: [i_brand_id#12 AS brand_id#28, i_class_id#13 AS class_id#29, i_category_id#14 AS category_id#30] +Input [5]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#27] + (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#28, class_id#29, category_id#30] (38) Exchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [id=#31] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#28, class_id#29, category_id#30] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Output [2]: [ws_item_sk#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Condition : isnotnull(ws_item_sk#32) -(43) ReusedExchange [Reuses operator id: 29] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(43) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#32] +Right keys [1]: [i_item_sk#34] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37] +Input [6]: [ws_item_sk#32, ws_sold_date_sk#33, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] -(46) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(46) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#38] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#38] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +Input [5]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] (49) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#39] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] +Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#28, class_id#29, category_id#30] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#28, class_id#29, category_id#30] (53) BroadcastExchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#40] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#15, class_id#16, category_id#17] +Right keys [3]: [brand_id#28, class_id#29, category_id#30] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#27] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] +Output [1]: [i_item_sk#6 AS ss_item_sk#41] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] (56) BroadcastExchange -Input [1]: [ss_item_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [ss_item_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#41] Join condition: None (58) Scan parquet default.item @@ -477,16 +477,16 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : isnotnull(i_item_sk#6) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#41] (62) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#41] Join condition: None (63) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] @@ -498,623 +498,623 @@ Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_ Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (66) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#30] +Output [3]: [d_date_sk#27, d_year#44, d_moy#45] 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 (67) ColumnarToRow [codegen id : 24] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] +Input [3]: [d_date_sk#27, d_year#44, d_moy#45] (68) Filter [codegen id : 24] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#30)) AND (d_year#13 = 2000)) AND (d_moy#30 = 11)) AND isnotnull(d_date_sk#12)) +Input [3]: [d_date_sk#27, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#27)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] +Output [1]: [d_date_sk#27] +Input [3]: [d_date_sk#27, d_year#44, d_moy#45] (70) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#27] Join condition: None (72) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#27] (73) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#32, isEmpty#33, count#34] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] +Aggregate Attributes [3]: [sum#47, isEmpty#48, count#49] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#53] (75) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] (76) Filter [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] +Output [6]: [store AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] (78) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Output [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Input [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] (80) Filter [codegen id : 51] -Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +Input [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Condition : isnotnull(cs_item_sk#15) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#41] (82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [ss_item_sk#41] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] (84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [i_item_sk#64] Join condition: None (85) Project [codegen id : 51] -Output [6]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_brand_id#65, i_class_id#66, i_category_id#67] +Input [8]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] (86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#68] (87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#68] Join condition: None (88) Project [codegen id : 51] -Output [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [cs_quantity#62, cs_list_price#63, i_brand_id#65, i_class_id#66, i_category_id#67] +Input [7]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_brand_id#65, i_class_id#66, i_category_id#67, d_date_sk#68] (89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#49, isEmpty#50, count#51] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] +Input [5]: [cs_quantity#62, cs_list_price#63, i_brand_id#65, i_class_id#66, i_category_id#67] +Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] +Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] (90) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] +Arguments: hashpartitioning(i_brand_id#65, i_class_id#66, i_category_id#67, 5), ENSURE_REQUIREMENTS, [id=#75] (91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56, count(1)#57] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sales#58, count(1)#57 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] +Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76, count(1)#77] +Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sales#78, count(1)#77 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] (92) Filter [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [catalog AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] +Output [6]: [catalog AS channel#81, i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79] +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] (94) Scan parquet default.web_sales -Output [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Output [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Input [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] (96) Filter [codegen id : 77] -Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Condition : isnotnull(ws_item_sk#32) (97) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#41] (98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [ws_item_sk#32] +Right keys [1]: [ss_item_sk#41] Join condition: None (99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] (100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#32] +Right keys [1]: [i_item_sk#84] Join condition: None (101) Project [codegen id : 77] -Output [6]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_brand_id#85, i_class_id#86, i_category_id#87] +Input [8]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] (102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#88] (103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#88] Join condition: None (104) Project [codegen id : 77] -Output [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#85, i_class_id#86, i_category_id#87] +Input [7]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_brand_id#85, i_class_id#86, i_category_id#87, d_date_sk#88] (105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] +Input [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#85, i_class_id#86, i_category_id#87] +Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#89, isEmpty#90, count#91] +Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] (106) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, 5), ENSURE_REQUIREMENTS, [id=#95] (107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71, count(1)#72] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sales#73, count(1)#72 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96, count(1)#97] +Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sales#98, count(1)#97 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] (108) Filter [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] +Output [6]: [web AS channel#101, i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] (110) Union (111) HashAggregate [codegen id : 79] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#77, isEmpty#78, sum#79] -Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#80, isEmpty#81, sum#82] +Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57] +Keys [4]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] +Aggregate Attributes [3]: [sum#102, isEmpty#103, sum#104] +Results [7]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum#105, isEmpty#106, sum#107] (112) Exchange -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#80, isEmpty#81, sum#82] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [7]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum#105, isEmpty#106, sum#107] +Arguments: hashpartitioning(channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#108] (113) HashAggregate [codegen id : 80] -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#80, isEmpty#81, sum#82] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#84, sum(number_sales#42)#85] -Results [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#41)#84 AS sum_sales#86, sum(number_sales#42)#85 AS number_sales#87] +Input [7]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum#105, isEmpty#106, sum#107] +Keys [4]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#56), sum(number_sales#57)] +Aggregate Attributes [2]: [sum(sales#56)#109, sum(number_sales#57)#110] +Results [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#56)#109 AS sum_sales#111, sum(number_sales#57)#110 AS number_sales#112] (114) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#88, isEmpty#89, count#90] +Output [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sum#116, isEmpty#117, count#118] (115) HashAggregate [codegen id : 106] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#88, isEmpty#89, count#90] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#41, count(1)#92 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93] +Input [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sum#116, isEmpty#117, count#118] +Keys [3]: [i_brand_id#113, i_class_id#114, i_category_id#115] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#121, count(1)#122] +Results [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#121 AS sales#56, count(1)#122 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#121 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123] (116) Filter [codegen id : 106] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (117) Project [codegen id : 106] -Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93] +Output [6]: [store AS channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sales#56, number_sales#57] +Input [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123] (118) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#94, isEmpty#95, count#96] +Output [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] (119) HashAggregate [codegen id : 132] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#94, isEmpty#95, count#96] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#58, count(1)#98 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99] +Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] +Keys [3]: [i_brand_id#124, i_class_id#125, i_category_id#126] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132, count(1)#133] +Results [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sales#78, count(1)#133 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] (120) Filter [codegen id : 132] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (121) Project [codegen id : 132] -Output [6]: [catalog AS channel#100, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99] +Output [6]: [catalog AS channel#81, i_brand_id#124, i_class_id#125, i_category_id#126, sales#78, number_sales#79] +Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] (122) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#101, isEmpty#102, count#103] +Output [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] (123) HashAggregate [codegen id : 158] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#101, isEmpty#102, count#103] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#104, count(1)#105] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sales#73, count(1)#105 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106] +Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] +Keys [3]: [i_brand_id#135, i_class_id#136, i_category_id#137] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143, count(1)#144] +Results [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sales#98, count(1)#144 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] (124) Filter [codegen id : 158] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (125) Project [codegen id : 158] -Output [6]: [web AS channel#107, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106] +Output [6]: [web AS channel#101, i_brand_id#135, i_class_id#136, i_category_id#137, sales#98, number_sales#99] +Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] (126) Union (127) HashAggregate [codegen id : 159] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#108, isEmpty#109, sum#110] -Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#111, isEmpty#112, sum#113] +Input [6]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sales#56, number_sales#57] +Keys [4]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115] +Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] +Aggregate Attributes [3]: [sum#146, isEmpty#147, sum#148] +Results [7]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sum#149, isEmpty#150, sum#151] (128) Exchange -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#111, isEmpty#112, sum#113] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#114] +Input [7]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sum#149, isEmpty#150, sum#151] +Arguments: hashpartitioning(channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, 5), ENSURE_REQUIREMENTS, [id=#152] (129) HashAggregate [codegen id : 160] -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#111, isEmpty#112, sum#113] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#115, sum(number_sales#42)#116] -Results [5]: [channel#46, i_brand_id#7, i_class_id#8, sum(sales#41)#115 AS sum_sales#86, sum(number_sales#42)#116 AS number_sales#87] +Input [7]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sum#149, isEmpty#150, sum#151] +Keys [4]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115] +Functions [2]: [sum(sales#56), sum(number_sales#57)] +Aggregate Attributes [2]: [sum(sales#56)#153, sum(number_sales#57)#154] +Results [5]: [channel#61, i_brand_id#113, i_class_id#114, sum(sales#56)#153 AS sum_sales#111, sum(number_sales#57)#154 AS number_sales#112] (130) HashAggregate [codegen id : 160] -Input [5]: [channel#46, i_brand_id#7, i_class_id#8, sum_sales#86, number_sales#87] -Keys [3]: [channel#46, i_brand_id#7, i_class_id#8] -Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] -Aggregate Attributes [3]: [sum#117, isEmpty#118, sum#119] -Results [6]: [channel#46, i_brand_id#7, i_class_id#8, sum#120, isEmpty#121, sum#122] +Input [5]: [channel#61, i_brand_id#113, i_class_id#114, sum_sales#111, number_sales#112] +Keys [3]: [channel#61, i_brand_id#113, i_class_id#114] +Functions [2]: [partial_sum(sum_sales#111), partial_sum(number_sales#112)] +Aggregate Attributes [3]: [sum#155, isEmpty#156, sum#157] +Results [6]: [channel#61, i_brand_id#113, i_class_id#114, sum#158, isEmpty#159, sum#160] (131) Exchange -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, sum#120, isEmpty#121, sum#122] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, 5), ENSURE_REQUIREMENTS, [id=#123] +Input [6]: [channel#61, i_brand_id#113, i_class_id#114, sum#158, isEmpty#159, sum#160] +Arguments: hashpartitioning(channel#61, i_brand_id#113, i_class_id#114, 5), ENSURE_REQUIREMENTS, [id=#161] (132) HashAggregate [codegen id : 161] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, sum#120, isEmpty#121, sum#122] -Keys [3]: [channel#46, i_brand_id#7, i_class_id#8] -Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] -Aggregate Attributes [2]: [sum(sum_sales#86)#124, sum(number_sales#87)#125] -Results [6]: [channel#46, i_brand_id#7, i_class_id#8, null AS i_category_id#126, sum(sum_sales#86)#124 AS sum(sum_sales)#127, sum(number_sales#87)#125 AS sum(number_sales)#128] +Input [6]: [channel#61, i_brand_id#113, i_class_id#114, sum#158, isEmpty#159, sum#160] +Keys [3]: [channel#61, i_brand_id#113, i_class_id#114] +Functions [2]: [sum(sum_sales#111), sum(number_sales#112)] +Aggregate Attributes [2]: [sum(sum_sales#111)#162, sum(number_sales#112)#163] +Results [6]: [channel#61, i_brand_id#113, i_class_id#114, null AS i_category_id#164, sum(sum_sales#111)#162 AS sum(sum_sales)#165, sum(number_sales#112)#163 AS sum(number_sales)#166] (133) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#129, isEmpty#130, count#131] +Output [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sum#170, isEmpty#171, count#172] (134) HashAggregate [codegen id : 187] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#129, isEmpty#130, count#131] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#132, count(1)#133] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sales#41, count(1)#133 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134] +Input [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sum#170, isEmpty#171, count#172] +Keys [3]: [i_brand_id#167, i_class_id#168, i_category_id#169] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#175, count(1)#176] +Results [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#175 AS sales#56, count(1)#176 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#175 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177] (135) Filter [codegen id : 187] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (136) Project [codegen id : 187] -Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134] +Output [6]: [store AS channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sales#56, number_sales#57] +Input [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177] (137) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#135, isEmpty#136, count#137] +Output [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] (138) HashAggregate [codegen id : 213] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#135, isEmpty#136, count#137] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#138, count(1)#139] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sales#58, count(1)#139 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140] +Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] +Keys [3]: [i_brand_id#178, i_class_id#179, i_category_id#180] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186, count(1)#187] +Results [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sales#78, count(1)#187 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] (139) Filter [codegen id : 213] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (140) Project [codegen id : 213] -Output [6]: [catalog AS channel#141, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140] +Output [6]: [catalog AS channel#81, i_brand_id#178, i_class_id#179, i_category_id#180, sales#78, number_sales#79] +Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] (141) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#142, isEmpty#143, count#144] +Output [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] (142) HashAggregate [codegen id : 239] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#142, isEmpty#143, count#144] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#145, count(1)#146] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sales#73, count(1)#146 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147] +Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] +Keys [3]: [i_brand_id#189, i_class_id#190, i_category_id#191] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197, count(1)#198] +Results [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sales#98, count(1)#198 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] (143) Filter [codegen id : 239] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (144) Project [codegen id : 239] -Output [6]: [web AS channel#148, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147] +Output [6]: [web AS channel#101, i_brand_id#189, i_class_id#190, i_category_id#191, sales#98, number_sales#99] +Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] (145) Union (146) HashAggregate [codegen id : 240] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#149, isEmpty#150, sum#151] -Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] +Input [6]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sales#56, number_sales#57] +Keys [4]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169] +Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] +Aggregate Attributes [3]: [sum#200, isEmpty#201, sum#202] +Results [7]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sum#203, isEmpty#204, sum#205] (147) Exchange -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#155] +Input [7]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sum#203, isEmpty#204, sum#205] +Arguments: hashpartitioning(channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, 5), ENSURE_REQUIREMENTS, [id=#206] (148) HashAggregate [codegen id : 241] -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#156, sum(number_sales#42)#157] -Results [4]: [channel#46, i_brand_id#7, sum(sales#41)#156 AS sum_sales#86, sum(number_sales#42)#157 AS number_sales#87] +Input [7]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sum#203, isEmpty#204, sum#205] +Keys [4]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169] +Functions [2]: [sum(sales#56), sum(number_sales#57)] +Aggregate Attributes [2]: [sum(sales#56)#207, sum(number_sales#57)#208] +Results [4]: [channel#61, i_brand_id#167, sum(sales#56)#207 AS sum_sales#111, sum(number_sales#57)#208 AS number_sales#112] (149) HashAggregate [codegen id : 241] -Input [4]: [channel#46, i_brand_id#7, sum_sales#86, number_sales#87] -Keys [2]: [channel#46, i_brand_id#7] -Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] -Aggregate Attributes [3]: [sum#158, isEmpty#159, sum#160] -Results [5]: [channel#46, i_brand_id#7, sum#161, isEmpty#162, sum#163] +Input [4]: [channel#61, i_brand_id#167, sum_sales#111, number_sales#112] +Keys [2]: [channel#61, i_brand_id#167] +Functions [2]: [partial_sum(sum_sales#111), partial_sum(number_sales#112)] +Aggregate Attributes [3]: [sum#209, isEmpty#210, sum#211] +Results [5]: [channel#61, i_brand_id#167, sum#212, isEmpty#213, sum#214] (150) Exchange -Input [5]: [channel#46, i_brand_id#7, sum#161, isEmpty#162, sum#163] -Arguments: hashpartitioning(channel#46, i_brand_id#7, 5), ENSURE_REQUIREMENTS, [id=#164] +Input [5]: [channel#61, i_brand_id#167, sum#212, isEmpty#213, sum#214] +Arguments: hashpartitioning(channel#61, i_brand_id#167, 5), ENSURE_REQUIREMENTS, [id=#215] (151) HashAggregate [codegen id : 242] -Input [5]: [channel#46, i_brand_id#7, sum#161, isEmpty#162, sum#163] -Keys [2]: [channel#46, i_brand_id#7] -Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] -Aggregate Attributes [2]: [sum(sum_sales#86)#165, sum(number_sales#87)#166] -Results [6]: [channel#46, i_brand_id#7, null AS i_class_id#167, null AS i_category_id#168, sum(sum_sales#86)#165 AS sum(sum_sales)#169, sum(number_sales#87)#166 AS sum(number_sales)#170] +Input [5]: [channel#61, i_brand_id#167, sum#212, isEmpty#213, sum#214] +Keys [2]: [channel#61, i_brand_id#167] +Functions [2]: [sum(sum_sales#111), sum(number_sales#112)] +Aggregate Attributes [2]: [sum(sum_sales#111)#216, sum(number_sales#112)#217] +Results [6]: [channel#61, i_brand_id#167, null AS i_class_id#218, null AS i_category_id#219, sum(sum_sales#111)#216 AS sum(sum_sales)#220, sum(number_sales#112)#217 AS sum(number_sales)#221] (152) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#171, isEmpty#172, count#173] +Output [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sum#225, isEmpty#226, count#227] (153) HashAggregate [codegen id : 268] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#171, isEmpty#172, count#173] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#174, count(1)#175] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sales#41, count(1)#175 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176] +Input [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sum#225, isEmpty#226, count#227] +Keys [3]: [i_brand_id#222, i_class_id#223, i_category_id#224] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#230, count(1)#231] +Results [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sales#56, count(1)#231 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232] (154) Filter [codegen id : 268] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (155) Project [codegen id : 268] -Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176] +Output [6]: [store AS channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sales#56, number_sales#57] +Input [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232] (156) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#177, isEmpty#178, count#179] +Output [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] (157) HashAggregate [codegen id : 294] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#177, isEmpty#178, count#179] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#180, count(1)#181] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sales#58, count(1)#181 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182] +Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] +Keys [3]: [i_brand_id#233, i_class_id#234, i_category_id#235] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241, count(1)#242] +Results [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sales#78, count(1)#242 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] (158) Filter [codegen id : 294] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (159) Project [codegen id : 294] -Output [6]: [catalog AS channel#183, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182] +Output [6]: [catalog AS channel#81, i_brand_id#233, i_class_id#234, i_category_id#235, sales#78, number_sales#79] +Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] (160) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#184, isEmpty#185, count#186] +Output [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] (161) HashAggregate [codegen id : 320] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#184, isEmpty#185, count#186] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#187, count(1)#188] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sales#73, count(1)#188 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189] +Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] +Keys [3]: [i_brand_id#244, i_class_id#245, i_category_id#246] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252, count(1)#253] +Results [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sales#98, count(1)#253 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] (162) Filter [codegen id : 320] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (163) Project [codegen id : 320] -Output [6]: [web AS channel#190, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189] +Output [6]: [web AS channel#101, i_brand_id#244, i_class_id#245, i_category_id#246, sales#98, number_sales#99] +Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] (164) Union (165) HashAggregate [codegen id : 321] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#191, isEmpty#192, sum#193] -Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#194, isEmpty#195, sum#196] +Input [6]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sales#56, number_sales#57] +Keys [4]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224] +Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] +Aggregate Attributes [3]: [sum#255, isEmpty#256, sum#257] +Results [7]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sum#258, isEmpty#259, sum#260] (166) Exchange -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#194, isEmpty#195, sum#196] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#197] +Input [7]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sum#258, isEmpty#259, sum#260] +Arguments: hashpartitioning(channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, 5), ENSURE_REQUIREMENTS, [id=#261] (167) HashAggregate [codegen id : 322] -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#194, isEmpty#195, sum#196] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#198, sum(number_sales#42)#199] -Results [3]: [channel#46, sum(sales#41)#198 AS sum_sales#86, sum(number_sales#42)#199 AS number_sales#87] +Input [7]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sum#258, isEmpty#259, sum#260] +Keys [4]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224] +Functions [2]: [sum(sales#56), sum(number_sales#57)] +Aggregate Attributes [2]: [sum(sales#56)#262, sum(number_sales#57)#263] +Results [3]: [channel#61, sum(sales#56)#262 AS sum_sales#111, sum(number_sales#57)#263 AS number_sales#112] (168) HashAggregate [codegen id : 322] -Input [3]: [channel#46, sum_sales#86, number_sales#87] -Keys [1]: [channel#46] -Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] -Aggregate Attributes [3]: [sum#200, isEmpty#201, sum#202] -Results [4]: [channel#46, sum#203, isEmpty#204, sum#205] +Input [3]: [channel#61, sum_sales#111, number_sales#112] +Keys [1]: [channel#61] +Functions [2]: [partial_sum(sum_sales#111), partial_sum(number_sales#112)] +Aggregate Attributes [3]: [sum#264, isEmpty#265, sum#266] +Results [4]: [channel#61, sum#267, isEmpty#268, sum#269] (169) Exchange -Input [4]: [channel#46, sum#203, isEmpty#204, sum#205] -Arguments: hashpartitioning(channel#46, 5), ENSURE_REQUIREMENTS, [id=#206] +Input [4]: [channel#61, sum#267, isEmpty#268, sum#269] +Arguments: hashpartitioning(channel#61, 5), ENSURE_REQUIREMENTS, [id=#270] (170) HashAggregate [codegen id : 323] -Input [4]: [channel#46, sum#203, isEmpty#204, sum#205] -Keys [1]: [channel#46] -Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] -Aggregate Attributes [2]: [sum(sum_sales#86)#207, sum(number_sales#87)#208] -Results [6]: [channel#46, null AS i_brand_id#209, null AS i_class_id#210, null AS i_category_id#211, sum(sum_sales#86)#207 AS sum(sum_sales)#212, sum(number_sales#87)#208 AS sum(number_sales)#213] +Input [4]: [channel#61, sum#267, isEmpty#268, sum#269] +Keys [1]: [channel#61] +Functions [2]: [sum(sum_sales#111), sum(number_sales#112)] +Aggregate Attributes [2]: [sum(sum_sales#111)#271, sum(number_sales#112)#272] +Results [6]: [channel#61, null AS i_brand_id#273, null AS i_class_id#274, null AS i_category_id#275, sum(sum_sales#111)#271 AS sum(sum_sales)#276, sum(number_sales#112)#272 AS sum(number_sales)#277] (171) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#214, isEmpty#215, count#216] +Output [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sum#281, isEmpty#282, count#283] (172) HashAggregate [codegen id : 349] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#214, isEmpty#215, count#216] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#217, count(1)#218] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#217 AS sales#41, count(1)#218 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#217 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219] +Input [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sum#281, isEmpty#282, count#283] +Keys [3]: [i_brand_id#278, i_class_id#279, i_category_id#280] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#286, count(1)#287] +Results [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#286 AS sales#56, count(1)#287 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#286 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288] (173) Filter [codegen id : 349] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (174) Project [codegen id : 349] -Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219] +Output [6]: [store AS channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sales#56, number_sales#57] +Input [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288] (175) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] +Output [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] (176) HashAggregate [codegen id : 375] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#223, count(1)#224] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sales#58, count(1)#224 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225] +Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] +Keys [3]: [i_brand_id#289, i_class_id#290, i_category_id#291] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297, count(1)#298] +Results [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sales#78, count(1)#298 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] (177) Filter [codegen id : 375] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (178) Project [codegen id : 375] -Output [6]: [catalog AS channel#226, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225] +Output [6]: [catalog AS channel#81, i_brand_id#289, i_class_id#290, i_category_id#291, sales#78, number_sales#79] +Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] (179) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#227, isEmpty#228, count#229] +Output [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] (180) HashAggregate [codegen id : 401] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#227, isEmpty#228, count#229] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#230, count(1)#231] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sales#73, count(1)#231 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232] +Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] +Keys [3]: [i_brand_id#300, i_class_id#301, i_category_id#302] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308, count(1)#309] +Results [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sales#98, count(1)#309 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] (181) Filter [codegen id : 401] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (182) Project [codegen id : 401] -Output [6]: [web AS channel#233, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232] +Output [6]: [web AS channel#101, i_brand_id#300, i_class_id#301, i_category_id#302, sales#98, number_sales#99] +Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] (183) Union (184) HashAggregate [codegen id : 402] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#234, isEmpty#235, sum#236] -Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#237, isEmpty#238, sum#239] +Input [6]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sales#56, number_sales#57] +Keys [4]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280] +Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] +Aggregate Attributes [3]: [sum#311, isEmpty#312, sum#313] +Results [7]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sum#314, isEmpty#315, sum#316] (185) Exchange -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#237, isEmpty#238, sum#239] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#240] +Input [7]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sum#314, isEmpty#315, sum#316] +Arguments: hashpartitioning(channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, 5), ENSURE_REQUIREMENTS, [id=#317] (186) HashAggregate [codegen id : 403] -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#237, isEmpty#238, sum#239] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#241, sum(number_sales#42)#242] -Results [2]: [sum(sales#41)#241 AS sum_sales#86, sum(number_sales#42)#242 AS number_sales#87] +Input [7]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sum#314, isEmpty#315, sum#316] +Keys [4]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280] +Functions [2]: [sum(sales#56), sum(number_sales#57)] +Aggregate Attributes [2]: [sum(sales#56)#318, sum(number_sales#57)#319] +Results [2]: [sum(sales#56)#318 AS sum_sales#111, sum(number_sales#57)#319 AS number_sales#112] (187) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#86, number_sales#87] +Input [2]: [sum_sales#111, number_sales#112] Keys: [] -Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] -Aggregate Attributes [3]: [sum#243, isEmpty#244, sum#245] -Results [3]: [sum#246, isEmpty#247, sum#248] +Functions [2]: [partial_sum(sum_sales#111), partial_sum(number_sales#112)] +Aggregate Attributes [3]: [sum#320, isEmpty#321, sum#322] +Results [3]: [sum#323, isEmpty#324, sum#325] (188) Exchange -Input [3]: [sum#246, isEmpty#247, sum#248] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#249] +Input [3]: [sum#323, isEmpty#324, sum#325] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#326] (189) HashAggregate [codegen id : 404] -Input [3]: [sum#246, isEmpty#247, sum#248] +Input [3]: [sum#323, isEmpty#324, sum#325] Keys: [] -Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] -Aggregate Attributes [2]: [sum(sum_sales#86)#250, sum(number_sales#87)#251] -Results [6]: [null AS channel#252, null AS i_brand_id#253, null AS i_class_id#254, null AS i_category_id#255, sum(sum_sales#86)#250 AS sum(sum_sales)#256, sum(number_sales#87)#251 AS sum(number_sales)#257] +Functions [2]: [sum(sum_sales#111), sum(number_sales#112)] +Aggregate Attributes [2]: [sum(sum_sales#111)#327, sum(number_sales#112)#328] +Results [6]: [null AS channel#329, null AS i_brand_id#330, null AS i_class_id#331, null AS i_category_id#332, sum(sum_sales#111)#327 AS sum(sum_sales)#333, sum(number_sales#112)#328 AS sum(number_sales)#334] (190) Union (191) HashAggregate [codegen id : 405] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] -Keys [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Keys [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Results [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] (192) Exchange -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87, 5), ENSURE_REQUIREMENTS, [id=#258] +Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Arguments: hashpartitioning(channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112, 5), ENSURE_REQUIREMENTS, [id=#335] (193) HashAggregate [codegen id : 406] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] -Keys [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Keys [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Results [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] (194) TakeOrderedAndProject -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] -Arguments: 100, [channel#46 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Arguments: 100, [channel#61 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#59, [id=#60] * HashAggregate (221) +- Exchange (220) +- * HashAggregate (219) @@ -1148,190 +1148,190 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#259)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#336)] ReadSchema: struct (196) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (197) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#27, d_year#44] 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 (198) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#27, d_year#44] (199) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#27, d_year#44] +Condition : (((isnotnull(d_year#44) AND (d_year#44 >= 1999)) AND (d_year#44 <= 2001)) AND isnotnull(d_date_sk#27)) (200) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#27] +Input [2]: [d_date_sk#27, d_year#44] (201) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#260] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#337] (202) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#27] Join condition: None (203) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#261, ss_list_price#3 AS list_price#262] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#2 AS quantity#338, ss_list_price#3 AS list_price#339] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#27] (204) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Output [3]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#263)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#340)] ReadSchema: struct (205) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Input [3]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] (206) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#341, d_year#342] 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 (207) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#341, d_year#342] (208) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#341, d_year#342] +Condition : (((isnotnull(d_year#342) AND (d_year#342 >= 1998)) AND (d_year#342 <= 2000)) AND isnotnull(d_date_sk#341)) (209) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#341] +Input [2]: [d_date_sk#341, d_year#342] (210) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#264] +Input [1]: [d_date_sk#341] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#343] (211) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#341] Join condition: None (212) Project [codegen id : 4] -Output [2]: [cs_quantity#47 AS quantity#265, cs_list_price#48 AS list_price#266] -Input [4]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, d_date_sk#12] +Output [2]: [cs_quantity#62 AS quantity#344, cs_list_price#63 AS list_price#345] +Input [4]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, d_date_sk#341] (213) Scan parquet default.web_sales -Output [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Output [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#263)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#340)] ReadSchema: struct (214) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Input [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] (215) ReusedExchange [Reuses operator id: 210] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#346] (216) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#346] Join condition: None (217) Project [codegen id : 6] -Output [2]: [ws_quantity#62 AS quantity#267, ws_list_price#63 AS list_price#268] -Input [4]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, d_date_sk#12] +Output [2]: [ws_quantity#82 AS quantity#347, ws_list_price#83 AS list_price#348] +Input [4]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, d_date_sk#346] (218) Union (219) HashAggregate [codegen id : 7] -Input [2]: [quantity#261, list_price#262] +Input [2]: [quantity#338, list_price#339] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#269, count#270] -Results [2]: [sum#271, count#272] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#338 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#339 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#349, count#350] +Results [2]: [sum#351, count#352] (220) Exchange -Input [2]: [sum#271, count#272] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#273] +Input [2]: [sum#351, count#352] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#353] (221) HashAggregate [codegen id : 8] -Input [2]: [sum#271, count#272] +Input [2]: [sum#351, count#352] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#274] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#274 AS average_sales#275] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#338 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#339 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#338 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#339 as decimal(12,2)))), DecimalType(18,2), true))#354] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#338 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#339 as decimal(12,2)))), DecimalType(18,2), true))#354 AS average_sales#355] -Subquery:2 Hosting operator id = 195 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#259 +Subquery:2 Hosting operator id = 195 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#336 ReusedExchange (222) (222) ReusedExchange [Reuses operator id: 201] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#27] -Subquery:3 Hosting operator id = 204 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#263 +Subquery:3 Hosting operator id = 204 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#340 ReusedExchange (223) (223) ReusedExchange [Reuses operator id: 210] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#341] -Subquery:4 Hosting operator id = 213 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#263 +Subquery:4 Hosting operator id = 213 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#340 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (224) (224) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#27] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 ReusedExchange (225) -(225) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(225) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#27] -Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#10 -Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 -Subquery:13 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:13 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:14 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:14 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:15 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:15 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:16 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:16 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:17 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:17 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:18 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:18 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:19 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:19 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:20 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:20 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:21 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:21 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:22 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:22 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:23 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:23 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:24 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:24 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index 1634c1e247a12..e351f9e687027 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -104,53 +104,53 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Exchange [brand_id,class_id,category_id] #7 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #11 + BroadcastExchange #10 WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (1) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #8 + BroadcastExchange #8 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter BroadcastExchange #12 WholeStageCodegen (9) From 9052170883783a4097a99805b28a70e870360819 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 9 Mar 2021 22:44:40 +0800 Subject: [PATCH 07/38] fix regression of reuse exchange --- .../analysis/DeduplicateRelations.scala | 44 +- .../approved-plans-v1_4/q32.sf100/explain.txt | 42 +- .../q32.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q32/explain.txt | 86 ++-- .../approved-plans-v1_4/q32/simplified.txt | 2 +- .../approved-plans-v1_4/q33.sf100/explain.txt | 271 +++++------- .../q33.sf100/simplified.txt | 20 +- .../approved-plans-v1_4/q33/explain.txt | 271 +++++------- .../approved-plans-v1_4/q33/simplified.txt | 20 +- .../approved-plans-v1_4/q41.sf100/explain.txt | 48 +-- .../q41.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q41/explain.txt | 48 +-- .../approved-plans-v1_4/q41/simplified.txt | 2 +- .../approved-plans-v1_4/q56.sf100/explain.txt | 273 +++++------- .../q56.sf100/simplified.txt | 20 +- .../approved-plans-v1_4/q56/explain.txt | 273 +++++------- .../approved-plans-v1_4/q56/simplified.txt | 20 +- .../approved-plans-v1_4/q6.sf100/explain.txt | 186 ++++---- .../q6.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q6/explain.txt | 100 ++--- .../approved-plans-v1_4/q6/simplified.txt | 2 +- .../approved-plans-v1_4/q60.sf100/explain.txt | 273 +++++------- .../q60.sf100/simplified.txt | 20 +- .../approved-plans-v1_4/q60/explain.txt | 273 +++++------- .../approved-plans-v1_4/q60/simplified.txt | 20 +- .../approved-plans-v1_4/q83.sf100/explain.txt | 402 ++++++++---------- .../q83.sf100/simplified.txt | 25 +- .../approved-plans-v1_4/q83/explain.txt | 366 +++++++--------- .../approved-plans-v1_4/q83/simplified.txt | 25 +- .../approved-plans-v1_4/q92.sf100/explain.txt | 64 +-- .../q92.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q92/explain.txt | 100 ++--- .../approved-plans-v1_4/q92/simplified.txt | 2 +- .../approved-plans-v2_7/q6.sf100/explain.txt | 186 ++++---- .../q6.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q6/explain.txt | 100 ++--- .../approved-plans-v2_7/q6/simplified.txt | 2 +- 37 files changed, 1534 insertions(+), 2062 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index f479bd0436833..8cadc812af6bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable.ArrayBuffer -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, AttributeSet, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, AttributeSet, NamedExpression, PlanExpression} import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Except, Expand, FlatMapCoGroupsInPandas, FlatMapGroupsInPandas, Generate, Intersect, Join, LogicalPlan, MapInPandas, Project, SerializeFromObject, Union, Window} import org.apache.spark.sql.catalyst.rules.Rule @@ -70,23 +70,39 @@ object DeduplicateRelations extends Rule[LogicalPlan] { (m, Seq(m)) } - case _ if plan.children.nonEmpty => - val newChildren = ArrayBuffer.empty[LogicalPlan] + case plan: LogicalPlan => val relations = ArrayBuffer.empty[MultiInstanceRelation] - for (c <- plan.children) { - val (renewed, collected) = renewDuplicatedRelations(existingRelations ++ relations, c) - newChildren += renewed - relations ++= collected - } + val newPlan = if (plan.children.nonEmpty) { + val newChildren = ArrayBuffer.empty[LogicalPlan] + for (c <- plan.children) { + val (renewed, collected) = renewDuplicatedRelations(existingRelations ++ relations, c) + newChildren += renewed + relations ++= collected + } - if (plan.childrenResolved) { - val attrMap = AttributeMap(plan.children.flatMap(_.output).zip( - newChildren.flatMap(_.output)).filter { case (a1, a2) => a1.exprId != a2.exprId }) - val newPlan = plan.withNewChildren(newChildren).rewriteAttrs(attrMap) - (newPlan, relations) + if (plan.childrenResolved) { + val attrMap = AttributeMap( + plan + .children + .flatMap(_.output).zip(newChildren.flatMap(_.output)) + .filter { case (a1, a2) => a1.exprId != a2.exprId } + ) + plan.withNewChildren(newChildren).rewriteAttrs(attrMap) + } else { + plan.withNewChildren(newChildren) + } } else { - (plan.withNewChildren(newChildren), relations) + plan + } + + val planWithNewSubquery = newPlan.transformExpressions { + case subquery: PlanExpression[LogicalPlan @unchecked] => + val (renewed, collected) = renewDuplicatedRelations( + existingRelations ++ relations, subquery.plan) + relations ++= collected + subquery.withNewPlan(renewed) } + (planWithNewSubquery, relations) case _ => (plan, Nil) } diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt index b70c36db4bc9d..5c886e9f2295d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt @@ -116,63 +116,63 @@ Input [3]: [cs_item_sk#4, sum#13, count#14] Keys [1]: [cs_item_sk#4] Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#5))] Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#5))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4 AS cs_item_sk#4#18] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4] (19) Filter -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4#18] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4] Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) (20) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#4#18] +Right keys [1]: [cs_item_sk#4] Join condition: None (21) Project [codegen id : 4] Output [2]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17] -Input [3]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4#18] +Input [3]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4] (22) BroadcastExchange Input [2]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (23) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] +Output [3]: [cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#6), dynamicpruningexpression(cs_sold_date_sk#6 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#7)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] ReadSchema: struct (24) ColumnarToRow -Input [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] +Input [3]: [cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] (25) Filter -Input [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] -Condition : (isnotnull(cs_item_sk#4) AND isnotnull(cs_ext_discount_amt#5)) +Input [3]: [cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] +Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_ext_discount_amt#20)) (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#4] -Join condition: (cast(cs_ext_discount_amt#5 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) +Right keys [1]: [cs_item_sk#19] +Join condition: (cast(cs_ext_discount_amt#20 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) (27) Project [codegen id : 6] -Output [1]: [cs_sold_date_sk#6] -Input [5]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] +Output [1]: [cs_sold_date_sk#21] +Input [5]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] (28) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 6] -Output [1]: [1 AS excess discount amount #20] -Input [2]: [cs_sold_date_sk#6, d_date_sk#8] +Output [1]: [1 AS excess discount amount #23] +Input [2]: [cs_sold_date_sk#21, d_date_sk#22] (31) CollectLimit -Input [1]: [excess discount amount #20] +Input [1]: [excess discount amount #23] Arguments: 100 ===== Subqueries ===== @@ -184,6 +184,6 @@ ReusedExchange (32) (32) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#6 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt index d885ad3178181..303bdf58604bf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt @@ -18,7 +18,7 @@ CollectLimit InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] Filter [(1.3 * avg(cs_ext_discount_amt))] - HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,sum,count] + HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index 9537689459170..91cdd05907a75 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -79,100 +79,100 @@ Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] (11) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] +Output [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#4)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] +Input [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] (13) Filter [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) +Input [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] +Condition : isnotnull(cs_item_sk#8) (14) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#9] +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#8, d_date#9] +Input [2]: [d_date_sk#11, d_date#12] (16) Filter [codegen id : 2] -Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10983)) AND (d_date#12 <= 11073)) AND isnotnull(d_date_sk#11)) (17) Project [codegen id : 2] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#9] +Output [1]: [d_date_sk#11] +Input [2]: [d_date_sk#11, d_date#12] (18) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] (19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#10] +Right keys [1]: [d_date_sk#11] Join condition: None (20) Project [codegen id : 3] -Output [2]: [cs_item_sk#1, cs_ext_discount_amt#2] -Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#8] +Output [2]: [cs_item_sk#8, cs_ext_discount_amt#9] +Input [4]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10, d_date_sk#11] (21) HashAggregate [codegen id : 3] -Input [2]: [cs_item_sk#1, cs_ext_discount_amt#2] -Keys [1]: [cs_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [2]: [sum#11, count#12] -Results [3]: [cs_item_sk#1, sum#13, count#14] +Input [2]: [cs_item_sk#8, cs_ext_discount_amt#9] +Keys [1]: [cs_item_sk#8] +Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#9))] +Aggregate Attributes [2]: [sum#14, count#15] +Results [3]: [cs_item_sk#8, sum#16, count#17] (22) Exchange -Input [3]: [cs_item_sk#1, sum#13, count#14] -Arguments: hashpartitioning(cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [3]: [cs_item_sk#8, sum#16, count#17] +Arguments: hashpartitioning(cs_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#18] (23) HashAggregate [codegen id : 4] -Input [3]: [cs_item_sk#1, sum#13, count#14] -Keys [1]: [cs_item_sk#1] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#2))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#2))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1 AS cs_item_sk#1#18] +Input [3]: [cs_item_sk#8, sum#16, count#17] +Keys [1]: [cs_item_sk#8] +Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#9))] +Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#9))#19] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#9))#19 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] (24) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] +Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#20) (25) BroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#19] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#21] (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] -Right keys [1]: [cs_item_sk#1#18] -Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) +Right keys [1]: [cs_item_sk#8] +Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#20) (27) Project [codegen id : 6] Output [1]: [cs_sold_date_sk#3] -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] +Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] (28) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 6] -Output [1]: [1 AS excess discount amount #20] -Input [2]: [cs_sold_date_sk#3, d_date_sk#8] +Output [1]: [1 AS excess discount amount #23] +Input [2]: [cs_sold_date_sk#3, d_date_sk#22] (31) CollectLimit -Input [1]: [excess discount amount #20] +Input [1]: [excess discount amount #23] Arguments: 100 ===== Subqueries ===== @@ -182,8 +182,8 @@ ReusedExchange (32) (32) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] -Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index 563281eef37bc..f3dd6d0954046 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -24,7 +24,7 @@ CollectLimit BroadcastExchange #3 WholeStageCodegen (4) Filter [(1.3 * avg(cs_ext_discount_amt))] - HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,sum,count] + HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] InputAdapter Exchange [cs_item_sk] #4 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index 613e56b58c17b..eeda8611876d6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- Union (72) +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- Union (63) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (76) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) + :- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,31 +50,22 @@ TakeOrderedAndProject (76) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- BroadcastExchange (51) - : +- * BroadcastHashJoin LeftSemi BuildRight (50) - : :- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet default.item (42) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- * Filter (47) - : +- * ColumnarToRow (46) - : +- Scan parquet default.item (45) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Filter (59) - : : : +- * ColumnarToRow (58) - : : : +- Scan parquet default.web_sales (57) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- ReusedExchange (66) + : +- ReusedExchange (42) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.web_sales (48) + : : +- ReusedExchange (51) + : +- ReusedExchange (54) + +- ReusedExchange (57) (1) Scan parquet default.store_sales @@ -169,30 +160,30 @@ Input [2]: [i_item_sk#13, i_manufact_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_category#15, i_manufact_id#14] +Output [2]: [i_category#15, i_manufact_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#14] +Input [2]: [i_category#15, i_manufact_id#16] (23) Filter [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#14] +Input [2]: [i_category#15, i_manufact_id#16] Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] -Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] -Input [2]: [i_category#15, i_manufact_id#14] +Output [1]: [i_manufact_id#16] +Input [2]: [i_category#15, i_manufact_id#16] (25) BroadcastExchange -Input [1]: [i_manufact_id#14#16] +Input [1]: [i_manufact_id#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_manufact_id#14] -Right keys [1]: [i_manufact_id#14#16] +Right keys [1]: [i_manufact_id#16] Join condition: None (27) BroadcastExchange @@ -265,182 +256,140 @@ Join condition: None Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) Scan parquet default.item +(42) ReusedExchange [Reuses operator id: 27] Output [2]: [i_item_sk#30, i_manufact_id#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 10] -Input [2]: [i_item_sk#30, i_manufact_id#31] - -(44) Filter [codegen id : 10] -Input [2]: [i_item_sk#30, i_manufact_id#31] -Condition : isnotnull(i_item_sk#30) - -(45) Scan parquet default.item -Output [2]: [i_category#15, i_manufact_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 9] -Input [2]: [i_category#15, i_manufact_id#14] - -(47) Filter [codegen id : 9] -Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) - -(48) Project [codegen id : 9] -Output [1]: [i_manufact_id#14] -Input [2]: [i_category#15, i_manufact_id#14] - -(49) BroadcastExchange -Input [1]: [i_manufact_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [i_manufact_id#31] -Right keys [1]: [i_manufact_id#14] -Join condition: None - -(51) BroadcastExchange -Input [2]: [i_item_sk#30, i_manufact_id#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] -(52) BroadcastHashJoin [codegen id : 11] +(43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] Right keys [1]: [i_item_sk#30] Join condition: None -(53) Project [codegen id : 11] +(44) Project [codegen id : 11] Output [2]: [cs_ext_sales_price#26, i_manufact_id#31] Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_manufact_id#31] -(54) HashAggregate [codegen id : 11] +(45) HashAggregate [codegen id : 11] Input [2]: [cs_ext_sales_price#26, i_manufact_id#31] Keys [1]: [i_manufact_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_manufact_id#31, sum#35] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_manufact_id#31, sum#33] -(55) Exchange -Input [2]: [i_manufact_id#31, sum#35] -Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] +(46) Exchange +Input [2]: [i_manufact_id#31, sum#33] +Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] -(56) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#31, sum#35] +(47) HashAggregate [codegen id : 12] +Input [2]: [i_manufact_id#31, sum#33] Keys [1]: [i_manufact_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] -Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] -(57) Scan parquet default.web_sales -Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(48) Scan parquet default.web_sales +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(49) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -(59) Filter [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) +(50) Filter [codegen id : 17] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) -(60) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#43] +(51) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#41] -(61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#43] +(52) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None -(62) Project [codegen id : 17] -Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] -Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] +(53) Project [codegen id : 17] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] -(63) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#44] +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#42] -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#40] -Right keys [1]: [ca_address_sk#44] +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None -(65) Project [codegen id : 17] -Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] +(56) Project [codegen id : 17] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] -(66) ReusedExchange [Reuses operator id: 51] -Output [2]: [i_item_sk#45, i_manufact_id#46] +(57) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#43, i_manufact_id#44] -(67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [i_item_sk#45] +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None -(68) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#41, i_manufact_id#46] -Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_manufact_id#46] +(59) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#39, i_manufact_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_manufact_id#44] -(69) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#41, i_manufact_id#46] -Keys [1]: [i_manufact_id#46] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum#47] -Results [2]: [i_manufact_id#46, sum#48] +(60) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#39, i_manufact_id#44] +Keys [1]: [i_manufact_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_manufact_id#44, sum#46] -(70) Exchange -Input [2]: [i_manufact_id#46, sum#48] -Arguments: hashpartitioning(i_manufact_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] +(61) Exchange +Input [2]: [i_manufact_id#44, sum#46] +Arguments: hashpartitioning(i_manufact_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] -(71) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#46, sum#48] -Keys [1]: [i_manufact_id#46] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] -Results [2]: [i_manufact_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] +(62) HashAggregate [codegen id : 18] +Input [2]: [i_manufact_id#44, sum#46] +Keys [1]: [i_manufact_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_manufact_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] -(72) Union +(63) Union -(73) HashAggregate [codegen id : 19] +(64) HashAggregate [codegen id : 19] Input [2]: [i_manufact_id#14, total_sales#23] Keys [1]: [i_manufact_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [3]: [i_manufact_id#14, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_manufact_id#14, sum#52, isEmpty#53] -(74) Exchange -Input [3]: [i_manufact_id#14, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] +(65) Exchange +Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] -(75) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#14, sum#54, isEmpty#55] +(66) HashAggregate [codegen id : 20] +Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] Keys [1]: [i_manufact_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#57] -Results [2]: [i_manufact_id#14, sum(total_sales#23)#57 AS total_sales#58] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_manufact_id#14, sum(total_sales#23)#55 AS total_sales#56] -(76) TakeOrderedAndProject -Input [2]: [i_manufact_id#14, total_sales#58] -Arguments: 100, [total_sales#58 ASC NULLS FIRST], [i_manufact_id#14, total_sales#58] +(67) TakeOrderedAndProject +Input [2]: [i_manufact_id#14, total_sales#56] +Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_manufact_id#14, total_sales#56] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (77) +ReusedExchange (68) -(77) ReusedExchange [Reuses operator id: 8] +(68) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt index ad5473e9da920..d1a115fa528d5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt @@ -79,25 +79,11 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_manufact_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_category,i_manufact_id] + ReusedExchange [i_item_sk,i_manufact_id] #5 WholeStageCodegen (18) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_manufact_id] #10 + Exchange [i_manufact_id] #8 WholeStageCodegen (17) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_manufact_id] @@ -116,4 +102,4 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #8 + ReusedExchange [i_item_sk,i_manufact_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index 613e56b58c17b..eeda8611876d6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- Union (72) +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- Union (63) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (76) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) + :- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,31 +50,22 @@ TakeOrderedAndProject (76) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- BroadcastExchange (51) - : +- * BroadcastHashJoin LeftSemi BuildRight (50) - : :- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet default.item (42) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- * Filter (47) - : +- * ColumnarToRow (46) - : +- Scan parquet default.item (45) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Filter (59) - : : : +- * ColumnarToRow (58) - : : : +- Scan parquet default.web_sales (57) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- ReusedExchange (66) + : +- ReusedExchange (42) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.web_sales (48) + : : +- ReusedExchange (51) + : +- ReusedExchange (54) + +- ReusedExchange (57) (1) Scan parquet default.store_sales @@ -169,30 +160,30 @@ Input [2]: [i_item_sk#13, i_manufact_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_category#15, i_manufact_id#14] +Output [2]: [i_category#15, i_manufact_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#14] +Input [2]: [i_category#15, i_manufact_id#16] (23) Filter [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#14] +Input [2]: [i_category#15, i_manufact_id#16] Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] -Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] -Input [2]: [i_category#15, i_manufact_id#14] +Output [1]: [i_manufact_id#16] +Input [2]: [i_category#15, i_manufact_id#16] (25) BroadcastExchange -Input [1]: [i_manufact_id#14#16] +Input [1]: [i_manufact_id#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_manufact_id#14] -Right keys [1]: [i_manufact_id#14#16] +Right keys [1]: [i_manufact_id#16] Join condition: None (27) BroadcastExchange @@ -265,182 +256,140 @@ Join condition: None Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) Scan parquet default.item +(42) ReusedExchange [Reuses operator id: 27] Output [2]: [i_item_sk#30, i_manufact_id#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 10] -Input [2]: [i_item_sk#30, i_manufact_id#31] - -(44) Filter [codegen id : 10] -Input [2]: [i_item_sk#30, i_manufact_id#31] -Condition : isnotnull(i_item_sk#30) - -(45) Scan parquet default.item -Output [2]: [i_category#15, i_manufact_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 9] -Input [2]: [i_category#15, i_manufact_id#14] - -(47) Filter [codegen id : 9] -Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) - -(48) Project [codegen id : 9] -Output [1]: [i_manufact_id#14] -Input [2]: [i_category#15, i_manufact_id#14] - -(49) BroadcastExchange -Input [1]: [i_manufact_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [i_manufact_id#31] -Right keys [1]: [i_manufact_id#14] -Join condition: None - -(51) BroadcastExchange -Input [2]: [i_item_sk#30, i_manufact_id#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] -(52) BroadcastHashJoin [codegen id : 11] +(43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] Right keys [1]: [i_item_sk#30] Join condition: None -(53) Project [codegen id : 11] +(44) Project [codegen id : 11] Output [2]: [cs_ext_sales_price#26, i_manufact_id#31] Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_manufact_id#31] -(54) HashAggregate [codegen id : 11] +(45) HashAggregate [codegen id : 11] Input [2]: [cs_ext_sales_price#26, i_manufact_id#31] Keys [1]: [i_manufact_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_manufact_id#31, sum#35] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_manufact_id#31, sum#33] -(55) Exchange -Input [2]: [i_manufact_id#31, sum#35] -Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] +(46) Exchange +Input [2]: [i_manufact_id#31, sum#33] +Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] -(56) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#31, sum#35] +(47) HashAggregate [codegen id : 12] +Input [2]: [i_manufact_id#31, sum#33] Keys [1]: [i_manufact_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] -Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] -(57) Scan parquet default.web_sales -Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(48) Scan parquet default.web_sales +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(49) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -(59) Filter [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) +(50) Filter [codegen id : 17] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) -(60) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#43] +(51) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#41] -(61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#43] +(52) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None -(62) Project [codegen id : 17] -Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] -Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] +(53) Project [codegen id : 17] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] -(63) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#44] +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#42] -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#40] -Right keys [1]: [ca_address_sk#44] +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None -(65) Project [codegen id : 17] -Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] +(56) Project [codegen id : 17] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] -(66) ReusedExchange [Reuses operator id: 51] -Output [2]: [i_item_sk#45, i_manufact_id#46] +(57) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#43, i_manufact_id#44] -(67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [i_item_sk#45] +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None -(68) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#41, i_manufact_id#46] -Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_manufact_id#46] +(59) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#39, i_manufact_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_manufact_id#44] -(69) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#41, i_manufact_id#46] -Keys [1]: [i_manufact_id#46] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum#47] -Results [2]: [i_manufact_id#46, sum#48] +(60) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#39, i_manufact_id#44] +Keys [1]: [i_manufact_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_manufact_id#44, sum#46] -(70) Exchange -Input [2]: [i_manufact_id#46, sum#48] -Arguments: hashpartitioning(i_manufact_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] +(61) Exchange +Input [2]: [i_manufact_id#44, sum#46] +Arguments: hashpartitioning(i_manufact_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] -(71) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#46, sum#48] -Keys [1]: [i_manufact_id#46] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] -Results [2]: [i_manufact_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] +(62) HashAggregate [codegen id : 18] +Input [2]: [i_manufact_id#44, sum#46] +Keys [1]: [i_manufact_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_manufact_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] -(72) Union +(63) Union -(73) HashAggregate [codegen id : 19] +(64) HashAggregate [codegen id : 19] Input [2]: [i_manufact_id#14, total_sales#23] Keys [1]: [i_manufact_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [3]: [i_manufact_id#14, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_manufact_id#14, sum#52, isEmpty#53] -(74) Exchange -Input [3]: [i_manufact_id#14, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] +(65) Exchange +Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] -(75) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#14, sum#54, isEmpty#55] +(66) HashAggregate [codegen id : 20] +Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] Keys [1]: [i_manufact_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#57] -Results [2]: [i_manufact_id#14, sum(total_sales#23)#57 AS total_sales#58] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_manufact_id#14, sum(total_sales#23)#55 AS total_sales#56] -(76) TakeOrderedAndProject -Input [2]: [i_manufact_id#14, total_sales#58] -Arguments: 100, [total_sales#58 ASC NULLS FIRST], [i_manufact_id#14, total_sales#58] +(67) TakeOrderedAndProject +Input [2]: [i_manufact_id#14, total_sales#56] +Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_manufact_id#14, total_sales#56] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (77) +ReusedExchange (68) -(77) ReusedExchange [Reuses operator id: 8] +(68) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index ad5473e9da920..d1a115fa528d5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -79,25 +79,11 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_manufact_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_category,i_manufact_id] + ReusedExchange [i_item_sk,i_manufact_id] #5 WholeStageCodegen (18) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_manufact_id] #10 + Exchange [i_manufact_id] #8 WholeStageCodegen (17) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_manufact_id] @@ -116,4 +102,4 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #8 + ReusedExchange [i_item_sk,i_manufact_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt index 13d73e61e1443..034c0fe9eb535 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt @@ -40,61 +40,61 @@ Output [2]: [i_manufact#2, i_product_name#3] Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] (5) Scan parquet default.item -Output [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [Or(Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,powder),EqualTo(i_color,khaki)),Or(EqualTo(i_units,Ounce),EqualTo(i_units,Oz))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,brown),EqualTo(i_color,honeydew)),Or(EqualTo(i_units,Bunch),EqualTo(i_units,Ton))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,floral),EqualTo(i_color,deep)),Or(EqualTo(i_units,N/A),EqualTo(i_units,Dozen))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,light),EqualTo(i_color,cornflower)),Or(EqualTo(i_units,Box),EqualTo(i_units,Pound))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large)))))),Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,midnight),EqualTo(i_color,snow)),Or(EqualTo(i_units,Pallet),EqualTo(i_units,Gross))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,cyan),EqualTo(i_color,papaya)),Or(EqualTo(i_units,Cup),EqualTo(i_units,Dram))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,orange),EqualTo(i_color,frosted)),Or(EqualTo(i_units,Each),EqualTo(i_units,Tbl))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,forest),EqualTo(i_color,ghost)),Or(EqualTo(i_units,Lb),EqualTo(i_units,Bundle))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))))))), IsNotNull(i_manufact)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] (7) Filter [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Condition : (((((i_category#4 = Women) AND (((((i_color#6 = powder) OR (i_color#6 = khaki)) AND ((i_units#7 = Ounce) OR (i_units#7 = Oz))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = brown) OR (i_color#6 = honeydew)) AND ((i_units#7 = Bunch) OR (i_units#7 = Ton))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = floral) OR (i_color#6 = deep)) AND ((i_units#7 = N/A) OR (i_units#7 = Dozen))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = light) OR (i_color#6 = cornflower)) AND ((i_units#7 = Box) OR (i_units#7 = Pound))) AND ((i_size#5 = medium) OR (i_size#5 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#6 = midnight) OR (i_color#6 = snow)) AND ((i_units#7 = Pallet) OR (i_units#7 = Gross))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = cyan) OR (i_color#6 = papaya)) AND ((i_units#7 = Cup) OR (i_units#7 = Dram))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = orange) OR (i_color#6 = frosted)) AND ((i_units#7 = Each) OR (i_units#7 = Tbl))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = forest) OR (i_color#6 = ghost)) AND ((i_units#7 = Lb) OR (i_units#7 = Bundle))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))))))) AND isnotnull(i_manufact#2)) +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Condition : (((((i_category#4 = Women) AND (((((i_color#7 = powder) OR (i_color#7 = khaki)) AND ((i_units#8 = Ounce) OR (i_units#8 = Oz))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = brown) OR (i_color#7 = honeydew)) AND ((i_units#8 = Bunch) OR (i_units#8 = Ton))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = floral) OR (i_color#7 = deep)) AND ((i_units#8 = N/A) OR (i_units#8 = Dozen))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = light) OR (i_color#7 = cornflower)) AND ((i_units#8 = Box) OR (i_units#8 = Pound))) AND ((i_size#6 = medium) OR (i_size#6 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#7 = midnight) OR (i_color#7 = snow)) AND ((i_units#8 = Pallet) OR (i_units#8 = Gross))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = cyan) OR (i_color#7 = papaya)) AND ((i_units#8 = Cup) OR (i_units#8 = Dram))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = orange) OR (i_color#7 = frosted)) AND ((i_units#8 = Each) OR (i_units#8 = Tbl))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = forest) OR (i_color#7 = ghost)) AND ((i_units#8 = Lb) OR (i_units#8 = Bundle))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))))))) AND isnotnull(i_manufact#5)) (8) Project [codegen id : 1] -Output [1]: [i_manufact#2] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Output [1]: [i_manufact#5] +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] (9) HashAggregate [codegen id : 1] -Input [1]: [i_manufact#2] -Keys [1]: [i_manufact#2] +Input [1]: [i_manufact#5] +Keys [1]: [i_manufact#5] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#8] -Results [2]: [i_manufact#2, count#9] +Aggregate Attributes [1]: [count#9] +Results [2]: [i_manufact#5, count#10] (10) Exchange -Input [2]: [i_manufact#2, count#9] -Arguments: hashpartitioning(i_manufact#2, 5), true, [id=#10] +Input [2]: [i_manufact#5, count#10] +Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [id=#11] (11) HashAggregate [codegen id : 2] -Input [2]: [i_manufact#2, count#9] -Keys [1]: [i_manufact#2] +Input [2]: [i_manufact#5, count#10] +Keys [1]: [i_manufact#5] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#11] -Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13] +Aggregate Attributes [1]: [count(1)#12] +Results [2]: [count(1)#12 AS item_cnt#13, i_manufact#5] (12) Filter [codegen id : 2] -Input [2]: [item_cnt#12, i_manufact#2#13] -Condition : (item_cnt#12 > 0) +Input [2]: [item_cnt#13, i_manufact#5] +Condition : (item_cnt#13 > 0) (13) Project [codegen id : 2] -Output [1]: [i_manufact#2#13] -Input [2]: [item_cnt#12, i_manufact#2#13] +Output [1]: [i_manufact#5] +Input [2]: [item_cnt#13, i_manufact#5] (14) BroadcastExchange -Input [1]: [i_manufact#2#13] +Input [1]: [i_manufact#5] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14] (15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_manufact#2] -Right keys [1]: [i_manufact#2#13] +Right keys [1]: [i_manufact#5] Join condition: None (16) Project [codegen id : 3] Output [1]: [i_product_name#3] -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#2#13] +Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] (17) HashAggregate [codegen id : 3] Input [1]: [i_product_name#3] @@ -105,7 +105,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] +Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt index 2d14d75ca9362..d36800823bb3f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (2) Project [i_manufact] Filter [item_cnt] - HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,count] + HashAggregate [i_manufact,count] [count(1),item_cnt,count] InputAdapter Exchange [i_manufact] #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index 13d73e61e1443..034c0fe9eb535 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -40,61 +40,61 @@ Output [2]: [i_manufact#2, i_product_name#3] Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] (5) Scan parquet default.item -Output [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [Or(Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,powder),EqualTo(i_color,khaki)),Or(EqualTo(i_units,Ounce),EqualTo(i_units,Oz))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,brown),EqualTo(i_color,honeydew)),Or(EqualTo(i_units,Bunch),EqualTo(i_units,Ton))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,floral),EqualTo(i_color,deep)),Or(EqualTo(i_units,N/A),EqualTo(i_units,Dozen))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,light),EqualTo(i_color,cornflower)),Or(EqualTo(i_units,Box),EqualTo(i_units,Pound))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large)))))),Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,midnight),EqualTo(i_color,snow)),Or(EqualTo(i_units,Pallet),EqualTo(i_units,Gross))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,cyan),EqualTo(i_color,papaya)),Or(EqualTo(i_units,Cup),EqualTo(i_units,Dram))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,orange),EqualTo(i_color,frosted)),Or(EqualTo(i_units,Each),EqualTo(i_units,Tbl))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,forest),EqualTo(i_color,ghost)),Or(EqualTo(i_units,Lb),EqualTo(i_units,Bundle))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))))))), IsNotNull(i_manufact)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] (7) Filter [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Condition : (((((i_category#4 = Women) AND (((((i_color#6 = powder) OR (i_color#6 = khaki)) AND ((i_units#7 = Ounce) OR (i_units#7 = Oz))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = brown) OR (i_color#6 = honeydew)) AND ((i_units#7 = Bunch) OR (i_units#7 = Ton))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = floral) OR (i_color#6 = deep)) AND ((i_units#7 = N/A) OR (i_units#7 = Dozen))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = light) OR (i_color#6 = cornflower)) AND ((i_units#7 = Box) OR (i_units#7 = Pound))) AND ((i_size#5 = medium) OR (i_size#5 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#6 = midnight) OR (i_color#6 = snow)) AND ((i_units#7 = Pallet) OR (i_units#7 = Gross))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = cyan) OR (i_color#6 = papaya)) AND ((i_units#7 = Cup) OR (i_units#7 = Dram))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = orange) OR (i_color#6 = frosted)) AND ((i_units#7 = Each) OR (i_units#7 = Tbl))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = forest) OR (i_color#6 = ghost)) AND ((i_units#7 = Lb) OR (i_units#7 = Bundle))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))))))) AND isnotnull(i_manufact#2)) +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Condition : (((((i_category#4 = Women) AND (((((i_color#7 = powder) OR (i_color#7 = khaki)) AND ((i_units#8 = Ounce) OR (i_units#8 = Oz))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = brown) OR (i_color#7 = honeydew)) AND ((i_units#8 = Bunch) OR (i_units#8 = Ton))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = floral) OR (i_color#7 = deep)) AND ((i_units#8 = N/A) OR (i_units#8 = Dozen))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = light) OR (i_color#7 = cornflower)) AND ((i_units#8 = Box) OR (i_units#8 = Pound))) AND ((i_size#6 = medium) OR (i_size#6 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#7 = midnight) OR (i_color#7 = snow)) AND ((i_units#8 = Pallet) OR (i_units#8 = Gross))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = cyan) OR (i_color#7 = papaya)) AND ((i_units#8 = Cup) OR (i_units#8 = Dram))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = orange) OR (i_color#7 = frosted)) AND ((i_units#8 = Each) OR (i_units#8 = Tbl))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = forest) OR (i_color#7 = ghost)) AND ((i_units#8 = Lb) OR (i_units#8 = Bundle))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))))))) AND isnotnull(i_manufact#5)) (8) Project [codegen id : 1] -Output [1]: [i_manufact#2] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Output [1]: [i_manufact#5] +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] (9) HashAggregate [codegen id : 1] -Input [1]: [i_manufact#2] -Keys [1]: [i_manufact#2] +Input [1]: [i_manufact#5] +Keys [1]: [i_manufact#5] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#8] -Results [2]: [i_manufact#2, count#9] +Aggregate Attributes [1]: [count#9] +Results [2]: [i_manufact#5, count#10] (10) Exchange -Input [2]: [i_manufact#2, count#9] -Arguments: hashpartitioning(i_manufact#2, 5), true, [id=#10] +Input [2]: [i_manufact#5, count#10] +Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [id=#11] (11) HashAggregate [codegen id : 2] -Input [2]: [i_manufact#2, count#9] -Keys [1]: [i_manufact#2] +Input [2]: [i_manufact#5, count#10] +Keys [1]: [i_manufact#5] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#11] -Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13] +Aggregate Attributes [1]: [count(1)#12] +Results [2]: [count(1)#12 AS item_cnt#13, i_manufact#5] (12) Filter [codegen id : 2] -Input [2]: [item_cnt#12, i_manufact#2#13] -Condition : (item_cnt#12 > 0) +Input [2]: [item_cnt#13, i_manufact#5] +Condition : (item_cnt#13 > 0) (13) Project [codegen id : 2] -Output [1]: [i_manufact#2#13] -Input [2]: [item_cnt#12, i_manufact#2#13] +Output [1]: [i_manufact#5] +Input [2]: [item_cnt#13, i_manufact#5] (14) BroadcastExchange -Input [1]: [i_manufact#2#13] +Input [1]: [i_manufact#5] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14] (15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_manufact#2] -Right keys [1]: [i_manufact#2#13] +Right keys [1]: [i_manufact#5] Join condition: None (16) Project [codegen id : 3] Output [1]: [i_product_name#3] -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#2#13] +Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] (17) HashAggregate [codegen id : 3] Input [1]: [i_product_name#3] @@ -105,7 +105,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] +Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index 2d14d75ca9362..d36800823bb3f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (2) Project [i_manufact] Filter [item_cnt] - HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,count] + HashAggregate [i_manufact,count] [count(1),item_cnt,count] InputAdapter Exchange [i_manufact] #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt index 3c6fe13107e04..47eda1483a06c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- Union (72) +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- Union (63) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (76) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) + :- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,31 +50,22 @@ TakeOrderedAndProject (76) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- BroadcastExchange (51) - : +- * BroadcastHashJoin LeftSemi BuildRight (50) - : :- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet default.item (42) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- * Filter (47) - : +- * ColumnarToRow (46) - : +- Scan parquet default.item (45) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Filter (59) - : : : +- * ColumnarToRow (58) - : : : +- Scan parquet default.web_sales (57) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- ReusedExchange (66) + : +- ReusedExchange (42) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.web_sales (48) + : : +- ReusedExchange (51) + : +- ReusedExchange (54) + +- ReusedExchange (57) (1) Scan parquet default.store_sales @@ -169,30 +160,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#14, i_color#15] +Output [2]: [i_item_id#15, i_color#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#14, i_color#15] +Input [2]: [i_item_id#15, i_color#16] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#14, i_color#15] -Condition : i_color#15 IN (slate ,blanched ,burnished ) +Input [2]: [i_item_id#15, i_color#16] +Condition : i_color#16 IN (slate ,blanched ,burnished ) (24) Project [codegen id : 3] -Output [1]: [i_item_id#14 AS i_item_id#14#16] -Input [2]: [i_item_id#14, i_color#15] +Output [1]: [i_item_id#15] +Input [2]: [i_item_id#15, i_color#16] (25) BroadcastExchange -Input [1]: [i_item_id#14#16] +Input [1]: [i_item_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#14#16] +Right keys [1]: [i_item_id#15] Join condition: None (27) BroadcastExchange @@ -265,182 +256,140 @@ Join condition: None Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) Scan parquet default.item +(42) ReusedExchange [Reuses operator id: 27] Output [2]: [i_item_sk#30, i_item_id#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 10] -Input [2]: [i_item_sk#30, i_item_id#31] - -(44) Filter [codegen id : 10] -Input [2]: [i_item_sk#30, i_item_id#31] -Condition : isnotnull(i_item_sk#30) - -(45) Scan parquet default.item -Output [2]: [i_item_id#14, i_color#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_id#14, i_color#15] - -(47) Filter [codegen id : 9] -Input [2]: [i_item_id#14, i_color#15] -Condition : i_color#15 IN (slate ,blanched ,burnished ) - -(48) Project [codegen id : 9] -Output [1]: [i_item_id#14] -Input [2]: [i_item_id#14, i_color#15] - -(49) BroadcastExchange -Input [1]: [i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [i_item_id#31] -Right keys [1]: [i_item_id#14] -Join condition: None - -(51) BroadcastExchange -Input [2]: [i_item_sk#30, i_item_id#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] -(52) BroadcastHashJoin [codegen id : 11] +(43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] Right keys [1]: [i_item_sk#30] Join condition: None -(53) Project [codegen id : 11] +(44) Project [codegen id : 11] Output [2]: [cs_ext_sales_price#26, i_item_id#31] Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] -(54) HashAggregate [codegen id : 11] +(45) HashAggregate [codegen id : 11] Input [2]: [cs_ext_sales_price#26, i_item_id#31] Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#31, sum#35] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#31, sum#33] -(55) Exchange -Input [2]: [i_item_id#31, sum#35] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] +(46) Exchange +Input [2]: [i_item_id#31, sum#33] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] -(56) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#31, sum#35] +(47) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#31, sum#33] Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] -Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] -(57) Scan parquet default.web_sales -Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(48) Scan parquet default.web_sales +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(49) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -(59) Filter [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) +(50) Filter [codegen id : 17] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) -(60) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#43] +(51) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#41] -(61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#43] +(52) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None -(62) Project [codegen id : 17] -Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] -Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] +(53) Project [codegen id : 17] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] -(63) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#44] +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#42] -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#40] -Right keys [1]: [ca_address_sk#44] +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None -(65) Project [codegen id : 17] -Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] +(56) Project [codegen id : 17] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] -(66) ReusedExchange [Reuses operator id: 51] -Output [2]: [i_item_sk#45, i_item_id#46] +(57) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#43, i_item_id#44] -(67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [i_item_sk#45] +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None -(68) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#41, i_item_id#46] -Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_item_id#46] +(59) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#39, i_item_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] -(69) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#41, i_item_id#46] -Keys [1]: [i_item_id#46] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum#47] -Results [2]: [i_item_id#46, sum#48] +(60) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#39, i_item_id#44] +Keys [1]: [i_item_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_item_id#44, sum#46] -(70) Exchange -Input [2]: [i_item_id#46, sum#48] -Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] +(61) Exchange +Input [2]: [i_item_id#44, sum#46] +Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] -(71) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#46, sum#48] -Keys [1]: [i_item_id#46] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] -Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] +(62) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#44, sum#46] +Keys [1]: [i_item_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] -(72) Union +(63) Union -(73) HashAggregate [codegen id : 19] +(64) HashAggregate [codegen id : 19] Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [3]: [i_item_id#14, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_item_id#14, sum#52, isEmpty#53] -(74) Exchange -Input [3]: [i_item_id#14, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] +(65) Exchange +Input [3]: [i_item_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] -(75) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#54, isEmpty#55] +(66) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#57] -Results [2]: [i_item_id#14, sum(total_sales#23)#57 AS total_sales#58] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] -(76) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#58] -Arguments: 100, [total_sales#58 ASC NULLS FIRST], [i_item_id#14, total_sales#58] +(67) TakeOrderedAndProject +Input [2]: [i_item_id#14, total_sales#56] +Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (77) +ReusedExchange (68) -(77) ReusedExchange [Reuses operator id: 8] +(68) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt index 34d320db2bb91..2d21010c26b8d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt @@ -79,25 +79,11 @@ TakeOrderedAndProject [total_sales,i_item_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_item_id] - Filter [i_color] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_id,i_color] + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #10 + Exchange [i_item_id] #8 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -116,4 +102,4 @@ TakeOrderedAndProject [total_sales,i_item_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #8 + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 3c6fe13107e04..47eda1483a06c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- Union (72) +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- Union (63) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (76) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) + :- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,31 +50,22 @@ TakeOrderedAndProject (76) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- BroadcastExchange (51) - : +- * BroadcastHashJoin LeftSemi BuildRight (50) - : :- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet default.item (42) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- * Filter (47) - : +- * ColumnarToRow (46) - : +- Scan parquet default.item (45) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Filter (59) - : : : +- * ColumnarToRow (58) - : : : +- Scan parquet default.web_sales (57) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- ReusedExchange (66) + : +- ReusedExchange (42) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.web_sales (48) + : : +- ReusedExchange (51) + : +- ReusedExchange (54) + +- ReusedExchange (57) (1) Scan parquet default.store_sales @@ -169,30 +160,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#14, i_color#15] +Output [2]: [i_item_id#15, i_color#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#14, i_color#15] +Input [2]: [i_item_id#15, i_color#16] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#14, i_color#15] -Condition : i_color#15 IN (slate ,blanched ,burnished ) +Input [2]: [i_item_id#15, i_color#16] +Condition : i_color#16 IN (slate ,blanched ,burnished ) (24) Project [codegen id : 3] -Output [1]: [i_item_id#14 AS i_item_id#14#16] -Input [2]: [i_item_id#14, i_color#15] +Output [1]: [i_item_id#15] +Input [2]: [i_item_id#15, i_color#16] (25) BroadcastExchange -Input [1]: [i_item_id#14#16] +Input [1]: [i_item_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#14#16] +Right keys [1]: [i_item_id#15] Join condition: None (27) BroadcastExchange @@ -265,182 +256,140 @@ Join condition: None Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) Scan parquet default.item +(42) ReusedExchange [Reuses operator id: 27] Output [2]: [i_item_sk#30, i_item_id#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 10] -Input [2]: [i_item_sk#30, i_item_id#31] - -(44) Filter [codegen id : 10] -Input [2]: [i_item_sk#30, i_item_id#31] -Condition : isnotnull(i_item_sk#30) - -(45) Scan parquet default.item -Output [2]: [i_item_id#14, i_color#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_id#14, i_color#15] - -(47) Filter [codegen id : 9] -Input [2]: [i_item_id#14, i_color#15] -Condition : i_color#15 IN (slate ,blanched ,burnished ) - -(48) Project [codegen id : 9] -Output [1]: [i_item_id#14] -Input [2]: [i_item_id#14, i_color#15] - -(49) BroadcastExchange -Input [1]: [i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [i_item_id#31] -Right keys [1]: [i_item_id#14] -Join condition: None - -(51) BroadcastExchange -Input [2]: [i_item_sk#30, i_item_id#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] -(52) BroadcastHashJoin [codegen id : 11] +(43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] Right keys [1]: [i_item_sk#30] Join condition: None -(53) Project [codegen id : 11] +(44) Project [codegen id : 11] Output [2]: [cs_ext_sales_price#26, i_item_id#31] Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] -(54) HashAggregate [codegen id : 11] +(45) HashAggregate [codegen id : 11] Input [2]: [cs_ext_sales_price#26, i_item_id#31] Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#31, sum#35] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#31, sum#33] -(55) Exchange -Input [2]: [i_item_id#31, sum#35] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] +(46) Exchange +Input [2]: [i_item_id#31, sum#33] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] -(56) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#31, sum#35] +(47) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#31, sum#33] Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] -Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] -(57) Scan parquet default.web_sales -Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(48) Scan parquet default.web_sales +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(49) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -(59) Filter [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) +(50) Filter [codegen id : 17] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) -(60) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#43] +(51) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#41] -(61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#43] +(52) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None -(62) Project [codegen id : 17] -Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] -Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] +(53) Project [codegen id : 17] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] -(63) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#44] +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#42] -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#40] -Right keys [1]: [ca_address_sk#44] +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None -(65) Project [codegen id : 17] -Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] +(56) Project [codegen id : 17] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] -(66) ReusedExchange [Reuses operator id: 51] -Output [2]: [i_item_sk#45, i_item_id#46] +(57) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#43, i_item_id#44] -(67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [i_item_sk#45] +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None -(68) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#41, i_item_id#46] -Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_item_id#46] +(59) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#39, i_item_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] -(69) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#41, i_item_id#46] -Keys [1]: [i_item_id#46] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum#47] -Results [2]: [i_item_id#46, sum#48] +(60) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#39, i_item_id#44] +Keys [1]: [i_item_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_item_id#44, sum#46] -(70) Exchange -Input [2]: [i_item_id#46, sum#48] -Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] +(61) Exchange +Input [2]: [i_item_id#44, sum#46] +Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] -(71) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#46, sum#48] -Keys [1]: [i_item_id#46] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] -Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] +(62) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#44, sum#46] +Keys [1]: [i_item_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] -(72) Union +(63) Union -(73) HashAggregate [codegen id : 19] +(64) HashAggregate [codegen id : 19] Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [3]: [i_item_id#14, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_item_id#14, sum#52, isEmpty#53] -(74) Exchange -Input [3]: [i_item_id#14, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] +(65) Exchange +Input [3]: [i_item_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] -(75) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#54, isEmpty#55] +(66) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#57] -Results [2]: [i_item_id#14, sum(total_sales#23)#57 AS total_sales#58] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] -(76) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#58] -Arguments: 100, [total_sales#58 ASC NULLS FIRST], [i_item_id#14, total_sales#58] +(67) TakeOrderedAndProject +Input [2]: [i_item_id#14, total_sales#56] +Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (77) +ReusedExchange (68) -(77) ReusedExchange [Reuses operator id: 8] +(68) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 34d320db2bb91..2d21010c26b8d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -79,25 +79,11 @@ TakeOrderedAndProject [total_sales,i_item_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_item_id] - Filter [i_color] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_id,i_color] + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #10 + Exchange [i_item_id] #8 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -116,4 +102,4 @@ TakeOrderedAndProject [total_sales,i_item_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #8 + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt index 4c5b5f25108e0..7e82b4d5df296 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt @@ -81,57 +81,57 @@ Input [3]: [i_item_sk#5, i_current_price#6, i_category#7] Condition : (isnotnull(i_current_price#6) AND isnotnull(i_item_sk#5)) (7) Scan parquet default.item -Output [2]: [i_current_price#6, i_category#7] +Output [2]: [i_current_price#8, i_category#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] +Input [2]: [i_current_price#8, i_category#9] (9) Filter [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] -Condition : isnotnull(i_category#7) +Input [2]: [i_current_price#8, i_category#9] +Condition : isnotnull(i_category#9) (10) HashAggregate [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] -Keys [1]: [i_category#7] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#6))] -Aggregate Attributes [2]: [sum#8, count#9] -Results [3]: [i_category#7, sum#10, count#11] +Input [2]: [i_current_price#8, i_category#9] +Keys [1]: [i_category#9] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#8))] +Aggregate Attributes [2]: [sum#10, count#11] +Results [3]: [i_category#9, sum#12, count#13] (11) Exchange -Input [3]: [i_category#7, sum#10, count#11] -Arguments: hashpartitioning(i_category#7, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [3]: [i_category#9, sum#12, count#13] +Arguments: hashpartitioning(i_category#9, 5), ENSURE_REQUIREMENTS, [id=#14] (12) HashAggregate [codegen id : 2] -Input [3]: [i_category#7, sum#10, count#11] -Keys [1]: [i_category#7] -Functions [1]: [avg(UnscaledValue(i_current_price#6))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#6))#13] -Results [2]: [cast((avg(UnscaledValue(i_current_price#6))#13 / 100.0) as decimal(11,6)) AS avg(i_current_price)#14, i_category#7 AS i_category#7#15] +Input [3]: [i_category#9, sum#12, count#13] +Keys [1]: [i_category#9] +Functions [1]: [avg(UnscaledValue(i_current_price#8))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#8))#15] +Results [2]: [cast((avg(UnscaledValue(i_current_price#8))#15 / 100.0) as decimal(11,6)) AS avg(i_current_price)#16, i_category#9] (13) BroadcastExchange -Input [2]: [avg(i_current_price)#14, i_category#7#15] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#16] +Input [2]: [avg(i_current_price)#16, i_category#9] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#17] (14) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_category#7] -Right keys [1]: [i_category#7#15] +Right keys [1]: [i_category#9] Join condition: None (15) Filter [codegen id : 3] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] -Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#14)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] +Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#16)), DecimalType(14,7), true)) (16) Project [codegen id : 3] Output [1]: [i_item_sk#5] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] (17) BroadcastExchange Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (18) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#1] @@ -143,143 +143,143 @@ Output [2]: [ss_customer_sk#2, ss_sold_date_sk#3] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, i_item_sk#5] (20) Scan parquet default.date_dim -Output [2]: [d_date_sk#18, d_month_seq#19] +Output [2]: [d_date_sk#19, d_month_seq#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#18, d_month_seq#19] +Input [2]: [d_date_sk#19, d_month_seq#20] (22) Filter [codegen id : 4] -Input [2]: [d_date_sk#18, d_month_seq#19] -Condition : ((isnotnull(d_month_seq#19) AND (d_month_seq#19 = Subquery scalar-subquery#20, [id=#21])) AND isnotnull(d_date_sk#18)) +Input [2]: [d_date_sk#19, d_month_seq#20] +Condition : ((isnotnull(d_month_seq#20) AND (d_month_seq#20 = Subquery scalar-subquery#21, [id=#22])) AND isnotnull(d_date_sk#19)) (23) Project [codegen id : 4] -Output [1]: [d_date_sk#18] -Input [2]: [d_date_sk#18, d_month_seq#19] +Output [1]: [d_date_sk#19] +Input [2]: [d_date_sk#19, d_month_seq#20] (24) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (25) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (26) Project [codegen id : 5] Output [1]: [ss_customer_sk#2] -Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#18] +Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#19] (27) Exchange Input [1]: [ss_customer_sk#2] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#23] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] (28) Sort [codegen id : 6] Input [1]: [ss_customer_sk#2] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#25, ca_state#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#24, ca_state#25] +Input [2]: [ca_address_sk#25, ca_state#26] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : isnotnull(ca_address_sk#24) +Input [2]: [ca_address_sk#25, ca_state#26] +Condition : isnotnull(ca_address_sk#25) (32) Exchange -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [ca_address_sk#25, ca_state#26] +Arguments: hashpartitioning(ca_address_sk#25, 5), ENSURE_REQUIREMENTS, [id=#27] (33) Sort [codegen id : 8] -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#25, ca_state#26] +Arguments: [ca_address_sk#25 ASC NULLS FIRST], false, 0 (34) Scan parquet default.customer -Output [2]: [c_customer_sk#27, c_current_addr_sk#28] +Output [2]: [c_customer_sk#28, c_current_addr_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 9] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] (36) Filter [codegen id : 9] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Condition : (isnotnull(c_current_addr_sk#28) AND isnotnull(c_customer_sk#27)) +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Condition : (isnotnull(c_current_addr_sk#29) AND isnotnull(c_customer_sk#28)) (37) Exchange -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Arguments: hashpartitioning(c_current_addr_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Arguments: hashpartitioning(c_current_addr_sk#29, 5), ENSURE_REQUIREMENTS, [id=#30] (38) Sort [codegen id : 10] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Arguments: [c_current_addr_sk#28 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Arguments: [c_current_addr_sk#29 ASC NULLS FIRST], false, 0 (39) SortMergeJoin [codegen id : 11] -Left keys [1]: [ca_address_sk#24] -Right keys [1]: [c_current_addr_sk#28] +Left keys [1]: [ca_address_sk#25] +Right keys [1]: [c_current_addr_sk#29] Join condition: None (40) Project [codegen id : 11] -Output [2]: [ca_state#25, c_customer_sk#27] -Input [4]: [ca_address_sk#24, ca_state#25, c_customer_sk#27, c_current_addr_sk#28] +Output [2]: [ca_state#26, c_customer_sk#28] +Input [4]: [ca_address_sk#25, ca_state#26, c_customer_sk#28, c_current_addr_sk#29] (41) Exchange -Input [2]: [ca_state#25, c_customer_sk#27] -Arguments: hashpartitioning(c_customer_sk#27, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [ca_state#26, c_customer_sk#28] +Arguments: hashpartitioning(c_customer_sk#28, 5), ENSURE_REQUIREMENTS, [id=#31] (42) Sort [codegen id : 12] -Input [2]: [ca_state#25, c_customer_sk#27] -Arguments: [c_customer_sk#27 ASC NULLS FIRST], false, 0 +Input [2]: [ca_state#26, c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 13] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#27] +Right keys [1]: [c_customer_sk#28] Join condition: None (44) Project [codegen id : 13] -Output [1]: [ca_state#25] -Input [3]: [ss_customer_sk#2, ca_state#25, c_customer_sk#27] +Output [1]: [ca_state#26] +Input [3]: [ss_customer_sk#2, ca_state#26, c_customer_sk#28] (45) HashAggregate [codegen id : 13] -Input [1]: [ca_state#25] -Keys [1]: [ca_state#25] +Input [1]: [ca_state#26] +Keys [1]: [ca_state#26] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#31] -Results [2]: [ca_state#25, count#32] +Aggregate Attributes [1]: [count#32] +Results [2]: [ca_state#26, count#33] (46) Exchange -Input [2]: [ca_state#25, count#32] -Arguments: hashpartitioning(ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [2]: [ca_state#26, count#33] +Arguments: hashpartitioning(ca_state#26, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 14] -Input [2]: [ca_state#25, count#32] -Keys [1]: [ca_state#25] +Input [2]: [ca_state#26, count#33] +Keys [1]: [ca_state#26] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#34] -Results [3]: [ca_state#25 AS state#35, count(1)#34 AS cnt#36, count(1)#34 AS count(1)#37] +Aggregate Attributes [1]: [count(1)#35] +Results [3]: [ca_state#26 AS state#36, count(1)#35 AS cnt#37, count(1)#35 AS count(1)#38] (48) Filter [codegen id : 14] -Input [3]: [state#35, cnt#36, count(1)#37] -Condition : (count(1)#37 >= 10) +Input [3]: [state#36, cnt#37, count(1)#38] +Condition : (count(1)#38 >= 10) (49) Project [codegen id : 14] -Output [2]: [state#35, cnt#36] -Input [3]: [state#35, cnt#36, count(1)#37] +Output [2]: [state#36, cnt#37] +Input [3]: [state#36, cnt#37, count(1)#38] (50) TakeOrderedAndProject -Input [2]: [state#35, cnt#36] -Arguments: 100, [cnt#36 ASC NULLS FIRST], [state#35, cnt#36] +Input [2]: [state#36, cnt#37] +Arguments: 100, [cnt#37 ASC NULLS FIRST], [state#36, cnt#37] ===== Subqueries ===== @@ -288,9 +288,9 @@ ReusedExchange (51) (51) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#19] -Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#20, [id=#21] +Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#21, [id=#22] * HashAggregate (58) +- Exchange (57) +- * HashAggregate (56) @@ -301,39 +301,39 @@ Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquer (52) Scan parquet default.date_dim -Output [3]: [d_month_seq#19, d_year#38, d_moy#39] +Output [3]: [d_month_seq#39, d_year#40, d_moy#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (53) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] (54) Filter [codegen id : 1] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] -Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] +Condition : (((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2000)) AND (d_moy#41 = 1)) (55) Project [codegen id : 1] -Output [1]: [d_month_seq#19] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] +Output [1]: [d_month_seq#39] +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] (56) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#19] -Keys [1]: [d_month_seq#19] +Input [1]: [d_month_seq#39] +Keys [1]: [d_month_seq#39] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#19] +Results [1]: [d_month_seq#39] (57) Exchange -Input [1]: [d_month_seq#19] -Arguments: hashpartitioning(d_month_seq#19, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [1]: [d_month_seq#39] +Arguments: hashpartitioning(d_month_seq#39, 5), ENSURE_REQUIREMENTS, [id=#42] (58) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#19] -Keys [1]: [d_month_seq#19] +Input [1]: [d_month_seq#39] +Keys [1]: [d_month_seq#39] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#19] +Results [1]: [d_month_seq#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt index a5097742ec146..1cdb7a5df8e6f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt @@ -38,7 +38,7 @@ TakeOrderedAndProject [cnt,state] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] InputAdapter Exchange [i_category] #6 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt index 58fabaa8e44be..0f98039fc0f7f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt @@ -160,57 +160,57 @@ Input [3]: [i_item_sk#16, i_current_price#17, i_category#18] Condition : (isnotnull(i_current_price#17) AND isnotnull(i_item_sk#16)) (26) Scan parquet default.item -Output [2]: [i_current_price#17, i_category#18] +Output [2]: [i_current_price#19, i_category#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (27) ColumnarToRow [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] +Input [2]: [i_current_price#19, i_category#20] (28) Filter [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] -Condition : isnotnull(i_category#18) +Input [2]: [i_current_price#19, i_category#20] +Condition : isnotnull(i_category#20) (29) HashAggregate [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] -Keys [1]: [i_category#18] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] -Aggregate Attributes [2]: [sum#19, count#20] -Results [3]: [i_category#18, sum#21, count#22] +Input [2]: [i_current_price#19, i_category#20] +Keys [1]: [i_category#20] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#19))] +Aggregate Attributes [2]: [sum#21, count#22] +Results [3]: [i_category#20, sum#23, count#24] (30) Exchange -Input [3]: [i_category#18, sum#21, count#22] -Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [i_category#20, sum#23, count#24] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#25] (31) HashAggregate [codegen id : 5] -Input [3]: [i_category#18, sum#21, count#22] -Keys [1]: [i_category#18] -Functions [1]: [avg(UnscaledValue(i_current_price#17))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#17))#24] -Results [2]: [cast((avg(UnscaledValue(i_current_price#17))#24 / 100.0) as decimal(11,6)) AS avg(i_current_price)#25, i_category#18 AS i_category#18#26] +Input [3]: [i_category#20, sum#23, count#24] +Keys [1]: [i_category#20] +Functions [1]: [avg(UnscaledValue(i_current_price#19))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#19))#26] +Results [2]: [cast((avg(UnscaledValue(i_current_price#19))#26 / 100.0) as decimal(11,6)) AS avg(i_current_price)#27, i_category#20] (32) BroadcastExchange -Input [2]: [avg(i_current_price)#25, i_category#18#26] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#27] +Input [2]: [avg(i_current_price)#27, i_category#20] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#28] (33) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_category#18] -Right keys [1]: [i_category#18#26] +Right keys [1]: [i_category#20] Join condition: None (34) Filter [codegen id : 6] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] -Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#25)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] +Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#27)), DecimalType(14,7), true)) (35) Project [codegen id : 6] Output [1]: [i_item_sk#16] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] (36) BroadcastExchange Input [1]: [i_item_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (37) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#6] @@ -225,31 +225,31 @@ Input [3]: [ca_state#2, ss_item_sk#6, i_item_sk#16] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [2]: [ca_state#2, count#30] +Aggregate Attributes [1]: [count#30] +Results [2]: [ca_state#2, count#31] (40) Exchange -Input [2]: [ca_state#2, count#30] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [2]: [ca_state#2, count#31] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#32] (41) HashAggregate [codegen id : 8] -Input [2]: [ca_state#2, count#30] +Input [2]: [ca_state#2, count#31] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [3]: [ca_state#2 AS state#33, count(1)#32 AS cnt#34, count(1)#32 AS count(1)#35] +Aggregate Attributes [1]: [count(1)#33] +Results [3]: [ca_state#2 AS state#34, count(1)#33 AS cnt#35, count(1)#33 AS count(1)#36] (42) Filter [codegen id : 8] -Input [3]: [state#33, cnt#34, count(1)#35] -Condition : (count(1)#35 >= 10) +Input [3]: [state#34, cnt#35, count(1)#36] +Condition : (count(1)#36 >= 10) (43) Project [codegen id : 8] -Output [2]: [state#33, cnt#34] -Input [3]: [state#33, cnt#34, count(1)#35] +Output [2]: [state#34, cnt#35] +Input [3]: [state#34, cnt#35, count(1)#36] (44) TakeOrderedAndProject -Input [2]: [state#33, cnt#34] -Arguments: 100, [cnt#34 ASC NULLS FIRST], [state#33, cnt#34] +Input [2]: [state#34, cnt#35] +Arguments: 100, [cnt#35 ASC NULLS FIRST], [state#34, cnt#35] ===== Subqueries ===== @@ -271,39 +271,39 @@ Subquery:2 Hosting operator id = 18 Hosting Expression = Subquery scalar-subquer (46) Scan parquet default.date_dim -Output [3]: [d_month_seq#12, d_year#36, d_moy#37] +Output [3]: [d_month_seq#37, d_year#38, d_moy#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (47) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] (48) Filter [codegen id : 1] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] -Condition : (((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2000)) AND (d_moy#37 = 1)) +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] +Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) (49) Project [codegen id : 1] -Output [1]: [d_month_seq#12] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] +Output [1]: [d_month_seq#37] +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] (50) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#12] -Keys [1]: [d_month_seq#12] +Input [1]: [d_month_seq#37] +Keys [1]: [d_month_seq#37] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#12] +Results [1]: [d_month_seq#37] (51) Exchange -Input [1]: [d_month_seq#12] -Arguments: hashpartitioning(d_month_seq#12, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [1]: [d_month_seq#37] +Arguments: hashpartitioning(d_month_seq#37, 5), ENSURE_REQUIREMENTS, [id=#40] (52) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#12] -Keys [1]: [d_month_seq#12] +Input [1]: [d_month_seq#37] +Keys [1]: [d_month_seq#37] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#12] +Results [1]: [d_month_seq#37] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index 8b60c2f86a280..d7fad5948f64b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -68,7 +68,7 @@ TakeOrderedAndProject [cnt,state] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] InputAdapter Exchange [i_category] #8 WholeStageCodegen (4) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt index 8774d6d342fb0..49cfd232239aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- Union (72) +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- Union (63) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (76) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) + :- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,31 +50,22 @@ TakeOrderedAndProject (76) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- BroadcastExchange (51) - : +- * BroadcastHashJoin LeftSemi BuildRight (50) - : :- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet default.item (42) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- * Filter (47) - : +- * ColumnarToRow (46) - : +- Scan parquet default.item (45) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Filter (59) - : : : +- * ColumnarToRow (58) - : : : +- Scan parquet default.web_sales (57) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- ReusedExchange (66) + : +- ReusedExchange (42) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.web_sales (48) + : : +- ReusedExchange (51) + : +- ReusedExchange (54) + +- ReusedExchange (57) (1) Scan parquet default.store_sales @@ -169,30 +160,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#14, i_category#15] +Output [2]: [i_item_id#15, i_category#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#14, i_category#15] +Input [2]: [i_item_id#15, i_category#16] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#14, i_category#15] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) +Input [2]: [i_item_id#15, i_category#16] +Condition : (isnotnull(i_category#16) AND (i_category#16 = Music )) (24) Project [codegen id : 3] -Output [1]: [i_item_id#14 AS i_item_id#14#16] -Input [2]: [i_item_id#14, i_category#15] +Output [1]: [i_item_id#15] +Input [2]: [i_item_id#15, i_category#16] (25) BroadcastExchange -Input [1]: [i_item_id#14#16] +Input [1]: [i_item_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#14#16] +Right keys [1]: [i_item_id#15] Join condition: None (27) BroadcastExchange @@ -265,182 +256,140 @@ Join condition: None Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) Scan parquet default.item +(42) ReusedExchange [Reuses operator id: 27] Output [2]: [i_item_sk#30, i_item_id#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 10] -Input [2]: [i_item_sk#30, i_item_id#31] - -(44) Filter [codegen id : 10] -Input [2]: [i_item_sk#30, i_item_id#31] -Condition : isnotnull(i_item_sk#30) - -(45) Scan parquet default.item -Output [2]: [i_item_id#14, i_category#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_id#14, i_category#15] - -(47) Filter [codegen id : 9] -Input [2]: [i_item_id#14, i_category#15] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) - -(48) Project [codegen id : 9] -Output [1]: [i_item_id#14] -Input [2]: [i_item_id#14, i_category#15] - -(49) BroadcastExchange -Input [1]: [i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [i_item_id#31] -Right keys [1]: [i_item_id#14] -Join condition: None - -(51) BroadcastExchange -Input [2]: [i_item_sk#30, i_item_id#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] -(52) BroadcastHashJoin [codegen id : 11] +(43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] Right keys [1]: [i_item_sk#30] Join condition: None -(53) Project [codegen id : 11] +(44) Project [codegen id : 11] Output [2]: [cs_ext_sales_price#26, i_item_id#31] Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] -(54) HashAggregate [codegen id : 11] +(45) HashAggregate [codegen id : 11] Input [2]: [cs_ext_sales_price#26, i_item_id#31] Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#31, sum#35] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#31, sum#33] -(55) Exchange -Input [2]: [i_item_id#31, sum#35] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] +(46) Exchange +Input [2]: [i_item_id#31, sum#33] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] -(56) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#31, sum#35] +(47) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#31, sum#33] Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] -Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] -(57) Scan parquet default.web_sales -Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(48) Scan parquet default.web_sales +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(49) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -(59) Filter [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) +(50) Filter [codegen id : 17] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) -(60) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#43] +(51) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#41] -(61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#43] +(52) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None -(62) Project [codegen id : 17] -Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] -Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] +(53) Project [codegen id : 17] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] -(63) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#44] +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#42] -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#40] -Right keys [1]: [ca_address_sk#44] +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None -(65) Project [codegen id : 17] -Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] +(56) Project [codegen id : 17] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] -(66) ReusedExchange [Reuses operator id: 51] -Output [2]: [i_item_sk#45, i_item_id#46] +(57) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#43, i_item_id#44] -(67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [i_item_sk#45] +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None -(68) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#41, i_item_id#46] -Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_item_id#46] +(59) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#39, i_item_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] -(69) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#41, i_item_id#46] -Keys [1]: [i_item_id#46] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum#47] -Results [2]: [i_item_id#46, sum#48] +(60) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#39, i_item_id#44] +Keys [1]: [i_item_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_item_id#44, sum#46] -(70) Exchange -Input [2]: [i_item_id#46, sum#48] -Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] +(61) Exchange +Input [2]: [i_item_id#44, sum#46] +Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] -(71) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#46, sum#48] -Keys [1]: [i_item_id#46] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] -Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] +(62) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#44, sum#46] +Keys [1]: [i_item_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] -(72) Union +(63) Union -(73) HashAggregate [codegen id : 19] +(64) HashAggregate [codegen id : 19] Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [3]: [i_item_id#14, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_item_id#14, sum#52, isEmpty#53] -(74) Exchange -Input [3]: [i_item_id#14, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] +(65) Exchange +Input [3]: [i_item_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] -(75) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#54, isEmpty#55] +(66) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#57] -Results [2]: [i_item_id#14, sum(total_sales#23)#57 AS total_sales#58] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] -(76) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#58] -Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#58 ASC NULLS FIRST], [i_item_id#14, total_sales#58] +(67) TakeOrderedAndProject +Input [2]: [i_item_id#14, total_sales#56] +Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (77) +ReusedExchange (68) -(77) ReusedExchange [Reuses operator id: 8] +(68) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt index 948e022cd11c8..46775417cbb70 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt @@ -79,25 +79,11 @@ TakeOrderedAndProject [i_item_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_item_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_id,i_category] + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #10 + Exchange [i_item_id] #8 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -116,4 +102,4 @@ TakeOrderedAndProject [i_item_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #8 + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 8774d6d342fb0..49cfd232239aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- Union (72) +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- Union (63) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (76) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) + :- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,31 +50,22 @@ TakeOrderedAndProject (76) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- BroadcastExchange (51) - : +- * BroadcastHashJoin LeftSemi BuildRight (50) - : :- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet default.item (42) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- * Filter (47) - : +- * ColumnarToRow (46) - : +- Scan parquet default.item (45) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Filter (59) - : : : +- * ColumnarToRow (58) - : : : +- Scan parquet default.web_sales (57) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- ReusedExchange (66) + : +- ReusedExchange (42) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.web_sales (48) + : : +- ReusedExchange (51) + : +- ReusedExchange (54) + +- ReusedExchange (57) (1) Scan parquet default.store_sales @@ -169,30 +160,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#14, i_category#15] +Output [2]: [i_item_id#15, i_category#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#14, i_category#15] +Input [2]: [i_item_id#15, i_category#16] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#14, i_category#15] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) +Input [2]: [i_item_id#15, i_category#16] +Condition : (isnotnull(i_category#16) AND (i_category#16 = Music )) (24) Project [codegen id : 3] -Output [1]: [i_item_id#14 AS i_item_id#14#16] -Input [2]: [i_item_id#14, i_category#15] +Output [1]: [i_item_id#15] +Input [2]: [i_item_id#15, i_category#16] (25) BroadcastExchange -Input [1]: [i_item_id#14#16] +Input [1]: [i_item_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#14#16] +Right keys [1]: [i_item_id#15] Join condition: None (27) BroadcastExchange @@ -265,182 +256,140 @@ Join condition: None Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) Scan parquet default.item +(42) ReusedExchange [Reuses operator id: 27] Output [2]: [i_item_sk#30, i_item_id#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 10] -Input [2]: [i_item_sk#30, i_item_id#31] - -(44) Filter [codegen id : 10] -Input [2]: [i_item_sk#30, i_item_id#31] -Condition : isnotnull(i_item_sk#30) - -(45) Scan parquet default.item -Output [2]: [i_item_id#14, i_category#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_id#14, i_category#15] - -(47) Filter [codegen id : 9] -Input [2]: [i_item_id#14, i_category#15] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) - -(48) Project [codegen id : 9] -Output [1]: [i_item_id#14] -Input [2]: [i_item_id#14, i_category#15] - -(49) BroadcastExchange -Input [1]: [i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [i_item_id#31] -Right keys [1]: [i_item_id#14] -Join condition: None - -(51) BroadcastExchange -Input [2]: [i_item_sk#30, i_item_id#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] -(52) BroadcastHashJoin [codegen id : 11] +(43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] Right keys [1]: [i_item_sk#30] Join condition: None -(53) Project [codegen id : 11] +(44) Project [codegen id : 11] Output [2]: [cs_ext_sales_price#26, i_item_id#31] Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] -(54) HashAggregate [codegen id : 11] +(45) HashAggregate [codegen id : 11] Input [2]: [cs_ext_sales_price#26, i_item_id#31] Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#31, sum#35] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#31, sum#33] -(55) Exchange -Input [2]: [i_item_id#31, sum#35] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] +(46) Exchange +Input [2]: [i_item_id#31, sum#33] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] -(56) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#31, sum#35] +(47) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#31, sum#33] Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] -Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] -(57) Scan parquet default.web_sales -Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(48) Scan parquet default.web_sales +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(49) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -(59) Filter [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) +(50) Filter [codegen id : 17] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) -(60) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#43] +(51) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#41] -(61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#43] +(52) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None -(62) Project [codegen id : 17] -Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] -Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] +(53) Project [codegen id : 17] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] -(63) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#44] +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#42] -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#40] -Right keys [1]: [ca_address_sk#44] +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None -(65) Project [codegen id : 17] -Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] +(56) Project [codegen id : 17] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] -(66) ReusedExchange [Reuses operator id: 51] -Output [2]: [i_item_sk#45, i_item_id#46] +(57) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#43, i_item_id#44] -(67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [i_item_sk#45] +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None -(68) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#41, i_item_id#46] -Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_item_id#46] +(59) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#39, i_item_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] -(69) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#41, i_item_id#46] -Keys [1]: [i_item_id#46] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum#47] -Results [2]: [i_item_id#46, sum#48] +(60) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#39, i_item_id#44] +Keys [1]: [i_item_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_item_id#44, sum#46] -(70) Exchange -Input [2]: [i_item_id#46, sum#48] -Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] +(61) Exchange +Input [2]: [i_item_id#44, sum#46] +Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] -(71) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#46, sum#48] -Keys [1]: [i_item_id#46] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] -Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] +(62) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#44, sum#46] +Keys [1]: [i_item_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] -(72) Union +(63) Union -(73) HashAggregate [codegen id : 19] +(64) HashAggregate [codegen id : 19] Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [3]: [i_item_id#14, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_item_id#14, sum#52, isEmpty#53] -(74) Exchange -Input [3]: [i_item_id#14, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] +(65) Exchange +Input [3]: [i_item_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] -(75) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#54, isEmpty#55] +(66) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#57] -Results [2]: [i_item_id#14, sum(total_sales#23)#57 AS total_sales#58] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] -(76) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#58] -Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#58 ASC NULLS FIRST], [i_item_id#14, total_sales#58] +(67) TakeOrderedAndProject +Input [2]: [i_item_id#14, total_sales#56] +Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (77) +ReusedExchange (68) -(77) ReusedExchange [Reuses operator id: 8] +(68) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index 948e022cd11c8..46775417cbb70 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -79,25 +79,11 @@ TakeOrderedAndProject [i_item_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_item_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_id,i_category] + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #10 + Exchange [i_item_id] #8 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -116,4 +102,4 @@ TakeOrderedAndProject [i_item_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #8 + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt index ef2ac135f64eb..a7c71a0668676 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (72) -+- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) +TakeOrderedAndProject (61) ++- * Project (60) + +- * BroadcastHashJoin Inner BuildRight (59) + :- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) : :- * HashAggregate (30) : : +- Exchange (29) : : +- * HashAggregate (28) @@ -34,43 +34,32 @@ TakeOrderedAndProject (72) : : +- * Filter (24) : : +- * ColumnarToRow (23) : : +- Scan parquet default.item (22) - : +- BroadcastExchange (54) - : +- * HashAggregate (53) - : +- Exchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) + : +- BroadcastExchange (43) + : +- * HashAggregate (42) + : +- Exchange (41) + : +- * HashAggregate (40) + : +- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (36) + : : +- * BroadcastHashJoin Inner BuildRight (35) : : :- * Filter (33) : : : +- * ColumnarToRow (32) : : : +- Scan parquet default.catalog_returns (31) - : : +- BroadcastExchange (45) - : : +- * Project (44) - : : +- * BroadcastHashJoin LeftSemi BuildRight (43) - : : :- * Filter (36) - : : : +- * ColumnarToRow (35) - : : : +- Scan parquet default.date_dim (34) - : : +- BroadcastExchange (42) - : : +- * Project (41) - : : +- * BroadcastHashJoin LeftSemi BuildRight (40) - : : :- * ColumnarToRow (38) - : : : +- Scan parquet default.date_dim (37) - : : +- ReusedExchange (39) - : +- ReusedExchange (48) - +- BroadcastExchange (69) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * Project (65) - +- * BroadcastHashJoin Inner BuildRight (64) - :- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Filter (59) - : : +- * ColumnarToRow (58) - : : +- Scan parquet default.web_returns (57) - : +- ReusedExchange (60) - +- ReusedExchange (63) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) + +- BroadcastExchange (58) + +- * HashAggregate (57) + +- Exchange (56) + +- * HashAggregate (55) + +- * Project (54) + +- * BroadcastHashJoin Inner BuildRight (53) + :- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Filter (48) + : : +- * ColumnarToRow (47) + : : +- Scan parquet default.web_returns (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet default.store_returns @@ -103,51 +92,51 @@ Input [2]: [d_date_sk#4, d_date#5] Condition : isnotnull(d_date_sk#4) (7) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#6, d_week_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (8) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#6, d_week_seq#7] (9) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (10) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#8, d_week_seq#9] (11) Filter [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] -Condition : cast(d_date#5 as string) IN (2000-06-30,2000-09-27,2000-11-17) +Input [2]: [d_date#8, d_week_seq#9] +Condition : cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) (12) Project [codegen id : 1] -Output [1]: [d_week_seq#6 AS d_week_seq#6#7] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_week_seq#9] +Input [2]: [d_date#8, d_week_seq#9] (13) BroadcastExchange -Input [1]: [d_week_seq#6#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] +Input [1]: [d_week_seq#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] (14) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_week_seq#6] -Right keys [1]: [d_week_seq#6#7] +Left keys [1]: [d_week_seq#7] +Right keys [1]: [d_week_seq#9] Join condition: None (15) Project [codegen id : 2] -Output [1]: [d_date#5 AS d_date#5#9] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_date#6] +Input [2]: [d_date#6, d_week_seq#7] (16) BroadcastExchange -Input [1]: [d_date#5#9] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] +Input [1]: [d_date#6] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#11] (17) BroadcastHashJoin [codegen id : 3] Left keys [1]: [d_date#5] -Right keys [1]: [d_date#5#9] +Right keys [1]: [d_date#6] Join condition: None (18) Project [codegen id : 3] @@ -156,7 +145,7 @@ Input [2]: [d_date_sk#4, d_date#5] (19) BroadcastExchange Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (20) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] @@ -168,240 +157,191 @@ Output [2]: [sr_item_sk#1, sr_return_quantity#2] Input [4]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, d_date_sk#4] (22) Scan parquet default.item -Output [2]: [i_item_sk#12, i_item_id#13] +Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (23) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#12, i_item_id#13] +Input [2]: [i_item_sk#13, i_item_id#14] (24) Filter [codegen id : 4] -Input [2]: [i_item_sk#12, i_item_id#13] -Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) +Input [2]: [i_item_sk#13, i_item_id#14] +Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) (25) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] +Input [2]: [i_item_sk#13, i_item_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] (26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_item_sk#1] -Right keys [1]: [cast(i_item_sk#12 as bigint)] +Right keys [1]: [cast(i_item_sk#13 as bigint)] Join condition: None (27) Project [codegen id : 5] -Output [2]: [sr_return_quantity#2, i_item_id#13] -Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#12, i_item_id#13] +Output [2]: [sr_return_quantity#2, i_item_id#14] +Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#13, i_item_id#14] (28) HashAggregate [codegen id : 5] -Input [2]: [sr_return_quantity#2, i_item_id#13] -Keys [1]: [i_item_id#13] +Input [2]: [sr_return_quantity#2, i_item_id#14] +Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#13, sum#16] +Aggregate Attributes [1]: [sum#16] +Results [2]: [i_item_id#14, sum#17] (29) Exchange -Input [2]: [i_item_id#13, sum#16] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [2]: [i_item_id#14, sum#17] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#18] (30) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#13, sum#16] -Keys [1]: [i_item_id#13] +Input [2]: [i_item_id#14, sum#17] +Keys [1]: [i_item_id#14] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#18] -Results [2]: [i_item_id#13 AS item_id#19, sum(sr_return_quantity#2)#18 AS sr_item_qty#20] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] +Results [2]: [i_item_id#14 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] (31) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] (33) Filter [codegen id : 10] -Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] -Condition : isnotnull(cr_item_sk#21) +Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +Condition : isnotnull(cr_item_sk#22) -(34) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_date#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) ColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#24, d_date#25] - -(36) Filter [codegen id : 8] -Input [2]: [d_date_sk#24, d_date#25] -Condition : isnotnull(d_date_sk#24) - -(37) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 7] -Input [2]: [d_date#5, d_week_seq#6] - -(39) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_week_seq#6#26] - -(40) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [d_week_seq#6] -Right keys [1]: [d_week_seq#6#26] -Join condition: None - -(41) Project [codegen id : 7] -Output [1]: [d_date#5] -Input [2]: [d_date#5, d_week_seq#6] - -(42) BroadcastExchange -Input [1]: [d_date#5] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#27] - -(43) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [d_date#25] -Right keys [1]: [d_date#5] -Join condition: None - -(44) Project [codegen id : 8] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_date#25] - -(45) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +(34) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#25] -(46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#23] -Right keys [1]: [d_date_sk#24] +(35) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_returned_date_sk#24] +Right keys [1]: [d_date_sk#25] Join condition: None -(47) Project [codegen id : 10] -Output [2]: [cr_item_sk#21, cr_return_quantity#22] -Input [4]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, d_date_sk#24] +(36) Project [codegen id : 10] +Output [2]: [cr_item_sk#22, cr_return_quantity#23] +Input [4]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, d_date_sk#25] -(48) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#29, i_item_id#30] +(37) ReusedExchange [Reuses operator id: 25] +Output [2]: [i_item_sk#26, i_item_id#27] -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#21] -Right keys [1]: [i_item_sk#29] +(38) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_item_sk#22] +Right keys [1]: [i_item_sk#26] Join condition: None -(50) Project [codegen id : 10] -Output [2]: [cr_return_quantity#22, i_item_id#30] -Input [4]: [cr_item_sk#21, cr_return_quantity#22, i_item_sk#29, i_item_id#30] - -(51) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#22, i_item_id#30] -Keys [1]: [i_item_id#30] -Functions [1]: [partial_sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum#31] -Results [2]: [i_item_id#30, sum#32] - -(52) Exchange -Input [2]: [i_item_id#30, sum#32] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [id=#33] - -(53) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#30, sum#32] -Keys [1]: [i_item_id#30] -Functions [1]: [sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum(cr_return_quantity#22)#34] -Results [2]: [i_item_id#30 AS item_id#35, sum(cr_return_quantity#22)#34 AS cr_item_qty#36] - -(54) BroadcastExchange -Input [2]: [item_id#35, cr_item_qty#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#37] - -(55) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#35] +(39) Project [codegen id : 10] +Output [2]: [cr_return_quantity#23, i_item_id#27] +Input [4]: [cr_item_sk#22, cr_return_quantity#23, i_item_sk#26, i_item_id#27] + +(40) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#23, i_item_id#27] +Keys [1]: [i_item_id#27] +Functions [1]: [partial_sum(cr_return_quantity#23)] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_item_id#27, sum#29] + +(41) Exchange +Input [2]: [i_item_id#27, sum#29] +Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, [id=#30] + +(42) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#27, sum#29] +Keys [1]: [i_item_id#27] +Functions [1]: [sum(cr_return_quantity#23)] +Aggregate Attributes [1]: [sum(cr_return_quantity#23)#31] +Results [2]: [i_item_id#27 AS item_id#32, sum(cr_return_quantity#23)#31 AS cr_item_qty#33] + +(43) BroadcastExchange +Input [2]: [item_id#32, cr_item_qty#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] + +(44) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#32] Join condition: None -(56) Project [codegen id : 18] -Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#36] -Input [4]: [item_id#19, sr_item_qty#20, item_id#35, cr_item_qty#36] +(45) Project [codegen id : 18] +Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#33] +Input [4]: [item_id#20, sr_item_qty#21, item_id#32, cr_item_qty#33] -(57) Scan parquet default.web_returns -Output [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] +(46) Scan parquet default.web_returns +Output [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#40), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(wr_returned_date_sk#37), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] +(47) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] -(59) Filter [codegen id : 16] -Input [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] -Condition : isnotnull(wr_item_sk#38) +(48) Filter [codegen id : 16] +Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +Condition : isnotnull(wr_item_sk#35) -(60) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#41] +(49) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#38] -(61) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#40] -Right keys [1]: [cast(d_date_sk#41 as bigint)] +(50) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#37] +Right keys [1]: [cast(d_date_sk#38 as bigint)] Join condition: None -(62) Project [codegen id : 16] -Output [2]: [wr_item_sk#38, wr_return_quantity#39] -Input [4]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40, d_date_sk#41] +(51) Project [codegen id : 16] +Output [2]: [wr_item_sk#35, wr_return_quantity#36] +Input [4]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37, d_date_sk#38] -(63) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#42, i_item_id#43] +(52) ReusedExchange [Reuses operator id: 25] +Output [2]: [i_item_sk#39, i_item_id#40] -(64) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#38] -Right keys [1]: [cast(i_item_sk#42 as bigint)] +(53) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#35] +Right keys [1]: [cast(i_item_sk#39 as bigint)] Join condition: None -(65) Project [codegen id : 16] -Output [2]: [wr_return_quantity#39, i_item_id#43] -Input [4]: [wr_item_sk#38, wr_return_quantity#39, i_item_sk#42, i_item_id#43] - -(66) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#39, i_item_id#43] -Keys [1]: [i_item_id#43] -Functions [1]: [partial_sum(wr_return_quantity#39)] -Aggregate Attributes [1]: [sum#44] -Results [2]: [i_item_id#43, sum#45] - -(67) Exchange -Input [2]: [i_item_id#43, sum#45] -Arguments: hashpartitioning(i_item_id#43, 5), ENSURE_REQUIREMENTS, [id=#46] - -(68) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#43, sum#45] -Keys [1]: [i_item_id#43] -Functions [1]: [sum(wr_return_quantity#39)] -Aggregate Attributes [1]: [sum(wr_return_quantity#39)#47] -Results [2]: [i_item_id#43 AS item_id#48, sum(wr_return_quantity#39)#47 AS wr_item_qty#49] - -(69) BroadcastExchange -Input [2]: [item_id#48, wr_item_qty#49] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] - -(70) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#48] +(54) Project [codegen id : 16] +Output [2]: [wr_return_quantity#36, i_item_id#40] +Input [4]: [wr_item_sk#35, wr_return_quantity#36, i_item_sk#39, i_item_id#40] + +(55) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#36, i_item_id#40] +Keys [1]: [i_item_id#40] +Functions [1]: [partial_sum(wr_return_quantity#36)] +Aggregate Attributes [1]: [sum#41] +Results [2]: [i_item_id#40, sum#42] + +(56) Exchange +Input [2]: [i_item_id#40, sum#42] +Arguments: hashpartitioning(i_item_id#40, 5), ENSURE_REQUIREMENTS, [id=#43] + +(57) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#40, sum#42] +Keys [1]: [i_item_id#40] +Functions [1]: [sum(wr_return_quantity#36)] +Aggregate Attributes [1]: [sum(wr_return_quantity#36)#44] +Results [2]: [i_item_id#40 AS item_id#45, sum(wr_return_quantity#36)#44 AS wr_item_qty#46] + +(58) BroadcastExchange +Input [2]: [item_id#45, wr_item_qty#46] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#47] + +(59) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#45] Join condition: None -(71) Project [codegen id : 18] -Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS sr_dev#51, cr_item_qty#36, (((cast(cr_item_qty#36 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS cr_dev#52, wr_item_qty#49, (((cast(wr_item_qty#49 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS wr_dev#53, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#54] -Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#36, item_id#48, wr_item_qty#49] +(60) Project [codegen id : 18] +Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS sr_dev#48, cr_item_qty#33, (((cast(cr_item_qty#33 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS cr_dev#49, wr_item_qty#46, (((cast(wr_item_qty#46 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS wr_dev#50, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#51] +Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#33, item_id#45, wr_item_qty#46] -(72) TakeOrderedAndProject -Input [8]: [item_id#19, sr_item_qty#20, sr_dev#51, cr_item_qty#36, cr_dev#52, wr_item_qty#49, wr_dev#53, average#54] -Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#51, cr_item_qty#36, cr_dev#52, wr_item_qty#49, wr_dev#53, average#54] +(61) TakeOrderedAndProject +Input [8]: [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] +Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt index 819d571695a83..b46131afd5ec7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt @@ -66,32 +66,15 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - Project [d_date] - BroadcastHashJoin [d_week_seq,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - InputAdapter - ReusedExchange [d_week_seq] #4 + ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 InputAdapter - BroadcastExchange #10 + BroadcastExchange #8 WholeStageCodegen (17) HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] InputAdapter - Exchange [i_item_id] #11 + Exchange [i_item_id] #9 WholeStageCodegen (16) HashAggregate [i_item_id,wr_return_quantity] [sum,sum] Project [wr_return_quantity,i_item_id] @@ -103,6 +86,6 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] InputAdapter - ReusedExchange [d_date_sk] #8 + ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index 8b14a282c2671..fd73cee998f71 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (72) -+- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) +TakeOrderedAndProject (61) ++- * Project (60) + +- * BroadcastHashJoin Inner BuildRight (59) + :- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) : :- * HashAggregate (30) : : +- Exchange (29) : : +- * HashAggregate (28) @@ -34,43 +34,32 @@ TakeOrderedAndProject (72) : : +- * Filter (17) : : +- * ColumnarToRow (16) : : +- Scan parquet default.date_dim (15) - : +- BroadcastExchange (54) - : +- * HashAggregate (53) - : +- Exchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) + : +- BroadcastExchange (43) + : +- * HashAggregate (42) + : +- Exchange (41) + : +- * HashAggregate (40) + : +- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) : :- * Project (36) : : +- * BroadcastHashJoin Inner BuildRight (35) : : :- * Filter (33) : : : +- * ColumnarToRow (32) : : : +- Scan parquet default.catalog_returns (31) : : +- ReusedExchange (34) - : +- BroadcastExchange (48) - : +- * Project (47) - : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : :- * Filter (39) - : : +- * ColumnarToRow (38) - : : +- Scan parquet default.date_dim (37) - : +- BroadcastExchange (45) - : +- * Project (44) - : +- * BroadcastHashJoin LeftSemi BuildRight (43) - : :- * ColumnarToRow (41) - : : +- Scan parquet default.date_dim (40) - : +- ReusedExchange (42) - +- BroadcastExchange (69) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * Project (65) - +- * BroadcastHashJoin Inner BuildRight (64) - :- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Filter (59) - : : +- * ColumnarToRow (58) - : : +- Scan parquet default.web_returns (57) - : +- ReusedExchange (60) - +- ReusedExchange (63) + : +- ReusedExchange (37) + +- BroadcastExchange (58) + +- * HashAggregate (57) + +- Exchange (56) + +- * HashAggregate (55) + +- * Project (54) + +- * BroadcastHashJoin Inner BuildRight (53) + :- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Filter (48) + : : +- * ColumnarToRow (47) + : : +- Scan parquet default.web_returns (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet default.store_returns @@ -130,51 +119,51 @@ Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) (13) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#9, d_week_seq#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#9, d_week_seq#10] (15) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#11, d_week_seq#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (16) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#11, d_week_seq#12] (17) Filter [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] -Condition : cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) +Input [2]: [d_date#11, d_week_seq#12] +Condition : cast(d_date#11 as string) IN (2000-06-30,2000-09-27,2000-11-17) (18) Project [codegen id : 2] -Output [1]: [d_week_seq#9 AS d_week_seq#9#10] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_week_seq#12] +Input [2]: [d_date#11, d_week_seq#12] (19) BroadcastExchange -Input [1]: [d_week_seq#9#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] +Input [1]: [d_week_seq#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] (20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_week_seq#9] -Right keys [1]: [d_week_seq#9#10] +Left keys [1]: [d_week_seq#10] +Right keys [1]: [d_week_seq#12] Join condition: None (21) Project [codegen id : 3] -Output [1]: [d_date#8 AS d_date#8#12] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_date#9] +Input [2]: [d_date#9, d_week_seq#10] (22) BroadcastExchange -Input [1]: [d_date#8#12] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] +Input [1]: [d_date#9] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#14] (23) BroadcastHashJoin [codegen id : 4] Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#12] +Right keys [1]: [d_date#9] Join condition: None (24) Project [codegen id : 4] @@ -183,7 +172,7 @@ Input [2]: [d_date_sk#7, d_date#8] (25) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] @@ -198,210 +187,161 @@ Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#5, d_date_sk# Input [2]: [sr_return_quantity#2, i_item_id#5] Keys [1]: [i_item_id#5] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#5, sum#16] +Aggregate Attributes [1]: [sum#16] +Results [2]: [i_item_id#5, sum#17] (29) Exchange -Input [2]: [i_item_id#5, sum#16] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [2]: [i_item_id#5, sum#17] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#18] (30) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#5, sum#16] +Input [2]: [i_item_id#5, sum#17] Keys [1]: [i_item_id#5] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#18] -Results [2]: [i_item_id#5 AS item_id#19, sum(sr_return_quantity#2)#18 AS sr_item_qty#20] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] +Results [2]: [i_item_id#5 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] (31) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] (33) Filter [codegen id : 10] -Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] -Condition : isnotnull(cr_item_sk#21) +Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +Condition : isnotnull(cr_item_sk#22) (34) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#24, i_item_id#25] +Output [2]: [i_item_sk#25, i_item_id#26] (35) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#21] -Right keys [1]: [i_item_sk#24] +Left keys [1]: [cr_item_sk#22] +Right keys [1]: [i_item_sk#25] Join condition: None (36) Project [codegen id : 10] -Output [3]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#25] -Input [5]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, i_item_sk#24, i_item_id#25] +Output [3]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#26] +Input [5]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, i_item_sk#25, i_item_id#26] -(37) Scan parquet default.date_dim -Output [2]: [d_date_sk#26, d_date#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#26, d_date#27] - -(39) Filter [codegen id : 9] -Input [2]: [d_date_sk#26, d_date#27] -Condition : isnotnull(d_date_sk#26) - -(40) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 8] -Input [2]: [d_date#8, d_week_seq#9] - -(42) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_week_seq#9#28] - -(43) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [d_week_seq#9] -Right keys [1]: [d_week_seq#9#28] -Join condition: None - -(44) Project [codegen id : 8] -Output [1]: [d_date#8] -Input [2]: [d_date#8, d_week_seq#9] - -(45) BroadcastExchange -Input [1]: [d_date#8] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#29] - -(46) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_date#27] -Right keys [1]: [d_date#8] -Join condition: None - -(47) Project [codegen id : 9] -Output [1]: [d_date_sk#26] -Input [2]: [d_date_sk#26, d_date#27] - -(48) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] +(37) ReusedExchange [Reuses operator id: 25] +Output [1]: [d_date_sk#27] -(49) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#23] -Right keys [1]: [d_date_sk#26] +(38) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_returned_date_sk#24] +Right keys [1]: [d_date_sk#27] Join condition: None -(50) Project [codegen id : 10] -Output [2]: [cr_return_quantity#22, i_item_id#25] -Input [4]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#25, d_date_sk#26] - -(51) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#22, i_item_id#25] -Keys [1]: [i_item_id#25] -Functions [1]: [partial_sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum#31] -Results [2]: [i_item_id#25, sum#32] - -(52) Exchange -Input [2]: [i_item_id#25, sum#32] -Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [id=#33] - -(53) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#25, sum#32] -Keys [1]: [i_item_id#25] -Functions [1]: [sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum(cr_return_quantity#22)#34] -Results [2]: [i_item_id#25 AS item_id#35, sum(cr_return_quantity#22)#34 AS cr_item_qty#36] - -(54) BroadcastExchange -Input [2]: [item_id#35, cr_item_qty#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#37] - -(55) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#35] +(39) Project [codegen id : 10] +Output [2]: [cr_return_quantity#23, i_item_id#26] +Input [4]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#26, d_date_sk#27] + +(40) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#23, i_item_id#26] +Keys [1]: [i_item_id#26] +Functions [1]: [partial_sum(cr_return_quantity#23)] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_item_id#26, sum#29] + +(41) Exchange +Input [2]: [i_item_id#26, sum#29] +Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [id=#30] + +(42) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#26, sum#29] +Keys [1]: [i_item_id#26] +Functions [1]: [sum(cr_return_quantity#23)] +Aggregate Attributes [1]: [sum(cr_return_quantity#23)#31] +Results [2]: [i_item_id#26 AS item_id#32, sum(cr_return_quantity#23)#31 AS cr_item_qty#33] + +(43) BroadcastExchange +Input [2]: [item_id#32, cr_item_qty#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] + +(44) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#32] Join condition: None -(56) Project [codegen id : 18] -Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#36] -Input [4]: [item_id#19, sr_item_qty#20, item_id#35, cr_item_qty#36] +(45) Project [codegen id : 18] +Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#33] +Input [4]: [item_id#20, sr_item_qty#21, item_id#32, cr_item_qty#33] -(57) Scan parquet default.web_returns -Output [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] +(46) Scan parquet default.web_returns +Output [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#40), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(wr_returned_date_sk#37), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] +(47) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] -(59) Filter [codegen id : 16] -Input [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] -Condition : isnotnull(wr_item_sk#38) +(48) Filter [codegen id : 16] +Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +Condition : isnotnull(wr_item_sk#35) -(60) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#41, i_item_id#42] +(49) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#38, i_item_id#39] -(61) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#38] -Right keys [1]: [cast(i_item_sk#41 as bigint)] +(50) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#35] +Right keys [1]: [cast(i_item_sk#38 as bigint)] Join condition: None -(62) Project [codegen id : 16] -Output [3]: [wr_return_quantity#39, wr_returned_date_sk#40, i_item_id#42] -Input [5]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40, i_item_sk#41, i_item_id#42] +(51) Project [codegen id : 16] +Output [3]: [wr_return_quantity#36, wr_returned_date_sk#37, i_item_id#39] +Input [5]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37, i_item_sk#38, i_item_id#39] -(63) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#43] +(52) ReusedExchange [Reuses operator id: 25] +Output [1]: [d_date_sk#40] -(64) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#40] -Right keys [1]: [cast(d_date_sk#43 as bigint)] +(53) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#37] +Right keys [1]: [cast(d_date_sk#40 as bigint)] Join condition: None -(65) Project [codegen id : 16] -Output [2]: [wr_return_quantity#39, i_item_id#42] -Input [4]: [wr_return_quantity#39, wr_returned_date_sk#40, i_item_id#42, d_date_sk#43] - -(66) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#39, i_item_id#42] -Keys [1]: [i_item_id#42] -Functions [1]: [partial_sum(wr_return_quantity#39)] -Aggregate Attributes [1]: [sum#44] -Results [2]: [i_item_id#42, sum#45] - -(67) Exchange -Input [2]: [i_item_id#42, sum#45] -Arguments: hashpartitioning(i_item_id#42, 5), ENSURE_REQUIREMENTS, [id=#46] - -(68) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#42, sum#45] -Keys [1]: [i_item_id#42] -Functions [1]: [sum(wr_return_quantity#39)] -Aggregate Attributes [1]: [sum(wr_return_quantity#39)#47] -Results [2]: [i_item_id#42 AS item_id#48, sum(wr_return_quantity#39)#47 AS wr_item_qty#49] - -(69) BroadcastExchange -Input [2]: [item_id#48, wr_item_qty#49] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] - -(70) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#48] +(54) Project [codegen id : 16] +Output [2]: [wr_return_quantity#36, i_item_id#39] +Input [4]: [wr_return_quantity#36, wr_returned_date_sk#37, i_item_id#39, d_date_sk#40] + +(55) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#36, i_item_id#39] +Keys [1]: [i_item_id#39] +Functions [1]: [partial_sum(wr_return_quantity#36)] +Aggregate Attributes [1]: [sum#41] +Results [2]: [i_item_id#39, sum#42] + +(56) Exchange +Input [2]: [i_item_id#39, sum#42] +Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, [id=#43] + +(57) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#39, sum#42] +Keys [1]: [i_item_id#39] +Functions [1]: [sum(wr_return_quantity#36)] +Aggregate Attributes [1]: [sum(wr_return_quantity#36)#44] +Results [2]: [i_item_id#39 AS item_id#45, sum(wr_return_quantity#36)#44 AS wr_item_qty#46] + +(58) BroadcastExchange +Input [2]: [item_id#45, wr_item_qty#46] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#47] + +(59) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#45] Join condition: None -(71) Project [codegen id : 18] -Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS sr_dev#51, cr_item_qty#36, (((cast(cr_item_qty#36 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS cr_dev#52, wr_item_qty#49, (((cast(wr_item_qty#49 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS wr_dev#53, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#54] -Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#36, item_id#48, wr_item_qty#49] +(60) Project [codegen id : 18] +Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS sr_dev#48, cr_item_qty#33, (((cast(cr_item_qty#33 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS cr_dev#49, wr_item_qty#46, (((cast(wr_item_qty#46 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS wr_dev#50, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#51] +Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#33, item_id#45, wr_item_qty#46] -(72) TakeOrderedAndProject -Input [8]: [item_id#19, sr_item_qty#20, sr_dev#51, cr_item_qty#36, cr_dev#52, wr_item_qty#49, wr_dev#53, average#54] -Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#51, cr_item_qty#36, cr_dev#52, wr_item_qty#49, wr_dev#53, average#54] +(61) TakeOrderedAndProject +Input [8]: [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] +Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index 9632348854668..7fc930f2885fd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -68,30 +68,13 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (9) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - Project [d_date] - BroadcastHashJoin [d_week_seq,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - InputAdapter - ReusedExchange [d_week_seq] #5 + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #10 + BroadcastExchange #8 WholeStageCodegen (17) HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] InputAdapter - Exchange [i_item_id] #11 + Exchange [i_item_id] #9 WholeStageCodegen (16) HashAggregate [i_item_id,wr_return_quantity] [sum,sum] Project [wr_return_quantity,i_item_id] @@ -105,4 +88,4 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - ReusedExchange [d_date_sk] #8 + ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt index 8297114e1e9ab..8ae9d897a7d79 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt @@ -119,82 +119,82 @@ Input [3]: [ws_item_sk#4, sum#13, count#14] Keys [1]: [ws_item_sk#4] Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#5))] Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#5))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4 AS ws_item_sk#4#18] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4] (19) Filter -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4#18] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4] Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) (20) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#1] -Right keys [1]: [ws_item_sk#4#18] +Right keys [1]: [ws_item_sk#4] Join condition: None (21) Project [codegen id : 4] Output [2]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17] -Input [3]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4#18] +Input [3]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4] (22) BroadcastExchange Input [2]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (23) Scan parquet default.web_sales -Output [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] +Output [3]: [ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_ext_discount_amt)] ReadSchema: struct (24) ColumnarToRow -Input [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] +Input [3]: [ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] (25) Filter -Input [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] -Condition : (isnotnull(ws_item_sk#4) AND isnotnull(ws_ext_discount_amt#5)) +Input [3]: [ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] +Condition : (isnotnull(ws_item_sk#19) AND isnotnull(ws_ext_discount_amt#20)) (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#1] -Right keys [1]: [ws_item_sk#4] -Join condition: (cast(ws_ext_discount_amt#5 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) +Right keys [1]: [ws_item_sk#19] +Join condition: (cast(ws_ext_discount_amt#20 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) (27) Project [codegen id : 6] -Output [2]: [ws_ext_discount_amt#5, ws_sold_date_sk#6] -Input [5]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] +Output [2]: [ws_ext_discount_amt#20, ws_sold_date_sk#21] +Input [5]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] (28) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ws_sold_date_sk#21] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 6] -Output [1]: [ws_ext_discount_amt#5] -Input [3]: [ws_ext_discount_amt#5, ws_sold_date_sk#6, d_date_sk#8] +Output [1]: [ws_ext_discount_amt#20] +Input [3]: [ws_ext_discount_amt#20, ws_sold_date_sk#21, d_date_sk#22] (31) HashAggregate [codegen id : 6] -Input [1]: [ws_ext_discount_amt#5] +Input [1]: [ws_ext_discount_amt#20] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#5))] -Aggregate Attributes [1]: [sum#20] -Results [1]: [sum#21] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#20))] +Aggregate Attributes [1]: [sum#23] +Results [1]: [sum#24] (32) Exchange -Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [sum#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#25] (33) HashAggregate [codegen id : 7] -Input [1]: [sum#21] +Input [1]: [sum#24] Keys: [] -Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#5))#23] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#5))#23,17,2) AS Excess Discount Amount #24] +Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#20))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#20))#26] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#20))#26,17,2) AS Excess Discount Amount #27] (34) Sort [codegen id : 7] -Input [1]: [Excess Discount Amount #24] -Arguments: [Excess Discount Amount #24 ASC NULLS FIRST], true, 0 +Input [1]: [Excess Discount Amount #27] +Arguments: [Excess Discount Amount #27 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -205,6 +205,6 @@ ReusedExchange (35) (35) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt index cc3ffa0de4bfd..e83a3e67e5c6f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt @@ -23,7 +23,7 @@ WholeStageCodegen (7) InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] Filter [(1.3 * avg(ws_ext_discount_amt))] - HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,sum,count] + HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] InputAdapter Exchange [ws_item_sk] #4 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index faf82026138f6..bae7dd2f3a240 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -82,119 +82,119 @@ Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] (11) Scan parquet default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] +Output [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] +Input [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] (13) Filter [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) +Input [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] +Condition : isnotnull(ws_item_sk#8) (14) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#9] +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#8, d_date#9] +Input [2]: [d_date_sk#11, d_date#12] (16) Filter [codegen id : 2] -Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10983)) AND (d_date#12 <= 11073)) AND isnotnull(d_date_sk#11)) (17) Project [codegen id : 2] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#9] +Output [1]: [d_date_sk#11] +Input [2]: [d_date_sk#11, d_date#12] (18) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] (19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ws_sold_date_sk#10] +Right keys [1]: [d_date_sk#11] Join condition: None (20) Project [codegen id : 3] -Output [2]: [ws_item_sk#1, ws_ext_discount_amt#2] -Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#8] +Output [2]: [ws_item_sk#8, ws_ext_discount_amt#9] +Input [4]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10, d_date_sk#11] (21) HashAggregate [codegen id : 3] -Input [2]: [ws_item_sk#1, ws_ext_discount_amt#2] -Keys [1]: [ws_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [2]: [sum#11, count#12] -Results [3]: [ws_item_sk#1, sum#13, count#14] +Input [2]: [ws_item_sk#8, ws_ext_discount_amt#9] +Keys [1]: [ws_item_sk#8] +Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#9))] +Aggregate Attributes [2]: [sum#14, count#15] +Results [3]: [ws_item_sk#8, sum#16, count#17] (22) Exchange -Input [3]: [ws_item_sk#1, sum#13, count#14] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [3]: [ws_item_sk#8, sum#16, count#17] +Arguments: hashpartitioning(ws_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#18] (23) HashAggregate [codegen id : 4] -Input [3]: [ws_item_sk#1, sum#13, count#14] -Keys [1]: [ws_item_sk#1] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#2))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#2))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1 AS ws_item_sk#1#18] +Input [3]: [ws_item_sk#8, sum#16, count#17] +Keys [1]: [ws_item_sk#8] +Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#9))] +Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#9))#19] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#9))#19 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] (24) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] +Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#20) (25) BroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#19] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#21] (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] -Right keys [1]: [ws_item_sk#1#18] -Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) +Right keys [1]: [ws_item_sk#8] +Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#20) (27) Project [codegen id : 6] Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] +Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] (28) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 6] Output [1]: [ws_ext_discount_amt#2] -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#8] +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#22] (31) HashAggregate [codegen id : 6] Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#20] -Results [1]: [sum#21] +Aggregate Attributes [1]: [sum#23] +Results [1]: [sum#24] (32) Exchange -Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [sum#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#25] (33) HashAggregate [codegen id : 7] -Input [1]: [sum#21] +Input [1]: [sum#24] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#23] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#23,17,2) AS Excess Discount Amount #24] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#26] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#26,17,2) AS Excess Discount Amount #27] (34) Sort [codegen id : 7] -Input [1]: [Excess Discount Amount #24] -Arguments: [Excess Discount Amount #24 ASC NULLS FIRST], true, 0 +Input [1]: [Excess Discount Amount #27] +Arguments: [Excess Discount Amount #27 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -203,8 +203,8 @@ ReusedExchange (35) (35) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index 3108c3bc82c7c..0df713c0e075e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -29,7 +29,7 @@ WholeStageCodegen (7) BroadcastExchange #4 WholeStageCodegen (4) Filter [(1.3 * avg(ws_ext_discount_amt))] - HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,sum,count] + HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] InputAdapter Exchange [ws_item_sk] #5 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt index b2934b6bb8242..3f2b62a7b8fa4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt @@ -81,57 +81,57 @@ Input [3]: [i_item_sk#5, i_current_price#6, i_category#7] Condition : (isnotnull(i_current_price#6) AND isnotnull(i_item_sk#5)) (7) Scan parquet default.item -Output [2]: [i_current_price#6, i_category#7] +Output [2]: [i_current_price#8, i_category#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] +Input [2]: [i_current_price#8, i_category#9] (9) Filter [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] -Condition : isnotnull(i_category#7) +Input [2]: [i_current_price#8, i_category#9] +Condition : isnotnull(i_category#9) (10) HashAggregate [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] -Keys [1]: [i_category#7] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#6))] -Aggregate Attributes [2]: [sum#8, count#9] -Results [3]: [i_category#7, sum#10, count#11] +Input [2]: [i_current_price#8, i_category#9] +Keys [1]: [i_category#9] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#8))] +Aggregate Attributes [2]: [sum#10, count#11] +Results [3]: [i_category#9, sum#12, count#13] (11) Exchange -Input [3]: [i_category#7, sum#10, count#11] -Arguments: hashpartitioning(i_category#7, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [3]: [i_category#9, sum#12, count#13] +Arguments: hashpartitioning(i_category#9, 5), ENSURE_REQUIREMENTS, [id=#14] (12) HashAggregate [codegen id : 2] -Input [3]: [i_category#7, sum#10, count#11] -Keys [1]: [i_category#7] -Functions [1]: [avg(UnscaledValue(i_current_price#6))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#6))#13] -Results [2]: [cast((avg(UnscaledValue(i_current_price#6))#13 / 100.0) as decimal(11,6)) AS avg(i_current_price)#14, i_category#7 AS i_category#7#15] +Input [3]: [i_category#9, sum#12, count#13] +Keys [1]: [i_category#9] +Functions [1]: [avg(UnscaledValue(i_current_price#8))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#8))#15] +Results [2]: [cast((avg(UnscaledValue(i_current_price#8))#15 / 100.0) as decimal(11,6)) AS avg(i_current_price)#16, i_category#9] (13) BroadcastExchange -Input [2]: [avg(i_current_price)#14, i_category#7#15] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#16] +Input [2]: [avg(i_current_price)#16, i_category#9] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#17] (14) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_category#7] -Right keys [1]: [i_category#7#15] +Right keys [1]: [i_category#9] Join condition: None (15) Filter [codegen id : 3] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] -Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#14)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] +Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#16)), DecimalType(14,7), true)) (16) Project [codegen id : 3] Output [1]: [i_item_sk#5] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] (17) BroadcastExchange Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (18) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#1] @@ -143,143 +143,143 @@ Output [2]: [ss_customer_sk#2, ss_sold_date_sk#3] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, i_item_sk#5] (20) Scan parquet default.date_dim -Output [2]: [d_date_sk#18, d_month_seq#19] +Output [2]: [d_date_sk#19, d_month_seq#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#18, d_month_seq#19] +Input [2]: [d_date_sk#19, d_month_seq#20] (22) Filter [codegen id : 4] -Input [2]: [d_date_sk#18, d_month_seq#19] -Condition : ((isnotnull(d_month_seq#19) AND (d_month_seq#19 = Subquery scalar-subquery#20, [id=#21])) AND isnotnull(d_date_sk#18)) +Input [2]: [d_date_sk#19, d_month_seq#20] +Condition : ((isnotnull(d_month_seq#20) AND (d_month_seq#20 = Subquery scalar-subquery#21, [id=#22])) AND isnotnull(d_date_sk#19)) (23) Project [codegen id : 4] -Output [1]: [d_date_sk#18] -Input [2]: [d_date_sk#18, d_month_seq#19] +Output [1]: [d_date_sk#19] +Input [2]: [d_date_sk#19, d_month_seq#20] (24) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (25) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (26) Project [codegen id : 5] Output [1]: [ss_customer_sk#2] -Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#18] +Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#19] (27) Exchange Input [1]: [ss_customer_sk#2] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#23] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] (28) Sort [codegen id : 6] Input [1]: [ss_customer_sk#2] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#25, ca_state#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#24, ca_state#25] +Input [2]: [ca_address_sk#25, ca_state#26] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : isnotnull(ca_address_sk#24) +Input [2]: [ca_address_sk#25, ca_state#26] +Condition : isnotnull(ca_address_sk#25) (32) Exchange -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [ca_address_sk#25, ca_state#26] +Arguments: hashpartitioning(ca_address_sk#25, 5), ENSURE_REQUIREMENTS, [id=#27] (33) Sort [codegen id : 8] -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#25, ca_state#26] +Arguments: [ca_address_sk#25 ASC NULLS FIRST], false, 0 (34) Scan parquet default.customer -Output [2]: [c_customer_sk#27, c_current_addr_sk#28] +Output [2]: [c_customer_sk#28, c_current_addr_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 9] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] (36) Filter [codegen id : 9] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Condition : (isnotnull(c_current_addr_sk#28) AND isnotnull(c_customer_sk#27)) +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Condition : (isnotnull(c_current_addr_sk#29) AND isnotnull(c_customer_sk#28)) (37) Exchange -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Arguments: hashpartitioning(c_current_addr_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Arguments: hashpartitioning(c_current_addr_sk#29, 5), ENSURE_REQUIREMENTS, [id=#30] (38) Sort [codegen id : 10] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Arguments: [c_current_addr_sk#28 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Arguments: [c_current_addr_sk#29 ASC NULLS FIRST], false, 0 (39) SortMergeJoin [codegen id : 11] -Left keys [1]: [ca_address_sk#24] -Right keys [1]: [c_current_addr_sk#28] +Left keys [1]: [ca_address_sk#25] +Right keys [1]: [c_current_addr_sk#29] Join condition: None (40) Project [codegen id : 11] -Output [2]: [ca_state#25, c_customer_sk#27] -Input [4]: [ca_address_sk#24, ca_state#25, c_customer_sk#27, c_current_addr_sk#28] +Output [2]: [ca_state#26, c_customer_sk#28] +Input [4]: [ca_address_sk#25, ca_state#26, c_customer_sk#28, c_current_addr_sk#29] (41) Exchange -Input [2]: [ca_state#25, c_customer_sk#27] -Arguments: hashpartitioning(c_customer_sk#27, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [ca_state#26, c_customer_sk#28] +Arguments: hashpartitioning(c_customer_sk#28, 5), ENSURE_REQUIREMENTS, [id=#31] (42) Sort [codegen id : 12] -Input [2]: [ca_state#25, c_customer_sk#27] -Arguments: [c_customer_sk#27 ASC NULLS FIRST], false, 0 +Input [2]: [ca_state#26, c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 13] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#27] +Right keys [1]: [c_customer_sk#28] Join condition: None (44) Project [codegen id : 13] -Output [1]: [ca_state#25] -Input [3]: [ss_customer_sk#2, ca_state#25, c_customer_sk#27] +Output [1]: [ca_state#26] +Input [3]: [ss_customer_sk#2, ca_state#26, c_customer_sk#28] (45) HashAggregate [codegen id : 13] -Input [1]: [ca_state#25] -Keys [1]: [ca_state#25] +Input [1]: [ca_state#26] +Keys [1]: [ca_state#26] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#31] -Results [2]: [ca_state#25, count#32] +Aggregate Attributes [1]: [count#32] +Results [2]: [ca_state#26, count#33] (46) Exchange -Input [2]: [ca_state#25, count#32] -Arguments: hashpartitioning(ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [2]: [ca_state#26, count#33] +Arguments: hashpartitioning(ca_state#26, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 14] -Input [2]: [ca_state#25, count#32] -Keys [1]: [ca_state#25] +Input [2]: [ca_state#26, count#33] +Keys [1]: [ca_state#26] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#34] -Results [4]: [ca_state#25 AS state#35, count(1)#34 AS cnt#36, count(1)#34 AS count(1)#37, ca_state#25] +Aggregate Attributes [1]: [count(1)#35] +Results [4]: [ca_state#26 AS state#36, count(1)#35 AS cnt#37, count(1)#35 AS count(1)#38, ca_state#26] (48) Filter [codegen id : 14] -Input [4]: [state#35, cnt#36, count(1)#37, ca_state#25] -Condition : (count(1)#37 >= 10) +Input [4]: [state#36, cnt#37, count(1)#38, ca_state#26] +Condition : (count(1)#38 >= 10) (49) Project [codegen id : 14] -Output [3]: [state#35, cnt#36, ca_state#25] -Input [4]: [state#35, cnt#36, count(1)#37, ca_state#25] +Output [3]: [state#36, cnt#37, ca_state#26] +Input [4]: [state#36, cnt#37, count(1)#38, ca_state#26] (50) TakeOrderedAndProject -Input [3]: [state#35, cnt#36, ca_state#25] -Arguments: 100, [cnt#36 ASC NULLS FIRST, ca_state#25 ASC NULLS FIRST], [state#35, cnt#36] +Input [3]: [state#36, cnt#37, ca_state#26] +Arguments: 100, [cnt#37 ASC NULLS FIRST, ca_state#26 ASC NULLS FIRST], [state#36, cnt#37] ===== Subqueries ===== @@ -288,9 +288,9 @@ ReusedExchange (51) (51) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#19] -Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#20, [id=#21] +Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#21, [id=#22] * HashAggregate (58) +- Exchange (57) +- * HashAggregate (56) @@ -301,39 +301,39 @@ Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquer (52) Scan parquet default.date_dim -Output [3]: [d_month_seq#19, d_year#38, d_moy#39] +Output [3]: [d_month_seq#39, d_year#40, d_moy#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (53) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] (54) Filter [codegen id : 1] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] -Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] +Condition : (((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2000)) AND (d_moy#41 = 1)) (55) Project [codegen id : 1] -Output [1]: [d_month_seq#19] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] +Output [1]: [d_month_seq#39] +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] (56) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#19] -Keys [1]: [d_month_seq#19] +Input [1]: [d_month_seq#39] +Keys [1]: [d_month_seq#39] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#19] +Results [1]: [d_month_seq#39] (57) Exchange -Input [1]: [d_month_seq#19] -Arguments: hashpartitioning(d_month_seq#19, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [1]: [d_month_seq#39] +Arguments: hashpartitioning(d_month_seq#39, 5), ENSURE_REQUIREMENTS, [id=#42] (58) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#19] -Keys [1]: [d_month_seq#19] +Input [1]: [d_month_seq#39] +Keys [1]: [d_month_seq#39] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#19] +Results [1]: [d_month_seq#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt index 23b705e3b97e7..f3badf6efe6b6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt @@ -38,7 +38,7 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] InputAdapter Exchange [i_category] #6 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt index 8598023b0276d..b37db85388e0f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt @@ -160,57 +160,57 @@ Input [3]: [i_item_sk#16, i_current_price#17, i_category#18] Condition : (isnotnull(i_current_price#17) AND isnotnull(i_item_sk#16)) (26) Scan parquet default.item -Output [2]: [i_current_price#17, i_category#18] +Output [2]: [i_current_price#19, i_category#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (27) ColumnarToRow [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] +Input [2]: [i_current_price#19, i_category#20] (28) Filter [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] -Condition : isnotnull(i_category#18) +Input [2]: [i_current_price#19, i_category#20] +Condition : isnotnull(i_category#20) (29) HashAggregate [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] -Keys [1]: [i_category#18] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] -Aggregate Attributes [2]: [sum#19, count#20] -Results [3]: [i_category#18, sum#21, count#22] +Input [2]: [i_current_price#19, i_category#20] +Keys [1]: [i_category#20] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#19))] +Aggregate Attributes [2]: [sum#21, count#22] +Results [3]: [i_category#20, sum#23, count#24] (30) Exchange -Input [3]: [i_category#18, sum#21, count#22] -Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [i_category#20, sum#23, count#24] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#25] (31) HashAggregate [codegen id : 5] -Input [3]: [i_category#18, sum#21, count#22] -Keys [1]: [i_category#18] -Functions [1]: [avg(UnscaledValue(i_current_price#17))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#17))#24] -Results [2]: [cast((avg(UnscaledValue(i_current_price#17))#24 / 100.0) as decimal(11,6)) AS avg(i_current_price)#25, i_category#18 AS i_category#18#26] +Input [3]: [i_category#20, sum#23, count#24] +Keys [1]: [i_category#20] +Functions [1]: [avg(UnscaledValue(i_current_price#19))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#19))#26] +Results [2]: [cast((avg(UnscaledValue(i_current_price#19))#26 / 100.0) as decimal(11,6)) AS avg(i_current_price)#27, i_category#20] (32) BroadcastExchange -Input [2]: [avg(i_current_price)#25, i_category#18#26] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#27] +Input [2]: [avg(i_current_price)#27, i_category#20] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#28] (33) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_category#18] -Right keys [1]: [i_category#18#26] +Right keys [1]: [i_category#20] Join condition: None (34) Filter [codegen id : 6] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] -Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#25)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] +Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#27)), DecimalType(14,7), true)) (35) Project [codegen id : 6] Output [1]: [i_item_sk#16] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] (36) BroadcastExchange Input [1]: [i_item_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (37) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#6] @@ -225,31 +225,31 @@ Input [3]: [ca_state#2, ss_item_sk#6, i_item_sk#16] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [2]: [ca_state#2, count#30] +Aggregate Attributes [1]: [count#30] +Results [2]: [ca_state#2, count#31] (40) Exchange -Input [2]: [ca_state#2, count#30] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [2]: [ca_state#2, count#31] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#32] (41) HashAggregate [codegen id : 8] -Input [2]: [ca_state#2, count#30] +Input [2]: [ca_state#2, count#31] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [4]: [ca_state#2 AS state#33, count(1)#32 AS cnt#34, count(1)#32 AS count(1)#35, ca_state#2] +Aggregate Attributes [1]: [count(1)#33] +Results [4]: [ca_state#2 AS state#34, count(1)#33 AS cnt#35, count(1)#33 AS count(1)#36, ca_state#2] (42) Filter [codegen id : 8] -Input [4]: [state#33, cnt#34, count(1)#35, ca_state#2] -Condition : (count(1)#35 >= 10) +Input [4]: [state#34, cnt#35, count(1)#36, ca_state#2] +Condition : (count(1)#36 >= 10) (43) Project [codegen id : 8] -Output [3]: [state#33, cnt#34, ca_state#2] -Input [4]: [state#33, cnt#34, count(1)#35, ca_state#2] +Output [3]: [state#34, cnt#35, ca_state#2] +Input [4]: [state#34, cnt#35, count(1)#36, ca_state#2] (44) TakeOrderedAndProject -Input [3]: [state#33, cnt#34, ca_state#2] -Arguments: 100, [cnt#34 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#33, cnt#34] +Input [3]: [state#34, cnt#35, ca_state#2] +Arguments: 100, [cnt#35 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#34, cnt#35] ===== Subqueries ===== @@ -271,39 +271,39 @@ Subquery:2 Hosting operator id = 18 Hosting Expression = Subquery scalar-subquer (46) Scan parquet default.date_dim -Output [3]: [d_month_seq#12, d_year#36, d_moy#37] +Output [3]: [d_month_seq#37, d_year#38, d_moy#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (47) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] (48) Filter [codegen id : 1] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] -Condition : (((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2000)) AND (d_moy#37 = 1)) +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] +Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) (49) Project [codegen id : 1] -Output [1]: [d_month_seq#12] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] +Output [1]: [d_month_seq#37] +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] (50) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#12] -Keys [1]: [d_month_seq#12] +Input [1]: [d_month_seq#37] +Keys [1]: [d_month_seq#37] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#12] +Results [1]: [d_month_seq#37] (51) Exchange -Input [1]: [d_month_seq#12] -Arguments: hashpartitioning(d_month_seq#12, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [1]: [d_month_seq#37] +Arguments: hashpartitioning(d_month_seq#37, 5), ENSURE_REQUIREMENTS, [id=#40] (52) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#12] -Keys [1]: [d_month_seq#12] +Input [1]: [d_month_seq#37] +Keys [1]: [d_month_seq#37] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#12] +Results [1]: [d_month_seq#37] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index ceddf1b04fc78..4ba09283e73cd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -68,7 +68,7 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] InputAdapter Exchange [i_category] #8 WholeStageCodegen (4) From ad2499d1f6789a26c1ecac55136d33cffa1bb106 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 9 Mar 2021 22:46:09 +0800 Subject: [PATCH 08/38] shortcut import --- .../spark/sql/catalyst/analysis/DeduplicateRelations.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index 8cadc812af6bf..9699a180bcadb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, AttributeSet, NamedExpression, PlanExpression} -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Except, Expand, FlatMapCoGroupsInPandas, FlatMapGroupsInPandas, Generate, Intersect, Join, LogicalPlan, MapInPandas, Project, SerializeFromObject, Union, Window} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule object DeduplicateRelations extends Rule[LogicalPlan] { From fec46948b37e625ec3da4bf1f9d1a8f38f41fc6f Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 9 Mar 2021 22:46:53 +0800 Subject: [PATCH 09/38] fix typo --- .../spark/sql/catalyst/analysis/DeduplicateRelations.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index 9699a180bcadb..358c935052d2d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -120,7 +120,7 @@ object DeduplicateRelations extends Rule[LogicalPlan] { * Generate a new logical plan for the right child with different expression IDs * for all conflicting attributes. */ - private def dedupRight (left: LogicalPlan, right: LogicalPlan): LogicalPlan = { + private def dedupRight(left: LogicalPlan, right: LogicalPlan): LogicalPlan = { val conflictingAttributes = left.outputSet.intersect(right.outputSet) logDebug(s"Conflicting attributes ${conflictingAttributes.mkString(",")} " + s"between $left and $right") From 33f596906e6f48cf6b65775384b36e34f3fcfd12 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 9 Mar 2021 22:47:28 +0800 Subject: [PATCH 10/38] add jira id --- .../sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala index 8968fd86024a0..f41c6753f4f74 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala @@ -379,7 +379,7 @@ class JoinReorderSuite extends JoinReorderPlanTestBase with StatsEstimationTestB assert(!plan2.betterThan(plan1, conf)) } - test("join reorder with self-join") { + test("SPARK-34354: join reorder with self-join") { val plan = t2.join(t1, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) .select(nameToAttr("t1.v-1-10")) .join(t2, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t2.k-1-5"))) From 272afdfc16c972ca97982f1d16840f5a6a020473 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 9 Mar 2021 22:59:04 +0800 Subject: [PATCH 11/38] add comment --- .../spark/sql/catalyst/analysis/DeduplicateRelations.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index 358c935052d2d..2d3bb77671675 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -56,6 +56,13 @@ object DeduplicateRelations extends Rule[LogicalPlan] { } } + /** + * Deduplicate any duplicated relations of a LogicalPlan + * @param existingRelations the known unique relations for a LogicalPlan + * @param plan the LogicalPlan that requires the deduplication + * @return (the new LogicalPlan which already deduplicate all duplicated relations (if any), + * all relations of the new LogicalPlan ) + */ private def renewDuplicatedRelations( existingRelations: Seq[MultiInstanceRelation], plan: LogicalPlan): (LogicalPlan, Seq[MultiInstanceRelation]) = plan match { From f17557e711aca979d5752807bc2ea3bcac5d87df Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 15 Mar 2021 14:27:41 +0800 Subject: [PATCH 12/38] fix AnalysisSuite --- .../sql/catalyst/analysis/AnalysisSuite.scala | 32 +++++++++---------- .../sql/catalyst/analysis/AnalysisTest.scala | 2 +- 2 files changed, 17 insertions(+), 17 deletions(-) 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 38f92c68af020..d48eea63d03f4 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 @@ -1051,7 +1051,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("SPARK-22748: Analyze __grouping__id as a literal function") { - assertAnalysisSuccess(parsePlan( + assertAnalysisSuccess(EliminateSubqueryAliases(parsePlan( """ |SELECT grouping__id FROM ( | SELECT grouping__id FROM ( @@ -1059,61 +1059,61 @@ class AnalysisSuite extends AnalysisTest with Matchers { | GROUP BY a, b WITH ROLLUP | ) |) - """.stripMargin), false) + """.stripMargin)), false) - assertAnalysisSuccess(parsePlan( + assertAnalysisSuccess(EliminateSubqueryAliases(parsePlan( """ |SELECT grouping__id FROM ( | SELECT a, b, count(1), grouping__id FROM TaBlE2 | GROUP BY a, b WITH CUBE |) - """.stripMargin), false) + """.stripMargin)), false) - assertAnalysisSuccess(parsePlan( + assertAnalysisSuccess(EliminateSubqueryAliases(parsePlan( """ |SELECT grouping__id FROM ( | SELECT a, b, count(1), grouping__id FROM TaBlE2 | GROUP BY a, b GROUPING SETS ((a, b), ()) |) - """.stripMargin), false) + """.stripMargin)), false) - assertAnalysisSuccess(parsePlan( + assertAnalysisSuccess(EliminateSubqueryAliases(parsePlan( """ |SELECT a, b, count(1) FROM TaBlE2 | GROUP BY CUBE(a, b) HAVING grouping__id > 0 - """.stripMargin), false) + """.stripMargin)), false) - assertAnalysisSuccess(parsePlan( + assertAnalysisSuccess(EliminateSubqueryAliases(parsePlan( """ |SELECT * FROM ( | SELECT a, b, count(1) FROM TaBlE2 | GROUP BY a, b GROUPING SETS ((a, b), ()) |) WHERE grouping__id > 0 - """.stripMargin), false) + """.stripMargin)), false) - assertAnalysisSuccess(parsePlan( + assertAnalysisSuccess(EliminateSubqueryAliases(parsePlan( """ |SELECT * FROM ( | SELECT a, b, count(1) FROM TaBlE2 | GROUP BY a, b GROUPING SETS ((a, b), ()) |) ORDER BY grouping__id > 0 - """.stripMargin), false) + """.stripMargin)), false) - assertAnalysisSuccess(parsePlan( + assertAnalysisSuccess(EliminateSubqueryAliases(parsePlan( """ |SELECT a, b, count(1) FROM TaBlE2 | GROUP BY a, b GROUPING SETS ((a, b), ()) | ORDER BY grouping__id > 0 - """.stripMargin), false) + """.stripMargin)), false) - assertAnalysisError(parsePlan( + assertAnalysisError(EliminateSubqueryAliases(parsePlan( """ |SELECT grouping__id FROM ( | SELECT a, b, count(1), grouping__id FROM TaBlE2 | GROUP BY a, b |) - """.stripMargin), + """.stripMargin)), Seq("grouping_id() can only be used with GroupingSets/Cube/Rollup"), false) } 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 e4c88df882d9d..39a068fccef72 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 @@ -101,7 +101,7 @@ trait AnalysisTest extends PlanTest { caseSensitive: Boolean = true): Unit = { withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { val actualPlan = getAnalyzer.executeAndCheck(inputPlan, new QueryPlanningTracker) - val transformed = actualPlan transformUp { + val transformed = EliminateSubqueryAliases(actualPlan) transformUp { case v: View if v.isTempViewStoringAnalyzedPlan => v.child } comparePlans(transformed, expectedPlan) From 53cc894b621aa8c65705f1bf95c843a18fef92c8 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 15 Mar 2021 14:35:00 +0800 Subject: [PATCH 13/38] fix LeftSemiPushdownSuite --- .../scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 3676f0cfd73dc..207ca9097f640 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -69,10 +69,10 @@ trait PlanTestBase extends PredicateHelper with SQLHelper with SQLConfHelper { s * Since attribute references are given globally unique ids during analysis, * we must normalize them to check if two different queries are identical. */ - protected def normalizeExprIds(plan: LogicalPlan) = { + protected def normalizeExprIds(plan: LogicalPlan): LogicalPlan = { plan transformAllExpressions { case s: ScalarSubquery => - s.copy(exprId = ExprId(0)) + s.copy(plan = normalizeExprIds(s.plan), exprId = ExprId(0)) case e: Exists => e.copy(exprId = ExprId(0)) case l: ListQuery => From 29c00d47a361641c9acd9fd9c348d1e610e70e0d Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 18 Mar 2021 20:29:45 +0800 Subject: [PATCH 14/38] fix --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 3 --- .../spark/sql/catalyst/analysis/DeduplicateRelations.scala | 2 ++ 2 files changed, 2 insertions(+), 3 deletions(-) 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 75816ffa1c11c..231605cf413ac 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 @@ -1604,9 +1604,6 @@ class Analyzer(override val catalogManager: CatalogManager) // implementation and should be resolved based on the table schema. o.copy(deleteExpr = resolveExpressionByPlanOutput(o.deleteExpr, o.table)) - case m @ MergeIntoTable(targetTable, sourceTable, _, _, _) if !m.duplicateResolved => - m.copy(sourceTable = dedupRight(targetTable, sourceTable)) - case m @ MergeIntoTable(targetTable, sourceTable, _, _, _) if !m.resolved && targetTable.resolved && sourceTable.resolved => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index 2d3bb77671675..306c63010d49c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -53,6 +53,8 @@ object DeduplicateRelations extends Rule[LogicalPlan] { } } u.copy(children = newChildren) + case m @ MergeIntoTable(targetTable, sourceTable, _, _, _) if !m.duplicateResolved => + m.copy(sourceTable = dedupRight(targetTable, sourceTable)) } } From 1acaea50336c29ba6dcdf9cbb62deadf6718f255 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Fri, 19 Mar 2021 15:44:08 +0800 Subject: [PATCH 15/38] fix --- .../sql/catalyst/analysis/Analyzer.scala | 58 ------------------- .../sql/catalyst/analysis/AnalysisSuite.scala | 2 +- 2 files changed, 1 insertion(+), 59 deletions(-) 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 231605cf413ac..49abe27286d88 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 @@ -1431,64 +1431,6 @@ class Analyzer(override val catalogManager: CatalogManager) * a logical plan node's children. */ object ResolveReferences extends Rule[LogicalPlan] { - /** - * Resolves the attribute and extract value expressions(s) by traversing the - * input expression in top down manner. The traversal is done in top-down manner as - * we need to skip over unbound lambda function expression. The lambda expressions are - * resolved in a different rule [[ResolveLambdaVariables]] - * - * Example : - * SELECT transform(array(1, 2, 3), (x, i) -> x + i)" - * - * In the case above, x and i are resolved as lambda variables in [[ResolveLambdaVariables]] - * - * Note : In this routine, the unresolved attributes are resolved from the input plan's - * children attributes. - * - * @param e The expression need to be resolved. - * @param q The LogicalPlan whose children are used to resolve expression's attribute. - * @param trimAlias When true, trim unnecessary alias of `GetStructField`. Note that, - * we cannot trim the alias of top-level `GetStructField`, as we should - * resolve `UnresolvedAttribute` to a named expression. The caller side - * can trim the alias of top-level `GetStructField` if it's safe to do so. - * @return resolved Expression. - */ - private def resolveExpressionTopDown( - e: Expression, - q: LogicalPlan, - trimAlias: Boolean = false): Expression = { - - def innerResolve(e: Expression, isTopLevel: Boolean): Expression = { - if (e.resolved) return e - e match { - case f: LambdaFunction if !f.bound => f - case u @ UnresolvedAttribute(nameParts) => - // Leave unchanged if resolution fails. Hopefully will be resolved next round. - val resolved = - withPosition(u) { - q.resolveChildren(nameParts, resolver) - .orElse(resolveLiteralFunction(nameParts, u, q)) - .getOrElse(u) - } - val result = resolved match { - // As the comment of method `resolveExpressionTopDown`'s param `trimAlias` said, - // when trimAlias = true, we will trim unnecessary alias of `GetStructField` and - // we won't trim the alias of top-level `GetStructField`. Since we will call - // CleanupAliases later in Analyzer, trim non top-level unnecessary alias of - // `GetStructField` here is safe. - case Alias(s: GetStructField, _) if trimAlias && !isTopLevel => s - case others => others - } - logDebug(s"Resolving $u to $result") - result - case UnresolvedExtractValue(child, fieldExpr) if child.resolved => - ExtractValue(child, fieldExpr, resolver) - case _ => e.mapChildren(innerResolve(_, isTopLevel = false)) - } - } - - innerResolve(e, isTopLevel = true) - } private def hasConflictingAttrs(p: LogicalPlan): Boolean = { p.children.length > 1 && { 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 d48eea63d03f4..1a694f81f8119 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 @@ -710,7 +710,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { test("CTE with non-existing column alias") { assertAnalysisError(parsePlan("WITH t(x) AS (SELECT 1) SELECT * FROM t WHERE y = 1"), - Seq("cannot resolve 'y' given input columns: [x]")) + Seq("cannot resolve 'y' given input columns: [t.x]")) } test("CTE with non-matching column alias") { From ba875b37d0f4b9b62307d2d973e4c10fc5abfc9c Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 22 Mar 2021 17:23:14 +0800 Subject: [PATCH 16/38] fix --- .../spark/sql/catalyst/analysis/AnalysisTest.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) 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 39a068fccef72..e3ad2f0e4986a 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 @@ -74,11 +74,11 @@ trait AnalysisTest extends PlanTest { catalog.createDatabase( CatalogDatabase("default", "", new URI("loc"), Map.empty), ignoreIfExists = false) - catalog.createTempView("TaBlE", TestRelations.testRelation, overrideIfExists = true) - catalog.createTempView("TaBlE2", TestRelations.testRelation2, overrideIfExists = true) - catalog.createTempView("TaBlE3", TestRelations.testRelation3, overrideIfExists = true) - catalog.createGlobalTempView("TaBlE4", TestRelations.testRelation4, overrideIfExists = true) - catalog.createGlobalTempView("TaBlE5", TestRelations.testRelation5, overrideIfExists = true) + createTempView(catalog, "TaBlE", TestRelations.testRelation, overrideIfExists = true) + createTempView(catalog, "TaBlE2", TestRelations.testRelation2, overrideIfExists = true) + createTempView(catalog, "TaBlE3", TestRelations.testRelation3, overrideIfExists = true) + createGlobalTempView(catalog, "TaBlE4", TestRelations.testRelation4, overrideIfExists = true) + createGlobalTempView(catalog, "TaBlE5", TestRelations.testRelation5, overrideIfExists = true) new Analyzer(catalog) { override val extendedResolutionRules = extendedAnalysisRules } From a67444c4a19040819c34a3a2a092b9b2839f4ae2 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 22 Mar 2021 17:25:38 +0800 Subject: [PATCH 17/38] regen --- .../q10.sf100/explain.txt | 102 +- .../approved-plans-modified/q10/explain.txt | 100 +- .../q27.sf100/explain.txt | 162 +-- .../approved-plans-modified/q27/explain.txt | 162 +-- .../q34.sf100/explain.txt | 4 +- .../approved-plans-modified/q34/explain.txt | 4 +- .../q46.sf100/explain.txt | 18 +- .../approved-plans-modified/q46/explain.txt | 14 +- .../q53.sf100/explain.txt | 6 +- .../approved-plans-modified/q53/explain.txt | 6 +- .../q59.sf100/explain.txt | 102 +- .../approved-plans-modified/q59/explain.txt | 102 +- .../q63.sf100/explain.txt | 6 +- .../approved-plans-modified/q63/explain.txt | 6 +- .../q65.sf100/explain.txt | 86 +- .../approved-plans-modified/q65/explain.txt | 48 +- .../q68.sf100/explain.txt | 20 +- .../approved-plans-modified/q68/explain.txt | 14 +- .../q7.sf100/explain.txt | 4 +- .../approved-plans-modified/q7/explain.txt | 4 +- .../q73.sf100/explain.txt | 4 +- .../approved-plans-modified/q73/explain.txt | 4 +- .../q89.sf100/explain.txt | 6 +- .../approved-plans-modified/q89/explain.txt | 6 +- .../q98.sf100/explain.txt | 6 +- .../approved-plans-modified/q98/explain.txt | 6 +- .../approved-plans-v1_4/q1.sf100/explain.txt | 122 +- .../approved-plans-v1_4/q1/explain.txt | 116 +- .../approved-plans-v1_4/q10.sf100/explain.txt | 104 +- .../approved-plans-v1_4/q10/explain.txt | 94 +- .../approved-plans-v1_4/q11.sf100/explain.txt | 272 ++--- .../approved-plans-v1_4/q11/explain.txt | 254 ++-- .../approved-plans-v1_4/q12.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q12/explain.txt | 6 +- .../approved-plans-v1_4/q13.sf100/explain.txt | 8 +- .../approved-plans-v1_4/q13/explain.txt | 8 +- .../q14a.sf100/explain.txt | 602 ++++----- .../approved-plans-v1_4/q14a/explain.txt | 536 ++++---- .../q14b.sf100/explain.txt | 542 ++++----- .../approved-plans-v1_4/q14b/explain.txt | 488 ++++---- .../approved-plans-v1_4/q16.sf100/explain.txt | 128 +- .../approved-plans-v1_4/q16/explain.txt | 128 +- .../approved-plans-v1_4/q17.sf100/explain.txt | 8 +- .../approved-plans-v1_4/q17/explain.txt | 8 +- .../approved-plans-v1_4/q18.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q18/explain.txt | 4 +- .../approved-plans-v1_4/q2.sf100/explain.txt | 52 +- .../approved-plans-v1_4/q2/explain.txt | 52 +- .../approved-plans-v1_4/q20.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q20/explain.txt | 6 +- .../approved-plans-v1_4/q21.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q21/explain.txt | 2 +- .../q23a.sf100/explain.txt | 362 +++--- .../approved-plans-v1_4/q23a/explain.txt | 274 ++--- .../q23b.sf100/explain.txt | 528 ++++---- .../approved-plans-v1_4/q23b/explain.txt | 404 +++--- .../q24a.sf100/explain.txt | 210 ++-- .../approved-plans-v1_4/q24a/explain.txt | 174 +-- .../q24b.sf100/explain.txt | 210 ++-- .../approved-plans-v1_4/q24b/explain.txt | 174 +-- .../approved-plans-v1_4/q26.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q26/explain.txt | 4 +- .../approved-plans-v1_4/q27.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q27/explain.txt | 4 +- .../approved-plans-v1_4/q28.sf100/explain.txt | 300 ++--- .../approved-plans-v1_4/q28/explain.txt | 300 ++--- .../approved-plans-v1_4/q30.sf100/explain.txt | 142 +-- .../approved-plans-v1_4/q30/explain.txt | 130 +- .../approved-plans-v1_4/q31.sf100/explain.txt | 430 +++---- .../approved-plans-v1_4/q31/explain.txt | 362 +++--- .../approved-plans-v1_4/q32.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q32/explain.txt | 2 +- .../approved-plans-v1_4/q34.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q34/explain.txt | 4 +- .../approved-plans-v1_4/q35.sf100/explain.txt | 116 +- .../approved-plans-v1_4/q35/explain.txt | 96 +- .../approved-plans-v1_4/q37.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q37/explain.txt | 2 +- .../approved-plans-v1_4/q38.sf100/explain.txt | 24 +- .../approved-plans-v1_4/q38/explain.txt | 12 +- .../q39a.sf100/explain.txt | 110 +- .../approved-plans-v1_4/q39a/explain.txt | 106 +- .../q39b.sf100/explain.txt | 110 +- .../approved-plans-v1_4/q39b/explain.txt | 106 +- .../approved-plans-v1_4/q4.sf100/explain.txt | 442 +++---- .../approved-plans-v1_4/q4/explain.txt | 412 +++---- .../approved-plans-v1_4/q44.sf100/explain.txt | 118 +- .../approved-plans-v1_4/q44/explain.txt | 118 +- .../approved-plans-v1_4/q45.sf100/explain.txt | 36 +- .../approved-plans-v1_4/q45/explain.txt | 36 +- .../approved-plans-v1_4/q46.sf100/explain.txt | 44 +- .../approved-plans-v1_4/q46/explain.txt | 14 +- .../approved-plans-v1_4/q47.sf100/explain.txt | 85 +- .../approved-plans-v1_4/q47/explain.txt | 77 +- .../approved-plans-v1_4/q48.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q48/explain.txt | 6 +- .../approved-plans-v1_4/q49.sf100/explain.txt | 198 +-- .../approved-plans-v1_4/q49/explain.txt | 152 +-- .../approved-plans-v1_4/q5.sf100/explain.txt | 164 +-- .../approved-plans-v1_4/q5/explain.txt | 176 +-- .../approved-plans-v1_4/q53.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q53/explain.txt | 6 +- .../approved-plans-v1_4/q54.sf100/explain.txt | 122 +- .../approved-plans-v1_4/q54/explain.txt | 124 +- .../approved-plans-v1_4/q57.sf100/explain.txt | 85 +- .../approved-plans-v1_4/q57/explain.txt | 77 +- .../approved-plans-v1_4/q58.sf100/explain.txt | 226 ++-- .../approved-plans-v1_4/q58/explain.txt | 228 ++-- .../approved-plans-v1_4/q59.sf100/explain.txt | 70 +- .../approved-plans-v1_4/q59/explain.txt | 70 +- .../approved-plans-v1_4/q61.sf100/explain.txt | 84 +- .../approved-plans-v1_4/q61/explain.txt | 94 +- .../approved-plans-v1_4/q63.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q63/explain.txt | 6 +- .../approved-plans-v1_4/q64.sf100/explain.txt | 320 ++--- .../approved-plans-v1_4/q64/explain.txt | 288 ++--- .../approved-plans-v1_4/q65.sf100/explain.txt | 98 +- .../approved-plans-v1_4/q65/explain.txt | 48 +- .../approved-plans-v1_4/q66.sf100/explain.txt | 70 +- .../approved-plans-v1_4/q66/explain.txt | 74 +- .../approved-plans-v1_4/q68.sf100/explain.txt | 44 +- .../approved-plans-v1_4/q68/explain.txt | 14 +- .../approved-plans-v1_4/q69.sf100/explain.txt | 96 +- .../approved-plans-v1_4/q69/explain.txt | 94 +- .../approved-plans-v1_4/q7.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q7/explain.txt | 4 +- .../approved-plans-v1_4/q70.sf100/explain.txt | 136 +-- .../approved-plans-v1_4/q70/explain.txt | 136 +-- .../approved-plans-v1_4/q71.sf100/explain.txt | 82 +- .../approved-plans-v1_4/q71/explain.txt | 82 +- .../approved-plans-v1_4/q72.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q72/explain.txt | 4 +- .../approved-plans-v1_4/q73.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q73/explain.txt | 4 +- .../approved-plans-v1_4/q74.sf100/explain.txt | 268 ++-- .../approved-plans-v1_4/q74/explain.txt | 250 ++-- .../approved-plans-v1_4/q75.sf100/explain.txt | 372 +++--- .../approved-plans-v1_4/q75/explain.txt | 372 +++--- .../approved-plans-v1_4/q76.sf100/explain.txt | 80 +- .../approved-plans-v1_4/q76/explain.txt | 64 +- .../approved-plans-v1_4/q77.sf100/explain.txt | 284 ++--- .../approved-plans-v1_4/q77/explain.txt | 300 ++--- .../approved-plans-v1_4/q78.sf100/explain.txt | 146 +-- .../approved-plans-v1_4/q78/explain.txt | 146 +-- .../approved-plans-v1_4/q8.sf100/explain.txt | 120 +- .../approved-plans-v1_4/q8/explain.txt | 106 +- .../approved-plans-v1_4/q80.sf100/explain.txt | 206 ++-- .../approved-plans-v1_4/q80/explain.txt | 210 ++-- .../approved-plans-v1_4/q81.sf100/explain.txt | 144 +-- .../approved-plans-v1_4/q81/explain.txt | 126 +- .../approved-plans-v1_4/q82.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q82/explain.txt | 2 +- .../approved-plans-v1_4/q85.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q85/explain.txt | 6 +- .../approved-plans-v1_4/q87.sf100/explain.txt | 24 +- .../approved-plans-v1_4/q87/explain.txt | 12 +- .../approved-plans-v1_4/q88.sf100/explain.txt | 504 ++++---- .../approved-plans-v1_4/q88/explain.txt | 504 ++++---- .../approved-plans-v1_4/q89.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q89/explain.txt | 6 +- .../approved-plans-v1_4/q9.sf100/explain.txt | 416 +++---- .../approved-plans-v1_4/q9/explain.txt | 416 +++---- .../approved-plans-v1_4/q90.sf100/explain.txt | 80 +- .../approved-plans-v1_4/q90/explain.txt | 80 +- .../approved-plans-v1_4/q91.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q91/explain.txt | 4 +- .../approved-plans-v1_4/q92.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q92/explain.txt | 2 +- .../approved-plans-v1_4/q93.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q93/explain.txt | 4 +- .../approved-plans-v1_4/q94.sf100/explain.txt | 136 +-- .../approved-plans-v1_4/q94/explain.txt | 136 +-- .../approved-plans-v1_4/q95.sf100/explain.txt | 196 +-- .../approved-plans-v1_4/q95/explain.txt | 196 +-- .../approved-plans-v1_4/q97.sf100/explain.txt | 40 +- .../approved-plans-v1_4/q97/explain.txt | 40 +- .../approved-plans-v1_4/q98.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q98/explain.txt | 6 +- .../q10a.sf100/explain.txt | 96 +- .../approved-plans-v2_7/q10a/explain.txt | 94 +- .../approved-plans-v2_7/q11.sf100/explain.txt | 268 ++-- .../approved-plans-v2_7/q11/explain.txt | 250 ++-- .../approved-plans-v2_7/q12.sf100/explain.txt | 6 +- .../approved-plans-v2_7/q12/explain.txt | 6 +- .../approved-plans-v2_7/q14.sf100/explain.txt | 542 ++++----- .../approved-plans-v2_7/q14/explain.txt | 488 ++++---- .../q14a.sf100/explain.txt | 1082 ++++++++--------- .../approved-plans-v2_7/q14a/explain.txt | 1016 ++++++++-------- .../q18a.sf100/explain.txt | 402 +++--- .../approved-plans-v2_7/q18a/explain.txt | 410 +++---- .../approved-plans-v2_7/q20.sf100/explain.txt | 6 +- .../approved-plans-v2_7/q20/explain.txt | 6 +- .../q22a.sf100/explain.txt | 122 +- .../approved-plans-v2_7/q22a/explain.txt | 122 +- .../approved-plans-v2_7/q24.sf100/explain.txt | 186 +-- .../approved-plans-v2_7/q24/explain.txt | 174 +-- .../q27a.sf100/explain.txt | 162 +-- .../approved-plans-v2_7/q27a/explain.txt | 162 +-- .../approved-plans-v2_7/q34.sf100/explain.txt | 4 +- .../approved-plans-v2_7/q34/explain.txt | 4 +- .../approved-plans-v2_7/q35.sf100/explain.txt | 116 +- .../approved-plans-v2_7/q35/explain.txt | 96 +- .../q35a.sf100/explain.txt | 116 +- .../approved-plans-v2_7/q35a/explain.txt | 96 +- .../q36a.sf100/explain.txt | 92 +- .../approved-plans-v2_7/q36a/explain.txt | 92 +- .../approved-plans-v2_7/q47.sf100/explain.txt | 85 +- .../approved-plans-v2_7/q47/explain.txt | 77 +- .../approved-plans-v2_7/q49.sf100/explain.txt | 198 +-- .../approved-plans-v2_7/q49/explain.txt | 152 +-- .../q51a.sf100/explain.txt | 267 +++- .../approved-plans-v2_7/q51a/explain.txt | 264 +++- .../approved-plans-v2_7/q57.sf100/explain.txt | 85 +- .../approved-plans-v2_7/q57/explain.txt | 77 +- .../approved-plans-v2_7/q5a.sf100/explain.txt | 232 ++-- .../approved-plans-v2_7/q5a/explain.txt | 244 ++-- .../approved-plans-v2_7/q64.sf100/explain.txt | 320 ++--- .../approved-plans-v2_7/q64/explain.txt | 288 ++--- .../q67a.sf100/explain.txt | 296 ++--- .../approved-plans-v2_7/q67a/explain.txt | 296 ++--- .../q70a.sf100/explain.txt | 212 ++-- .../approved-plans-v2_7/q70a/explain.txt | 212 ++-- .../approved-plans-v2_7/q72.sf100/explain.txt | 4 +- .../approved-plans-v2_7/q72/explain.txt | 4 +- .../approved-plans-v2_7/q74.sf100/explain.txt | 268 ++-- .../approved-plans-v2_7/q74/explain.txt | 250 ++-- .../approved-plans-v2_7/q75.sf100/explain.txt | 372 +++--- .../approved-plans-v2_7/q75/explain.txt | 372 +++--- .../q77a.sf100/explain.txt | 364 +++--- .../approved-plans-v2_7/q77a/explain.txt | 380 +++--- .../approved-plans-v2_7/q78.sf100/explain.txt | 146 +-- .../approved-plans-v2_7/q78/explain.txt | 146 +-- .../q80a.sf100/explain.txt | 270 ++-- .../approved-plans-v2_7/q80a/explain.txt | 274 ++--- .../q86a.sf100/explain.txt | 92 +- .../approved-plans-v2_7/q86a/explain.txt | 92 +- .../approved-plans-v2_7/q98.sf100/explain.txt | 6 +- .../approved-plans-v2_7/q98/explain.txt | 6 +- 238 files changed, 16877 insertions(+), 15890 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt index 8a4b341ac925b..1fdcba1ed4de4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt @@ -137,22 +137,22 @@ Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] Condition : isnotnull(cs_ship_customer_sk#13) (19) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#15] (20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (21) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13 AS customer_sk#15] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#8] +Output [1]: [cs_ship_customer_sk#13 AS customer_sk#16] +Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] (22) Union (23) Exchange Input [1]: [customer_sk#12] -Arguments: hashpartitioning(customer_sk#12, 5), ENSURE_REQUIREMENTS, [id=#16] +Arguments: hashpartitioning(customer_sk#12, 5), ENSURE_REQUIREMENTS, [id=#17] (24) Sort [codegen id : 7] Input [1]: [customer_sk#12] @@ -164,43 +164,43 @@ Right keys [1]: [customer_sk#12] Join condition: None (26) Scan parquet default.store_sales -Output [2]: [ss_customer_sk#17, ss_sold_date_sk#18] +Output [2]: [ss_customer_sk#18, ss_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#19), dynamicpruningexpression(ss_sold_date_sk#19 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 9] -Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] +Input [2]: [ss_customer_sk#18, ss_sold_date_sk#19] (28) Filter [codegen id : 9] -Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#17) +Input [2]: [ss_customer_sk#18, ss_sold_date_sk#19] +Condition : isnotnull(ss_customer_sk#18) (29) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#20] (30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#18] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ss_sold_date_sk#19] +Right keys [1]: [d_date_sk#20] Join condition: None (31) Project [codegen id : 9] -Output [1]: [ss_customer_sk#17 AS customer_sk#19] -Input [3]: [ss_customer_sk#17, ss_sold_date_sk#18, d_date_sk#8] +Output [1]: [ss_customer_sk#18 AS customer_sk#21] +Input [3]: [ss_customer_sk#18, ss_sold_date_sk#19, d_date_sk#20] (32) Exchange -Input [1]: [customer_sk#19] -Arguments: hashpartitioning(customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [customer_sk#21] +Arguments: hashpartitioning(customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#22] (33) Sort [codegen id : 10] -Input [1]: [customer_sk#19] -Arguments: [customer_sk#19 ASC NULLS FIRST], false, 0 +Input [1]: [customer_sk#21] +Arguments: [customer_sk#21 ASC NULLS FIRST], false, 0 (34) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#19] +Right keys [1]: [customer_sk#21] Join condition: None (35) Project [codegen id : 12] @@ -208,84 +208,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (36) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_county#22] +Output [2]: [ca_address_sk#23, ca_county#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 11] -Input [2]: [ca_address_sk#21, ca_county#22] +Input [2]: [ca_address_sk#23, ca_county#24] (38) Filter [codegen id : 11] -Input [2]: [ca_address_sk#21, ca_county#22] -Condition : (ca_county#22 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#21)) +Input [2]: [ca_address_sk#23, ca_county#24] +Condition : (ca_county#24 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#23)) (39) Project [codegen id : 11] -Output [1]: [ca_address_sk#21] -Input [2]: [ca_address_sk#21, ca_county#22] +Output [1]: [ca_address_sk#23] +Input [2]: [ca_address_sk#23, ca_county#24] (40) BroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [ca_address_sk#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#23] Join condition: None (42) Project [codegen id : 12] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#23] (43) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (44) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (45) ColumnarToRow -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] (46) Filter -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Condition : isnotnull(cd_demo_sk#25) +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Condition : isnotnull(cd_demo_sk#27) (47) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#27] Join condition: None (48) Project [codegen id : 13] -Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] (49) HashAggregate [codegen id : 13] -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#34] -Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#35] +Aggregate Attributes [1]: [count#36] +Results [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] (50) Exchange -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#35] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] +Arguments: hashpartitioning(cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, 5), ENSURE_REQUIREMENTS, [id=#38] (51) HashAggregate [codegen id : 14] -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#35] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] +Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#37] -Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#37 AS cnt1#38, cd_purchase_estimate#29, count(1)#37 AS cnt2#39, cd_credit_rating#30, count(1)#37 AS cnt3#40, cd_dep_count#31, count(1)#37 AS cnt4#41, cd_dep_employed_count#32, count(1)#37 AS cnt5#42, cd_dep_college_count#33, count(1)#37 AS cnt6#43] +Aggregate Attributes [1]: [count(1)#39] +Results [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, count(1)#39 AS cnt1#40, cd_purchase_estimate#31, count(1)#39 AS cnt2#41, cd_credit_rating#32, count(1)#39 AS cnt3#42, cd_dep_count#33, count(1)#39 AS cnt4#43, cd_dep_employed_count#34, count(1)#39 AS cnt5#44, cd_dep_college_count#35, count(1)#39 AS cnt6#45] (52) TakeOrderedAndProject -Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#38, cd_purchase_estimate#29, cnt2#39, cd_credit_rating#30, cnt3#40, cd_dep_count#31, cnt4#41, cd_dep_employed_count#32, cnt5#42, cd_dep_college_count#33, cnt6#43] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#38, cd_purchase_estimate#29, cnt2#39, cd_credit_rating#30, cnt3#40, cd_dep_count#31, cnt4#41, cd_dep_employed_count#32, cnt5#42, cd_dep_college_count#33, cnt6#43] +Input [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#40, cd_purchase_estimate#31, cnt2#41, cd_credit_rating#32, cnt3#42, cd_dep_count#33, cnt4#43, cd_dep_employed_count#34, cnt5#44, cd_dep_college_count#35, cnt6#45] +Arguments: 100, [cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_education_status#30 ASC NULLS FIRST, cd_purchase_estimate#31 ASC NULLS FIRST, cd_credit_rating#32 ASC NULLS FIRST, cd_dep_count#33 ASC NULLS FIRST, cd_dep_employed_count#34 ASC NULLS FIRST, cd_dep_college_count#35 ASC NULLS FIRST], [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#40, cd_purchase_estimate#31, cnt2#41, cd_credit_rating#32, cnt3#42, cd_dep_count#33, cnt4#43, cd_dep_employed_count#34, cnt5#44, cd_dep_college_count#35, cnt6#45] ===== Subqueries ===== @@ -298,6 +298,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#19 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt index 6f64579d09c78..ff55b1c511456 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt @@ -125,22 +125,22 @@ Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] Condition : isnotnull(cs_ship_customer_sk#12) (17) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] -Output [1]: [cs_ship_customer_sk#12 AS customer_sk#14] -Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#7] +Output [1]: [cs_ship_customer_sk#12 AS customer_sk#15] +Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] (20) Union (21) BroadcastExchange Input [1]: [customer_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (22) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] @@ -148,39 +148,39 @@ Right keys [1]: [customer_sk#11] Join condition: None (23) Scan parquet default.store_sales -Output [2]: [ss_customer_sk#16, ss_sold_date_sk#17] +Output [2]: [ss_customer_sk#17, ss_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#17), dynamicpruningexpression(ss_sold_date_sk#17 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [2]: [ss_customer_sk#16, ss_sold_date_sk#17] +Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] (25) Filter [codegen id : 6] -Input [2]: [ss_customer_sk#16, ss_sold_date_sk#17] -Condition : isnotnull(ss_customer_sk#16) +Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] +Condition : isnotnull(ss_customer_sk#17) (26) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#19] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#17] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [ss_sold_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (28) Project [codegen id : 6] -Output [1]: [ss_customer_sk#16 AS customer_sk#18] -Input [3]: [ss_customer_sk#16, ss_sold_date_sk#17, d_date_sk#7] +Output [1]: [ss_customer_sk#17 AS customer_sk#20] +Input [3]: [ss_customer_sk#17, ss_sold_date_sk#18, d_date_sk#19] (29) BroadcastExchange -Input [1]: [customer_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [customer_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (30) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#18] +Right keys [1]: [customer_sk#20] Join condition: None (31) Project [codegen id : 9] @@ -188,84 +188,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (32) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] +Output [2]: [ca_address_sk#22, ca_county#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (33) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_county#21] +Input [2]: [ca_address_sk#22, ca_county#23] (34) Filter [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) +Input [2]: [ca_address_sk#22, ca_county#23] +Condition : (ca_county#23 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#22)) (35) Project [codegen id : 7] -Output [1]: [ca_address_sk#20] -Input [2]: [ca_address_sk#20, ca_county#21] +Output [1]: [ca_address_sk#22] +Input [2]: [ca_address_sk#22, ca_county#23] (36) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [ca_address_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#22] Join condition: None (38) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#22] (39) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (41) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Condition : isnotnull(cd_demo_sk#23) +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Condition : isnotnull(cd_demo_sk#25) (42) BroadcastExchange -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (43) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#25] Join condition: None (44) Project [codegen id : 9] -Output [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (45) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] +Aggregate Attributes [1]: [count#35] +Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] (46) Exchange -Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] -Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] (47) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] -Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#36] -Results [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#36 AS cnt1#37, cd_purchase_estimate#27, count(1)#36 AS cnt2#38, cd_credit_rating#28, count(1)#36 AS cnt3#39, cd_dep_count#29, count(1)#36 AS cnt4#40, cd_dep_employed_count#30, count(1)#36 AS cnt5#41, cd_dep_college_count#31, count(1)#36 AS cnt6#42] +Aggregate Attributes [1]: [count(1)#38] +Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] (48) TakeOrderedAndProject -Input [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] -Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] +Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] +Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] ===== Subqueries ===== @@ -278,6 +278,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 14 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#17 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#6 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 baed833e42a47..8f7cbc7d0ffbc 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -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)] +PartitionFilters: [isnotnull(ss_sold_date_sk#61), (ss_sold_date_sk#61 >= 2451545), (ss_sold_date_sk#61 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -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] +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] (35) Filter [codegen id : 11] -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)) +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#62] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#61] +Right keys [1]: [d_date_sk#62] Join condition: None (38) Project [codegen id : 11] -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] +Output [7]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#62] (39) Scan parquet default.store -Output [2]: [s_store_sk#18, s_state#19] +Output [2]: [s_store_sk#63, s_state#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_state, [TN,AL,SD]), IsNotNull(s_store_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 8] -Input [2]: [s_store_sk#18, s_state#19] +Input [2]: [s_store_sk#63, s_state#64] (41) Filter [codegen id : 8] -Input [2]: [s_store_sk#18, s_state#19] -Condition : (s_state#19 IN (TN,AL,SD) AND isnotnull(s_store_sk#18)) +Input [2]: [s_store_sk#63, s_state#64] +Condition : (s_state#64 IN (TN,AL,SD) AND isnotnull(s_store_sk#63)) (42) Project [codegen id : 8] -Output [1]: [s_store_sk#18] -Input [2]: [s_store_sk#18, s_state#19] +Output [1]: [s_store_sk#63] +Input [2]: [s_store_sk#63, s_state#64] (43) BroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [s_store_sk#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] (44) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#56] +Right keys [1]: [s_store_sk#63] Join condition: None (45) Project [codegen id : 11] -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#18] +Output [6]: [ss_item_sk#54, ss_cdemo_sk#55, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, s_store_sk#63] (46) ReusedExchange [Reuses operator id: 15] -Output [1]: [cd_demo_sk#13] +Output [1]: [cd_demo_sk#66] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#13] +Left keys [1]: [ss_cdemo_sk#55] +Right keys [1]: [cd_demo_sk#66] Join condition: None (48) Project [codegen id : 11] -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#13] +Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [7]: [ss_item_sk#54, ss_cdemo_sk#55, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, cd_demo_sk#66] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#21, i_item_id#22] +Output [2]: [i_item_sk#67, i_item_id#68] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#67] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] +Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] +Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#22] +Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#68] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] +Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] (53) Exchange -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [1]: [i_item_id#22] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Keys [1]: [i_item_id#68] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] -Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] +Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] +Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] (55) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -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)] +PartitionFilters: [isnotnull(ss_sold_date_sk#103), (ss_sold_date_sk#103 >= 2451545), (ss_sold_date_sk#103 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 17] -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] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] (57) Filter [codegen id : 17] -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)) +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#104] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#103] +Right keys [1]: [d_date_sk#104] Join condition: None (60) Project [codegen id : 17] -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] +Output [7]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#104] (61) ReusedExchange [Reuses operator id: 43] -Output [1]: [s_store_sk#18] +Output [1]: [s_store_sk#105] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#98] +Right keys [1]: [s_store_sk#105] Join condition: None (63) Project [codegen id : 17] -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#18] +Output [6]: [ss_item_sk#96, ss_cdemo_sk#97, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#105] (64) ReusedExchange [Reuses operator id: 15] -Output [1]: [cd_demo_sk#13] +Output [1]: [cd_demo_sk#106] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#13] +Left keys [1]: [ss_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#106] Join condition: None (66) Project [codegen id : 17] -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#13] +Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [7]: [ss_item_sk#96, ss_cdemo_sk#97, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, cd_demo_sk#106] (67) Scan parquet default.item -Output [1]: [i_item_sk#21] +Output [1]: [i_item_sk#107] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#21] +Input [1]: [i_item_sk#107] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#21] -Condition : isnotnull(i_item_sk#21) +Input [1]: [i_item_sk#107] +Condition : isnotnull(i_item_sk#107) (70) BroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] +Input [1]: [i_item_sk#107] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#96] +Right keys [1]: [i_item_sk#107] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] +Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] +Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] -Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] +Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] (74) Exchange -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] -Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] +Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] +Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 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 735f763ae5d20..bfb9e8384817d 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -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)] +PartitionFilters: [isnotnull(ss_sold_date_sk#61), (ss_sold_date_sk#61 >= 2451545), (ss_sold_date_sk#61 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -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] +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] (35) Filter [codegen id : 11] -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)) +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#62] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#55] +Right keys [1]: [cd_demo_sk#62] Join condition: None (38) Project [codegen id : 11] -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] +Output [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, cd_demo_sk#62] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#63] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#61] +Right keys [1]: [d_date_sk#63] Join condition: None (41) Project [codegen id : 11] -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] +Output [6]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [8]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#63] (42) Scan parquet default.store -Output [2]: [s_store_sk#18, s_state#19] +Output [2]: [s_store_sk#64, s_state#65] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_state, [TN,AL,SD]), IsNotNull(s_store_sk)] ReadSchema: struct (43) ColumnarToRow [codegen id : 9] -Input [2]: [s_store_sk#18, s_state#19] +Input [2]: [s_store_sk#64, s_state#65] (44) Filter [codegen id : 9] -Input [2]: [s_store_sk#18, s_state#19] -Condition : (s_state#19 IN (TN,AL,SD) AND isnotnull(s_store_sk#18)) +Input [2]: [s_store_sk#64, s_state#65] +Condition : (s_state#65 IN (TN,AL,SD) AND isnotnull(s_store_sk#64)) (45) Project [codegen id : 9] -Output [1]: [s_store_sk#18] -Input [2]: [s_store_sk#18, s_state#19] +Output [1]: [s_store_sk#64] +Input [2]: [s_store_sk#64, s_state#65] (46) BroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [s_store_sk#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#66] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#56] +Right keys [1]: [s_store_sk#64] Join condition: None (48) Project [codegen id : 11] -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#18] +Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, s_store_sk#64] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#21, i_item_id#22] +Output [2]: [i_item_sk#67, i_item_id#68] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#67] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] +Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] +Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#22] +Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#68] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] +Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] (53) Exchange -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [1]: [i_item_id#22] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Keys [1]: [i_item_id#68] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] -Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] +Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] +Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] (55) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -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)] +PartitionFilters: [isnotnull(ss_sold_date_sk#103), (ss_sold_date_sk#103 >= 2451545), (ss_sold_date_sk#103 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 17] -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] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] (57) Filter [codegen id : 17] -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)) +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#104] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#104] Join condition: None (60) Project [codegen id : 17] -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] +Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] (61) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#105] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#103] +Right keys [1]: [d_date_sk#105] Join condition: None (63) Project [codegen id : 17] -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] +Output [6]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#105] (64) ReusedExchange [Reuses operator id: 46] -Output [1]: [s_store_sk#18] +Output [1]: [s_store_sk#106] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#98] +Right keys [1]: [s_store_sk#106] Join condition: None (66) Project [codegen id : 17] -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#18] +Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#106] (67) Scan parquet default.item -Output [1]: [i_item_sk#21] +Output [1]: [i_item_sk#107] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#21] +Input [1]: [i_item_sk#107] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#21] -Condition : isnotnull(i_item_sk#21) +Input [1]: [i_item_sk#107] +Condition : isnotnull(i_item_sk#107) (70) BroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] +Input [1]: [i_item_sk#107] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#96] +Right keys [1]: [i_item_sk#107] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] +Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] +Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] -Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] +Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] (74) Exchange -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] -Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] +Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] +Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#15] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt index ff33c0b00120b..ebc6009080bfa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt @@ -121,7 +121,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -129,7 +129,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = Unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = Unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt index 5ebb8e180e1e0..cd556c0bccb46 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = Unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = Unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt index 4eb212c336669..e62003f82faec 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt @@ -261,24 +261,24 @@ Input [7]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_add Arguments: [c_current_addr_sk#36 ASC NULLS FIRST], false, 0 (47) ReusedExchange [Reuses operator id: 30] -Output [2]: [ca_address_sk#22, ca_city#23] +Output [2]: [ca_address_sk#41, ca_city#42] (48) Sort [codegen id : 15] -Input [2]: [ca_address_sk#22, ca_city#23] -Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#41, ca_city#42] +Arguments: [ca_address_sk#41 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#36] -Right keys [1]: [ca_address_sk#22] -Join condition: NOT (ca_city#23 = bought_city#31) +Right keys [1]: [ca_address_sk#41] +Join condition: NOT (ca_city#42 = bought_city#31) (50) Project [codegen id : 16] -Output [7]: [c_last_name#38, c_first_name#37, ca_city#23, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#36, c_first_name#37, c_last_name#38, ca_address_sk#22, ca_city#23] +Output [7]: [c_last_name#38, c_first_name#37, ca_city#42, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#36, c_first_name#37, c_last_name#38, ca_address_sk#41, ca_city#42] (51) TakeOrderedAndProject -Input [7]: [c_last_name#38, c_first_name#37, ca_city#23, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Arguments: 100, [c_last_name#38 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, ca_city#23 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#38, c_first_name#37, ca_city#23, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [7]: [c_last_name#38, c_first_name#37, ca_city#42, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Arguments: 100, [c_last_name#38 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, ca_city#42 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#38, c_first_name#37, ca_city#42, bought_city#31, ss_ticket_number#5, amt#32, profit#33] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt index 951558ca1b130..a00e2eeac49b7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt @@ -225,20 +225,20 @@ Output [7]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_ad Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#31, amt#32, profit#33, c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#21, ca_city#22] +Output [2]: [ca_address_sk#39, ca_city#40] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#21] -Join condition: NOT (ca_city#22 = bought_city#31) +Right keys [1]: [ca_address_sk#39] +Join condition: NOT (ca_city#40 = bought_city#31) (42) Project [codegen id : 8] -Output [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#21, ca_city#22] +Output [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#39, ca_city#40] (43) TakeOrderedAndProject -Input [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#22 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#40 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt index 92db706d594f2..a3223d60c96be 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] (3) Filter [codegen id : 1] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 1] Output [2]: [i_item_sk#1, i_manufact_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (30) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#26, _w0#27, avg_quarterly_sales#29] -Condition : ((isnotnull(avg_quarterly_sales#29) AND (avg_quarterly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_quarterly_sales#29) AND ((avg_quarterly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (31) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#26, avg_quarterly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt index 8b29f7c168765..172e66022046d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] (3) Filter [codegen id : 4] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 4] Output [2]: [i_item_sk#1, i_manufact_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (30) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#26, _w0#27, avg_quarterly_sales#29] -Condition : ((isnotnull(avg_quarterly_sales#29) AND (avg_quarterly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_quarterly_sales#29) AND ((avg_quarterly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (31) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#26, avg_quarterly_sales#29] 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 4cdcd9685acbe..d20f7cf4a2857 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#1, ss_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3)] +PartitionFilters: [isnotnull(ss_sold_date_sk#56)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 6] -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] (28) Filter [codegen id : 6] -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] +Condition : isnotnull(ss_store_sk#54) (29) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] +Output [3]: [d_date_sk#57, d_week_seq#58, d_day_name#59] (30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#4] +Left keys [1]: [ss_sold_date_sk#56] +Right keys [1]: [d_date_sk#57] Join condition: None (31) Project [codegen id : 6] -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] +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] (32) HashAggregate [codegen id : 6] -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] +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] (33) Exchange -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] +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] (34) HashAggregate [codegen id : 9] -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))#67, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72] -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))#67,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72,17,2) AS sat_sales#36] +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] (35) Scan parquet default.store -Output [2]: [s_store_sk#37, s_store_id#38] +Output [2]: [s_store_sk#79, s_store_id#80] 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#37, s_store_id#38] +Input [2]: [s_store_sk#79, s_store_id#80] (37) Filter [codegen id : 7] -Input [2]: [s_store_sk#37, s_store_id#38] -Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) +Input [2]: [s_store_sk#79, s_store_id#80] +Condition : (isnotnull(s_store_sk#79) AND isnotnull(s_store_id#80)) (38) BroadcastExchange -Input [2]: [s_store_sk#37, s_store_id#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [s_store_sk#79, s_store_id#80] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] (39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#37] +Left keys [1]: [ss_store_sk#54] +Right keys [1]: [s_store_sk#79] Join condition: None (40) Project [codegen id : 9] -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#38] -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#37, s_store_id#38] +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] (41) Scan parquet default.date_dim -Output [2]: [d_month_seq#74, d_week_seq#75] +Output [2]: [d_month_seq#82, d_week_seq#83] 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#74, d_week_seq#75] +Input [2]: [d_month_seq#82, d_week_seq#83] (43) Filter [codegen id : 8] -Input [2]: [d_month_seq#74, d_week_seq#75] -Condition : (((isnotnull(d_month_seq#74) AND (d_month_seq#74 >= 1197)) AND (d_month_seq#74 <= 1208)) AND isnotnull(d_week_seq#75)) +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)) (44) Project [codegen id : 8] -Output [1]: [d_week_seq#75] -Input [2]: [d_month_seq#74, d_week_seq#75] +Output [1]: [d_week_seq#83] +Input [2]: [d_month_seq#82, d_week_seq#83] (45) BroadcastExchange -Input [1]: [d_week_seq#75] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] +Input [1]: [d_week_seq#83] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#84] (46) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#75] +Left keys [1]: [d_week_seq#58] +Right keys [1]: [d_week_seq#83] Join condition: None (47) Project [codegen id : 9] -Output [8]: [d_week_seq#5 AS d_week_seq2#77, s_store_id#38 AS s_store_id2#78, sun_sales#30 AS sun_sales2#79, mon_sales#31 AS mon_sales2#80, wed_sales#33 AS wed_sales2#81, thu_sales#34 AS thu_sales2#82, fri_sales#35 AS fri_sales2#83, sat_sales#36 AS sat_sales2#84] -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#38, d_week_seq#75] +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] (48) BroadcastExchange -Input [8]: [d_week_seq2#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#85] +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] (49) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#78, (d_week_seq2#77 - 52)] +Right keys [2]: [s_store_id2#86, (d_week_seq2#85 - 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#79)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#86, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#80)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#87, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#88, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#81)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#89, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#82)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#90, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#83)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#91, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#84)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#92] -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#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] +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] (51) TakeOrderedAndProject -Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] -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)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] +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] 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 4cdcd9685acbe..d20f7cf4a2857 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#1, ss_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3)] +PartitionFilters: [isnotnull(ss_sold_date_sk#56)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 6] -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] (28) Filter [codegen id : 6] -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] +Condition : isnotnull(ss_store_sk#54) (29) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] +Output [3]: [d_date_sk#57, d_week_seq#58, d_day_name#59] (30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#4] +Left keys [1]: [ss_sold_date_sk#56] +Right keys [1]: [d_date_sk#57] Join condition: None (31) Project [codegen id : 6] -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] +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] (32) HashAggregate [codegen id : 6] -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] +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] (33) Exchange -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] +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] (34) HashAggregate [codegen id : 9] -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))#67, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72] -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))#67,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72,17,2) AS sat_sales#36] +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] (35) Scan parquet default.store -Output [2]: [s_store_sk#37, s_store_id#38] +Output [2]: [s_store_sk#79, s_store_id#80] 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#37, s_store_id#38] +Input [2]: [s_store_sk#79, s_store_id#80] (37) Filter [codegen id : 7] -Input [2]: [s_store_sk#37, s_store_id#38] -Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) +Input [2]: [s_store_sk#79, s_store_id#80] +Condition : (isnotnull(s_store_sk#79) AND isnotnull(s_store_id#80)) (38) BroadcastExchange -Input [2]: [s_store_sk#37, s_store_id#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [s_store_sk#79, s_store_id#80] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] (39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#37] +Left keys [1]: [ss_store_sk#54] +Right keys [1]: [s_store_sk#79] Join condition: None (40) Project [codegen id : 9] -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#38] -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#37, s_store_id#38] +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] (41) Scan parquet default.date_dim -Output [2]: [d_month_seq#74, d_week_seq#75] +Output [2]: [d_month_seq#82, d_week_seq#83] 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#74, d_week_seq#75] +Input [2]: [d_month_seq#82, d_week_seq#83] (43) Filter [codegen id : 8] -Input [2]: [d_month_seq#74, d_week_seq#75] -Condition : (((isnotnull(d_month_seq#74) AND (d_month_seq#74 >= 1197)) AND (d_month_seq#74 <= 1208)) AND isnotnull(d_week_seq#75)) +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)) (44) Project [codegen id : 8] -Output [1]: [d_week_seq#75] -Input [2]: [d_month_seq#74, d_week_seq#75] +Output [1]: [d_week_seq#83] +Input [2]: [d_month_seq#82, d_week_seq#83] (45) BroadcastExchange -Input [1]: [d_week_seq#75] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] +Input [1]: [d_week_seq#83] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#84] (46) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#75] +Left keys [1]: [d_week_seq#58] +Right keys [1]: [d_week_seq#83] Join condition: None (47) Project [codegen id : 9] -Output [8]: [d_week_seq#5 AS d_week_seq2#77, s_store_id#38 AS s_store_id2#78, sun_sales#30 AS sun_sales2#79, mon_sales#31 AS mon_sales2#80, wed_sales#33 AS wed_sales2#81, thu_sales#34 AS thu_sales2#82, fri_sales#35 AS fri_sales2#83, sat_sales#36 AS sat_sales2#84] -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#38, d_week_seq#75] +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] (48) BroadcastExchange -Input [8]: [d_week_seq2#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#85] +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] (49) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#78, (d_week_seq2#77 - 52)] +Right keys [2]: [s_store_id2#86, (d_week_seq2#85 - 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#79)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#86, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#80)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#87, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#88, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#81)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#89, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#82)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#90, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#83)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#91, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#84)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#92] -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#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] +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] (51) TakeOrderedAndProject -Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] -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)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] +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] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt index 8e0aaf2c479b8..75cdaecea7595 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] (3) Filter [codegen id : 1] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 1] Output [2]: [i_item_sk#1, i_manager_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manager_id#5, specifiedwindowfram (30) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#26, _w0#27, avg_monthly_sales#29] -Condition : ((isnotnull(avg_monthly_sales#29) AND (avg_monthly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_monthly_sales#29) AND ((avg_monthly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (31) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#26, avg_monthly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt index 712ba76f2878a..c8e0821dfacda 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] (3) Filter [codegen id : 4] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 4] Output [2]: [i_item_sk#1, i_manager_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manager_id#5, specifiedwindowfram (30) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#26, _w0#27, avg_monthly_sales#29] -Condition : ((isnotnull(avg_monthly_sales#29) AND (avg_monthly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_monthly_sales#29) AND ((avg_monthly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (31) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#26, avg_monthly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt index 741a644c4fdbc..240f579025038 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt @@ -127,123 +127,123 @@ Input [4]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17 Condition : isnotnull(ss_store_sk#15) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#18] (19) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#17] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#18] Join condition: None (20) Project [codegen id : 4] Output [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] -Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#6] +Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#18] (21) HashAggregate [codegen id : 4] Input [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] +Aggregate Attributes [1]: [sum#19] +Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] (22) Exchange -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] -Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] +Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#21] (23) HashAggregate [codegen id : 5] -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#21] -Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#21,17,2) AS revenue#22] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#22] +Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#22,17,2) AS revenue#23] (24) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#15, revenue#22] +Input [2]: [ss_store_sk#15, revenue#23] Keys [1]: [ss_store_sk#15] -Functions [1]: [partial_avg(revenue#22)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ss_store_sk#15, sum#25, count#26] +Functions [1]: [partial_avg(revenue#23)] +Aggregate Attributes [2]: [sum#24, count#25] +Results [3]: [ss_store_sk#15, sum#26, count#27] (25) Exchange -Input [3]: [ss_store_sk#15, sum#25, count#26] -Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ss_store_sk#15, sum#26, count#27] +Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#28] (26) HashAggregate [codegen id : 6] -Input [3]: [ss_store_sk#15, sum#25, count#26] +Input [3]: [ss_store_sk#15, sum#26, count#27] Keys [1]: [ss_store_sk#15] -Functions [1]: [avg(revenue#22)] -Aggregate Attributes [1]: [avg(revenue#22)#28] -Results [2]: [ss_store_sk#15, avg(revenue#22)#28 AS ave#29] +Functions [1]: [avg(revenue#23)] +Aggregate Attributes [1]: [avg(revenue#23)#29] +Results [2]: [ss_store_sk#15, avg(revenue#23)#29 AS ave#30] (27) BroadcastExchange -Input [2]: [ss_store_sk#15, ave#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] +Input [2]: [ss_store_sk#15, ave#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (28) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_store_sk#2] Right keys [1]: [ss_store_sk#15] -Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#29)), DecimalType(23,7), true)) +Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#30)), DecimalType(23,7), true)) (29) Project [codegen id : 7] Output [3]: [ss_store_sk#2, ss_item_sk#1, revenue#13] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#29] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#30] (30) BroadcastExchange Input [3]: [ss_store_sk#2, ss_item_sk#1, revenue#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] (31) Scan parquet default.store -Output [2]: [s_store_sk#32, s_store_name#33] +Output [2]: [s_store_sk#33, s_store_name#34] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (32) ColumnarToRow -Input [2]: [s_store_sk#32, s_store_name#33] +Input [2]: [s_store_sk#33, s_store_name#34] (33) Filter -Input [2]: [s_store_sk#32, s_store_name#33] -Condition : isnotnull(s_store_sk#32) +Input [2]: [s_store_sk#33, s_store_name#34] +Condition : isnotnull(s_store_sk#33) (34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#32] +Right keys [1]: [s_store_sk#33] Join condition: None (35) Project [codegen id : 8] -Output [3]: [ss_item_sk#1, revenue#13, s_store_name#33] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#32, s_store_name#33] +Output [3]: [ss_item_sk#1, revenue#13, s_store_name#34] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#33, s_store_name#34] (36) BroadcastExchange -Input [3]: [ss_item_sk#1, revenue#13, s_store_name#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] +Input [3]: [ss_item_sk#1, revenue#13, s_store_name#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] (37) Scan parquet default.item -Output [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Output [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (38) ColumnarToRow -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] (39) Filter -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Condition : isnotnull(i_item_sk#35) +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Condition : isnotnull(i_item_sk#36) (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#35] +Right keys [1]: [i_item_sk#36] Join condition: None (41) Project [codegen id : 9] -Output [6]: [s_store_name#33, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Input [8]: [ss_item_sk#1, revenue#13, s_store_name#33, i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Output [6]: [s_store_name#34, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Input [8]: [ss_item_sk#1, revenue#13, s_store_name#34, i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] (42) TakeOrderedAndProject -Input [6]: [s_store_name#33, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Arguments: 100, [s_store_name#33 ASC NULLS FIRST, i_item_desc#36 ASC NULLS FIRST], [s_store_name#33, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [6]: [s_store_name#34, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Arguments: 100, [s_store_name#34 ASC NULLS FIRST, i_item_desc#37 ASC NULLS FIRST], [s_store_name#34, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt index 0851190e54fe0..43ef320ed235f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt @@ -181,65 +181,65 @@ Input [4]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26 Condition : isnotnull(ss_store_sk#24) (30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#27] (31) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#27] Join condition: None (32) Project [codegen id : 6] Output [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] -Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#8] +Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#27] (33) HashAggregate [codegen id : 6] Input [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#27] -Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] +Aggregate Attributes [1]: [sum#28] +Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] (34) Exchange -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] -Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] +Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#30] (35) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#30] -Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#30,17,2) AS revenue#31] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] +Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS revenue#32] (36) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#24, revenue#31] +Input [2]: [ss_store_sk#24, revenue#32] Keys [1]: [ss_store_sk#24] -Functions [1]: [partial_avg(revenue#31)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [3]: [ss_store_sk#24, sum#34, count#35] +Functions [1]: [partial_avg(revenue#32)] +Aggregate Attributes [2]: [sum#33, count#34] +Results [3]: [ss_store_sk#24, sum#35, count#36] (37) Exchange -Input [3]: [ss_store_sk#24, sum#34, count#35] -Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ss_store_sk#24, sum#35, count#36] +Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#37] (38) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#24, sum#34, count#35] +Input [3]: [ss_store_sk#24, sum#35, count#36] Keys [1]: [ss_store_sk#24] -Functions [1]: [avg(revenue#31)] -Aggregate Attributes [1]: [avg(revenue#31)#37] -Results [2]: [ss_store_sk#24, avg(revenue#31)#37 AS ave#38] +Functions [1]: [avg(revenue#32)] +Aggregate Attributes [1]: [avg(revenue#32)#38] +Results [2]: [ss_store_sk#24, avg(revenue#32)#38 AS ave#39] (39) BroadcastExchange -Input [2]: [ss_store_sk#24, ave#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Input [2]: [ss_store_sk#24, ave#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#24] -Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#38)), DecimalType(23,7), true)) +Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#39)), DecimalType(23,7), true)) (41) Project [codegen id : 9] Output [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#38] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#39] (42) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt index 10ac72a6b44de..0b96b76a0ffc4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt @@ -232,31 +232,31 @@ Input [8]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37 Arguments: HashedRelationBroadcastMode(List(cast(input[5, int, true] as bigint)),false), [id=#44] (41) Scan parquet default.customer_address -Output [2]: [ca_address_sk#23, ca_city#24] +Output [2]: [ca_address_sk#45, ca_city#46] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] ReadSchema: struct (42) ColumnarToRow -Input [2]: [ca_address_sk#23, ca_city#24] +Input [2]: [ca_address_sk#45, ca_city#46] (43) Filter -Input [2]: [ca_address_sk#23, ca_city#24] -Condition : (isnotnull(ca_address_sk#23) AND isnotnull(ca_city#24)) +Input [2]: [ca_address_sk#45, ca_city#46] +Condition : (isnotnull(ca_address_sk#45) AND isnotnull(ca_city#46)) (44) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#41] -Right keys [1]: [ca_address_sk#23] -Join condition: NOT (ca_city#24 = bought_city#35) +Right keys [1]: [ca_address_sk#45] +Join condition: NOT (ca_city#46 = bought_city#35) (45) Project [codegen id : 8] -Output [8]: [c_last_name#43, c_first_name#42, ca_city#24, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#41, c_first_name#42, c_last_name#43, ca_address_sk#23, ca_city#24] +Output [8]: [c_last_name#43, c_first_name#42, ca_city#46, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#41, c_first_name#42, c_last_name#43, ca_address_sk#45, ca_city#46] (46) TakeOrderedAndProject -Input [8]: [c_last_name#43, c_first_name#42, ca_city#24, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Arguments: 100, [c_last_name#43 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#43, c_first_name#42, ca_city#24, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [8]: [c_last_name#43, c_first_name#42, ca_city#46, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Arguments: 100, [c_last_name#43 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#43, c_first_name#42, ca_city#46, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt index 81faa98299f82..f50329db7e05e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt @@ -225,20 +225,20 @@ Output [8]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#3 Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_customer_sk#39, c_current_addr_sk#40, c_first_name#41, c_last_name#42] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#22, ca_city#23] +Output [2]: [ca_address_sk#44, ca_city#45] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#40] -Right keys [1]: [ca_address_sk#22] -Join condition: NOT (ca_city#23 = bought_city#35) +Right keys [1]: [ca_address_sk#44] +Join condition: NOT (ca_city#45 = bought_city#35) (42) Project [codegen id : 8] -Output [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#22, ca_city#23] +Output [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#44, ca_city#45] (43) TakeOrderedAndProject -Input [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt index fe4db3541cbd9..daa0ca7b16562 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt @@ -116,7 +116,7 @@ Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_p Output [4]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary ), IsNotNull(cd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -124,7 +124,7 @@ Input [4]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_stat (20) Filter [codegen id : 3] Input [4]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20] -Condition : ((((((isnotnull(cd_gender#18) AND isnotnull(cd_marital_status#19)) AND isnotnull(cd_education_status#20)) AND (cd_gender#18 = F)) AND (cd_marital_status#19 = W)) AND (cd_education_status#20 = Primary)) AND isnotnull(cd_demo_sk#17)) +Condition : ((((((isnotnull(cd_gender#18) AND isnotnull(cd_marital_status#19)) AND isnotnull(cd_education_status#20)) AND (cd_gender#18 = F)) AND (cd_marital_status#19 = W)) AND (cd_education_status#20 = Primary )) AND isnotnull(cd_demo_sk#17)) (21) Project [codegen id : 3] Output [1]: [cd_demo_sk#17] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt index c1f0c1bd01451..a9da9118323b6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnul Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = F)) AND (cd_marital_status#12 = W)) AND (cd_education_status#13 = Primary)) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = F)) AND (cd_marital_status#12 = W)) AND (cd_education_status#13 = Primary )) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt index 9dc7b54bf5bb6..c2909d9f5531f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt @@ -60,7 +60,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint) Output [4]: [hd_demo_sk#7, hd_buy_potential#8, hd_dep_count#9, hd_vehicle_count#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (6) ColumnarToRow @@ -68,7 +68,7 @@ Input [4]: [hd_demo_sk#7, hd_buy_potential#8, hd_dep_count#9, hd_vehicle_count#1 (7) Filter Input [4]: [hd_demo_sk#7, hd_buy_potential#8, hd_dep_count#9, hd_vehicle_count#10] -Condition : (((((isnotnull(hd_vehicle_count#10) AND isnotnull(hd_dep_count#9)) AND ((hd_buy_potential#8 = >10000) OR (hd_buy_potential#8 = Unknown))) AND (hd_vehicle_count#10 > 0)) AND ((cast(hd_dep_count#9 as double) / cast(hd_vehicle_count#10 as double)) > 1.0)) AND isnotnull(hd_demo_sk#7)) +Condition : (((((isnotnull(hd_vehicle_count#10) AND isnotnull(hd_dep_count#9)) AND ((hd_buy_potential#8 = >10000 ) OR (hd_buy_potential#8 = Unknown ))) AND (hd_vehicle_count#10 > 0)) AND ((cast(hd_dep_count#9 as double) / cast(hd_vehicle_count#10 as double)) > 1.0)) AND isnotnull(hd_demo_sk#7)) (8) Project Output [1]: [hd_demo_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt index 559ef93fb70ac..5ec772ca3d638 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = Unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = Unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt index 393b6a4da4255..a325bf820a4f9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt @@ -109,7 +109,7 @@ Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, d_moy#8, s_store_sk#1 Output [4]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(In(i_category, [Home,Books,Electronics]),In(i_class, [wallpaper,parenting,musical])),And(In(i_category, [Shoes,Jewelry,Men]),In(i_class, [womens,birdal,pants]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(In(i_category, [Home ,Books ,Electronics ]),In(i_class, [wallpaper ,parenting ,musical ])),And(In(i_category, [Shoes ,Jewelry ,Men ]),In(i_class, [womens ,birdal ,pants ]))), IsNotNull(i_item_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] @@ -117,7 +117,7 @@ Input [4]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17] (19) Filter [codegen id : 3] Input [4]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17] -Condition : (((i_category#17 IN (Home,Books,Electronics) AND i_class#16 IN (wallpaper,parenting,musical)) OR (i_category#17 IN (Shoes,Jewelry,Men) AND i_class#16 IN (womens,birdal,pants))) AND isnotnull(i_item_sk#14)) +Condition : (((i_category#17 IN (Home ,Books ,Electronics ) AND i_class#16 IN (wallpaper ,parenting ,musical )) OR (i_category#17 IN (Shoes ,Jewelry ,Men ) AND i_class#16 IN (womens ,birdal ,pants ))) AND isnotnull(i_item_sk#14)) (20) BroadcastExchange Input [4]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17] @@ -164,7 +164,7 @@ Arguments: [avg(_w0#24) windowspecdefinition(i_category#17, i_brand#15, s_store_ (29) Filter [codegen id : 7] Input [9]: [i_category#17, i_class#16, i_brand#15, s_store_name#11, s_company_name#12, d_moy#8, sum_sales#23, _w0#24, avg_monthly_sales#26] -Condition : ((isnotnull(avg_monthly_sales#26) AND NOT (avg_monthly_sales#26 = 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_monthly_sales#26) AND (NOT (avg_monthly_sales#26 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (30) Project [codegen id : 7] Output [8]: [i_category#17, i_class#16, i_brand#15, s_store_name#11, s_company_name#12, d_moy#8, sum_sales#23, avg_monthly_sales#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt index 0210877799a98..2ec1ea034361a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject (31) Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(In(i_category, [Home,Books,Electronics]),In(i_class, [wallpaper,parenting,musical])),And(In(i_category, [Shoes,Jewelry,Men]),In(i_class, [womens,birdal,pants]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(In(i_category, [Home ,Books ,Electronics ]),In(i_class, [wallpaper ,parenting ,musical ])),And(In(i_category, [Shoes ,Jewelry ,Men ]),In(i_class, [womens ,birdal ,pants ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -44,7 +44,7 @@ Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] (3) Filter [codegen id : 4] Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((i_category#4 IN (Home,Books,Electronics) AND i_class#3 IN (wallpaper,parenting,musical)) OR (i_category#4 IN (Shoes,Jewelry,Men) AND i_class#3 IN (womens,birdal,pants))) AND isnotnull(i_item_sk#1)) +Condition : (((i_category#4 IN (Home ,Books ,Electronics ) AND i_class#3 IN (wallpaper ,parenting ,musical )) OR (i_category#4 IN (Shoes ,Jewelry ,Men ) AND i_class#3 IN (womens ,birdal ,pants ))) AND isnotnull(i_item_sk#1)) (4) Scan parquet default.store_sales Output [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] @@ -164,7 +164,7 @@ Arguments: [avg(_w0#24) windowspecdefinition(i_category#4, i_brand#2, s_store_na (29) Filter [codegen id : 7] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, _w0#24, avg_monthly_sales#26] -Condition : ((isnotnull(avg_monthly_sales#26) AND NOT (avg_monthly_sales#26 = 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_monthly_sales#26) AND (NOT (avg_monthly_sales#26 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (30) Project [codegen id : 7] Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, avg_monthly_sales#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt index 09e8dbc57e336..9d369b5d211fc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt @@ -57,7 +57,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((((isnotnull(d_date#6) AND (d_date#6 >= 11323)) AND (d_date#6 <= 11353)) AND (d_date_sk#5 >= 2451911)) AND (d_date_sk#5 <= 2451941)) AND isnotnull(d_date_sk#5)) +Condition : (((((isnotnull(d_date#6) AND (d_date#6 >= 2001-01-01)) AND (d_date#6 <= 2001-01-31)) AND (d_date_sk#5 >= 2451911)) AND (d_date_sk#5 <= 2451941)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -88,7 +88,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Jewelry,Sports,Books]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Jewelry ,Sports ,Books ]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -96,7 +96,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Jewelry,Sports,Books) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Jewelry ,Sports ,Books ) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt index d46901132fcc3..e31c029b4932e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt @@ -46,7 +46,7 @@ Condition : isnotnull(ss_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Jewelry,Sports,Books]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Jewelry ,Sports ,Books ]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -54,7 +54,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Jewelry,Sports,Books) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Jewelry ,Sports ,Books ) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -81,7 +81,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((((isnotnull(d_date#13) AND (d_date#13 >= 11323)) AND (d_date#13 <= 11353)) AND (d_date_sk#12 >= 2451911)) AND (d_date_sk#12 <= 2451941)) AND isnotnull(d_date_sk#12)) +Condition : (((((isnotnull(d_date#13) AND (d_date#13 >= 2001-01-01)) AND (d_date#13 <= 2001-01-31)) AND (d_date_sk#12 >= 2451911)) AND (d_date_sk#12 <= 2451941)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] 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 9c68e55238aee..a950e7b9e8639 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 @@ -117,158 +117,158 @@ Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) (15) Scan parquet default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Output [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#5)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct (16) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] (17) Filter [codegen id : 4] -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) +Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] +Condition : isnotnull(sr_store_sk#17) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#20] (19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#4] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [sr_returned_date_sk#19] +Right keys [1]: [cast(d_date_sk#20 as bigint)] Join condition: None (20) Project [codegen id : 4] -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] +Output [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] +Input [5]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19, d_date_sk#20] (21) HashAggregate [codegen id : 4] -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#16] -Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] +Keys [2]: [sr_customer_sk#16, sr_store_sk#17] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#18))] +Aggregate Attributes [1]: [sum#21] +Results [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] (22) Exchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] +Arguments: hashpartitioning(sr_customer_sk#16, sr_store_sk#17, 5), ENSURE_REQUIREMENTS, [id=#23] (23) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] -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))#19] -Results [2]: [sr_store_sk#2 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#19,17,2) AS ctr_total_return#15] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] +Keys [2]: [sr_customer_sk#16, sr_store_sk#17] +Functions [1]: [sum(UnscaledValue(sr_return_amt#18))] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#18))#24] +Results [2]: [sr_store_sk#17 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#18))#24,17,2) AS ctr_total_return#15] (24) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#14, ctr_total_return#15] Keys [1]: [ctr_store_sk#14] Functions [1]: [partial_avg(ctr_total_return#15)] -Aggregate Attributes [2]: [sum#20, count#21] -Results [3]: [ctr_store_sk#14, sum#22, count#23] +Aggregate Attributes [2]: [sum#25, count#26] +Results [3]: [ctr_store_sk#14, sum#27, count#28] (25) Exchange -Input [3]: [ctr_store_sk#14, sum#22, count#23] -Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [ctr_store_sk#14, sum#27, count#28] +Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#29] (26) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#14, sum#22, count#23] +Input [3]: [ctr_store_sk#14, sum#27, count#28] Keys [1]: [ctr_store_sk#14] Functions [1]: [avg(ctr_total_return#15)] -Aggregate Attributes [1]: [avg(ctr_total_return#15)#25] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#25) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14 AS ctr_store_sk#14#27] +Aggregate Attributes [1]: [avg(ctr_total_return#15)#30] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14 AS ctr_store_sk#14#32] (27) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#26) +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) (28) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#28] +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] +Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#33] (29) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [ctr_store_sk#14#27] -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#26) +Right keys [1]: [ctr_store_sk#14#32] +Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) (30) Project [codegen id : 8] Output [2]: [ctr_customer_sk#13, ctr_store_sk#14] -Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] +Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] (31) Scan parquet default.store -Output [2]: [s_store_sk#29, s_state#30] +Output [2]: [s_store_sk#34, s_state#35] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#29, s_state#30] +Input [2]: [s_store_sk#34, s_state#35] (33) Filter [codegen id : 7] -Input [2]: [s_store_sk#29, s_state#30] -Condition : ((isnotnull(s_state#30) AND (s_state#30 = TN)) AND isnotnull(s_store_sk#29)) +Input [2]: [s_store_sk#34, s_state#35] +Condition : ((isnotnull(s_state#35) AND (s_state#35 = TN)) AND isnotnull(s_store_sk#34)) (34) Project [codegen id : 7] -Output [1]: [s_store_sk#29] -Input [2]: [s_store_sk#29, s_state#30] +Output [1]: [s_store_sk#34] +Input [2]: [s_store_sk#34, s_state#35] (35) BroadcastExchange -Input [1]: [s_store_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [1]: [s_store_sk#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (36) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [cast(s_store_sk#29 as bigint)] +Right keys [1]: [cast(s_store_sk#34 as bigint)] Join condition: None (37) Project [codegen id : 8] Output [1]: [ctr_customer_sk#13] -Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#29] +Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#34] (38) Exchange Input [1]: [ctr_customer_sk#13] -Arguments: hashpartitioning(ctr_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#32] +Arguments: hashpartitioning(ctr_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#37] (39) Sort [codegen id : 9] Input [1]: [ctr_customer_sk#13] Arguments: [ctr_customer_sk#13 ASC NULLS FIRST], false, 0 (40) Scan parquet default.customer -Output [2]: [c_customer_sk#33, c_customer_id#34] +Output [2]: [c_customer_sk#38, c_customer_id#39] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [2]: [c_customer_sk#33, c_customer_id#34] +Input [2]: [c_customer_sk#38, c_customer_id#39] (42) Filter [codegen id : 10] -Input [2]: [c_customer_sk#33, c_customer_id#34] -Condition : isnotnull(c_customer_sk#33) +Input [2]: [c_customer_sk#38, c_customer_id#39] +Condition : isnotnull(c_customer_sk#38) (43) Exchange -Input [2]: [c_customer_sk#33, c_customer_id#34] -Arguments: hashpartitioning(cast(c_customer_sk#33 as bigint), 5), ENSURE_REQUIREMENTS, [id=#35] +Input [2]: [c_customer_sk#38, c_customer_id#39] +Arguments: hashpartitioning(cast(c_customer_sk#38 as bigint), 5), ENSURE_REQUIREMENTS, [id=#40] (44) Sort [codegen id : 11] -Input [2]: [c_customer_sk#33, c_customer_id#34] -Arguments: [cast(c_customer_sk#33 as bigint) ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#38, c_customer_id#39] +Arguments: [cast(c_customer_sk#38 as bigint) ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 12] Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [cast(c_customer_sk#33 as bigint)] +Right keys [1]: [cast(c_customer_sk#38 as bigint)] Join condition: None (46) Project [codegen id : 12] -Output [1]: [c_customer_id#34] -Input [3]: [ctr_customer_sk#13, c_customer_sk#33, c_customer_id#34] +Output [1]: [c_customer_id#39] +Input [3]: [ctr_customer_sk#13, c_customer_sk#38, c_customer_id#39] (47) TakeOrderedAndProject -Input [1]: [c_customer_id#34] -Arguments: 100, [c_customer_id#34 ASC NULLS FIRST], [c_customer_id#34] +Input [1]: [c_customer_id#39] +Arguments: 100, [c_customer_id#39 ASC NULLS FIRST], [c_customer_id#39] ===== Subqueries ===== @@ -279,6 +279,6 @@ ReusedExchange (48) (48) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 15 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 15 Hosting Expression = sr_returned_date_sk#19 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 4487f04c8293d..015228686d39b 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 @@ -114,146 +114,146 @@ Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) (15) Scan parquet default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Output [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#5)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct (16) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] (17) Filter [codegen id : 4] -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) +Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] +Condition : isnotnull(sr_store_sk#17) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#20] (19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#4] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [sr_returned_date_sk#19] +Right keys [1]: [cast(d_date_sk#20 as bigint)] Join condition: None (20) Project [codegen id : 4] -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] +Output [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] +Input [5]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19, d_date_sk#20] (21) HashAggregate [codegen id : 4] -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#16] -Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] +Keys [2]: [sr_customer_sk#16, sr_store_sk#17] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#18))] +Aggregate Attributes [1]: [sum#21] +Results [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] (22) Exchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] +Arguments: hashpartitioning(sr_customer_sk#16, sr_store_sk#17, 5), ENSURE_REQUIREMENTS, [id=#23] (23) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] -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))#19] -Results [2]: [sr_store_sk#2 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#19,17,2) AS ctr_total_return#15] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] +Keys [2]: [sr_customer_sk#16, sr_store_sk#17] +Functions [1]: [sum(UnscaledValue(sr_return_amt#18))] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#18))#24] +Results [2]: [sr_store_sk#17 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#18))#24,17,2) AS ctr_total_return#15] (24) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#14, ctr_total_return#15] Keys [1]: [ctr_store_sk#14] Functions [1]: [partial_avg(ctr_total_return#15)] -Aggregate Attributes [2]: [sum#20, count#21] -Results [3]: [ctr_store_sk#14, sum#22, count#23] +Aggregate Attributes [2]: [sum#25, count#26] +Results [3]: [ctr_store_sk#14, sum#27, count#28] (25) Exchange -Input [3]: [ctr_store_sk#14, sum#22, count#23] -Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [ctr_store_sk#14, sum#27, count#28] +Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#29] (26) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#14, sum#22, count#23] +Input [3]: [ctr_store_sk#14, sum#27, count#28] Keys [1]: [ctr_store_sk#14] Functions [1]: [avg(ctr_total_return#15)] -Aggregate Attributes [1]: [avg(ctr_total_return#15)#25] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#25) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14 AS ctr_store_sk#14#27] +Aggregate Attributes [1]: [avg(ctr_total_return#15)#30] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14 AS ctr_store_sk#14#32] (27) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#26) +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) (28) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#28] +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] +Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#33] (29) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [ctr_store_sk#14#27] -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#26) +Right keys [1]: [ctr_store_sk#14#32] +Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) (30) Project [codegen id : 9] Output [2]: [ctr_customer_sk#13, ctr_store_sk#14] -Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] +Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] (31) Scan parquet default.store -Output [2]: [s_store_sk#29, s_state#30] +Output [2]: [s_store_sk#34, s_state#35] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#29, s_state#30] +Input [2]: [s_store_sk#34, s_state#35] (33) Filter [codegen id : 7] -Input [2]: [s_store_sk#29, s_state#30] -Condition : ((isnotnull(s_state#30) AND (s_state#30 = TN)) AND isnotnull(s_store_sk#29)) +Input [2]: [s_store_sk#34, s_state#35] +Condition : ((isnotnull(s_state#35) AND (s_state#35 = TN)) AND isnotnull(s_store_sk#34)) (34) Project [codegen id : 7] -Output [1]: [s_store_sk#29] -Input [2]: [s_store_sk#29, s_state#30] +Output [1]: [s_store_sk#34] +Input [2]: [s_store_sk#34, s_state#35] (35) BroadcastExchange -Input [1]: [s_store_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [1]: [s_store_sk#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [cast(s_store_sk#29 as bigint)] +Right keys [1]: [cast(s_store_sk#34 as bigint)] Join condition: None (37) Project [codegen id : 9] Output [1]: [ctr_customer_sk#13] -Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#29] +Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#34] (38) Scan parquet default.customer -Output [2]: [c_customer_sk#32, c_customer_id#33] +Output [2]: [c_customer_sk#37, c_customer_id#38] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 8] -Input [2]: [c_customer_sk#32, c_customer_id#33] +Input [2]: [c_customer_sk#37, c_customer_id#38] (40) Filter [codegen id : 8] -Input [2]: [c_customer_sk#32, c_customer_id#33] -Condition : isnotnull(c_customer_sk#32) +Input [2]: [c_customer_sk#37, c_customer_id#38] +Condition : isnotnull(c_customer_sk#37) (41) BroadcastExchange -Input [2]: [c_customer_sk#32, c_customer_id#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [2]: [c_customer_sk#37, c_customer_id#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#39] (42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [cast(c_customer_sk#32 as bigint)] +Right keys [1]: [cast(c_customer_sk#37 as bigint)] Join condition: None (43) Project [codegen id : 9] -Output [1]: [c_customer_id#33] -Input [3]: [ctr_customer_sk#13, c_customer_sk#32, c_customer_id#33] +Output [1]: [c_customer_id#38] +Input [3]: [ctr_customer_sk#13, c_customer_sk#37, c_customer_id#38] (44) TakeOrderedAndProject -Input [1]: [c_customer_id#33] -Arguments: 100, [c_customer_id#33 ASC NULLS FIRST], [c_customer_id#33] +Input [1]: [c_customer_id#38] +Arguments: 100, [c_customer_id#38 ASC NULLS FIRST], [c_customer_id#38] ===== Subqueries ===== @@ -264,6 +264,6 @@ ReusedExchange (45) (45) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 15 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 15 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt index c25973f527384..2dd0a13e74a74 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt @@ -143,20 +143,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#17] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#10] +Right keys [1]: [d_date_sk#17] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] -Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] +Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] (23) Exchange Input [1]: [ws_bill_customer_sk#15] -Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] @@ -168,38 +168,38 @@ Right keys [1]: [ws_bill_customer_sk#15] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Output [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#21] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [cs_sold_date_sk#20] +Right keys [1]: [d_date_sk#21] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#18] -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] +Output [1]: [cs_ship_customer_sk#19] +Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] (31) Exchange -Input [1]: [cs_ship_customer_sk#18] -Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [cs_ship_customer_sk#19] +Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#18] -Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#19] +Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#19] Join condition: None (34) Filter [codegen id : 13] @@ -211,96 +211,96 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (36) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_county#22] +Output [2]: [ca_address_sk#23, ca_county#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 12] -Input [2]: [ca_address_sk#21, ca_county#22] +Input [2]: [ca_address_sk#23, ca_county#24] (38) Filter [codegen id : 12] -Input [2]: [ca_address_sk#21, ca_county#22] -Condition : (ca_county#22 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#21)) +Input [2]: [ca_address_sk#23, ca_county#24] +Condition : (ca_county#24 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#23)) (39) Project [codegen id : 12] -Output [1]: [ca_address_sk#21] -Input [2]: [ca_address_sk#21, ca_county#22] +Output [1]: [ca_address_sk#23] +Input [2]: [ca_address_sk#23, ca_county#24] (40) BroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [ca_address_sk#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (41) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#23] Join condition: None (42) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#21] +Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#23] (43) Exchange Input [1]: [c_current_cdemo_sk#4] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#26] (44) Sort [codegen id : 14] Input [1]: [c_current_cdemo_sk#4] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 (45) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (46) ColumnarToRow [codegen id : 15] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] (47) Filter [codegen id : 15] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Condition : isnotnull(cd_demo_sk#25) +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Condition : isnotnull(cd_demo_sk#27) (48) Exchange -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: hashpartitioning(cd_demo_sk#25, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Arguments: hashpartitioning(cd_demo_sk#27, 5), ENSURE_REQUIREMENTS, [id=#36] (49) Sort [codegen id : 16] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: [cd_demo_sk#25 ASC NULLS FIRST], false, 0 +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Arguments: [cd_demo_sk#27 ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 17] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#27] Join condition: None (51) Project [codegen id : 17] -Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] (52) HashAggregate [codegen id : 17] -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#35] -Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Aggregate Attributes [1]: [count#37] +Results [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] (53) Exchange -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] +Arguments: hashpartitioning(cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, 5), ENSURE_REQUIREMENTS, [id=#39] (54) HashAggregate [codegen id : 18] -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] +Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] +Aggregate Attributes [1]: [count(1)#40] +Results [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, count(1)#40 AS cnt1#41, cd_purchase_estimate#31, count(1)#40 AS cnt2#42, cd_credit_rating#32, count(1)#40 AS cnt3#43, cd_dep_count#33, count(1)#40 AS cnt4#44, cd_dep_employed_count#34, count(1)#40 AS cnt5#45, cd_dep_college_count#35, count(1)#40 AS cnt6#46] (55) TakeOrderedAndProject -Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] +Input [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#41, cd_purchase_estimate#31, cnt2#42, cd_credit_rating#32, cnt3#43, cd_dep_count#33, cnt4#44, cd_dep_employed_count#34, cnt5#45, cd_dep_college_count#35, cnt6#46] +Arguments: 100, [cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_education_status#30 ASC NULLS FIRST, cd_purchase_estimate#31 ASC NULLS FIRST, cd_credit_rating#32 ASC NULLS FIRST, cd_dep_count#33 ASC NULLS FIRST, cd_dep_employed_count#34 ASC NULLS FIRST, cd_dep_college_count#35 ASC NULLS FIRST], [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#41, cd_purchase_estimate#31, cnt2#42, cd_credit_rating#32, cnt3#43, cd_dep_count#33, cnt4#44, cd_dep_employed_count#34, cnt5#45, cd_dep_college_count#35, cnt6#46] ===== Subqueries ===== @@ -313,6 +313,6 @@ Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index 07bcd309e9ce6..370551a2de652 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -123,20 +123,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#16] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] -Right keys [1]: [d_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] +Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#16] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] @@ -144,34 +144,34 @@ Right keys [1]: [ws_bill_customer_sk#14] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#8)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#20] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#9] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#20] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#17] -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] +Output [1]: [cs_ship_customer_sk#18] +Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#20] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [cs_ship_customer_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#17] +Right keys [1]: [cs_ship_customer_sk#18] Join condition: None (29) Filter [codegen id : 9] @@ -183,84 +183,84 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] +Output [2]: [ca_address_sk#22, ca_county#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_county#21] +Input [2]: [ca_address_sk#22, ca_county#23] (33) Filter [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#20)) +Input [2]: [ca_address_sk#22, ca_county#23] +Condition : (ca_county#23 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#22)) (34) Project [codegen id : 7] -Output [1]: [ca_address_sk#20] -Input [2]: [ca_address_sk#20, ca_county#21] +Output [1]: [ca_address_sk#22] +Input [2]: [ca_address_sk#22, ca_county#23] (35) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [ca_address_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#22] Join condition: None (37) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] +Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22] (38) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (40) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Condition : isnotnull(cd_demo_sk#23) +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Condition : isnotnull(cd_demo_sk#25) (41) BroadcastExchange -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#25] Join condition: None (43) Project [codegen id : 9] -Output [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (44) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] +Aggregate Attributes [1]: [count#35] +Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] (45) Exchange -Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] -Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] (46) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] -Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#36] -Results [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#36 AS cnt1#37, cd_purchase_estimate#27, count(1)#36 AS cnt2#38, cd_credit_rating#28, count(1)#36 AS cnt3#39, cd_dep_count#29, count(1)#36 AS cnt4#40, cd_dep_employed_count#30, count(1)#36 AS cnt5#41, cd_dep_college_count#31, count(1)#36 AS cnt6#42] +Aggregate Attributes [1]: [count(1)#38] +Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] (47) TakeOrderedAndProject -Input [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] -Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] +Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] +Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] ===== Subqueries ===== @@ -273,6 +273,6 @@ Output [1]: [d_date_sk#9] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 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 da2143dc20a90..92e8e38634ea6 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 @@ -200,289 +200,289 @@ Input [2]: [customer_id#23, year_total#24] Arguments: [customer_id#23 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] (27) Filter [codegen id : 10] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_customer_sk#1) +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] +Condition : isnotnull(ss_customer_sk#26) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#31, d_year#32] 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 (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#6, d_year#7] +Input [2]: [d_date_sk#31, d_year#32] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#31, d_year#32] +Condition : ((isnotnull(d_year#32) AND (d_year#32 = 2002)) AND isnotnull(d_date_sk#31)) (31) BroadcastExchange -Input [2]: [d_date_sk#6, d_year#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [2]: [d_date_sk#31, d_year#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ss_sold_date_sk#29] +Right keys [1]: [d_date_sk#31] Join condition: None (33) Project [codegen id : 10] -Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4, d_date_sk#6, d_year#7] +Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Input [6]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29, d_date_sk#31, d_year#32] (34) Exchange -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#34] (35) Sort [codegen id : 11] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] (37) Sort [codegen id : 13] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Arguments: [c_customer_sk#35 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#35] Join condition: None (39) Project [codegen id : 14] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Input [12]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Input [12]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32, c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] (40) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#29] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] +Input [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#43] +Results [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] (41) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] +Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, 5), ENSURE_REQUIREMENTS, [id=#45] (42) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32] -Results [3]: [c_customer_id#11 AS customer_id#33, c_preferred_cust_flag#14 AS customer_preferred_cust_flag#34, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32,18,2) AS year_total#35] +Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] +Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46] +Results [3]: [c_customer_id#36 AS customer_id#47, c_preferred_cust_flag#39 AS customer_preferred_cust_flag#48, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46,18,2) AS year_total#49] (43) Exchange -Input [3]: [customer_id#33, customer_preferred_cust_flag#34, year_total#35] -Arguments: hashpartitioning(customer_id#33, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] +Arguments: hashpartitioning(customer_id#47, 5), ENSURE_REQUIREMENTS, [id=#50] (44) Sort [codegen id : 16] -Input [3]: [customer_id#33, customer_preferred_cust_flag#34, year_total#35] -Arguments: [customer_id#33 ASC NULLS FIRST], false, 0 +Input [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] +Arguments: [customer_id#47 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#33] +Right keys [1]: [customer_id#47] Join condition: None (46) Project [codegen id : 17] -Output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35] -Input [5]: [customer_id#23, year_total#24, customer_id#33, customer_preferred_cust_flag#34, year_total#35] +Output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49] +Input [5]: [customer_id#23, year_total#24, customer_id#47, customer_preferred_cust_flag#48, year_total#49] (47) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 19] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] (49) Filter [codegen id : 19] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_customer_sk#37) +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) (50) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#55, d_year#56] (51) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#54] +Right keys [1]: [d_date_sk#55] Join condition: None (52) Project [codegen id : 19] -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Input [6]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#6, d_year#7] +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] (53) Exchange -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Arguments: hashpartitioning(ws_bill_customer_sk#37, 5), ENSURE_REQUIREMENTS, [id=#41] +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] (54) Sort [codegen id : 20] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Arguments: [ws_bill_customer_sk#37 ASC NULLS FIRST], false, 0 +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 (55) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +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] (56) Sort [codegen id : 22] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +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 (57) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#37] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ws_bill_customer_sk#51] +Right keys [1]: [c_customer_sk#58] Join condition: None (58) Project [codegen id : 23] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Input [12]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +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] (59) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#42] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#43] +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] (60) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#43] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#44] +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] (61) HashAggregate [codegen id : 24] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#43] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45] -Results [2]: [c_customer_id#11 AS customer_id#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45,18,2) AS year_total#47] +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] (62) Filter [codegen id : 24] -Input [2]: [customer_id#46, year_total#47] -Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) +Input [2]: [customer_id#70, year_total#71] +Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.00)) (63) Project [codegen id : 24] -Output [2]: [customer_id#46 AS customer_id#48, year_total#47 AS year_total#49] -Input [2]: [customer_id#46, year_total#47] +Output [2]: [customer_id#70 AS customer_id#72, year_total#71 AS year_total#73] +Input [2]: [customer_id#70, year_total#71] (64) Exchange -Input [2]: [customer_id#48, year_total#49] -Arguments: hashpartitioning(customer_id#48, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [customer_id#72, year_total#73] +Arguments: hashpartitioning(customer_id#72, 5), ENSURE_REQUIREMENTS, [id=#74] (65) Sort [codegen id : 25] -Input [2]: [customer_id#48, year_total#49] -Arguments: [customer_id#48 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#72, year_total#73] +Arguments: [customer_id#72 ASC NULLS FIRST], false, 0 (66) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#48] +Right keys [1]: [customer_id#72] Join condition: None (67) Project [codegen id : 26] -Output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35, year_total#49] -Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35, customer_id#48, year_total#49] +Output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73] +Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, customer_id#72, year_total#73] (68) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ws_sold_date_sk#78), dynamicpruningexpression(ws_sold_date_sk#78 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] (70) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_customer_sk#37) +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] +Condition : isnotnull(ws_bill_customer_sk#75) (71) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#79, d_year#80] (72) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#78] +Right keys [1]: [d_date_sk#79] Join condition: None (73) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Input [6]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#6, d_year#7] +Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Input [6]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78, d_date_sk#79, d_year#80] (74) Exchange -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Arguments: hashpartitioning(ws_bill_customer_sk#37, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Arguments: hashpartitioning(ws_bill_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] (75) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Arguments: [ws_bill_customer_sk#37 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Arguments: [ws_bill_customer_sk#75 ASC NULLS FIRST], false, 0 (76) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] (77) Sort [codegen id : 31] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] +Arguments: [c_customer_sk#82 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#37] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ws_bill_customer_sk#75] +Right keys [1]: [c_customer_sk#82] Join condition: None (79) Project [codegen id : 32] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Input [12]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Input [12]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80, c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] (80) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#52] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#53] +Input [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#90] +Results [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] (81) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#53] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] +Arguments: hashpartitioning(c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, 5), ENSURE_REQUIREMENTS, [id=#92] (82) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#53] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55] -Results [2]: [c_customer_id#11 AS customer_id#56, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55,18,2) AS year_total#57] +Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] +Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93] +Results [2]: [c_customer_id#83 AS customer_id#94, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93,18,2) AS year_total#95] (83) Exchange -Input [2]: [customer_id#56, year_total#57] -Arguments: hashpartitioning(customer_id#56, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [customer_id#94, year_total#95] +Arguments: hashpartitioning(customer_id#94, 5), ENSURE_REQUIREMENTS, [id=#96] (84) Sort [codegen id : 34] -Input [2]: [customer_id#56, year_total#57] -Arguments: [customer_id#56 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#94, year_total#95] +Arguments: [customer_id#94 ASC NULLS FIRST], false, 0 (85) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#56] -Join condition: (CASE WHEN (year_total#49 > 0.00) THEN CheckOverflow((promote_precision(year_total#57) / promote_precision(year_total#49)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#35) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE null END) +Right keys [1]: [customer_id#94] +Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#95) / promote_precision(year_total#73)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#49) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE null END) (86) Project [codegen id : 35] -Output [1]: [customer_preferred_cust_flag#34] -Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35, year_total#49, customer_id#56, year_total#57] +Output [1]: [customer_preferred_cust_flag#48] +Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73, customer_id#94, year_total#95] (87) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#34] -Arguments: 100, [customer_preferred_cust_flag#34 ASC NULLS FIRST], [customer_preferred_cust_flag#34] +Input [1]: [customer_preferred_cust_flag#48] +Arguments: 100, [customer_preferred_cust_flag#48 ASC NULLS FIRST], [customer_preferred_cust_flag#48] ===== Subqueries ===== @@ -493,15 +493,15 @@ ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#6, d_year#7] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#26 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#30 ReusedExchange (89) (89) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#31, d_year#32] -Subquery:3 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#26 +Subquery:4 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#78 IN dynamicpruning#30 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 5a5249a3aa81b..4081f69732168 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 @@ -172,286 +172,286 @@ Input [2]: [customer_id#22, year_total#23] Condition : (isnotnull(year_total#23) AND (year_total#23 > 0.00)) (20) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] (22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_customer_id#25)) (23) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Output [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ss_sold_date_sk#35), dynamicpruningexpression(ss_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] (25) Filter [codegen id : 4] -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_customer_sk#9) +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Condition : isnotnull(ss_customer_sk#32) (26) BroadcastExchange -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#9] +Left keys [1]: [c_customer_sk#24] +Right keys [1]: [ss_customer_sk#32] Join condition: None (28) Project [codegen id : 6] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Input [12]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#38, d_year#39] 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 (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#15, d_year#16] +Input [2]: [d_date_sk#38, d_year#39] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#15, d_year#16] -Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2002)) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#38, d_year#39] +Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) (32) BroadcastExchange -Input [2]: [d_date_sk#15, d_year#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +Input [2]: [d_date_sk#38, d_year#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#35] +Right keys [1]: [d_date_sk#38] Join condition: None (34) Project [codegen id : 6] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#15, d_year#16] +Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] +Input [12]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#38, d_year#39] (35) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#27] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] +Input [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] +Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#41] +Results [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] (36) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] +Arguments: hashpartitioning(c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, 5), ENSURE_REQUIREMENTS, [id=#43] (37) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30] -Results [3]: [c_customer_id#2 AS customer_id#31, c_preferred_cust_flag#5 AS customer_preferred_cust_flag#32, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30,18,2) AS year_total#33] +Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] +Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44] +Results [3]: [c_customer_id#25 AS customer_id#45, c_preferred_cust_flag#28 AS customer_preferred_cust_flag#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44,18,2) AS year_total#47] (38) BroadcastExchange -Input [3]: [customer_id#31, customer_preferred_cust_flag#32, year_total#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] +Input [3]: [customer_id#45, customer_preferred_cust_flag#46, year_total#47] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#31] +Right keys [1]: [customer_id#45] Join condition: None (40) Project [codegen id : 16] -Output [4]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33] -Input [5]: [customer_id#22, year_total#23, customer_id#31, customer_preferred_cust_flag#32, year_total#33] +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] (41) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +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] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (42) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +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] (43) Filter [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +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)) (44) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Output [4]: [ws_bill_customer_sk#57, ws_ext_discount_amt#58, ws_ext_list_price#59, ws_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (45) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Input [4]: [ws_bill_customer_sk#57, ws_ext_discount_amt#58, ws_ext_list_price#59, ws_sold_date_sk#60] (46) Filter [codegen id : 8] -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_bill_customer_sk#35) +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) (47) BroadcastExchange -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#39] +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] (48) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#35] +Left keys [1]: [c_customer_sk#49] +Right keys [1]: [ws_bill_customer_sk#57] Join condition: None (49) Project [codegen id : 10] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +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] (50) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#62, d_year#63] (51) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ws_sold_date_sk#60] +Right keys [1]: [d_date_sk#62] Join condition: None (52) Project [codegen id : 10] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38, d_date_sk#15, d_year#16] +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] (53) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#40] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#41] +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] (54) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#41] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#42] +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] (55) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#41] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#43] -Results [2]: [c_customer_id#2 AS customer_id#44, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#43,18,2) AS year_total#45] +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] (56) Filter [codegen id : 11] -Input [2]: [customer_id#44, year_total#45] -Condition : (isnotnull(year_total#45) AND (year_total#45 > 0.00)) +Input [2]: [customer_id#68, year_total#69] +Condition : (isnotnull(year_total#69) AND (year_total#69 > 0.00)) (57) Project [codegen id : 11] -Output [2]: [customer_id#44 AS customer_id#46, year_total#45 AS year_total#47] -Input [2]: [customer_id#44, year_total#45] +Output [2]: [customer_id#68 AS customer_id#70, year_total#69 AS year_total#71] +Input [2]: [customer_id#68, year_total#69] (58) BroadcastExchange -Input [2]: [customer_id#46, year_total#47] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] +Input [2]: [customer_id#70, year_total#71] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#72] (59) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#46] +Right keys [1]: [customer_id#70] Join condition: None (60) Project [codegen id : 16] -Output [5]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33, year_total#47] -Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33, customer_id#46, year_total#47] +Output [5]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, year_total#71] +Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, customer_id#70, year_total#71] (61) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (62) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] (63) Filter [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_customer_id#74)) (64) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Output [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ws_sold_date_sk#84), dynamicpruningexpression(ws_sold_date_sk#84 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] (66) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_bill_customer_sk#35) +Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Condition : isnotnull(ws_bill_customer_sk#81) (67) BroadcastExchange -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#85] (68) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#35] +Left keys [1]: [c_customer_sk#73] +Right keys [1]: [ws_bill_customer_sk#81] Join condition: None (69) Project [codegen id : 14] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Output [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Input [12]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] (70) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#86, d_year#87] (71) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ws_sold_date_sk#84] +Right keys [1]: [d_date_sk#86] Join condition: None (72) Project [codegen id : 14] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38, d_date_sk#15, d_year#16] +Output [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, d_year#87] +Input [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84, d_date_sk#86, d_year#87] (73) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#50] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#51] +Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, d_year#87] +Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#88] +Results [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] (74) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#51] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] +Arguments: hashpartitioning(c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, 5), ENSURE_REQUIREMENTS, [id=#90] (75) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#51] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#53] -Results [2]: [c_customer_id#2 AS customer_id#54, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#53,18,2) AS year_total#55] +Input [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] +Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))#91] +Results [2]: [c_customer_id#74 AS customer_id#92, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))#91,18,2) AS year_total#93] (76) BroadcastExchange -Input [2]: [customer_id#54, year_total#55] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] +Input [2]: [customer_id#92, year_total#93] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#94] (77) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#54] -Join condition: (CASE WHEN (year_total#47 > 0.00) THEN CheckOverflow((promote_precision(year_total#55) / promote_precision(year_total#47)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#33) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE null END) +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 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) (78) Project [codegen id : 16] -Output [1]: [customer_preferred_cust_flag#32] -Input [7]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33, year_total#47, customer_id#54, year_total#55] +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#71, customer_id#92, year_total#93] (79) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#32] -Arguments: 100, [customer_preferred_cust_flag#32 ASC NULLS FIRST], [customer_preferred_cust_flag#32] +Input [1]: [customer_preferred_cust_flag#46] +Arguments: 100, [customer_preferred_cust_flag#46 ASC NULLS FIRST], [customer_preferred_cust_flag#46] ===== Subqueries ===== @@ -462,15 +462,15 @@ ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#15, d_year#16] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#24 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 ReusedExchange (81) (81) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#38, d_year#39] -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#24 +Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#84 IN dynamicpruning#36 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt index 4e258bb08b1d8..039a38acb9f22 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt @@ -55,7 +55,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -86,7 +86,7 @@ Arguments: [ws_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -94,7 +94,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt index 0ae1f849bca42..0c22f6333e5a9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt @@ -44,7 +44,7 @@ Condition : isnotnull(ws_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -52,7 +52,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -79,7 +79,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt index e400eea2cea66..55dca59270306 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt @@ -58,7 +58,7 @@ Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnot Output [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -66,7 +66,7 @@ Input [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] (6) Filter [codegen id : 1] Input [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] -Condition : (isnotnull(cd_demo_sk#12) AND ((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree)) OR ((cd_marital_status#13 = S) AND (cd_education_status#14 = College))) OR ((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree)))) +Condition : (isnotnull(cd_demo_sk#12) AND ((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree )) OR ((cd_marital_status#13 = S) AND (cd_education_status#14 = College ))) OR ((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree )))) (7) BroadcastExchange Input [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] @@ -75,7 +75,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (8) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#12] -Join condition: ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) +Join condition: ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) (9) Project [codegen id : 6] Output [11]: [ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, cd_marital_status#13, cd_education_status#14] @@ -102,7 +102,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (14) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#16] -Join condition: (((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#17 = 3)) OR (((((cd_marital_status#13 = S) AND (cd_education_status#14 = College)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#17 = 1))) OR (((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#17 = 1))) +Join condition: (((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#17 = 3)) OR (((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#17 = 1))) OR (((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#17 = 1))) (15) Project [codegen id : 6] Output [7]: [ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt index 560738591c930..027fde38ff7fa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt @@ -147,7 +147,7 @@ Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ex Output [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct (25) ColumnarToRow [codegen id : 4] @@ -155,7 +155,7 @@ Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] (26) Filter [codegen id : 4] Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] -Condition : (isnotnull(cd_demo_sk#21) AND ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) OR ((cd_marital_status#22 = S) AND (cd_education_status#23 = College))) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)))) +Condition : (isnotnull(cd_demo_sk#21) AND ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) OR ((cd_marital_status#22 = S) AND (cd_education_status#23 = College ))) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )))) (27) BroadcastExchange Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] @@ -164,7 +164,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (28) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#21] -Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) +Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) (29) Project [codegen id : 6] Output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23] @@ -191,7 +191,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (34) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#25] -Join condition: (((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#26 = 3)) OR (((((cd_marital_status#22 = S) AND (cd_education_status#23 = College)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#26 = 1))) OR (((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#26 = 1))) +Join condition: (((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#26 = 3)) OR (((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#26 = 1))) OR (((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#26 = 1))) (35) Project [codegen id : 6] Output [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt index 6033ad52be4e9..e4ec487623d2c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt @@ -173,580 +173,580 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (10) ColumnarToRow [codegen id : 10] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] (11) Filter [codegen id : 10] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_item_sk#11) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#14, d_year#15] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#14, d_year#15] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1999)) AND (d_year#15 <= 2001)) AND isnotnull(d_date_sk#14)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#14] +Input [2]: [d_date_sk#14, d_year#15] (16) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (17) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#14] Join condition: None (18) Project [codegen id : 10] -Output [1]: [ss_item_sk#1] -Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] +Output [1]: [ss_item_sk#11] +Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] (19) Scan parquet default.item -Output [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Condition : (((isnotnull(i_item_sk#15) AND isnotnull(i_brand_id#16)) AND isnotnull(i_class_id#17)) AND isnotnull(i_category_id#18)) +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) (22) Exchange -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: hashpartitioning(coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] (23) Sort [codegen id : 5] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: [coalesce(i_brand_id#16, 0) ASC NULLS FIRST, isnull(i_brand_id#16) ASC NULLS FIRST, coalesce(i_class_id#17, 0) ASC NULLS FIRST, isnull(i_class_id#17) ASC NULLS FIRST, coalesce(i_category_id#18, 0) ASC NULLS FIRST, isnull(i_category_id#18) ASC NULLS FIRST], false, 0 +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 (24) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] +Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 8] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] (26) Filter [codegen id : 8] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#22) (27) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#24] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None (29) Project [codegen id : 8] -Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#22] +Output [1]: [cs_item_sk#22] +Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] (30) Scan parquet default.item -Output [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] (32) Filter [codegen id : 7] -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -Condition : isnotnull(i_item_sk#23) +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Condition : isnotnull(i_item_sk#25) (33) BroadcastExchange -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#23] +Left keys [1]: [cs_item_sk#22] +Right keys [1]: [i_item_sk#25] Join condition: None (35) Project [codegen id : 8] -Output [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Input [5]: [cs_item_sk#20, i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] (36) Exchange -Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: hashpartitioning(coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26), 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] (37) Sort [codegen id : 9] -Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: [coalesce(i_brand_id#24, 0) ASC NULLS FIRST, isnull(i_brand_id#24) ASC NULLS FIRST, coalesce(i_class_id#25, 0) ASC NULLS FIRST, isnull(i_class_id#25) ASC NULLS FIRST, coalesce(i_category_id#26, 0) ASC NULLS FIRST, isnull(i_category_id#26) ASC NULLS FIRST], false, 0 +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 (38) SortMergeJoin -Left keys [6]: [coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18)] -Right keys [6]: [coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26)] +Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] Join condition: None (39) BroadcastExchange -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] +Left keys [1]: [ss_item_sk#11] +Right keys [1]: [i_item_sk#17] Join condition: None (41) Project [codegen id : 10] -Output [3]: [i_brand_id#16 AS brand_id#30, i_class_id#17 AS class_id#31, i_category_id#18 AS category_id#32] -Input [5]: [ss_item_sk#1, i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] +Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (42) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#32, class_id#33, category_id#34] (43) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] (44) HashAggregate [codegen id : 11] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#32, class_id#33, category_id#34] (45) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32), 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] (46) Sort [codegen id : 12] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: [coalesce(brand_id#30, 0) ASC NULLS FIRST, isnull(brand_id#30) ASC NULLS FIRST, coalesce(class_id#31, 0) ASC NULLS FIRST, isnull(class_id#31) ASC NULLS FIRST, coalesce(category_id#32, 0) ASC NULLS FIRST, isnull(category_id#32) ASC NULLS FIRST], false, 0 +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 15] -Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] (49) Filter [codegen id : 15] -Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#35) +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_item_sk#37) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#37] +Output [1]: [d_date_sk#39] (51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#37] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#39] Join condition: None (52) Project [codegen id : 15] -Output [1]: [ws_item_sk#35] -Input [3]: [ws_item_sk#35, ws_sold_date_sk#36, d_date_sk#37] +Output [1]: [ws_item_sk#37] +Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] (53) ReusedExchange [Reuses operator id: 33] -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] (54) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [i_item_sk#38] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#40] Join condition: None (55) Project [codegen id : 15] -Output [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Input [5]: [ws_item_sk#35, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] (56) Exchange -Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: hashpartitioning(coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41), 5), ENSURE_REQUIREMENTS, [id=#42] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] (57) Sort [codegen id : 16] -Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: [coalesce(i_brand_id#39, 0) ASC NULLS FIRST, isnull(i_brand_id#39) ASC NULLS FIRST, coalesce(i_class_id#40, 0) ASC NULLS FIRST, isnull(i_class_id#40) ASC NULLS FIRST, coalesce(i_category_id#41, 0) ASC NULLS FIRST, isnull(i_category_id#41) ASC NULLS FIRST], false, 0 +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] -Right keys [6]: [coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41)] +Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] +Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] Join condition: None (59) HashAggregate [codegen id : 17] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#32, class_id#33, category_id#34] (60) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] (61) HashAggregate [codegen id : 18] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#32, class_id#33, category_id#34] (62) BroadcastExchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#44] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] (63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#30, class_id#31, category_id#32] +Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join condition: None (64) Project [codegen id : 19] -Output [1]: [i_item_sk#7 AS ss_item_sk#45] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#30, class_id#31, category_id#32] +Output [1]: [i_item_sk#7 AS ss_item_sk#47] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] (65) Exchange -Input [1]: [ss_item_sk#45] -Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [1]: [ss_item_sk#47] +Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] (66) Sort [codegen id : 20] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#47] Join condition: None (68) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#47] +Output [3]: [d_date_sk#49, d_year#50, d_moy#51] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 21] -Input [3]: [d_date_sk#12, d_year#13, d_moy#47] +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] (70) Filter [codegen id : 21] -Input [3]: [d_date_sk#12, d_year#13, d_moy#47] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#47)) AND (d_year#13 = 2001)) AND (d_moy#47 = 11)) AND isnotnull(d_date_sk#12)) +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] +Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2001)) AND (d_moy#51 = 11)) AND isnotnull(d_date_sk#49)) (71) Project [codegen id : 21] -Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#47] +Output [1]: [d_date_sk#49] +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] (72) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] +Input [1]: [d_date_sk#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] (73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#49] Join condition: None (74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] (75) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 22] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] (77) Filter [codegen id : 22] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Condition : isnotnull(i_item_sk#53) (78) Exchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: hashpartitioning(i_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#57] (79) Sort [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: [i_item_sk#53 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#47] (81) Sort [codegen id : 41] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#45] +Left keys [1]: [i_item_sk#53] +Right keys [1]: [ss_item_sk#47] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] (84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#53] Join condition: None (85) Project [codegen id : 42] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] (86) HashAggregate [codegen id : 42] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] +Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#51, isEmpty#52, count#53] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] +Aggregate Attributes [3]: [sum#59, isEmpty#60, count#61] +Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] +Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#65] (88) HashAggregate [codegen id : 43] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] +Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66, count(1)#67] +Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] (89) Filter [codegen id : 43] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(32,6)) > cast(Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (90) Project [codegen id : 43] -Output [6]: [sales#60, number_sales#61, store AS channel#65, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] +Output [6]: [sales#68, number_sales#69, store AS channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] (91) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Output [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#77), dynamicpruningexpression(cs_sold_date_sk#77 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 44] -Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] (93) Filter [codegen id : 44] -Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Condition : isnotnull(cs_item_sk#74) (94) Exchange -Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] -Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Arguments: hashpartitioning(cs_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#78] (95) Sort [codegen id : 45] -Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] -Arguments: [cs_item_sk#20 ASC NULLS FIRST], false, 0 +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Arguments: [cs_item_sk#74 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#47] (97) Sort [codegen id : 63] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [ss_item_sk#45] +Left keys [1]: [cs_item_sk#74] +Right keys [1]: [ss_item_sk#47] Join condition: None (99) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#69] +Output [1]: [d_date_sk#79] (100) BroadcastHashJoin [codegen id : 85] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#69] +Left keys [1]: [cs_sold_date_sk#77] +Right keys [1]: [d_date_sk#79] Join condition: None (101) Project [codegen id : 85] -Output [3]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67] -Input [5]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21, d_date_sk#69] +Output [3]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76] +Input [5]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77, d_date_sk#79] (102) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73] +Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] (103) BroadcastHashJoin [codegen id : 85] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#70] +Left keys [1]: [cs_item_sk#74] +Right keys [1]: [i_item_sk#80] Join condition: None (104) Project [codegen id : 85] -Output [5]: [cs_quantity#66, cs_list_price#67, i_brand_id#71, i_class_id#72, i_category_id#73] -Input [7]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73] +Output [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] +Input [7]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] (105) HashAggregate [codegen id : 85] -Input [5]: [cs_quantity#66, cs_list_price#67, i_brand_id#71, i_class_id#72, i_category_id#73] -Keys [3]: [i_brand_id#71, i_class_id#72, i_category_id#73] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#74, isEmpty#75, count#76] -Results [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] +Input [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] +Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] +Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] (106) Exchange -Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] -Arguments: hashpartitioning(i_brand_id#71, i_class_id#72, i_category_id#73, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, [id=#90] (107) HashAggregate [codegen id : 86] -Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] -Keys [3]: [i_brand_id#71, i_class_id#72, i_category_id#73] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81, count(1)#82] -Results [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sales#83, count(1)#82 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] +Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] (108) Filter [codegen id : 86] -Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (109) Project [codegen id : 86] -Output [6]: [sales#83, number_sales#84, catalog AS channel#86, i_brand_id#71, i_class_id#72, i_category_id#73] -Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] +Output [6]: [sales#93, number_sales#94, catalog AS channel#96, i_brand_id#81, i_class_id#82, i_category_id#83] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] (110) Scan parquet default.web_sales -Output [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Output [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#100), dynamicpruningexpression(ws_sold_date_sk#100 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (111) ColumnarToRow [codegen id : 87] -Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] (112) Filter [codegen id : 87] -Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#35) +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Condition : isnotnull(ws_item_sk#97) (113) Exchange -Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] -Arguments: hashpartitioning(ws_item_sk#35, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Arguments: hashpartitioning(ws_item_sk#97, 5), ENSURE_REQUIREMENTS, [id=#101] (114) Sort [codegen id : 88] -Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] -Arguments: [ws_item_sk#35 ASC NULLS FIRST], false, 0 +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Arguments: [ws_item_sk#97 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#47] (116) Sort [codegen id : 106] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (117) SortMergeJoin -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [ss_item_sk#45] +Left keys [1]: [ws_item_sk#97] +Right keys [1]: [ss_item_sk#47] Join condition: None (118) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#90] +Output [1]: [d_date_sk#102] (119) BroadcastHashJoin [codegen id : 128] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#90] +Left keys [1]: [ws_sold_date_sk#100] +Right keys [1]: [d_date_sk#102] Join condition: None (120) Project [codegen id : 128] -Output [3]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88] -Input [5]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36, d_date_sk#90] +Output [3]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99] +Input [5]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100, d_date_sk#102] (121) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94] +Output [4]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] (122) BroadcastHashJoin [codegen id : 128] -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [i_item_sk#91] +Left keys [1]: [ws_item_sk#97] +Right keys [1]: [i_item_sk#103] Join condition: None (123) Project [codegen id : 128] -Output [5]: [ws_quantity#87, ws_list_price#88, i_brand_id#92, i_class_id#93, i_category_id#94] -Input [7]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94] +Output [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] +Input [7]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] (124) HashAggregate [codegen id : 128] -Input [5]: [ws_quantity#87, ws_list_price#88, i_brand_id#92, i_class_id#93, i_category_id#94] -Keys [3]: [i_brand_id#92, i_class_id#93, i_category_id#94] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#95, isEmpty#96, count#97] -Results [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] +Input [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] +Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#107, isEmpty#108, count#109] +Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] (125) Exchange -Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] -Arguments: hashpartitioning(i_brand_id#92, i_class_id#93, i_category_id#94, 5), ENSURE_REQUIREMENTS, [id=#101] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +Arguments: hashpartitioning(i_brand_id#104, i_class_id#105, i_category_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] (126) HashAggregate [codegen id : 129] -Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] -Keys [3]: [i_brand_id#92, i_class_id#93, i_category_id#94] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102, count(1)#103] -Results [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sales#104, count(1)#103 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114, count(1)#115] +Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sales#116, count(1)#115 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] (127) Filter [codegen id : 129] -Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (128) Project [codegen id : 129] -Output [6]: [sales#104, number_sales#105, web AS channel#107, i_brand_id#92, i_class_id#93, i_category_id#94] -Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] +Output [6]: [sales#116, number_sales#117, web AS channel#119, i_brand_id#104, i_class_id#105, i_category_id#106] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] (129) Union (130) Expand [codegen id : 130] -Input [6]: [sales#60, number_sales#61, channel#65, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [List(sales#60, number_sales#61, channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, 0), List(sales#60, number_sales#61, channel#65, i_brand_id#8, i_class_id#9, null, 1), List(sales#60, number_sales#61, channel#65, i_brand_id#8, null, null, 3), List(sales#60, number_sales#61, channel#65, null, null, null, 7), List(sales#60, number_sales#61, null, null, null, null, 15)], [sales#60, number_sales#61, channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112] +Input [6]: [sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: [List(sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, 0), List(sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, null, 1), List(sales#68, number_sales#69, channel#73, i_brand_id#54, null, null, 3), List(sales#68, number_sales#69, channel#73, null, null, null, 7), List(sales#68, number_sales#69, null, null, null, null, 15)], [sales#68, number_sales#69, channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] (131) HashAggregate [codegen id : 130] -Input [7]: [sales#60, number_sales#61, channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112] -Keys [5]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112] -Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] -Aggregate Attributes [3]: [sum#113, isEmpty#114, sum#115] -Results [8]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112, sum#116, isEmpty#117, sum#118] +Input [7]: [sales#68, number_sales#69, channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] +Keys [5]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#125, isEmpty#126, sum#127] +Results [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] (132) Exchange -Input [8]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112, sum#116, isEmpty#117, sum#118] -Arguments: hashpartitioning(channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112, 5), ENSURE_REQUIREMENTS, [id=#119] +Input [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] +Arguments: hashpartitioning(channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, 5), ENSURE_REQUIREMENTS, [id=#131] (133) HashAggregate [codegen id : 131] -Input [8]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112, sum#116, isEmpty#117, sum#118] -Keys [5]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112] -Functions [2]: [sum(sales#60), sum(number_sales#61)] -Aggregate Attributes [2]: [sum(sales#60)#120, sum(number_sales#61)#121] -Results [6]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, sum(sales#60)#120 AS sum(sales)#122, sum(number_sales#61)#121 AS sum(number_sales)#123] +Input [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] +Keys [5]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#132, sum(number_sales#69)#133] +Results [6]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales#68)#132 AS sum(sales)#134, sum(number_sales#69)#133 AS sum(number_sales)#135] (134) TakeOrderedAndProject -Input [6]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, sum(sales)#122, sum(number_sales)#123] -Arguments: 100, [channel#108 ASC NULLS FIRST, i_brand_id#109 ASC NULLS FIRST, i_class_id#110 ASC NULLS FIRST, i_category_id#111 ASC NULLS FIRST], [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, sum(sales)#122, sum(number_sales)#123] +Input [6]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales)#134, sum(number_sales)#135] +Arguments: 100, [channel#120 ASC NULLS FIRST, i_brand_id#121 ASC NULLS FIRST, i_class_id#122 ASC NULLS FIRST, i_category_id#123 ASC NULLS FIRST], [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales)#134, sum(number_sales)#135] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#63, [id=#64] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#71, [id=#72] * HashAggregate (157) +- Exchange (156) +- * HashAggregate (155) @@ -773,145 +773,145 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (135) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#124)] +PartitionFilters: [isnotnull(ss_sold_date_sk#138), dynamicpruningexpression(ss_sold_date_sk#138 IN dynamicpruning#139)] ReadSchema: struct (136) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138] (137) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#140, d_year#141] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (138) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#140, d_year#141] (139) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#140, d_year#141] +Condition : (((isnotnull(d_year#141) AND (d_year#141 >= 1999)) AND (d_year#141 <= 2001)) AND isnotnull(d_date_sk#140)) (140) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#140] +Input [2]: [d_date_sk#140, d_year#141] (141) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#125] +Input [1]: [d_date_sk#140] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#142] (142) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#138] +Right keys [1]: [d_date_sk#140] Join condition: None (143) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#126, ss_list_price#3 AS list_price#127] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#136 AS quantity#143, ss_list_price#137 AS list_price#144] +Input [4]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138, d_date_sk#140] (144) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Output [3]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#124)] +PartitionFilters: [isnotnull(cs_sold_date_sk#147), dynamicpruningexpression(cs_sold_date_sk#147 IN dynamicpruning#139)] ReadSchema: struct (145) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Input [3]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147] (146) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#128] +Output [1]: [d_date_sk#148] (147) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#128] +Left keys [1]: [cs_sold_date_sk#147] +Right keys [1]: [d_date_sk#148] Join condition: None (148) Project [codegen id : 4] -Output [2]: [cs_quantity#66 AS quantity#129, cs_list_price#67 AS list_price#130] -Input [4]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21, d_date_sk#128] +Output [2]: [cs_quantity#145 AS quantity#149, cs_list_price#146 AS list_price#150] +Input [4]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147, d_date_sk#148] (149) Scan parquet default.web_sales -Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Output [3]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#124)] +PartitionFilters: [isnotnull(ws_sold_date_sk#153), dynamicpruningexpression(ws_sold_date_sk#153 IN dynamicpruning#139)] ReadSchema: struct (150) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Input [3]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153] (151) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#131] +Output [1]: [d_date_sk#154] (152) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#131] +Left keys [1]: [ws_sold_date_sk#153] +Right keys [1]: [d_date_sk#154] Join condition: None (153) Project [codegen id : 6] -Output [2]: [ws_quantity#87 AS quantity#132, ws_list_price#88 AS list_price#133] -Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36, d_date_sk#131] +Output [2]: [ws_quantity#151 AS quantity#155, ws_list_price#152 AS list_price#156] +Input [4]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153, d_date_sk#154] (154) Union (155) HashAggregate [codegen id : 7] -Input [2]: [quantity#126, list_price#127] +Input [2]: [quantity#143, list_price#144] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#126 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#127 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#134, count#135] -Results [2]: [sum#136, count#137] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#157, count#158] +Results [2]: [sum#159, count#160] (156) Exchange -Input [2]: [sum#136, count#137] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#138] +Input [2]: [sum#159, count#160] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#161] (157) HashAggregate [codegen id : 8] -Input [2]: [sum#136, count#137] +Input [2]: [sum#159, count#160] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#126 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#127 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#126 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#127 as decimal(12,2)))), DecimalType(18,2), true))#139] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#126 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#127 as decimal(12,2)))), DecimalType(18,2), true))#139 AS average_sales#140] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))#162] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))#162 AS average_sales#163] -Subquery:2 Hosting operator id = 135 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#124 +Subquery:2 Hosting operator id = 135 Hosting Expression = ss_sold_date_sk#138 IN dynamicpruning#139 ReusedExchange (158) (158) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#140] -Subquery:3 Hosting operator id = 144 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#124 +Subquery:3 Hosting operator id = 144 Hosting Expression = cs_sold_date_sk#147 IN dynamicpruning#139 -Subquery:4 Hosting operator id = 149 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#124 +Subquery:4 Hosting operator id = 149 Hosting Expression = ws_sold_date_sk#153 IN dynamicpruning#139 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (159) (159) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#49] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 ReusedExchange (160) (160) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#14] -Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 -Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#77 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 34aa615cf910c..6f61fc8e96ae1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -146,512 +146,512 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_item_sk#10) (10) Scan parquet default.item -Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (11) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] (12) Filter [codegen id : 4] -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14)) +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) (13) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] (15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#15) +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Condition : isnotnull(cs_item_sk#17) (16) Scan parquet default.item -Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] (18) Filter [codegen id : 1] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Condition : isnotnull(i_item_sk#17) +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Condition : isnotnull(i_item_sk#19) (19) BroadcastExchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] (20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#15] -Right keys [1]: [i_item_sk#17] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#19] Join condition: None (21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20] -Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] (22) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_year#23] +Output [2]: [d_date_sk#24, d_year#25] 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 (23) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#22, d_year#23] +Input [2]: [d_date_sk#24, d_year#25] (24) Filter [codegen id : 2] -Input [2]: [d_date_sk#22, d_year#23] -Condition : (((isnotnull(d_year#23) AND (d_year#23 >= 1999)) AND (d_year#23 <= 2001)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) (25) Project [codegen id : 2] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_year#23] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_year#25] (26) 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#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (27) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] Join condition: None (28) Project [codegen id : 3] -Output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] -Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#22] +Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] (29) BroadcastExchange -Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] (30) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)] -Right keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join condition: None (31) BroadcastExchange -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] (32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#11] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] Join condition: None (33) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] (34) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#29] (35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#29] Join condition: None (36) Project [codegen id : 6] -Output [3]: [i_brand_id#12 AS brand_id#28, i_class_id#13 AS class_id#29, i_category_id#14 AS category_id#30] -Input [5]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#27] +Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#30, class_id#31, category_id#32] (38) Exchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#30, class_id#31, category_id#32] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_item_sk#32) +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_item_sk#34) (43) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#34] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#36] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37] -Input [6]: [ws_item_sk#32, ws_sold_date_sk#33, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] +Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] (46) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#38] +Output [1]: [d_date_sk#40] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#40] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] -Input [5]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] +Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] (49) BroadcastExchange -Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#39] +Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] -Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#30, class_id#31, category_id#32] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#30, class_id#31, category_id#32] (53) BroadcastExchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#40] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#28, class_id#29, category_id#30] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#41] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] +Output [1]: [i_item_sk#6 AS ss_item_sk#43] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] (56) BroadcastExchange -Input [1]: [ss_item_sk#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [ss_item_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#41] +Right keys [1]: [ss_item_sk#43] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Condition : isnotnull(i_item_sk#45) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#43] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#41] +Left keys [1]: [i_item_sk#45] +Right keys [1]: [ss_item_sk#43] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#45] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (66) Scan parquet default.date_dim -Output [3]: [d_date_sk#27, d_year#44, d_moy#45] +Output [3]: [d_date_sk#50, d_year#51, d_moy#52] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [3]: [d_date_sk#27, d_year#44, d_moy#45] +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] (68) Filter [codegen id : 24] -Input [3]: [d_date_sk#27, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#27)) +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] +Condition : ((((isnotnull(d_year#51) AND isnotnull(d_moy#52)) AND (d_year#51 = 2001)) AND (d_moy#52 = 11)) AND isnotnull(d_date_sk#50)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#27] -Input [3]: [d_date_sk#27, d_year#44, d_moy#45] +Output [1]: [d_date_sk#50] +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] (70) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Input [1]: [d_date_sk#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#50] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#27] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#47, isEmpty#48, count#49] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] +Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] +Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#60] (75) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] (76) Filter [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [sales#56, number_sales#57, store AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] +Output [6]: [sales#63, number_sales#64, store AS channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] (78) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Output [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] (80) Filter [codegen id : 51] -Input [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#15) +Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +Condition : isnotnull(cs_item_sk#69) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#43] (82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#15] -Right keys [1]: [ss_item_sk#41] +Left keys [1]: [cs_item_sk#69] +Right keys [1]: [ss_item_sk#43] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +Output [4]: [i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] (84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#15] -Right keys [1]: [i_item_sk#64] +Left keys [1]: [cs_item_sk#69] +Right keys [1]: [i_item_sk#73] Join condition: None (85) Project [codegen id : 51] -Output [6]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [8]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +Output [6]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [8]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] (86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#68] +Output [1]: [d_date_sk#77] (87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#68] +Left keys [1]: [cs_sold_date_sk#72] +Right keys [1]: [d_date_sk#77] Join condition: None (88) Project [codegen id : 51] -Output [5]: [cs_quantity#62, cs_list_price#63, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [7]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_brand_id#65, i_class_id#66, i_category_id#67, d_date_sk#68] +Output [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [7]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76, d_date_sk#77] (89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#62, cs_list_price#63, i_brand_id#65, i_class_id#66, i_category_id#67] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] -Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] +Input [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] +Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#78, isEmpty#79, count#80] +Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] (90) Exchange -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Arguments: hashpartitioning(i_brand_id#65, i_class_id#66, i_category_id#67, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Arguments: hashpartitioning(i_brand_id#74, i_class_id#75, i_category_id#76, 5), ENSURE_REQUIREMENTS, [id=#84] (91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76, count(1)#77] -Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sales#78, count(1)#77 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85, count(1)#86] +Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sales#87, count(1)#86 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] (92) Filter [codegen id : 52] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [sales#78, number_sales#79, catalog AS channel#81, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] +Output [6]: [sales#87, number_sales#88, catalog AS channel#90, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] (94) Scan parquet default.web_sales -Output [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Output [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] (96) Filter [codegen id : 77] -Input [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] -Condition : isnotnull(ws_item_sk#32) +Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Condition : isnotnull(ws_item_sk#91) (97) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#43] (98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [ss_item_sk#41] +Left keys [1]: [ws_item_sk#91] +Right keys [1]: [ss_item_sk#43] Join condition: None (99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] +Output [4]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] (100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#84] +Left keys [1]: [ws_item_sk#91] +Right keys [1]: [i_item_sk#95] Join condition: None (101) Project [codegen id : 77] -Output [6]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_brand_id#85, i_class_id#86, i_category_id#87] -Input [8]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] +Output [6]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [8]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] (102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#88] +Output [1]: [d_date_sk#99] (103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#88] +Left keys [1]: [ws_sold_date_sk#94] +Right keys [1]: [d_date_sk#99] Join condition: None (104) Project [codegen id : 77] -Output [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#85, i_class_id#86, i_category_id#87] -Input [7]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_brand_id#85, i_class_id#86, i_category_id#87, d_date_sk#88] +Output [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [7]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98, d_date_sk#99] (105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#85, i_class_id#86, i_category_id#87] -Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#89, isEmpty#90, count#91] -Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Input [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] +Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#100, isEmpty#101, count#102] +Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] (106) Exchange -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] -Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, 5), ENSURE_REQUIREMENTS, [id=#95] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Arguments: hashpartitioning(i_brand_id#96, i_class_id#97, i_category_id#98, 5), ENSURE_REQUIREMENTS, [id=#106] (107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] -Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96, count(1)#97] -Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sales#98, count(1)#97 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] +Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#109, count(1)#108 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] (108) Filter [codegen id : 78] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [sales#98, number_sales#99, web AS channel#101, i_brand_id#85, i_class_id#86, i_category_id#87] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] +Output [6]: [sales#109, number_sales#110, web AS channel#112, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] (110) Union (111) Expand [codegen id : 79] -Input [6]: [sales#56, number_sales#57, channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: [List(sales#56, number_sales#57, channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#56, number_sales#57, channel#61, i_brand_id#7, i_class_id#8, null, 1), List(sales#56, number_sales#57, channel#61, i_brand_id#7, null, null, 3), List(sales#56, number_sales#57, channel#61, null, null, null, 7), List(sales#56, number_sales#57, null, null, null, null, 15)], [sales#56, number_sales#57, channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] +Input [6]: [sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] +Arguments: [List(sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, 0), List(sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, null, 1), List(sales#63, number_sales#64, channel#68, i_brand_id#46, null, null, 3), List(sales#63, number_sales#64, channel#68, null, null, null, 7), List(sales#63, number_sales#64, null, null, null, null, 15)], [sales#63, number_sales#64, channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] (112) HashAggregate [codegen id : 79] -Input [7]: [sales#56, number_sales#57, channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] -Keys [5]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] -Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] -Aggregate Attributes [3]: [sum#107, isEmpty#108, sum#109] -Results [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] +Input [7]: [sales#63, number_sales#64, channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] +Keys [5]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#118, isEmpty#119, sum#120] +Results [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] (113) Exchange -Input [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] -Arguments: hashpartitioning(channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] +Input [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] +Arguments: hashpartitioning(channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, 5), ENSURE_REQUIREMENTS, [id=#124] (114) HashAggregate [codegen id : 80] -Input [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] -Keys [5]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] -Functions [2]: [sum(sales#56), sum(number_sales#57)] -Aggregate Attributes [2]: [sum(sales#56)#114, sum(number_sales#57)#115] -Results [6]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales#56)#114 AS sum(sales)#116, sum(number_sales#57)#115 AS sum(number_sales)#117] +Input [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] +Keys [5]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#125, sum(number_sales#64)#126] +Results [6]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales#63)#125 AS sum(sales)#127, sum(number_sales#64)#126 AS sum(number_sales)#128] (115) TakeOrderedAndProject -Input [6]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales)#116, sum(number_sales)#117] -Arguments: 100, [channel#102 ASC NULLS FIRST, i_brand_id#103 ASC NULLS FIRST, i_class_id#104 ASC NULLS FIRST, i_category_id#105 ASC NULLS FIRST], [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales)#116, sum(number_sales)#117] +Input [6]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales)#127, sum(number_sales)#128] +Arguments: 100, [channel#113 ASC NULLS FIRST, i_brand_id#114 ASC NULLS FIRST, i_class_id#115 ASC NULLS FIRST, i_category_id#116 ASC NULLS FIRST], [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales)#127, sum(number_sales)#128] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#59, [id=#60] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#66, [id=#67] * HashAggregate (138) +- Exchange (137) +- * HashAggregate (136) @@ -678,145 +678,145 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (116) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#118)] +PartitionFilters: [isnotnull(ss_sold_date_sk#131), dynamicpruningexpression(ss_sold_date_sk#131 IN dynamicpruning#132)] ReadSchema: struct (117) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131] (118) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_year#44] +Output [2]: [d_date_sk#133, d_year#134] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (119) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#44] +Input [2]: [d_date_sk#133, d_year#134] (120) Filter [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#44] -Condition : (((isnotnull(d_year#44) AND (d_year#44 >= 1999)) AND (d_year#44 <= 2001)) AND isnotnull(d_date_sk#27)) +Input [2]: [d_date_sk#133, d_year#134] +Condition : (((isnotnull(d_year#134) AND (d_year#134 >= 1999)) AND (d_year#134 <= 2001)) AND isnotnull(d_date_sk#133)) (121) Project [codegen id : 1] -Output [1]: [d_date_sk#27] -Input [2]: [d_date_sk#27, d_year#44] +Output [1]: [d_date_sk#133] +Input [2]: [d_date_sk#133, d_year#134] (122) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#119] +Input [1]: [d_date_sk#133] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#135] (123) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Left keys [1]: [ss_sold_date_sk#131] +Right keys [1]: [d_date_sk#133] Join condition: None (124) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#120, ss_list_price#3 AS list_price#121] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#27] +Output [2]: [ss_quantity#129 AS quantity#136, ss_list_price#130 AS list_price#137] +Input [4]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131, d_date_sk#133] (125) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Output [3]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#118)] +PartitionFilters: [isnotnull(cs_sold_date_sk#140), dynamicpruningexpression(cs_sold_date_sk#140 IN dynamicpruning#132)] ReadSchema: struct (126) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Input [3]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140] (127) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#122] +Output [1]: [d_date_sk#141] (128) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#122] +Left keys [1]: [cs_sold_date_sk#140] +Right keys [1]: [d_date_sk#141] Join condition: None (129) Project [codegen id : 4] -Output [2]: [cs_quantity#62 AS quantity#123, cs_list_price#63 AS list_price#124] -Input [4]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, d_date_sk#122] +Output [2]: [cs_quantity#138 AS quantity#142, cs_list_price#139 AS list_price#143] +Input [4]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140, d_date_sk#141] (130) Scan parquet default.web_sales -Output [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Output [3]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#118)] +PartitionFilters: [isnotnull(ws_sold_date_sk#146), dynamicpruningexpression(ws_sold_date_sk#146 IN dynamicpruning#132)] ReadSchema: struct (131) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Input [3]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146] (132) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#125] +Output [1]: [d_date_sk#147] (133) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#125] +Left keys [1]: [ws_sold_date_sk#146] +Right keys [1]: [d_date_sk#147] Join condition: None (134) Project [codegen id : 6] -Output [2]: [ws_quantity#82 AS quantity#126, ws_list_price#83 AS list_price#127] -Input [4]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, d_date_sk#125] +Output [2]: [ws_quantity#144 AS quantity#148, ws_list_price#145 AS list_price#149] +Input [4]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146, d_date_sk#147] (135) Union (136) HashAggregate [codegen id : 7] -Input [2]: [quantity#120, list_price#121] +Input [2]: [quantity#136, list_price#137] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#120 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#121 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#128, count#129] -Results [2]: [sum#130, count#131] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#150, count#151] +Results [2]: [sum#152, count#153] (137) Exchange -Input [2]: [sum#130, count#131] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#132] +Input [2]: [sum#152, count#153] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#154] (138) HashAggregate [codegen id : 8] -Input [2]: [sum#130, count#131] +Input [2]: [sum#152, count#153] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#120 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#121 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#120 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#121 as decimal(12,2)))), DecimalType(18,2), true))#133] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#120 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#121 as decimal(12,2)))), DecimalType(18,2), true))#133 AS average_sales#134] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#155] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#155 AS average_sales#156] -Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#118 +Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#131 IN dynamicpruning#132 ReusedExchange (139) (139) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#133] -Subquery:3 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#118 +Subquery:3 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#140 IN dynamicpruning#132 -Subquery:4 Hosting operator id = 130 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#118 +Subquery:4 Hosting operator id = 130 Hosting Expression = ws_sold_date_sk#146 IN dynamicpruning#132 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#50] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 ReusedExchange (141) (141) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#29] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt index 013c3339f6ec3..fab231dc80a6c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt @@ -155,499 +155,499 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (10) ColumnarToRow [codegen id : 10] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] (11) Filter [codegen id : 10] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_item_sk#11) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#14, d_year#15] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#14, d_year#15] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1999)) AND (d_year#15 <= 2001)) AND isnotnull(d_date_sk#14)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#14] +Input [2]: [d_date_sk#14, d_year#15] (16) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (17) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#14] Join condition: None (18) Project [codegen id : 10] -Output [1]: [ss_item_sk#1] -Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] +Output [1]: [ss_item_sk#11] +Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] (19) Scan parquet default.item -Output [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Condition : (((isnotnull(i_item_sk#15) AND isnotnull(i_brand_id#16)) AND isnotnull(i_class_id#17)) AND isnotnull(i_category_id#18)) +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) (22) Exchange -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: hashpartitioning(coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] (23) Sort [codegen id : 5] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: [coalesce(i_brand_id#16, 0) ASC NULLS FIRST, isnull(i_brand_id#16) ASC NULLS FIRST, coalesce(i_class_id#17, 0) ASC NULLS FIRST, isnull(i_class_id#17) ASC NULLS FIRST, coalesce(i_category_id#18, 0) ASC NULLS FIRST, isnull(i_category_id#18) ASC NULLS FIRST], false, 0 +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 (24) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] +Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 8] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] (26) Filter [codegen id : 8] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#22) (27) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#24] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None (29) Project [codegen id : 8] -Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#22] +Output [1]: [cs_item_sk#22] +Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] (30) Scan parquet default.item -Output [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] (32) Filter [codegen id : 7] -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -Condition : isnotnull(i_item_sk#23) +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Condition : isnotnull(i_item_sk#25) (33) BroadcastExchange -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#23] +Left keys [1]: [cs_item_sk#22] +Right keys [1]: [i_item_sk#25] Join condition: None (35) Project [codegen id : 8] -Output [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Input [5]: [cs_item_sk#20, i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] (36) Exchange -Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: hashpartitioning(coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26), 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] (37) Sort [codegen id : 9] -Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: [coalesce(i_brand_id#24, 0) ASC NULLS FIRST, isnull(i_brand_id#24) ASC NULLS FIRST, coalesce(i_class_id#25, 0) ASC NULLS FIRST, isnull(i_class_id#25) ASC NULLS FIRST, coalesce(i_category_id#26, 0) ASC NULLS FIRST, isnull(i_category_id#26) ASC NULLS FIRST], false, 0 +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 (38) SortMergeJoin -Left keys [6]: [coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18)] -Right keys [6]: [coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26)] +Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] Join condition: None (39) BroadcastExchange -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] +Left keys [1]: [ss_item_sk#11] +Right keys [1]: [i_item_sk#17] Join condition: None (41) Project [codegen id : 10] -Output [3]: [i_brand_id#16 AS brand_id#30, i_class_id#17 AS class_id#31, i_category_id#18 AS category_id#32] -Input [5]: [ss_item_sk#1, i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] +Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (42) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#32, class_id#33, category_id#34] (43) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] (44) HashAggregate [codegen id : 11] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#32, class_id#33, category_id#34] (45) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32), 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] (46) Sort [codegen id : 12] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: [coalesce(brand_id#30, 0) ASC NULLS FIRST, isnull(brand_id#30) ASC NULLS FIRST, coalesce(class_id#31, 0) ASC NULLS FIRST, isnull(class_id#31) ASC NULLS FIRST, coalesce(category_id#32, 0) ASC NULLS FIRST, isnull(category_id#32) ASC NULLS FIRST], false, 0 +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 15] -Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] (49) Filter [codegen id : 15] -Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#35) +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_item_sk#37) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#37] +Output [1]: [d_date_sk#39] (51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#37] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#39] Join condition: None (52) Project [codegen id : 15] -Output [1]: [ws_item_sk#35] -Input [3]: [ws_item_sk#35, ws_sold_date_sk#36, d_date_sk#37] +Output [1]: [ws_item_sk#37] +Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] (53) ReusedExchange [Reuses operator id: 33] -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] (54) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [i_item_sk#38] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#40] Join condition: None (55) Project [codegen id : 15] -Output [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Input [5]: [ws_item_sk#35, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] (56) Exchange -Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: hashpartitioning(coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41), 5), ENSURE_REQUIREMENTS, [id=#42] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] (57) Sort [codegen id : 16] -Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: [coalesce(i_brand_id#39, 0) ASC NULLS FIRST, isnull(i_brand_id#39) ASC NULLS FIRST, coalesce(i_class_id#40, 0) ASC NULLS FIRST, isnull(i_class_id#40) ASC NULLS FIRST, coalesce(i_category_id#41, 0) ASC NULLS FIRST, isnull(i_category_id#41) ASC NULLS FIRST], false, 0 +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] -Right keys [6]: [coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41)] +Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] +Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] Join condition: None (59) HashAggregate [codegen id : 17] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#32, class_id#33, category_id#34] (60) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] (61) HashAggregate [codegen id : 18] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#32, class_id#33, category_id#34] (62) BroadcastExchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#44] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] (63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#30, class_id#31, category_id#32] +Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join condition: None (64) Project [codegen id : 19] -Output [1]: [i_item_sk#7 AS ss_item_sk#45] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#30, class_id#31, category_id#32] +Output [1]: [i_item_sk#7 AS ss_item_sk#47] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] (65) Exchange -Input [1]: [ss_item_sk#45] -Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [1]: [ss_item_sk#47] +Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] (66) Sort [codegen id : 20] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#47] Join condition: None (68) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#47] +Output [2]: [d_date_sk#49, d_week_seq#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 21] -Input [2]: [d_date_sk#12, d_week_seq#47] +Input [2]: [d_date_sk#49, d_week_seq#50] (70) Filter [codegen id : 21] -Input [2]: [d_date_sk#12, d_week_seq#47] -Condition : ((isnotnull(d_week_seq#47) AND (d_week_seq#47 = Subquery scalar-subquery#48, [id=#49])) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#49, d_week_seq#50] +Condition : ((isnotnull(d_week_seq#50) AND (d_week_seq#50 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#49)) (71) Project [codegen id : 21] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#47] +Output [1]: [d_date_sk#49] +Input [2]: [d_date_sk#49, d_week_seq#50] (72) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] +Input [1]: [d_date_sk#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] (73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#49] Join condition: None (74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] (75) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (76) ColumnarToRow [codegen id : 22] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] (77) Filter [codegen id : 22] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Condition : (((isnotnull(i_item_sk#54) AND isnotnull(i_brand_id#55)) AND isnotnull(i_class_id#56)) AND isnotnull(i_category_id#57)) (78) Exchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: hashpartitioning(i_item_sk#54, 5), ENSURE_REQUIREMENTS, [id=#58] (79) Sort [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: [i_item_sk#54 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#47] (81) Sort [codegen id : 41] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#45] +Left keys [1]: [i_item_sk#54] +Right keys [1]: [ss_item_sk#47] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#59] (84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#54] Join condition: None (85) Project [codegen id : 42] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] (86) HashAggregate [codegen id : 42] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] +Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#53, isEmpty#54, count#55] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] +Aggregate Attributes [3]: [sum#60, isEmpty#61, count#62] +Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] +Arguments: hashpartitioning(i_brand_id#55, i_class_id#56, i_category_id#57, 5), ENSURE_REQUIREMENTS, [id=#66] (88) HashAggregate [codegen id : 86] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] +Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60, count(1)#61] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sales#62, count(1)#61 AS number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67, count(1)#68] +Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#69, count(1)#68 AS number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] (89) Filter [codegen id : 86] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 as decimal(32,6)) > cast(Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(32,6)) > cast(Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) (90) Project [codegen id : 86] -Output [6]: [store AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] +Output [6]: [store AS channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] (91) Scan parquet default.store_sales -Output [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Output [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#71), dynamicpruningexpression(ss_sold_date_sk#71 IN dynamicpruning#72)] +PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 43] -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] (93) Filter [codegen id : 43] -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] -Condition : isnotnull(ss_item_sk#68) +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Condition : isnotnull(ss_item_sk#75) (94) Exchange -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] -Arguments: hashpartitioning(ss_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Arguments: hashpartitioning(ss_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#80] (95) Sort [codegen id : 44] -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] -Arguments: [ss_item_sk#68 ASC NULLS FIRST], false, 0 +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Arguments: [ss_item_sk#75 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#47] (97) Sort [codegen id : 62] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ss_item_sk#68] -Right keys [1]: [ss_item_sk#45] +Left keys [1]: [ss_item_sk#75] +Right keys [1]: [ss_item_sk#47] Join condition: None (99) Scan parquet default.date_dim -Output [2]: [d_date_sk#74, d_week_seq#75] +Output [2]: [d_date_sk#81, d_week_seq#82] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (100) ColumnarToRow [codegen id : 63] -Input [2]: [d_date_sk#74, d_week_seq#75] +Input [2]: [d_date_sk#81, d_week_seq#82] (101) Filter [codegen id : 63] -Input [2]: [d_date_sk#74, d_week_seq#75] -Condition : ((isnotnull(d_week_seq#75) AND (d_week_seq#75 = Subquery scalar-subquery#76, [id=#77])) AND isnotnull(d_date_sk#74)) +Input [2]: [d_date_sk#81, d_week_seq#82] +Condition : ((isnotnull(d_week_seq#82) AND (d_week_seq#82 = Subquery scalar-subquery#83, [id=#84])) AND isnotnull(d_date_sk#81)) (102) Project [codegen id : 63] -Output [1]: [d_date_sk#74] -Input [2]: [d_date_sk#74, d_week_seq#75] +Output [1]: [d_date_sk#81] +Input [2]: [d_date_sk#81, d_week_seq#82] (103) BroadcastExchange -Input [1]: [d_date_sk#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] +Input [1]: [d_date_sk#81] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#85] (104) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_sold_date_sk#71] -Right keys [1]: [d_date_sk#74] +Left keys [1]: [ss_sold_date_sk#78] +Right keys [1]: [d_date_sk#81] Join condition: None (105) Project [codegen id : 84] -Output [3]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70] -Input [5]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71, d_date_sk#74] +Output [3]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77] +Input [5]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#81] (106) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] +Output [4]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] (107) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#68] -Right keys [1]: [i_item_sk#79] +Left keys [1]: [ss_item_sk#75] +Right keys [1]: [i_item_sk#86] Join condition: None (108) Project [codegen id : 84] -Output [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#80, i_class_id#81, i_category_id#82] -Input [7]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] +Output [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] +Input [7]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] (109) HashAggregate [codegen id : 84] -Input [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#80, i_class_id#81, i_category_id#82] -Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] -Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] +Input [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] +Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] +Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] (110) Exchange -Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] -Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] +Arguments: hashpartitioning(i_brand_id#87, i_class_id#88, i_category_id#89, 5), ENSURE_REQUIREMENTS, [id=#96] (111) HashAggregate [codegen id : 85] -Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] -Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90, count(1)#91] -Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sales#92, count(1)#91 AS number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] +Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] +Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#99, count(1)#98 AS number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] (112) Filter [codegen id : 85] -Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) (113) Project [codegen id : 85] -Output [6]: [store AS channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] -Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] +Output [6]: [store AS channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] (114) BroadcastExchange -Input [6]: [channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#96] +Input [6]: [channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#103] (115) BroadcastHashJoin [codegen id : 86] -Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] +Left keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] +Right keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] Join condition: None (116) TakeOrderedAndProject -Input [12]: [channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] -Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Input [12]: [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Arguments: 100, [i_brand_id#55 ASC NULLS FIRST, i_class_id#56 ASC NULLS FIRST, i_category_id#57 ASC NULLS FIRST], [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#65, [id=#66] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#72, [id=#73] * HashAggregate (139) +- Exchange (138) +- * HashAggregate (137) @@ -674,140 +674,140 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (117) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#97)] +PartitionFilters: [isnotnull(ss_sold_date_sk#106), dynamicpruningexpression(ss_sold_date_sk#106 IN dynamicpruning#107)] ReadSchema: struct (118) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] (119) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#108, d_year#109] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (120) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#108, d_year#109] (121) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#108, d_year#109] +Condition : (((isnotnull(d_year#109) AND (d_year#109 >= 1999)) AND (d_year#109 <= 2001)) AND isnotnull(d_date_sk#108)) (122) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#108] +Input [2]: [d_date_sk#108, d_year#109] (123) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] +Input [1]: [d_date_sk#108] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#110] (124) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#106] +Right keys [1]: [d_date_sk#108] Join condition: None (125) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#99, ss_list_price#3 AS list_price#100] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#104 AS quantity#111, ss_list_price#105 AS list_price#112] +Input [4]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106, d_date_sk#108] (126) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21] +Output [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#97)] +PartitionFilters: [isnotnull(cs_sold_date_sk#115), dynamicpruningexpression(cs_sold_date_sk#115 IN dynamicpruning#107)] ReadSchema: struct (127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21] +Input [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] (128) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#103] +Output [1]: [d_date_sk#116] (129) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#103] +Left keys [1]: [cs_sold_date_sk#115] +Right keys [1]: [d_date_sk#116] Join condition: None (130) Project [codegen id : 4] -Output [2]: [cs_quantity#101 AS quantity#104, cs_list_price#102 AS list_price#105] -Input [4]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21, d_date_sk#103] +Output [2]: [cs_quantity#113 AS quantity#117, cs_list_price#114 AS list_price#118] +Input [4]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115, d_date_sk#116] (131) Scan parquet default.web_sales -Output [3]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36] +Output [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#97)] +PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#107)] ReadSchema: struct (132) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36] +Input [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] (133) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#108] +Output [1]: [d_date_sk#122] (134) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#108] +Left keys [1]: [ws_sold_date_sk#121] +Right keys [1]: [d_date_sk#122] Join condition: None (135) Project [codegen id : 6] -Output [2]: [ws_quantity#106 AS quantity#109, ws_list_price#107 AS list_price#110] -Input [4]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36, d_date_sk#108] +Output [2]: [ws_quantity#119 AS quantity#123, ws_list_price#120 AS list_price#124] +Input [4]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121, d_date_sk#122] (136) Union (137) HashAggregate [codegen id : 7] -Input [2]: [quantity#99, list_price#100] +Input [2]: [quantity#111, list_price#112] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#111, count#112] -Results [2]: [sum#113, count#114] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#125, count#126] +Results [2]: [sum#127, count#128] (138) Exchange -Input [2]: [sum#113, count#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] +Input [2]: [sum#127, count#128] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] (139) HashAggregate [codegen id : 8] -Input [2]: [sum#113, count#114] +Input [2]: [sum#127, count#128] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#116] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#116 AS average_sales#117] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130 AS average_sales#131] -Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#97 +Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#106 IN dynamicpruning#107 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#108] -Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#97 +Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#115 IN dynamicpruning#107 -Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#97 +Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#107 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (141) (141) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#49] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 ReusedExchange (142) (142) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#14] -Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 -Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#51, [id=#52] * Project (146) +- * Filter (145) +- * ColumnarToRow (144) @@ -815,33 +815,33 @@ Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquer (143) Scan parquet default.date_dim -Output [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Output [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] (145) Filter [codegen id : 1] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#13 = 2000)) AND (d_moy#118 = 12)) AND (d_dom#119 = 11)) +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Condition : (((((isnotnull(d_year#133) AND isnotnull(d_moy#134)) AND isnotnull(d_dom#135)) AND (d_year#133 = 2000)) AND (d_moy#134 = 12)) AND (d_dom#135 = 11)) (146) Project [codegen id : 1] -Output [1]: [d_week_seq#47] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Output [1]: [d_week_seq#132] +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] -Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#65, [id=#66] +Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] -Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#71 IN dynamicpruning#72 +Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#79 ReusedExchange (147) (147) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#74] +Output [1]: [d_date_sk#81] -Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#76, [id=#77] +Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#83, [id=#84] * Project (151) +- * Filter (150) +- * ColumnarToRow (149) @@ -849,21 +849,21 @@ Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subqu (148) Scan parquet default.date_dim -Output [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Output [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (149) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] (150) Filter [codegen id : 1] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#13 = 1999)) AND (d_moy#118 = 12)) AND (d_dom#119 = 11)) +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Condition : (((((isnotnull(d_year#137) AND isnotnull(d_moy#138)) AND isnotnull(d_dom#139)) AND (d_year#137 = 1999)) AND (d_moy#138 = 12)) AND (d_dom#139 = 11)) (151) Project [codegen id : 1] -Output [1]: [d_week_seq#47] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Output [1]: [d_week_seq#136] +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 83604fd5cdff8..ae653b43971db 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -131,443 +131,443 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_item_sk#10) (10) Scan parquet default.item -Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (11) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] (12) Filter [codegen id : 4] -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14)) +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) (13) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] (15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#15) +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Condition : isnotnull(cs_item_sk#17) (16) Scan parquet default.item -Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] (18) Filter [codegen id : 1] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Condition : isnotnull(i_item_sk#17) +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Condition : isnotnull(i_item_sk#19) (19) BroadcastExchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] (20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#15] -Right keys [1]: [i_item_sk#17] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#19] Join condition: None (21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20] -Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] (22) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_year#23] +Output [2]: [d_date_sk#24, d_year#25] 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 (23) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#22, d_year#23] +Input [2]: [d_date_sk#24, d_year#25] (24) Filter [codegen id : 2] -Input [2]: [d_date_sk#22, d_year#23] -Condition : (((isnotnull(d_year#23) AND (d_year#23 >= 1999)) AND (d_year#23 <= 2001)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) (25) Project [codegen id : 2] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_year#23] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_year#25] (26) 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#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (27) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] Join condition: None (28) Project [codegen id : 3] -Output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] -Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#22] +Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] (29) BroadcastExchange -Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] (30) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)] -Right keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join condition: None (31) BroadcastExchange -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] (32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#11] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] Join condition: None (33) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] (34) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#29] (35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#29] Join condition: None (36) Project [codegen id : 6] -Output [3]: [i_brand_id#12 AS brand_id#28, i_class_id#13 AS class_id#29, i_category_id#14 AS category_id#30] -Input [5]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#27] +Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#30, class_id#31, category_id#32] (38) Exchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#30, class_id#31, category_id#32] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_item_sk#32) +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_item_sk#34) (43) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#34] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#36] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37] -Input [6]: [ws_item_sk#32, ws_sold_date_sk#33, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] +Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] (46) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#38] +Output [1]: [d_date_sk#40] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#40] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] -Input [5]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] +Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] (49) BroadcastExchange -Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#39] +Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] -Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#30, class_id#31, category_id#32] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#30, class_id#31, category_id#32] (53) BroadcastExchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#40] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#28, class_id#29, category_id#30] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#41] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] +Output [1]: [i_item_sk#6 AS ss_item_sk#43] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] (56) BroadcastExchange -Input [1]: [ss_item_sk#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [ss_item_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#41] +Right keys [1]: [ss_item_sk#43] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Condition : (((isnotnull(i_item_sk#45) AND isnotnull(i_brand_id#46)) AND isnotnull(i_class_id#47)) AND isnotnull(i_category_id#48)) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#43] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#41] +Left keys [1]: [i_item_sk#45] +Right keys [1]: [ss_item_sk#43] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#45] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (66) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_week_seq#44] +Output [2]: [d_date_sk#50, d_week_seq#51] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [2]: [d_date_sk#27, d_week_seq#44] +Input [2]: [d_date_sk#50, d_week_seq#51] (68) Filter [codegen id : 24] -Input [2]: [d_date_sk#27, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#27)) +Input [2]: [d_date_sk#50, d_week_seq#51] +Condition : ((isnotnull(d_week_seq#51) AND (d_week_seq#51 = Subquery scalar-subquery#52, [id=#53])) AND isnotnull(d_date_sk#50)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#27] -Input [2]: [d_date_sk#27, d_week_seq#44] +Output [1]: [d_date_sk#50] +Input [2]: [d_date_sk#50, d_week_seq#51] (70) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] +Input [1]: [d_date_sk#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#50] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#27] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#48, isEmpty#49, count#50] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] +Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] +Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#61] (75) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55, count(1)#56] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sales#57, count(1)#56 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] (76) Filter [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59 as decimal(32,6)) > cast(Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) (77) Project [codegen id : 52] -Output [6]: [store AS channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] +Output [6]: [store AS channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] (78) Scan parquet default.store_sales -Output [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] +Output [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] +PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] +Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] (80) Filter [codegen id : 50] -Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] -Condition : isnotnull(ss_item_sk#63) +Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +Condition : isnotnull(ss_item_sk#70) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#43] (82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#63] -Right keys [1]: [ss_item_sk#41] +Left keys [1]: [ss_item_sk#70] +Right keys [1]: [ss_item_sk#43] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] +Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] (84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#63] -Right keys [1]: [i_item_sk#68] +Left keys [1]: [ss_item_sk#70] +Right keys [1]: [i_item_sk#75] Join condition: None (85) Project [codegen id : 50] -Output [6]: [ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_brand_id#69, i_class_id#70, i_category_id#71] -Input [8]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] +Output [6]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] +Input [8]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#72, d_week_seq#73] +Output [2]: [d_date_sk#79, d_week_seq#80] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#72, d_week_seq#73] +Input [2]: [d_date_sk#79, d_week_seq#80] (88) Filter [codegen id : 49] -Input [2]: [d_date_sk#72, d_week_seq#73] -Condition : ((isnotnull(d_week_seq#73) AND (d_week_seq#73 = Subquery scalar-subquery#74, [id=#75])) AND isnotnull(d_date_sk#72)) +Input [2]: [d_date_sk#79, d_week_seq#80] +Condition : ((isnotnull(d_week_seq#80) AND (d_week_seq#80 = Subquery scalar-subquery#81, [id=#82])) AND isnotnull(d_date_sk#79)) (89) Project [codegen id : 49] -Output [1]: [d_date_sk#72] -Input [2]: [d_date_sk#72, d_week_seq#73] +Output [1]: [d_date_sk#79] +Input [2]: [d_date_sk#79, d_week_seq#80] (90) BroadcastExchange -Input [1]: [d_date_sk#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] +Input [1]: [d_date_sk#79] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#83] (91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#66] -Right keys [1]: [d_date_sk#72] +Left keys [1]: [ss_sold_date_sk#73] +Right keys [1]: [d_date_sk#79] Join condition: None (92) Project [codegen id : 50] -Output [5]: [ss_quantity#64, ss_list_price#65, i_brand_id#69, i_class_id#70, i_category_id#71] -Input [7]: [ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_brand_id#69, i_class_id#70, i_category_id#71, d_date_sk#72] +Output [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] +Input [7]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78, d_date_sk#79] (93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#64, ss_list_price#65, i_brand_id#69, i_class_id#70, i_category_id#71] -Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#77, isEmpty#78, count#79] -Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] +Input [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] +Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] +Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] (94) Exchange -Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] -Arguments: hashpartitioning(i_brand_id#69, i_class_id#70, i_category_id#71, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] +Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, [id=#90] (95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] -Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84, count(1)#85] -Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sales#86, count(1)#85 AS number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] +Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] +Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] (96) Filter [codegen id : 51] -Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) (97) Project [codegen id : 51] -Output [6]: [store AS channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] -Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] +Output [6]: [store AS channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] (98) BroadcastExchange -Input [6]: [channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#90] +Input [6]: [channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#97] (99) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] +Left keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Right keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] Join condition: None (100) TakeOrderedAndProject -Input [12]: [channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] -Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] +Input [12]: [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Arguments: 100, [i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#60, [id=#61] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#67, [id=#68] * HashAggregate (123) +- Exchange (122) +- * HashAggregate (121) @@ -594,140 +594,140 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#91)] +PartitionFilters: [isnotnull(ss_sold_date_sk#100), dynamicpruningexpression(ss_sold_date_sk#100 IN dynamicpruning#101)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] (103) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_year#92] +Output [2]: [d_date_sk#102, d_year#103] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#92] +Input [2]: [d_date_sk#102, d_year#103] (105) Filter [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#92] -Condition : (((isnotnull(d_year#92) AND (d_year#92 >= 1999)) AND (d_year#92 <= 2001)) AND isnotnull(d_date_sk#27)) +Input [2]: [d_date_sk#102, d_year#103] +Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 <= 2001)) AND isnotnull(d_date_sk#102)) (106) Project [codegen id : 1] -Output [1]: [d_date_sk#27] -Input [2]: [d_date_sk#27, d_year#92] +Output [1]: [d_date_sk#102] +Input [2]: [d_date_sk#102, d_year#103] (107) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] +Input [1]: [d_date_sk#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#104] (108) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Left keys [1]: [ss_sold_date_sk#100] +Right keys [1]: [d_date_sk#102] Join condition: None (109) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#94, ss_list_price#3 AS list_price#95] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#27] +Output [2]: [ss_quantity#98 AS quantity#105, ss_list_price#99 AS list_price#106] +Input [4]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100, d_date_sk#102] (110) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16] +Output [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#91)] +PartitionFilters: [isnotnull(cs_sold_date_sk#109), dynamicpruningexpression(cs_sold_date_sk#109 IN dynamicpruning#101)] ReadSchema: struct (111) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16] +Input [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] (112) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#98] +Output [1]: [d_date_sk#110] (113) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#98] +Left keys [1]: [cs_sold_date_sk#109] +Right keys [1]: [d_date_sk#110] Join condition: None (114) Project [codegen id : 4] -Output [2]: [cs_quantity#96 AS quantity#99, cs_list_price#97 AS list_price#100] -Input [4]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16, d_date_sk#98] +Output [2]: [cs_quantity#107 AS quantity#111, cs_list_price#108 AS list_price#112] +Input [4]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109, d_date_sk#110] (115) Scan parquet default.web_sales -Output [3]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33] +Output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#91)] +PartitionFilters: [isnotnull(ws_sold_date_sk#115), dynamicpruningexpression(ws_sold_date_sk#115 IN dynamicpruning#101)] ReadSchema: struct (116) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33] +Input [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] (117) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#103] +Output [1]: [d_date_sk#116] (118) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#103] +Left keys [1]: [ws_sold_date_sk#115] +Right keys [1]: [d_date_sk#116] Join condition: None (119) Project [codegen id : 6] -Output [2]: [ws_quantity#101 AS quantity#104, ws_list_price#102 AS list_price#105] -Input [4]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33, d_date_sk#103] +Output [2]: [ws_quantity#113 AS quantity#117, ws_list_price#114 AS list_price#118] +Input [4]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115, d_date_sk#116] (120) Union (121) HashAggregate [codegen id : 7] -Input [2]: [quantity#94, list_price#95] +Input [2]: [quantity#105, list_price#106] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#106, count#107] -Results [2]: [sum#108, count#109] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#119, count#120] +Results [2]: [sum#121, count#122] (122) Exchange -Input [2]: [sum#108, count#109] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] +Input [2]: [sum#121, count#122] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#123] (123) HashAggregate [codegen id : 8] -Input [2]: [sum#108, count#109] +Input [2]: [sum#121, count#122] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))#111] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))#111 AS average_sales#112] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124 AS average_sales#125] -Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#91 +Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#100 IN dynamicpruning#101 ReusedExchange (124) (124) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#102] -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#91 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#109 IN dynamicpruning#101 -Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#91 +Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#115 IN dynamicpruning#101 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#50] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 ReusedExchange (126) (126) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#29] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] * Project (130) +- * Filter (129) +- * ColumnarToRow (128) @@ -735,33 +735,33 @@ Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer (127) Scan parquet default.date_dim -Output [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Output [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (128) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] (129) Filter [codegen id : 1] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] -Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#92 = 2000)) AND (d_moy#113 = 12)) AND (d_dom#114 = 11)) +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Condition : (((((isnotnull(d_year#127) AND isnotnull(d_moy#128)) AND isnotnull(d_dom#129)) AND (d_year#127 = 2000)) AND (d_moy#128 = 12)) AND (d_dom#129 = 11)) (130) Project [codegen id : 1] -Output [1]: [d_week_seq#44] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Output [1]: [d_week_seq#126] +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] -Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#60, [id=#61] +Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] -Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67 +Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 ReusedExchange (131) (131) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#72] +Output [1]: [d_date_sk#79] -Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#74, [id=#75] +Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#81, [id=#82] * Project (135) +- * Filter (134) +- * ColumnarToRow (133) @@ -769,21 +769,21 @@ Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subque (132) Scan parquet default.date_dim -Output [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Output [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (133) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] (134) Filter [codegen id : 1] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] -Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#92 = 1999)) AND (d_moy#113 = 12)) AND (d_dom#114 = 11)) +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Condition : (((((isnotnull(d_year#131) AND isnotnull(d_moy#132)) AND isnotnull(d_dom#133)) AND (d_year#131 = 1999)) AND (d_moy#132 = 12)) AND (d_dom#133 = 11)) (135) Project [codegen id : 1] -Output [1]: [d_week_seq#44] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Output [1]: [d_week_seq#130] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index adfa3e44f996d..7604ea1e786d8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -74,187 +74,187 @@ Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet default.catalog_sales -Output [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Output [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Input [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] (9) Project [codegen id : 3] -Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#10, cs_order_number#5 AS cs_order_number#5#11] -Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Output [2]: [cs_warehouse_sk#10, cs_order_number#11] +Input [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] (10) Exchange -Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] -Arguments: hashpartitioning(cs_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [2]: [cs_warehouse_sk#10, cs_order_number#11] +Arguments: hashpartitioning(cs_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 4] -Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] -Arguments: [cs_order_number#5#11 ASC NULLS FIRST], false, 0 +Input [2]: [cs_warehouse_sk#10, cs_order_number#11] +Arguments: [cs_order_number#11 ASC NULLS FIRST], false, 0 (12) SortMergeJoin Left keys [1]: [cs_order_number#5] -Right keys [1]: [cs_order_number#5#11] -Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#10) +Right keys [1]: [cs_order_number#11] +Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#10) (13) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] (14) Scan parquet default.catalog_returns -Output [2]: [cr_order_number#13, cr_returned_date_sk#14] +Output [2]: [cr_order_number#14, cr_returned_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] ReadSchema: struct (15) ColumnarToRow [codegen id : 6] -Input [2]: [cr_order_number#13, cr_returned_date_sk#14] +Input [2]: [cr_order_number#14, cr_returned_date_sk#15] (16) Project [codegen id : 6] -Output [1]: [cr_order_number#13] -Input [2]: [cr_order_number#13, cr_returned_date_sk#14] +Output [1]: [cr_order_number#14] +Input [2]: [cr_order_number#14, cr_returned_date_sk#15] (17) Exchange -Input [1]: [cr_order_number#13] -Arguments: hashpartitioning(cr_order_number#13, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [1]: [cr_order_number#14] +Arguments: hashpartitioning(cr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] (18) Sort [codegen id : 7] -Input [1]: [cr_order_number#13] -Arguments: [cr_order_number#13 ASC NULLS FIRST], false, 0 +Input [1]: [cr_order_number#14] +Arguments: [cr_order_number#14 ASC NULLS FIRST], false, 0 (19) SortMergeJoin Left keys [1]: [cs_order_number#5] -Right keys [1]: [cr_order_number#13] +Right keys [1]: [cr_order_number#14] Join condition: None (20) Scan parquet default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] +Output [2]: [ca_address_sk#17, ca_state#18] 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 (21) ColumnarToRow [codegen id : 8] -Input [2]: [ca_address_sk#16, ca_state#17] +Input [2]: [ca_address_sk#17, ca_state#18] (22) Filter [codegen id : 8] -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_address_sk#16)) +Input [2]: [ca_address_sk#17, ca_state#18] +Condition : ((isnotnull(ca_state#18) AND (ca_state#18 = GA)) AND isnotnull(ca_address_sk#17)) (23) Project [codegen id : 8] -Output [1]: [ca_address_sk#16] -Input [2]: [ca_address_sk#16, ca_state#17] +Output [1]: [ca_address_sk#17] +Input [2]: [ca_address_sk#17, ca_state#18] (24) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Input [1]: [ca_address_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#16] +Right keys [1]: [ca_address_sk#17] Join condition: None (26) Project [codegen id : 11] Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] +Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#17] (27) Scan parquet default.call_center -Output [2]: [cc_call_center_sk#19, cc_county#20] +Output [2]: [cc_call_center_sk#20, cc_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 9] -Input [2]: [cc_call_center_sk#19, cc_county#20] +Input [2]: [cc_call_center_sk#20, cc_county#21] (29) Filter [codegen id : 9] -Input [2]: [cc_call_center_sk#19, cc_county#20] -Condition : ((isnotnull(cc_county#20) AND (cc_county#20 = Williamson County)) AND isnotnull(cc_call_center_sk#19)) +Input [2]: [cc_call_center_sk#20, cc_county#21] +Condition : ((isnotnull(cc_county#21) AND (cc_county#21 = Williamson County)) AND isnotnull(cc_call_center_sk#20)) (30) Project [codegen id : 9] -Output [1]: [cc_call_center_sk#19] -Input [2]: [cc_call_center_sk#19, cc_county#20] +Output [1]: [cc_call_center_sk#20] +Input [2]: [cc_call_center_sk#20, cc_county#21] (31) BroadcastExchange -Input [1]: [cc_call_center_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [cc_call_center_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] -Right keys [1]: [cc_call_center_sk#19] +Right keys [1]: [cc_call_center_sk#20] Join condition: None (33) Project [codegen id : 11] Output [4]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [6]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#19] +Input [6]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#20] (34) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_date#23] +Output [2]: [d_date_sk#23, d_date#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 10] -Input [2]: [d_date_sk#22, d_date#23] +Input [2]: [d_date_sk#23, d_date#24] (36) Filter [codegen id : 10] -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11719)) AND (d_date#23 <= 11779)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#23, d_date#24] +Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 2002-02-01)) AND (d_date#24 <= 2002-04-02)) AND isnotnull(d_date_sk#23)) (37) Project [codegen id : 10] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_date#23] +Output [1]: [d_date_sk#23] +Input [2]: [d_date_sk#23, d_date#24] (38) 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#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#23] Join condition: None (40) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [5]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#22] +Input [5]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#23] (41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] -Results [3]: [cs_order_number#5, sum#27, sum#28] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] +Results [3]: [cs_order_number#5, sum#28, sum#29] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_order_number#5, sum#27, sum#28] +Input [3]: [cs_order_number#5, sum#28, sum#29] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] -Results [3]: [cs_order_number#5, sum#27, sum#28] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] +Results [3]: [cs_order_number#5, sum#28, sum#29] (43) HashAggregate [codegen id : 11] -Input [3]: [cs_order_number#5, sum#27, sum#28] +Input [3]: [cs_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] -Results [3]: [sum#27, sum#28, count#30] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] +Results [3]: [sum#28, sum#29, count#31] (44) Exchange -Input [3]: [sum#27, sum#28, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [sum#28, sum#29, count#31] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] (45) HashAggregate [codegen id : 12] -Input [3]: [sum#27, sum#28, count#30] +Input [3]: [sum#28, sum#29, count#31] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] -Results [3]: [count(cs_order_number#5)#29 AS order count #32, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#25,17,2) AS total shipping cost #33, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#26,17,2) AS total net profit #34] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] +Results [3]: [count(cs_order_number#5)#30 AS order count #33, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#26,17,2) AS total shipping cost #34, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#27,17,2) AS total net profit #35] (46) Sort [codegen id : 12] -Input [3]: [order count #32, total shipping cost #33, total net profit #34] -Arguments: [order count #32 ASC NULLS FIRST], true, 0 +Input [3]: [order count #33, total shipping cost #34, total net profit #35] +Arguments: [order count #33 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index 5640564564396..683d83235cce5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -74,187 +74,187 @@ Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet default.catalog_sales -Output [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Output [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Input [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] (9) Project [codegen id : 3] -Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#10, cs_order_number#5 AS cs_order_number#5#11] -Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Output [2]: [cs_warehouse_sk#10, cs_order_number#11] +Input [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] (10) Exchange -Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] -Arguments: hashpartitioning(cs_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [2]: [cs_warehouse_sk#10, cs_order_number#11] +Arguments: hashpartitioning(cs_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 4] -Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] -Arguments: [cs_order_number#5#11 ASC NULLS FIRST], false, 0 +Input [2]: [cs_warehouse_sk#10, cs_order_number#11] +Arguments: [cs_order_number#11 ASC NULLS FIRST], false, 0 (12) SortMergeJoin Left keys [1]: [cs_order_number#5] -Right keys [1]: [cs_order_number#5#11] -Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#10) +Right keys [1]: [cs_order_number#11] +Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#10) (13) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] (14) Scan parquet default.catalog_returns -Output [2]: [cr_order_number#13, cr_returned_date_sk#14] +Output [2]: [cr_order_number#14, cr_returned_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] ReadSchema: struct (15) ColumnarToRow [codegen id : 6] -Input [2]: [cr_order_number#13, cr_returned_date_sk#14] +Input [2]: [cr_order_number#14, cr_returned_date_sk#15] (16) Project [codegen id : 6] -Output [1]: [cr_order_number#13] -Input [2]: [cr_order_number#13, cr_returned_date_sk#14] +Output [1]: [cr_order_number#14] +Input [2]: [cr_order_number#14, cr_returned_date_sk#15] (17) Exchange -Input [1]: [cr_order_number#13] -Arguments: hashpartitioning(cr_order_number#13, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [1]: [cr_order_number#14] +Arguments: hashpartitioning(cr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] (18) Sort [codegen id : 7] -Input [1]: [cr_order_number#13] -Arguments: [cr_order_number#13 ASC NULLS FIRST], false, 0 +Input [1]: [cr_order_number#14] +Arguments: [cr_order_number#14 ASC NULLS FIRST], false, 0 (19) SortMergeJoin Left keys [1]: [cs_order_number#5] -Right keys [1]: [cr_order_number#13] +Right keys [1]: [cr_order_number#14] Join condition: None (20) Scan parquet default.date_dim -Output [2]: [d_date_sk#16, d_date#17] +Output [2]: [d_date_sk#17, d_date#18] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#16, d_date#17] +Input [2]: [d_date_sk#17, d_date#18] (22) Filter [codegen id : 8] -Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 11719)) AND (d_date#17 <= 11779)) AND isnotnull(d_date_sk#16)) +Input [2]: [d_date_sk#17, d_date#18] +Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 2002-02-01)) AND (d_date#18 <= 2002-04-02)) AND isnotnull(d_date_sk#17)) (23) Project [codegen id : 8] -Output [1]: [d_date_sk#16] -Input [2]: [d_date_sk#16, d_date#17] +Output [1]: [d_date_sk#17] +Input [2]: [d_date_sk#17, d_date#18] (24) BroadcastExchange -Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Input [1]: [d_date_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#17] Join condition: None (26) Project [codegen id : 11] Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#16] +Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#17] (27) Scan parquet default.customer_address -Output [2]: [ca_address_sk#19, ca_state#20] +Output [2]: [ca_address_sk#20, ca_state#21] 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 (28) ColumnarToRow [codegen id : 9] -Input [2]: [ca_address_sk#19, ca_state#20] +Input [2]: [ca_address_sk#20, ca_state#21] (29) Filter [codegen id : 9] -Input [2]: [ca_address_sk#19, ca_state#20] -Condition : ((isnotnull(ca_state#20) AND (ca_state#20 = GA)) AND isnotnull(ca_address_sk#19)) +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : ((isnotnull(ca_state#21) AND (ca_state#21 = GA)) AND isnotnull(ca_address_sk#20)) (30) Project [codegen id : 9] -Output [1]: [ca_address_sk#19] -Input [2]: [ca_address_sk#19, ca_state#20] +Output [1]: [ca_address_sk#20] +Input [2]: [ca_address_sk#20, ca_state#21] (31) BroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [ca_address_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#19] +Right keys [1]: [ca_address_sk#20] Join condition: None (33) Project [codegen id : 11] Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#19] +Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#20] (34) Scan parquet default.call_center -Output [2]: [cc_call_center_sk#22, cc_county#23] +Output [2]: [cc_call_center_sk#23, cc_county#24] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 10] -Input [2]: [cc_call_center_sk#22, cc_county#23] +Input [2]: [cc_call_center_sk#23, cc_county#24] (36) Filter [codegen id : 10] -Input [2]: [cc_call_center_sk#22, cc_county#23] -Condition : ((isnotnull(cc_county#23) AND (cc_county#23 = Williamson County)) AND isnotnull(cc_call_center_sk#22)) +Input [2]: [cc_call_center_sk#23, cc_county#24] +Condition : ((isnotnull(cc_county#24) AND (cc_county#24 = Williamson County)) AND isnotnull(cc_call_center_sk#23)) (37) Project [codegen id : 10] -Output [1]: [cc_call_center_sk#22] -Input [2]: [cc_call_center_sk#22, cc_county#23] +Output [1]: [cc_call_center_sk#23] +Input [2]: [cc_call_center_sk#23, cc_county#24] (38) BroadcastExchange -Input [1]: [cc_call_center_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +Input [1]: [cc_call_center_sk#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] -Right keys [1]: [cc_call_center_sk#22] +Right keys [1]: [cc_call_center_sk#23] Join condition: None (40) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#22] +Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#23] (41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] -Results [3]: [cs_order_number#5, sum#27, sum#28] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] +Results [3]: [cs_order_number#5, sum#28, sum#29] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_order_number#5, sum#27, sum#28] +Input [3]: [cs_order_number#5, sum#28, sum#29] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] -Results [3]: [cs_order_number#5, sum#27, sum#28] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] +Results [3]: [cs_order_number#5, sum#28, sum#29] (43) HashAggregate [codegen id : 11] -Input [3]: [cs_order_number#5, sum#27, sum#28] +Input [3]: [cs_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] -Results [3]: [sum#27, sum#28, count#30] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] +Results [3]: [sum#28, sum#29, count#31] (44) Exchange -Input [3]: [sum#27, sum#28, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [sum#28, sum#29, count#31] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] (45) HashAggregate [codegen id : 12] -Input [3]: [sum#27, sum#28, count#30] +Input [3]: [sum#28, sum#29, count#31] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] -Results [3]: [count(cs_order_number#5)#29 AS order count #32, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#25,17,2) AS total shipping cost #33, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#26,17,2) AS total net profit #34] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] +Results [3]: [count(cs_order_number#5)#30 AS order count #33, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#26,17,2) AS total shipping cost #34, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#27,17,2) AS total net profit #35] (46) Sort [codegen id : 12] -Input [3]: [order count #32, total shipping cost #33, total net profit #34] -Arguments: [order count #32 ASC NULLS FIRST], true, 0 +Input [3]: [order count #33, total shipping cost #34, total net profit #35] +Arguments: [order count #33 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index 97467fe1dc1db..8918c2a36e2ec 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -77,7 +77,7 @@ Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isno Output [2]: [d_date_sk#8, d_quarter_name#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -85,7 +85,7 @@ Input [2]: [d_date_sk#8, d_quarter_name#9] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#8, d_quarter_name#9] -Condition : ((isnotnull(d_quarter_name#9) AND (d_quarter_name#9 = 2001Q1)) AND isnotnull(d_date_sk#8)) +Condition : ((isnotnull(d_quarter_name#9) AND (rpad(d_quarter_name#9, 6, ) = 2001Q1)) AND isnotnull(d_date_sk#8)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#8] @@ -197,7 +197,7 @@ Condition : ((isnotnull(sr_customer_sk#21) AND isnotnull(sr_item_sk#20)) AND isn Output [2]: [d_date_sk#26, d_quarter_name#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 9] @@ -205,7 +205,7 @@ Input [2]: [d_date_sk#26, d_quarter_name#27] (33) Filter [codegen id : 9] Input [2]: [d_date_sk#26, d_quarter_name#27] -Condition : (d_quarter_name#27 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#26)) +Condition : (rpad(d_quarter_name#27, 6, ) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#26)) (34) Project [codegen id : 9] Output [1]: [d_date_sk#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index 32c8de9a1d16e..dafa24943be82 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -124,7 +124,7 @@ Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_i Output [2]: [d_date_sk#21, d_quarter_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 3] @@ -132,7 +132,7 @@ Input [2]: [d_date_sk#21, d_quarter_name#22] (18) Filter [codegen id : 3] Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : ((isnotnull(d_quarter_name#22) AND (d_quarter_name#22 = 2001Q1)) AND isnotnull(d_date_sk#21)) +Condition : ((isnotnull(d_quarter_name#22) AND (rpad(d_quarter_name#22, 6, ) = 2001Q1)) AND isnotnull(d_date_sk#21)) (19) Project [codegen id : 3] Output [1]: [d_date_sk#21] @@ -155,7 +155,7 @@ Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_re Output [2]: [d_date_sk#24, d_quarter_name#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] @@ -163,7 +163,7 @@ Input [2]: [d_date_sk#24, d_quarter_name#25] (25) Filter [codegen id : 4] Input [2]: [d_date_sk#24, d_quarter_name#25] -Condition : (d_quarter_name#25 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) +Condition : (rpad(d_quarter_name#25, 6, ) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) (26) Project [codegen id : 4] Output [1]: [d_date_sk#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt index 020e92edb9d38..85aed12cc9d4f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt @@ -73,7 +73,7 @@ Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1) Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -81,7 +81,7 @@ Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14 (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown)) AND isnotnull(cd_demo_sk#11)) +Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown )) AND isnotnull(cd_demo_sk#11)) (7) Project [codegen id : 1] Output [2]: [cd_demo_sk#11, cd_dep_count#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt index b3b8cabfb6880..ed68f03734b13 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt @@ -67,7 +67,7 @@ Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1) Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -75,7 +75,7 @@ Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14 (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown)) AND isnotnull(cd_demo_sk#11)) +Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown )) AND isnotnull(cd_demo_sk#11)) (7) Project [codegen id : 1] Output [2]: [cd_demo_sk#11, cd_dep_count#14] 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 f540e067bbc92..c31bb7470648f 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#10, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] +Output [8]: [d_week_seq#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] (25) HashAggregate [codegen id : 11] -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))#60, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#60,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66,17,2) AS sat_sales#41] +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] (26) Scan parquet default.date_dim -Output [2]: [d_week_seq#67, d_year#68] +Output [2]: [d_week_seq#69, d_year#70] 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#67, d_year#68] +Input [2]: [d_week_seq#69, d_year#70] (28) Filter [codegen id : 10] -Input [2]: [d_week_seq#67, d_year#68] -Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) +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)) (29) Project [codegen id : 10] -Output [1]: [d_week_seq#67] -Input [2]: [d_week_seq#67, d_year#68] +Output [1]: [d_week_seq#69] +Input [2]: [d_week_seq#69, d_year#70] (30) BroadcastExchange -Input [1]: [d_week_seq#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] +Input [1]: [d_week_seq#69] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] (31) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#67] +Left keys [1]: [d_week_seq#53] +Right keys [1]: [d_week_seq#69] Join condition: None (32) Project [codegen id : 11] -Output [8]: [d_week_seq#10 AS d_week_seq2#70, sun_sales#35 AS sun_sales2#71, mon_sales#36 AS mon_sales2#72, tue_sales#37 AS tue_sales2#73, wed_sales#38 AS wed_sales2#74, thu_sales#39 AS thu_sales2#75, fri_sales#40 AS fri_sales2#76, sat_sales#41 AS sat_sales2#77] -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#67] +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] (33) BroadcastExchange -Input [8]: [d_week_seq2#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#78] +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] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq1#45] -Right keys [1]: [(d_week_seq2#70 - 53)] +Right keys [1]: [(d_week_seq2#72 - 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#71)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#79, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#72)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#80, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#73)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#81, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#74)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#82, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#75)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#83, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#76)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#84, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#77)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#85] -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#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] +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] (36) Exchange -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] -Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#86] +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] (37) Sort [codegen id : 13] -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] +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: [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 f540e067bbc92..c31bb7470648f 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#10, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] +Output [8]: [d_week_seq#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] (25) HashAggregate [codegen id : 11] -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))#60, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#60,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66,17,2) AS sat_sales#41] +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] (26) Scan parquet default.date_dim -Output [2]: [d_week_seq#67, d_year#68] +Output [2]: [d_week_seq#69, d_year#70] 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#67, d_year#68] +Input [2]: [d_week_seq#69, d_year#70] (28) Filter [codegen id : 10] -Input [2]: [d_week_seq#67, d_year#68] -Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) +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)) (29) Project [codegen id : 10] -Output [1]: [d_week_seq#67] -Input [2]: [d_week_seq#67, d_year#68] +Output [1]: [d_week_seq#69] +Input [2]: [d_week_seq#69, d_year#70] (30) BroadcastExchange -Input [1]: [d_week_seq#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] +Input [1]: [d_week_seq#69] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] (31) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#67] +Left keys [1]: [d_week_seq#53] +Right keys [1]: [d_week_seq#69] Join condition: None (32) Project [codegen id : 11] -Output [8]: [d_week_seq#10 AS d_week_seq2#70, sun_sales#35 AS sun_sales2#71, mon_sales#36 AS mon_sales2#72, tue_sales#37 AS tue_sales2#73, wed_sales#38 AS wed_sales2#74, thu_sales#39 AS thu_sales2#75, fri_sales#40 AS fri_sales2#76, sat_sales#41 AS sat_sales2#77] -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#67] +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] (33) BroadcastExchange -Input [8]: [d_week_seq2#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#78] +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] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq1#45] -Right keys [1]: [(d_week_seq2#70 - 53)] +Right keys [1]: [(d_week_seq2#72 - 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#71)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#79, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#72)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#80, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#73)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#81, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#74)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#82, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#75)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#83, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#76)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#84, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#77)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#85] -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#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] +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] (36) Exchange -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] -Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#86] +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] (37) Sort [codegen id : 13] -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] +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: [d_week_seq1#45 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt index adb5346969d06..a40286f9f8f83 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt @@ -55,7 +55,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -86,7 +86,7 @@ Arguments: [cs_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -94,7 +94,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt index 1816433bdd129..4f5eebc0c300c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt @@ -44,7 +44,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -52,7 +52,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -79,7 +79,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt index a04d4e6f5a5c2..efb45b5ccdb7f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt @@ -148,7 +148,7 @@ Results [4]: [w_warehouse_name#14, i_item_id#7, sum(CASE WHEN (d_date#11 < 2000- (26) Filter [codegen id : 5] Input [4]: [w_warehouse_name#14, i_item_id#7, inv_before#23, inv_after#24] -Condition : (((((isnotnull(inv_before#23) AND isnotnull(inv_after#24)) AND (inv_before#23 > 0)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) >= 0.666667)) AND (inv_before#23 > 0)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) <= 1.5)) +Condition : ((isnotnull(inv_before#23) AND isnotnull(inv_after#24)) AND (((inv_before#23 > 0) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) >= 0.666667)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) <= 1.5))) (27) TakeOrderedAndProject Input [4]: [w_warehouse_name#14, i_item_id#7, inv_before#23, inv_after#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt index 38d49852134cd..7fcbe1befa6b4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt @@ -148,7 +148,7 @@ Results [4]: [w_warehouse_name#7, i_item_id#10, sum(CASE WHEN (d_date#14 < 2000- (26) Filter [codegen id : 5] Input [4]: [w_warehouse_name#7, i_item_id#10, inv_before#23, inv_after#24] -Condition : (((((isnotnull(inv_before#23) AND isnotnull(inv_after#24)) AND (inv_before#23 > 0)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) >= 0.666667)) AND (inv_before#23 > 0)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) <= 1.5)) +Condition : ((isnotnull(inv_before#23) AND isnotnull(inv_after#24)) AND (((inv_before#23 > 0) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) >= 0.666667)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) <= 1.5))) (27) TakeOrderedAndProject Input [4]: [w_warehouse_name#7, i_item_id#10, inv_before#23, inv_after#24] 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 aa96edd4c7fba..f42a7615324c2 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 @@ -246,91 +246,91 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (33) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (35) Filter [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Condition : isnotnull(ss_customer_sk#26) (36) Project [codegen id : 11] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (37) Exchange Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#29] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#30] (38) Sort [codegen id : 12] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (39) Scan parquet default.customer -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 13] -Input [1]: [c_customer_sk#30] +Input [1]: [c_customer_sk#31] (41) Filter [codegen id : 13] -Input [1]: [c_customer_sk#30] -Condition : isnotnull(c_customer_sk#30) +Input [1]: [c_customer_sk#31] +Condition : isnotnull(c_customer_sk#31) (42) Exchange -Input [1]: [c_customer_sk#30] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [1]: [c_customer_sk#31] +Arguments: hashpartitioning(c_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] (43) Sort [codegen id : 14] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (44) SortMergeJoin [codegen id : 15] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#31] Join condition: None (45) Project [codegen id : 15] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#31] (46) HashAggregate [codegen id : 15] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Keys [1]: [c_customer_sk#31] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Aggregate Attributes [2]: [sum#33, isEmpty#34] +Results [3]: [c_customer_sk#31, sum#35, isEmpty#36] (47) HashAggregate [codegen id : 15] -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Keys [1]: [c_customer_sk#30] +Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] +Keys [1]: [c_customer_sk#31] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (48) Filter [codegen id : 15] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (49) Project [codegen id : 15] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#31] +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (50) Sort [codegen id : 15] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (51) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#31] Join condition: None (52) Project [codegen id : 17] @@ -338,209 +338,209 @@ Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (53) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_year#13, d_moy#40] +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 16] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (55) Filter [codegen id : 16] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#40)) AND (d_year#13 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#11)) +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) (56) Project [codegen id : 16] -Output [1]: [d_date_sk#11] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] +Output [1]: [d_date_sk#43] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (57) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] (58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#11] +Right keys [1]: [d_date_sk#43] Join condition: None (59) Project [codegen id : 17] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#42] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#11] +Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#47] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] (60) Scan parquet default.web_sales -Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#6)] ReadSchema: struct (61) ColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (62) Exchange -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: hashpartitioning(ws_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: hashpartitioning(ws_item_sk#48, 5), ENSURE_REQUIREMENTS, [id=#53] (63) Sort [codegen id : 19] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: [ws_item_sk#43 ASC NULLS FIRST], false, 0 +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: [ws_item_sk#48 ASC NULLS FIRST], false, 0 (64) ReusedExchange [Reuses operator id: unknown] -Output [2]: [ss_item_sk#8, d_date#12] +Output [2]: [ss_item_sk#54, d_date#55] (65) Sort [codegen id : 22] -Input [2]: [ss_item_sk#8, d_date#12] -Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 +Input [2]: [ss_item_sk#54, d_date#55] +Arguments: [ss_item_sk#54 ASC NULLS FIRST], false, 0 (66) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#16, i_item_desc#17] +Output [2]: [i_item_sk#56, i_item_desc#57] (67) Sort [codegen id : 24] -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16 ASC NULLS FIRST], false, 0 +Input [2]: [i_item_sk#56, i_item_desc#57] +Arguments: [i_item_sk#56 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#8] -Right keys [1]: [i_item_sk#16] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#56] Join condition: None (69) Project [codegen id : 25] -Output [3]: [d_date#12, i_item_sk#16, i_item_desc#17] -Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#16, i_item_desc#17] +Output [3]: [d_date#55, i_item_sk#56, i_item_desc#57] +Input [4]: [ss_item_sk#54, d_date#55, i_item_sk#56, i_item_desc#57] (70) HashAggregate [codegen id : 25] -Input [3]: [d_date#12, i_item_sk#16, i_item_desc#17] -Keys [3]: [substr(i_item_desc#17, 1, 30) AS substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12] +Input [3]: [d_date#55, i_item_sk#56, i_item_desc#57] +Keys [3]: [substr(i_item_desc#57, 1, 30) AS substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#50] -Results [4]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12, count#51] +Aggregate Attributes [1]: [count#59] +Results [4]: [substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55, count#60] (71) HashAggregate [codegen id : 25] -Input [4]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12, count#51] -Keys [3]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12] +Input [4]: [substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55, count#60] +Keys [3]: [substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#52] -Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#52 AS count(1)#53] +Aggregate Attributes [1]: [count(1)#61] +Results [2]: [i_item_sk#56 AS item_sk#23, count(1)#61 AS count(1)#62] (72) Filter [codegen id : 25] -Input [2]: [item_sk#23, count(1)#53] -Condition : (count(1)#53 > 4) +Input [2]: [item_sk#23, count(1)#62] +Condition : (count(1)#62 > 4) (73) Project [codegen id : 25] Output [1]: [item_sk#23] -Input [2]: [item_sk#23, count(1)#53] +Input [2]: [item_sk#23, count(1)#62] (74) Sort [codegen id : 25] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 (75) SortMergeJoin -Left keys [1]: [ws_item_sk#43] +Left keys [1]: [ws_item_sk#48] Right keys [1]: [item_sk#23] Join condition: None (76) Project [codegen id : 26] -Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (77) Exchange -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: hashpartitioning(ws_bill_customer_sk#49, 5), ENSURE_REQUIREMENTS, [id=#63] (78) Sort [codegen id : 27] -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: [ws_bill_customer_sk#49 ASC NULLS FIRST], false, 0 (79) ReusedExchange [Reuses operator id: 37] -Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Output [3]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66] (80) Sort [codegen id : 29] -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66] +Arguments: [ss_customer_sk#64 ASC NULLS FIRST], false, 0 (81) ReusedExchange [Reuses operator id: 42] -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#67] (82) Sort [codegen id : 31] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#67] +Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 (83) SortMergeJoin [codegen id : 32] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ss_customer_sk#64] +Right keys [1]: [c_customer_sk#67] Join condition: None (84) Project [codegen id : 32] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#65, ss_sales_price#66, c_customer_sk#67] +Input [4]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66, c_customer_sk#67] (85) HashAggregate [codegen id : 32] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#55, isEmpty#56] -Results [3]: [c_customer_sk#30, sum#57, isEmpty#58] +Input [3]: [ss_quantity#65, ss_sales_price#66, c_customer_sk#67] +Keys [1]: [c_customer_sk#67] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#68, isEmpty#69] +Results [3]: [c_customer_sk#67, sum#70, isEmpty#71] (86) HashAggregate [codegen id : 32] -Input [3]: [c_customer_sk#30, sum#57, isEmpty#58] -Keys [1]: [c_customer_sk#30] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#59] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#59 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] +Input [3]: [c_customer_sk#67, sum#70, isEmpty#71] +Keys [1]: [c_customer_sk#67] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))#72] +Results [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))#72 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] (87) Filter [codegen id : 32] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (88) Project [codegen id : 32] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] +Output [1]: [c_customer_sk#67] +Input [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] (89) Sort [codegen id : 32] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#67] +Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#44] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ws_bill_customer_sk#49] +Right keys [1]: [c_customer_sk#67] Join condition: None (91) Project [codegen id : 34] -Output [3]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [3]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (92) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#74] (93) BroadcastHashJoin [codegen id : 34] -Left keys [1]: [ws_sold_date_sk#47] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ws_sold_date_sk#52] +Right keys [1]: [d_date_sk#74] Join condition: None (94) Project [codegen id : 34] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2), true) AS sales#61] -Input [4]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#11] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#51 as decimal(12,2)))), DecimalType(18,2), true) AS sales#75] +Input [4]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52, d_date_sk#74] (95) Union (96) HashAggregate [codegen id : 35] -Input [1]: [sales#42] +Input [1]: [sales#47] Keys: [] -Functions [1]: [partial_sum(sales#42)] -Aggregate Attributes [2]: [sum#62, isEmpty#63] -Results [2]: [sum#64, isEmpty#65] +Functions [1]: [partial_sum(sales#47)] +Aggregate Attributes [2]: [sum#76, isEmpty#77] +Results [2]: [sum#78, isEmpty#79] (97) Exchange -Input [2]: [sum#64, isEmpty#65] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] +Input [2]: [sum#78, isEmpty#79] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#80] (98) HashAggregate [codegen id : 36] -Input [2]: [sum#64, isEmpty#65] +Input [2]: [sum#78, isEmpty#79] Keys: [] -Functions [1]: [sum(sales#42)] -Aggregate Attributes [1]: [sum(sales#42)#67] -Results [1]: [sum(sales#42)#67 AS sum(sales)#68] +Functions [1]: [sum(sales#47)] +Aggregate Attributes [1]: [sum(sales#47)#81] +Results [1]: [sum(sales#47)#81 AS sum(sales)#82] ===== Subqueries ===== @@ -549,7 +549,7 @@ ReusedExchange (99) (99) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#43] Subquery:2 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 ReusedExchange (100) @@ -558,7 +558,7 @@ ReusedExchange (100) (100) ReusedExchange [Reuses operator id: 12] Output [2]: [d_date_sk#11, d_date#12] -Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (124) +- Exchange (123) +- * HashAggregate (122) @@ -586,131 +586,131 @@ Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Output [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#69)] +PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#87)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] (103) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] -Condition : isnotnull(ss_customer_sk#26) +Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] +Condition : isnotnull(ss_customer_sk#83) (104) Scan parquet default.date_dim -Output [2]: [d_date_sk#11, d_year#13] +Output [2]: [d_date_sk#88, d_year#89] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (105) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#11, d_year#13] +Input [2]: [d_date_sk#88, d_year#89] (106) Filter [codegen id : 1] -Input [2]: [d_date_sk#11, d_year#13] -Condition : (d_year#13 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) +Input [2]: [d_date_sk#88, d_year#89] +Condition : (d_year#89 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#88)) (107) Project [codegen id : 1] -Output [1]: [d_date_sk#11] -Input [2]: [d_date_sk#11, d_year#13] +Output [1]: [d_date_sk#88] +Input [2]: [d_date_sk#88, d_year#89] (108) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#70] +Input [1]: [d_date_sk#88] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#90] (109) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ss_sold_date_sk#86] +Right keys [1]: [d_date_sk#88] Join condition: None (110) Project [codegen id : 2] -Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9, d_date_sk#11] +Output [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] +Input [5]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86, d_date_sk#88] (111) Exchange -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] +Arguments: hashpartitioning(ss_customer_sk#83, 5), ENSURE_REQUIREMENTS, [id=#91] (112) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] +Arguments: [ss_customer_sk#83 ASC NULLS FIRST], false, 0 (113) Scan parquet default.customer -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#92] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (114) ColumnarToRow [codegen id : 4] -Input [1]: [c_customer_sk#30] +Input [1]: [c_customer_sk#92] (115) Filter [codegen id : 4] -Input [1]: [c_customer_sk#30] -Condition : isnotnull(c_customer_sk#30) +Input [1]: [c_customer_sk#92] +Condition : isnotnull(c_customer_sk#92) (116) Exchange -Input [1]: [c_customer_sk#30] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [1]: [c_customer_sk#92] +Arguments: hashpartitioning(c_customer_sk#92, 5), ENSURE_REQUIREMENTS, [id=#93] (117) Sort [codegen id : 5] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#92] +Arguments: [c_customer_sk#92 ASC NULLS FIRST], false, 0 (118) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ss_customer_sk#83] +Right keys [1]: [c_customer_sk#92] Join condition: None (119) Project [codegen id : 6] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#92] +Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, c_customer_sk#92] (120) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#73, isEmpty#74] -Results [3]: [c_customer_sk#30, sum#75, isEmpty#76] +Input [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#92] +Keys [1]: [c_customer_sk#92] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#94, isEmpty#95] +Results [3]: [c_customer_sk#92, sum#96, isEmpty#97] (121) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#30, sum#75, isEmpty#76] -Keys [1]: [c_customer_sk#30] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#77] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#77 AS csales#78] +Input [3]: [c_customer_sk#92, sum#96, isEmpty#97] +Keys [1]: [c_customer_sk#92] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98 AS csales#99] (122) HashAggregate [codegen id : 6] -Input [1]: [csales#78] +Input [1]: [csales#99] Keys: [] -Functions [1]: [partial_max(csales#78)] -Aggregate Attributes [1]: [max#79] -Results [1]: [max#80] +Functions [1]: [partial_max(csales#99)] +Aggregate Attributes [1]: [max#100] +Results [1]: [max#101] (123) Exchange -Input [1]: [max#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +Input [1]: [max#101] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#102] (124) HashAggregate [codegen id : 7] -Input [1]: [max#80] +Input [1]: [max#101] Keys: [] -Functions [1]: [max(csales#78)] -Aggregate Attributes [1]: [max(csales#78)#82] -Results [1]: [max(csales#78)#82 AS tpcds_cmax#83] +Functions [1]: [max(csales#99)] +Aggregate Attributes [1]: [max(csales#99)#103] +Results [1]: [max(csales#99)#103 AS tpcds_cmax#104] -Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#69 +Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#87 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 108] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#88] -Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:6 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] 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 692671c9f287d..8c2aed03ce0cc 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 @@ -206,83 +206,83 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (29) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (31) Filter [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Condition : isnotnull(ss_customer_sk#26) (32) Project [codegen id : 8] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (33) Scan parquet default.customer -Output [1]: [c_customer_sk#29] +Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 7] -Input [1]: [c_customer_sk#29] +Input [1]: [c_customer_sk#30] (35) Filter [codegen id : 7] -Input [1]: [c_customer_sk#29] -Condition : isnotnull(c_customer_sk#29) +Input [1]: [c_customer_sk#30] +Condition : isnotnull(c_customer_sk#30) (36) BroadcastExchange -Input [1]: [c_customer_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [1]: [c_customer_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (37) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#30] Join condition: None (38) Project [codegen id : 8] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (39) HashAggregate [codegen id : 8] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#31, isEmpty#32] -Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] (40) Exchange -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#36] (41) HashAggregate [codegen id : 9] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (42) Filter [codegen id : 9] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (43) Project [codegen id : 9] -Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (44) Sort [codegen id : 9] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (45) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#30] Join condition: None (46) Project [codegen id : 11] @@ -290,128 +290,128 @@ Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (47) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_year#12, d_moy#40] +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 10] -Input [3]: [d_date_sk#10, d_year#12, d_moy#40] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (49) Filter [codegen id : 10] -Input [3]: [d_date_sk#10, d_year#12, d_moy#40] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#40)) AND (d_year#12 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#10)) +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) (50) Project [codegen id : 10] -Output [1]: [d_date_sk#10] -Input [3]: [d_date_sk#10, d_year#12, d_moy#40] +Output [1]: [d_date_sk#43] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (51) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] (52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#10] +Right keys [1]: [d_date_sk#43] Join condition: None (53) Project [codegen id : 11] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#42] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#10] +Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#47] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] (54) Scan parquet default.web_sales -Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#6)] ReadSchema: struct (55) ColumnarToRow [codegen id : 16] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (56) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] (57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#43] +Left keys [1]: [ws_item_sk#48] Right keys [1]: [item_sk#22] Join condition: None (58) Project [codegen id : 16] -Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (59) Exchange -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: hashpartitioning(ws_bill_customer_sk#49, 5), ENSURE_REQUIREMENTS, [id=#53] (60) Sort [codegen id : 17] -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: [ws_bill_customer_sk#49 ASC NULLS FIRST], false, 0 (61) ReusedExchange [Reuses operator id: 40] -Output [3]: [c_customer_sk#29, sum#49, isEmpty#50] +Output [3]: [c_customer_sk#54, sum#55, isEmpty#56] (62) HashAggregate [codegen id : 20] -Input [3]: [c_customer_sk#29, sum#49, isEmpty#50] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#51] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#51 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] +Input [3]: [c_customer_sk#54, sum#55, isEmpty#56] +Keys [1]: [c_customer_sk#54] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))#59] +Results [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))#59 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] (63) Filter [codegen id : 20] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (64) Project [codegen id : 20] -Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] +Output [1]: [c_customer_sk#54] +Input [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] (65) Sort [codegen id : 20] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#54] +Arguments: [c_customer_sk#54 ASC NULLS FIRST], false, 0 (66) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#44] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ws_bill_customer_sk#49] +Right keys [1]: [c_customer_sk#54] Join condition: None (67) Project [codegen id : 22] -Output [3]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [3]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (68) ReusedExchange [Reuses operator id: 51] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#61] (69) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#47] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#52] +Right keys [1]: [d_date_sk#61] Join condition: None (70) Project [codegen id : 22] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2), true) AS sales#53] -Input [4]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#10] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#51 as decimal(12,2)))), DecimalType(18,2), true) AS sales#62] +Input [4]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52, d_date_sk#61] (71) Union (72) HashAggregate [codegen id : 23] -Input [1]: [sales#42] +Input [1]: [sales#47] Keys: [] -Functions [1]: [partial_sum(sales#42)] -Aggregate Attributes [2]: [sum#54, isEmpty#55] -Results [2]: [sum#56, isEmpty#57] +Functions [1]: [partial_sum(sales#47)] +Aggregate Attributes [2]: [sum#63, isEmpty#64] +Results [2]: [sum#65, isEmpty#66] (73) Exchange -Input [2]: [sum#56, isEmpty#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [sum#65, isEmpty#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] (74) HashAggregate [codegen id : 24] -Input [2]: [sum#56, isEmpty#57] +Input [2]: [sum#65, isEmpty#66] Keys: [] -Functions [1]: [sum(sales#42)] -Aggregate Attributes [1]: [sum(sales#42)#59] -Results [1]: [sum(sales#42)#59 AS sum(sales)#60] +Functions [1]: [sum(sales#47)] +Aggregate Attributes [1]: [sum(sales#47)#68] +Results [1]: [sum(sales#47)#68 AS sum(sales)#69] ===== Subqueries ===== @@ -420,7 +420,7 @@ ReusedExchange (75) (75) ReusedExchange [Reuses operator id: 51] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#43] Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 ReusedExchange (76) @@ -429,7 +429,7 @@ ReusedExchange (76) (76) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#10, d_date#11] -Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (98) +- Exchange (97) +- * HashAggregate (96) @@ -455,123 +455,123 @@ Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquer (77) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Output [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (78) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] (79) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] -Condition : isnotnull(ss_customer_sk#26) +Input [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] +Condition : isnotnull(ss_customer_sk#70) (80) Scan parquet default.customer -Output [1]: [c_customer_sk#29] +Output [1]: [c_customer_sk#75] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (81) ColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#29] +Input [1]: [c_customer_sk#75] (82) Filter [codegen id : 1] -Input [1]: [c_customer_sk#29] -Condition : isnotnull(c_customer_sk#29) +Input [1]: [c_customer_sk#75] +Condition : isnotnull(c_customer_sk#75) (83) BroadcastExchange -Input [1]: [c_customer_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#62] +Input [1]: [c_customer_sk#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] (84) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ss_customer_sk#70] +Right keys [1]: [c_customer_sk#75] Join condition: None (85) Project [codegen id : 3] -Output [4]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] -Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] +Output [4]: [ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75] +Input [5]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#12] +Output [2]: [d_date_sk#77, d_year#78] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#10, d_year#12] +Input [2]: [d_date_sk#77, d_year#78] (88) Filter [codegen id : 2] -Input [2]: [d_date_sk#10, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +Input [2]: [d_date_sk#77, d_year#78] +Condition : (d_year#78 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#77)) (89) Project [codegen id : 2] -Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_year#12] +Output [1]: [d_date_sk#77] +Input [2]: [d_date_sk#77, d_year#78] (90) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] +Input [1]: [d_date_sk#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#79] (91) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#73] +Right keys [1]: [d_date_sk#77] Join condition: None (92) Project [codegen id : 3] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Input [5]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29, d_date_sk#10] +Output [3]: [ss_quantity#71, ss_sales_price#72, c_customer_sk#75] +Input [5]: [ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75, d_date_sk#77] (93) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#64, isEmpty#65] -Results [3]: [c_customer_sk#29, sum#66, isEmpty#67] +Input [3]: [ss_quantity#71, ss_sales_price#72, c_customer_sk#75] +Keys [1]: [c_customer_sk#75] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#80, isEmpty#81] +Results [3]: [c_customer_sk#75, sum#82, isEmpty#83] (94) Exchange -Input [3]: [c_customer_sk#29, sum#66, isEmpty#67] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [3]: [c_customer_sk#75, sum#82, isEmpty#83] +Arguments: hashpartitioning(c_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#84] (95) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#29, sum#66, isEmpty#67] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS csales#70] +Input [3]: [c_customer_sk#75, sum#82, isEmpty#83] +Keys [1]: [c_customer_sk#75] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))#85] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))#85 AS csales#86] (96) HashAggregate [codegen id : 4] -Input [1]: [csales#70] +Input [1]: [csales#86] Keys: [] -Functions [1]: [partial_max(csales#70)] -Aggregate Attributes [1]: [max#71] -Results [1]: [max#72] +Functions [1]: [partial_max(csales#86)] +Aggregate Attributes [1]: [max#87] +Results [1]: [max#88] (97) Exchange -Input [1]: [max#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#73] +Input [1]: [max#88] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#89] (98) HashAggregate [codegen id : 5] -Input [1]: [max#72] +Input [1]: [max#88] Keys: [] -Functions [1]: [max(csales#70)] -Aggregate Attributes [1]: [max(csales#70)#74] -Results [1]: [max(csales#70)#74 AS tpcds_cmax#75] +Functions [1]: [max(csales#86)] +Aggregate Attributes [1]: [max(csales#86)#90] +Results [1]: [max(csales#86)#90 AS tpcds_cmax#91] -Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#61 +Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 ReusedExchange (99) (99) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#77] -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:6 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] 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 35a42fa256325..e75b223ea599a 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 @@ -290,145 +290,145 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (34) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (36) Filter [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Condition : isnotnull(ss_customer_sk#26) (37) Project [codegen id : 11] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (38) Exchange Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#29] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#30] (39) Sort [codegen id : 12] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (40) Scan parquet default.customer -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 13] -Input [1]: [c_customer_sk#30] +Input [1]: [c_customer_sk#31] (42) Filter [codegen id : 13] -Input [1]: [c_customer_sk#30] -Condition : isnotnull(c_customer_sk#30) +Input [1]: [c_customer_sk#31] +Condition : isnotnull(c_customer_sk#31) (43) Exchange -Input [1]: [c_customer_sk#30] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [1]: [c_customer_sk#31] +Arguments: hashpartitioning(c_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] (44) Sort [codegen id : 14] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 15] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#31] Join condition: None (46) Project [codegen id : 15] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#31] (47) HashAggregate [codegen id : 15] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Keys [1]: [c_customer_sk#31] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Aggregate Attributes [2]: [sum#33, isEmpty#34] +Results [3]: [c_customer_sk#31, sum#35, isEmpty#36] (48) HashAggregate [codegen id : 15] -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Keys [1]: [c_customer_sk#30] +Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] +Keys [1]: [c_customer_sk#31] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (49) Filter [codegen id : 15] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (50) Project [codegen id : 15] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#31] +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (51) Sort [codegen id : 15] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (52) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#31] Join condition: None (53) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_year#13, d_moy#40] +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 16] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (55) Filter [codegen id : 16] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#40)) AND (d_year#13 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#11)) +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) (56) Project [codegen id : 16] -Output [1]: [d_date_sk#11] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] +Output [1]: [d_date_sk#43] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (57) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] (58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#11] +Right keys [1]: [d_date_sk#43] Join condition: None (59) Project [codegen id : 17] Output [3]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4] -Input [5]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#11] +Input [5]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] (60) Scan parquet default.customer -Output [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Output [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 18] -Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] (62) Filter [codegen id : 18] -Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] -Condition : isnotnull(c_customer_sk#30) +Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] +Condition : isnotnull(c_customer_sk#47) (63) Exchange -Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] +Arguments: hashpartitioning(c_customer_sk#47, 5), ENSURE_REQUIREMENTS, [id=#50] (64) Sort [codegen id : 19] -Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] +Arguments: [c_customer_sk#47 ASC NULLS FIRST], false, 0 (65) ReusedExchange [Reuses operator id: 38] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] @@ -438,327 +438,327 @@ Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (67) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#31] (68) Sort [codegen id : 23] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (69) SortMergeJoin [codegen id : 24] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#31] Join condition: None (70) Project [codegen id : 24] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#31] (71) HashAggregate [codegen id : 24] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Keys [1]: [c_customer_sk#31] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Aggregate Attributes [2]: [sum#33, isEmpty#34] +Results [3]: [c_customer_sk#31, sum#35, isEmpty#36] (72) HashAggregate [codegen id : 24] -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Keys [1]: [c_customer_sk#30] +Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] +Keys [1]: [c_customer_sk#31] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (73) Filter [codegen id : 24] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (74) Project [codegen id : 24] -Output [1]: [c_customer_sk#30 AS c_customer_sk#30#45] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#31] +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (75) Sort [codegen id : 24] -Input [1]: [c_customer_sk#30#45] -Arguments: [c_customer_sk#30#45 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (76) SortMergeJoin -Left keys [1]: [c_customer_sk#30] -Right keys [1]: [c_customer_sk#30#45] +Left keys [1]: [c_customer_sk#47] +Right keys [1]: [c_customer_sk#31] Join condition: None (77) SortMergeJoin [codegen id : 25] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#47] Join condition: None (78) Project [codegen id : 25] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#42, c_last_name#43] -Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#30, c_first_name#42, c_last_name#43] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#48, c_last_name#49] +Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#47, c_first_name#48, c_last_name#49] (79) HashAggregate [codegen id : 25] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#42, c_last_name#43] -Keys [2]: [c_last_name#43, c_first_name#42] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#48, c_last_name#49] +Keys [2]: [c_last_name#49, c_first_name#48] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#46, isEmpty#47] -Results [4]: [c_last_name#43, c_first_name#42, sum#48, isEmpty#49] +Aggregate Attributes [2]: [sum#51, isEmpty#52] +Results [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] (80) Exchange -Input [4]: [c_last_name#43, c_first_name#42, sum#48, isEmpty#49] -Arguments: hashpartitioning(c_last_name#43, c_first_name#42, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] +Arguments: hashpartitioning(c_last_name#49, c_first_name#48, 5), ENSURE_REQUIREMENTS, [id=#55] (81) HashAggregate [codegen id : 26] -Input [4]: [c_last_name#43, c_first_name#42, sum#48, isEmpty#49] -Keys [2]: [c_last_name#43, c_first_name#42] +Input [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] +Keys [2]: [c_last_name#49, c_first_name#48] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#51] -Results [3]: [c_last_name#43, c_first_name#42, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#51 AS sales#52] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#56] +Results [3]: [c_last_name#49, c_first_name#48, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sales#57] (82) Scan parquet default.web_sales -Output [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Output [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#57), dynamicpruningexpression(ws_sold_date_sk#57 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (83) ColumnarToRow [codegen id : 27] -Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] (84) Filter [codegen id : 27] -Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Condition : isnotnull(ws_bill_customer_sk#54) +Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Condition : isnotnull(ws_bill_customer_sk#59) (85) Exchange -Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Arguments: hashpartitioning(ws_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Arguments: hashpartitioning(ws_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#63] (86) Sort [codegen id : 28] -Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Arguments: [ws_item_sk#53 ASC NULLS FIRST], false, 0 +Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Arguments: [ws_item_sk#58 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: unknown] -Output [2]: [ss_item_sk#8, d_date#12] +Output [2]: [ss_item_sk#64, d_date#65] (88) Sort [codegen id : 31] -Input [2]: [ss_item_sk#8, d_date#12] -Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 +Input [2]: [ss_item_sk#64, d_date#65] +Arguments: [ss_item_sk#64 ASC NULLS FIRST], false, 0 (89) ReusedExchange [Reuses operator id: 21] -Output [2]: [i_item_sk#16, i_item_desc#17] +Output [2]: [i_item_sk#66, i_item_desc#67] (90) Sort [codegen id : 33] -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16 ASC NULLS FIRST], false, 0 +Input [2]: [i_item_sk#66, i_item_desc#67] +Arguments: [i_item_sk#66 ASC NULLS FIRST], false, 0 (91) SortMergeJoin [codegen id : 34] -Left keys [1]: [ss_item_sk#8] -Right keys [1]: [i_item_sk#16] +Left keys [1]: [ss_item_sk#64] +Right keys [1]: [i_item_sk#66] Join condition: None (92) Project [codegen id : 34] -Output [3]: [d_date#12, i_item_sk#16, i_item_desc#17] -Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#16, i_item_desc#17] +Output [3]: [d_date#65, i_item_sk#66, i_item_desc#67] +Input [4]: [ss_item_sk#64, d_date#65, i_item_sk#66, i_item_desc#67] (93) HashAggregate [codegen id : 34] -Input [3]: [d_date#12, i_item_sk#16, i_item_desc#17] -Keys [3]: [substr(i_item_desc#17, 1, 30) AS substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12] +Input [3]: [d_date#65, i_item_sk#66, i_item_desc#67] +Keys [3]: [substr(i_item_desc#67, 1, 30) AS substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#60] -Results [4]: [substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12, count#61] +Aggregate Attributes [1]: [count#69] +Results [4]: [substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65, count#70] (94) HashAggregate [codegen id : 34] -Input [4]: [substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12, count#61] -Keys [3]: [substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12] +Input [4]: [substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65, count#70] +Keys [3]: [substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#62] -Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#62 AS count(1)#63] +Aggregate Attributes [1]: [count(1)#71] +Results [2]: [i_item_sk#66 AS item_sk#23, count(1)#71 AS count(1)#72] (95) Filter [codegen id : 34] -Input [2]: [item_sk#23, count(1)#63] -Condition : (count(1)#63 > 4) +Input [2]: [item_sk#23, count(1)#72] +Condition : (count(1)#72 > 4) (96) Project [codegen id : 34] Output [1]: [item_sk#23] -Input [2]: [item_sk#23, count(1)#63] +Input [2]: [item_sk#23, count(1)#72] (97) Sort [codegen id : 34] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ws_item_sk#53] +Left keys [1]: [ws_item_sk#58] Right keys [1]: [item_sk#23] Join condition: None (99) Project [codegen id : 35] -Output [4]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Output [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] (100) Exchange -Input [4]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Arguments: hashpartitioning(ws_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Arguments: hashpartitioning(ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#73] (101) Sort [codegen id : 36] -Input [4]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Arguments: [ws_bill_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Arguments: [ws_bill_customer_sk#59 ASC NULLS FIRST], false, 0 (102) ReusedExchange [Reuses operator id: 38] -Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Output [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] (103) Sort [codegen id : 38] -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] +Arguments: [ss_customer_sk#74 ASC NULLS FIRST], false, 0 (104) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#77] (105) Sort [codegen id : 40] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#77] +Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 (106) SortMergeJoin [codegen id : 41] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ss_customer_sk#74] +Right keys [1]: [c_customer_sk#77] Join condition: None (107) Project [codegen id : 41] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, c_customer_sk#77] (108) HashAggregate [codegen id : 41] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#65, isEmpty#66] -Results [3]: [c_customer_sk#30, sum#67, isEmpty#68] +Input [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Keys [1]: [c_customer_sk#77] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#78, isEmpty#79] +Results [3]: [c_customer_sk#77, sum#80, isEmpty#81] (109) HashAggregate [codegen id : 41] -Input [3]: [c_customer_sk#30, sum#67, isEmpty#68] -Keys [1]: [c_customer_sk#30] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] +Input [3]: [c_customer_sk#77, sum#80, isEmpty#81] +Keys [1]: [c_customer_sk#77] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82] +Results [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] (110) Filter [codegen id : 41] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (111) Project [codegen id : 41] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] +Output [1]: [c_customer_sk#77] +Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] (112) Sort [codegen id : 41] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#77] +Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 (113) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#54] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ws_bill_customer_sk#59] +Right keys [1]: [c_customer_sk#77] Join condition: None (114) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#84] (115) BroadcastHashJoin [codegen id : 43] -Left keys [1]: [ws_sold_date_sk#57] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ws_sold_date_sk#62] +Right keys [1]: [d_date_sk#84] Join condition: None (116) Project [codegen id : 43] -Output [3]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56] -Input [5]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57, d_date_sk#11] +Output [3]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61] +Input [5]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62, d_date_sk#84] (117) ReusedExchange [Reuses operator id: 63] -Output [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Output [3]: [c_customer_sk#85, c_first_name#86, c_last_name#87] (118) Sort [codegen id : 45] -Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#85, c_first_name#86, c_last_name#87] +Arguments: [c_customer_sk#85 ASC NULLS FIRST], false, 0 (119) ReusedExchange [Reuses operator id: 38] -Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Output [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] (120) Sort [codegen id : 47] -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] +Arguments: [ss_customer_sk#74 ASC NULLS FIRST], false, 0 (121) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#77] (122) Sort [codegen id : 49] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#77] +Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 (123) SortMergeJoin [codegen id : 50] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ss_customer_sk#74] +Right keys [1]: [c_customer_sk#77] Join condition: None (124) Project [codegen id : 50] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, c_customer_sk#77] (125) HashAggregate [codegen id : 50] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#65, isEmpty#66] -Results [3]: [c_customer_sk#30, sum#67, isEmpty#68] +Input [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Keys [1]: [c_customer_sk#77] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#78, isEmpty#79] +Results [3]: [c_customer_sk#77, sum#80, isEmpty#81] (126) HashAggregate [codegen id : 50] -Input [3]: [c_customer_sk#30, sum#67, isEmpty#68] -Keys [1]: [c_customer_sk#30] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] +Input [3]: [c_customer_sk#77, sum#80, isEmpty#81] +Keys [1]: [c_customer_sk#77] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82] +Results [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] (127) Filter [codegen id : 50] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (128) Project [codegen id : 50] -Output [1]: [c_customer_sk#30 AS c_customer_sk#30#71] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] +Output [1]: [c_customer_sk#77] +Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] (129) Sort [codegen id : 50] -Input [1]: [c_customer_sk#30#71] -Arguments: [c_customer_sk#30#71 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#77] +Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 (130) SortMergeJoin -Left keys [1]: [c_customer_sk#30] -Right keys [1]: [c_customer_sk#30#71] +Left keys [1]: [c_customer_sk#85] +Right keys [1]: [c_customer_sk#77] Join condition: None (131) SortMergeJoin [codegen id : 51] -Left keys [1]: [ws_bill_customer_sk#54] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ws_bill_customer_sk#59] +Right keys [1]: [c_customer_sk#85] Join condition: None (132) Project [codegen id : 51] -Output [4]: [ws_quantity#55, ws_list_price#56, c_first_name#42, c_last_name#43] -Input [6]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, c_customer_sk#30, c_first_name#42, c_last_name#43] +Output [4]: [ws_quantity#60, ws_list_price#61, c_first_name#86, c_last_name#87] +Input [6]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, c_customer_sk#85, c_first_name#86, c_last_name#87] (133) HashAggregate [codegen id : 51] -Input [4]: [ws_quantity#55, ws_list_price#56, c_first_name#42, c_last_name#43] -Keys [2]: [c_last_name#43, c_first_name#42] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#72, isEmpty#73] -Results [4]: [c_last_name#43, c_first_name#42, sum#74, isEmpty#75] +Input [4]: [ws_quantity#60, ws_list_price#61, c_first_name#86, c_last_name#87] +Keys [2]: [c_last_name#87, c_first_name#86] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#88, isEmpty#89] +Results [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] (134) Exchange -Input [4]: [c_last_name#43, c_first_name#42, sum#74, isEmpty#75] -Arguments: hashpartitioning(c_last_name#43, c_first_name#42, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] +Arguments: hashpartitioning(c_last_name#87, c_first_name#86, 5), ENSURE_REQUIREMENTS, [id=#92] (135) HashAggregate [codegen id : 52] -Input [4]: [c_last_name#43, c_first_name#42, sum#74, isEmpty#75] -Keys [2]: [c_last_name#43, c_first_name#42] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))#77] -Results [3]: [c_last_name#43, c_first_name#42, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))#77 AS sales#78] +Input [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] +Keys [2]: [c_last_name#87, c_first_name#86] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#93] +Results [3]: [c_last_name#87, c_first_name#86, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#93 AS sales#94] (136) Union (137) TakeOrderedAndProject -Input [3]: [c_last_name#43, c_first_name#42, sales#52] -Arguments: 100, [c_last_name#43 ASC NULLS FIRST, c_first_name#42 ASC NULLS FIRST, sales#52 ASC NULLS FIRST], [c_last_name#43, c_first_name#42, sales#52] +Input [3]: [c_last_name#49, c_first_name#48, sales#57] +Arguments: 100, [c_last_name#49 ASC NULLS FIRST, c_first_name#48 ASC NULLS FIRST, sales#57 ASC NULLS FIRST], [c_last_name#49, c_first_name#48, sales#57] ===== Subqueries ===== @@ -767,7 +767,7 @@ ReusedExchange (138) (138) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#43] Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 ReusedExchange (139) @@ -776,7 +776,7 @@ ReusedExchange (139) (139) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#11, d_date#12] -Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (163) +- Exchange (162) +- * HashAggregate (161) @@ -804,135 +804,135 @@ Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (140) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Output [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#79)] +PartitionFilters: [isnotnull(ss_sold_date_sk#98), dynamicpruningexpression(ss_sold_date_sk#98 IN dynamicpruning#99)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (141) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] (142) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] -Condition : isnotnull(ss_customer_sk#26) +Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] +Condition : isnotnull(ss_customer_sk#95) (143) Scan parquet default.date_dim -Output [2]: [d_date_sk#11, d_year#13] +Output [2]: [d_date_sk#100, d_year#101] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#11, d_year#13] +Input [2]: [d_date_sk#100, d_year#101] (145) Filter [codegen id : 1] -Input [2]: [d_date_sk#11, d_year#13] -Condition : (d_year#13 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) +Input [2]: [d_date_sk#100, d_year#101] +Condition : (d_year#101 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#100)) (146) Project [codegen id : 1] -Output [1]: [d_date_sk#11] -Input [2]: [d_date_sk#11, d_year#13] +Output [1]: [d_date_sk#100] +Input [2]: [d_date_sk#100, d_year#101] (147) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#80] +Input [1]: [d_date_sk#100] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#102] (148) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ss_sold_date_sk#98] +Right keys [1]: [d_date_sk#100] Join condition: None (149) Project [codegen id : 2] -Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9, d_date_sk#11] +Output [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] +Input [5]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98, d_date_sk#100] (150) Exchange -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] +Arguments: hashpartitioning(ss_customer_sk#95, 5), ENSURE_REQUIREMENTS, [id=#103] (151) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] +Arguments: [ss_customer_sk#95 ASC NULLS FIRST], false, 0 (152) Scan parquet default.customer -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#104] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (153) ColumnarToRow [codegen id : 4] -Input [1]: [c_customer_sk#30] +Input [1]: [c_customer_sk#104] (154) Filter [codegen id : 4] -Input [1]: [c_customer_sk#30] -Condition : isnotnull(c_customer_sk#30) +Input [1]: [c_customer_sk#104] +Condition : isnotnull(c_customer_sk#104) (155) Exchange -Input [1]: [c_customer_sk#30] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#82] +Input [1]: [c_customer_sk#104] +Arguments: hashpartitioning(c_customer_sk#104, 5), ENSURE_REQUIREMENTS, [id=#105] (156) Sort [codegen id : 5] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#104] +Arguments: [c_customer_sk#104 ASC NULLS FIRST], false, 0 (157) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ss_customer_sk#95] +Right keys [1]: [c_customer_sk#104] Join condition: None (158) Project [codegen id : 6] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#96, ss_sales_price#97, c_customer_sk#104] +Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, c_customer_sk#104] (159) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#83, isEmpty#84] -Results [3]: [c_customer_sk#30, sum#85, isEmpty#86] +Input [3]: [ss_quantity#96, ss_sales_price#97, c_customer_sk#104] +Keys [1]: [c_customer_sk#104] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#106, isEmpty#107] +Results [3]: [c_customer_sk#104, sum#108, isEmpty#109] (160) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#30, sum#85, isEmpty#86] -Keys [1]: [c_customer_sk#30] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#87] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#87 AS csales#88] +Input [3]: [c_customer_sk#104, sum#108, isEmpty#109] +Keys [1]: [c_customer_sk#104] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))#110] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))#110 AS csales#111] (161) HashAggregate [codegen id : 6] -Input [1]: [csales#88] +Input [1]: [csales#111] Keys: [] -Functions [1]: [partial_max(csales#88)] -Aggregate Attributes [1]: [max#89] -Results [1]: [max#90] +Functions [1]: [partial_max(csales#111)] +Aggregate Attributes [1]: [max#112] +Results [1]: [max#113] (162) Exchange -Input [1]: [max#90] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#91] +Input [1]: [max#113] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] (163) HashAggregate [codegen id : 7] -Input [1]: [max#90] +Input [1]: [max#113] Keys: [] -Functions [1]: [max(csales#88)] -Aggregate Attributes [1]: [max(csales#88)#92] -Results [1]: [max(csales#88)#92 AS tpcds_cmax#93] +Functions [1]: [max(csales#111)] +Aggregate Attributes [1]: [max(csales#111)#115] +Results [1]: [max(csales#111)#115 AS tpcds_cmax#116] -Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#79 +Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#98 IN dynamicpruning#99 ReusedExchange (164) (164) ReusedExchange [Reuses operator id: 147] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#100] -Subquery:5 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:5 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] -Subquery:6 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#57 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 110 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:7 Hosting operator id = 110 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] -Subquery:8 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:8 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] 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 385cf017e58c3..f5c64908c8fb9 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 @@ -240,340 +240,340 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (30) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (32) Filter [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Condition : isnotnull(ss_customer_sk#26) (33) Project [codegen id : 8] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (34) Scan parquet default.customer -Output [1]: [c_customer_sk#29] +Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 7] -Input [1]: [c_customer_sk#29] +Input [1]: [c_customer_sk#30] (36) Filter [codegen id : 7] -Input [1]: [c_customer_sk#29] -Condition : isnotnull(c_customer_sk#29) +Input [1]: [c_customer_sk#30] +Condition : isnotnull(c_customer_sk#30) (37) BroadcastExchange -Input [1]: [c_customer_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [1]: [c_customer_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (38) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#30] Join condition: None (39) Project [codegen id : 8] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (40) HashAggregate [codegen id : 8] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#31, isEmpty#32] -Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] (41) Exchange -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#36] (42) HashAggregate [codegen id : 9] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (43) Filter [codegen id : 9] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (44) Project [codegen id : 9] -Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (45) Sort [codegen id : 9] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (46) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#30] Join condition: None (47) Scan parquet default.customer -Output [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Output [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 10] -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] (49) Filter [codegen id : 10] -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Condition : isnotnull(c_customer_sk#29) +Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Condition : isnotnull(c_customer_sk#43) (50) Exchange -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Arguments: hashpartitioning(c_customer_sk#43, 5), ENSURE_REQUIREMENTS, [id=#46] (51) Sort [codegen id : 11] -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Arguments: [c_customer_sk#43 ASC NULLS FIRST], false, 0 (52) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Output [3]: [c_customer_sk#30, sum#34, isEmpty#35] (53) HashAggregate [codegen id : 14] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (54) Filter [codegen id : 14] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (55) Project [codegen id : 14] -Output [1]: [c_customer_sk#29 AS c_customer_sk#29#43] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (56) Sort [codegen id : 14] -Input [1]: [c_customer_sk#29#43] -Arguments: [c_customer_sk#29#43 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (57) SortMergeJoin -Left keys [1]: [c_customer_sk#29] -Right keys [1]: [c_customer_sk#29#43] +Left keys [1]: [c_customer_sk#43] +Right keys [1]: [c_customer_sk#30] Join condition: None (58) BroadcastExchange -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] +Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] (59) BroadcastHashJoin [codegen id : 16] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#43] Join condition: None (60) Project [codegen id : 16] -Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#40, c_last_name#41] -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#29, c_first_name#40, c_last_name#41] +Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#44, c_last_name#45] +Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#43, c_first_name#44, c_last_name#45] (61) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_year#12, d_moy#45] +Output [3]: [d_date_sk#48, d_year#49, d_moy#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 15] -Input [3]: [d_date_sk#10, d_year#12, d_moy#45] +Input [3]: [d_date_sk#48, d_year#49, d_moy#50] (63) Filter [codegen id : 15] -Input [3]: [d_date_sk#10, d_year#12, d_moy#45] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#45)) AND (d_year#12 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#10)) +Input [3]: [d_date_sk#48, d_year#49, d_moy#50] +Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2000)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#48)) (64) Project [codegen id : 15] -Output [1]: [d_date_sk#10] -Input [3]: [d_date_sk#10, d_year#12, d_moy#45] +Output [1]: [d_date_sk#48] +Input [3]: [d_date_sk#48, d_year#49, d_moy#50] (65) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Input [1]: [d_date_sk#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] (66) BroadcastHashJoin [codegen id : 16] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#10] +Right keys [1]: [d_date_sk#48] Join condition: None (67) Project [codegen id : 16] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#40, c_last_name#41] -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#40, c_last_name#41, d_date_sk#10] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#44, c_last_name#45] +Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#44, c_last_name#45, d_date_sk#48] (68) HashAggregate [codegen id : 16] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#40, c_last_name#41] -Keys [2]: [c_last_name#41, c_first_name#40] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#44, c_last_name#45] +Keys [2]: [c_last_name#45, c_first_name#44] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#47, isEmpty#48] -Results [4]: [c_last_name#41, c_first_name#40, sum#49, isEmpty#50] +Aggregate Attributes [2]: [sum#52, isEmpty#53] +Results [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] (69) Exchange -Input [4]: [c_last_name#41, c_first_name#40, sum#49, isEmpty#50] -Arguments: hashpartitioning(c_last_name#41, c_first_name#40, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] +Arguments: hashpartitioning(c_last_name#45, c_first_name#44, 5), ENSURE_REQUIREMENTS, [id=#56] (70) HashAggregate [codegen id : 17] -Input [4]: [c_last_name#41, c_first_name#40, sum#49, isEmpty#50] -Keys [2]: [c_last_name#41, c_first_name#40] +Input [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] +Keys [2]: [c_last_name#45, c_first_name#44] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#52] -Results [3]: [c_last_name#41, c_first_name#40, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#52 AS sales#53] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#57] +Results [3]: [c_last_name#45, c_first_name#44, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#57 AS sales#58] (71) Scan parquet default.web_sales -Output [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Output [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 22] -Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] (73) Filter [codegen id : 22] -Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] -Condition : isnotnull(ws_bill_customer_sk#55) +Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_bill_customer_sk#60) (74) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] (75) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_item_sk#54] +Left keys [1]: [ws_item_sk#59] Right keys [1]: [item_sk#22] Join condition: None (76) Project [codegen id : 22] -Output [4]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] -Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Output [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] (77) Exchange -Input [4]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_bill_customer_sk#55, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Arguments: hashpartitioning(ws_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#64] (78) Sort [codegen id : 23] -Input [4]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] -Arguments: [ws_bill_customer_sk#55 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Arguments: [ws_bill_customer_sk#60 ASC NULLS FIRST], false, 0 (79) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#29, sum#60, isEmpty#61] +Output [3]: [c_customer_sk#65, sum#66, isEmpty#67] (80) HashAggregate [codegen id : 26] -Input [3]: [c_customer_sk#29, sum#60, isEmpty#61] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] +Input [3]: [c_customer_sk#65, sum#66, isEmpty#67] +Keys [1]: [c_customer_sk#65] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70] +Results [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] (81) Filter [codegen id : 26] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (82) Project [codegen id : 26] -Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] +Output [1]: [c_customer_sk#65] +Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] (83) Sort [codegen id : 26] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#65] +Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 (84) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#55] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ws_bill_customer_sk#60] +Right keys [1]: [c_customer_sk#65] Join condition: None (85) ReusedExchange [Reuses operator id: 50] -Output [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Output [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] (86) Sort [codegen id : 28] -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] +Arguments: [c_customer_sk#72 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#29, sum#60, isEmpty#61] +Output [3]: [c_customer_sk#65, sum#66, isEmpty#67] (88) HashAggregate [codegen id : 31] -Input [3]: [c_customer_sk#29, sum#60, isEmpty#61] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] +Input [3]: [c_customer_sk#65, sum#66, isEmpty#67] +Keys [1]: [c_customer_sk#65] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70] +Results [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] (89) Filter [codegen id : 31] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (90) Project [codegen id : 31] -Output [1]: [c_customer_sk#29 AS c_customer_sk#29#64] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] +Output [1]: [c_customer_sk#65] +Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] (91) Sort [codegen id : 31] -Input [1]: [c_customer_sk#29#64] -Arguments: [c_customer_sk#29#64 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#65] +Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 (92) SortMergeJoin -Left keys [1]: [c_customer_sk#29] -Right keys [1]: [c_customer_sk#29#64] +Left keys [1]: [c_customer_sk#72] +Right keys [1]: [c_customer_sk#65] Join condition: None (93) BroadcastExchange -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] +Input [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] (94) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ws_bill_customer_sk#55] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ws_bill_customer_sk#60] +Right keys [1]: [c_customer_sk#72] Join condition: None (95) Project [codegen id : 33] -Output [5]: [ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58, c_first_name#40, c_last_name#41] -Input [7]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58, c_customer_sk#29, c_first_name#40, c_last_name#41] +Output [5]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74] +Input [7]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_customer_sk#72, c_first_name#73, c_last_name#74] (96) ReusedExchange [Reuses operator id: 65] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#76] (97) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ws_sold_date_sk#58] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#63] +Right keys [1]: [d_date_sk#76] Join condition: None (98) Project [codegen id : 33] -Output [4]: [ws_quantity#56, ws_list_price#57, c_first_name#40, c_last_name#41] -Input [6]: [ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58, c_first_name#40, c_last_name#41, d_date_sk#10] +Output [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74] +Input [6]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74, d_date_sk#76] (99) HashAggregate [codegen id : 33] -Input [4]: [ws_quantity#56, ws_list_price#57, c_first_name#40, c_last_name#41] -Keys [2]: [c_last_name#41, c_first_name#40] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#66, isEmpty#67] -Results [4]: [c_last_name#41, c_first_name#40, sum#68, isEmpty#69] +Input [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74] +Keys [2]: [c_last_name#74, c_first_name#73] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#77, isEmpty#78] +Results [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] (100) Exchange -Input [4]: [c_last_name#41, c_first_name#40, sum#68, isEmpty#69] -Arguments: hashpartitioning(c_last_name#41, c_first_name#40, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] +Arguments: hashpartitioning(c_last_name#74, c_first_name#73, 5), ENSURE_REQUIREMENTS, [id=#81] (101) HashAggregate [codegen id : 34] -Input [4]: [c_last_name#41, c_first_name#40, sum#68, isEmpty#69] -Keys [2]: [c_last_name#41, c_first_name#40] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))#71] -Results [3]: [c_last_name#41, c_first_name#40, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sales#72] +Input [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] +Keys [2]: [c_last_name#74, c_first_name#73] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#82] +Results [3]: [c_last_name#74, c_first_name#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sales#83] (102) Union (103) TakeOrderedAndProject -Input [3]: [c_last_name#41, c_first_name#40, sales#53] -Arguments: 100, [c_last_name#41 ASC NULLS FIRST, c_first_name#40 ASC NULLS FIRST, sales#53 ASC NULLS FIRST], [c_last_name#41, c_first_name#40, sales#53] +Input [3]: [c_last_name#45, c_first_name#44, sales#58] +Arguments: 100, [c_last_name#45 ASC NULLS FIRST, c_first_name#44 ASC NULLS FIRST, sales#58 ASC NULLS FIRST], [c_last_name#45, c_first_name#44, sales#58] ===== Subqueries ===== @@ -582,7 +582,7 @@ ReusedExchange (104) (104) ReusedExchange [Reuses operator id: 65] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#48] Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 ReusedExchange (105) @@ -591,7 +591,7 @@ ReusedExchange (105) (105) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#10, d_date#11] -Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (127) +- Exchange (126) +- * HashAggregate (125) @@ -617,127 +617,127 @@ Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquer (106) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Output [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (107) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] (108) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] -Condition : isnotnull(ss_customer_sk#26) +Input [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] +Condition : isnotnull(ss_customer_sk#84) (109) Scan parquet default.customer -Output [1]: [c_customer_sk#29] +Output [1]: [c_customer_sk#89] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (110) ColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#29] +Input [1]: [c_customer_sk#89] (111) Filter [codegen id : 1] -Input [1]: [c_customer_sk#29] -Condition : isnotnull(c_customer_sk#29) +Input [1]: [c_customer_sk#89] +Condition : isnotnull(c_customer_sk#89) (112) BroadcastExchange -Input [1]: [c_customer_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [1]: [c_customer_sk#89] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#90] (113) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ss_customer_sk#84] +Right keys [1]: [c_customer_sk#89] Join condition: None (114) Project [codegen id : 3] -Output [4]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] -Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] +Output [4]: [ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89] +Input [5]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89] (115) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#12] +Output [2]: [d_date_sk#91, d_year#92] 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 (116) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#10, d_year#12] +Input [2]: [d_date_sk#91, d_year#92] (117) Filter [codegen id : 2] -Input [2]: [d_date_sk#10, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +Input [2]: [d_date_sk#91, d_year#92] +Condition : (d_year#92 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#91)) (118) Project [codegen id : 2] -Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_year#12] +Output [1]: [d_date_sk#91] +Input [2]: [d_date_sk#91, d_year#92] (119) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#75] +Input [1]: [d_date_sk#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] (120) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#87] +Right keys [1]: [d_date_sk#91] Join condition: None (121) Project [codegen id : 3] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Input [5]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29, d_date_sk#10] +Output [3]: [ss_quantity#85, ss_sales_price#86, c_customer_sk#89] +Input [5]: [ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89, d_date_sk#91] (122) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#76, isEmpty#77] -Results [3]: [c_customer_sk#29, sum#78, isEmpty#79] +Input [3]: [ss_quantity#85, ss_sales_price#86, c_customer_sk#89] +Keys [1]: [c_customer_sk#89] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#94, isEmpty#95] +Results [3]: [c_customer_sk#89, sum#96, isEmpty#97] (123) Exchange -Input [3]: [c_customer_sk#29, sum#78, isEmpty#79] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [3]: [c_customer_sk#89, sum#96, isEmpty#97] +Arguments: hashpartitioning(c_customer_sk#89, 5), ENSURE_REQUIREMENTS, [id=#98] (124) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#29, sum#78, isEmpty#79] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#81] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#81 AS csales#82] +Input [3]: [c_customer_sk#89, sum#96, isEmpty#97] +Keys [1]: [c_customer_sk#89] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))#99] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))#99 AS csales#100] (125) HashAggregate [codegen id : 4] -Input [1]: [csales#82] +Input [1]: [csales#100] Keys: [] -Functions [1]: [partial_max(csales#82)] -Aggregate Attributes [1]: [max#83] -Results [1]: [max#84] +Functions [1]: [partial_max(csales#100)] +Aggregate Attributes [1]: [max#101] +Results [1]: [max#102] (126) Exchange -Input [1]: [max#84] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#85] +Input [1]: [max#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103] (127) HashAggregate [codegen id : 5] -Input [1]: [max#84] +Input [1]: [max#102] Keys: [] -Functions [1]: [max(csales#82)] -Aggregate Attributes [1]: [max(csales#82)#86] -Results [1]: [max(csales#82)#86 AS tpcds_cmax#87] +Functions [1]: [max(csales#100)] +Aggregate Attributes [1]: [max(csales#100)#104] +Results [1]: [max(csales#100)#104 AS tpcds_cmax#105] -Subquery:4 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#73 +Subquery:4 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#88 ReusedExchange (128) (128) ReusedExchange [Reuses operator id: 119] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#91] -Subquery:5 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:5 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] -Subquery:6 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:7 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] -Subquery:8 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:8 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] 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 e9a0ada3801b4..7895ee1838ad7 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 @@ -73,7 +73,7 @@ Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s 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_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -81,7 +81,7 @@ Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_ (7) Filter [codegen id : 1] 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_color#10) AND (i_color#10 = pale)) AND isnotnull(i_item_sk#7)) +Condition : ((isnotnull(i_color#10) AND (i_color#10 = pale )) AND isnotnull(i_item_sk#7)) (8) BroadcastExchange Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] @@ -340,248 +340,248 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (51) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] (53) Filter [codegen id : 2] -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)) +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) (54) Project [codegen id : 2] -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] +Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] (55) Scan parquet default.store -Output [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (56) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] (57) Filter [codegen id : 1] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -Condition : (((isnotnull(s_market_id#22) AND (s_market_id#22 = 8)) AND isnotnull(s_store_sk#20)) AND isnotnull(s_zip#24)) +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) (58) Project [codegen id : 1] -Output [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] (59) BroadcastExchange -Input [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] +Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] (60) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#20] +Left keys [1]: [ss_store_sk#52] +Right keys [1]: [s_store_sk#56] Join condition: None (61) Project [codegen id : 2] -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -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#20, s_store_name#21, s_state#23, s_zip#24] +Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] (62) Exchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#62] (63) Sort [codegen id : 3] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 (64) Scan parquet default.customer -Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Output [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (65) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] (66) Filter [codegen id : 4] -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_birth_country#18)) +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_birth_country#66)) (67) Exchange -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, [id=#67] (68) Sort [codegen id : 5] -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 +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: [c_customer_sk#63 ASC NULLS FIRST], false, 0 (69) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#15] +Left keys [1]: [ss_customer_sk#51] +Right keys [1]: [c_customer_sk#63] Join condition: None (70) Project [codegen id : 6] -Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Output [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Input [11]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] (71) Exchange -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#68] (72) Sort [codegen id : 7] -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 (73) Scan parquet default.item -Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (75) Filter [codegen id : 8] -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) +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Condition : isnotnull(i_item_sk#69) (76) Exchange -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=#54] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: hashpartitioning(i_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] (77) Sort [codegen id : 9] -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 +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: [i_item_sk#69 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ss_item_sk#50] +Right keys [1]: [i_item_sk#69] Join condition: None (79) Project [codegen id : 10] -Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (80) Exchange -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Arguments: hashpartitioning(c_birth_country#18, s_zip#24, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: hashpartitioning(c_birth_country#66, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] (81) Sort [codegen id : 11] -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Arguments: [c_birth_country#18 ASC NULLS FIRST, s_zip#24 ASC NULLS FIRST], false, 0 +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: [c_birth_country#66 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 (82) Scan parquet default.customer_address -Output [3]: [ca_state#26, ca_zip#27, ca_country#28] +Output [3]: [ca_state#77, ca_zip#78, ca_country#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (83) ColumnarToRow [codegen id : 12] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] (84) Filter [codegen id : 12] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Condition : (isnotnull(ca_country#28) AND isnotnull(ca_zip#27)) +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) (85) Exchange -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Arguments: hashpartitioning(upper(ca_country#28), ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: hashpartitioning(upper(ca_country#79), ca_zip#78, 5), ENSURE_REQUIREMENTS, [id=#80] (86) Sort [codegen id : 13] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Arguments: [upper(ca_country#28) ASC NULLS FIRST, ca_zip#27 ASC NULLS FIRST], false, 0 +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: [upper(ca_country#79) ASC NULLS FIRST, ca_zip#78 ASC NULLS FIRST], false, 0 (87) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#18, s_zip#24] -Right keys [2]: [upper(ca_country#28), ca_zip#27] +Left keys [2]: [c_birth_country#66, s_zip#60] +Right keys [2]: [upper(ca_country#79), ca_zip#78] Join condition: None (88) Project [codegen id : 14] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, ca_zip#27, ca_country#28] +Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, ca_zip#78, ca_country#79] (89) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#57] +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Arguments: hashpartitioning(cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint), 5), ENSURE_REQUIREMENTS, [id=#81] (90) Sort [codegen id : 15] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Arguments: [cast(ss_ticket_number#53 as bigint) ASC NULLS FIRST, cast(ss_item_sk#50 as bigint) ASC NULLS FIRST], false, 0 (91) Scan parquet default.store_returns -Output [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Output [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 16] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] (93) Filter [codegen id : 16] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] -Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Condition : (isnotnull(sr_ticket_number#83) AND isnotnull(sr_item_sk#82)) (94) Project [codegen id : 16] -Output [2]: [sr_item_sk#31, sr_ticket_number#32] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Output [2]: [sr_item_sk#82, sr_ticket_number#83] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] (95) Exchange -Input [2]: [sr_item_sk#31, sr_ticket_number#32] -Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: hashpartitioning(sr_ticket_number#83, sr_item_sk#82, 5), ENSURE_REQUIREMENTS, [id=#85] (96) Sort [codegen id : 17] -Input [2]: [sr_item_sk#31, sr_ticket_number#32] -Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 (97) SortMergeJoin [codegen id : 18] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] +Left keys [2]: [cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint)] +Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] Join condition: None (98) Project [codegen id : 18] -Output [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, sr_item_sk#31, sr_ticket_number#32] +Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, sr_item_sk#82, sr_ticket_number#83] (99) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] -Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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#59] -Results [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] +Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] +Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum#86] +Results [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] (100) Exchange -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Arguments: hashpartitioning(c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#88] (101) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] -Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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))#62] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#62,17,2) AS netpaid#39] +Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] (102) HashAggregate [codegen id : 19] Input [1]: [netpaid#39] Keys: [] Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#63, count#64] -Results [2]: [sum#65, count#66] +Aggregate Attributes [2]: [sum#90, count#91] +Results [2]: [sum#92, count#93] (103) Exchange -Input [2]: [sum#65, count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] +Input [2]: [sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] (104) HashAggregate [codegen id : 20] -Input [2]: [sum#65, count#66] +Input [2]: [sum#92, count#93] Keys: [] Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#68] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#68)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#69] +Aggregate Attributes [1]: [avg(netpaid#39)#95] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] 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 3b304ebe3f649..0565ae76a440d 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 @@ -144,7 +144,7 @@ Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_stor 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_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] @@ -152,7 +152,7 @@ Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, (24) Filter [codegen id : 6] 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_color#21) AND (i_color#21 = pale)) AND isnotnull(i_item_sk#18)) +Condition : ((isnotnull(i_color#21) AND (i_color#21 = pale )) AND isnotnull(i_item_sk#18)) (25) BroadcastExchange Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] @@ -316,212 +316,212 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (48) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 1] -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] +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] (50) Filter [codegen id : 1] -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)) +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)) (51) Project [codegen id : 1] -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] +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] (52) Exchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#49] +Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Arguments: hashpartitioning(cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] (53) Sort [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Arguments: [cast(ss_ticket_number#52 as bigint) ASC NULLS FIRST, cast(ss_item_sk#49 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.store_returns -Output [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] (56) Filter [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] -Condition : (isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#8)) +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Condition : (isnotnull(sr_ticket_number#57) AND isnotnull(sr_item_sk#56)) (57) Project [codegen id : 3] -Output [2]: [sr_item_sk#8, sr_ticket_number#9] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [2]: [sr_item_sk#56, sr_ticket_number#57] +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] (58) Exchange -Input [2]: [sr_item_sk#8, sr_ticket_number#9] -Arguments: hashpartitioning(sr_ticket_number#9, sr_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [sr_item_sk#56, sr_ticket_number#57] +Arguments: hashpartitioning(sr_ticket_number#57, sr_item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#59] (59) Sort [codegen id : 4] -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 +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 (60) SortMergeJoin [codegen id : 9] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] +Left keys [2]: [cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint)] +Right keys [2]: [sr_ticket_number#57, sr_item_sk#56] Join condition: None (61) Project [codegen id : 9] -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] +Output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] +Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#56, sr_ticket_number#57] (62) Scan parquet default.store -Output [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (63) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] (64) Filter [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] -Condition : (((isnotnull(s_market_id#14) AND (s_market_id#14 = 8)) AND isnotnull(s_store_sk#12)) AND isnotnull(s_zip#16)) +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Condition : (((isnotnull(s_market_id#62) AND (s_market_id#62 = 8)) AND isnotnull(s_store_sk#60)) AND isnotnull(s_zip#64)) (65) Project [codegen id : 5] -Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] (66) BroadcastExchange -Input [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] +Input [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] (67) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] +Left keys [1]: [ss_store_sk#51] +Right keys [1]: [s_store_sk#60] Join condition: None (68) Project [codegen id : 9] -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] +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64] +Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] (69) Scan parquet default.item -Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Output [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] (71) Filter [codegen id : 6] -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) +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Condition : isnotnull(i_item_sk#66) (72) BroadcastExchange -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=#52] +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#72] (73) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [ss_item_sk#49] +Right keys [1]: [i_item_sk#66] Join condition: None (74) Project [codegen id : 9] -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] +Output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] (75) Scan parquet default.customer -Output [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Output [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (76) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] (77) Filter [codegen id : 7] -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_birth_country#28)) +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_birth_country#76)) (78) BroadcastExchange -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] (79) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#25] +Left keys [1]: [ss_customer_sk#50] +Right keys [1]: [c_customer_sk#73] Join condition: None (80) Project [codegen id : 9] -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] +Output [12]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76] +Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] (81) Scan parquet default.customer_address -Output [3]: [ca_state#30, ca_zip#31, ca_country#32] +Output [3]: [ca_state#78, ca_zip#79, ca_country#80] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (82) ColumnarToRow [codegen id : 8] -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] (83) Filter [codegen id : 8] -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] -Condition : (isnotnull(ca_country#32) AND isnotnull(ca_zip#31)) +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) (84) BroadcastExchange -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#54] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#81] (85) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#28, s_zip#16] -Right keys [2]: [upper(ca_country#32), ca_zip#31] +Left keys [2]: [c_birth_country#76, s_zip#64] +Right keys [2]: [upper(ca_country#80), ca_zip#79] Join condition: None (86) Project [codegen id : 9] -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] +Output [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] +Input [15]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76, ca_state#78, ca_zip#79, ca_country#80] (87) HashAggregate [codegen id : 9] -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#55] -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#56] +Input [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] +Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum#82] +Results [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] (88) Exchange -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#56] -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=#57] +Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +Arguments: hashpartitioning(c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, [id=#84] (89) HashAggregate [codegen id : 10] -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#56] -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))#58] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#58,17,2) AS netpaid#38] +Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] +Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#85] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#38] (90) HashAggregate [codegen id : 10] Input [1]: [netpaid#38] Keys: [] Functions [1]: [partial_avg(netpaid#38)] -Aggregate Attributes [2]: [sum#59, count#60] -Results [2]: [sum#61, count#62] +Aggregate Attributes [2]: [sum#86, count#87] +Results [2]: [sum#88, count#89] (91) Exchange -Input [2]: [sum#61, count#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] +Input [2]: [sum#88, count#89] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#90] (92) HashAggregate [codegen id : 11] -Input [2]: [sum#61, count#62] +Input [2]: [sum#88, count#89] Keys: [] Functions [1]: [avg(netpaid#38)] -Aggregate Attributes [1]: [avg(netpaid#38)#64] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#64)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#65] +Aggregate Attributes [1]: [avg(netpaid#38)#91] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] 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 695f6efe560e4..e45459d529104 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 @@ -73,7 +73,7 @@ Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s 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_color), EqualTo(i_color,chiffon), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -81,7 +81,7 @@ Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_ (7) Filter [codegen id : 1] 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_color#10) AND (i_color#10 = chiffon)) AND isnotnull(i_item_sk#7)) +Condition : ((isnotnull(i_color#10) AND (i_color#10 = chiffon )) AND isnotnull(i_item_sk#7)) (8) BroadcastExchange Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] @@ -340,248 +340,248 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (51) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] (53) Filter [codegen id : 2] -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)) +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) (54) Project [codegen id : 2] -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] +Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] (55) Scan parquet default.store -Output [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (56) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] (57) Filter [codegen id : 1] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -Condition : (((isnotnull(s_market_id#22) AND (s_market_id#22 = 8)) AND isnotnull(s_store_sk#20)) AND isnotnull(s_zip#24)) +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) (58) Project [codegen id : 1] -Output [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] (59) BroadcastExchange -Input [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] +Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] (60) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#20] +Left keys [1]: [ss_store_sk#52] +Right keys [1]: [s_store_sk#56] Join condition: None (61) Project [codegen id : 2] -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -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#20, s_store_name#21, s_state#23, s_zip#24] +Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] (62) Exchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#62] (63) Sort [codegen id : 3] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 (64) Scan parquet default.customer -Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Output [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (65) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] (66) Filter [codegen id : 4] -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_birth_country#18)) +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_birth_country#66)) (67) Exchange -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, [id=#67] (68) Sort [codegen id : 5] -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 +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: [c_customer_sk#63 ASC NULLS FIRST], false, 0 (69) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#15] +Left keys [1]: [ss_customer_sk#51] +Right keys [1]: [c_customer_sk#63] Join condition: None (70) Project [codegen id : 6] -Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Output [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Input [11]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] (71) Exchange -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#68] (72) Sort [codegen id : 7] -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 (73) Scan parquet default.item -Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (75) Filter [codegen id : 8] -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) +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Condition : isnotnull(i_item_sk#69) (76) Exchange -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=#54] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: hashpartitioning(i_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] (77) Sort [codegen id : 9] -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 +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: [i_item_sk#69 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ss_item_sk#50] +Right keys [1]: [i_item_sk#69] Join condition: None (79) Project [codegen id : 10] -Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (80) Exchange -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Arguments: hashpartitioning(c_birth_country#18, s_zip#24, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: hashpartitioning(c_birth_country#66, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] (81) Sort [codegen id : 11] -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Arguments: [c_birth_country#18 ASC NULLS FIRST, s_zip#24 ASC NULLS FIRST], false, 0 +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: [c_birth_country#66 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 (82) Scan parquet default.customer_address -Output [3]: [ca_state#26, ca_zip#27, ca_country#28] +Output [3]: [ca_state#77, ca_zip#78, ca_country#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (83) ColumnarToRow [codegen id : 12] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] (84) Filter [codegen id : 12] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Condition : (isnotnull(ca_country#28) AND isnotnull(ca_zip#27)) +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) (85) Exchange -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Arguments: hashpartitioning(upper(ca_country#28), ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: hashpartitioning(upper(ca_country#79), ca_zip#78, 5), ENSURE_REQUIREMENTS, [id=#80] (86) Sort [codegen id : 13] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Arguments: [upper(ca_country#28) ASC NULLS FIRST, ca_zip#27 ASC NULLS FIRST], false, 0 +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: [upper(ca_country#79) ASC NULLS FIRST, ca_zip#78 ASC NULLS FIRST], false, 0 (87) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#18, s_zip#24] -Right keys [2]: [upper(ca_country#28), ca_zip#27] +Left keys [2]: [c_birth_country#66, s_zip#60] +Right keys [2]: [upper(ca_country#79), ca_zip#78] Join condition: None (88) Project [codegen id : 14] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, ca_zip#27, ca_country#28] +Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, ca_zip#78, ca_country#79] (89) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#57] +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Arguments: hashpartitioning(cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint), 5), ENSURE_REQUIREMENTS, [id=#81] (90) Sort [codegen id : 15] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Arguments: [cast(ss_ticket_number#53 as bigint) ASC NULLS FIRST, cast(ss_item_sk#50 as bigint) ASC NULLS FIRST], false, 0 (91) Scan parquet default.store_returns -Output [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Output [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 16] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] (93) Filter [codegen id : 16] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] -Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Condition : (isnotnull(sr_ticket_number#83) AND isnotnull(sr_item_sk#82)) (94) Project [codegen id : 16] -Output [2]: [sr_item_sk#31, sr_ticket_number#32] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Output [2]: [sr_item_sk#82, sr_ticket_number#83] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] (95) Exchange -Input [2]: [sr_item_sk#31, sr_ticket_number#32] -Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: hashpartitioning(sr_ticket_number#83, sr_item_sk#82, 5), ENSURE_REQUIREMENTS, [id=#85] (96) Sort [codegen id : 17] -Input [2]: [sr_item_sk#31, sr_ticket_number#32] -Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 (97) SortMergeJoin [codegen id : 18] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] +Left keys [2]: [cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint)] +Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] Join condition: None (98) Project [codegen id : 18] -Output [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, sr_item_sk#31, sr_ticket_number#32] +Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, sr_item_sk#82, sr_ticket_number#83] (99) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] -Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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#59] -Results [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] +Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] +Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum#86] +Results [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] (100) Exchange -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Arguments: hashpartitioning(c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#88] (101) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] -Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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))#62] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#62,17,2) AS netpaid#39] +Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] (102) HashAggregate [codegen id : 19] Input [1]: [netpaid#39] Keys: [] Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#63, count#64] -Results [2]: [sum#65, count#66] +Aggregate Attributes [2]: [sum#90, count#91] +Results [2]: [sum#92, count#93] (103) Exchange -Input [2]: [sum#65, count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] +Input [2]: [sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] (104) HashAggregate [codegen id : 20] -Input [2]: [sum#65, count#66] +Input [2]: [sum#92, count#93] Keys: [] Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#68] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#68)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#69] +Aggregate Attributes [1]: [avg(netpaid#39)#95] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] 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 49f76525f88d8..aeaf3f56b9e13 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 @@ -144,7 +144,7 @@ Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_stor 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_color), EqualTo(i_color,chiffon), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] @@ -152,7 +152,7 @@ Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, (24) Filter [codegen id : 6] 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_color#21) AND (i_color#21 = chiffon)) AND isnotnull(i_item_sk#18)) +Condition : ((isnotnull(i_color#21) AND (i_color#21 = chiffon )) AND isnotnull(i_item_sk#18)) (25) BroadcastExchange Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] @@ -316,212 +316,212 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (48) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 1] -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] +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] (50) Filter [codegen id : 1] -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)) +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)) (51) Project [codegen id : 1] -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] +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] (52) Exchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#49] +Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Arguments: hashpartitioning(cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] (53) Sort [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Arguments: [cast(ss_ticket_number#52 as bigint) ASC NULLS FIRST, cast(ss_item_sk#49 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.store_returns -Output [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] (56) Filter [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] -Condition : (isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#8)) +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Condition : (isnotnull(sr_ticket_number#57) AND isnotnull(sr_item_sk#56)) (57) Project [codegen id : 3] -Output [2]: [sr_item_sk#8, sr_ticket_number#9] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [2]: [sr_item_sk#56, sr_ticket_number#57] +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] (58) Exchange -Input [2]: [sr_item_sk#8, sr_ticket_number#9] -Arguments: hashpartitioning(sr_ticket_number#9, sr_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [sr_item_sk#56, sr_ticket_number#57] +Arguments: hashpartitioning(sr_ticket_number#57, sr_item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#59] (59) Sort [codegen id : 4] -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 +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 (60) SortMergeJoin [codegen id : 9] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] +Left keys [2]: [cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint)] +Right keys [2]: [sr_ticket_number#57, sr_item_sk#56] Join condition: None (61) Project [codegen id : 9] -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] +Output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] +Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#56, sr_ticket_number#57] (62) Scan parquet default.store -Output [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (63) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] (64) Filter [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] -Condition : (((isnotnull(s_market_id#14) AND (s_market_id#14 = 8)) AND isnotnull(s_store_sk#12)) AND isnotnull(s_zip#16)) +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Condition : (((isnotnull(s_market_id#62) AND (s_market_id#62 = 8)) AND isnotnull(s_store_sk#60)) AND isnotnull(s_zip#64)) (65) Project [codegen id : 5] -Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] (66) BroadcastExchange -Input [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] +Input [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] (67) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] +Left keys [1]: [ss_store_sk#51] +Right keys [1]: [s_store_sk#60] Join condition: None (68) Project [codegen id : 9] -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] +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64] +Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] (69) Scan parquet default.item -Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Output [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] (71) Filter [codegen id : 6] -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) +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Condition : isnotnull(i_item_sk#66) (72) BroadcastExchange -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=#52] +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#72] (73) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [ss_item_sk#49] +Right keys [1]: [i_item_sk#66] Join condition: None (74) Project [codegen id : 9] -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] +Output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] (75) Scan parquet default.customer -Output [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Output [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (76) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] (77) Filter [codegen id : 7] -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_birth_country#28)) +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_birth_country#76)) (78) BroadcastExchange -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] (79) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#25] +Left keys [1]: [ss_customer_sk#50] +Right keys [1]: [c_customer_sk#73] Join condition: None (80) Project [codegen id : 9] -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] +Output [12]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76] +Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] (81) Scan parquet default.customer_address -Output [3]: [ca_state#30, ca_zip#31, ca_country#32] +Output [3]: [ca_state#78, ca_zip#79, ca_country#80] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (82) ColumnarToRow [codegen id : 8] -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] (83) Filter [codegen id : 8] -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] -Condition : (isnotnull(ca_country#32) AND isnotnull(ca_zip#31)) +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) (84) BroadcastExchange -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#54] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#81] (85) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#28, s_zip#16] -Right keys [2]: [upper(ca_country#32), ca_zip#31] +Left keys [2]: [c_birth_country#76, s_zip#64] +Right keys [2]: [upper(ca_country#80), ca_zip#79] Join condition: None (86) Project [codegen id : 9] -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] +Output [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] +Input [15]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76, ca_state#78, ca_zip#79, ca_country#80] (87) HashAggregate [codegen id : 9] -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#55] -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#56] +Input [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] +Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum#82] +Results [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] (88) Exchange -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#56] -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=#57] +Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +Arguments: hashpartitioning(c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, [id=#84] (89) HashAggregate [codegen id : 10] -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#56] -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))#58] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#58,17,2) AS netpaid#38] +Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] +Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#85] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#38] (90) HashAggregate [codegen id : 10] Input [1]: [netpaid#38] Keys: [] Functions [1]: [partial_avg(netpaid#38)] -Aggregate Attributes [2]: [sum#59, count#60] -Results [2]: [sum#61, count#62] +Aggregate Attributes [2]: [sum#86, count#87] +Results [2]: [sum#88, count#89] (91) Exchange -Input [2]: [sum#61, count#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] +Input [2]: [sum#88, count#89] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#90] (92) HashAggregate [codegen id : 11] -Input [2]: [sum#61, count#62] +Input [2]: [sum#88, count#89] Keys: [] Functions [1]: [avg(netpaid#38)] -Aggregate Attributes [1]: [avg(netpaid#38)#64] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#64)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#65] +Aggregate Attributes [1]: [avg(netpaid#38)#91] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt index e38776ac2784b..40565dc76a7ba 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isn Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt index 0de1f44569c19..c86f121ad35a9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isn Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt index f4a07e6ac8f21..8c530f5a37885 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnu Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt index f4a07e6ac8f21..8c530f5a37885 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnu Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt index 9f15199d8c8b7..eec45ea549531 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt @@ -126,306 +126,306 @@ Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#6 / 100.0) as decimal(11,6)) AS B1_LP#15, count(ss_list_price#3)#7 AS B1_CNT#16, count(ss_list_price#3)#12 AS B1_CNTD#17] (11) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] (13) Filter [codegen id : 3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 6)) AND (ss_quantity#1 <= 10)) AND ((((ss_list_price#3 >= 90.00) AND (ss_list_price#3 <= 100.00)) OR ((ss_coupon_amt#4 >= 2323.00) AND (ss_coupon_amt#4 <= 3323.00))) OR ((ss_wholesale_cost#2 >= 31.00) AND (ss_wholesale_cost#2 <= 51.00)))) +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Condition : (((isnotnull(ss_quantity#18) AND (ss_quantity#18 >= 6)) AND (ss_quantity#18 <= 10)) AND ((((ss_list_price#20 >= 90.00) AND (ss_list_price#20 <= 100.00)) OR ((ss_coupon_amt#21 >= 2323.00) AND (ss_coupon_amt#21 <= 3323.00))) OR ((ss_wholesale_cost#19 >= 31.00) AND (ss_wholesale_cost#19 <= 51.00)))) (14) Project [codegen id : 3] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#20] +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] (15) HashAggregate [codegen id : 3] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] -Results [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [1]: [ss_list_price#20] +Keys [1]: [ss_list_price#20] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#20)), partial_count(ss_list_price#20)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] +Results [4]: [ss_list_price#20, sum#25, count#26, count#27] (16) Exchange -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] +Arguments: hashpartitioning(ss_list_price#20, 5), ENSURE_REQUIREMENTS, [id=#28] (17) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] -Results [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] +Keys [1]: [ss_list_price#20] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] +Results [4]: [ss_list_price#20, sum#25, count#26, count#27] (18) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] -Results [4]: [sum#20, count#21, count#22, count#25] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20), partial_count(distinct ss_list_price#20)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] +Results [4]: [sum#25, count#26, count#27, count#30] (19) Exchange -Input [4]: [sum#20, count#21, count#22, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] +Input [4]: [sum#25, count#26, count#27, count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] (20) HashAggregate [codegen id : 5] -Input [4]: [sum#20, count#21, count#22, count#25] +Input [4]: [sum#25, count#26, count#27, count#30] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#18 / 100.0) as decimal(11,6)) AS B2_LP#27, count(ss_list_price#3)#19 AS B2_CNT#28, count(ss_list_price#3)#24 AS B2_CNTD#29] +Functions [3]: [avg(UnscaledValue(ss_list_price#20)), count(ss_list_price#20), count(distinct ss_list_price#20)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#20))#23 / 100.0) as decimal(11,6)) AS B2_LP#32, count(ss_list_price#20)#24 AS B2_CNT#33, count(ss_list_price#20)#29 AS B2_CNTD#34] (21) BroadcastExchange -Input [3]: [B2_LP#27, B2_CNT#28, B2_CNTD#29] -Arguments: IdentityBroadcastMode, [id=#30] +Input [3]: [B2_LP#32, B2_CNT#33, B2_CNTD#34] +Arguments: IdentityBroadcastMode, [id=#35] (22) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (23) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] (25) Filter [codegen id : 6] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 11)) AND (ss_quantity#1 <= 15)) AND ((((ss_list_price#3 >= 142.00) AND (ss_list_price#3 <= 152.00)) OR ((ss_coupon_amt#4 >= 12214.00) AND (ss_coupon_amt#4 <= 13214.00))) OR ((ss_wholesale_cost#2 >= 79.00) AND (ss_wholesale_cost#2 <= 99.00)))) +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Condition : (((isnotnull(ss_quantity#36) AND (ss_quantity#36 >= 11)) AND (ss_quantity#36 <= 15)) AND ((((ss_list_price#38 >= 142.00) AND (ss_list_price#38 <= 152.00)) OR ((ss_coupon_amt#39 >= 12214.00) AND (ss_coupon_amt#39 <= 13214.00))) OR ((ss_wholesale_cost#37 >= 79.00) AND (ss_wholesale_cost#37 <= 99.00)))) (26) Project [codegen id : 6] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#38] +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] (27) HashAggregate [codegen id : 6] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] -Results [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [1]: [ss_list_price#38] +Keys [1]: [ss_list_price#38] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#38)), partial_count(ss_list_price#38)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] +Results [4]: [ss_list_price#38, sum#43, count#44, count#45] (28) Exchange -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] +Arguments: hashpartitioning(ss_list_price#38, 5), ENSURE_REQUIREMENTS, [id=#46] (29) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] -Results [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] +Keys [1]: [ss_list_price#38] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] +Results [4]: [ss_list_price#38, sum#43, count#44, count#45] (30) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] -Results [4]: [sum#33, count#34, count#35, count#38] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38), partial_count(distinct ss_list_price#38)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] +Results [4]: [sum#43, count#44, count#45, count#48] (31) Exchange -Input [4]: [sum#33, count#34, count#35, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#39] +Input [4]: [sum#43, count#44, count#45, count#48] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] (32) HashAggregate [codegen id : 8] -Input [4]: [sum#33, count#34, count#35, count#38] +Input [4]: [sum#43, count#44, count#45, count#48] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#31 / 100.0) as decimal(11,6)) AS B3_LP#40, count(ss_list_price#3)#32 AS B3_CNT#41, count(ss_list_price#3)#37 AS B3_CNTD#42] +Functions [3]: [avg(UnscaledValue(ss_list_price#38)), count(ss_list_price#38), count(distinct ss_list_price#38)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#38))#41 / 100.0) as decimal(11,6)) AS B3_LP#50, count(ss_list_price#38)#42 AS B3_CNT#51, count(ss_list_price#38)#47 AS B3_CNTD#52] (33) BroadcastExchange -Input [3]: [B3_LP#40, B3_CNT#41, B3_CNTD#42] -Arguments: IdentityBroadcastMode, [id=#43] +Input [3]: [B3_LP#50, B3_CNT#51, B3_CNTD#52] +Arguments: IdentityBroadcastMode, [id=#53] (34) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (35) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] ReadSchema: struct (36) ColumnarToRow [codegen id : 9] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] (37) Filter [codegen id : 9] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 16)) AND (ss_quantity#1 <= 20)) AND ((((ss_list_price#3 >= 135.00) AND (ss_list_price#3 <= 145.00)) OR ((ss_coupon_amt#4 >= 6071.00) AND (ss_coupon_amt#4 <= 7071.00))) OR ((ss_wholesale_cost#2 >= 38.00) AND (ss_wholesale_cost#2 <= 58.00)))) +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Condition : (((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 16)) AND (ss_quantity#54 <= 20)) AND ((((ss_list_price#56 >= 135.00) AND (ss_list_price#56 <= 145.00)) OR ((ss_coupon_amt#57 >= 6071.00) AND (ss_coupon_amt#57 <= 7071.00))) OR ((ss_wholesale_cost#55 >= 38.00) AND (ss_wholesale_cost#55 <= 58.00)))) (38) Project [codegen id : 9] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#56] +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] (39) HashAggregate [codegen id : 9] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] -Results [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [1]: [ss_list_price#56] +Keys [1]: [ss_list_price#56] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#56)), partial_count(ss_list_price#56)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] +Results [4]: [ss_list_price#56, sum#61, count#62, count#63] (40) Exchange -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] +Arguments: hashpartitioning(ss_list_price#56, 5), ENSURE_REQUIREMENTS, [id=#64] (41) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] -Results [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] +Keys [1]: [ss_list_price#56] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] +Results [4]: [ss_list_price#56, sum#61, count#62, count#63] (42) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] -Results [4]: [sum#46, count#47, count#48, count#51] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56), partial_count(distinct ss_list_price#56)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] +Results [4]: [sum#61, count#62, count#63, count#66] (43) Exchange -Input [4]: [sum#46, count#47, count#48, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] +Input [4]: [sum#61, count#62, count#63, count#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] (44) HashAggregate [codegen id : 11] -Input [4]: [sum#46, count#47, count#48, count#51] +Input [4]: [sum#61, count#62, count#63, count#66] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#44 / 100.0) as decimal(11,6)) AS B4_LP#53, count(ss_list_price#3)#45 AS B4_CNT#54, count(ss_list_price#3)#50 AS B4_CNTD#55] +Functions [3]: [avg(UnscaledValue(ss_list_price#56)), count(ss_list_price#56), count(distinct ss_list_price#56)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#56))#59 / 100.0) as decimal(11,6)) AS B4_LP#68, count(ss_list_price#56)#60 AS B4_CNT#69, count(ss_list_price#56)#65 AS B4_CNTD#70] (45) BroadcastExchange -Input [3]: [B4_LP#53, B4_CNT#54, B4_CNTD#55] -Arguments: IdentityBroadcastMode, [id=#56] +Input [3]: [B4_LP#68, B4_CNT#69, B4_CNTD#70] +Arguments: IdentityBroadcastMode, [id=#71] (46) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (47) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] ReadSchema: struct (48) ColumnarToRow [codegen id : 12] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] (49) Filter [codegen id : 12] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 21)) AND (ss_quantity#1 <= 25)) AND ((((ss_list_price#3 >= 122.00) AND (ss_list_price#3 <= 132.00)) OR ((ss_coupon_amt#4 >= 836.00) AND (ss_coupon_amt#4 <= 1836.00))) OR ((ss_wholesale_cost#2 >= 17.00) AND (ss_wholesale_cost#2 <= 37.00)))) +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Condition : (((isnotnull(ss_quantity#72) AND (ss_quantity#72 >= 21)) AND (ss_quantity#72 <= 25)) AND ((((ss_list_price#74 >= 122.00) AND (ss_list_price#74 <= 132.00)) OR ((ss_coupon_amt#75 >= 836.00) AND (ss_coupon_amt#75 <= 1836.00))) OR ((ss_wholesale_cost#73 >= 17.00) AND (ss_wholesale_cost#73 <= 37.00)))) (50) Project [codegen id : 12] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#74] +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] (51) HashAggregate [codegen id : 12] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] -Results [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [1]: [ss_list_price#74] +Keys [1]: [ss_list_price#74] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#74)), partial_count(ss_list_price#74)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] +Results [4]: [ss_list_price#74, sum#79, count#80, count#81] (52) Exchange -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] +Arguments: hashpartitioning(ss_list_price#74, 5), ENSURE_REQUIREMENTS, [id=#82] (53) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] -Results [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] +Keys [1]: [ss_list_price#74] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] +Results [4]: [ss_list_price#74, sum#79, count#80, count#81] (54) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] -Results [4]: [sum#59, count#60, count#61, count#64] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74), partial_count(distinct ss_list_price#74)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] +Results [4]: [sum#79, count#80, count#81, count#84] (55) Exchange -Input [4]: [sum#59, count#60, count#61, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] +Input [4]: [sum#79, count#80, count#81, count#84] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#85] (56) HashAggregate [codegen id : 14] -Input [4]: [sum#59, count#60, count#61, count#64] +Input [4]: [sum#79, count#80, count#81, count#84] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#57 / 100.0) as decimal(11,6)) AS B5_LP#66, count(ss_list_price#3)#58 AS B5_CNT#67, count(ss_list_price#3)#63 AS B5_CNTD#68] +Functions [3]: [avg(UnscaledValue(ss_list_price#74)), count(ss_list_price#74), count(distinct ss_list_price#74)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#74))#77 / 100.0) as decimal(11,6)) AS B5_LP#86, count(ss_list_price#74)#78 AS B5_CNT#87, count(ss_list_price#74)#83 AS B5_CNTD#88] (57) BroadcastExchange -Input [3]: [B5_LP#66, B5_CNT#67, B5_CNTD#68] -Arguments: IdentityBroadcastMode, [id=#69] +Input [3]: [B5_LP#86, B5_CNT#87, B5_CNTD#88] +Arguments: IdentityBroadcastMode, [id=#89] (58) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (59) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] ReadSchema: struct (60) ColumnarToRow [codegen id : 15] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] (61) Filter [codegen id : 15] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 26)) AND (ss_quantity#1 <= 30)) AND ((((ss_list_price#3 >= 154.00) AND (ss_list_price#3 <= 164.00)) OR ((ss_coupon_amt#4 >= 7326.00) AND (ss_coupon_amt#4 <= 8326.00))) OR ((ss_wholesale_cost#2 >= 7.00) AND (ss_wholesale_cost#2 <= 27.00)))) +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Condition : (((isnotnull(ss_quantity#90) AND (ss_quantity#90 >= 26)) AND (ss_quantity#90 <= 30)) AND ((((ss_list_price#92 >= 154.00) AND (ss_list_price#92 <= 164.00)) OR ((ss_coupon_amt#93 >= 7326.00) AND (ss_coupon_amt#93 <= 8326.00))) OR ((ss_wholesale_cost#91 >= 7.00) AND (ss_wholesale_cost#91 <= 27.00)))) (62) Project [codegen id : 15] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#92] +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] (63) HashAggregate [codegen id : 15] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] -Results [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [1]: [ss_list_price#92] +Keys [1]: [ss_list_price#92] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#92)), partial_count(ss_list_price#92)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] +Results [4]: [ss_list_price#92, sum#97, count#98, count#99] (64) Exchange -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] +Arguments: hashpartitioning(ss_list_price#92, 5), ENSURE_REQUIREMENTS, [id=#100] (65) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] -Results [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] +Keys [1]: [ss_list_price#92] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] +Results [4]: [ss_list_price#92, sum#97, count#98, count#99] (66) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] -Results [4]: [sum#72, count#73, count#74, count#77] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92), partial_count(distinct ss_list_price#92)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] +Results [4]: [sum#97, count#98, count#99, count#102] (67) Exchange -Input [4]: [sum#72, count#73, count#74, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] +Input [4]: [sum#97, count#98, count#99, count#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103] (68) HashAggregate [codegen id : 17] -Input [4]: [sum#72, count#73, count#74, count#77] +Input [4]: [sum#97, count#98, count#99, count#102] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#70 / 100.0) as decimal(11,6)) AS B6_LP#79, count(ss_list_price#3)#71 AS B6_CNT#80, count(ss_list_price#3)#76 AS B6_CNTD#81] +Functions [3]: [avg(UnscaledValue(ss_list_price#92)), count(ss_list_price#92), count(distinct ss_list_price#92)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#92))#95 / 100.0) as decimal(11,6)) AS B6_LP#104, count(ss_list_price#92)#96 AS B6_CNT#105, count(ss_list_price#92)#101 AS B6_CNTD#106] (69) BroadcastExchange -Input [3]: [B6_LP#79, B6_CNT#80, B6_CNTD#81] -Arguments: IdentityBroadcastMode, [id=#82] +Input [3]: [B6_LP#104, B6_CNT#105, B6_CNTD#106] +Arguments: IdentityBroadcastMode, [id=#107] (70) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt index 9f15199d8c8b7..eec45ea549531 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt @@ -126,306 +126,306 @@ Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#6 / 100.0) as decimal(11,6)) AS B1_LP#15, count(ss_list_price#3)#7 AS B1_CNT#16, count(ss_list_price#3)#12 AS B1_CNTD#17] (11) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] (13) Filter [codegen id : 3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 6)) AND (ss_quantity#1 <= 10)) AND ((((ss_list_price#3 >= 90.00) AND (ss_list_price#3 <= 100.00)) OR ((ss_coupon_amt#4 >= 2323.00) AND (ss_coupon_amt#4 <= 3323.00))) OR ((ss_wholesale_cost#2 >= 31.00) AND (ss_wholesale_cost#2 <= 51.00)))) +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Condition : (((isnotnull(ss_quantity#18) AND (ss_quantity#18 >= 6)) AND (ss_quantity#18 <= 10)) AND ((((ss_list_price#20 >= 90.00) AND (ss_list_price#20 <= 100.00)) OR ((ss_coupon_amt#21 >= 2323.00) AND (ss_coupon_amt#21 <= 3323.00))) OR ((ss_wholesale_cost#19 >= 31.00) AND (ss_wholesale_cost#19 <= 51.00)))) (14) Project [codegen id : 3] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#20] +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] (15) HashAggregate [codegen id : 3] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] -Results [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [1]: [ss_list_price#20] +Keys [1]: [ss_list_price#20] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#20)), partial_count(ss_list_price#20)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] +Results [4]: [ss_list_price#20, sum#25, count#26, count#27] (16) Exchange -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] +Arguments: hashpartitioning(ss_list_price#20, 5), ENSURE_REQUIREMENTS, [id=#28] (17) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] -Results [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] +Keys [1]: [ss_list_price#20] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] +Results [4]: [ss_list_price#20, sum#25, count#26, count#27] (18) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] -Results [4]: [sum#20, count#21, count#22, count#25] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20), partial_count(distinct ss_list_price#20)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] +Results [4]: [sum#25, count#26, count#27, count#30] (19) Exchange -Input [4]: [sum#20, count#21, count#22, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] +Input [4]: [sum#25, count#26, count#27, count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] (20) HashAggregate [codegen id : 5] -Input [4]: [sum#20, count#21, count#22, count#25] +Input [4]: [sum#25, count#26, count#27, count#30] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#18 / 100.0) as decimal(11,6)) AS B2_LP#27, count(ss_list_price#3)#19 AS B2_CNT#28, count(ss_list_price#3)#24 AS B2_CNTD#29] +Functions [3]: [avg(UnscaledValue(ss_list_price#20)), count(ss_list_price#20), count(distinct ss_list_price#20)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#20))#23 / 100.0) as decimal(11,6)) AS B2_LP#32, count(ss_list_price#20)#24 AS B2_CNT#33, count(ss_list_price#20)#29 AS B2_CNTD#34] (21) BroadcastExchange -Input [3]: [B2_LP#27, B2_CNT#28, B2_CNTD#29] -Arguments: IdentityBroadcastMode, [id=#30] +Input [3]: [B2_LP#32, B2_CNT#33, B2_CNTD#34] +Arguments: IdentityBroadcastMode, [id=#35] (22) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (23) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] (25) Filter [codegen id : 6] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 11)) AND (ss_quantity#1 <= 15)) AND ((((ss_list_price#3 >= 142.00) AND (ss_list_price#3 <= 152.00)) OR ((ss_coupon_amt#4 >= 12214.00) AND (ss_coupon_amt#4 <= 13214.00))) OR ((ss_wholesale_cost#2 >= 79.00) AND (ss_wholesale_cost#2 <= 99.00)))) +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Condition : (((isnotnull(ss_quantity#36) AND (ss_quantity#36 >= 11)) AND (ss_quantity#36 <= 15)) AND ((((ss_list_price#38 >= 142.00) AND (ss_list_price#38 <= 152.00)) OR ((ss_coupon_amt#39 >= 12214.00) AND (ss_coupon_amt#39 <= 13214.00))) OR ((ss_wholesale_cost#37 >= 79.00) AND (ss_wholesale_cost#37 <= 99.00)))) (26) Project [codegen id : 6] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#38] +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] (27) HashAggregate [codegen id : 6] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] -Results [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [1]: [ss_list_price#38] +Keys [1]: [ss_list_price#38] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#38)), partial_count(ss_list_price#38)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] +Results [4]: [ss_list_price#38, sum#43, count#44, count#45] (28) Exchange -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] +Arguments: hashpartitioning(ss_list_price#38, 5), ENSURE_REQUIREMENTS, [id=#46] (29) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] -Results [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] +Keys [1]: [ss_list_price#38] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] +Results [4]: [ss_list_price#38, sum#43, count#44, count#45] (30) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] -Results [4]: [sum#33, count#34, count#35, count#38] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38), partial_count(distinct ss_list_price#38)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] +Results [4]: [sum#43, count#44, count#45, count#48] (31) Exchange -Input [4]: [sum#33, count#34, count#35, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#39] +Input [4]: [sum#43, count#44, count#45, count#48] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] (32) HashAggregate [codegen id : 8] -Input [4]: [sum#33, count#34, count#35, count#38] +Input [4]: [sum#43, count#44, count#45, count#48] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#31 / 100.0) as decimal(11,6)) AS B3_LP#40, count(ss_list_price#3)#32 AS B3_CNT#41, count(ss_list_price#3)#37 AS B3_CNTD#42] +Functions [3]: [avg(UnscaledValue(ss_list_price#38)), count(ss_list_price#38), count(distinct ss_list_price#38)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#38))#41 / 100.0) as decimal(11,6)) AS B3_LP#50, count(ss_list_price#38)#42 AS B3_CNT#51, count(ss_list_price#38)#47 AS B3_CNTD#52] (33) BroadcastExchange -Input [3]: [B3_LP#40, B3_CNT#41, B3_CNTD#42] -Arguments: IdentityBroadcastMode, [id=#43] +Input [3]: [B3_LP#50, B3_CNT#51, B3_CNTD#52] +Arguments: IdentityBroadcastMode, [id=#53] (34) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (35) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] ReadSchema: struct (36) ColumnarToRow [codegen id : 9] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] (37) Filter [codegen id : 9] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 16)) AND (ss_quantity#1 <= 20)) AND ((((ss_list_price#3 >= 135.00) AND (ss_list_price#3 <= 145.00)) OR ((ss_coupon_amt#4 >= 6071.00) AND (ss_coupon_amt#4 <= 7071.00))) OR ((ss_wholesale_cost#2 >= 38.00) AND (ss_wholesale_cost#2 <= 58.00)))) +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Condition : (((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 16)) AND (ss_quantity#54 <= 20)) AND ((((ss_list_price#56 >= 135.00) AND (ss_list_price#56 <= 145.00)) OR ((ss_coupon_amt#57 >= 6071.00) AND (ss_coupon_amt#57 <= 7071.00))) OR ((ss_wholesale_cost#55 >= 38.00) AND (ss_wholesale_cost#55 <= 58.00)))) (38) Project [codegen id : 9] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#56] +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] (39) HashAggregate [codegen id : 9] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] -Results [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [1]: [ss_list_price#56] +Keys [1]: [ss_list_price#56] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#56)), partial_count(ss_list_price#56)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] +Results [4]: [ss_list_price#56, sum#61, count#62, count#63] (40) Exchange -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] +Arguments: hashpartitioning(ss_list_price#56, 5), ENSURE_REQUIREMENTS, [id=#64] (41) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] -Results [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] +Keys [1]: [ss_list_price#56] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] +Results [4]: [ss_list_price#56, sum#61, count#62, count#63] (42) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] -Results [4]: [sum#46, count#47, count#48, count#51] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56), partial_count(distinct ss_list_price#56)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] +Results [4]: [sum#61, count#62, count#63, count#66] (43) Exchange -Input [4]: [sum#46, count#47, count#48, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] +Input [4]: [sum#61, count#62, count#63, count#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] (44) HashAggregate [codegen id : 11] -Input [4]: [sum#46, count#47, count#48, count#51] +Input [4]: [sum#61, count#62, count#63, count#66] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#44 / 100.0) as decimal(11,6)) AS B4_LP#53, count(ss_list_price#3)#45 AS B4_CNT#54, count(ss_list_price#3)#50 AS B4_CNTD#55] +Functions [3]: [avg(UnscaledValue(ss_list_price#56)), count(ss_list_price#56), count(distinct ss_list_price#56)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#56))#59 / 100.0) as decimal(11,6)) AS B4_LP#68, count(ss_list_price#56)#60 AS B4_CNT#69, count(ss_list_price#56)#65 AS B4_CNTD#70] (45) BroadcastExchange -Input [3]: [B4_LP#53, B4_CNT#54, B4_CNTD#55] -Arguments: IdentityBroadcastMode, [id=#56] +Input [3]: [B4_LP#68, B4_CNT#69, B4_CNTD#70] +Arguments: IdentityBroadcastMode, [id=#71] (46) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (47) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] ReadSchema: struct (48) ColumnarToRow [codegen id : 12] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] (49) Filter [codegen id : 12] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 21)) AND (ss_quantity#1 <= 25)) AND ((((ss_list_price#3 >= 122.00) AND (ss_list_price#3 <= 132.00)) OR ((ss_coupon_amt#4 >= 836.00) AND (ss_coupon_amt#4 <= 1836.00))) OR ((ss_wholesale_cost#2 >= 17.00) AND (ss_wholesale_cost#2 <= 37.00)))) +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Condition : (((isnotnull(ss_quantity#72) AND (ss_quantity#72 >= 21)) AND (ss_quantity#72 <= 25)) AND ((((ss_list_price#74 >= 122.00) AND (ss_list_price#74 <= 132.00)) OR ((ss_coupon_amt#75 >= 836.00) AND (ss_coupon_amt#75 <= 1836.00))) OR ((ss_wholesale_cost#73 >= 17.00) AND (ss_wholesale_cost#73 <= 37.00)))) (50) Project [codegen id : 12] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#74] +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] (51) HashAggregate [codegen id : 12] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] -Results [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [1]: [ss_list_price#74] +Keys [1]: [ss_list_price#74] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#74)), partial_count(ss_list_price#74)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] +Results [4]: [ss_list_price#74, sum#79, count#80, count#81] (52) Exchange -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] +Arguments: hashpartitioning(ss_list_price#74, 5), ENSURE_REQUIREMENTS, [id=#82] (53) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] -Results [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] +Keys [1]: [ss_list_price#74] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] +Results [4]: [ss_list_price#74, sum#79, count#80, count#81] (54) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] -Results [4]: [sum#59, count#60, count#61, count#64] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74), partial_count(distinct ss_list_price#74)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] +Results [4]: [sum#79, count#80, count#81, count#84] (55) Exchange -Input [4]: [sum#59, count#60, count#61, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] +Input [4]: [sum#79, count#80, count#81, count#84] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#85] (56) HashAggregate [codegen id : 14] -Input [4]: [sum#59, count#60, count#61, count#64] +Input [4]: [sum#79, count#80, count#81, count#84] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#57 / 100.0) as decimal(11,6)) AS B5_LP#66, count(ss_list_price#3)#58 AS B5_CNT#67, count(ss_list_price#3)#63 AS B5_CNTD#68] +Functions [3]: [avg(UnscaledValue(ss_list_price#74)), count(ss_list_price#74), count(distinct ss_list_price#74)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#74))#77 / 100.0) as decimal(11,6)) AS B5_LP#86, count(ss_list_price#74)#78 AS B5_CNT#87, count(ss_list_price#74)#83 AS B5_CNTD#88] (57) BroadcastExchange -Input [3]: [B5_LP#66, B5_CNT#67, B5_CNTD#68] -Arguments: IdentityBroadcastMode, [id=#69] +Input [3]: [B5_LP#86, B5_CNT#87, B5_CNTD#88] +Arguments: IdentityBroadcastMode, [id=#89] (58) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (59) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] ReadSchema: struct (60) ColumnarToRow [codegen id : 15] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] (61) Filter [codegen id : 15] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 26)) AND (ss_quantity#1 <= 30)) AND ((((ss_list_price#3 >= 154.00) AND (ss_list_price#3 <= 164.00)) OR ((ss_coupon_amt#4 >= 7326.00) AND (ss_coupon_amt#4 <= 8326.00))) OR ((ss_wholesale_cost#2 >= 7.00) AND (ss_wholesale_cost#2 <= 27.00)))) +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Condition : (((isnotnull(ss_quantity#90) AND (ss_quantity#90 >= 26)) AND (ss_quantity#90 <= 30)) AND ((((ss_list_price#92 >= 154.00) AND (ss_list_price#92 <= 164.00)) OR ((ss_coupon_amt#93 >= 7326.00) AND (ss_coupon_amt#93 <= 8326.00))) OR ((ss_wholesale_cost#91 >= 7.00) AND (ss_wholesale_cost#91 <= 27.00)))) (62) Project [codegen id : 15] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#92] +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] (63) HashAggregate [codegen id : 15] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] -Results [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [1]: [ss_list_price#92] +Keys [1]: [ss_list_price#92] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#92)), partial_count(ss_list_price#92)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] +Results [4]: [ss_list_price#92, sum#97, count#98, count#99] (64) Exchange -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] +Arguments: hashpartitioning(ss_list_price#92, 5), ENSURE_REQUIREMENTS, [id=#100] (65) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] -Results [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] +Keys [1]: [ss_list_price#92] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] +Results [4]: [ss_list_price#92, sum#97, count#98, count#99] (66) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] -Results [4]: [sum#72, count#73, count#74, count#77] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92), partial_count(distinct ss_list_price#92)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] +Results [4]: [sum#97, count#98, count#99, count#102] (67) Exchange -Input [4]: [sum#72, count#73, count#74, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] +Input [4]: [sum#97, count#98, count#99, count#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103] (68) HashAggregate [codegen id : 17] -Input [4]: [sum#72, count#73, count#74, count#77] +Input [4]: [sum#97, count#98, count#99, count#102] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#70 / 100.0) as decimal(11,6)) AS B6_LP#79, count(ss_list_price#3)#71 AS B6_CNT#80, count(ss_list_price#3)#76 AS B6_CNTD#81] +Functions [3]: [avg(UnscaledValue(ss_list_price#92)), count(ss_list_price#92), count(distinct ss_list_price#92)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#92))#95 / 100.0) as decimal(11,6)) AS B6_LP#104, count(ss_list_price#92)#96 AS B6_CNT#105, count(ss_list_price#92)#101 AS B6_CNTD#106] (69) BroadcastExchange -Input [3]: [B6_LP#79, B6_CNT#80, B6_CNTD#81] -Arguments: IdentityBroadcastMode, [id=#82] +Input [3]: [B6_LP#104, B6_CNT#105, B6_CNTD#106] +Arguments: IdentityBroadcastMode, [id=#107] (70) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None 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 ca09ec6ea1a42..9e84e385be302 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 @@ -168,170 +168,170 @@ Input [3]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19] Condition : isnotnull(ctr_total_return#19) (24) Scan parquet default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +Output [4]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#23), dynamicpruningexpression(wr_returned_date_sk#23 IN dynamicpruning#5)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 8] -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +Input [4]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23] (26) Filter [codegen id : 8] -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) +Input [4]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23] +Condition : isnotnull(wr_returning_addr_sk#21) (27) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#24] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_returned_date_sk#4] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#23] +Right keys [1]: [cast(d_date_sk#24 as bigint)] Join condition: None (29) Project [codegen id : 8] -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] +Output [3]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22] +Input [5]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23, d_date_sk#24] (30) Exchange -Input [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Arguments: hashpartitioning(wr_returning_addr_sk#2, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [3]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22] +Arguments: hashpartitioning(wr_returning_addr_sk#21, 5), ENSURE_REQUIREMENTS, [id=#25] (31) Sort [codegen id : 9] -Input [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Arguments: [wr_returning_addr_sk#2 ASC NULLS FIRST], false, 0 +Input [3]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22] +Arguments: [wr_returning_addr_sk#21 ASC NULLS FIRST], false, 0 (32) ReusedExchange [Reuses operator id: 16] -Output [2]: [ca_address_sk#10, ca_state#11] +Output [2]: [ca_address_sk#26, ca_state#27] (33) Sort [codegen id : 11] -Input [2]: [ca_address_sk#10, ca_state#11] -Arguments: [cast(ca_address_sk#10 as bigint) ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#26, ca_state#27] +Arguments: [cast(ca_address_sk#26 as bigint) ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 12] -Left keys [1]: [wr_returning_addr_sk#2] -Right keys [1]: [cast(ca_address_sk#10 as bigint)] +Left keys [1]: [wr_returning_addr_sk#21] +Right keys [1]: [cast(ca_address_sk#26 as bigint)] Join condition: None (35) Project [codegen id : 12] -Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#11] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#10, ca_state#11] +Output [3]: [wr_returning_customer_sk#20, wr_return_amt#22, ca_state#27] +Input [5]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, ca_address_sk#26, ca_state#27] (36) HashAggregate [codegen id : 12] -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#11] -Keys [2]: [wr_returning_customer_sk#1, ca_state#11] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum#21] -Results [3]: [wr_returning_customer_sk#1, ca_state#11, sum#22] +Input [3]: [wr_returning_customer_sk#20, wr_return_amt#22, ca_state#27] +Keys [2]: [wr_returning_customer_sk#20, ca_state#27] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#22))] +Aggregate Attributes [1]: [sum#28] +Results [3]: [wr_returning_customer_sk#20, ca_state#27, sum#29] (37) Exchange -Input [3]: [wr_returning_customer_sk#1, ca_state#11, sum#22] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#11, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [wr_returning_customer_sk#20, ca_state#27, sum#29] +Arguments: hashpartitioning(wr_returning_customer_sk#20, ca_state#27, 5), ENSURE_REQUIREMENTS, [id=#30] (38) HashAggregate [codegen id : 13] -Input [3]: [wr_returning_customer_sk#1, ca_state#11, sum#22] -Keys [2]: [wr_returning_customer_sk#1, ca_state#11] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#24] -Results [2]: [ca_state#11 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#24,17,2) AS ctr_total_return#19] +Input [3]: [wr_returning_customer_sk#20, ca_state#27, sum#29] +Keys [2]: [wr_returning_customer_sk#20, ca_state#27] +Functions [1]: [sum(UnscaledValue(wr_return_amt#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#22))#31] +Results [2]: [ca_state#27 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(wr_return_amt#22))#31,17,2) AS ctr_total_return#19] (39) HashAggregate [codegen id : 13] Input [2]: [ctr_state#18, ctr_total_return#19] Keys [1]: [ctr_state#18] Functions [1]: [partial_avg(ctr_total_return#19)] -Aggregate Attributes [2]: [sum#25, count#26] -Results [3]: [ctr_state#18, sum#27, count#28] +Aggregate Attributes [2]: [sum#32, count#33] +Results [3]: [ctr_state#18, sum#34, count#35] (40) Exchange -Input [3]: [ctr_state#18, sum#27, count#28] -Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ctr_state#18, sum#34, count#35] +Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#36] (41) HashAggregate [codegen id : 14] -Input [3]: [ctr_state#18, sum#27, count#28] +Input [3]: [ctr_state#18, sum#34, count#35] Keys [1]: [ctr_state#18] Functions [1]: [avg(ctr_total_return#19)] -Aggregate Attributes [1]: [avg(ctr_total_return#19)#30] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#18 AS ctr_state#18#32] +Aggregate Attributes [1]: [avg(ctr_total_return#19)#37] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#37) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#38, ctr_state#18 AS ctr_state#18#39] (42) Filter [codegen id : 14] -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) +Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#38) (43) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#33] +Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#40] (44) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ctr_state#18] -Right keys [1]: [ctr_state#18#32] -Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) +Right keys [1]: [ctr_state#18#39] +Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#38) (45) Project [codegen id : 17] Output [2]: [ctr_customer_sk#17, ctr_total_return#19] -Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] +Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] (46) Scan parquet default.customer -Output [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Output [14]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 16] -Input [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Input [14]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] (48) Filter [codegen id : 16] -Input [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] -Condition : (isnotnull(c_customer_sk#34) AND isnotnull(c_current_addr_sk#36)) +Input [14]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] +Condition : (isnotnull(c_customer_sk#41) AND isnotnull(c_current_addr_sk#43)) (49) Scan parquet default.customer_address -Output [2]: [ca_address_sk#10, ca_state#11] +Output [2]: [ca_address_sk#55, ca_state#56] 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 (50) ColumnarToRow [codegen id : 15] -Input [2]: [ca_address_sk#10, ca_state#11] +Input [2]: [ca_address_sk#55, ca_state#56] (51) Filter [codegen id : 15] -Input [2]: [ca_address_sk#10, ca_state#11] -Condition : ((isnotnull(ca_state#11) AND (ca_state#11 = GA)) AND isnotnull(ca_address_sk#10)) +Input [2]: [ca_address_sk#55, ca_state#56] +Condition : ((isnotnull(ca_state#56) AND (ca_state#56 = GA)) AND isnotnull(ca_address_sk#55)) (52) Project [codegen id : 15] -Output [1]: [ca_address_sk#10] -Input [2]: [ca_address_sk#10, ca_state#11] +Output [1]: [ca_address_sk#55] +Input [2]: [ca_address_sk#55, ca_state#56] (53) BroadcastExchange -Input [1]: [ca_address_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] +Input [1]: [ca_address_sk#55] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#57] (54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_addr_sk#36] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [c_current_addr_sk#43] +Right keys [1]: [ca_address_sk#55] Join condition: None (55) Project [codegen id : 16] -Output [13]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] -Input [15]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ca_address_sk#10] +Output [13]: [c_customer_sk#41, c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] +Input [15]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ca_address_sk#55] (56) BroadcastExchange -Input [13]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] +Input [13]: [c_customer_sk#41, c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] (57) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ctr_customer_sk#17] -Right keys [1]: [cast(c_customer_sk#34 as bigint)] +Right keys [1]: [cast(c_customer_sk#41 as bigint)] Join condition: None (58) Project [codegen id : 17] -Output [13]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#19] -Input [15]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Output [13]: [c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ctr_total_return#19] +Input [15]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#41, c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] (59) TakeOrderedAndProject -Input [13]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#19] -Arguments: 100, [c_customer_id#35 ASC NULLS FIRST, c_salutation#37 ASC NULLS FIRST, c_first_name#38 ASC NULLS FIRST, c_last_name#39 ASC NULLS FIRST, c_preferred_cust_flag#40 ASC NULLS FIRST, c_birth_day#41 ASC NULLS FIRST, c_birth_month#42 ASC NULLS FIRST, c_birth_year#43 ASC NULLS FIRST, c_birth_country#44 ASC NULLS FIRST, c_login#45 ASC NULLS FIRST, c_email_address#46 ASC NULLS FIRST, c_last_review_date#47 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#19] +Input [13]: [c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ctr_total_return#19] +Arguments: 100, [c_customer_id#42 ASC NULLS FIRST, c_salutation#44 ASC NULLS FIRST, c_first_name#45 ASC NULLS FIRST, c_last_name#46 ASC NULLS FIRST, c_preferred_cust_flag#47 ASC NULLS FIRST, c_birth_day#48 ASC NULLS FIRST, c_birth_month#49 ASC NULLS FIRST, c_birth_year#50 ASC NULLS FIRST, c_birth_country#51 ASC NULLS FIRST, c_login#52 ASC NULLS FIRST, c_email_address#53 ASC NULLS FIRST, c_last_review_date#54 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ctr_total_return#19] ===== Subqueries ===== @@ -342,6 +342,6 @@ ReusedExchange (60) (60) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 24 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 24 Hosting Expression = wr_returned_date_sk#23 IN dynamicpruning#5 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 57b85bc9db340..1aaebad9f1920 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 @@ -150,158 +150,158 @@ Input [3]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18] Condition : isnotnull(ctr_total_return#18) (21) Scan parquet default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +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#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#22), dynamicpruningexpression(wr_returned_date_sk#22 IN dynamicpruning#5)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct (22) ColumnarToRow [codegen id : 6] -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +Input [4]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, wr_returned_date_sk#22] (23) Filter [codegen id : 6] -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) +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) (24) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#23] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returned_date_sk#4] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#22] +Right keys [1]: [cast(d_date_sk#23 as bigint)] Join condition: None (26) Project [codegen id : 6] -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] +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#23] (27) ReusedExchange [Reuses operator id: 14] -Output [2]: [ca_address_sk#9, ca_state#10] +Output [2]: [ca_address_sk#24, ca_state#25] (28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returning_addr_sk#2] -Right keys [1]: [cast(ca_address_sk#9 as bigint)] +Left keys [1]: [wr_returning_addr_sk#20] +Right keys [1]: [cast(ca_address_sk#24 as bigint)] Join condition: None (29) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#9, ca_state#10] +Output [3]: [wr_returning_customer_sk#19, wr_return_amt#21, ca_state#25] +Input [5]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, ca_address_sk#24, ca_state#25] (30) HashAggregate [codegen id : 6] -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [wr_returning_customer_sk#1, ca_state#10, sum#20] +Input [3]: [wr_returning_customer_sk#19, wr_return_amt#21, ca_state#25] +Keys [2]: [wr_returning_customer_sk#19, ca_state#25] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#21))] +Aggregate Attributes [1]: [sum#26] +Results [3]: [wr_returning_customer_sk#19, ca_state#25, sum#27] (31) Exchange -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#20] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [3]: [wr_returning_customer_sk#19, ca_state#25, sum#27] +Arguments: hashpartitioning(wr_returning_customer_sk#19, ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#28] (32) HashAggregate [codegen id : 7] -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#20] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#22] -Results [2]: [ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#22,17,2) AS ctr_total_return#18] +Input [3]: [wr_returning_customer_sk#19, ca_state#25, sum#27] +Keys [2]: [wr_returning_customer_sk#19, ca_state#25] +Functions [1]: [sum(UnscaledValue(wr_return_amt#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#21))#29] +Results [2]: [ca_state#25 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(wr_return_amt#21))#29,17,2) AS ctr_total_return#18] (33) HashAggregate [codegen id : 7] Input [2]: [ctr_state#17, ctr_total_return#18] Keys [1]: [ctr_state#17] Functions [1]: [partial_avg(ctr_total_return#18)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ctr_state#17, sum#25, count#26] +Aggregate Attributes [2]: [sum#30, count#31] +Results [3]: [ctr_state#17, sum#32, count#33] (34) Exchange -Input [3]: [ctr_state#17, sum#25, count#26] -Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ctr_state#17, sum#32, count#33] +Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#34] (35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#17, sum#25, count#26] +Input [3]: [ctr_state#17, sum#32, count#33] Keys [1]: [ctr_state#17] Functions [1]: [avg(ctr_total_return#18)] -Aggregate Attributes [1]: [avg(ctr_total_return#18)#28] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#28) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#29, ctr_state#17 AS ctr_state#17#30] +Aggregate Attributes [1]: [avg(ctr_total_return#18)#35] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#35) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#36, ctr_state#17 AS ctr_state#17#37] (36) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) +Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#36) (37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#31] +Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#38] (38) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#17] -Right keys [1]: [ctr_state#17#30] -Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) +Right keys [1]: [ctr_state#17#37] +Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#36) (39) Project [codegen id : 11] Output [2]: [ctr_customer_sk#16, ctr_total_return#18] -Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] +Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] (40) Scan parquet default.customer -Output [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +Output [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +Input [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] (42) Filter [codegen id : 9] -Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34)) +Input [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] +Condition : (isnotnull(c_customer_sk#39) AND isnotnull(c_current_addr_sk#41)) (43) BroadcastExchange -Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#46] +Input [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] (44) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#16] -Right keys [1]: [cast(c_customer_sk#32 as bigint)] +Right keys [1]: [cast(c_customer_sk#39 as bigint)] Join condition: None (45) Project [codegen id : 11] -Output [14]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] -Input [16]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +Output [14]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] +Input [16]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] (46) Scan parquet default.customer_address -Output [2]: [ca_address_sk#9, ca_state#10] +Output [2]: [ca_address_sk#54, ca_state#55] 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 (47) ColumnarToRow [codegen id : 10] -Input [2]: [ca_address_sk#9, ca_state#10] +Input [2]: [ca_address_sk#54, ca_state#55] (48) Filter [codegen id : 10] -Input [2]: [ca_address_sk#9, ca_state#10] -Condition : ((isnotnull(ca_state#10) AND (ca_state#10 = GA)) AND isnotnull(ca_address_sk#9)) +Input [2]: [ca_address_sk#54, ca_state#55] +Condition : ((isnotnull(ca_state#55) AND (ca_state#55 = GA)) AND isnotnull(ca_address_sk#54)) (49) Project [codegen id : 10] -Output [1]: [ca_address_sk#9] -Input [2]: [ca_address_sk#9, ca_state#10] +Output [1]: [ca_address_sk#54] +Input [2]: [ca_address_sk#54, ca_state#55] (50) BroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] +Input [1]: [ca_address_sk#54] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] (51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#34] -Right keys [1]: [ca_address_sk#9] +Left keys [1]: [c_current_addr_sk#41] +Right keys [1]: [ca_address_sk#54] Join condition: None (52) Project [codegen id : 11] -Output [13]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#18] -Input [15]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ca_address_sk#9] +Output [13]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ctr_total_return#18] +Input [15]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ca_address_sk#54] (53) TakeOrderedAndProject -Input [13]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#18] -Arguments: 100, [c_customer_id#33 ASC NULLS FIRST, c_salutation#35 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, c_last_name#37 ASC NULLS FIRST, c_preferred_cust_flag#38 ASC NULLS FIRST, c_birth_day#39 ASC NULLS FIRST, c_birth_month#40 ASC NULLS FIRST, c_birth_year#41 ASC NULLS FIRST, c_birth_country#42 ASC NULLS FIRST, c_login#43 ASC NULLS FIRST, c_email_address#44 ASC NULLS FIRST, c_last_review_date#45 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#18] +Input [13]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ctr_total_return#18] +Arguments: 100, [c_customer_id#40 ASC NULLS FIRST, c_salutation#42 ASC NULLS FIRST, c_first_name#43 ASC NULLS FIRST, c_last_name#44 ASC NULLS FIRST, c_preferred_cust_flag#45 ASC NULLS FIRST, c_birth_day#46 ASC NULLS FIRST, c_birth_month#47 ASC NULLS FIRST, c_birth_year#48 ASC NULLS FIRST, c_birth_country#49 ASC NULLS FIRST, c_login#50 ASC NULLS FIRST, c_email_address#51 ASC NULLS FIRST, c_last_review_date#52 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ctr_total_return#18] ===== Subqueries ===== @@ -312,6 +312,6 @@ ReusedExchange (54) (54) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 21 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 21 Hosting Expression = wr_returned_date_sk#22 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 61aa5ea5e7e5c..1703da84e7914 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 @@ -220,452 +220,452 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#16] Results [2]: [ca_county#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#16,17,2) AS store_sales#17] (22) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#18)] +PartitionFilters: [isnotnull(ss_sold_date_sk#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 8] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20] (24) Filter [codegen id : 8] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) +Input [3]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20] +Condition : isnotnull(ss_addr_sk#18) (25) Scan parquet default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_qoy#21] +Output [3]: [d_date_sk#22, d_year#23, d_qoy#24] 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 (26) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] +Input [3]: [d_date_sk#22, d_year#23, d_qoy#24] (27) Filter [codegen id : 7] -Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] -Condition : ((((isnotnull(d_qoy#21) AND isnotnull(d_year#20)) AND (d_qoy#21 = 3)) AND (d_year#20 = 2000)) AND isnotnull(d_date_sk#19)) +Input [3]: [d_date_sk#22, d_year#23, d_qoy#24] +Condition : ((((isnotnull(d_qoy#24) AND isnotnull(d_year#23)) AND (d_qoy#24 = 3)) AND (d_year#23 = 2000)) AND isnotnull(d_date_sk#22)) (28) BroadcastExchange -Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +Input [3]: [d_date_sk#22, d_year#23, d_qoy#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] (29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#19] +Left keys [1]: [ss_sold_date_sk#20] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 8] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#19, d_year#20, d_qoy#21] +Output [4]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24] +Input [6]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23, d_qoy#24] (31) Exchange -Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21] -Arguments: hashpartitioning(ss_addr_sk#1, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [4]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24] +Arguments: hashpartitioning(ss_addr_sk#18, 5), ENSURE_REQUIREMENTS, [id=#26] (32) Sort [codegen id : 9] -Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21] -Arguments: [ss_addr_sk#1 ASC NULLS FIRST], false, 0 +Input [4]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24] +Arguments: [ss_addr_sk#18 ASC NULLS FIRST], false, 0 (33) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#24, ca_county#25] +Output [2]: [ca_address_sk#27, ca_county#28] (34) Sort [codegen id : 11] -Input [2]: [ca_address_sk#24, ca_county#25] -Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#27, ca_county#28] +Arguments: [ca_address_sk#27 ASC NULLS FIRST], false, 0 (35) SortMergeJoin [codegen id : 12] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [ss_addr_sk#18] +Right keys [1]: [ca_address_sk#27] Join condition: None (36) Project [codegen id : 12] -Output [4]: [ss_ext_sales_price#2, d_year#20, d_qoy#21, ca_county#25] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21, ca_address_sk#24, ca_county#25] +Output [4]: [ss_ext_sales_price#19, d_year#23, d_qoy#24, ca_county#28] +Input [6]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24, ca_address_sk#27, ca_county#28] (37) HashAggregate [codegen id : 12] -Input [4]: [ss_ext_sales_price#2, d_year#20, d_qoy#21, ca_county#25] -Keys [3]: [ca_county#25, d_qoy#21, d_year#20] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#26] -Results [4]: [ca_county#25, d_qoy#21, d_year#20, sum#27] +Input [4]: [ss_ext_sales_price#19, d_year#23, d_qoy#24, ca_county#28] +Keys [3]: [ca_county#28, d_qoy#24, d_year#23] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#19))] +Aggregate Attributes [1]: [sum#29] +Results [4]: [ca_county#28, d_qoy#24, d_year#23, sum#30] (38) Exchange -Input [4]: [ca_county#25, d_qoy#21, d_year#20, sum#27] -Arguments: hashpartitioning(ca_county#25, d_qoy#21, d_year#20, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [4]: [ca_county#28, d_qoy#24, d_year#23, sum#30] +Arguments: hashpartitioning(ca_county#28, d_qoy#24, d_year#23, 5), ENSURE_REQUIREMENTS, [id=#31] (39) HashAggregate [codegen id : 13] -Input [4]: [ca_county#25, d_qoy#21, d_year#20, sum#27] -Keys [3]: [ca_county#25, d_qoy#21, d_year#20] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#29] -Results [2]: [ca_county#25, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#29,17,2) AS store_sales#30] +Input [4]: [ca_county#28, d_qoy#24, d_year#23, sum#30] +Keys [3]: [ca_county#28, d_qoy#24, d_year#23] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#19))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#19))#32] +Results [2]: [ca_county#28, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#19))#32,17,2) AS store_sales#33] (40) BroadcastExchange -Input [2]: [ca_county#25, store_sales#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#31] +Input [2]: [ca_county#28, store_sales#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] (41) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ca_county#11] -Right keys [1]: [ca_county#25] +Right keys [1]: [ca_county#28] Join condition: None (42) Project [codegen id : 42] -Output [3]: [ca_county#11, store_sales#17, store_sales#30] -Input [4]: [ca_county#11, store_sales#17, ca_county#25, store_sales#30] +Output [3]: [ca_county#11, store_sales#17, store_sales#33] +Input [4]: [ca_county#11, store_sales#17, ca_county#28, store_sales#33] (43) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(ss_sold_date_sk#37), dynamicpruningexpression(ss_sold_date_sk#37 IN dynamicpruning#38)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 15] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37] (45) Filter [codegen id : 15] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) +Input [3]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37] +Condition : isnotnull(ss_addr_sk#35) (46) Scan parquet default.date_dim -Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Output [3]: [d_date_sk#39, d_year#40, d_qoy#41] 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 (47) ColumnarToRow [codegen id : 14] -Input [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] (48) Filter [codegen id : 14] -Input [3]: [d_date_sk#33, d_year#34, d_qoy#35] -Condition : ((((isnotnull(d_qoy#35) AND isnotnull(d_year#34)) AND (d_qoy#35 = 1)) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) +Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] +Condition : ((((isnotnull(d_qoy#41) AND isnotnull(d_year#40)) AND (d_qoy#41 = 1)) AND (d_year#40 = 2000)) AND isnotnull(d_date_sk#39)) (49) BroadcastExchange -Input [3]: [d_date_sk#33, d_year#34, d_qoy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] +Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#42] (50) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [ss_sold_date_sk#37] +Right keys [1]: [d_date_sk#39] Join condition: None (51) Project [codegen id : 15] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#33, d_year#34, d_qoy#35] +Output [4]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41] +Input [6]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37, d_date_sk#39, d_year#40, d_qoy#41] (52) Exchange -Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35] -Arguments: hashpartitioning(ss_addr_sk#1, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41] +Arguments: hashpartitioning(ss_addr_sk#35, 5), ENSURE_REQUIREMENTS, [id=#43] (53) Sort [codegen id : 16] -Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35] -Arguments: [ss_addr_sk#1 ASC NULLS FIRST], false, 0 +Input [4]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41] +Arguments: [ss_addr_sk#35 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#38, ca_county#39] +Output [2]: [ca_address_sk#44, ca_county#45] (55) Sort [codegen id : 18] -Input [2]: [ca_address_sk#38, ca_county#39] -Arguments: [ca_address_sk#38 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#44, ca_county#45] +Arguments: [ca_address_sk#44 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 19] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#38] +Left keys [1]: [ss_addr_sk#35] +Right keys [1]: [ca_address_sk#44] Join condition: None (57) Project [codegen id : 19] -Output [4]: [ss_ext_sales_price#2, d_year#34, d_qoy#35, ca_county#39] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35, ca_address_sk#38, ca_county#39] +Output [4]: [ss_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#45] +Input [6]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41, ca_address_sk#44, ca_county#45] (58) HashAggregate [codegen id : 19] -Input [4]: [ss_ext_sales_price#2, d_year#34, d_qoy#35, ca_county#39] -Keys [3]: [ca_county#39, d_qoy#35, d_year#34] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#40] -Results [4]: [ca_county#39, d_qoy#35, d_year#34, sum#41] +Input [4]: [ss_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#45] +Keys [3]: [ca_county#45, d_qoy#41, d_year#40] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#36))] +Aggregate Attributes [1]: [sum#46] +Results [4]: [ca_county#45, d_qoy#41, d_year#40, sum#47] (59) Exchange -Input [4]: [ca_county#39, d_qoy#35, d_year#34, sum#41] -Arguments: hashpartitioning(ca_county#39, d_qoy#35, d_year#34, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [4]: [ca_county#45, d_qoy#41, d_year#40, sum#47] +Arguments: hashpartitioning(ca_county#45, d_qoy#41, d_year#40, 5), ENSURE_REQUIREMENTS, [id=#48] (60) HashAggregate [codegen id : 20] -Input [4]: [ca_county#39, d_qoy#35, d_year#34, sum#41] -Keys [3]: [ca_county#39, d_qoy#35, d_year#34] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#43] -Results [3]: [ca_county#39, d_year#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#43,17,2) AS store_sales#44] +Input [4]: [ca_county#45, d_qoy#41, d_year#40, sum#47] +Keys [3]: [ca_county#45, d_qoy#41, d_year#40] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#36))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#36))#49] +Results [3]: [ca_county#45, d_year#40, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#36))#49,17,2) AS store_sales#50] (61) BroadcastExchange -Input [3]: [ca_county#39, d_year#34, store_sales#44] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#45] +Input [3]: [ca_county#45, d_year#40, store_sales#50] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#51] (62) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ca_county#11] -Right keys [1]: [ca_county#39] +Right keys [1]: [ca_county#45] Join condition: None (63) Project [codegen id : 42] -Output [5]: [store_sales#17, store_sales#30, ca_county#39, d_year#34, store_sales#44] -Input [6]: [ca_county#11, store_sales#17, store_sales#30, ca_county#39, d_year#34, store_sales#44] +Output [5]: [store_sales#17, store_sales#33, ca_county#45, d_year#40, store_sales#50] +Input [6]: [ca_county#11, store_sales#17, store_sales#33, ca_county#45, d_year#40, store_sales#50] (64) 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#52, ws_ext_sales_price#53, ws_sold_date_sk#54] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#38)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Input [3]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, ws_sold_date_sk#54] (66) Filter [codegen id : 22] -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#52, ws_ext_sales_price#53, ws_sold_date_sk#54] +Condition : isnotnull(ws_bill_addr_sk#52) (67) ReusedExchange [Reuses operator id: 49] -Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] +Output [3]: [d_date_sk#55, d_year#56, d_qoy#57] (68) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#48] -Right keys [1]: [d_date_sk#49] +Left keys [1]: [ws_sold_date_sk#54] +Right keys [1]: [d_date_sk#55] Join condition: None (69) Project [codegen id : 22] -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#52, ws_ext_sales_price#53, d_year#56, d_qoy#57] +Input [6]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, ws_sold_date_sk#54, d_date_sk#55, d_year#56, d_qoy#57] (70) Exchange -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51] -Arguments: hashpartitioning(ws_bill_addr_sk#46, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [4]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57] +Arguments: hashpartitioning(ws_bill_addr_sk#52, 5), ENSURE_REQUIREMENTS, [id=#58] (71) Sort [codegen id : 23] -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51] -Arguments: [ws_bill_addr_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57] +Arguments: [ws_bill_addr_sk#52 ASC NULLS FIRST], false, 0 (72) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#53, ca_county#54] +Output [2]: [ca_address_sk#59, ca_county#60] (73) Sort [codegen id : 25] -Input [2]: [ca_address_sk#53, ca_county#54] -Arguments: [ca_address_sk#53 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#59, ca_county#60] +Arguments: [ca_address_sk#59 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 26] -Left keys [1]: [ws_bill_addr_sk#46] -Right keys [1]: [ca_address_sk#53] +Left keys [1]: [ws_bill_addr_sk#52] +Right keys [1]: [ca_address_sk#59] Join condition: None (75) Project [codegen id : 26] -Output [4]: [ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_county#54] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_address_sk#53, ca_county#54] +Output [4]: [ws_ext_sales_price#53, d_year#56, d_qoy#57, ca_county#60] +Input [6]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57, ca_address_sk#59, ca_county#60] (76) HashAggregate [codegen id : 26] -Input [4]: [ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_county#54] -Keys [3]: [ca_county#54, d_qoy#51, d_year#50] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum#55] -Results [4]: [ca_county#54, d_qoy#51, d_year#50, sum#56] +Input [4]: [ws_ext_sales_price#53, d_year#56, d_qoy#57, ca_county#60] +Keys [3]: [ca_county#60, d_qoy#57, d_year#56] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#53))] +Aggregate Attributes [1]: [sum#61] +Results [4]: [ca_county#60, d_qoy#57, d_year#56, sum#62] (77) Exchange -Input [4]: [ca_county#54, d_qoy#51, d_year#50, sum#56] -Arguments: hashpartitioning(ca_county#54, d_qoy#51, d_year#50, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [4]: [ca_county#60, d_qoy#57, d_year#56, sum#62] +Arguments: hashpartitioning(ca_county#60, d_qoy#57, d_year#56, 5), ENSURE_REQUIREMENTS, [id=#63] (78) HashAggregate [codegen id : 41] -Input [4]: [ca_county#54, d_qoy#51, d_year#50, sum#56] -Keys [3]: [ca_county#54, 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))#58] -Results [2]: [ca_county#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#47))#58,17,2) AS web_sales#59] +Input [4]: [ca_county#60, d_qoy#57, d_year#56, sum#62] +Keys [3]: [ca_county#60, d_qoy#57, d_year#56] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#53))#64] +Results [2]: [ca_county#60, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#53))#64,17,2) AS web_sales#65] (79) 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#66, ws_ext_sales_price#67, ws_sold_date_sk#68] 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#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (80) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Input [3]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, ws_sold_date_sk#68] (81) Filter [codegen id : 28] -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#66, ws_ext_sales_price#67, ws_sold_date_sk#68] +Condition : isnotnull(ws_bill_addr_sk#66) (82) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#60, d_year#61, d_qoy#62] +Output [3]: [d_date_sk#69, d_year#70, d_qoy#71] (83) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#48] -Right keys [1]: [d_date_sk#60] +Left keys [1]: [ws_sold_date_sk#68] +Right keys [1]: [d_date_sk#69] Join condition: None (84) Project [codegen id : 28] -Output [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48, d_date_sk#60, d_year#61, d_qoy#62] +Output [4]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71] +Input [6]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, ws_sold_date_sk#68, d_date_sk#69, d_year#70, d_qoy#71] (85) Exchange -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62] -Arguments: hashpartitioning(ws_bill_addr_sk#46, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [4]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71] +Arguments: hashpartitioning(ws_bill_addr_sk#66, 5), ENSURE_REQUIREMENTS, [id=#72] (86) Sort [codegen id : 29] -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62] -Arguments: [ws_bill_addr_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71] +Arguments: [ws_bill_addr_sk#66 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#64, ca_county#65] +Output [2]: [ca_address_sk#73, ca_county#74] (88) Sort [codegen id : 31] -Input [2]: [ca_address_sk#64, ca_county#65] -Arguments: [ca_address_sk#64 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#73, ca_county#74] +Arguments: [ca_address_sk#73 ASC NULLS FIRST], false, 0 (89) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_addr_sk#46] -Right keys [1]: [ca_address_sk#64] +Left keys [1]: [ws_bill_addr_sk#66] +Right keys [1]: [ca_address_sk#73] Join condition: None (90) Project [codegen id : 32] -Output [4]: [ws_ext_sales_price#47, d_year#61, d_qoy#62, ca_county#65] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62, ca_address_sk#64, ca_county#65] +Output [4]: [ws_ext_sales_price#67, d_year#70, d_qoy#71, ca_county#74] +Input [6]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71, ca_address_sk#73, ca_county#74] (91) HashAggregate [codegen id : 32] -Input [4]: [ws_ext_sales_price#47, d_year#61, d_qoy#62, ca_county#65] -Keys [3]: [ca_county#65, d_qoy#62, d_year#61] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum#66] -Results [4]: [ca_county#65, d_qoy#62, d_year#61, sum#67] +Input [4]: [ws_ext_sales_price#67, d_year#70, d_qoy#71, ca_county#74] +Keys [3]: [ca_county#74, d_qoy#71, d_year#70] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#67))] +Aggregate Attributes [1]: [sum#75] +Results [4]: [ca_county#74, d_qoy#71, d_year#70, sum#76] (92) Exchange -Input [4]: [ca_county#65, d_qoy#62, d_year#61, sum#67] -Arguments: hashpartitioning(ca_county#65, d_qoy#62, d_year#61, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [4]: [ca_county#74, d_qoy#71, d_year#70, sum#76] +Arguments: hashpartitioning(ca_county#74, d_qoy#71, d_year#70, 5), ENSURE_REQUIREMENTS, [id=#77] (93) HashAggregate [codegen id : 33] -Input [4]: [ca_county#65, d_qoy#62, d_year#61, sum#67] -Keys [3]: [ca_county#65, d_qoy#62, d_year#61] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#47))#69] -Results [2]: [ca_county#65, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#47))#69,17,2) AS web_sales#70] +Input [4]: [ca_county#74, d_qoy#71, d_year#70, sum#76] +Keys [3]: [ca_county#74, d_qoy#71, d_year#70] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#67))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#67))#78] +Results [2]: [ca_county#74, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#67))#78,17,2) AS web_sales#79] (94) BroadcastExchange -Input [2]: [ca_county#65, web_sales#70] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#71] +Input [2]: [ca_county#74, web_sales#79] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#80] (95) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#54] -Right keys [1]: [ca_county#65] +Left keys [1]: [ca_county#60] +Right keys [1]: [ca_county#74] Join condition: None (96) Project [codegen id : 41] -Output [3]: [ca_county#54, web_sales#59, web_sales#70] -Input [4]: [ca_county#54, web_sales#59, ca_county#65, web_sales#70] +Output [3]: [ca_county#60, web_sales#65, web_sales#79] +Input [4]: [ca_county#60, web_sales#65, ca_county#74, web_sales#79] (97) 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#81, ws_ext_sales_price#82, ws_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#18)] +PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (98) ColumnarToRow [codegen id : 35] -Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Input [3]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, ws_sold_date_sk#83] (99) Filter [codegen id : 35] -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#81, ws_ext_sales_price#82, ws_sold_date_sk#83] +Condition : isnotnull(ws_bill_addr_sk#81) (100) ReusedExchange [Reuses operator id: 28] -Output [3]: [d_date_sk#72, d_year#73, d_qoy#74] +Output [3]: [d_date_sk#84, d_year#85, d_qoy#86] (101) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_sold_date_sk#48] -Right keys [1]: [d_date_sk#72] +Left keys [1]: [ws_sold_date_sk#83] +Right keys [1]: [d_date_sk#84] Join condition: None (102) Project [codegen id : 35] -Output [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48, d_date_sk#72, d_year#73, d_qoy#74] +Output [4]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86] +Input [6]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, ws_sold_date_sk#83, d_date_sk#84, d_year#85, d_qoy#86] (103) Exchange -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74] -Arguments: hashpartitioning(ws_bill_addr_sk#46, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [4]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86] +Arguments: hashpartitioning(ws_bill_addr_sk#81, 5), ENSURE_REQUIREMENTS, [id=#87] (104) Sort [codegen id : 36] -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74] -Arguments: [ws_bill_addr_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86] +Arguments: [ws_bill_addr_sk#81 ASC NULLS FIRST], false, 0 (105) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#76, ca_county#77] +Output [2]: [ca_address_sk#88, ca_county#89] (106) Sort [codegen id : 38] -Input [2]: [ca_address_sk#76, ca_county#77] -Arguments: [ca_address_sk#76 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#88, ca_county#89] +Arguments: [ca_address_sk#88 ASC NULLS FIRST], false, 0 (107) SortMergeJoin [codegen id : 39] -Left keys [1]: [ws_bill_addr_sk#46] -Right keys [1]: [ca_address_sk#76] +Left keys [1]: [ws_bill_addr_sk#81] +Right keys [1]: [ca_address_sk#88] Join condition: None (108) Project [codegen id : 39] -Output [4]: [ws_ext_sales_price#47, d_year#73, d_qoy#74, ca_county#77] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74, ca_address_sk#76, ca_county#77] +Output [4]: [ws_ext_sales_price#82, d_year#85, d_qoy#86, ca_county#89] +Input [6]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86, ca_address_sk#88, ca_county#89] (109) HashAggregate [codegen id : 39] -Input [4]: [ws_ext_sales_price#47, d_year#73, d_qoy#74, ca_county#77] -Keys [3]: [ca_county#77, d_qoy#74, d_year#73] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum#78] -Results [4]: [ca_county#77, d_qoy#74, d_year#73, sum#79] +Input [4]: [ws_ext_sales_price#82, d_year#85, d_qoy#86, ca_county#89] +Keys [3]: [ca_county#89, d_qoy#86, d_year#85] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#82))] +Aggregate Attributes [1]: [sum#90] +Results [4]: [ca_county#89, d_qoy#86, d_year#85, sum#91] (110) Exchange -Input [4]: [ca_county#77, d_qoy#74, d_year#73, sum#79] -Arguments: hashpartitioning(ca_county#77, d_qoy#74, d_year#73, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [4]: [ca_county#89, d_qoy#86, d_year#85, sum#91] +Arguments: hashpartitioning(ca_county#89, d_qoy#86, d_year#85, 5), ENSURE_REQUIREMENTS, [id=#92] (111) HashAggregate [codegen id : 40] -Input [4]: [ca_county#77, d_qoy#74, d_year#73, sum#79] -Keys [3]: [ca_county#77, d_qoy#74, d_year#73] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#47))#81] -Results [2]: [ca_county#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#47))#81,17,2) AS web_sales#82] +Input [4]: [ca_county#89, d_qoy#86, d_year#85, sum#91] +Keys [3]: [ca_county#89, d_qoy#86, d_year#85] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#82))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#82))#93] +Results [2]: [ca_county#89, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#82))#93,17,2) AS web_sales#94] (112) BroadcastExchange -Input [2]: [ca_county#77, web_sales#82] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#83] +Input [2]: [ca_county#89, web_sales#94] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#95] (113) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#54] -Right keys [1]: [ca_county#77] +Left keys [1]: [ca_county#60] +Right keys [1]: [ca_county#89] Join condition: None (114) Project [codegen id : 41] -Output [4]: [ca_county#54, web_sales#59, web_sales#70, web_sales#82] -Input [5]: [ca_county#54, web_sales#59, web_sales#70, ca_county#77, web_sales#82] +Output [4]: [ca_county#60, web_sales#65, web_sales#79, web_sales#94] +Input [5]: [ca_county#60, web_sales#65, web_sales#79, ca_county#89, web_sales#94] (115) BroadcastExchange -Input [4]: [ca_county#54, web_sales#59, web_sales#70, web_sales#82] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#84] +Input [4]: [ca_county#60, web_sales#65, web_sales#79, web_sales#94] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#96] (116) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ca_county#39] -Right keys [1]: [ca_county#54] -Join condition: ((CASE WHEN (web_sales#59 > 0.00) THEN CheckOverflow((promote_precision(web_sales#70) / promote_precision(web_sales#59)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#44 > 0.00) THEN CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#44)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#70 > 0.00) THEN CheckOverflow((promote_precision(web_sales#82) / promote_precision(web_sales#70)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#17 > 0.00) THEN CheckOverflow((promote_precision(store_sales#30) / promote_precision(store_sales#17)), DecimalType(37,20), true) ELSE null END)) +Left keys [1]: [ca_county#45] +Right keys [1]: [ca_county#60] +Join condition: ((CASE WHEN (web_sales#65 > 0.00) THEN CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#65)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#50 > 0.00) THEN CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#50)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#79 > 0.00) THEN CheckOverflow((promote_precision(web_sales#94) / promote_precision(web_sales#79)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#17 > 0.00) THEN CheckOverflow((promote_precision(store_sales#33) / promote_precision(store_sales#17)), DecimalType(37,20), true) ELSE null END)) (117) Project [codegen id : 42] -Output [6]: [ca_county#39, d_year#34, CheckOverflow((promote_precision(web_sales#70) / promote_precision(web_sales#59)), DecimalType(37,20), true) AS web_q1_q2_increase#85, CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#44)), DecimalType(37,20), true) AS store_q1_q2_increase#86, CheckOverflow((promote_precision(web_sales#82) / promote_precision(web_sales#70)), DecimalType(37,20), true) AS web_q2_q3_increase#87, CheckOverflow((promote_precision(store_sales#30) / promote_precision(store_sales#17)), DecimalType(37,20), true) AS store_q2_q3_increase#88] -Input [9]: [store_sales#17, store_sales#30, ca_county#39, d_year#34, store_sales#44, ca_county#54, web_sales#59, web_sales#70, web_sales#82] +Output [6]: [ca_county#45, d_year#40, CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#65)), DecimalType(37,20), true) AS web_q1_q2_increase#97, CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#50)), DecimalType(37,20), true) AS store_q1_q2_increase#98, CheckOverflow((promote_precision(web_sales#94) / promote_precision(web_sales#79)), DecimalType(37,20), true) AS web_q2_q3_increase#99, CheckOverflow((promote_precision(store_sales#33) / promote_precision(store_sales#17)), DecimalType(37,20), true) AS store_q2_q3_increase#100] +Input [9]: [store_sales#17, store_sales#33, ca_county#45, d_year#40, store_sales#50, ca_county#60, web_sales#65, web_sales#79, web_sales#94] (118) Exchange -Input [6]: [ca_county#39, d_year#34, web_q1_q2_increase#85, store_q1_q2_increase#86, web_q2_q3_increase#87, store_q2_q3_increase#88] -Arguments: rangepartitioning(ca_county#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [6]: [ca_county#45, d_year#40, web_q1_q2_increase#97, store_q1_q2_increase#98, web_q2_q3_increase#99, store_q2_q3_increase#100] +Arguments: rangepartitioning(ca_county#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#101] (119) Sort [codegen id : 43] -Input [6]: [ca_county#39, d_year#34, web_q1_q2_increase#85, store_q1_q2_increase#86, web_q2_q3_increase#87, store_q2_q3_increase#88] -Arguments: [ca_county#39 ASC NULLS FIRST], true, 0 +Input [6]: [ca_county#45, d_year#40, web_q1_q2_increase#97, store_q1_q2_increase#98, web_q2_q3_increase#99, store_q2_q3_increase#100] +Arguments: [ca_county#45 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -676,24 +676,24 @@ ReusedExchange (120) (120) ReusedExchange [Reuses operator id: 7] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#18 +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 ReusedExchange (121) (121) ReusedExchange [Reuses operator id: 28] -Output [3]: [d_date_sk#19, d_year#20, d_qoy#21] +Output [3]: [d_date_sk#22, d_year#23, d_qoy#24] -Subquery:3 Hosting operator id = 43 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#32 +Subquery:3 Hosting operator id = 43 Hosting Expression = ss_sold_date_sk#37 IN dynamicpruning#38 ReusedExchange (122) (122) ReusedExchange [Reuses operator id: 49] -Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Output [3]: [d_date_sk#39, d_year#40, d_qoy#41] -Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#32 +Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#38 -Subquery:5 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#4 +Subquery:5 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#4 -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#18 +Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#21 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 81a4a7fed99b2..f47a7af5a15e3 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 @@ -188,383 +188,383 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#15] Results [3]: [ca_county#10, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#15,17,2) AS store_sales#16] (19) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(ss_sold_date_sk#19), dynamicpruningexpression(ss_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (20) ColumnarToRow [codegen id : 6] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19] (21) Filter [codegen id : 6] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) +Input [3]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19] +Condition : isnotnull(ss_addr_sk#17) (22) Scan parquet default.date_dim -Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 4] -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] (24) Filter [codegen id : 4] -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) +Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] +Condition : ((((isnotnull(d_qoy#23) AND isnotnull(d_year#22)) AND (d_qoy#23 = 2)) AND (d_year#22 = 2000)) AND isnotnull(d_date_sk#21)) (25) BroadcastExchange -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] (26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#18] +Left keys [1]: [ss_sold_date_sk#19] +Right keys [1]: [d_date_sk#21] Join condition: None (27) Project [codegen id : 6] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#19, d_qoy#20] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#18, d_year#19, d_qoy#20] +Output [4]: [ss_addr_sk#17, ss_ext_sales_price#18, d_year#22, d_qoy#23] +Input [6]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19, d_date_sk#21, d_year#22, d_qoy#23] (28) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#22, ca_county#23] +Output [2]: [ca_address_sk#25, ca_county#26] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#22] +Left keys [1]: [ss_addr_sk#17] +Right keys [1]: [ca_address_sk#25] Join condition: None (30) Project [codegen id : 6] -Output [4]: [ss_ext_sales_price#2, d_year#19, d_qoy#20, ca_county#23] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#19, d_qoy#20, ca_address_sk#22, ca_county#23] +Output [4]: [ss_ext_sales_price#18, d_year#22, d_qoy#23, ca_county#26] +Input [6]: [ss_addr_sk#17, ss_ext_sales_price#18, d_year#22, d_qoy#23, ca_address_sk#25, ca_county#26] (31) HashAggregate [codegen id : 6] -Input [4]: [ss_ext_sales_price#2, d_year#19, d_qoy#20, ca_county#23] -Keys [3]: [ca_county#23, d_qoy#20, d_year#19] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#24] -Results [4]: [ca_county#23, d_qoy#20, d_year#19, sum#25] +Input [4]: [ss_ext_sales_price#18, d_year#22, d_qoy#23, ca_county#26] +Keys [3]: [ca_county#26, d_qoy#23, d_year#22] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#18))] +Aggregate Attributes [1]: [sum#27] +Results [4]: [ca_county#26, d_qoy#23, d_year#22, sum#28] (32) Exchange -Input [4]: [ca_county#23, d_qoy#20, d_year#19, sum#25] -Arguments: hashpartitioning(ca_county#23, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [4]: [ca_county#26, d_qoy#23, d_year#22, sum#28] +Arguments: hashpartitioning(ca_county#26, d_qoy#23, d_year#22, 5), ENSURE_REQUIREMENTS, [id=#29] (33) HashAggregate [codegen id : 7] -Input [4]: [ca_county#23, d_qoy#20, d_year#19, sum#25] -Keys [3]: [ca_county#23, d_qoy#20, d_year#19] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#27] -Results [2]: [ca_county#23, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#27,17,2) AS store_sales#28] +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] (34) BroadcastExchange -Input [2]: [ca_county#23, store_sales#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#29] +Input [2]: [ca_county#26, store_sales#31] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] (35) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#10] -Right keys [1]: [ca_county#23] +Right keys [1]: [ca_county#26] Join condition: None (36) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_addr_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ss_sold_date_sk#35), dynamicpruningexpression(ss_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 10] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_addr_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] (38) Filter [codegen id : 10] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) +Input [3]: [ss_addr_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] +Condition : isnotnull(ss_addr_sk#33) (39) Scan parquet default.date_dim -Output [3]: [d_date_sk#31, d_year#32, d_qoy#33] +Output [3]: [d_date_sk#37, d_year#38, d_qoy#39] 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 (40) ColumnarToRow [codegen id : 8] -Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] +Input [3]: [d_date_sk#37, d_year#38, d_qoy#39] (41) Filter [codegen id : 8] -Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] -Condition : ((((isnotnull(d_qoy#33) AND isnotnull(d_year#32)) AND (d_qoy#33 = 3)) AND (d_year#32 = 2000)) AND isnotnull(d_date_sk#31)) +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 = 3)) AND (d_year#38 = 2000)) AND isnotnull(d_date_sk#37)) (42) BroadcastExchange -Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [3]: [d_date_sk#37, d_year#38, d_qoy#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] (43) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#31] +Left keys [1]: [ss_sold_date_sk#35] +Right keys [1]: [d_date_sk#37] Join condition: None (44) Project [codegen id : 10] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#32, d_qoy#33] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#31, d_year#32, d_qoy#33] +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] (45) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#35, ca_county#36] +Output [2]: [ca_address_sk#41, ca_county#42] (46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#35] +Left keys [1]: [ss_addr_sk#33] +Right keys [1]: [ca_address_sk#41] Join condition: None (47) Project [codegen id : 10] -Output [4]: [ss_ext_sales_price#2, d_year#32, d_qoy#33, ca_county#36] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#32, d_qoy#33, ca_address_sk#35, ca_county#36] +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] (48) HashAggregate [codegen id : 10] -Input [4]: [ss_ext_sales_price#2, d_year#32, d_qoy#33, ca_county#36] -Keys [3]: [ca_county#36, d_qoy#33, d_year#32] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#37] -Results [4]: [ca_county#36, d_qoy#33, d_year#32, sum#38] +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] (49) Exchange -Input [4]: [ca_county#36, d_qoy#33, d_year#32, sum#38] -Arguments: hashpartitioning(ca_county#36, d_qoy#33, d_year#32, 5), ENSURE_REQUIREMENTS, [id=#39] +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] (50) HashAggregate [codegen id : 11] -Input [4]: [ca_county#36, d_qoy#33, d_year#32, sum#38] -Keys [3]: [ca_county#36, d_qoy#33, d_year#32] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#40] -Results [2]: [ca_county#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#40,17,2) AS store_sales#41] +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 [2]: [ca_county#42, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#46,17,2) AS store_sales#47] (51) BroadcastExchange -Input [2]: [ca_county#36, store_sales#41] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#42] +Input [2]: [ca_county#42, store_sales#47] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] (52) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#23] -Right keys [1]: [ca_county#36] +Left keys [1]: [ca_county#26] +Right keys [1]: [ca_county#42] Join condition: None (53) Project [codegen id : 24] -Output [5]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41] -Input [7]: [ca_county#10, d_year#6, store_sales#16, ca_county#23, store_sales#28, ca_county#36, store_sales#41] +Output [5]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47] +Input [7]: [ca_county#10, d_year#6, store_sales#16, ca_county#26, store_sales#31, ca_county#42, store_sales#47] (54) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Output [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 14] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] (56) Filter [codegen id : 14] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_bill_addr_sk#43) +Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_bill_addr_sk#49) (57) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#46, d_year#47, d_qoy#48] +Output [3]: [d_date_sk#52, d_year#53, d_qoy#54] (58) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#46] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#52] Join condition: None (59) Project [codegen id : 14] -Output [4]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#47, d_qoy#48] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45, d_date_sk#46, d_year#47, d_qoy#48] +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] (60) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#49, ca_county#50] +Output [2]: [ca_address_sk#55, ca_county#56] (61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#43] -Right keys [1]: [ca_address_sk#49] +Left keys [1]: [ws_bill_addr_sk#49] +Right keys [1]: [ca_address_sk#55] Join condition: None (62) Project [codegen id : 14] -Output [4]: [ws_ext_sales_price#44, d_year#47, d_qoy#48, ca_county#50] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#47, d_qoy#48, ca_address_sk#49, ca_county#50] +Output [4]: [ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_county#56] +Input [6]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_address_sk#55, ca_county#56] (63) HashAggregate [codegen id : 14] -Input [4]: [ws_ext_sales_price#44, d_year#47, d_qoy#48, ca_county#50] -Keys [3]: [ca_county#50, d_qoy#48, d_year#47] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum#51] -Results [4]: [ca_county#50, d_qoy#48, d_year#47, sum#52] +Input [4]: [ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_county#56] +Keys [3]: [ca_county#56, d_qoy#54, d_year#53] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#50))] +Aggregate Attributes [1]: [sum#57] +Results [4]: [ca_county#56, d_qoy#54, d_year#53, sum#58] (64) Exchange -Input [4]: [ca_county#50, d_qoy#48, d_year#47, sum#52] -Arguments: hashpartitioning(ca_county#50, d_qoy#48, d_year#47, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [4]: [ca_county#56, d_qoy#54, d_year#53, sum#58] +Arguments: hashpartitioning(ca_county#56, d_qoy#54, d_year#53, 5), ENSURE_REQUIREMENTS, [id=#59] (65) HashAggregate [codegen id : 15] -Input [4]: [ca_county#50, d_qoy#48, d_year#47, sum#52] -Keys [3]: [ca_county#50, d_qoy#48, d_year#47] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#44))#54] -Results [2]: [ca_county#50, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#44))#54,17,2) AS web_sales#55] +Input [4]: [ca_county#56, d_qoy#54, d_year#53, sum#58] +Keys [3]: [ca_county#56, 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))#60] +Results [2]: [ca_county#56, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#50))#60,17,2) AS web_sales#61] (66) BroadcastExchange -Input [2]: [ca_county#50, web_sales#55] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] +Input [2]: [ca_county#56, web_sales#61] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#62] (67) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#10] -Right keys [1]: [ca_county#50] +Right keys [1]: [ca_county#56] Join condition: None (68) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Output [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#20)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 18] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] (70) Filter [codegen id : 18] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_bill_addr_sk#43) +Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +Condition : isnotnull(ws_bill_addr_sk#63) (71) ReusedExchange [Reuses operator id: 25] -Output [3]: [d_date_sk#57, d_year#58, d_qoy#59] +Output [3]: [d_date_sk#66, d_year#67, d_qoy#68] (72) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#57] +Left keys [1]: [ws_sold_date_sk#65] +Right keys [1]: [d_date_sk#66] Join condition: None (73) Project [codegen id : 18] -Output [4]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#58, d_qoy#59] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45, d_date_sk#57, d_year#58, d_qoy#59] +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] (74) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#60, ca_county#61] +Output [2]: [ca_address_sk#69, ca_county#70] (75) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_addr_sk#43] -Right keys [1]: [ca_address_sk#60] +Left keys [1]: [ws_bill_addr_sk#63] +Right keys [1]: [ca_address_sk#69] Join condition: None (76) Project [codegen id : 18] -Output [4]: [ws_ext_sales_price#44, d_year#58, d_qoy#59, ca_county#61] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#58, d_qoy#59, ca_address_sk#60, ca_county#61] +Output [4]: [ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_county#70] +Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_address_sk#69, ca_county#70] (77) HashAggregate [codegen id : 18] -Input [4]: [ws_ext_sales_price#44, d_year#58, d_qoy#59, ca_county#61] -Keys [3]: [ca_county#61, d_qoy#59, d_year#58] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum#62] -Results [4]: [ca_county#61, d_qoy#59, d_year#58, sum#63] +Input [4]: [ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_county#70] +Keys [3]: [ca_county#70, d_qoy#68, d_year#67] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#64))] +Aggregate Attributes [1]: [sum#71] +Results [4]: [ca_county#70, d_qoy#68, d_year#67, sum#72] (78) Exchange -Input [4]: [ca_county#61, d_qoy#59, d_year#58, sum#63] -Arguments: hashpartitioning(ca_county#61, d_qoy#59, d_year#58, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [4]: [ca_county#70, d_qoy#68, d_year#67, sum#72] +Arguments: hashpartitioning(ca_county#70, d_qoy#68, d_year#67, 5), ENSURE_REQUIREMENTS, [id=#73] (79) HashAggregate [codegen id : 19] -Input [4]: [ca_county#61, d_qoy#59, d_year#58, sum#63] -Keys [3]: [ca_county#61, d_qoy#59, d_year#58] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#44))#65] -Results [2]: [ca_county#61, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#44))#65,17,2) AS web_sales#66] +Input [4]: [ca_county#70, d_qoy#68, d_year#67, sum#72] +Keys [3]: [ca_county#70, 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))#74] +Results [2]: [ca_county#70, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#64))#74,17,2) AS web_sales#75] (80) BroadcastExchange -Input [2]: [ca_county#61, web_sales#66] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#67] +Input [2]: [ca_county#70, web_sales#75] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#76] (81) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#50] -Right keys [1]: [ca_county#61] -Join condition: (CASE WHEN (web_sales#55 > 0.00) THEN CheckOverflow((promote_precision(web_sales#66) / promote_precision(web_sales#55)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#16 > 0.00) THEN CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#16)), DecimalType(37,20), true) ELSE null END) +Left keys [1]: [ca_county#56] +Right keys [1]: [ca_county#70] +Join condition: (CASE WHEN (web_sales#61 > 0.00) THEN CheckOverflow((promote_precision(web_sales#75) / promote_precision(web_sales#61)), 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) (82) Project [codegen id : 24] -Output [8]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41, ca_county#50, web_sales#55, web_sales#66] -Input [9]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41, ca_county#50, web_sales#55, ca_county#61, web_sales#66] +Output [8]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47, ca_county#56, web_sales#61, web_sales#75] +Input [9]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47, ca_county#56, web_sales#61, ca_county#70, web_sales#75] (83) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Output [3]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (84) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Input [3]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79] (85) Filter [codegen id : 22] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_bill_addr_sk#43) +Input [3]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79] +Condition : isnotnull(ws_bill_addr_sk#77) (86) ReusedExchange [Reuses operator id: 42] -Output [3]: [d_date_sk#68, d_year#69, d_qoy#70] +Output [3]: [d_date_sk#80, d_year#81, d_qoy#82] (87) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#68] +Left keys [1]: [ws_sold_date_sk#79] +Right keys [1]: [d_date_sk#80] Join condition: None (88) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#69, d_qoy#70] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45, d_date_sk#68, d_year#69, d_qoy#70] +Output [4]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, d_year#81, d_qoy#82] +Input [6]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79, d_date_sk#80, d_year#81, d_qoy#82] (89) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#71, ca_county#72] +Output [2]: [ca_address_sk#83, ca_county#84] (90) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_bill_addr_sk#43] -Right keys [1]: [ca_address_sk#71] +Left keys [1]: [ws_bill_addr_sk#77] +Right keys [1]: [ca_address_sk#83] Join condition: None (91) Project [codegen id : 22] -Output [4]: [ws_ext_sales_price#44, d_year#69, d_qoy#70, ca_county#72] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#69, d_qoy#70, ca_address_sk#71, ca_county#72] +Output [4]: [ws_ext_sales_price#78, d_year#81, d_qoy#82, ca_county#84] +Input [6]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, d_year#81, d_qoy#82, ca_address_sk#83, ca_county#84] (92) HashAggregate [codegen id : 22] -Input [4]: [ws_ext_sales_price#44, d_year#69, d_qoy#70, ca_county#72] -Keys [3]: [ca_county#72, d_qoy#70, d_year#69] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum#73] -Results [4]: [ca_county#72, d_qoy#70, d_year#69, sum#74] +Input [4]: [ws_ext_sales_price#78, d_year#81, d_qoy#82, ca_county#84] +Keys [3]: [ca_county#84, d_qoy#82, d_year#81] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#78))] +Aggregate Attributes [1]: [sum#85] +Results [4]: [ca_county#84, d_qoy#82, d_year#81, sum#86] (93) Exchange -Input [4]: [ca_county#72, d_qoy#70, d_year#69, sum#74] -Arguments: hashpartitioning(ca_county#72, d_qoy#70, d_year#69, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [4]: [ca_county#84, d_qoy#82, d_year#81, sum#86] +Arguments: hashpartitioning(ca_county#84, d_qoy#82, d_year#81, 5), ENSURE_REQUIREMENTS, [id=#87] (94) HashAggregate [codegen id : 23] -Input [4]: [ca_county#72, d_qoy#70, d_year#69, sum#74] -Keys [3]: [ca_county#72, d_qoy#70, d_year#69] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#44))#76] -Results [2]: [ca_county#72, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#44))#76,17,2) AS web_sales#77] +Input [4]: [ca_county#84, d_qoy#82, d_year#81, sum#86] +Keys [3]: [ca_county#84, d_qoy#82, d_year#81] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#78))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#78))#88] +Results [2]: [ca_county#84, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#78))#88,17,2) AS web_sales#89] (95) BroadcastExchange -Input [2]: [ca_county#72, web_sales#77] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#78] +Input [2]: [ca_county#84, web_sales#89] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#90] (96) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#50] -Right keys [1]: [ca_county#72] -Join condition: (CASE WHEN (web_sales#66 > 0.00) THEN CheckOverflow((promote_precision(web_sales#77) / promote_precision(web_sales#66)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#28 > 0.00) THEN CheckOverflow((promote_precision(store_sales#41) / promote_precision(store_sales#28)), DecimalType(37,20), true) ELSE null END) +Left keys [1]: [ca_county#56] +Right keys [1]: [ca_county#84] +Join condition: (CASE WHEN (web_sales#75 > 0.00) THEN CheckOverflow((promote_precision(web_sales#89) / promote_precision(web_sales#75)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#31 > 0.00) THEN CheckOverflow((promote_precision(store_sales#47) / promote_precision(store_sales#31)), DecimalType(37,20), true) ELSE null END) (97) Project [codegen id : 24] -Output [6]: [ca_county#10, d_year#6, CheckOverflow((promote_precision(web_sales#66) / promote_precision(web_sales#55)), DecimalType(37,20), true) AS web_q1_q2_increase#79, CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q1_q2_increase#80, CheckOverflow((promote_precision(web_sales#77) / promote_precision(web_sales#66)), DecimalType(37,20), true) AS web_q2_q3_increase#81, CheckOverflow((promote_precision(store_sales#41) / promote_precision(store_sales#28)), DecimalType(37,20), true) AS store_q2_q3_increase#82] -Input [10]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41, ca_county#50, web_sales#55, web_sales#66, ca_county#72, web_sales#77] +Output [6]: [ca_county#10, d_year#6, CheckOverflow((promote_precision(web_sales#75) / promote_precision(web_sales#61)), DecimalType(37,20), true) AS web_q1_q2_increase#91, CheckOverflow((promote_precision(store_sales#31) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q1_q2_increase#92, CheckOverflow((promote_precision(web_sales#89) / promote_precision(web_sales#75)), DecimalType(37,20), true) AS web_q2_q3_increase#93, CheckOverflow((promote_precision(store_sales#47) / promote_precision(store_sales#31)), DecimalType(37,20), true) AS store_q2_q3_increase#94] +Input [10]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47, ca_county#56, web_sales#61, web_sales#75, ca_county#84, web_sales#89] (98) Exchange -Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#79, store_q1_q2_increase#80, web_q2_q3_increase#81, store_q2_q3_increase#82] -Arguments: rangepartitioning(ca_county#10 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#91, store_q1_q2_increase#92, web_q2_q3_increase#93, store_q2_q3_increase#94] +Arguments: rangepartitioning(ca_county#10 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#95] (99) Sort [codegen id : 25] -Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#79, store_q1_q2_increase#80, web_q2_q3_increase#81, store_q2_q3_increase#82] +Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#91, store_q1_q2_increase#92, web_q2_q3_increase#93, store_q2_q3_increase#94] Arguments: [ca_county#10 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -576,24 +576,24 @@ ReusedExchange (100) (100) ReusedExchange [Reuses operator id: 7] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#17 +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#19 IN dynamicpruning#20 ReusedExchange (101) (101) ReusedExchange [Reuses operator id: 25] -Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] -Subquery:3 Hosting operator id = 36 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#30 +Subquery:3 Hosting operator id = 36 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 ReusedExchange (102) (102) ReusedExchange [Reuses operator id: 42] -Output [3]: [d_date_sk#31, d_year#32, d_qoy#33] +Output [3]: [d_date_sk#37, d_year#38, d_qoy#39] -Subquery:4 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#17 +Subquery:5 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#20 -Subquery:6 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#30 +Subquery:6 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#36 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt index 5c886e9f2295d..5d171e5f595b9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt @@ -81,7 +81,7 @@ Input [2]: [d_date_sk#8, d_date#9] (11) Filter [codegen id : 2] Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) (12) Project [codegen id : 2] Output [1]: [d_date_sk#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index 91cdd05907a75..ad918310a918a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -105,7 +105,7 @@ Input [2]: [d_date_sk#11, d_date#12] (16) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10983)) AND (d_date#12 <= 11073)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) (17) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt index 60b7965371e06..ea9994910c3b9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt @@ -121,7 +121,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -129,7 +129,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt index 6979de4765247..df5b93da51771 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt index 7098389df6776..9da5f36eb0553 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt @@ -145,20 +145,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#17] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#10] +Right keys [1]: [d_date_sk#17] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] -Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] +Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] (23) Exchange Input [1]: [ws_bill_customer_sk#15] -Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] @@ -170,38 +170,38 @@ Right keys [1]: [ws_bill_customer_sk#15] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Output [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#21] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [cs_sold_date_sk#20] +Right keys [1]: [d_date_sk#21] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#18] -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] +Output [1]: [cs_ship_customer_sk#19] +Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] (31) Exchange -Input [1]: [cs_ship_customer_sk#18] -Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [cs_ship_customer_sk#19] +Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#18] -Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#19] +Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#19] Join condition: None (34) Filter [codegen id : 12] @@ -214,103 +214,103 @@ Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2 (36) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#21] +Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#23] (37) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 (38) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_state#23] +Output [2]: [ca_address_sk#24, ca_state#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 14] -Input [2]: [ca_address_sk#22, ca_state#23] +Input [2]: [ca_address_sk#24, ca_state#25] (40) Filter [codegen id : 14] -Input [2]: [ca_address_sk#22, ca_state#23] -Condition : isnotnull(ca_address_sk#22) +Input [2]: [ca_address_sk#24, ca_state#25] +Condition : isnotnull(ca_address_sk#24) (41) Exchange -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: hashpartitioning(ca_address_sk#22, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] (42) Sort [codegen id : 15] -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#24] Join condition: None (44) Project [codegen id : 16] -Output [2]: [c_current_cdemo_sk#4, ca_state#23] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] +Output [2]: [c_current_cdemo_sk#4, ca_state#25] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24, ca_state#25] (45) Exchange -Input [2]: [c_current_cdemo_sk#4, ca_state#23] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [2]: [c_current_cdemo_sk#4, ca_state#25] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#27] (46) Sort [codegen id : 17] -Input [2]: [c_current_cdemo_sk#4, ca_state#23] +Input [2]: [c_current_cdemo_sk#4, ca_state#25] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 (47) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 18] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (49) Filter [codegen id : 18] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Condition : isnotnull(cd_demo_sk#26) +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Condition : isnotnull(cd_demo_sk#28) (50) Exchange -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: hashpartitioning(cd_demo_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#34] (51) Sort [codegen id : 19] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: [cd_demo_sk#26 ASC NULLS FIRST], false, 0 +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#26] +Right keys [1]: [cd_demo_sk#28] Join condition: None (53) Project [codegen id : 20] -Output [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [8]: [c_current_cdemo_sk#4, ca_state#25, cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (54) HashAggregate [codegen id : 20] -Input [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#29), partial_max(cd_dep_count#29), partial_avg(cd_dep_count#29), partial_min(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_avg(cd_dep_employed_count#30), partial_min(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_avg(cd_dep_college_count#31)] -Aggregate Attributes [13]: [count#33, min#34, max#35, sum#36, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45] -Results [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] +Input [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Functions [10]: [partial_count(1), partial_min(cd_dep_count#31), partial_max(cd_dep_count#31), partial_avg(cd_dep_count#31), partial_min(cd_dep_employed_count#32), partial_max(cd_dep_employed_count#32), partial_avg(cd_dep_employed_count#32), partial_min(cd_dep_college_count#33), partial_max(cd_dep_college_count#33), partial_avg(cd_dep_college_count#33)] +Aggregate Attributes [13]: [count#35, min#36, max#37, sum#38, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47] +Results [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] (55) Exchange -Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] -Arguments: hashpartitioning(ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] +Arguments: hashpartitioning(ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#61] (56) HashAggregate [codegen id : 21] -Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] -Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [count(1), min(cd_dep_count#29), max(cd_dep_count#29), avg(cd_dep_count#29), min(cd_dep_employed_count#30), max(cd_dep_employed_count#30), avg(cd_dep_employed_count#30), min(cd_dep_college_count#31), max(cd_dep_college_count#31), avg(cd_dep_college_count#31)] -Aggregate Attributes [10]: [count(1)#60, min(cd_dep_count#29)#61, max(cd_dep_count#29)#62, avg(cd_dep_count#29)#63, min(cd_dep_employed_count#30)#64, max(cd_dep_employed_count#30)#65, avg(cd_dep_employed_count#30)#66, min(cd_dep_college_count#31)#67, max(cd_dep_college_count#31)#68, avg(cd_dep_college_count#31)#69] -Results [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, count(1)#60 AS cnt1#70, min(cd_dep_count#29)#61 AS min(cd_dep_count)#71, max(cd_dep_count#29)#62 AS max(cd_dep_count)#72, avg(cd_dep_count#29)#63 AS avg(cd_dep_count)#73, cd_dep_employed_count#30, count(1)#60 AS cnt2#74, min(cd_dep_employed_count#30)#64 AS min(cd_dep_employed_count)#75, max(cd_dep_employed_count#30)#65 AS max(cd_dep_employed_count)#76, avg(cd_dep_employed_count#30)#66 AS avg(cd_dep_employed_count)#77, cd_dep_college_count#31, count(1)#60 AS cnt3#78, min(cd_dep_college_count#31)#67 AS min(cd_dep_college_count)#79, max(cd_dep_college_count#31)#68 AS max(cd_dep_college_count)#80, avg(cd_dep_college_count#31)#69 AS avg(cd_dep_college_count)#81, cd_dep_count#29 AS aggOrder#82] +Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] +Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Functions [10]: [count(1), min(cd_dep_count#31), max(cd_dep_count#31), avg(cd_dep_count#31), min(cd_dep_employed_count#32), max(cd_dep_employed_count#32), avg(cd_dep_employed_count#32), min(cd_dep_college_count#33), max(cd_dep_college_count#33), avg(cd_dep_college_count#33)] +Aggregate Attributes [10]: [count(1)#62, min(cd_dep_count#31)#63, max(cd_dep_count#31)#64, avg(cd_dep_count#31)#65, min(cd_dep_employed_count#32)#66, max(cd_dep_employed_count#32)#67, avg(cd_dep_employed_count#32)#68, min(cd_dep_college_count#33)#69, max(cd_dep_college_count#33)#70, avg(cd_dep_college_count#33)#71] +Results [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, count(1)#62 AS cnt1#72, min(cd_dep_count#31)#63 AS min(cd_dep_count)#73, max(cd_dep_count#31)#64 AS max(cd_dep_count)#74, avg(cd_dep_count#31)#65 AS avg(cd_dep_count)#75, cd_dep_employed_count#32, count(1)#62 AS cnt2#76, min(cd_dep_employed_count#32)#66 AS min(cd_dep_employed_count)#77, max(cd_dep_employed_count#32)#67 AS max(cd_dep_employed_count)#78, avg(cd_dep_employed_count#32)#68 AS avg(cd_dep_employed_count)#79, cd_dep_college_count#33, count(1)#62 AS cnt3#80, min(cd_dep_college_count#33)#69 AS min(cd_dep_college_count)#81, max(cd_dep_college_count#33)#70 AS max(cd_dep_college_count)#82, avg(cd_dep_college_count#33)#71 AS avg(cd_dep_college_count)#83, cd_dep_count#31 AS aggOrder#84] (57) TakeOrderedAndProject -Input [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cnt1#70, min(cd_dep_count)#71, max(cd_dep_count)#72, avg(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, min(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, avg(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, min(cd_dep_college_count)#79, max(cd_dep_college_count)#80, avg(cd_dep_college_count)#81, aggOrder#82] -Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, aggOrder#82 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#23, cd_gender#27, cd_marital_status#28, cnt1#70, min(cd_dep_count)#71, max(cd_dep_count)#72, avg(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, min(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, avg(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, min(cd_dep_college_count)#79, max(cd_dep_college_count)#80, avg(cd_dep_college_count)#81] +Input [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cnt1#72, min(cd_dep_count)#73, max(cd_dep_count)#74, avg(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, min(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, avg(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, min(cd_dep_college_count)#81, max(cd_dep_college_count)#82, avg(cd_dep_college_count)#83, aggOrder#84] +Arguments: 100, [ca_state#25 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, aggOrder#84 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [ca_state#25, cd_gender#29, cd_marital_status#30, cnt1#72, min(cd_dep_count)#73, max(cd_dep_count)#74, avg(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, min(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, avg(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, min(cd_dep_college_count)#81, max(cd_dep_college_count)#82, avg(cd_dep_college_count)#83] ===== Subqueries ===== @@ -323,6 +323,6 @@ Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index 4dab13491784a..48ae824834450 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -122,20 +122,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#16] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] -Right keys [1]: [d_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] +Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#16] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] @@ -143,34 +143,34 @@ Right keys [1]: [ws_bill_customer_sk#14] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#8)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#20] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#9] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#20] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#17] -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] +Output [1]: [cs_ship_customer_sk#18] +Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#20] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [cs_ship_customer_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#17] +Right keys [1]: [cs_ship_customer_sk#18] Join condition: None (29) Filter [codegen id : 9] @@ -182,80 +182,80 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +Output [2]: [ca_address_sk#22, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_state#21] +Input [2]: [ca_address_sk#22, ca_state#23] (33) Filter [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) +Input [2]: [ca_address_sk#22, ca_state#23] +Condition : isnotnull(ca_address_sk#22) (34) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +Input [2]: [ca_address_sk#22, ca_state#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] (35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#22] Join condition: None (36) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#21] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] +Output [2]: [c_current_cdemo_sk#4, ca_state#23] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] (37) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (39) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#25) (40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#25] Join condition: None (42) Project [codegen id : 9] -Output [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Output [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (43) HashAggregate [codegen id : 9] -Input [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#26), partial_max(cd_dep_count#26), partial_avg(cd_dep_count#26), partial_min(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_avg(cd_dep_employed_count#27), partial_min(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_avg(cd_dep_college_count#28)] -Aggregate Attributes [13]: [count#30, min#31, max#32, sum#33, count#34, min#35, max#36, sum#37, count#38, min#39, max#40, sum#41, count#42] -Results [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] +Input [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Functions [10]: [partial_count(1), partial_min(cd_dep_count#28), partial_max(cd_dep_count#28), partial_avg(cd_dep_count#28), partial_min(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_avg(cd_dep_employed_count#29), partial_min(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_avg(cd_dep_college_count#30)] +Aggregate Attributes [13]: [count#32, min#33, max#34, sum#35, count#36, min#37, max#38, sum#39, count#40, min#41, max#42, sum#43, count#44] +Results [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] (44) Exchange -Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] -Arguments: hashpartitioning(ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] +Arguments: hashpartitioning(ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] (45) HashAggregate [codegen id : 10] -Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] -Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), min(cd_dep_count#26), max(cd_dep_count#26), avg(cd_dep_count#26), min(cd_dep_employed_count#27), max(cd_dep_employed_count#27), avg(cd_dep_employed_count#27), min(cd_dep_college_count#28), max(cd_dep_college_count#28), avg(cd_dep_college_count#28)] -Aggregate Attributes [10]: [count(1)#57, min(cd_dep_count#26)#58, max(cd_dep_count#26)#59, avg(cd_dep_count#26)#60, min(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, avg(cd_dep_employed_count#27)#63, min(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, avg(cd_dep_college_count#28)#66] -Results [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, count(1)#57 AS cnt1#67, min(cd_dep_count#26)#58 AS min(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, avg(cd_dep_count#26)#60 AS avg(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, min(cd_dep_employed_count#27)#61 AS min(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, avg(cd_dep_employed_count#27)#63 AS avg(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, min(cd_dep_college_count#28)#64 AS min(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, avg(cd_dep_college_count#28)#66 AS avg(cd_dep_college_count)#78, cd_dep_count#26 AS aggOrder#79] +Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] +Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Functions [10]: [count(1), min(cd_dep_count#28), max(cd_dep_count#28), avg(cd_dep_count#28), min(cd_dep_employed_count#29), max(cd_dep_employed_count#29), avg(cd_dep_employed_count#29), min(cd_dep_college_count#30), max(cd_dep_college_count#30), avg(cd_dep_college_count#30)] +Aggregate Attributes [10]: [count(1)#59, min(cd_dep_count#28)#60, max(cd_dep_count#28)#61, avg(cd_dep_count#28)#62, min(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, avg(cd_dep_employed_count#29)#65, min(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, avg(cd_dep_college_count#30)#68] +Results [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, count(1)#59 AS cnt1#69, min(cd_dep_count#28)#60 AS min(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, avg(cd_dep_count#28)#62 AS avg(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, min(cd_dep_employed_count#29)#63 AS min(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, avg(cd_dep_employed_count#29)#65 AS avg(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, min(cd_dep_college_count#30)#66 AS min(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, avg(cd_dep_college_count#30)#68 AS avg(cd_dep_college_count)#80, cd_dep_count#28 AS aggOrder#81] (46) TakeOrderedAndProject -Input [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78, aggOrder#79] -Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, aggOrder#79 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#21, cd_gender#24, cd_marital_status#25, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78] +Input [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80, aggOrder#81] +Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, aggOrder#81 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#23, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80] ===== Subqueries ===== @@ -268,6 +268,6 @@ Output [1]: [d_date_sk#9] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt index 3f5ed7b48a9db..1f2c169ba0921 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt @@ -95,7 +95,7 @@ Input [2]: [d_date_sk#11, d_date#12] (14) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10988)) AND (d_date#12 <= 11048)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) (15) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt index 04c3f07b903f0..9cc89345f5a1c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt @@ -92,7 +92,7 @@ Input [2]: [d_date_sk#11, d_date#12] (14) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10988)) AND (d_date#12 <= 11048)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) (15) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt index 776585e5bd97f..71accc7295b0f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt @@ -177,11 +177,11 @@ Results [3]: [c_last_name#11, c_first_name#10, d_date#5] (23) Exchange Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#14] (24) Sort [codegen id : 8] Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 0) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 (25) Scan parquet default.catalog_sales Output [2]: [cs_bill_customer_sk#15, cs_sold_date_sk#16] @@ -254,15 +254,15 @@ Results [3]: [c_last_name#22, c_first_name#21, d_date#18] (40) Exchange Input [3]: [c_last_name#22, c_first_name#21, d_date#18] -Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 0), isnull(d_date#18), 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18), 5), ENSURE_REQUIREMENTS, [id=#24] (41) Sort [codegen id : 16] Input [3]: [c_last_name#22, c_first_name#21, d_date#18] -Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 0) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 1970-01-01) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 (42) SortMergeJoin -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 0), isnull(d_date#18)] +Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18)] Join condition: None (43) HashAggregate [codegen id : 17] @@ -285,11 +285,11 @@ Results [3]: [c_last_name#11, c_first_name#10, d_date#5] (46) Exchange Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#26] +Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#26] (47) Sort [codegen id : 19] Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 0) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 (48) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#27, ws_sold_date_sk#28] @@ -362,15 +362,15 @@ Results [3]: [c_last_name#34, c_first_name#33, d_date#30] (63) Exchange Input [3]: [c_last_name#34, c_first_name#33, d_date#30] -Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 0), isnull(d_date#30), 5), ENSURE_REQUIREMENTS, [id=#36] +Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30), 5), ENSURE_REQUIREMENTS, [id=#36] (64) Sort [codegen id : 27] Input [3]: [c_last_name#34, c_first_name#33, d_date#30] -Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 0) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 1970-01-01) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 (65) SortMergeJoin -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 0), isnull(d_date#30)] +Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30)] Join condition: None (66) HashAggregate [codegen id : 28] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt index cedc6c7f4c576..c4e679eb902e0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt @@ -205,11 +205,11 @@ Results [3]: [c_last_name#19, c_first_name#18, d_date#16] (32) BroadcastExchange Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 0), isnull(input[2, date, true])),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [id=#21] (33) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 0), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 0), isnull(d_date#16)] +Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)] Join condition: None (34) HashAggregate [codegen id : 12] @@ -285,11 +285,11 @@ Results [3]: [c_last_name#28, c_first_name#27, d_date#25] (48) BroadcastExchange Input [3]: [c_last_name#28, c_first_name#27, d_date#25] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 0), isnull(input[2, date, true])),false), [id=#30] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [id=#30] (49) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 0), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#28, ), isnull(c_last_name#28), coalesce(c_first_name#27, ), isnull(c_first_name#27), coalesce(d_date#25, 0), isnull(d_date#25)] +Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#28, ), isnull(c_last_name#28), coalesce(c_first_name#27, ), isnull(c_first_name#27), coalesce(d_date#25, 1970-01-01), isnull(d_date#25)] Join condition: None (50) HashAggregate [codegen id : 12] 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 a53bec582637d..5eb63f2a046cf 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 @@ -176,7 +176,7 @@ Results [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stddev_samp(cast(inv_qua (26) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stdev#28, mean#29] -Condition : (((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) +Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND (NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0))) (27) Project [codegen id : 5] Output [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -191,121 +191,121 @@ Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30] Arguments: [i_item_sk#10 ASC NULLS FIRST, w_warehouse_sk#12 ASC NULLS FIRST], false, 0 (30) Scan parquet default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(inv_date_sk#35), dynamicpruningexpression(inv_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 10] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] (32) Filter [codegen id : 10] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] +Condition : (isnotnull(inv_item_sk#32) AND isnotnull(inv_warehouse_sk#33)) (33) Scan parquet default.date_dim -Output [3]: [d_date_sk#33, d_year#34, d_moy#35] +Output [3]: [d_date_sk#37, d_year#38, d_moy#39] 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 (34) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] (35) Filter [codegen id : 7] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] -Condition : ((((isnotnull(d_year#34) AND isnotnull(d_moy#35)) AND (d_year#34 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#33)) +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] +Condition : ((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2001)) AND (d_moy#39 = 2)) AND isnotnull(d_date_sk#37)) (36) Project [codegen id : 7] -Output [2]: [d_date_sk#33, d_moy#35] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] +Output [2]: [d_date_sk#37, d_moy#39] +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] (37) BroadcastExchange -Input [2]: [d_date_sk#33, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [2]: [d_date_sk#37, d_moy#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [inv_date_sk#35] +Right keys [1]: [d_date_sk#37] Join condition: None (39) Project [codegen id : 10] -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35] -Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#33, d_moy#35] +Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39] +Input [6]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35, d_date_sk#37, d_moy#39] (40) ReusedExchange [Reuses operator id: 14] -Output [1]: [i_item_sk#37] +Output [1]: [i_item_sk#41] (41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#37] +Left keys [1]: [inv_item_sk#32] +Right keys [1]: [i_item_sk#41] Join condition: None (42) Project [codegen id : 10] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37] +Output [4]: [inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41] +Input [5]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41] (43) ReusedExchange [Reuses operator id: 20] -Output [2]: [w_warehouse_sk#38, w_warehouse_name#39] +Output [2]: [w_warehouse_sk#42, w_warehouse_name#43] (44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#38] +Left keys [1]: [inv_warehouse_sk#33] +Right keys [1]: [w_warehouse_sk#42] Join condition: None (45) Project [codegen id : 10] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39, d_moy#35] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39] +Output [5]: [inv_quantity_on_hand#34, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43, d_moy#39] +Input [6]: [inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43] (46) HashAggregate [codegen id : 10] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39, d_moy#35] -Keys [4]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#40, avg#41, m2#42, sum#43, count#44] -Results [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, n#45, avg#46, m2#47, sum#48, count#49] +Input [5]: [inv_quantity_on_hand#34, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43, d_moy#39] +Keys [4]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#34 as double)), partial_avg(inv_quantity_on_hand#34)] +Aggregate Attributes [5]: [n#44, avg#45, m2#46, sum#47, count#48] +Results [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] (47) Exchange -Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, n#45, avg#46, m2#47, sum#48, count#49] -Arguments: hashpartitioning(w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] +Arguments: hashpartitioning(w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, 5), ENSURE_REQUIREMENTS, [id=#54] (48) HashAggregate [codegen id : 11] -Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#51, avg(inv_quantity_on_hand#3)#52] -Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#3 as double))#51 AS stdev#53, avg(inv_quantity_on_hand#3)#52 AS mean#54] +Input [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] +Keys [4]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#34 as double)), avg(inv_quantity_on_hand#34)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#34 as double))#55, avg(inv_quantity_on_hand#34)#56] +Results [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stddev_samp(cast(inv_quantity_on_hand#34 as double))#55 AS stdev#57, avg(inv_quantity_on_hand#34)#56 AS mean#58] (49) Filter [codegen id : 11] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#42, i_item_sk#41, d_moy#39, stdev#57, mean#58] +Condition : ((isnotnull(mean#58) AND isnotnull(stdev#57)) AND (NOT (mean#58 = 0.0) AND ((stdev#57 / mean#58) > 1.0))) (50) Project [codegen id : 11] -Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#35, stdev#53, mean#54] +Output [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, CASE WHEN (mean#58 = 0.0) THEN null ELSE (stdev#57 / mean#58) END AS cov#59] +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stdev#57, mean#58] (51) Exchange -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: hashpartitioning(i_item_sk#37, w_warehouse_sk#38, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: hashpartitioning(i_item_sk#41, w_warehouse_sk#42, 5), ENSURE_REQUIREMENTS, [id=#60] (52) Sort [codegen id : 12] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: [i_item_sk#37 ASC NULLS FIRST, w_warehouse_sk#38 ASC NULLS FIRST], false, 0 +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: [i_item_sk#41 ASC NULLS FIRST, w_warehouse_sk#42 ASC NULLS FIRST], false, 0 (53) SortMergeJoin [codegen id : 13] Left keys [2]: [i_item_sk#10, w_warehouse_sk#12] -Right keys [2]: [i_item_sk#37, w_warehouse_sk#38] +Right keys [2]: [i_item_sk#41, w_warehouse_sk#42] Join condition: None (54) Exchange -Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: rangepartitioning(w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: rangepartitioning(w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, mean#58 ASC NULLS FIRST, cov#59 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#61] (55) Sort [codegen id : 14] -Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: [w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: [w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, mean#58 ASC NULLS FIRST, cov#59 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -316,11 +316,11 @@ ReusedExchange (56) (56) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#6, d_moy#8] -Subquery:2 Hosting operator id = 30 Hosting Expression = inv_date_sk#4 IN dynamicpruning#32 +Subquery:2 Hosting operator id = 30 Hosting Expression = inv_date_sk#35 IN dynamicpruning#36 ReusedExchange (57) (57) ReusedExchange [Reuses operator id: 37] -Output [2]: [d_date_sk#33, d_moy#35] +Output [2]: [d_date_sk#37, d_moy#39] 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 c61480a05a4cf..e77de53c5adcb 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 @@ -173,124 +173,124 @@ Results [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stddev_samp(cast(inv_quan (26) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] -Condition : (((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) +Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND (NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0))) (27) Project [codegen id : 10] Output [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] (28) Scan parquet default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Output [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(inv_date_sk#34), dynamicpruningexpression(inv_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] (30) Filter [codegen id : 8] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) +Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] +Condition : (isnotnull(inv_item_sk#31) AND isnotnull(inv_warehouse_sk#32)) (31) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#32] +Output [1]: [i_item_sk#36] (32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [inv_item_sk#31] +Right keys [1]: [i_item_sk#36] Join condition: None (33) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32] +Output [4]: [inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36] +Input [5]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36] (34) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#33, w_warehouse_name#34] +Output [2]: [w_warehouse_sk#37, w_warehouse_name#38] (35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#33] +Left keys [1]: [inv_warehouse_sk#32] +Right keys [1]: [w_warehouse_sk#37] Join condition: None (36) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] +Output [5]: [inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38] +Input [6]: [inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38] (37) Scan parquet default.date_dim -Output [3]: [d_date_sk#35, d_year#36, d_moy#37] +Output [3]: [d_date_sk#39, d_year#40, d_moy#41] 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 (38) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] (39) Filter [codegen id : 7] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2001)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] +Condition : ((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2001)) AND (d_moy#41 = 2)) AND isnotnull(d_date_sk#39)) (40) Project [codegen id : 7] -Output [2]: [d_date_sk#35, d_moy#37] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Output [2]: [d_date_sk#39, d_moy#41] +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] (41) BroadcastExchange -Input [2]: [d_date_sk#35, d_moy#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] +Input [2]: [d_date_sk#39, d_moy#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (42) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#35] +Left keys [1]: [inv_date_sk#34] +Right keys [1]: [d_date_sk#39] Join condition: None (43) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_date_sk#35, d_moy#37] +Output [5]: [inv_quantity_on_hand#33, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_moy#41] +Input [7]: [inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_date_sk#39, d_moy#41] (44) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] -Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#39, avg#40, m2#41, sum#42, count#43] -Results [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] +Input [5]: [inv_quantity_on_hand#33, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_moy#41] +Keys [4]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#33 as double)), partial_avg(inv_quantity_on_hand#33)] +Aggregate Attributes [5]: [n#43, avg#44, m2#45, sum#46, count#47] +Results [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] (45) Exchange -Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] -Arguments: hashpartitioning(w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] +Arguments: hashpartitioning(w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, 5), ENSURE_REQUIREMENTS, [id=#53] (46) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] -Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#50, avg(inv_quantity_on_hand#3)#51] -Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_quantity_on_hand#3 as double))#50 AS stdev#52, avg(inv_quantity_on_hand#3)#51 AS mean#53] +Input [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] +Keys [4]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#33 as double)), avg(inv_quantity_on_hand#33)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#33 as double))#54, avg(inv_quantity_on_hand#33)#55] +Results [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stddev_samp(cast(inv_quantity_on_hand#33 as double))#54 AS stdev#56, avg(inv_quantity_on_hand#33)#55 AS mean#57] (47) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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#37, i_item_sk#36, d_moy#41, stdev#56, mean#57] +Condition : ((isnotnull(mean#57) AND isnotnull(stdev#56)) AND (NOT (mean#57 = 0.0) AND ((stdev#56 / mean#57) > 1.0))) (48) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#52, mean#53] +Output [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, CASE WHEN (mean#57 = 0.0) THEN null ELSE (stdev#56 / mean#57) END AS cov#58] +Input [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stdev#56, mean#57] (49) BroadcastExchange -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [id=#59] (50) BroadcastHashJoin [codegen id : 10] Left keys [2]: [i_item_sk#6, w_warehouse_sk#8] -Right keys [2]: [i_item_sk#32, w_warehouse_sk#33] +Right keys [2]: [i_item_sk#36, w_warehouse_sk#37] Join condition: None (51) Exchange -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, cov#54] -Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#37 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#13, mean#29, cov#30, w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, mean#57 ASC NULLS FIRST, cov#58 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#60] (52) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, cov#54] -Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#37 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#13, mean#29, cov#30, w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, mean#57 ASC NULLS FIRST, cov#58 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -301,11 +301,11 @@ ReusedExchange (53) (53) ReusedExchange [Reuses operator id: 20] Output [2]: [d_date_sk#11, d_moy#13] -Subquery:2 Hosting operator id = 28 Hosting Expression = inv_date_sk#4 IN dynamicpruning#31 +Subquery:2 Hosting operator id = 28 Hosting Expression = inv_date_sk#34 IN dynamicpruning#35 ReusedExchange (54) (54) ReusedExchange [Reuses operator id: 41] -Output [2]: [d_date_sk#35, d_moy#37] +Output [2]: [d_date_sk#39, d_moy#41] 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 6bc5980231252..e2e3760fe2d03 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 @@ -176,7 +176,7 @@ Results [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stddev_samp(cast(inv_qua (26) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stdev#28, mean#29] -Condition : (((((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.5)) +Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND ((NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0)) AND ((stdev#28 / mean#29) > 1.5))) (27) Project [codegen id : 5] Output [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -191,121 +191,121 @@ Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30] Arguments: [i_item_sk#10 ASC NULLS FIRST, w_warehouse_sk#12 ASC NULLS FIRST], false, 0 (30) Scan parquet default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(inv_date_sk#35), dynamicpruningexpression(inv_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 10] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] (32) Filter [codegen id : 10] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] +Condition : (isnotnull(inv_item_sk#32) AND isnotnull(inv_warehouse_sk#33)) (33) Scan parquet default.date_dim -Output [3]: [d_date_sk#33, d_year#34, d_moy#35] +Output [3]: [d_date_sk#37, d_year#38, d_moy#39] 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 (34) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] (35) Filter [codegen id : 7] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] -Condition : ((((isnotnull(d_year#34) AND isnotnull(d_moy#35)) AND (d_year#34 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#33)) +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] +Condition : ((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2001)) AND (d_moy#39 = 2)) AND isnotnull(d_date_sk#37)) (36) Project [codegen id : 7] -Output [2]: [d_date_sk#33, d_moy#35] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] +Output [2]: [d_date_sk#37, d_moy#39] +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] (37) BroadcastExchange -Input [2]: [d_date_sk#33, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [2]: [d_date_sk#37, d_moy#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [inv_date_sk#35] +Right keys [1]: [d_date_sk#37] Join condition: None (39) Project [codegen id : 10] -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35] -Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#33, d_moy#35] +Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39] +Input [6]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35, d_date_sk#37, d_moy#39] (40) ReusedExchange [Reuses operator id: 14] -Output [1]: [i_item_sk#37] +Output [1]: [i_item_sk#41] (41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#37] +Left keys [1]: [inv_item_sk#32] +Right keys [1]: [i_item_sk#41] Join condition: None (42) Project [codegen id : 10] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37] +Output [4]: [inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41] +Input [5]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41] (43) ReusedExchange [Reuses operator id: 20] -Output [2]: [w_warehouse_sk#38, w_warehouse_name#39] +Output [2]: [w_warehouse_sk#42, w_warehouse_name#43] (44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#38] +Left keys [1]: [inv_warehouse_sk#33] +Right keys [1]: [w_warehouse_sk#42] Join condition: None (45) Project [codegen id : 10] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39, d_moy#35] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39] +Output [5]: [inv_quantity_on_hand#34, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43, d_moy#39] +Input [6]: [inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43] (46) HashAggregate [codegen id : 10] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39, d_moy#35] -Keys [4]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#40, avg#41, m2#42, sum#43, count#44] -Results [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, n#45, avg#46, m2#47, sum#48, count#49] +Input [5]: [inv_quantity_on_hand#34, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43, d_moy#39] +Keys [4]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#34 as double)), partial_avg(inv_quantity_on_hand#34)] +Aggregate Attributes [5]: [n#44, avg#45, m2#46, sum#47, count#48] +Results [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] (47) Exchange -Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, n#45, avg#46, m2#47, sum#48, count#49] -Arguments: hashpartitioning(w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] +Arguments: hashpartitioning(w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, 5), ENSURE_REQUIREMENTS, [id=#54] (48) HashAggregate [codegen id : 11] -Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#51, avg(inv_quantity_on_hand#3)#52] -Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#3 as double))#51 AS stdev#53, avg(inv_quantity_on_hand#3)#52 AS mean#54] +Input [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] +Keys [4]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#34 as double)), avg(inv_quantity_on_hand#34)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#34 as double))#55, avg(inv_quantity_on_hand#34)#56] +Results [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stddev_samp(cast(inv_quantity_on_hand#34 as double))#55 AS stdev#57, avg(inv_quantity_on_hand#34)#56 AS mean#58] (49) Filter [codegen id : 11] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#42, i_item_sk#41, d_moy#39, stdev#57, mean#58] +Condition : ((isnotnull(mean#58) AND isnotnull(stdev#57)) AND (NOT (mean#58 = 0.0) AND ((stdev#57 / mean#58) > 1.0))) (50) Project [codegen id : 11] -Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#35, stdev#53, mean#54] +Output [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, CASE WHEN (mean#58 = 0.0) THEN null ELSE (stdev#57 / mean#58) END AS cov#59] +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stdev#57, mean#58] (51) Exchange -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: hashpartitioning(i_item_sk#37, w_warehouse_sk#38, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: hashpartitioning(i_item_sk#41, w_warehouse_sk#42, 5), ENSURE_REQUIREMENTS, [id=#60] (52) Sort [codegen id : 12] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: [i_item_sk#37 ASC NULLS FIRST, w_warehouse_sk#38 ASC NULLS FIRST], false, 0 +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: [i_item_sk#41 ASC NULLS FIRST, w_warehouse_sk#42 ASC NULLS FIRST], false, 0 (53) SortMergeJoin [codegen id : 13] Left keys [2]: [i_item_sk#10, w_warehouse_sk#12] -Right keys [2]: [i_item_sk#37, w_warehouse_sk#38] +Right keys [2]: [i_item_sk#41, w_warehouse_sk#42] Join condition: None (54) Exchange -Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: rangepartitioning(w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: rangepartitioning(w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, mean#58 ASC NULLS FIRST, cov#59 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#61] (55) Sort [codegen id : 14] -Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: [w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: [w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, mean#58 ASC NULLS FIRST, cov#59 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -316,11 +316,11 @@ ReusedExchange (56) (56) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#6, d_moy#8] -Subquery:2 Hosting operator id = 30 Hosting Expression = inv_date_sk#4 IN dynamicpruning#32 +Subquery:2 Hosting operator id = 30 Hosting Expression = inv_date_sk#35 IN dynamicpruning#36 ReusedExchange (57) (57) ReusedExchange [Reuses operator id: 37] -Output [2]: [d_date_sk#33, d_moy#35] +Output [2]: [d_date_sk#37, d_moy#39] 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 0ffe40240be2b..d60cd37ce7bf1 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 @@ -173,124 +173,124 @@ Results [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stddev_samp(cast(inv_quan (26) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] -Condition : (((((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.5)) +Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND ((NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0)) AND ((stdev#28 / mean#29) > 1.5))) (27) Project [codegen id : 10] Output [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] (28) Scan parquet default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Output [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(inv_date_sk#34), dynamicpruningexpression(inv_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] (30) Filter [codegen id : 8] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) +Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] +Condition : (isnotnull(inv_item_sk#31) AND isnotnull(inv_warehouse_sk#32)) (31) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#32] +Output [1]: [i_item_sk#36] (32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [inv_item_sk#31] +Right keys [1]: [i_item_sk#36] Join condition: None (33) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32] +Output [4]: [inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36] +Input [5]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36] (34) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#33, w_warehouse_name#34] +Output [2]: [w_warehouse_sk#37, w_warehouse_name#38] (35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#33] +Left keys [1]: [inv_warehouse_sk#32] +Right keys [1]: [w_warehouse_sk#37] Join condition: None (36) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] +Output [5]: [inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38] +Input [6]: [inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38] (37) Scan parquet default.date_dim -Output [3]: [d_date_sk#35, d_year#36, d_moy#37] +Output [3]: [d_date_sk#39, d_year#40, d_moy#41] 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 (38) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] (39) Filter [codegen id : 7] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2001)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] +Condition : ((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2001)) AND (d_moy#41 = 2)) AND isnotnull(d_date_sk#39)) (40) Project [codegen id : 7] -Output [2]: [d_date_sk#35, d_moy#37] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Output [2]: [d_date_sk#39, d_moy#41] +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] (41) BroadcastExchange -Input [2]: [d_date_sk#35, d_moy#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] +Input [2]: [d_date_sk#39, d_moy#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (42) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#35] +Left keys [1]: [inv_date_sk#34] +Right keys [1]: [d_date_sk#39] Join condition: None (43) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_date_sk#35, d_moy#37] +Output [5]: [inv_quantity_on_hand#33, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_moy#41] +Input [7]: [inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_date_sk#39, d_moy#41] (44) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] -Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#39, avg#40, m2#41, sum#42, count#43] -Results [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] +Input [5]: [inv_quantity_on_hand#33, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_moy#41] +Keys [4]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#33 as double)), partial_avg(inv_quantity_on_hand#33)] +Aggregate Attributes [5]: [n#43, avg#44, m2#45, sum#46, count#47] +Results [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] (45) Exchange -Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] -Arguments: hashpartitioning(w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] +Arguments: hashpartitioning(w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, 5), ENSURE_REQUIREMENTS, [id=#53] (46) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] -Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#50, avg(inv_quantity_on_hand#3)#51] -Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_quantity_on_hand#3 as double))#50 AS stdev#52, avg(inv_quantity_on_hand#3)#51 AS mean#53] +Input [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] +Keys [4]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#33 as double)), avg(inv_quantity_on_hand#33)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#33 as double))#54, avg(inv_quantity_on_hand#33)#55] +Results [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stddev_samp(cast(inv_quantity_on_hand#33 as double))#54 AS stdev#56, avg(inv_quantity_on_hand#33)#55 AS mean#57] (47) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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#37, i_item_sk#36, d_moy#41, stdev#56, mean#57] +Condition : ((isnotnull(mean#57) AND isnotnull(stdev#56)) AND (NOT (mean#57 = 0.0) AND ((stdev#56 / mean#57) > 1.0))) (48) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#52, mean#53] +Output [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, CASE WHEN (mean#57 = 0.0) THEN null ELSE (stdev#56 / mean#57) END AS cov#58] +Input [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stdev#56, mean#57] (49) BroadcastExchange -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [id=#59] (50) BroadcastHashJoin [codegen id : 10] Left keys [2]: [i_item_sk#6, w_warehouse_sk#8] -Right keys [2]: [i_item_sk#32, w_warehouse_sk#33] +Right keys [2]: [i_item_sk#36, w_warehouse_sk#37] Join condition: None (51) Exchange -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, cov#54] -Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#37 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#13, mean#29, cov#30, w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, mean#57 ASC NULLS FIRST, cov#58 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#60] (52) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, cov#54] -Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#37 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#13, mean#29, cov#30, w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, mean#57 ASC NULLS FIRST, cov#58 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -301,11 +301,11 @@ ReusedExchange (53) (53) ReusedExchange [Reuses operator id: 20] Output [2]: [d_date_sk#11, d_moy#13] -Subquery:2 Hosting operator id = 28 Hosting Expression = inv_date_sk#4 IN dynamicpruning#31 +Subquery:2 Hosting operator id = 28 Hosting Expression = inv_date_sk#34 IN dynamicpruning#35 ReusedExchange (54) (54) ReusedExchange [Reuses operator id: 41] -Output [2]: [d_date_sk#35, d_moy#37] +Output [2]: [d_date_sk#39, d_moy#41] 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 7b6bd35bfe180..ce8b8bed5a26e 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 @@ -239,465 +239,465 @@ Input [2]: [customer_id#27, year_total#28] Arguments: [customer_id#27 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6] +Output [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ss_sold_date_sk#35), dynamicpruningexpression(ss_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6] +Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] (27) Filter [codegen id : 10] -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_customer_sk#1) +Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Condition : isnotnull(ss_customer_sk#30) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#37, d_year#38] 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 (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#8, d_year#9] +Input [2]: [d_date_sk#37, d_year#38] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#8, d_year#9] -Condition : ((isnotnull(d_year#9) AND (d_year#9 = 2002)) AND isnotnull(d_date_sk#8)) +Input [2]: [d_date_sk#37, d_year#38] +Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_sk#37)) (31) BroadcastExchange -Input [2]: [d_date_sk#8, d_year#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [2]: [d_date_sk#37, d_year#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#39] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ss_sold_date_sk#35] +Right keys [1]: [d_date_sk#37] Join condition: None (33) Project [codegen id : 10] -Output [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] -Input [8]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6, d_date_sk#8, d_year#9] +Output [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Input [8]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#37, d_year#38] (34) Exchange -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] -Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Arguments: hashpartitioning(ss_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#40] (35) Sort [codegen id : 11] -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] -Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Arguments: [ss_customer_sk#30 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] (37) Sort [codegen id : 13] -Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] -Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +Arguments: [c_customer_sk#41 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#12] +Left keys [1]: [ss_customer_sk#30] +Right keys [1]: [c_customer_sk#41] Join condition: None (39) Project [codegen id : 14] -Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] -Input [14]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Input [14]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38, c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] (40) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#35, isEmpty#36] +Input [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#49, isEmpty#50] +Results [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, sum#51, isEmpty#52] (41) Exchange -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#35, isEmpty#36] -Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, sum#51, isEmpty#52] +Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, 5), ENSURE_REQUIREMENTS, [id=#53] (42) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#35, isEmpty#36] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 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#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#38] -Results [8]: [c_customer_id#13 AS customer_id#39, c_first_name#14 AS customer_first_name#40, c_last_name#15 AS customer_last_name#41, c_preferred_cust_flag#16 AS customer_preferred_cust_flag#42, c_birth_country#17 AS customer_birth_country#43, c_login#18 AS customer_login#44, c_email_address#19 AS customer_email_address#45, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#38 AS year_total#46] +Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, sum#51, isEmpty#52] +Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 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#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#54] +Results [8]: [c_customer_id#42 AS customer_id#55, c_first_name#43 AS customer_first_name#56, c_last_name#44 AS customer_last_name#57, c_preferred_cust_flag#45 AS customer_preferred_cust_flag#58, c_birth_country#46 AS customer_birth_country#59, c_login#47 AS customer_login#60, c_email_address#48 AS customer_email_address#61, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#54 AS year_total#62] (43) Exchange -Input [8]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46] -Arguments: hashpartitioning(customer_id#39, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [8]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62] +Arguments: hashpartitioning(customer_id#55, 5), ENSURE_REQUIREMENTS, [id=#63] (44) Sort [codegen id : 16] -Input [8]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46] -Arguments: [customer_id#39 ASC NULLS FIRST], false, 0 +Input [8]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62] +Arguments: [customer_id#55 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#39] +Right keys [1]: [customer_id#55] Join condition: None (46) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] +Output [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#53), dynamicpruningexpression(cs_sold_date_sk#53 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#69), dynamicpruningexpression(cs_sold_date_sk#69 IN dynamicpruning#7)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] +Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69] (48) Filter [codegen id : 19] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] -Condition : isnotnull(cs_bill_customer_sk#48) +Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69] +Condition : isnotnull(cs_bill_customer_sk#64) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#70, d_year#71] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#69] +Right keys [1]: [d_date_sk#70] Join condition: None (51) Project [codegen id : 19] -Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Input [8]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53, d_date_sk#8, d_year#9] +Output [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] +Input [8]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69, d_date_sk#70, d_year#71] (52) Exchange -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Arguments: hashpartitioning(cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] +Arguments: hashpartitioning(cs_bill_customer_sk#64, 5), ENSURE_REQUIREMENTS, [id=#72] (53) Sort [codegen id : 20] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Arguments: [cs_bill_customer_sk#48 ASC NULLS FIRST], false, 0 +Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] +Arguments: [cs_bill_customer_sk#64 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] (55) Sort [codegen id : 22] -Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] -Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Arguments: [c_customer_sk#73 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#48] -Right keys [1]: [c_customer_sk#12] +Left keys [1]: [cs_bill_customer_sk#64] +Right keys [1]: [c_customer_sk#73] Join condition: None (57) Project [codegen id : 23] -Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Input [14]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] +Input [14]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71, c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] (58) HashAggregate [codegen id : 23] -Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#55, isEmpty#56] -Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#57, isEmpty#58] +Input [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] +Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#81, isEmpty#82] +Results [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, sum#83, isEmpty#84] (59) Exchange -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#57, isEmpty#58] -Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, sum#83, isEmpty#84] +Arguments: hashpartitioning(c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, 5), ENSURE_REQUIREMENTS, [id=#85] (60) HashAggregate [codegen id : 24] -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#57, isEmpty#58] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 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#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#60] -Results [2]: [c_customer_id#13 AS customer_id#61, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#60 AS year_total#62] +Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, sum#83, isEmpty#84] +Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 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#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86] +Results [2]: [c_customer_id#74 AS customer_id#87, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86 AS year_total#88] (61) Filter [codegen id : 24] -Input [2]: [customer_id#61, year_total#62] -Condition : (isnotnull(year_total#62) AND (year_total#62 > 0.000000)) +Input [2]: [customer_id#87, year_total#88] +Condition : (isnotnull(year_total#88) AND (year_total#88 > 0.000000)) (62) Project [codegen id : 24] -Output [2]: [customer_id#61 AS customer_id#63, year_total#62 AS year_total#64] -Input [2]: [customer_id#61, year_total#62] +Output [2]: [customer_id#87 AS customer_id#89, year_total#88 AS year_total#90] +Input [2]: [customer_id#87, year_total#88] (63) Exchange -Input [2]: [customer_id#63, year_total#64] -Arguments: hashpartitioning(customer_id#63, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [2]: [customer_id#89, year_total#90] +Arguments: hashpartitioning(customer_id#89, 5), ENSURE_REQUIREMENTS, [id=#91] (64) Sort [codegen id : 25] -Input [2]: [customer_id#63, year_total#64] -Arguments: [customer_id#63 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#89, year_total#90] +Arguments: [customer_id#89 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#63] +Right keys [1]: [customer_id#89] Join condition: None (66) Project [codegen id : 26] -Output [11]: [customer_id#27, year_total#28, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46, year_total#64] -Input [12]: [customer_id#27, year_total#28, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46, customer_id#63, year_total#64] +Output [11]: [customer_id#27, year_total#28, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62, year_total#90] +Input [12]: [customer_id#27, year_total#28, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62, customer_id#89, year_total#90] (67) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] +Output [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#53), dynamicpruningexpression(cs_sold_date_sk#53 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(cs_sold_date_sk#97), dynamicpruningexpression(cs_sold_date_sk#97 IN dynamicpruning#36)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] +Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97] (69) Filter [codegen id : 28] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] -Condition : isnotnull(cs_bill_customer_sk#48) +Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97] +Condition : isnotnull(cs_bill_customer_sk#92) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#98, d_year#99] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#97] +Right keys [1]: [d_date_sk#98] Join condition: None (72) Project [codegen id : 28] -Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Input [8]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53, d_date_sk#8, d_year#9] +Output [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] +Input [8]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97, d_date_sk#98, d_year#99] (73) Exchange -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Arguments: hashpartitioning(cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] +Arguments: hashpartitioning(cs_bill_customer_sk#92, 5), ENSURE_REQUIREMENTS, [id=#100] (74) Sort [codegen id : 29] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Arguments: [cs_bill_customer_sk#48 ASC NULLS FIRST], false, 0 +Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] +Arguments: [cs_bill_customer_sk#92 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] (76) Sort [codegen id : 31] -Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] -Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] +Arguments: [c_customer_sk#101 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [cs_bill_customer_sk#48] -Right keys [1]: [c_customer_sk#12] +Left keys [1]: [cs_bill_customer_sk#92] +Right keys [1]: [c_customer_sk#101] Join condition: None (78) Project [codegen id : 32] -Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Input [14]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] +Input [14]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99, c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] (79) HashAggregate [codegen id : 32] -Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#67, isEmpty#68] -Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#69, isEmpty#70] +Input [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] +Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#109, isEmpty#110] +Results [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, sum#111, isEmpty#112] (80) Exchange -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#69, isEmpty#70] -Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, sum#111, isEmpty#112] +Arguments: hashpartitioning(c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, 5), ENSURE_REQUIREMENTS, [id=#113] (81) HashAggregate [codegen id : 33] -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#69, isEmpty#70] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 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#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#72] -Results [2]: [c_customer_id#13 AS customer_id#73, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#72 AS year_total#74] +Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, sum#111, isEmpty#112] +Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 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#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#114] +Results [2]: [c_customer_id#102 AS customer_id#115, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#114 AS year_total#116] (82) Exchange -Input [2]: [customer_id#73, year_total#74] -Arguments: hashpartitioning(customer_id#73, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [2]: [customer_id#115, year_total#116] +Arguments: hashpartitioning(customer_id#115, 5), ENSURE_REQUIREMENTS, [id=#117] (83) Sort [codegen id : 34] -Input [2]: [customer_id#73, year_total#74] -Arguments: [customer_id#73 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#115, year_total#116] +Arguments: [customer_id#115 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#73] -Join condition: (CASE WHEN (year_total#64 > 0.000000) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#64)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#28 > 0.000000) THEN CheckOverflow((promote_precision(year_total#46) / promote_precision(year_total#28)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#115] +Join condition: (CASE WHEN (year_total#90 > 0.000000) THEN CheckOverflow((promote_precision(year_total#116) / promote_precision(year_total#90)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#28 > 0.000000) THEN CheckOverflow((promote_precision(year_total#62) / promote_precision(year_total#28)), DecimalType(38,14), true) ELSE null END) (85) Project [codegen id : 35] -Output [10]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74] -Input [13]: [customer_id#27, year_total#28, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46, year_total#64, customer_id#73, year_total#74] +Output [10]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116] +Input [13]: [customer_id#27, year_total#28, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62, year_total#90, customer_id#115, year_total#116] (86) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] +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#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ws_sold_date_sk#123), dynamicpruningexpression(ws_sold_date_sk#123 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 37] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] +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] (88) Filter [codegen id : 37] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_bill_customer_sk#76) +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) (89) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#124, d_year#125] (90) BroadcastHashJoin [codegen id : 37] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ws_sold_date_sk#123] +Right keys [1]: [d_date_sk#124] Join condition: None (91) Project [codegen id : 37] -Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Input [8]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#8, d_year#9] +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, d_year#125] +Input [8]: [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, d_date_sk#124, d_year#125] (92) Exchange -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Arguments: hashpartitioning(ws_bill_customer_sk#76, 5), ENSURE_REQUIREMENTS, [id=#82] +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, d_year#125] +Arguments: hashpartitioning(ws_bill_customer_sk#118, 5), ENSURE_REQUIREMENTS, [id=#126] (93) Sort [codegen id : 38] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Arguments: [ws_bill_customer_sk#76 ASC NULLS FIRST], false, 0 +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, d_year#125] +Arguments: [ws_bill_customer_sk#118 ASC NULLS FIRST], false, 0 (94) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [8]: [c_customer_sk#127, c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134] (95) Sort [codegen id : 40] -Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] -Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#127, c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134] +Arguments: [c_customer_sk#127 ASC NULLS FIRST], false, 0 (96) SortMergeJoin [codegen id : 41] -Left keys [1]: [ws_bill_customer_sk#76] -Right keys [1]: [c_customer_sk#12] +Left keys [1]: [ws_bill_customer_sk#118] +Right keys [1]: [c_customer_sk#127] Join condition: None (97) Project [codegen id : 41] -Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Input [14]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [12]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, d_year#125] +Input [14]: [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, d_year#125, c_customer_sk#127, c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134] (98) HashAggregate [codegen id : 41] -Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#83, isEmpty#84] -Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#85, isEmpty#86] +Input [12]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, d_year#125] +Keys [8]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125] +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#135, isEmpty#136] +Results [10]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, sum#137, isEmpty#138] (99) Exchange -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#85, isEmpty#86] -Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#87] +Input [10]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, sum#137, isEmpty#138] +Arguments: hashpartitioning(c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, 5), ENSURE_REQUIREMENTS, [id=#139] (100) HashAggregate [codegen id : 42] -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#85, isEmpty#86] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 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#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#88] -Results [2]: [c_customer_id#13 AS customer_id#89, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#88 AS year_total#90] +Input [10]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, sum#137, isEmpty#138] +Keys [8]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125] +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))#140] +Results [2]: [c_customer_id#128 AS customer_id#141, 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))#140 AS year_total#142] (101) Filter [codegen id : 42] -Input [2]: [customer_id#89, year_total#90] -Condition : (isnotnull(year_total#90) AND (year_total#90 > 0.000000)) +Input [2]: [customer_id#141, year_total#142] +Condition : (isnotnull(year_total#142) AND (year_total#142 > 0.000000)) (102) Project [codegen id : 42] -Output [2]: [customer_id#89 AS customer_id#91, year_total#90 AS year_total#92] -Input [2]: [customer_id#89, year_total#90] +Output [2]: [customer_id#141 AS customer_id#143, year_total#142 AS year_total#144] +Input [2]: [customer_id#141, year_total#142] (103) Exchange -Input [2]: [customer_id#91, year_total#92] -Arguments: hashpartitioning(customer_id#91, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [2]: [customer_id#143, year_total#144] +Arguments: hashpartitioning(customer_id#143, 5), ENSURE_REQUIREMENTS, [id=#145] (104) Sort [codegen id : 43] -Input [2]: [customer_id#91, year_total#92] -Arguments: [customer_id#91 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#143, year_total#144] +Arguments: [customer_id#143 ASC NULLS FIRST], false, 0 (105) SortMergeJoin [codegen id : 44] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#91] +Right keys [1]: [customer_id#143] Join condition: None (106) Project [codegen id : 44] -Output [11]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74, year_total#92] -Input [12]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74, customer_id#91, year_total#92] +Output [11]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116, year_total#144] +Input [12]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116, customer_id#143, year_total#144] (107) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] +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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ws_sold_date_sk#151), dynamicpruningexpression(ws_sold_date_sk#151 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 46] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] +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] (109) Filter [codegen id : 46] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_bill_customer_sk#76) +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) (110) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#152, d_year#153] (111) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ws_sold_date_sk#151] +Right keys [1]: [d_date_sk#152] Join condition: None (112) Project [codegen id : 46] -Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Input [8]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#8, d_year#9] +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, d_year#153] +Input [8]: [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, d_date_sk#152, d_year#153] (113) Exchange -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Arguments: hashpartitioning(ws_bill_customer_sk#76, 5), ENSURE_REQUIREMENTS, [id=#94] +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, d_year#153] +Arguments: hashpartitioning(ws_bill_customer_sk#146, 5), ENSURE_REQUIREMENTS, [id=#154] (114) Sort [codegen id : 47] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Arguments: [ws_bill_customer_sk#76 ASC NULLS FIRST], false, 0 +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, d_year#153] +Arguments: [ws_bill_customer_sk#146 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] (116) Sort [codegen id : 49] -Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] -Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] +Arguments: [c_customer_sk#155 ASC NULLS FIRST], false, 0 (117) SortMergeJoin [codegen id : 50] -Left keys [1]: [ws_bill_customer_sk#76] -Right keys [1]: [c_customer_sk#12] +Left keys [1]: [ws_bill_customer_sk#146] +Right keys [1]: [c_customer_sk#155] Join condition: None (118) Project [codegen id : 50] -Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Input [14]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] +Input [14]: [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, d_year#153, c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] (119) HashAggregate [codegen id : 50] -Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#95, isEmpty#96] -Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#97, isEmpty#98] +Input [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] +Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] +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#163, isEmpty#164] +Results [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] (120) Exchange -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#97, isEmpty#98] -Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#99] +Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] +Arguments: hashpartitioning(c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, 5), ENSURE_REQUIREMENTS, [id=#167] (121) HashAggregate [codegen id : 51] -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#97, isEmpty#98] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 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#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#100] -Results [2]: [c_customer_id#13 AS customer_id#101, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#100 AS year_total#102] +Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] +Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] +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))#168] +Results [2]: [c_customer_id#156 AS customer_id#169, 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))#168 AS year_total#170] (122) Exchange -Input [2]: [customer_id#101, year_total#102] -Arguments: hashpartitioning(customer_id#101, 5), ENSURE_REQUIREMENTS, [id=#103] +Input [2]: [customer_id#169, year_total#170] +Arguments: hashpartitioning(customer_id#169, 5), ENSURE_REQUIREMENTS, [id=#171] (123) Sort [codegen id : 52] -Input [2]: [customer_id#101, year_total#102] -Arguments: [customer_id#101 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#169, year_total#170] +Arguments: [customer_id#169 ASC NULLS FIRST], false, 0 (124) SortMergeJoin [codegen id : 53] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#101] -Join condition: (CASE WHEN (year_total#64 > 0.000000) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#64)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#92 > 0.000000) THEN CheckOverflow((promote_precision(year_total#102) / promote_precision(year_total#92)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#169] +Join condition: (CASE WHEN (year_total#90 > 0.000000) THEN CheckOverflow((promote_precision(year_total#116) / promote_precision(year_total#90)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#144 > 0.000000) THEN CheckOverflow((promote_precision(year_total#170) / promote_precision(year_total#144)), DecimalType(38,14), true) ELSE null END) (125) Project [codegen id : 53] -Output [7]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45] -Input [13]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74, year_total#92, customer_id#101, year_total#102] +Output [7]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61] +Input [13]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116, year_total#144, customer_id#169, year_total#170] (126) TakeOrderedAndProject -Input [7]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45] -Arguments: 100, [customer_id#39 ASC NULLS FIRST, customer_first_name#40 ASC NULLS FIRST, customer_last_name#41 ASC NULLS FIRST, customer_preferred_cust_flag#42 ASC NULLS FIRST, customer_birth_country#43 ASC NULLS FIRST, customer_login#44 ASC NULLS FIRST, customer_email_address#45 ASC NULLS FIRST], [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45] +Input [7]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61] +Arguments: 100, [customer_id#55 ASC NULLS FIRST, customer_first_name#56 ASC NULLS FIRST, customer_last_name#57 ASC NULLS FIRST, customer_preferred_cust_flag#58 ASC NULLS FIRST, customer_birth_country#59 ASC NULLS FIRST, customer_login#60 ASC NULLS FIRST, customer_email_address#61 ASC NULLS FIRST], [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61] ===== Subqueries ===== @@ -708,19 +708,19 @@ ReusedExchange (127) (127) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#8, d_year#9] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#30 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 ReusedExchange (128) (128) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#37, d_year#38] -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#69 IN dynamicpruning#7 -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#30 +Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#97 IN dynamicpruning#36 -Subquery:5 Hosting operator id = 86 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#7 +Subquery:5 Hosting operator id = 86 Hosting Expression = ws_sold_date_sk#123 IN dynamicpruning#7 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#30 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#151 IN dynamicpruning#36 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 80dee3d164511..1cf27630d2994 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 @@ -209,460 +209,460 @@ Input [2]: [customer_id#26, year_total#27] Condition : (isnotnull(year_total#27) AND (year_total#27 > 0.000000)) (20) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] (22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] +Condition : (isnotnull(c_customer_sk#28) AND isnotnull(c_customer_id#29)) (23) Scan parquet default.store_sales -Output [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Output [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(ss_sold_date_sk#41), dynamicpruningexpression(ss_sold_date_sk#41 IN dynamicpruning#42)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] (25) Filter [codegen id : 4] -Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -Condition : isnotnull(ss_customer_sk#9) +Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] +Condition : isnotnull(ss_customer_sk#36) (26) BroadcastExchange -Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#9] +Left keys [1]: [c_customer_sk#28] +Right keys [1]: [ss_customer_sk#36] Join condition: None (28) Project [codegen id : 6] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Output [12]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] +Input [14]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#44, d_year#45] 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 (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#17, d_year#18] +Input [2]: [d_date_sk#44, d_year#45] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#17, d_year#18] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2002)) AND isnotnull(d_date_sk#17)) +Input [2]: [d_date_sk#44, d_year#45] +Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) (32) BroadcastExchange -Input [2]: [d_date_sk#17, d_year#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [2]: [d_date_sk#44, d_year#45] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#46] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#14] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ss_sold_date_sk#41] +Right keys [1]: [d_date_sk#44] Join condition: None (34) Project [codegen id : 6] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#18] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14, d_date_sk#17, d_year#18] +Output [12]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#45] +Input [14]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41, d_date_sk#44, d_year#45] (35) HashAggregate [codegen id : 6] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#18] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#31, isEmpty#32] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#33, isEmpty#34] +Input [12]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#45] +Keys [8]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#47, isEmpty#48] +Results [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, sum#49, isEmpty#50] (36) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#33, isEmpty#34] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, sum#49, isEmpty#50] +Arguments: hashpartitioning(c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, 5), ENSURE_REQUIREMENTS, [id=#51] (37) HashAggregate [codegen id : 7] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#33, isEmpty#34] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 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#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#36] -Results [8]: [c_customer_id#2 AS customer_id#37, c_first_name#3 AS customer_first_name#38, c_last_name#4 AS customer_last_name#39, c_preferred_cust_flag#5 AS customer_preferred_cust_flag#40, c_birth_country#6 AS customer_birth_country#41, c_login#7 AS customer_login#42, c_email_address#8 AS customer_email_address#43, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#36 AS year_total#44] +Input [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, sum#49, isEmpty#50] +Keys [8]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 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#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#52] +Results [8]: [c_customer_id#29 AS customer_id#53, c_first_name#30 AS customer_first_name#54, c_last_name#31 AS customer_last_name#55, c_preferred_cust_flag#32 AS customer_preferred_cust_flag#56, c_birth_country#33 AS customer_birth_country#57, c_login#34 AS customer_login#58, c_email_address#35 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#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#52 AS year_total#60] (38) BroadcastExchange -Input [8]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#45] +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: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#61] (39) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#37] +Right keys [1]: [customer_id#53] Join condition: None (40) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] (42) Filter [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] +Condition : (isnotnull(c_customer_sk#62) AND isnotnull(c_customer_id#63)) (43) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Output [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#51), dynamicpruningexpression(cs_sold_date_sk#51 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cs_sold_date_sk#75), dynamicpruningexpression(cs_sold_date_sk#75 IN dynamicpruning#15)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Input [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] (45) Filter [codegen id : 8] -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Condition : isnotnull(cs_bill_customer_sk#46) +Input [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] +Condition : isnotnull(cs_bill_customer_sk#70) (46) BroadcastExchange -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] +Input [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_bill_customer_sk#46] +Left keys [1]: [c_customer_sk#62] +Right keys [1]: [cs_bill_customer_sk#70] Join condition: None (48) Project [codegen id : 10] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] +Input [14]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#77, d_year#78] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#51] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [cs_sold_date_sk#75] +Right keys [1]: [d_date_sk#77] Join condition: None (51) Project [codegen id : 10] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51, d_date_sk#17, d_year#18] +Output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, d_year#78] +Input [14]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75, d_date_sk#77, d_year#78] (52) HashAggregate [codegen id : 10] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#53, isEmpty#54] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#55, isEmpty#56] +Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, d_year#78] +Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 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#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, sum#81, isEmpty#82] (53) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#55, isEmpty#56] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, sum#81, isEmpty#82] +Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#83] (54) HashAggregate [codegen id : 11] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#55, isEmpty#56] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 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#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#58] -Results [2]: [c_customer_id#2 AS customer_id#59, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#58 AS year_total#60] +Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, sum#81, isEmpty#82] +Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 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#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#84] +Results [2]: [c_customer_id#63 AS customer_id#85, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#84 AS year_total#86] (55) Filter [codegen id : 11] -Input [2]: [customer_id#59, year_total#60] -Condition : (isnotnull(year_total#60) AND (year_total#60 > 0.000000)) +Input [2]: [customer_id#85, year_total#86] +Condition : (isnotnull(year_total#86) AND (year_total#86 > 0.000000)) (56) Project [codegen id : 11] -Output [2]: [customer_id#59 AS customer_id#61, year_total#60 AS year_total#62] -Input [2]: [customer_id#59, year_total#60] +Output [2]: [customer_id#85 AS customer_id#87, year_total#86 AS year_total#88] +Input [2]: [customer_id#85, year_total#86] (57) BroadcastExchange -Input [2]: [customer_id#61, year_total#62] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#63] +Input [2]: [customer_id#87, year_total#88] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#89] (58) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#61] +Right keys [1]: [customer_id#87] Join condition: None (59) Project [codegen id : 24] -Output [11]: [customer_id#26, year_total#27, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44, year_total#62] -Input [12]: [customer_id#26, year_total#27, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44, customer_id#61, year_total#62] +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#88] +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#87, year_total#88] (60) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97] (62) Filter [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97] +Condition : (isnotnull(c_customer_sk#90) AND isnotnull(c_customer_id#91)) (63) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Output [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#51), dynamicpruningexpression(cs_sold_date_sk#51 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(cs_sold_date_sk#103), dynamicpruningexpression(cs_sold_date_sk#103 IN dynamicpruning#42)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Input [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] (65) Filter [codegen id : 12] -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Condition : isnotnull(cs_bill_customer_sk#46) +Input [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] +Condition : isnotnull(cs_bill_customer_sk#98) (66) BroadcastExchange -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] +Input [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#104] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_bill_customer_sk#46] +Left keys [1]: [c_customer_sk#90] +Right keys [1]: [cs_bill_customer_sk#98] Join condition: None (68) Project [codegen id : 14] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Output [12]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] +Input [14]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#105, d_year#106] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#51] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [cs_sold_date_sk#103] +Right keys [1]: [d_date_sk#105] Join condition: None (71) Project [codegen id : 14] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51, d_date_sk#17, d_year#18] +Output [12]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, d_year#106] +Input [14]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103, d_date_sk#105, d_year#106] (72) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#65, isEmpty#66] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#67, isEmpty#68] +Input [12]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, d_year#106] +Keys [8]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#107, isEmpty#108] +Results [10]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, sum#109, isEmpty#110] (73) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#67, isEmpty#68] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [10]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, sum#109, isEmpty#110] +Arguments: hashpartitioning(c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, 5), ENSURE_REQUIREMENTS, [id=#111] (74) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#67, isEmpty#68] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 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#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#70] -Results [2]: [c_customer_id#2 AS customer_id#71, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#70 AS year_total#72] +Input [10]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, sum#109, isEmpty#110] +Keys [8]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 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#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112] +Results [2]: [c_customer_id#91 AS customer_id#113, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112 AS year_total#114] (75) BroadcastExchange -Input [2]: [customer_id#71, year_total#72] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#73] +Input [2]: [customer_id#113, year_total#114] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#115] (76) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#71] -Join condition: (CASE WHEN (year_total#62 > 0.000000) THEN CheckOverflow((promote_precision(year_total#72) / promote_precision(year_total#62)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#27 > 0.000000) THEN CheckOverflow((promote_precision(year_total#44) / promote_precision(year_total#27)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#113] +Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), 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) (77) Project [codegen id : 24] -Output [10]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72] -Input [13]: [customer_id#26, year_total#27, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44, year_total#62, customer_id#71, year_total#72] +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#88, year_total#114] +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#88, customer_id#113, year_total#114] (78) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (79) ColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123] (80) Filter [codegen id : 18] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123] +Condition : (isnotnull(c_customer_sk#116) AND isnotnull(c_customer_id#117)) (81) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Output [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(ws_sold_date_sk#129), dynamicpruningexpression(ws_sold_date_sk#129 IN dynamicpruning#15)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (82) ColumnarToRow [codegen id : 16] -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Input [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] (83) Filter [codegen id : 16] -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Condition : isnotnull(ws_bill_customer_sk#74) +Input [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] +Condition : isnotnull(ws_bill_customer_sk#124) (84) BroadcastExchange -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#80] +Input [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#130] (85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#74] +Left keys [1]: [c_customer_sk#116] +Right keys [1]: [ws_bill_customer_sk#124] Join condition: None (86) Project [codegen id : 18] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Output [12]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] +Input [14]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] (87) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#131, d_year#132] (88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#79] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ws_sold_date_sk#129] +Right keys [1]: [d_date_sk#131] Join condition: None (89) Project [codegen id : 18] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#17, d_year#18] +Output [12]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, d_year#132] +Input [14]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129, d_date_sk#131, d_year#132] (90) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#81, isEmpty#82] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#83, isEmpty#84] +Input [12]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, d_year#132] +Keys [8]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#133, isEmpty#134] +Results [10]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, sum#135, isEmpty#136] (91) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#83, isEmpty#84] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [10]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, sum#135, isEmpty#136] +Arguments: hashpartitioning(c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, 5), ENSURE_REQUIREMENTS, [id=#137] (92) HashAggregate [codegen id : 19] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#83, isEmpty#84] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 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#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86] -Results [2]: [c_customer_id#2 AS customer_id#87, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86 AS year_total#88] +Input [10]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, sum#135, isEmpty#136] +Keys [8]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 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#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138] +Results [2]: [c_customer_id#117 AS customer_id#139, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138 AS year_total#140] (93) Filter [codegen id : 19] -Input [2]: [customer_id#87, year_total#88] -Condition : (isnotnull(year_total#88) AND (year_total#88 > 0.000000)) +Input [2]: [customer_id#139, year_total#140] +Condition : (isnotnull(year_total#140) AND (year_total#140 > 0.000000)) (94) Project [codegen id : 19] -Output [2]: [customer_id#87 AS customer_id#89, year_total#88 AS year_total#90] -Input [2]: [customer_id#87, year_total#88] +Output [2]: [customer_id#139 AS customer_id#141, year_total#140 AS year_total#142] +Input [2]: [customer_id#139, year_total#140] (95) BroadcastExchange -Input [2]: [customer_id#89, year_total#90] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#91] +Input [2]: [customer_id#141, year_total#142] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#143] (96) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#89] +Right keys [1]: [customer_id#141] Join condition: None (97) Project [codegen id : 24] -Output [11]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72, year_total#90] -Input [12]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72, customer_id#89, year_total#90] +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#88, year_total#114, year_total#142] +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#88, year_total#114, customer_id#141, year_total#142] (98) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (99) ColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] (100) Filter [codegen id : 22] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] +Condition : (isnotnull(c_customer_sk#144) AND isnotnull(c_customer_id#145)) (101) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Output [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(ws_sold_date_sk#157), dynamicpruningexpression(ws_sold_date_sk#157 IN dynamicpruning#42)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 20] -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] (103) Filter [codegen id : 20] -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Condition : isnotnull(ws_bill_customer_sk#74) +Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Condition : isnotnull(ws_bill_customer_sk#152) (104) BroadcastExchange -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] +Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#158] (105) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#74] +Left keys [1]: [c_customer_sk#144] +Right keys [1]: [ws_bill_customer_sk#152] Join condition: None (106) Project [codegen id : 22] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Output [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Input [14]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] (107) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#159, d_year#160] (108) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#79] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ws_sold_date_sk#157] +Right keys [1]: [d_date_sk#159] Join condition: None (109) Project [codegen id : 22] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#17, d_year#18] +Output [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, d_year#160] +Input [14]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157, d_date_sk#159, d_year#160] (110) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#93, isEmpty#94] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#95, isEmpty#96] +Input [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, d_year#160] +Keys [8]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#161, isEmpty#162] +Results [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] (111) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#95, isEmpty#96] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#97] +Input [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] +Arguments: hashpartitioning(c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, 5), ENSURE_REQUIREMENTS, [id=#165] (112) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#95, isEmpty#96] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 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#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#98] -Results [2]: [c_customer_id#2 AS customer_id#99, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#98 AS year_total#100] +Input [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] +Keys [8]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 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#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166] +Results [2]: [c_customer_id#145 AS customer_id#167, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166 AS year_total#168] (113) BroadcastExchange -Input [2]: [customer_id#99, year_total#100] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#101] +Input [2]: [customer_id#167, year_total#168] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#169] (114) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#99] -Join condition: (CASE WHEN (year_total#62 > 0.000000) THEN CheckOverflow((promote_precision(year_total#72) / promote_precision(year_total#62)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#90 > 0.000000) THEN CheckOverflow((promote_precision(year_total#100) / promote_precision(year_total#90)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#167] +Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#142 > 0.000000) THEN CheckOverflow((promote_precision(year_total#168) / promote_precision(year_total#142)), DecimalType(38,14), true) ELSE null END) (115) Project [codegen id : 24] -Output [7]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43] -Input [13]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72, year_total#90, customer_id#99, year_total#100] +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#88, year_total#114, year_total#142, customer_id#167, year_total#168] (116) TakeOrderedAndProject -Input [7]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43] -Arguments: 100, [customer_id#37 ASC NULLS FIRST, customer_first_name#38 ASC NULLS FIRST, customer_last_name#39 ASC NULLS FIRST, customer_preferred_cust_flag#40 ASC NULLS FIRST, customer_birth_country#41 ASC NULLS FIRST, customer_login#42 ASC NULLS FIRST, customer_email_address#43 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43] +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] ===== Subqueries ===== @@ -673,19 +673,19 @@ ReusedExchange (117) (117) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#17, d_year#18] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#28 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#41 IN dynamicpruning#42 ReusedExchange (118) (118) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#44, d_year#45] -Subquery:3 Hosting operator id = 43 Hosting Expression = cs_sold_date_sk#51 IN dynamicpruning#15 +Subquery:3 Hosting operator id = 43 Hosting Expression = cs_sold_date_sk#75 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 63 Hosting Expression = cs_sold_date_sk#51 IN dynamicpruning#28 +Subquery:4 Hosting operator id = 63 Hosting Expression = cs_sold_date_sk#103 IN dynamicpruning#42 -Subquery:5 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#129 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#28 +Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#157 IN dynamicpruning#42 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt index 35924d2ffd9a9..a6270c8ab533a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt @@ -102,98 +102,98 @@ Output [2]: [item_sk#11, rnk#17] Input [3]: [item_sk#11, rank_col#12, rnk#17] (15) ReusedExchange [Reuses operator id: 6] -Output [3]: [ss_item_sk#1, sum#18, count#19] +Output [3]: [ss_item_sk#18, sum#19, count#20] (16) HashAggregate [codegen id : 5] -Input [3]: [ss_item_sk#1, sum#18, count#19] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#20] -Results [3]: [ss_item_sk#1 AS item_sk#21, cast((avg(UnscaledValue(ss_net_profit#3))#20 / 100.0) as decimal(11,6)) AS rank_col#22, cast((avg(UnscaledValue(ss_net_profit#3))#20 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#3)#23] +Input [3]: [ss_item_sk#18, sum#19, count#20] +Keys [1]: [ss_item_sk#18] +Functions [1]: [avg(UnscaledValue(ss_net_profit#21))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#21))#22] +Results [3]: [ss_item_sk#18 AS item_sk#23, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS rank_col#24, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#21)#25] (17) Filter [codegen id : 5] -Input [3]: [item_sk#21, rank_col#22, avg(ss_net_profit#3)#23] -Condition : (isnotnull(avg(ss_net_profit#3)#23) AND (cast(avg(ss_net_profit#3)#23 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) +Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] +Condition : (isnotnull(avg(ss_net_profit#21)#25) AND (cast(avg(ss_net_profit#21)#25 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) (18) Project [codegen id : 5] -Output [2]: [item_sk#21, rank_col#22] -Input [3]: [item_sk#21, rank_col#22, avg(ss_net_profit#3)#23] +Output [2]: [item_sk#23, rank_col#24] +Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] (19) Exchange -Input [2]: [item_sk#21, rank_col#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [2]: [item_sk#23, rank_col#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] (20) Sort [codegen id : 6] -Input [2]: [item_sk#21, rank_col#22] -Arguments: [rank_col#22 DESC NULLS LAST], false, 0 +Input [2]: [item_sk#23, rank_col#24] +Arguments: [rank_col#24 DESC NULLS LAST], false, 0 (21) Window -Input [2]: [item_sk#21, rank_col#22] -Arguments: [rank(rank_col#22) windowspecdefinition(rank_col#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#25], [rank_col#22 DESC NULLS LAST] +Input [2]: [item_sk#23, rank_col#24] +Arguments: [rank(rank_col#24) windowspecdefinition(rank_col#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#27], [rank_col#24 DESC NULLS LAST] (22) Filter [codegen id : 7] -Input [3]: [item_sk#21, rank_col#22, rnk#25] -Condition : ((isnotnull(rnk#25) AND (rnk#25 < 11)) AND isnotnull(item_sk#21)) +Input [3]: [item_sk#23, rank_col#24, rnk#27] +Condition : ((isnotnull(rnk#27) AND (rnk#27 < 11)) AND isnotnull(item_sk#23)) (23) Project [codegen id : 7] -Output [2]: [item_sk#21, rnk#25] -Input [3]: [item_sk#21, rank_col#22, rnk#25] +Output [2]: [item_sk#23, rnk#27] +Input [3]: [item_sk#23, rank_col#24, rnk#27] (24) BroadcastExchange -Input [2]: [item_sk#21, rnk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#26] +Input [2]: [item_sk#23, rnk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#28] (25) BroadcastHashJoin [codegen id : 10] Left keys [1]: [rnk#17] -Right keys [1]: [rnk#25] +Right keys [1]: [rnk#27] Join condition: None (26) Project [codegen id : 10] -Output [3]: [item_sk#11, rnk#17, item_sk#21] -Input [4]: [item_sk#11, rnk#17, item_sk#21, rnk#25] +Output [3]: [item_sk#11, rnk#17, item_sk#23] +Input [4]: [item_sk#11, rnk#17, item_sk#23, rnk#27] (27) Scan parquet default.item -Output [2]: [i_item_sk#27, i_product_name#28] +Output [2]: [i_item_sk#29, i_product_name#30] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 8] -Input [2]: [i_item_sk#27, i_product_name#28] +Input [2]: [i_item_sk#29, i_product_name#30] (29) Filter [codegen id : 8] -Input [2]: [i_item_sk#27, i_product_name#28] -Condition : isnotnull(i_item_sk#27) +Input [2]: [i_item_sk#29, i_product_name#30] +Condition : isnotnull(i_item_sk#29) (30) BroadcastExchange -Input [2]: [i_item_sk#27, i_product_name#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [2]: [i_item_sk#29, i_product_name#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [item_sk#11] -Right keys [1]: [i_item_sk#27] +Right keys [1]: [i_item_sk#29] Join condition: None (32) Project [codegen id : 10] -Output [3]: [rnk#17, item_sk#21, i_product_name#28] -Input [5]: [item_sk#11, rnk#17, item_sk#21, i_item_sk#27, i_product_name#28] +Output [3]: [rnk#17, item_sk#23, i_product_name#30] +Input [5]: [item_sk#11, rnk#17, item_sk#23, i_item_sk#29, i_product_name#30] (33) ReusedExchange [Reuses operator id: 30] -Output [2]: [i_item_sk#30, i_product_name#31] +Output [2]: [i_item_sk#32, i_product_name#33] (34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [item_sk#21] -Right keys [1]: [i_item_sk#30] +Left keys [1]: [item_sk#23] +Right keys [1]: [i_item_sk#32] Join condition: None (35) Project [codegen id : 10] -Output [3]: [rnk#17, i_product_name#28 AS best_performing#32, i_product_name#31 AS worst_performing#33] -Input [5]: [rnk#17, item_sk#21, i_product_name#28, i_item_sk#30, i_product_name#31] +Output [3]: [rnk#17, i_product_name#30 AS best_performing#34, i_product_name#33 AS worst_performing#35] +Input [5]: [rnk#17, item_sk#23, i_product_name#30, i_item_sk#32, i_product_name#33] (36) TakeOrderedAndProject -Input [3]: [rnk#17, best_performing#32, worst_performing#33] -Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#32, worst_performing#33] +Input [3]: [rnk#17, best_performing#34, worst_performing#35] +Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#34, worst_performing#35] ===== Subqueries ===== @@ -208,40 +208,40 @@ Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery (37) Scan parquet default.store_sales -Output [4]: [ss_addr_sk#34, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Output [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 1] -Input [4]: [ss_addr_sk#34, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] (39) Filter [codegen id : 1] -Input [4]: [ss_addr_sk#34, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) AND isnull(ss_addr_sk#34)) +Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] +Condition : ((isnotnull(ss_store_sk#37) AND (ss_store_sk#37 = 4)) AND isnull(ss_addr_sk#36)) (40) Project [codegen id : 1] -Output [2]: [ss_store_sk#2, ss_net_profit#3] -Input [4]: [ss_addr_sk#34, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Output [2]: [ss_store_sk#37, ss_net_profit#38] +Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] (41) HashAggregate [codegen id : 1] -Input [2]: [ss_store_sk#2, ss_net_profit#3] -Keys [1]: [ss_store_sk#2] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#35, count#36] -Results [3]: [ss_store_sk#2, sum#37, count#38] +Input [2]: [ss_store_sk#37, ss_net_profit#38] +Keys [1]: [ss_store_sk#37] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#38))] +Aggregate Attributes [2]: [sum#40, count#41] +Results [3]: [ss_store_sk#37, sum#42, count#43] (42) Exchange -Input [3]: [ss_store_sk#2, sum#37, count#38] -Arguments: hashpartitioning(ss_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [3]: [ss_store_sk#37, sum#42, count#43] +Arguments: hashpartitioning(ss_store_sk#37, 5), ENSURE_REQUIREMENTS, [id=#44] (43) HashAggregate [codegen id : 2] -Input [3]: [ss_store_sk#2, sum#37, count#38] -Keys [1]: [ss_store_sk#2] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#40] -Results [1]: [cast((avg(UnscaledValue(ss_net_profit#3))#40 / 100.0) as decimal(11,6)) AS rank_col#41] +Input [3]: [ss_store_sk#37, sum#42, count#43] +Keys [1]: [ss_store_sk#37] +Functions [1]: [avg(UnscaledValue(ss_net_profit#38))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#38))#45] +Results [1]: [cast((avg(UnscaledValue(ss_net_profit#38))#45 / 100.0) as decimal(11,6)) AS rank_col#46] Subquery:2 Hosting operator id = 17 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index 4ca133ffd838d..a220cf9ff546f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -107,98 +107,98 @@ Input [2]: [item_sk#11, rnk#17] Arguments: [rnk#17 ASC NULLS FIRST], false, 0 (16) ReusedExchange [Reuses operator id: 6] -Output [3]: [ss_item_sk#1, sum#18, count#19] +Output [3]: [ss_item_sk#18, sum#19, count#20] (17) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#1, sum#18, count#19] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#20] -Results [3]: [ss_item_sk#1 AS item_sk#21, cast((avg(UnscaledValue(ss_net_profit#3))#20 / 100.0) as decimal(11,6)) AS rank_col#22, cast((avg(UnscaledValue(ss_net_profit#3))#20 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#3)#23] +Input [3]: [ss_item_sk#18, sum#19, count#20] +Keys [1]: [ss_item_sk#18] +Functions [1]: [avg(UnscaledValue(ss_net_profit#21))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#21))#22] +Results [3]: [ss_item_sk#18 AS item_sk#23, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS rank_col#24, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#21)#25] (18) Filter [codegen id : 6] -Input [3]: [item_sk#21, rank_col#22, avg(ss_net_profit#3)#23] -Condition : (isnotnull(avg(ss_net_profit#3)#23) AND (cast(avg(ss_net_profit#3)#23 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) +Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] +Condition : (isnotnull(avg(ss_net_profit#21)#25) AND (cast(avg(ss_net_profit#21)#25 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) (19) Project [codegen id : 6] -Output [2]: [item_sk#21, rank_col#22] -Input [3]: [item_sk#21, rank_col#22, avg(ss_net_profit#3)#23] +Output [2]: [item_sk#23, rank_col#24] +Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] (20) Exchange -Input [2]: [item_sk#21, rank_col#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [2]: [item_sk#23, rank_col#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] (21) Sort [codegen id : 7] -Input [2]: [item_sk#21, rank_col#22] -Arguments: [rank_col#22 DESC NULLS LAST], false, 0 +Input [2]: [item_sk#23, rank_col#24] +Arguments: [rank_col#24 DESC NULLS LAST], false, 0 (22) Window -Input [2]: [item_sk#21, rank_col#22] -Arguments: [rank(rank_col#22) windowspecdefinition(rank_col#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#25], [rank_col#22 DESC NULLS LAST] +Input [2]: [item_sk#23, rank_col#24] +Arguments: [rank(rank_col#24) windowspecdefinition(rank_col#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#27], [rank_col#24 DESC NULLS LAST] (23) Filter [codegen id : 8] -Input [3]: [item_sk#21, rank_col#22, rnk#25] -Condition : ((isnotnull(rnk#25) AND (rnk#25 < 11)) AND isnotnull(item_sk#21)) +Input [3]: [item_sk#23, rank_col#24, rnk#27] +Condition : ((isnotnull(rnk#27) AND (rnk#27 < 11)) AND isnotnull(item_sk#23)) (24) Project [codegen id : 8] -Output [2]: [item_sk#21, rnk#25] -Input [3]: [item_sk#21, rank_col#22, rnk#25] +Output [2]: [item_sk#23, rnk#27] +Input [3]: [item_sk#23, rank_col#24, rnk#27] (25) Sort [codegen id : 8] -Input [2]: [item_sk#21, rnk#25] -Arguments: [rnk#25 ASC NULLS FIRST], false, 0 +Input [2]: [item_sk#23, rnk#27] +Arguments: [rnk#27 ASC NULLS FIRST], false, 0 (26) SortMergeJoin [codegen id : 11] Left keys [1]: [rnk#17] -Right keys [1]: [rnk#25] +Right keys [1]: [rnk#27] Join condition: None (27) Project [codegen id : 11] -Output [3]: [item_sk#11, rnk#17, item_sk#21] -Input [4]: [item_sk#11, rnk#17, item_sk#21, rnk#25] +Output [3]: [item_sk#11, rnk#17, item_sk#23] +Input [4]: [item_sk#11, rnk#17, item_sk#23, rnk#27] (28) Scan parquet default.item -Output [2]: [i_item_sk#26, i_product_name#27] +Output [2]: [i_item_sk#28, i_product_name#29] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_sk#26, i_product_name#27] +Input [2]: [i_item_sk#28, i_product_name#29] (30) Filter [codegen id : 9] -Input [2]: [i_item_sk#26, i_product_name#27] -Condition : isnotnull(i_item_sk#26) +Input [2]: [i_item_sk#28, i_product_name#29] +Condition : isnotnull(i_item_sk#28) (31) BroadcastExchange -Input [2]: [i_item_sk#26, i_product_name#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] +Input [2]: [i_item_sk#28, i_product_name#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [item_sk#11] -Right keys [1]: [i_item_sk#26] +Right keys [1]: [i_item_sk#28] Join condition: None (33) Project [codegen id : 11] -Output [3]: [rnk#17, item_sk#21, i_product_name#27] -Input [5]: [item_sk#11, rnk#17, item_sk#21, i_item_sk#26, i_product_name#27] +Output [3]: [rnk#17, item_sk#23, i_product_name#29] +Input [5]: [item_sk#11, rnk#17, item_sk#23, i_item_sk#28, i_product_name#29] (34) ReusedExchange [Reuses operator id: 31] -Output [2]: [i_item_sk#29, i_product_name#30] +Output [2]: [i_item_sk#31, i_product_name#32] (35) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [item_sk#21] -Right keys [1]: [i_item_sk#29] +Left keys [1]: [item_sk#23] +Right keys [1]: [i_item_sk#31] Join condition: None (36) Project [codegen id : 11] -Output [3]: [rnk#17, i_product_name#27 AS best_performing#31, i_product_name#30 AS worst_performing#32] -Input [5]: [rnk#17, item_sk#21, i_product_name#27, i_item_sk#29, i_product_name#30] +Output [3]: [rnk#17, i_product_name#29 AS best_performing#33, i_product_name#32 AS worst_performing#34] +Input [5]: [rnk#17, item_sk#23, i_product_name#29, i_item_sk#31, i_product_name#32] (37) TakeOrderedAndProject -Input [3]: [rnk#17, best_performing#31, worst_performing#32] -Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#31, worst_performing#32] +Input [3]: [rnk#17, best_performing#33, worst_performing#34] +Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#33, worst_performing#34] ===== Subqueries ===== @@ -213,40 +213,40 @@ Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery (38) Scan parquet default.store_sales -Output [4]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Output [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 1] -Input [4]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] (40) Filter [codegen id : 1] -Input [4]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) AND isnull(ss_addr_sk#33)) +Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_store_sk#36) AND (ss_store_sk#36 = 4)) AND isnull(ss_addr_sk#35)) (41) Project [codegen id : 1] -Output [2]: [ss_store_sk#2, ss_net_profit#3] -Input [4]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Output [2]: [ss_store_sk#36, ss_net_profit#37] +Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] (42) HashAggregate [codegen id : 1] -Input [2]: [ss_store_sk#2, ss_net_profit#3] -Keys [1]: [ss_store_sk#2] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#34, count#35] -Results [3]: [ss_store_sk#2, sum#36, count#37] +Input [2]: [ss_store_sk#36, ss_net_profit#37] +Keys [1]: [ss_store_sk#36] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#37))] +Aggregate Attributes [2]: [sum#39, count#40] +Results [3]: [ss_store_sk#36, sum#41, count#42] (43) Exchange -Input [3]: [ss_store_sk#2, sum#36, count#37] -Arguments: hashpartitioning(ss_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [3]: [ss_store_sk#36, sum#41, count#42] +Arguments: hashpartitioning(ss_store_sk#36, 5), ENSURE_REQUIREMENTS, [id=#43] (44) HashAggregate [codegen id : 2] -Input [3]: [ss_store_sk#2, sum#36, count#37] -Keys [1]: [ss_store_sk#2] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#39] -Results [1]: [cast((avg(UnscaledValue(ss_net_profit#3))#39 / 100.0) as decimal(11,6)) AS rank_col#40] +Input [3]: [ss_store_sk#36, sum#41, count#42] +Keys [1]: [ss_store_sk#36] +Functions [1]: [avg(UnscaledValue(ss_net_profit#37))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#37))#44] +Results [1]: [cast((avg(UnscaledValue(ss_net_profit#37))#44 / 100.0) as decimal(11,6)) AS rank_col#45] Subquery:2 Hosting operator id = 18 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt index 22f746eb1d91a..701414b22eb80 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt @@ -199,30 +199,30 @@ Output [4]: [ws_sales_price#4, ca_city#19, ca_zip#20, i_item_id#12] Input [6]: [ws_bill_customer_sk#3, ws_sales_price#4, i_item_id#12, c_customer_sk#15, ca_city#19, ca_zip#20] (35) Scan parquet default.item -Output [2]: [i_item_sk#11, i_item_id#12] +Output [2]: [i_item_sk#23, i_item_id#24] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_item_sk, [2,3,5,7,11,13,17,19,23,29])] ReadSchema: struct (36) ColumnarToRow [codegen id : 11] -Input [2]: [i_item_sk#11, i_item_id#12] +Input [2]: [i_item_sk#23, i_item_id#24] (37) Filter [codegen id : 11] -Input [2]: [i_item_sk#11, i_item_id#12] -Condition : i_item_sk#11 IN (2,3,5,7,11,13,17,19,23,29) +Input [2]: [i_item_sk#23, i_item_id#24] +Condition : i_item_sk#23 IN (2,3,5,7,11,13,17,19,23,29) (38) Project [codegen id : 11] -Output [1]: [i_item_id#12 AS i_item_id#12#23] -Input [2]: [i_item_sk#11, i_item_id#12] +Output [1]: [i_item_id#24] +Input [2]: [i_item_sk#23, i_item_id#24] (39) BroadcastExchange -Input [1]: [i_item_id#12#23] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#24] +Input [1]: [i_item_id#24] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#25] (40) BroadcastHashJoin [codegen id : 12] Left keys [1]: [i_item_id#12] -Right keys [1]: [i_item_id#12#23] +Right keys [1]: [i_item_id#24] Join condition: None (41) Filter [codegen id : 12] @@ -237,23 +237,23 @@ Input [5]: [ws_sales_price#4, ca_city#19, ca_zip#20, i_item_id#12, exists#1] Input [3]: [ws_sales_price#4, ca_city#19, ca_zip#20] Keys [2]: [ca_zip#20, ca_city#19] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#25] -Results [3]: [ca_zip#20, ca_city#19, sum#26] +Aggregate Attributes [1]: [sum#26] +Results [3]: [ca_zip#20, ca_city#19, sum#27] (44) Exchange -Input [3]: [ca_zip#20, ca_city#19, sum#26] -Arguments: hashpartitioning(ca_zip#20, ca_city#19, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ca_zip#20, ca_city#19, sum#27] +Arguments: hashpartitioning(ca_zip#20, ca_city#19, 5), ENSURE_REQUIREMENTS, [id=#28] (45) HashAggregate [codegen id : 13] -Input [3]: [ca_zip#20, ca_city#19, sum#26] +Input [3]: [ca_zip#20, ca_city#19, sum#27] Keys [2]: [ca_zip#20, ca_city#19] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#28] -Results [3]: [ca_zip#20, ca_city#19, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#28,17,2) AS sum(ws_sales_price)#29] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#29] +Results [3]: [ca_zip#20, ca_city#19, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#29,17,2) AS sum(ws_sales_price)#30] (46) TakeOrderedAndProject -Input [3]: [ca_zip#20, ca_city#19, sum(ws_sales_price)#29] -Arguments: 100, [ca_zip#20 ASC NULLS FIRST, ca_city#19 ASC NULLS FIRST], [ca_zip#20, ca_city#19, sum(ws_sales_price)#29] +Input [3]: [ca_zip#20, ca_city#19, sum(ws_sales_price)#30] +Arguments: 100, [ca_zip#20 ASC NULLS FIRST, ca_city#19 ASC NULLS FIRST], [ca_zip#20, ca_city#19, sum(ws_sales_price)#30] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt index b81498ddcea34..f3a37f9e8767e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt @@ -169,30 +169,30 @@ Output [4]: [ws_sales_price#4, ca_city#11, ca_zip#12, i_item_id#19] Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#11, ca_zip#12, i_item_sk#18, i_item_id#19] (29) Scan parquet default.item -Output [2]: [i_item_sk#18, i_item_id#19] +Output [2]: [i_item_sk#21, i_item_id#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_item_sk, [2,3,5,7,11,13,17,19,23,29])] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] -Input [2]: [i_item_sk#18, i_item_id#19] +Input [2]: [i_item_sk#21, i_item_id#22] (31) Filter [codegen id : 5] -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : i_item_sk#18 IN (2,3,5,7,11,13,17,19,23,29) +Input [2]: [i_item_sk#21, i_item_id#22] +Condition : i_item_sk#21 IN (2,3,5,7,11,13,17,19,23,29) (32) Project [codegen id : 5] -Output [1]: [i_item_id#19 AS i_item_id#19#21] -Input [2]: [i_item_sk#18, i_item_id#19] +Output [1]: [i_item_id#22] +Input [2]: [i_item_sk#21, i_item_id#22] (33) BroadcastExchange -Input [1]: [i_item_id#19#21] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#22] +Input [1]: [i_item_id#22] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#23] (34) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_id#19] -Right keys [1]: [i_item_id#19#21] +Right keys [1]: [i_item_id#22] Join condition: None (35) Filter [codegen id : 6] @@ -207,23 +207,23 @@ Input [5]: [ws_sales_price#4, ca_city#11, ca_zip#12, i_item_id#19, exists#1] Input [3]: [ws_sales_price#4, ca_city#11, ca_zip#12] Keys [2]: [ca_zip#12, ca_city#11] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#23] -Results [3]: [ca_zip#12, ca_city#11, sum#24] +Aggregate Attributes [1]: [sum#24] +Results [3]: [ca_zip#12, ca_city#11, sum#25] (38) Exchange -Input [3]: [ca_zip#12, ca_city#11, sum#24] -Arguments: hashpartitioning(ca_zip#12, ca_city#11, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [ca_zip#12, ca_city#11, sum#25] +Arguments: hashpartitioning(ca_zip#12, ca_city#11, 5), ENSURE_REQUIREMENTS, [id=#26] (39) HashAggregate [codegen id : 7] -Input [3]: [ca_zip#12, ca_city#11, sum#24] +Input [3]: [ca_zip#12, ca_city#11, sum#25] Keys [2]: [ca_zip#12, ca_city#11] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#26] -Results [3]: [ca_zip#12, ca_city#11, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#26,17,2) AS sum(ws_sales_price)#27] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#27] +Results [3]: [ca_zip#12, ca_city#11, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#27,17,2) AS sum(ws_sales_price)#28] (40) TakeOrderedAndProject -Input [3]: [ca_zip#12, ca_city#11, sum(ws_sales_price)#27] -Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#11 ASC NULLS FIRST], [ca_zip#12, ca_city#11, sum(ws_sales_price)#27] +Input [3]: [ca_zip#12, ca_city#11, sum(ws_sales_price)#28] +Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#11 ASC NULLS FIRST], [ca_zip#12, ca_city#11, sum(ws_sales_price)#28] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt index 78ade994364a7..ac35640c7b004 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt @@ -230,55 +230,55 @@ Input [5]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt Arguments: [ss_addr_sk#12 ASC NULLS FIRST], false, 0 (41) ReusedExchange [Reuses operator id: 9] -Output [2]: [ca_address_sk#6, ca_city#7] +Output [2]: [ca_address_sk#31, ca_city#32] (42) Sort [codegen id : 13] -Input [2]: [ca_address_sk#6, ca_city#7] -Arguments: [ca_address_sk#6 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#31, ca_city#32] +Arguments: [ca_address_sk#31 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 14] Left keys [1]: [ss_addr_sk#12] -Right keys [1]: [ca_address_sk#6] +Right keys [1]: [ca_address_sk#31] Join condition: None (44) Project [codegen id : 14] -Output [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#7] -Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_address_sk#6, ca_city#7] +Output [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#32] +Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_address_sk#31, ca_city#32] (45) HashAggregate [codegen id : 14] -Input [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#7] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] +Input [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#32] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32] Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#15)), partial_sum(UnscaledValue(ss_net_profit#16))] -Aggregate Attributes [2]: [sum#31, sum#32] -Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#33, sum#34] +Aggregate Attributes [2]: [sum#33, sum#34] +Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32, sum#35, sum#36] (46) HashAggregate [codegen id : 14] -Input [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#33, sum#34] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] +Input [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32, sum#35, sum#36] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#15)), sum(UnscaledValue(ss_net_profit#16))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#15))#35, sum(UnscaledValue(ss_net_profit#16))#36] -Results [5]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#7 AS bought_city#37, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#15))#35,17,2) AS amt#38, MakeDecimal(sum(UnscaledValue(ss_net_profit#16))#36,17,2) AS profit#39] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#15))#37, sum(UnscaledValue(ss_net_profit#16))#38] +Results [5]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#32 AS bought_city#39, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#15))#37,17,2) AS amt#40, MakeDecimal(sum(UnscaledValue(ss_net_profit#16))#38,17,2) AS profit#41] (47) Exchange -Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#37, amt#38, profit#39] -Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#39, amt#40, profit#41] +Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#42] (48) Sort [codegen id : 15] -Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#37, amt#38, profit#39] +Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#39, amt#40, profit#41] Arguments: [ss_customer_sk#10 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 16] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#10] -Join condition: NOT (ca_city#7 = bought_city#37) +Join condition: NOT (ca_city#7 = bought_city#39) (50) Project [codegen id : 16] -Output [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#37, ss_ticket_number#14, amt#38, profit#39] -Input [9]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#37, amt#38, profit#39] +Output [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#39, ss_ticket_number#14, amt#40, profit#41] +Input [9]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#39, amt#40, profit#41] (51) TakeOrderedAndProject -Input [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#37, ss_ticket_number#14, amt#38, profit#39] -Arguments: 100, [c_last_name#4 ASC NULLS FIRST, c_first_name#3 ASC NULLS FIRST, ca_city#7 ASC NULLS FIRST, bought_city#37 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#37, ss_ticket_number#14, amt#38, profit#39] +Input [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#39, ss_ticket_number#14, amt#40, profit#41] +Arguments: 100, [c_last_name#4 ASC NULLS FIRST, c_first_name#3 ASC NULLS FIRST, ca_city#7 ASC NULLS FIRST, bought_city#39 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#39, ss_ticket_number#14, amt#40, profit#41] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt index e9bc22389d8a6..2532a14d7907f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt @@ -225,20 +225,20 @@ Output [7]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_ad Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#31, amt#32, profit#33, c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#21, ca_city#22] +Output [2]: [ca_address_sk#39, ca_city#40] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#21] -Join condition: NOT (ca_city#22 = bought_city#31) +Right keys [1]: [ca_address_sk#39] +Join condition: NOT (ca_city#40 = bought_city#31) (42) Project [codegen id : 8] -Output [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#21, ca_city#22] +Output [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#39, ca_city#40] (43) TakeOrderedAndProject -Input [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#22 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#40 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] ===== Subqueries ===== 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 dc19996f9957b..3a356e35707f7 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 @@ -213,7 +213,11 @@ Arguments: [rank(d_year#7, d_moy#8) windowspecdefinition(i_category#17, i_brand# (35) Filter [codegen id : 12] Input [9]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28] +<<<<<<< HEAD Condition : ((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#26)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#26 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +======= +Condition : (((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#26)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#26 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#28)) +>>>>>>> regen (36) Exchange Input [9]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28] @@ -229,22 +233,23 @@ Output [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_ye (39) HashAggregate [codegen id : 21] Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum#36] Keys [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35] -Functions [1]: [sum(UnscaledValue(ss_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#3))#37] -Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#3))#37,17,2) AS sum_sales#38] +Functions [1]: [sum(UnscaledValue(ss_sales_price#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#37))#38] +Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#37))#38,17,2) AS sum_sales#39] (40) Exchange -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#40] (41) Sort [codegen id : 22] -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST], false, 0 (42) Window -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +<<<<<<< HEAD (43) Project [codegen id : 23] Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38, rn#40] @@ -256,12 +261,30 @@ Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_compan (45) Sort [codegen id : 24] Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#40 + 1) ASC NULLS FIRST], false, 0 +======= +(43) Filter [codegen id : 23] +Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] +Condition : isnotnull(rn#41) + +(44) Project [codegen id : 23] +Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] + +(45) Exchange +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1), 5), ENSURE_REQUIREMENTS, [id=#42] + +(46) Sort [codegen id : 24] +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#41 + 1) ASC NULLS FIRST], false, 0 +>>>>>>> regen (46) SortMergeJoin [codegen id : 25] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1)] +Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1)] Join condition: None +<<<<<<< HEAD (47) Project [codegen id : 25] Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38] Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] @@ -288,12 +311,45 @@ Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_compan (53) Sort [codegen id : 36] Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (rn#49 - 1) ASC NULLS FIRST], false, 0 +======= +(48) Project [codegen id : 25] +Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39] +Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] + +(49) ReusedExchange [Reuses operator id: 40] +Output [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] + +(50) Sort [codegen id : 34] +Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] +Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST], false, 0 + +(51) Window +Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] +Arguments: [rank(d_year#47, d_moy#48) windowspecdefinition(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#50], [i_category#43, i_brand#44, s_store_name#45, s_company_name#46], [d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST] + +(52) Filter [codegen id : 35] +Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] +Condition : isnotnull(rn#50) + +(53) Project [codegen id : 35] +Output [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] + +(54) Exchange +Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Arguments: hashpartitioning(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1), 5), ENSURE_REQUIREMENTS, [id=#51] + +(55) Sort [codegen id : 36] +Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, (rn#50 - 1) ASC NULLS FIRST], false, 0 +>>>>>>> regen (54) SortMergeJoin [codegen id : 37] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1)] +Right keys [5]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1)] Join condition: None +<<<<<<< HEAD (55) Project [codegen id : 37] Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#38 AS psum#51, sum_sales#48 AS nsum#52] Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] @@ -301,6 +357,15 @@ Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_ye (56) TakeOrderedAndProject Input [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#11 ASC NULLS FIRST], [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] +======= +(57) Project [codegen id : 37] +Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#39 AS psum#52, sum_sales#49 AS nsum#53] +Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39, i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] + +(58) TakeOrderedAndProject +Input [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#11 ASC NULLS FIRST], [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] +>>>>>>> regen ===== Subqueries ===== 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 3d58d26e6a3a4..4a96c307dff2f 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 @@ -194,7 +194,11 @@ Arguments: [rank(d_year#11, d_moy#12) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [9]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27] +<<<<<<< HEAD Condition : ((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#25)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +======= +Condition : (((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#25)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#27)) +>>>>>>> regen (33) ReusedExchange [Reuses operator id: unknown] Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34] @@ -202,22 +206,23 @@ Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_ye (34) HashAggregate [codegen id : 13] 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#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#35] -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#6))#35,17,2) AS sum_sales#36] +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] (35) 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#36] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#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#37] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] (36) Sort [codegen id : 14] -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36] +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: [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 (37) 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#36] -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#38], [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#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] +<<<<<<< HEAD (38) Project [codegen id : 15] Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] @@ -225,12 +230,26 @@ Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_yea (39) BroadcastExchange Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] 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=#39] +======= +(38) Filter [codegen id : 15] +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] +Condition : isnotnull(rn#39) + +(39) Project [codegen id : 15] +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] + +(40) BroadcastExchange +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] + 1)),false), [id=#40] +>>>>>>> regen (40) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#38 + 1)] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] Join condition: None +<<<<<<< HEAD (41) Project [codegen id : 23] Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] @@ -253,12 +272,41 @@ Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_yea (46) BroadcastExchange Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] 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=#48] +======= +(42) Project [codegen id : 23] +Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] +Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] + +(43) ReusedExchange [Reuses operator id: 35] +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] + +(44) Sort [codegen id : 21] +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 + +(45) 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] + +(46) Filter [codegen id : 22] +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] +Condition : isnotnull(rn#48) + +(47) Project [codegen id : 22] +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] + +(48) BroadcastExchange +Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] - 1)),false), [id=#49] +>>>>>>> regen (47) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -Right keys [5]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#47 - 1)] +Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] Join condition: None +<<<<<<< HEAD (48) Project [codegen id : 23] Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#49, sum_sales#46 AS nsum#50] Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] @@ -266,6 +314,15 @@ Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year (49) TakeOrderedAndProject Input [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] +======= +(50) Project [codegen id : 23] +Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] +Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] + +(51) TakeOrderedAndProject +Input [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, psum#50, nsum#51] +>>>>>>> regen ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/explain.txt index bd8d979c32d17..2f87af685f2e5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/explain.txt @@ -79,7 +79,7 @@ Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_ Output [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree)),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College)))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree )),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree ))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College )))] ReadSchema: struct (11) ColumnarToRow [codegen id : 2] @@ -87,7 +87,7 @@ Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] (12) Filter [codegen id : 2] Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] -Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree)) OR ((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree))) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College)))) +Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree )) OR ((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree ))) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College )))) (13) BroadcastExchange Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] @@ -96,7 +96,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (14) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#11] -Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree)) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree)) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College)) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) +Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) (15) Project [codegen id : 5] Output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt index bd8d979c32d17..2f87af685f2e5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt @@ -79,7 +79,7 @@ Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_ Output [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree)),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College)))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree )),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree ))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College )))] ReadSchema: struct (11) ColumnarToRow [codegen id : 2] @@ -87,7 +87,7 @@ Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] (12) Filter [codegen id : 2] Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] -Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree)) OR ((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree))) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College)))) +Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree )) OR ((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree ))) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College )))) (13) BroadcastExchange Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] @@ -96,7 +96,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (14) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#11] -Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree)) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree)) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College)) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) +Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) (15) Project [codegen id : 5] Output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt index e93d0e9503493..3f30183af3a94 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt @@ -250,225 +250,225 @@ Output [5]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_net_profit#47, cs_sold_date_sk#48] (36) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#49] (37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_sold_date_sk#48] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#49] Join condition: None (38) Project [codegen id : 12] Output [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#8] +Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#49] (39) Exchange Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#49] +Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#50] (40) Sort [codegen id : 13] Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] Arguments: [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST], false, 0 (41) Scan parquet default.catalog_returns -Output [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Output [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (42) ColumnarToRow [codegen id : 14] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] (43) Filter [codegen id : 14] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] -Condition : (((isnotnull(cr_return_amount#53) AND (cr_return_amount#53 > 10000.00)) AND isnotnull(cr_order_number#51)) AND isnotnull(cr_item_sk#50)) +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Condition : (((isnotnull(cr_return_amount#54) AND (cr_return_amount#54 > 10000.00)) AND isnotnull(cr_order_number#52)) AND isnotnull(cr_item_sk#51)) (44) Project [codegen id : 14] -Output [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Output [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] (45) Exchange -Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Arguments: hashpartitioning(cr_order_number#52, cr_item_sk#51, 5), ENSURE_REQUIREMENTS, [id=#56] (46) Sort [codegen id : 15] -Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Arguments: [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Arguments: [cr_order_number#52 ASC NULLS FIRST, cr_item_sk#51 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 16] Left keys [2]: [cs_order_number#44, cs_item_sk#43] -Right keys [2]: [cr_order_number#51, cr_item_sk#50] +Right keys [2]: [cr_order_number#52, cr_item_sk#51] Join condition: None (48) Project [codegen id : 16] -Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] -Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] +Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] (49) HashAggregate [codegen id : 16] -Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] +Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] Keys [1]: [cs_item_sk#43] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#52, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#56, sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] -Results [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#53, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] +Results [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] (50) Exchange -Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] -Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#69] (51) HashAggregate [codegen id : 17] -Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] Keys [1]: [cs_item_sk#43] -Functions [4]: [sum(coalesce(cr_return_quantity#52, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#52, 0))#69, sum(coalesce(cs_quantity#45, 0))#70, sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72] -Results [3]: [cs_item_sk#43 AS item#73, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#52, 0))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#70 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#74, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#75] +Functions [4]: [sum(coalesce(cr_return_quantity#53, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#53, 0))#70, sum(coalesce(cs_quantity#45, 0))#71, sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73] +Results [3]: [cs_item_sk#43 AS item#74, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#53, 0))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#71 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#75, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#76] (52) Exchange -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#77] (53) Sort [codegen id : 18] -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: [return_ratio#74 ASC NULLS FIRST], false, 0 +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: [return_ratio#75 ASC NULLS FIRST], false, 0 (54) Window -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: [rank(return_ratio#74) windowspecdefinition(return_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#77], [return_ratio#74 ASC NULLS FIRST] +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: [rank(return_ratio#75) windowspecdefinition(return_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#78], [return_ratio#75 ASC NULLS FIRST] (55) Sort [codegen id : 19] -Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] -Arguments: [currency_ratio#75 ASC NULLS FIRST], false, 0 +Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] +Arguments: [currency_ratio#76 ASC NULLS FIRST], false, 0 (56) Window -Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] -Arguments: [rank(currency_ratio#75) windowspecdefinition(currency_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#78], [currency_ratio#75 ASC NULLS FIRST] +Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] +Arguments: [rank(currency_ratio#76) windowspecdefinition(currency_ratio#76 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#79], [currency_ratio#76 ASC NULLS FIRST] (57) Filter [codegen id : 20] -Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] -Condition : ((return_rank#77 <= 10) OR (currency_rank#78 <= 10)) +Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] +Condition : ((return_rank#78 <= 10) OR (currency_rank#79 <= 10)) (58) Project [codegen id : 20] -Output [5]: [catalog AS channel#79, item#73, return_ratio#74, return_rank#77, currency_rank#78] -Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] +Output [5]: [catalog AS channel#80, item#74, return_ratio#75, return_rank#78, currency_rank#79] +Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] (59) Scan parquet default.store_sales -Output [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Output [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_sold_date_sk#85 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 22] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] (61) Filter [codegen id : 22] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] -Condition : (((((((isnotnull(ss_net_profit#84) AND isnotnull(ss_net_paid#83)) AND isnotnull(ss_quantity#82)) AND (ss_net_profit#84 > 1.00)) AND (ss_net_paid#83 > 0.00)) AND (ss_quantity#82 > 0)) AND isnotnull(ss_ticket_number#81)) AND isnotnull(ss_item_sk#80)) +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Condition : (((((((isnotnull(ss_net_profit#85) AND isnotnull(ss_net_paid#84)) AND isnotnull(ss_quantity#83)) AND (ss_net_profit#85 > 1.00)) AND (ss_net_paid#84 > 0.00)) AND (ss_quantity#83 > 0)) AND isnotnull(ss_ticket_number#82)) AND isnotnull(ss_item_sk#81)) (62) Project [codegen id : 22] -Output [5]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Output [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] (63) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#87] (64) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ss_sold_date_sk#85] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ss_sold_date_sk#86] +Right keys [1]: [d_date_sk#87] Join condition: None (65) Project [codegen id : 22] -Output [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85, d_date_sk#8] +Output [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86, d_date_sk#87] (66) Exchange -Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Arguments: hashpartitioning(cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint), 5), ENSURE_REQUIREMENTS, [id=#86] +Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Arguments: hashpartitioning(cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint), 5), ENSURE_REQUIREMENTS, [id=#88] (67) Sort [codegen id : 23] -Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Arguments: [cast(ss_ticket_number#81 as bigint) ASC NULLS FIRST, cast(ss_item_sk#80 as bigint) ASC NULLS FIRST], false, 0 +Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Arguments: [cast(ss_ticket_number#82 as bigint) ASC NULLS FIRST, cast(ss_item_sk#81 as bigint) ASC NULLS FIRST], false, 0 (68) Scan parquet default.store_returns -Output [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Output [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 24] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] (70) Filter [codegen id : 24] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] -Condition : (((isnotnull(sr_return_amt#90) AND (sr_return_amt#90 > 10000.00)) AND isnotnull(sr_ticket_number#88)) AND isnotnull(sr_item_sk#87)) +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Condition : (((isnotnull(sr_return_amt#92) AND (sr_return_amt#92 > 10000.00)) AND isnotnull(sr_ticket_number#90)) AND isnotnull(sr_item_sk#89)) (71) Project [codegen id : 24] -Output [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Output [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] (72) Exchange -Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Arguments: hashpartitioning(sr_ticket_number#88, sr_item_sk#87, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Arguments: hashpartitioning(sr_ticket_number#90, sr_item_sk#89, 5), ENSURE_REQUIREMENTS, [id=#94] (73) Sort [codegen id : 25] -Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Arguments: [sr_ticket_number#88 ASC NULLS FIRST, sr_item_sk#87 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Arguments: [sr_ticket_number#90 ASC NULLS FIRST, sr_item_sk#89 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 26] -Left keys [2]: [cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint)] -Right keys [2]: [sr_ticket_number#88, sr_item_sk#87] +Left keys [2]: [cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint)] +Right keys [2]: [sr_ticket_number#90, sr_item_sk#89] Join condition: None (75) Project [codegen id : 26] -Output [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] -Input [8]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Output [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] +Input [8]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] (76) HashAggregate [codegen id : 26] -Input [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] -Keys [1]: [ss_item_sk#80] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#89, 0)), partial_sum(coalesce(ss_quantity#82, 0)), partial_sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#93, sum#94, sum#95, isEmpty#96, sum#97, isEmpty#98] -Results [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Input [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] +Keys [1]: [ss_item_sk#81] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#91, 0)), partial_sum(coalesce(ss_quantity#83, 0)), partial_sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#95, sum#96, sum#97, isEmpty#98, sum#99, isEmpty#100] +Results [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] (77) Exchange -Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Arguments: hashpartitioning(ss_item_sk#80, 5), ENSURE_REQUIREMENTS, [id=#105] +Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] +Arguments: hashpartitioning(ss_item_sk#81, 5), ENSURE_REQUIREMENTS, [id=#107] (78) HashAggregate [codegen id : 27] -Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Keys [1]: [ss_item_sk#80] -Functions [4]: [sum(coalesce(sr_return_quantity#89, 0)), sum(coalesce(ss_quantity#82, 0)), sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#89, 0))#106, sum(coalesce(ss_quantity#82, 0))#107, sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108, sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109] -Results [3]: [ss_item_sk#80 AS item#110, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#89, 0))#106 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#82, 0))#107 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#111, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#112] +Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] +Keys [1]: [ss_item_sk#81] +Functions [4]: [sum(coalesce(sr_return_quantity#91, 0)), sum(coalesce(ss_quantity#83, 0)), sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#91, 0))#108, sum(coalesce(ss_quantity#83, 0))#109, sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110, sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111] +Results [3]: [ss_item_sk#81 AS item#112, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#91, 0))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#83, 0))#109 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#113, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#114] (79) Exchange -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] (80) Sort [codegen id : 28] -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: [return_ratio#111 ASC NULLS FIRST], false, 0 +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: [return_ratio#113 ASC NULLS FIRST], false, 0 (81) Window -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: [rank(return_ratio#111) windowspecdefinition(return_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#114], [return_ratio#111 ASC NULLS FIRST] +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: [rank(return_ratio#113) windowspecdefinition(return_ratio#113 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#116], [return_ratio#113 ASC NULLS FIRST] (82) Sort [codegen id : 29] -Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] -Arguments: [currency_ratio#112 ASC NULLS FIRST], false, 0 +Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] +Arguments: [currency_ratio#114 ASC NULLS FIRST], false, 0 (83) Window -Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] -Arguments: [rank(currency_ratio#112) windowspecdefinition(currency_ratio#112 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#115], [currency_ratio#112 ASC NULLS FIRST] +Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] +Arguments: [rank(currency_ratio#114) windowspecdefinition(currency_ratio#114 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#117], [currency_ratio#114 ASC NULLS FIRST] (84) Filter [codegen id : 30] -Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] -Condition : ((return_rank#114 <= 10) OR (currency_rank#115 <= 10)) +Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] +Condition : ((return_rank#116 <= 10) OR (currency_rank#117 <= 10)) (85) Project [codegen id : 30] -Output [5]: [store AS channel#116, item#110, return_ratio#111, return_rank#114, currency_rank#115] -Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] +Output [5]: [store AS channel#118, item#112, return_ratio#113, return_rank#116, currency_rank#117] +Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] (86) Union @@ -481,7 +481,7 @@ Results [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_ran (88) Exchange Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] -Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#117] +Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#119] (89) HashAggregate [codegen id : 32] Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] @@ -505,6 +505,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#85 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index aeaca4deaae0c..a98f59f84be7d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -260,170 +260,170 @@ Output [6]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, Input [9]: [cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_item_sk#49, cr_order_number#50, cr_return_quantity#51, cr_return_amount#52] (40) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#54] (41) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#47] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#54] Join condition: None (42) Project [codegen id : 10] Output [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] -Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#14] +Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#54] (43) HashAggregate [codegen id : 10] Input [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] Keys [1]: [cs_item_sk#42] Functions [4]: [partial_sum(coalesce(cr_return_quantity#51, 0)), partial_sum(coalesce(cs_quantity#44, 0)), partial_sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Results [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] +Aggregate Attributes [6]: [sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Results [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] (44) Exchange -Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] -Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#67] (45) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] +Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] Keys [1]: [cs_item_sk#42] Functions [4]: [sum(coalesce(cr_return_quantity#51, 0)), sum(coalesce(cs_quantity#44, 0)), sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#67, sum(coalesce(cs_quantity#44, 0))#68, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70] -Results [3]: [cs_item_sk#42 AS item#71, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#67 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#68 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#72, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#73] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#68, sum(coalesce(cs_quantity#44, 0))#69, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71] +Results [3]: [cs_item_sk#42 AS item#72, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#68 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#69 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#73, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#74] (46) Exchange -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#74] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#75] (47) Sort [codegen id : 12] -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: [return_ratio#72 ASC NULLS FIRST], false, 0 +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [return_ratio#73 ASC NULLS FIRST], false, 0 (48) Window -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: [rank(return_ratio#72) windowspecdefinition(return_ratio#72 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#72 ASC NULLS FIRST] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#76], [return_ratio#73 ASC NULLS FIRST] (49) Sort [codegen id : 13] -Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] -Arguments: [currency_ratio#73 ASC NULLS FIRST], false, 0 +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 (50) Window -Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] -Arguments: [rank(currency_ratio#73) windowspecdefinition(currency_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#73 ASC NULLS FIRST] +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#77], [currency_ratio#74 ASC NULLS FIRST] (51) Filter [codegen id : 14] -Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] +Condition : ((return_rank#76 <= 10) OR (currency_rank#77 <= 10)) (52) Project [codegen id : 14] -Output [5]: [catalog AS channel#77, item#71, return_ratio#72, return_rank#75, currency_rank#76] -Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] +Output [5]: [catalog AS channel#78, item#72, return_ratio#73, return_rank#76, currency_rank#77] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] (53) Scan parquet default.store_sales -Output [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Output [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#83), dynamicpruningexpression(ss_sold_date_sk#83 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#84), dynamicpruningexpression(ss_sold_date_sk#84 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 15] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] (55) Filter [codegen id : 15] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] -Condition : (((((((isnotnull(ss_net_profit#82) AND isnotnull(ss_net_paid#81)) AND isnotnull(ss_quantity#80)) AND (ss_net_profit#82 > 1.00)) AND (ss_net_paid#81 > 0.00)) AND (ss_quantity#80 > 0)) AND isnotnull(ss_ticket_number#79)) AND isnotnull(ss_item_sk#78)) +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Condition : (((((((isnotnull(ss_net_profit#83) AND isnotnull(ss_net_paid#82)) AND isnotnull(ss_quantity#81)) AND (ss_net_profit#83 > 1.00)) AND (ss_net_paid#82 > 0.00)) AND (ss_quantity#81 > 0)) AND isnotnull(ss_ticket_number#80)) AND isnotnull(ss_item_sk#79)) (56) Project [codegen id : 15] -Output [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Output [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] (57) BroadcastExchange -Input [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#84] +Input [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#85] (58) Scan parquet default.store_returns -Output [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Output [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (59) ColumnarToRow -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] (60) Filter -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] -Condition : (((isnotnull(sr_return_amt#88) AND (sr_return_amt#88 > 10000.00)) AND isnotnull(sr_ticket_number#86)) AND isnotnull(sr_item_sk#85)) +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Condition : (((isnotnull(sr_return_amt#89) AND (sr_return_amt#89 > 10000.00)) AND isnotnull(sr_ticket_number#87)) AND isnotnull(sr_item_sk#86)) (61) Project -Output [4]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Output [4]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] (62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [cast(ss_ticket_number#79 as bigint), cast(ss_item_sk#78 as bigint)] -Right keys [2]: [sr_ticket_number#86, sr_item_sk#85] +Left keys [2]: [cast(ss_ticket_number#80 as bigint), cast(ss_item_sk#79 as bigint)] +Right keys [2]: [sr_ticket_number#87, sr_item_sk#86] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88] -Input [9]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] +Output [6]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89] +Input [9]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] (64) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#91] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#83] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ss_sold_date_sk#84] +Right keys [1]: [d_date_sk#91] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] -Input [7]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88, d_date_sk#14] +Output [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] +Input [7]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89, d_date_sk#91] (67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] -Keys [1]: [ss_item_sk#78] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#87, 0)), partial_sum(coalesce(ss_quantity#80, 0)), partial_sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Results [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Input [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] +Keys [1]: [ss_item_sk#79] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#88, 0)), partial_sum(coalesce(ss_quantity#81, 0)), partial_sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] +Results [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] (68) Exchange -Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Arguments: hashpartitioning(ss_item_sk#78, 5), ENSURE_REQUIREMENTS, [id=#102] +Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Arguments: hashpartitioning(ss_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#104] (69) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Keys [1]: [ss_item_sk#78] -Functions [4]: [sum(coalesce(sr_return_quantity#87, 0)), sum(coalesce(ss_quantity#80, 0)), sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#87, 0))#103, sum(coalesce(ss_quantity#80, 0))#104, sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105, sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106] -Results [3]: [ss_item_sk#78 AS item#107, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#87, 0))#103 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#80, 0))#104 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#108, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#109] +Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Keys [1]: [ss_item_sk#79] +Functions [4]: [sum(coalesce(sr_return_quantity#88, 0)), sum(coalesce(ss_quantity#81, 0)), sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#88, 0))#105, sum(coalesce(ss_quantity#81, 0))#106, sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107, sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108] +Results [3]: [ss_item_sk#79 AS item#109, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#88, 0))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#81, 0))#106 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#110, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#111] (70) Exchange -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#112] (71) Sort [codegen id : 19] -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: [return_ratio#108 ASC NULLS FIRST], false, 0 +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: [return_ratio#110 ASC NULLS FIRST], false, 0 (72) Window -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: [rank(return_ratio#108) windowspecdefinition(return_ratio#108 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#111], [return_ratio#108 ASC NULLS FIRST] +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: [rank(return_ratio#110) windowspecdefinition(return_ratio#110 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#113], [return_ratio#110 ASC NULLS FIRST] (73) Sort [codegen id : 20] -Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] -Arguments: [currency_ratio#109 ASC NULLS FIRST], false, 0 +Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] +Arguments: [currency_ratio#111 ASC NULLS FIRST], false, 0 (74) Window -Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] -Arguments: [rank(currency_ratio#109) windowspecdefinition(currency_ratio#109 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#112], [currency_ratio#109 ASC NULLS FIRST] +Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] +Arguments: [rank(currency_ratio#111) windowspecdefinition(currency_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#114], [currency_ratio#111 ASC NULLS FIRST] (75) Filter [codegen id : 21] -Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] -Condition : ((return_rank#111 <= 10) OR (currency_rank#112 <= 10)) +Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] +Condition : ((return_rank#113 <= 10) OR (currency_rank#114 <= 10)) (76) Project [codegen id : 21] -Output [5]: [store AS channel#113, item#107, return_ratio#108, return_rank#111, currency_rank#112] -Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] +Output [5]: [store AS channel#115, item#109, return_ratio#110, return_rank#113, currency_rank#114] +Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] (77) Union @@ -436,7 +436,7 @@ Results [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_ran (79) Exchange Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] -Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#114] +Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#116] (80) HashAggregate [codegen id : 23] Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] @@ -460,6 +460,6 @@ Output [1]: [d_date_sk#14] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#83 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#84 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index 2d3dea5d84565..28a457258eff7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -160,7 +160,7 @@ Input [2]: [d_date_sk#25, d_date#26] (18) Filter [codegen id : 4] Input [2]: [d_date_sk#25, d_date#26] -Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 11192)) AND (d_date#26 <= 11206)) AND isnotnull(d_date_sk#25)) +Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 2000-08-23)) AND (d_date#26 <= 2000-09-06)) AND isnotnull(d_date_sk#25)) (19) Project [codegen id : 4] Output [1]: [d_date_sk#25] @@ -265,193 +265,193 @@ Output [6]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, Input [8]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] (41) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#70] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#25] +Right keys [1]: [d_date_sk#70] Join condition: None (43) Project [codegen id : 11] Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#25] +Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#70] (44) HashAggregate [codegen id : 11] Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] -Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] +Results [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] (45) Exchange -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] +Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#79] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#83, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS RETURNS#84, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#85, catalog channel AS channel#86, concat(catalog_page, cp_catalog_page_id#68) AS id#87] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#84, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS RETURNS#85, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#86, catalog channel AS channel#87, concat(catalog_page, cp_catalog_page_id#68) AS id#88] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] -Condition : isnotnull(ws_web_site_sk#88) +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : isnotnull(ws_web_site_sk#89) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] (53) Exchange -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: hashpartitioning(wr_item_sk#98, wr_order_number#99, 5), ENSURE_REQUIREMENTS, [id=#103] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: hashpartitioning(wr_item_sk#99, wr_order_number#100, 5), ENSURE_REQUIREMENTS, [id=#104] (54) Sort [codegen id : 15] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: [wr_item_sk#98 ASC NULLS FIRST, wr_order_number#99 ASC NULLS FIRST], false, 0 +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: [wr_item_sk#99 ASC NULLS FIRST, wr_order_number#100 ASC NULLS FIRST], false, 0 (55) Scan parquet default.web_sales -Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 16] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (57) Filter [codegen id : 16] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) (58) Project [codegen id : 16] -Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (59) Exchange -Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Arguments: hashpartitioning(cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint), 5), ENSURE_REQUIREMENTS, [id=#106] +Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Arguments: hashpartitioning(cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint), 5), ENSURE_REQUIREMENTS, [id=#109] (60) Sort [codegen id : 17] -Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Arguments: [cast(ws_item_sk#104 as bigint) ASC NULLS FIRST, cast(ws_order_number#105 as bigint) ASC NULLS FIRST], false, 0 +Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Arguments: [cast(ws_item_sk#105 as bigint) ASC NULLS FIRST, cast(ws_order_number#107 as bigint) ASC NULLS FIRST], false, 0 (61) SortMergeJoin [codegen id : 18] -Left keys [2]: [wr_item_sk#98, wr_order_number#99] -Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] +Left keys [2]: [wr_item_sk#99, wr_order_number#100] +Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] Join condition: None (62) Project [codegen id : 18] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#107, wr_returned_date_sk#102 AS date_sk#108, 0.00 AS sales_price#109, 0.00 AS profit#110, wr_return_amt#100 AS return_amt#111, wr_net_loss#101 AS net_loss#112] -Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#110, wr_returned_date_sk#103 AS date_sk#111, 0.00 AS sales_price#112, 0.00 AS profit#113, wr_return_amt#101 AS return_amt#114, wr_net_loss#102 AS net_loss#115] +Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] (63) Union (64) Scan parquet default.web_site -Output [2]: [web_site_sk#113, web_site_id#114] +Output [2]: [web_site_sk#116, web_site_id#117] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 19] -Input [2]: [web_site_sk#113, web_site_id#114] +Input [2]: [web_site_sk#116, web_site_id#117] (66) Filter [codegen id : 19] -Input [2]: [web_site_sk#113, web_site_id#114] -Condition : isnotnull(web_site_sk#113) +Input [2]: [web_site_sk#116, web_site_id#117] +Condition : isnotnull(web_site_sk#116) (67) BroadcastExchange -Input [2]: [web_site_sk#113, web_site_id#114] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] +Input [2]: [web_site_sk#116, web_site_id#117] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] (68) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [wsr_web_site_sk#92] -Right keys [1]: [web_site_sk#113] +Left keys [1]: [wsr_web_site_sk#93] +Right keys [1]: [web_site_sk#116] Join condition: None (69) Project [codegen id : 21] -Output [6]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Input [8]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#113, web_site_id#114] +Output [6]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [8]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] (70) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#119] (71) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [date_sk#93] -Right keys [1]: [cast(d_date_sk#25 as bigint)] +Left keys [1]: [date_sk#94] +Right keys [1]: [cast(d_date_sk#119 as bigint)] Join condition: None (72) Project [codegen id : 21] -Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Input [7]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114, d_date_sk#25] +Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [7]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117, d_date_sk#119] (73) HashAggregate [codegen id : 21] -Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Keys [1]: [web_site_id#114] -Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] -Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] +Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Keys [1]: [web_site_id#117] +Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum#120, sum#121, sum#122, sum#123] +Results [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] (74) Exchange -Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] +Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#128] (75) HashAggregate [codegen id : 22] -Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -Keys [1]: [web_site_id#114] -Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#125, sum(UnscaledValue(return_amt#96))#126, sum(UnscaledValue(profit#95))#127, sum(UnscaledValue(net_loss#97))#128] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#94))#125,17,2) AS sales#129, MakeDecimal(sum(UnscaledValue(return_amt#96))#126,17,2) AS RETURNS#130, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#131, web channel AS channel#132, concat(web_site, web_site_id#114) AS id#133] +Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] +Keys [1]: [web_site_id#117] +Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#129, sum(UnscaledValue(return_amt#97))#130, sum(UnscaledValue(profit#96))#131, sum(UnscaledValue(net_loss#98))#132] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#95))#129,17,2) AS sales#133, MakeDecimal(sum(UnscaledValue(return_amt#97))#130,17,2) AS RETURNS#134, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#131,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#132,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#135, web channel AS channel#136, concat(web_site, web_site_id#117) AS id#137] (76) Union (77) Expand [codegen id : 23] Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45] -Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#134, id#135, spark_grouping_id#136] +Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#138, id#139, spark_grouping_id#140] (78) HashAggregate [codegen id : 23] -Input [6]: [sales#41, returns#42, profit#43, channel#134, id#135, spark_grouping_id#136] -Keys [3]: [channel#134, id#135, spark_grouping_id#136] +Input [6]: [sales#41, returns#42, profit#43, channel#138, id#139, spark_grouping_id#140] +Keys [3]: [channel#138, id#139, spark_grouping_id#140] Functions [3]: [partial_sum(sales#41), partial_sum(returns#42), partial_sum(profit#43)] -Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] -Results [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Aggregate Attributes [6]: [sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Results [9]: [channel#138, id#139, spark_grouping_id#140, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] (79) Exchange -Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -Arguments: hashpartitioning(channel#134, id#135, spark_grouping_id#136, 5), ENSURE_REQUIREMENTS, [id=#149] +Input [9]: [channel#138, id#139, spark_grouping_id#140, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] +Arguments: hashpartitioning(channel#138, id#139, spark_grouping_id#140, 5), ENSURE_REQUIREMENTS, [id=#153] (80) HashAggregate [codegen id : 24] -Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -Keys [3]: [channel#134, id#135, spark_grouping_id#136] +Input [9]: [channel#138, id#139, spark_grouping_id#140, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] +Keys [3]: [channel#138, id#139, spark_grouping_id#140] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] -Aggregate Attributes [3]: [sum(sales#41)#150, sum(returns#42)#151, sum(profit#43)#152] -Results [5]: [channel#134, id#135, sum(sales#41)#150 AS sales#153, sum(returns#42)#151 AS returns#154, sum(profit#43)#152 AS profit#155] +Aggregate Attributes [3]: [sum(sales#41)#154, sum(returns#42)#155, sum(profit#43)#156] +Results [5]: [channel#138, id#139, sum(sales#41)#154 AS sales#157, sum(returns#42)#155 AS returns#158, sum(profit#43)#156 AS profit#159] (81) TakeOrderedAndProject -Input [5]: [channel#134, id#135, sales#153, returns#154, profit#155] -Arguments: 100, [channel#134 ASC NULLS FIRST, id#135 ASC NULLS FIRST], [channel#134, id#135, sales#153, returns#154, profit#155] +Input [5]: [channel#138, id#139, sales#157, returns#158, profit#159] +Arguments: 100, [channel#138 ASC NULLS FIRST, id#139 ASC NULLS FIRST], [channel#138, id#139, sales#157, returns#158, profit#159] ===== Subqueries ===== @@ -469,12 +469,12 @@ ReusedExchange (83) (83) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#70] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index f6fe8bee89369..cb130ce17795a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -130,7 +130,7 @@ Input [2]: [d_date_sk#22, d_date#23] (12) Filter [codegen id : 3] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11192)) AND (d_date#23 <= 11206)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) (13) Project [codegen id : 3] Output [1]: [d_date_sk#22] @@ -235,208 +235,208 @@ Input [4]: [cr_catalog_page_sk#57, cr_return_amount#58, cr_net_loss#59, cr_retur (34) Union (35) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#67] (36) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#67] Join condition: None (37) Project [codegen id : 11] Output [5]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56] -Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#22] +Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#67] (38) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Output [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] (40) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] -Condition : isnotnull(cp_catalog_page_sk#67) +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Condition : isnotnull(cp_catalog_page_sk#68) (41) BroadcastExchange -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#69] +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#70] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [page_sk#51] -Right keys [1]: [cp_catalog_page_sk#67] +Right keys [1]: [cp_catalog_page_sk#68] Join condition: None (43) Project [codegen id : 11] -Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] +Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] +Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#68, cp_catalog_page_id#69] (44) HashAggregate [codegen id : 11] -Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Keys [1]: [cp_catalog_page_id#68] +Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] +Keys [1]: [cp_catalog_page_id#69] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] -Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] +Results [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] (45) Exchange -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] +Arguments: hashpartitioning(cp_catalog_page_id#69, 5), ENSURE_REQUIREMENTS, [id=#79] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Keys [1]: [cp_catalog_page_id#68] +Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] +Keys [1]: [cp_catalog_page_id#69] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#83, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS RETURNS#84, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#85, catalog channel AS channel#86, concat(catalog_page, cp_catalog_page_id#68) AS id#87] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#84, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS RETURNS#85, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#86, catalog channel AS channel#87, concat(catalog_page, cp_catalog_page_id#69) AS id#88] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] -Condition : isnotnull(ws_web_site_sk#88) +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : isnotnull(ws_web_site_sk#89) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] (53) BroadcastExchange -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#103] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#104] (54) Scan parquet default.web_sales -Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (55) ColumnarToRow -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (56) Filter -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) (57) Project -Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#98, wr_order_number#99] -Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] +Left keys [2]: [wr_item_sk#99, wr_order_number#100] +Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] Join condition: None (59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#106, wr_returned_date_sk#102 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#100 AS return_amt#110, wr_net_loss#101 AS net_loss#111] -Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#109, wr_returned_date_sk#103 AS date_sk#110, 0.00 AS sales_price#111, 0.00 AS profit#112, wr_return_amt#101 AS return_amt#113, wr_net_loss#102 AS net_loss#114] +Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] (60) Union (61) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#115] (62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#93] -Right keys [1]: [cast(d_date_sk#22 as bigint)] +Left keys [1]: [date_sk#94] +Right keys [1]: [cast(d_date_sk#115 as bigint)] Join condition: None (63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97] -Input [7]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, d_date_sk#22] +Output [5]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98] +Input [7]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, d_date_sk#115] (64) Scan parquet default.web_site -Output [2]: [web_site_sk#112, web_site_id#113] +Output [2]: [web_site_sk#116, web_site_id#117] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#112, web_site_id#113] +Input [2]: [web_site_sk#116, web_site_id#117] (66) Filter [codegen id : 17] -Input [2]: [web_site_sk#112, web_site_id#113] -Condition : isnotnull(web_site_sk#112) +Input [2]: [web_site_sk#116, web_site_id#117] +Condition : isnotnull(web_site_sk#116) (67) BroadcastExchange -Input [2]: [web_site_sk#112, web_site_id#113] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#114] +Input [2]: [web_site_sk#116, web_site_id#117] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] (68) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#92] -Right keys [1]: [web_site_sk#112] +Left keys [1]: [wsr_web_site_sk#93] +Right keys [1]: [web_site_sk#116] Join condition: None (69) Project [codegen id : 18] -Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] -Input [7]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#112, web_site_id#113] +Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [7]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] (70) HashAggregate [codegen id : 18] -Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] -Keys [1]: [web_site_id#113] -Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] -Results [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] +Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Keys [1]: [web_site_id#117] +Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum#119, sum#120, sum#121, sum#122] +Results [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] (71) Exchange -Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -Arguments: hashpartitioning(web_site_id#113, 5), ENSURE_REQUIREMENTS, [id=#123] +Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] +Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#127] (72) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -Keys [1]: [web_site_id#113] -Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#124, sum(UnscaledValue(return_amt#96))#125, sum(UnscaledValue(profit#95))#126, sum(UnscaledValue(net_loss#97))#127] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#94))#124,17,2) AS sales#128, MakeDecimal(sum(UnscaledValue(return_amt#96))#125,17,2) AS RETURNS#129, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#126,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#127,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#130, web channel AS channel#131, concat(web_site, web_site_id#113) AS id#132] +Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] +Keys [1]: [web_site_id#117] +Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#128, sum(UnscaledValue(return_amt#97))#129, sum(UnscaledValue(profit#96))#130, sum(UnscaledValue(net_loss#98))#131] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#95))#128,17,2) AS sales#132, MakeDecimal(sum(UnscaledValue(return_amt#97))#129,17,2) AS RETURNS#133, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#130,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#131,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#134, web channel AS channel#135, concat(web_site, web_site_id#117) AS id#136] (73) Union (74) Expand [codegen id : 20] Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45] -Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#133, id#134, spark_grouping_id#135] +Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#137, id#138, spark_grouping_id#139] (75) HashAggregate [codegen id : 20] -Input [6]: [sales#41, returns#42, profit#43, channel#133, id#134, spark_grouping_id#135] -Keys [3]: [channel#133, id#134, spark_grouping_id#135] +Input [6]: [sales#41, returns#42, profit#43, channel#137, id#138, spark_grouping_id#139] +Keys [3]: [channel#137, id#138, spark_grouping_id#139] Functions [3]: [partial_sum(sales#41), partial_sum(returns#42), partial_sum(profit#43)] -Aggregate Attributes [6]: [sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Results [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Results [9]: [channel#137, id#138, spark_grouping_id#139, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] (76) Exchange -Input [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -Arguments: hashpartitioning(channel#133, id#134, spark_grouping_id#135, 5), ENSURE_REQUIREMENTS, [id=#148] +Input [9]: [channel#137, id#138, spark_grouping_id#139, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Arguments: hashpartitioning(channel#137, id#138, spark_grouping_id#139, 5), ENSURE_REQUIREMENTS, [id=#152] (77) HashAggregate [codegen id : 21] -Input [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -Keys [3]: [channel#133, id#134, spark_grouping_id#135] +Input [9]: [channel#137, id#138, spark_grouping_id#139, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Keys [3]: [channel#137, id#138, spark_grouping_id#139] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] -Aggregate Attributes [3]: [sum(sales#41)#149, sum(returns#42)#150, sum(profit#43)#151] -Results [5]: [channel#133, id#134, sum(sales#41)#149 AS sales#152, sum(returns#42)#150 AS returns#153, sum(profit#43)#151 AS profit#154] +Aggregate Attributes [3]: [sum(sales#41)#153, sum(returns#42)#154, sum(profit#43)#155] +Results [5]: [channel#137, id#138, sum(sales#41)#153 AS sales#156, sum(returns#42)#154 AS returns#157, sum(profit#43)#155 AS profit#158] (78) TakeOrderedAndProject -Input [5]: [channel#133, id#134, sales#152, returns#153, profit#154] -Arguments: 100, [channel#133 ASC NULLS FIRST, id#134 ASC NULLS FIRST], [channel#133, id#134, sales#152, returns#153, profit#154] +Input [5]: [channel#137, id#138, sales#156, returns#157, profit#158] +Arguments: 100, [channel#137 ASC NULLS FIRST, id#138 ASC NULLS FIRST], [channel#137, id#138, sales#156, returns#157, profit#158] ===== Subqueries ===== @@ -454,12 +454,12 @@ ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#67] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt index 1c18ab278209b..e0cd0729e0e32 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] (3) Filter [codegen id : 1] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 1] Output [2]: [i_item_sk#1, i_manufact_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (30) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#26, _w0#27, avg_quarterly_sales#29] -Condition : ((isnotnull(avg_quarterly_sales#29) AND (avg_quarterly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_quarterly_sales#29) AND ((avg_quarterly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (31) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#26, avg_quarterly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt index 3c41db90ad1aa..3bf06577e3212 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] (3) Filter [codegen id : 4] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 4] Output [2]: [i_item_sk#1, i_manufact_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (30) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#26, _w0#27, avg_quarterly_sales#29] -Condition : ((isnotnull(avg_quarterly_sales#29) AND (avg_quarterly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_quarterly_sales#29) AND ((avg_quarterly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (31) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#26, avg_quarterly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt index c8058eeee7ca1..b149bdd3e1e3f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt @@ -188,7 +188,7 @@ Input [4]: [sold_date_sk#12, customer_sk#13, item_sk#14, d_date_sk#21] Output [3]: [i_item_sk#25, i_class#26, i_category#27] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women), EqualTo(i_class,maternity), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women ), EqualTo(i_class,maternity ), IsNotNull(i_item_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 6] @@ -196,7 +196,7 @@ Input [3]: [i_item_sk#25, i_class#26, i_category#27] (29) Filter [codegen id : 6] Input [3]: [i_item_sk#25, i_class#26, i_category#27] -Condition : ((((isnotnull(i_category#27) AND isnotnull(i_class#26)) AND (i_category#27 = Women)) AND (i_class#26 = maternity)) AND isnotnull(i_item_sk#25)) +Condition : ((((isnotnull(i_category#27) AND isnotnull(i_class#26)) AND (i_category#27 = Women )) AND (i_class#26 = maternity )) AND isnotnull(i_item_sk#25)) (30) Project [codegen id : 6] Output [1]: [i_item_sk#25] @@ -297,39 +297,39 @@ Input [3]: [ss_customer_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] Condition : isnotnull(ss_customer_sk#33) (51) Scan parquet default.date_dim -Output [2]: [d_date_sk#21, d_month_seq#37] +Output [2]: [d_date_sk#37, d_month_seq#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#21, d_month_seq#37] +Input [2]: [d_date_sk#37, d_month_seq#38] (53) Filter [codegen id : 12] -Input [2]: [d_date_sk#21, d_month_seq#37] -Condition : (((isnotnull(d_month_seq#37) AND (d_month_seq#37 >= Subquery scalar-subquery#38, [id=#39])) AND (d_month_seq#37 <= Subquery scalar-subquery#40, [id=#41])) AND isnotnull(d_date_sk#21)) +Input [2]: [d_date_sk#37, d_month_seq#38] +Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= Subquery scalar-subquery#39, [id=#40])) AND (d_month_seq#38 <= Subquery scalar-subquery#41, [id=#42])) AND isnotnull(d_date_sk#37)) (54) Project [codegen id : 12] -Output [1]: [d_date_sk#21] -Input [2]: [d_date_sk#21, d_month_seq#37] +Output [1]: [d_date_sk#37] +Input [2]: [d_date_sk#37, d_month_seq#38] (55) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [d_date_sk#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#43] (56) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ss_sold_date_sk#35] -Right keys [1]: [d_date_sk#21] +Right keys [1]: [d_date_sk#37] Join condition: None (57) Project [codegen id : 13] Output [2]: [ss_customer_sk#33, ss_ext_sales_price#34] -Input [4]: [ss_customer_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35, d_date_sk#21] +Input [4]: [ss_customer_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35, d_date_sk#37] (58) Exchange Input [2]: [ss_customer_sk#33, ss_ext_sales_price#34] -Arguments: hashpartitioning(ss_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#43] +Arguments: hashpartitioning(ss_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#44] (59) Sort [codegen id : 14] Input [2]: [ss_customer_sk#33, ss_ext_sales_price#34] @@ -348,37 +348,37 @@ Input [3]: [c_customer_sk#30, ss_customer_sk#33, ss_ext_sales_price#34] Input [2]: [c_customer_sk#30, ss_ext_sales_price#34] Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#34))] -Aggregate Attributes [1]: [sum#44] -Results [2]: [c_customer_sk#30, sum#45] +Aggregate Attributes [1]: [sum#45] +Results [2]: [c_customer_sk#30, sum#46] (63) HashAggregate [codegen id : 15] -Input [2]: [c_customer_sk#30, sum#45] +Input [2]: [c_customer_sk#30, sum#46] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#34))#46] -Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#46,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#47] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#34))#47] +Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#47,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#48] (64) HashAggregate [codegen id : 15] -Input [1]: [segment#47] -Keys [1]: [segment#47] +Input [1]: [segment#48] +Keys [1]: [segment#48] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#48] -Results [2]: [segment#47, count#49] +Aggregate Attributes [1]: [count#49] +Results [2]: [segment#48, count#50] (65) Exchange -Input [2]: [segment#47, count#49] -Arguments: hashpartitioning(segment#47, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [segment#48, count#50] +Arguments: hashpartitioning(segment#48, 5), ENSURE_REQUIREMENTS, [id=#51] (66) HashAggregate [codegen id : 16] -Input [2]: [segment#47, count#49] -Keys [1]: [segment#47] +Input [2]: [segment#48, count#50] +Keys [1]: [segment#48] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#51] -Results [3]: [segment#47, count(1)#51 AS num_customers#52, (segment#47 * 50) AS segment_base#53] +Aggregate Attributes [1]: [count(1)#52] +Results [3]: [segment#48, count(1)#52 AS num_customers#53, (segment#48 * 50) AS segment_base#54] (67) TakeOrderedAndProject -Input [3]: [segment#47, num_customers#52, segment_base#53] -Arguments: 100, [segment#47 ASC NULLS FIRST, num_customers#52 ASC NULLS FIRST], [segment#47, num_customers#52, segment_base#53] +Input [3]: [segment#48, num_customers#53, segment_base#54] +Arguments: 100, [segment#48 ASC NULLS FIRST, num_customers#53 ASC NULLS FIRST], [segment#48, num_customers#53, segment_base#54] ===== Subqueries ===== @@ -396,9 +396,9 @@ ReusedExchange (69) (69) ReusedExchange [Reuses operator id: 55] -Output [1]: [d_date_sk#21] +Output [1]: [d_date_sk#37] -Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#39, [id=#40] * HashAggregate (76) +- Exchange (75) +- * HashAggregate (74) @@ -409,42 +409,42 @@ Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (70) Scan parquet default.date_dim -Output [3]: [d_month_seq#37, d_year#22, d_moy#23] +Output [3]: [d_month_seq#55, 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,1998), EqualTo(d_moy,12)] ReadSchema: struct (71) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] +Input [3]: [d_month_seq#55, d_year#56, d_moy#57] (72) Filter [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] -Condition : (((isnotnull(d_year#22) AND isnotnull(d_moy#23)) AND (d_year#22 = 1998)) AND (d_moy#23 = 12)) +Input [3]: [d_month_seq#55, d_year#56, d_moy#57] +Condition : (((isnotnull(d_year#56) AND isnotnull(d_moy#57)) AND (d_year#56 = 1998)) AND (d_moy#57 = 12)) (73) Project [codegen id : 1] -Output [1]: [(d_month_seq#37 + 1) AS (d_month_seq + 1)#54] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] +Output [1]: [(d_month_seq#55 + 1) AS (d_month_seq + 1)#58] +Input [3]: [d_month_seq#55, d_year#56, d_moy#57] (74) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 1)#54] -Keys [1]: [(d_month_seq + 1)#54] +Input [1]: [(d_month_seq + 1)#58] +Keys [1]: [(d_month_seq + 1)#58] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#54] +Results [1]: [(d_month_seq + 1)#58] (75) Exchange -Input [1]: [(d_month_seq + 1)#54] -Arguments: hashpartitioning((d_month_seq + 1)#54, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [1]: [(d_month_seq + 1)#58] +Arguments: hashpartitioning((d_month_seq + 1)#58, 5), ENSURE_REQUIREMENTS, [id=#59] (76) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#54] -Keys [1]: [(d_month_seq + 1)#54] +Input [1]: [(d_month_seq + 1)#58] +Keys [1]: [(d_month_seq + 1)#58] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#54] +Results [1]: [(d_month_seq + 1)#58] -Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#40, [id=#41] +Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (83) +- Exchange (82) +- * HashAggregate (81) @@ -455,39 +455,39 @@ Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (77) Scan parquet default.date_dim -Output [3]: [d_month_seq#37, d_year#22, d_moy#23] +Output [3]: [d_month_seq#60, d_year#61, d_moy#62] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (78) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] +Input [3]: [d_month_seq#60, d_year#61, d_moy#62] (79) Filter [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] -Condition : (((isnotnull(d_year#22) AND isnotnull(d_moy#23)) AND (d_year#22 = 1998)) AND (d_moy#23 = 12)) +Input [3]: [d_month_seq#60, d_year#61, d_moy#62] +Condition : (((isnotnull(d_year#61) AND isnotnull(d_moy#62)) AND (d_year#61 = 1998)) AND (d_moy#62 = 12)) (80) Project [codegen id : 1] -Output [1]: [(d_month_seq#37 + 3) AS (d_month_seq + 3)#56] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] +Output [1]: [(d_month_seq#60 + 3) AS (d_month_seq + 3)#63] +Input [3]: [d_month_seq#60, d_year#61, d_moy#62] (81) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 3)#56] -Keys [1]: [(d_month_seq + 3)#56] +Input [1]: [(d_month_seq + 3)#63] +Keys [1]: [(d_month_seq + 3)#63] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#56] +Results [1]: [(d_month_seq + 3)#63] (82) Exchange -Input [1]: [(d_month_seq + 3)#56] -Arguments: hashpartitioning((d_month_seq + 3)#56, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [1]: [(d_month_seq + 3)#63] +Arguments: hashpartitioning((d_month_seq + 3)#63, 5), ENSURE_REQUIREMENTS, [id=#64] (83) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#56] -Keys [1]: [(d_month_seq + 3)#56] +Input [1]: [(d_month_seq + 3)#63] +Keys [1]: [(d_month_seq + 3)#63] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#56] +Results [1]: [(d_month_seq + 3)#63] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index b14ae55d75093..d0638d7606bb5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -109,7 +109,7 @@ Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] Output [3]: [i_item_sk#14, i_class#15, i_category#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women), EqualTo(i_class,maternity), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women ), EqualTo(i_class,maternity ), IsNotNull(i_item_sk)] ReadSchema: struct (11) ColumnarToRow [codegen id : 3] @@ -117,7 +117,7 @@ Input [3]: [i_item_sk#14, i_class#15, i_category#16] (12) Filter [codegen id : 3] Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Condition : ((((isnotnull(i_category#16) AND isnotnull(i_class#15)) AND (i_category#16 = Women)) AND (i_class#15 = maternity)) AND isnotnull(i_item_sk#14)) +Condition : ((((isnotnull(i_category#16) AND isnotnull(i_class#15)) AND (i_category#16 = Women )) AND (i_class#15 = maternity )) AND isnotnull(i_item_sk#14)) (13) Project [codegen id : 3] Output [1]: [i_item_sk#14] @@ -295,75 +295,75 @@ Output [3]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28] Input [7]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28, ca_county#32, ca_state#33, s_county#35, s_state#36] (51) Scan parquet default.date_dim -Output [2]: [d_date_sk#18, d_month_seq#38] +Output [2]: [d_date_sk#38, d_month_seq#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 10] -Input [2]: [d_date_sk#18, d_month_seq#38] +Input [2]: [d_date_sk#38, d_month_seq#39] (53) Filter [codegen id : 10] -Input [2]: [d_date_sk#18, d_month_seq#38] -Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= Subquery scalar-subquery#39, [id=#40])) AND (d_month_seq#38 <= Subquery scalar-subquery#41, [id=#42])) AND isnotnull(d_date_sk#18)) +Input [2]: [d_date_sk#38, d_month_seq#39] +Condition : (((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= Subquery scalar-subquery#40, [id=#41])) AND (d_month_seq#39 <= Subquery scalar-subquery#42, [id=#43])) AND isnotnull(d_date_sk#38)) (54) Project [codegen id : 10] -Output [1]: [d_date_sk#18] -Input [2]: [d_date_sk#18, d_month_seq#38] +Output [1]: [d_date_sk#38] +Input [2]: [d_date_sk#38, d_month_seq#39] (55) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#43] +Input [1]: [d_date_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (56) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#38] Join condition: None (57) Project [codegen id : 11] Output [2]: [c_customer_sk#22, ss_ext_sales_price#27] -Input [4]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#18] +Input [4]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#38] (58) HashAggregate [codegen id : 11] Input [2]: [c_customer_sk#22, ss_ext_sales_price#27] Keys [1]: [c_customer_sk#22] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum#44] -Results [2]: [c_customer_sk#22, sum#45] +Aggregate Attributes [1]: [sum#45] +Results [2]: [c_customer_sk#22, sum#46] (59) Exchange -Input [2]: [c_customer_sk#22, sum#45] -Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [2]: [c_customer_sk#22, sum#46] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#47] (60) HashAggregate [codegen id : 12] -Input [2]: [c_customer_sk#22, sum#45] +Input [2]: [c_customer_sk#22, sum#46] Keys [1]: [c_customer_sk#22] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#47] -Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#47,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#48] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#48] +Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#48,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#49] (61) HashAggregate [codegen id : 12] -Input [1]: [segment#48] -Keys [1]: [segment#48] +Input [1]: [segment#49] +Keys [1]: [segment#49] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#49] -Results [2]: [segment#48, count#50] +Aggregate Attributes [1]: [count#50] +Results [2]: [segment#49, count#51] (62) Exchange -Input [2]: [segment#48, count#50] -Arguments: hashpartitioning(segment#48, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [2]: [segment#49, count#51] +Arguments: hashpartitioning(segment#49, 5), ENSURE_REQUIREMENTS, [id=#52] (63) HashAggregate [codegen id : 13] -Input [2]: [segment#48, count#50] -Keys [1]: [segment#48] +Input [2]: [segment#49, count#51] +Keys [1]: [segment#49] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#52] -Results [3]: [segment#48, count(1)#52 AS num_customers#53, (segment#48 * 50) AS segment_base#54] +Aggregate Attributes [1]: [count(1)#53] +Results [3]: [segment#49, count(1)#53 AS num_customers#54, (segment#49 * 50) AS segment_base#55] (64) TakeOrderedAndProject -Input [3]: [segment#48, num_customers#53, segment_base#54] -Arguments: 100, [segment#48 ASC NULLS FIRST, num_customers#53 ASC NULLS FIRST], [segment#48, num_customers#53, segment_base#54] +Input [3]: [segment#49, num_customers#54, segment_base#55] +Arguments: 100, [segment#49 ASC NULLS FIRST, num_customers#54 ASC NULLS FIRST], [segment#49, num_customers#54, segment_base#55] ===== Subqueries ===== @@ -381,9 +381,9 @@ ReusedExchange (66) (66) ReusedExchange [Reuses operator id: 55] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#38] -Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#39, [id=#40] +Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#40, [id=#41] * HashAggregate (73) +- Exchange (72) +- * HashAggregate (71) @@ -394,42 +394,42 @@ Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (67) Scan parquet default.date_dim -Output [3]: [d_month_seq#38, d_year#19, d_moy#20] +Output [3]: [d_month_seq#56, d_year#57, d_moy#58] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (68) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] +Input [3]: [d_month_seq#56, d_year#57, d_moy#58] (69) Filter [codegen id : 1] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] -Condition : (((isnotnull(d_year#19) AND isnotnull(d_moy#20)) AND (d_year#19 = 1998)) AND (d_moy#20 = 12)) +Input [3]: [d_month_seq#56, d_year#57, d_moy#58] +Condition : (((isnotnull(d_year#57) AND isnotnull(d_moy#58)) AND (d_year#57 = 1998)) AND (d_moy#58 = 12)) (70) Project [codegen id : 1] -Output [1]: [(d_month_seq#38 + 1) AS (d_month_seq + 1)#55] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] +Output [1]: [(d_month_seq#56 + 1) AS (d_month_seq + 1)#59] +Input [3]: [d_month_seq#56, d_year#57, d_moy#58] (71) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 1)#55] -Keys [1]: [(d_month_seq + 1)#55] +Input [1]: [(d_month_seq + 1)#59] +Keys [1]: [(d_month_seq + 1)#59] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#55] +Results [1]: [(d_month_seq + 1)#59] (72) Exchange -Input [1]: [(d_month_seq + 1)#55] -Arguments: hashpartitioning((d_month_seq + 1)#55, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [1]: [(d_month_seq + 1)#59] +Arguments: hashpartitioning((d_month_seq + 1)#59, 5), ENSURE_REQUIREMENTS, [id=#60] (73) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#55] -Keys [1]: [(d_month_seq + 1)#55] +Input [1]: [(d_month_seq + 1)#59] +Keys [1]: [(d_month_seq + 1)#59] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#55] +Results [1]: [(d_month_seq + 1)#59] -Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#42, [id=#43] * HashAggregate (80) +- Exchange (79) +- * HashAggregate (78) @@ -440,39 +440,39 @@ Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (74) Scan parquet default.date_dim -Output [3]: [d_month_seq#38, d_year#19, d_moy#20] +Output [3]: [d_month_seq#61, d_year#62, d_moy#63] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (75) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] +Input [3]: [d_month_seq#61, d_year#62, d_moy#63] (76) Filter [codegen id : 1] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] -Condition : (((isnotnull(d_year#19) AND isnotnull(d_moy#20)) AND (d_year#19 = 1998)) AND (d_moy#20 = 12)) +Input [3]: [d_month_seq#61, d_year#62, d_moy#63] +Condition : (((isnotnull(d_year#62) AND isnotnull(d_moy#63)) AND (d_year#62 = 1998)) AND (d_moy#63 = 12)) (77) Project [codegen id : 1] -Output [1]: [(d_month_seq#38 + 3) AS (d_month_seq + 3)#57] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] +Output [1]: [(d_month_seq#61 + 3) AS (d_month_seq + 3)#64] +Input [3]: [d_month_seq#61, d_year#62, d_moy#63] (78) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 3)#57] -Keys [1]: [(d_month_seq + 3)#57] +Input [1]: [(d_month_seq + 3)#64] +Keys [1]: [(d_month_seq + 3)#64] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#57] +Results [1]: [(d_month_seq + 3)#64] (79) Exchange -Input [1]: [(d_month_seq + 3)#57] -Arguments: hashpartitioning((d_month_seq + 3)#57, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [1]: [(d_month_seq + 3)#64] +Arguments: hashpartitioning((d_month_seq + 3)#64, 5), ENSURE_REQUIREMENTS, [id=#65] (80) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#57] -Keys [1]: [(d_month_seq + 3)#57] +Input [1]: [(d_month_seq + 3)#64] +Keys [1]: [(d_month_seq + 3)#64] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#57] +Results [1]: [(d_month_seq + 3)#64] 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 f0be099379418..173125b197079 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 @@ -213,7 +213,11 @@ Arguments: [rank(d_year#7, d_moy#8) windowspecdefinition(i_category#16, i_brand# (35) Filter [codegen id : 12] Input [8]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27] +<<<<<<< HEAD Condition : ((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#25)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +======= +Condition : (((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#25)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#27)) +>>>>>>> regen (36) Exchange Input [8]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27] @@ -229,22 +233,23 @@ Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] (39) HashAggregate [codegen id : 21] Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] Keys [5]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33] -Functions [1]: [sum(UnscaledValue(cs_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#3))#35] -Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#3))#35,17,2) AS sum_sales#36] +Functions [1]: [sum(UnscaledValue(cs_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#35))#36] +Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#35))#36,17,2) AS sum_sales#37] (40) Exchange -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] (41) Sort [codegen id : 22] -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 (42) Window -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] -Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +<<<<<<< HEAD (43) Project [codegen id : 23] Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] @@ -256,12 +261,30 @@ Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#38 + 1), (45) Sort [codegen id : 24] Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#38 + 1) ASC NULLS FIRST], false, 0 +======= +(43) Filter [codegen id : 23] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] +Condition : isnotnull(rn#39) + +(44) Project [codegen id : 23] +Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] + +(45) Exchange +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] + +(46) Sort [codegen id : 24] +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 +>>>>>>> regen (46) SortMergeJoin [codegen id : 25] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#38 + 1)] +Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#39 + 1)] Join condition: None +<<<<<<< HEAD (47) Project [codegen id : 25] Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] @@ -288,12 +311,45 @@ Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (rn#46 - 1), (53) Sort [codegen id : 36] Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 +======= +(48) Project [codegen id : 25] +Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] +Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] + +(49) ReusedExchange [Reuses operator id: 40] +Output [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] + +(50) Sort [codegen id : 34] +Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 + +(51) Window +Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] +Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#41, i_brand#42, cc_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#41, i_brand#42, cc_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] + +(52) Filter [codegen id : 35] +Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] +Condition : isnotnull(rn#47) + +(53) Project [codegen id : 35] +Output [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] + +(54) Exchange +Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Arguments: hashpartitioning(i_category#41, i_brand#42, cc_name#43, (rn#47 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] + +(55) Sort [codegen id : 36] +Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, (rn#47 - 1) ASC NULLS FIRST], false, 0 +>>>>>>> regen (54) SortMergeJoin [codegen id : 37] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (rn#46 - 1)] +Right keys [4]: [i_category#41, i_brand#42, cc_name#43, (rn#47 - 1)] Join condition: None +<<<<<<< HEAD (55) Project [codegen id : 37] Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#48, sum_sales#45 AS nsum#49] Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] @@ -301,6 +357,15 @@ Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales (56) TakeOrderedAndProject Input [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] +======= +(57) Project [codegen id : 37] +Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#49, sum_sales#46 AS nsum#50] +Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] + +(58) TakeOrderedAndProject +Input [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] +>>>>>>> regen ===== Subqueries ===== 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 1eb7dbe89d6e1..d5b269b02d199 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 @@ -194,7 +194,11 @@ Arguments: [rank(d_year#11, d_moy#12) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [8]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26] +<<<<<<< HEAD Condition : ((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#24)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +======= +Condition : (((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#24)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#26)) +>>>>>>> regen (33) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] @@ -202,22 +206,23 @@ Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] (34) HashAggregate [codegen id : 13] 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#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#33] -Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#33,17,2) AS sum_sales#34] +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] (35) Exchange -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#35] +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] (36) Sort [codegen id : 14] -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35] 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 (37) Window -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] -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#36], [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#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] +<<<<<<< HEAD (38) Project [codegen id : 15] Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34, rn#36] @@ -225,12 +230,26 @@ Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sale (39) BroadcastExchange Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#37] +======= +(38) Filter [codegen id : 15] +Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] +Condition : isnotnull(rn#37) + +(39) Project [codegen id : 15] +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] + +(40) BroadcastExchange +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] + 1)),false), [id=#38] +>>>>>>> regen (40) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#36 + 1)] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] Join condition: None +<<<<<<< HEAD (41) Project [codegen id : 23] Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34] Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] @@ -253,12 +272,41 @@ Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sale (46) BroadcastExchange Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#45] +======= +(42) Project [codegen id : 23] +Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35] +Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] + +(43) ReusedExchange [Reuses operator id: 35] +Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] + +(44) Sort [codegen id : 21] +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 + +(45) 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] + +(46) Filter [codegen id : 22] +Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] +Condition : isnotnull(rn#45) + +(47) Project [codegen id : 22] +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] + +(48) BroadcastExchange +Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] - 1)),false), [id=#46] +>>>>>>> regen (47) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -Right keys [4]: [i_category#38, i_brand#39, cc_name#40, (rn#44 - 1)] +Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] Join condition: None +<<<<<<< HEAD (48) Project [codegen id : 23] Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#34 AS psum#46, sum_sales#43 AS nsum#47] Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34, i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] @@ -266,6 +314,15 @@ Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales (49) TakeOrderedAndProject Input [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, 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#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, psum#46, nsum#47] +======= +(50) Project [codegen id : 23] +Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] +Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] + +(51) TakeOrderedAndProject +Input [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, 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#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, psum#47, nsum#48] +>>>>>>> regen ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index ee3abb39ed053..b339df0707d2f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -110,30 +110,30 @@ Input [2]: [d_date_sk#4, d_date#5] Condition : isnotnull(d_date_sk#4) (7) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#6, d_week_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#6, d_week_seq#7] (9) Filter [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = Subquery scalar-subquery#7, [id=#8])) +Input [2]: [d_date#6, d_week_seq#7] +Condition : (isnotnull(d_week_seq#7) AND (d_week_seq#7 = Subquery scalar-subquery#8, [id=#9])) (10) Project [codegen id : 1] -Output [1]: [d_date#5 AS d_date#5#9] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_date#6] +Input [2]: [d_date#6, d_week_seq#7] (11) BroadcastExchange -Input [1]: [d_date#5#9] +Input [1]: [d_date#6] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] (12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [d_date#5] -Right keys [1]: [d_date#5#9] +Right keys [1]: [d_date#6] Join condition: None (13) Project [codegen id : 2] @@ -218,237 +218,237 @@ Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Condition : isnotnull(cs_item_sk#21) (30) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] +Output [2]: [d_date_sk#24, d_date#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#4, d_date#5] +Input [2]: [d_date_sk#24, d_date#25] (32) Filter [codegen id : 6] -Input [2]: [d_date_sk#4, d_date#5] -Condition : isnotnull(d_date_sk#4) +Input [2]: [d_date_sk#24, d_date#25] +Condition : isnotnull(d_date_sk#24) (33) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#26, d_week_seq#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (34) ColumnarToRow [codegen id : 5] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#26, d_week_seq#27] (35) Filter [codegen id : 5] -Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) +Input [2]: [d_date#26, d_week_seq#27] +Condition : (isnotnull(d_week_seq#27) AND (d_week_seq#27 = ReusedSubquery Subquery scalar-subquery#8, [id=#9])) (36) Project [codegen id : 5] -Output [1]: [d_date#5 AS d_date#5#24] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_date#26] +Input [2]: [d_date#26, d_week_seq#27] (37) BroadcastExchange -Input [1]: [d_date#5#24] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#25] +Input [1]: [d_date#26] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#28] (38) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [d_date#5] -Right keys [1]: [d_date#5#24] +Left keys [1]: [d_date#25] +Right keys [1]: [d_date#26] Join condition: None (39) Project [codegen id : 6] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_date#25] (40) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#4] +Right keys [1]: [d_date_sk#24] Join condition: None (42) Project [codegen id : 8] Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#4] +Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] (43) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#12, i_item_id#13] +Output [2]: [i_item_sk#30, i_item_id#31] (44) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#12] +Right keys [1]: [i_item_sk#30] Join condition: None (45) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#22, i_item_id#13] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#12, i_item_id#13] +Output [2]: [cs_ext_sales_price#22, i_item_id#31] +Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#30, i_item_id#31] (46) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#22, i_item_id#13] -Keys [1]: [i_item_id#13] +Input [2]: [cs_ext_sales_price#22, i_item_id#31] +Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_item_id#13, sum#28] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#31, sum#33] (47) Exchange -Input [2]: [i_item_id#13, sum#28] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [2]: [i_item_id#31, sum#33] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] (48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#13, sum#28] -Keys [1]: [i_item_id#13] +Input [2]: [i_item_id#31, sum#33] +Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#13 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#35] +Results [2]: [i_item_id#31 AS item_id#36, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#35,17,2) AS cs_item_rev#37] (49) Filter [codegen id : 9] -Input [2]: [item_id#31, cs_item_rev#32] -Condition : isnotnull(cs_item_rev#32) +Input [2]: [item_id#36, cs_item_rev#37] +Condition : isnotnull(cs_item_rev#37) (50) BroadcastExchange -Input [2]: [item_id#31, cs_item_rev#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] +Input [2]: [item_id#36, cs_item_rev#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#38] (51) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#31] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) +Right keys [1]: [item_id#36] +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) (52) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] -Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] +Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#37] +Input [4]: [item_id#19, ss_item_rev#20, item_id#36, cs_item_rev#37] (53) Scan parquet default.web_sales -Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] (55) Filter [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#34) +Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] +Condition : isnotnull(ws_item_sk#39) (56) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] +Output [2]: [d_date_sk#42, d_date#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 11] -Input [2]: [d_date_sk#4, d_date#5] +Input [2]: [d_date_sk#42, d_date#43] (58) Filter [codegen id : 11] -Input [2]: [d_date_sk#4, d_date#5] -Condition : isnotnull(d_date_sk#4) +Input [2]: [d_date_sk#42, d_date#43] +Condition : isnotnull(d_date_sk#42) (59) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#44, d_week_seq#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (60) ColumnarToRow [codegen id : 10] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#44, d_week_seq#45] (61) Filter [codegen id : 10] -Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) +Input [2]: [d_date#44, d_week_seq#45] +Condition : (isnotnull(d_week_seq#45) AND (d_week_seq#45 = ReusedSubquery Subquery scalar-subquery#8, [id=#9])) (62) Project [codegen id : 10] -Output [1]: [d_date#5 AS d_date#5#37] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_date#44] +Input [2]: [d_date#44, d_week_seq#45] (63) BroadcastExchange -Input [1]: [d_date#5#37] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#38] +Input [1]: [d_date#44] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#46] (64) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_date#5] -Right keys [1]: [d_date#5#37] +Left keys [1]: [d_date#43] +Right keys [1]: [d_date#44] Join condition: None (65) Project [codegen id : 11] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +Output [1]: [d_date_sk#42] +Input [2]: [d_date_sk#42, d_date#43] (66) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Input [1]: [d_date_sk#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] (67) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#4] +Left keys [1]: [ws_sold_date_sk#41] +Right keys [1]: [d_date_sk#42] Join condition: None (68) Project [codegen id : 13] -Output [2]: [ws_item_sk#34, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#4] +Output [2]: [ws_item_sk#39, ws_ext_sales_price#40] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41, d_date_sk#42] (69) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#12, i_item_id#13] +Output [2]: [i_item_sk#48, i_item_id#49] (70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#12] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#48] Join condition: None (71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#35, i_item_id#13] -Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, i_item_sk#12, i_item_id#13] +Output [2]: [ws_ext_sales_price#40, i_item_id#49] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#40, i_item_sk#48, i_item_id#49] (72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#35, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#13, sum#41] +Input [2]: [ws_ext_sales_price#40, i_item_id#49] +Keys [1]: [i_item_id#49] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#40))] +Aggregate Attributes [1]: [sum#50] +Results [2]: [i_item_id#49, sum#51] (73) Exchange -Input [2]: [i_item_id#13, sum#41] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [2]: [i_item_id#49, sum#51] +Arguments: hashpartitioning(i_item_id#49, 5), ENSURE_REQUIREMENTS, [id=#52] (74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#13, sum#41] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] -Results [2]: [i_item_id#13 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] +Input [2]: [i_item_id#49, sum#51] +Keys [1]: [i_item_id#49] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#40))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#40))#53] +Results [2]: [i_item_id#49 AS item_id#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#40))#53,17,2) AS ws_item_rev#55] (75) Filter [codegen id : 14] -Input [2]: [item_id#44, ws_item_rev#45] -Condition : isnotnull(ws_item_rev#45) +Input [2]: [item_id#54, ws_item_rev#55] +Condition : isnotnull(ws_item_rev#55) (76) BroadcastExchange -Input [2]: [item_id#44, ws_item_rev#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] +Input [2]: [item_id#54, ws_item_rev#55] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] (77) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#44] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) +Right keys [1]: [item_id#54] +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) (78) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#57, cs_item_rev#37, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#37 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#58, ws_item_rev#55, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#55 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#59, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#60] +Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#37, item_id#54, ws_item_rev#55] (79) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] ===== Subqueries ===== -Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#7, [id=#8] +Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#8, [id=#9] * Project (83) +- * Filter (82) +- * ColumnarToRow (81) @@ -456,25 +456,25 @@ Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery (80) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#61, d_week_seq#62] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct (81) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#61, d_week_seq#62] (82) Filter [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_date#5) AND (d_date#5 = 10959)) +Input [2]: [d_date#61, d_week_seq#62] +Condition : (isnotnull(d_date#61) AND (d_date#61 = 2000-01-03)) (83) Project [codegen id : 1] -Output [1]: [d_week_seq#6] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_week_seq#62] +Input [2]: [d_date#61, d_week_seq#62] -Subquery:2 Hosting operator id = 35 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] +Subquery:2 Hosting operator id = 35 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] -Subquery:3 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] +Subquery:3 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index dd37b8801e957..1f3e6853a3c41 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -137,30 +137,30 @@ Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) (13) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#9, d_week_seq#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (14) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#9, d_week_seq#10] (15) Filter [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = Subquery scalar-subquery#10, [id=#11])) +Input [2]: [d_date#9, d_week_seq#10] +Condition : (isnotnull(d_week_seq#10) AND (d_week_seq#10 = Subquery scalar-subquery#11, [id=#12])) (16) Project [codegen id : 2] -Output [1]: [d_date#8 AS d_date#8#12] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_date#9] +Input [2]: [d_date#9, d_week_seq#10] (17) BroadcastExchange -Input [1]: [d_date#8#12] +Input [1]: [d_date#9] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] (18) BroadcastHashJoin [codegen id : 3] Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#12] +Right keys [1]: [d_date#9] Join condition: None (19) Project [codegen id : 3] @@ -218,237 +218,237 @@ Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Condition : isnotnull(cs_item_sk#21) (30) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#24, i_item_id#25] (31) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#4] +Right keys [1]: [i_item_sk#24] Join condition: None (32) Project [codegen id : 8] -Output [3]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#5] -Input [5]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_sk#4, i_item_id#5] +Output [3]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25] +Input [5]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_sk#24, i_item_id#25] (33) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] +Output [2]: [d_date_sk#26, d_date#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#7, d_date#8] +Input [2]: [d_date_sk#26, d_date#27] (35) Filter [codegen id : 7] -Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) +Input [2]: [d_date_sk#26, d_date#27] +Condition : isnotnull(d_date_sk#26) (36) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#28, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#28, d_week_seq#29] (38) Filter [codegen id : 6] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) +Input [2]: [d_date#28, d_week_seq#29] +Condition : (isnotnull(d_week_seq#29) AND (d_week_seq#29 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) (39) Project [codegen id : 6] -Output [1]: [d_date#8 AS d_date#8#24] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_date#28] +Input [2]: [d_date#28, d_week_seq#29] (40) BroadcastExchange -Input [1]: [d_date#8#24] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#25] +Input [1]: [d_date#28] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#30] (41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#24] +Left keys [1]: [d_date#27] +Right keys [1]: [d_date#28] Join condition: None (42) Project [codegen id : 7] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] +Output [1]: [d_date_sk#26] +Input [2]: [d_date_sk#26, d_date#27] (43) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (44) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#26] Join condition: None (45) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#22, i_item_id#5] -Input [4]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#5, d_date_sk#7] +Output [2]: [cs_ext_sales_price#22, i_item_id#25] +Input [4]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25, d_date_sk#26] (46) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#22, i_item_id#5] -Keys [1]: [i_item_id#5] +Input [2]: [cs_ext_sales_price#22, i_item_id#25] +Keys [1]: [i_item_id#25] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_item_id#5, sum#28] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#25, sum#33] (47) Exchange -Input [2]: [i_item_id#5, sum#28] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [2]: [i_item_id#25, sum#33] +Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [id=#34] (48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#5, sum#28] -Keys [1]: [i_item_id#5] +Input [2]: [i_item_id#25, sum#33] +Keys [1]: [i_item_id#25] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#5 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#35] +Results [2]: [i_item_id#25 AS item_id#36, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#35,17,2) AS cs_item_rev#37] (49) Filter [codegen id : 9] -Input [2]: [item_id#31, cs_item_rev#32] -Condition : isnotnull(cs_item_rev#32) +Input [2]: [item_id#36, cs_item_rev#37] +Condition : isnotnull(cs_item_rev#37) (50) BroadcastExchange -Input [2]: [item_id#31, cs_item_rev#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] +Input [2]: [item_id#36, cs_item_rev#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#38] (51) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#31] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) +Right keys [1]: [item_id#36] +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) (52) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] -Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] +Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#37] +Input [4]: [item_id#19, ss_item_rev#20, item_id#36, cs_item_rev#37] (53) Scan parquet default.web_sales -Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] (55) Filter [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#34) +Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] +Condition : isnotnull(ws_item_sk#39) (56) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#42, i_item_id#43] (57) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#4] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#42] Join condition: None (58) Project [codegen id : 13] -Output [3]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#5] -Input [5]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_sk#4, i_item_id#5] +Output [3]: [ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_id#43] +Input [5]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_sk#42, i_item_id#43] (59) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] +Output [2]: [d_date_sk#44, d_date#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#7, d_date#8] +Input [2]: [d_date_sk#44, d_date#45] (61) Filter [codegen id : 12] -Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) +Input [2]: [d_date_sk#44, d_date#45] +Condition : isnotnull(d_date_sk#44) (62) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#46, d_week_seq#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (63) ColumnarToRow [codegen id : 11] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#46, d_week_seq#47] (64) Filter [codegen id : 11] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) +Input [2]: [d_date#46, d_week_seq#47] +Condition : (isnotnull(d_week_seq#47) AND (d_week_seq#47 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) (65) Project [codegen id : 11] -Output [1]: [d_date#8 AS d_date#8#37] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_date#46] +Input [2]: [d_date#46, d_week_seq#47] (66) BroadcastExchange -Input [1]: [d_date#8#37] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#38] +Input [1]: [d_date#46] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#48] (67) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#37] +Left keys [1]: [d_date#45] +Right keys [1]: [d_date#46] Join condition: None (68) Project [codegen id : 12] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] +Output [1]: [d_date_sk#44] +Input [2]: [d_date_sk#44, d_date#45] (69) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Input [1]: [d_date_sk#44] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] (70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [ws_sold_date_sk#41] +Right keys [1]: [d_date_sk#44] Join condition: None (71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#35, i_item_id#5] -Input [4]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#5, d_date_sk#7] +Output [2]: [ws_ext_sales_price#40, i_item_id#43] +Input [4]: [ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_id#43, d_date_sk#44] (72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#35, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#5, sum#41] +Input [2]: [ws_ext_sales_price#40, i_item_id#43] +Keys [1]: [i_item_id#43] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#40))] +Aggregate Attributes [1]: [sum#50] +Results [2]: [i_item_id#43, sum#51] (73) Exchange -Input [2]: [i_item_id#5, sum#41] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [2]: [i_item_id#43, sum#51] +Arguments: hashpartitioning(i_item_id#43, 5), ENSURE_REQUIREMENTS, [id=#52] (74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#5, sum#41] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] -Results [2]: [i_item_id#5 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] +Input [2]: [i_item_id#43, sum#51] +Keys [1]: [i_item_id#43] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#40))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#40))#53] +Results [2]: [i_item_id#43 AS item_id#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#40))#53,17,2) AS ws_item_rev#55] (75) Filter [codegen id : 14] -Input [2]: [item_id#44, ws_item_rev#45] -Condition : isnotnull(ws_item_rev#45) +Input [2]: [item_id#54, ws_item_rev#55] +Condition : isnotnull(ws_item_rev#55) (76) BroadcastExchange -Input [2]: [item_id#44, ws_item_rev#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] +Input [2]: [item_id#54, ws_item_rev#55] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] (77) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#44] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) +Right keys [1]: [item_id#54] +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) (78) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#57, cs_item_rev#37, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#37 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#58, ws_item_rev#55, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#55 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#59, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#60] +Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#37, item_id#54, ws_item_rev#55] (79) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] ===== Subqueries ===== -Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#10, [id=#11] +Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#11, [id=#12] * Project (83) +- * Filter (82) +- * ColumnarToRow (81) @@ -456,25 +456,25 @@ Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquer (80) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#61, d_week_seq#62] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct (81) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#61, d_week_seq#62] (82) Filter [codegen id : 1] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_date#8) AND (d_date#8 = 10959)) +Input [2]: [d_date#61, d_week_seq#62] +Condition : (isnotnull(d_date#61) AND (d_date#61 = 2000-01-03)) (83) Project [codegen id : 1] -Output [1]: [d_week_seq#9] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_week_seq#62] +Input [2]: [d_date#61, d_week_seq#62] -Subquery:2 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] +Subquery:2 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] -Subquery:3 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] +Subquery:3 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] 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 fb57c1b00ecf3..cf22f178dd33f 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#5, ss_store_sk#1, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] +Output [9]: [d_week_seq#54, ss_store_sk#55, sum#56, sum#57, sum#58, sum#59, sum#60, sum#61, sum#62] (27) HashAggregate [codegen id : 9] -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))#61, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67] -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))#61,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63,17,2) AS tue_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67,17,2) AS sat_sales#36] +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] (28) Scan parquet default.store -Output [2]: [s_store_sk#37, s_store_id#38] +Output [2]: [s_store_sk#72, s_store_id#73] 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#37, s_store_id#38] +Input [2]: [s_store_sk#72, s_store_id#73] (30) Filter [codegen id : 7] -Input [2]: [s_store_sk#37, s_store_id#38] -Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) +Input [2]: [s_store_sk#72, s_store_id#73] +Condition : (isnotnull(s_store_sk#72) AND isnotnull(s_store_id#73)) (31) BroadcastExchange -Input [2]: [s_store_sk#37, s_store_id#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#68] +Input [2]: [s_store_sk#72, s_store_id#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#37] +Left keys [1]: [ss_store_sk#55] +Right keys [1]: [s_store_sk#72] Join condition: None (33) Project [codegen id : 9] -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#38] -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#37, s_store_id#38] +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] (34) Scan parquet default.date_dim -Output [2]: [d_month_seq#69, d_week_seq#70] +Output [2]: [d_month_seq#75, d_week_seq#76] 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#69, d_week_seq#70] +Input [2]: [d_month_seq#75, d_week_seq#76] (36) Filter [codegen id : 8] -Input [2]: [d_month_seq#69, d_week_seq#70] -Condition : (((isnotnull(d_month_seq#69) AND (d_month_seq#69 >= 1224)) AND (d_month_seq#69 <= 1235)) AND isnotnull(d_week_seq#70)) +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)) (37) Project [codegen id : 8] -Output [1]: [d_week_seq#70] -Input [2]: [d_month_seq#69, d_week_seq#70] +Output [1]: [d_week_seq#76] +Input [2]: [d_month_seq#75, d_week_seq#76] (38) BroadcastExchange -Input [1]: [d_week_seq#70] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] +Input [1]: [d_week_seq#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] (39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#70] +Left keys [1]: [d_week_seq#54] +Right keys [1]: [d_week_seq#76] Join condition: None (40) Project [codegen id : 9] -Output [9]: [d_week_seq#5 AS d_week_seq2#72, s_store_id#38 AS s_store_id2#73, sun_sales#30 AS sun_sales2#74, mon_sales#31 AS mon_sales2#75, tue_sales#32 AS tue_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 [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#38, d_week_seq#70] +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] (41) BroadcastExchange -Input [9]: [d_week_seq2#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_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] +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] (42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#73, (d_week_seq2#72 - 52)] +Right keys [2]: [s_store_id2#79, (d_week_seq2#78 - 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#74)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#82, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#75)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#83, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#76)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#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 [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#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] +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] (44) TakeOrderedAndProject -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_sales2)#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_sales2)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] +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] 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 fb57c1b00ecf3..cf22f178dd33f 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#5, ss_store_sk#1, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] +Output [9]: [d_week_seq#54, ss_store_sk#55, sum#56, sum#57, sum#58, sum#59, sum#60, sum#61, sum#62] (27) HashAggregate [codegen id : 9] -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))#61, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67] -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))#61,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63,17,2) AS tue_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67,17,2) AS sat_sales#36] +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] (28) Scan parquet default.store -Output [2]: [s_store_sk#37, s_store_id#38] +Output [2]: [s_store_sk#72, s_store_id#73] 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#37, s_store_id#38] +Input [2]: [s_store_sk#72, s_store_id#73] (30) Filter [codegen id : 7] -Input [2]: [s_store_sk#37, s_store_id#38] -Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) +Input [2]: [s_store_sk#72, s_store_id#73] +Condition : (isnotnull(s_store_sk#72) AND isnotnull(s_store_id#73)) (31) BroadcastExchange -Input [2]: [s_store_sk#37, s_store_id#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#68] +Input [2]: [s_store_sk#72, s_store_id#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#37] +Left keys [1]: [ss_store_sk#55] +Right keys [1]: [s_store_sk#72] Join condition: None (33) Project [codegen id : 9] -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#38] -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#37, s_store_id#38] +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] (34) Scan parquet default.date_dim -Output [2]: [d_month_seq#69, d_week_seq#70] +Output [2]: [d_month_seq#75, d_week_seq#76] 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#69, d_week_seq#70] +Input [2]: [d_month_seq#75, d_week_seq#76] (36) Filter [codegen id : 8] -Input [2]: [d_month_seq#69, d_week_seq#70] -Condition : (((isnotnull(d_month_seq#69) AND (d_month_seq#69 >= 1224)) AND (d_month_seq#69 <= 1235)) AND isnotnull(d_week_seq#70)) +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)) (37) Project [codegen id : 8] -Output [1]: [d_week_seq#70] -Input [2]: [d_month_seq#69, d_week_seq#70] +Output [1]: [d_week_seq#76] +Input [2]: [d_month_seq#75, d_week_seq#76] (38) BroadcastExchange -Input [1]: [d_week_seq#70] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] +Input [1]: [d_week_seq#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] (39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#70] +Left keys [1]: [d_week_seq#54] +Right keys [1]: [d_week_seq#76] Join condition: None (40) Project [codegen id : 9] -Output [9]: [d_week_seq#5 AS d_week_seq2#72, s_store_id#38 AS s_store_id2#73, sun_sales#30 AS sun_sales2#74, mon_sales#31 AS mon_sales2#75, tue_sales#32 AS tue_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 [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#38, d_week_seq#70] +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] (41) BroadcastExchange -Input [9]: [d_week_seq2#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_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] +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] (42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#73, (d_week_seq2#72 - 52)] +Right keys [2]: [s_store_id2#79, (d_week_seq2#78 - 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#74)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#82, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#75)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#83, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#76)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#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 [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#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] +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] (44) TakeOrderedAndProject -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_sales2)#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_sales2)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] +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] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt index 23807f0561551..410fd9bc3d4e7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt @@ -286,100 +286,100 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#32] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#32,17,2) AS promotions#33] (48) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] +Output [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] (50) Filter [codegen id : 13] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_store_sk#36) AND isnotnull(ss_customer_sk#35)) AND isnotnull(ss_item_sk#34)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#39] (52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ss_sold_date_sk#38] +Right keys [1]: [d_date_sk#39] Join condition: None (53) Project [codegen id : 13] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] +Output [4]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37] +Input [6]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38, d_date_sk#39] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [i_item_sk#12] +Output [1]: [i_item_sk#40] (55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#12] +Left keys [1]: [ss_item_sk#34] +Right keys [1]: [i_item_sk#40] Join condition: None (56) Project [codegen id : 13] -Output [3]: [ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, i_item_sk#12] +Output [3]: [ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37] +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, i_item_sk#40] (57) ReusedExchange [Reuses operator id: 29] -Output [1]: [s_store_sk#20] +Output [1]: [s_store_sk#41] (58) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#20] +Left keys [1]: [ss_store_sk#36] +Right keys [1]: [s_store_sk#41] Join condition: None (59) Project [codegen id : 13] -Output [2]: [ss_customer_sk#2, ss_ext_sales_price#5] -Input [4]: [ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, s_store_sk#20] +Output [2]: [ss_customer_sk#35, ss_ext_sales_price#37] +Input [4]: [ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, s_store_sk#41] (60) ReusedExchange [Reuses operator id: 42] -Output [1]: [c_customer_sk#23] +Output [1]: [c_customer_sk#42] (61) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#23] +Left keys [1]: [ss_customer_sk#35] +Right keys [1]: [c_customer_sk#42] Join condition: None (62) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#5] -Input [3]: [ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#23] +Output [1]: [ss_ext_sales_price#37] +Input [3]: [ss_customer_sk#35, ss_ext_sales_price#37, c_customer_sk#42] (63) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#5] +Input [1]: [ss_ext_sales_price#37] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum#34] -Results [1]: [sum#35] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#37))] +Aggregate Attributes [1]: [sum#43] +Results [1]: [sum#44] (64) Exchange -Input [1]: [sum#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#36] +Input [1]: [sum#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45] (65) HashAggregate [codegen id : 14] -Input [1]: [sum#35] +Input [1]: [sum#44] Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#37] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#37,17,2) AS total#38] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#37))#46] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#37))#46,17,2) AS total#47] (66) BroadcastExchange -Input [1]: [total#38] -Arguments: IdentityBroadcastMode, [id=#39] +Input [1]: [total#47] +Arguments: IdentityBroadcastMode, [id=#48] (67) BroadcastNestedLoopJoin [codegen id : 15] Join condition: None (68) Project [codegen id : 15] -Output [3]: [promotions#33, total#38, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#38 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] -Input [2]: [promotions#33, total#38] +Output [3]: [promotions#33, total#47, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#47 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#49] +Input [2]: [promotions#33, total#47] (69) Sort [codegen id : 15] -Input [3]: [promotions#33, total#38, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] -Arguments: [promotions#33 ASC NULLS FIRST, total#38 ASC NULLS FIRST], true, 0 +Input [3]: [promotions#33, total#47, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#49] +Arguments: [promotions#33 ASC NULLS FIRST, total#47 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -390,6 +390,6 @@ ReusedExchange (70) (70) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt index c0fad04ce4026..e0bee37e047cb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt @@ -289,112 +289,112 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#32] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#32,17,2) AS promotions#33] (48) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] +Output [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] (50) Filter [codegen id : 13] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_store_sk#36) AND isnotnull(ss_customer_sk#35)) AND isnotnull(ss_item_sk#34)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [s_store_sk#8] +Output [1]: [s_store_sk#39] (52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] +Left keys [1]: [ss_store_sk#36] +Right keys [1]: [s_store_sk#39] Join condition: None (53) Project [codegen id : 13] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] +Output [4]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37, ss_sold_date_sk#38] +Input [6]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38, s_store_sk#39] (54) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#40] (55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#16] +Left keys [1]: [ss_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] Join condition: None (56) Project [codegen id : 13] -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#16] +Output [3]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37] +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37, ss_sold_date_sk#38, d_date_sk#40] (57) ReusedExchange [Reuses operator id: 28] -Output [2]: [c_customer_sk#20, c_current_addr_sk#21] +Output [2]: [c_customer_sk#41, c_current_addr_sk#42] (58) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#20] +Left keys [1]: [ss_customer_sk#35] +Right keys [1]: [c_customer_sk#41] Join condition: None (59) Project [codegen id : 13] -Output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#21] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#20, c_current_addr_sk#21] +Output [3]: [ss_item_sk#34, ss_ext_sales_price#37, c_current_addr_sk#42] +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37, c_customer_sk#41, c_current_addr_sk#42] (60) ReusedExchange [Reuses operator id: 35] -Output [1]: [ca_address_sk#23] +Output [1]: [ca_address_sk#43] (61) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [c_current_addr_sk#21] -Right keys [1]: [ca_address_sk#23] +Left keys [1]: [c_current_addr_sk#42] +Right keys [1]: [ca_address_sk#43] Join condition: None (62) Project [codegen id : 13] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#5] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#21, ca_address_sk#23] +Output [2]: [ss_item_sk#34, ss_ext_sales_price#37] +Input [4]: [ss_item_sk#34, ss_ext_sales_price#37, c_current_addr_sk#42, ca_address_sk#43] (63) ReusedExchange [Reuses operator id: 42] -Output [1]: [i_item_sk#26] +Output [1]: [i_item_sk#44] (64) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#26] +Left keys [1]: [ss_item_sk#34] +Right keys [1]: [i_item_sk#44] Join condition: None (65) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#5] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#26] +Output [1]: [ss_ext_sales_price#37] +Input [3]: [ss_item_sk#34, ss_ext_sales_price#37, i_item_sk#44] (66) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#5] +Input [1]: [ss_ext_sales_price#37] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum#34] -Results [1]: [sum#35] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#37))] +Aggregate Attributes [1]: [sum#45] +Results [1]: [sum#46] (67) Exchange -Input [1]: [sum#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#36] +Input [1]: [sum#46] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#47] (68) HashAggregate [codegen id : 14] -Input [1]: [sum#35] +Input [1]: [sum#46] Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#37] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#37,17,2) AS total#38] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#37))#48] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#37))#48,17,2) AS total#49] (69) BroadcastExchange -Input [1]: [total#38] -Arguments: IdentityBroadcastMode, [id=#39] +Input [1]: [total#49] +Arguments: IdentityBroadcastMode, [id=#50] (70) BroadcastNestedLoopJoin [codegen id : 15] Join condition: None (71) Project [codegen id : 15] -Output [3]: [promotions#33, total#38, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#38 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] -Input [2]: [promotions#33, total#38] +Output [3]: [promotions#33, total#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#49 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#51] +Input [2]: [promotions#33, total#49] (72) Sort [codegen id : 15] -Input [3]: [promotions#33, total#38, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] -Arguments: [promotions#33 ASC NULLS FIRST, total#38 ASC NULLS FIRST], true, 0 +Input [3]: [promotions#33, total#49, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#51] +Arguments: [promotions#33 ASC NULLS FIRST, total#49 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -405,6 +405,6 @@ ReusedExchange (73) (73) ReusedExchange [Reuses operator id: 22] Output [1]: [d_date_sk#16] -Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt index 3533986d86b84..2b8dbaddacad8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,refernece,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,refernece ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] (3) Filter [codegen id : 1] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,refernece,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,refernece ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 1] Output [2]: [i_item_sk#1, i_manager_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manager_id#5, specifiedwindowfram (30) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#26, _w0#27, avg_monthly_sales#29] -Condition : ((isnotnull(avg_monthly_sales#29) AND (avg_monthly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_monthly_sales#29) AND ((avg_monthly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (31) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#26, avg_monthly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt index 622b073a4ae10..efb27d8cd8b11 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,refernece,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,refernece ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] (3) Filter [codegen id : 4] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,refernece,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,refernece ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 4] Output [2]: [i_item_sk#1, i_manager_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manager_id#5, specifiedwindowfram (30) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#26, _w0#27, avg_monthly_sales#29] -Condition : ((isnotnull(avg_monthly_sales#29) AND (avg_monthly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_monthly_sales#29) AND ((avg_monthly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (31) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#26, avg_monthly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt index 5d74065179f44..931e72bba7182 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt @@ -741,7 +741,7 @@ Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt# Output [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple,burlywood,indian,spring,floral,medium]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple ,burlywood ,indian ,spring ,floral ,medium ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct (121) ColumnarToRow [codegen id : 40] @@ -749,7 +749,7 @@ Input [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] (122) Filter [codegen id : 40] Input [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] -Condition : ((((((isnotnull(i_current_price#94) AND i_color#95 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#94 >= 64.00)) AND (i_current_price#94 <= 74.00)) AND (i_current_price#94 >= 65.00)) AND (i_current_price#94 <= 79.00)) AND isnotnull(i_item_sk#93)) +Condition : ((((((isnotnull(i_current_price#94) AND i_color#95 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#94 >= 64.00)) AND (i_current_price#94 <= 74.00)) AND (i_current_price#94 >= 65.00)) AND (i_current_price#94 <= 79.00)) AND isnotnull(i_item_sk#93)) (123) Project [codegen id : 40] Output [2]: [i_item_sk#93, i_product_name#96] @@ -795,353 +795,353 @@ Input [17]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_str Arguments: [item_sk#112 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, store_zip#114 ASC NULLS FIRST], false, 0 (132) Scan parquet default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#129)] +PartitionFilters: [isnotnull(ss_sold_date_sk#140), dynamicpruningexpression(ss_sold_date_sk#140 IN dynamicpruning#141)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (133) ColumnarToRow [codegen id : 44] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] (134) Filter [codegen id : 44] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Condition : (((((((isnotnull(ss_item_sk#129) AND isnotnull(ss_ticket_number#136)) AND isnotnull(ss_store_sk#134)) AND isnotnull(ss_customer_sk#130)) AND isnotnull(ss_cdemo_sk#131)) AND isnotnull(ss_promo_sk#135)) AND isnotnull(ss_hdemo_sk#132)) AND isnotnull(ss_addr_sk#133)) (135) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint), 5), ENSURE_REQUIREMENTS, [id=#130] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Arguments: hashpartitioning(cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint), 5), ENSURE_REQUIREMENTS, [id=#142] (136) Sort [codegen id : 45] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [cast(ss_item_sk#1 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#8 as bigint) ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Arguments: [cast(ss_item_sk#129 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#136 as bigint) ASC NULLS FIRST], false, 0 (137) ReusedExchange [Reuses operator id: 10] -Output [2]: [sr_item_sk#15, sr_ticket_number#16] +Output [2]: [sr_item_sk#143, sr_ticket_number#144] (138) Sort [codegen id : 47] -Input [2]: [sr_item_sk#15, sr_ticket_number#16] -Arguments: [sr_item_sk#15 ASC NULLS FIRST, sr_ticket_number#16 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#143, sr_ticket_number#144] +Arguments: [sr_item_sk#143 ASC NULLS FIRST, sr_ticket_number#144 ASC NULLS FIRST], false, 0 (139) SortMergeJoin [codegen id : 56] -Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] -Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] +Left keys [2]: [cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint)] +Right keys [2]: [sr_item_sk#143, sr_ticket_number#144] Join condition: None (140) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, sr_item_sk#143, sr_ticket_number#144] (141) ReusedExchange [Reuses operator id: 33] -Output [1]: [cs_item_sk#19] +Output [1]: [cs_item_sk#145] (142) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#19] +Left keys [1]: [ss_item_sk#129] +Right keys [1]: [cs_item_sk#145] Join condition: None (143) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, cs_item_sk#145] (144) Scan parquet default.date_dim -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#146, d_year#147] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (145) ColumnarToRow [codegen id : 54] -Input [2]: [d_date_sk#43, d_year#44] +Input [2]: [d_date_sk#146, d_year#147] (146) Filter [codegen id : 54] -Input [2]: [d_date_sk#43, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#43)) +Input [2]: [d_date_sk#146, d_year#147] +Condition : ((isnotnull(d_year#147) AND (d_year#147 = 2000)) AND isnotnull(d_date_sk#146)) (147) BroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#131] +Input [2]: [d_date_sk#146, d_year#147] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#148] (148) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#43] +Left keys [1]: [ss_sold_date_sk#140] +Right keys [1]: [d_date_sk#146] Join condition: None (149) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#43, d_year#44] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147] +Input [13]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, d_date_sk#146, d_year#147] (150) ReusedExchange [Reuses operator id: 45] -Output [3]: [s_store_sk#46, s_store_name#47, s_zip#48] +Output [3]: [s_store_sk#149, s_store_name#150, s_zip#151] (151) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#46] +Left keys [1]: [ss_store_sk#134] +Right keys [1]: [s_store_sk#149] Join condition: None (152) Project [codegen id : 56] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_sk#46, s_store_name#47, s_zip#48] +Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_sk#149, s_store_name#150, s_zip#151] (153) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#132] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Arguments: hashpartitioning(ss_customer_sk#130, 5), ENSURE_REQUIREMENTS, [id=#152] (154) Sort [codegen id : 57] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Arguments: [ss_customer_sk#130 ASC NULLS FIRST], false, 0 (155) ReusedExchange [Reuses operator id: 53] -Output [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Output [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] (156) Sort [codegen id : 59] -Input [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] -Arguments: [c_customer_sk#51 ASC NULLS FIRST], false, 0 +Input [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Arguments: [c_customer_sk#153 ASC NULLS FIRST], false, 0 (157) SortMergeJoin [codegen id : 62] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#51] +Left keys [1]: [ss_customer_sk#130] +Right keys [1]: [c_customer_sk#153] Join condition: None (158) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Input [18]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] (159) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#133, d_year#134] +Output [2]: [d_date_sk#159, d_year#160] (160) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_sales_date_sk#56] -Right keys [1]: [d_date_sk#133] +Left keys [1]: [c_first_sales_date_sk#158] +Right keys [1]: [d_date_sk#159] Join condition: None (161) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56, d_date_sk#133, d_year#134] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158, d_date_sk#159, d_year#160] (162) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#135, d_year#136] +Output [2]: [d_date_sk#161, d_year#162] (163) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_shipto_date_sk#55] -Right keys [1]: [d_date_sk#135] +Left keys [1]: [c_first_shipto_date_sk#157] +Right keys [1]: [d_date_sk#161] Join condition: None (164) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134, d_date_sk#135, d_year#136] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160, d_date_sk#161, d_year#162] (165) Exchange -Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Arguments: hashpartitioning(ss_cdemo_sk#3, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Arguments: hashpartitioning(ss_cdemo_sk#131, 5), ENSURE_REQUIREMENTS, [id=#163] (166) Sort [codegen id : 63] -Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Arguments: [ss_cdemo_sk#3 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Arguments: [ss_cdemo_sk#131 ASC NULLS FIRST], false, 0 (167) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#64, cd_marital_status#65] +Output [2]: [cd_demo_sk#164, cd_marital_status#165] (168) Sort [codegen id : 65] -Input [2]: [cd_demo_sk#64, cd_marital_status#65] -Arguments: [cd_demo_sk#64 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#164, cd_marital_status#165] +Arguments: [cd_demo_sk#164 ASC NULLS FIRST], false, 0 (169) SortMergeJoin [codegen id : 66] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#64] +Left keys [1]: [ss_cdemo_sk#131] +Right keys [1]: [cd_demo_sk#164] Join condition: None (170) Project [codegen id : 66] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_demo_sk#64, cd_marital_status#65] +Output [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_demo_sk#164, cd_marital_status#165] (171) Exchange -Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Arguments: hashpartitioning(c_current_cdemo_sk#52, 5), ENSURE_REQUIREMENTS, [id=#138] +Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Arguments: hashpartitioning(c_current_cdemo_sk#154, 5), ENSURE_REQUIREMENTS, [id=#166] (172) Sort [codegen id : 67] -Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Arguments: [c_current_cdemo_sk#52 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Arguments: [c_current_cdemo_sk#154 ASC NULLS FIRST], false, 0 (173) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#139, cd_marital_status#140] +Output [2]: [cd_demo_sk#167, cd_marital_status#168] (174) Sort [codegen id : 69] -Input [2]: [cd_demo_sk#139, cd_marital_status#140] -Arguments: [cd_demo_sk#139 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#167, cd_marital_status#168] +Arguments: [cd_demo_sk#167 ASC NULLS FIRST], false, 0 (175) SortMergeJoin [codegen id : 73] -Left keys [1]: [c_current_cdemo_sk#52] -Right keys [1]: [cd_demo_sk#139] -Join condition: NOT (cd_marital_status#65 = cd_marital_status#140) +Left keys [1]: [c_current_cdemo_sk#154] +Right keys [1]: [cd_demo_sk#167] +Join condition: NOT (cd_marital_status#165 = cd_marital_status#168) (176) Project [codegen id : 73] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65, cd_demo_sk#139, cd_marital_status#140] +Output [14]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [18]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165, cd_demo_sk#167, cd_marital_status#168] (177) ReusedExchange [Reuses operator id: 84] -Output [1]: [p_promo_sk#70] +Output [1]: [p_promo_sk#169] (178) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#70] +Left keys [1]: [ss_promo_sk#135] +Right keys [1]: [p_promo_sk#169] Join condition: None (179) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, p_promo_sk#70] +Output [13]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, p_promo_sk#169] (180) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#72, hd_income_band_sk#73] +Output [2]: [hd_demo_sk#170, hd_income_band_sk#171] (181) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#72] +Left keys [1]: [ss_hdemo_sk#132] +Right keys [1]: [hd_demo_sk#170] Join condition: None (182) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_demo_sk#72, hd_income_band_sk#73] +Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171] +Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_demo_sk#170, hd_income_band_sk#171] (183) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] +Output [2]: [hd_demo_sk#172, hd_income_band_sk#173] (184) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [c_current_hdemo_sk#53] -Right keys [1]: [hd_demo_sk#141] +Left keys [1]: [c_current_hdemo_sk#155] +Right keys [1]: [hd_demo_sk#172] Join condition: None (185) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_demo_sk#141, hd_income_band_sk#142] +Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Input [15]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_demo_sk#172, hd_income_band_sk#173] (186) Exchange -Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Arguments: hashpartitioning(ss_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#143] +Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Arguments: hashpartitioning(ss_addr_sk#133, 5), ENSURE_REQUIREMENTS, [id=#174] (187) Sort [codegen id : 74] -Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Arguments: [ss_addr_sk#5 ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Arguments: [ss_addr_sk#133 ASC NULLS FIRST], false, 0 (188) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Output [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] (189) Sort [codegen id : 76] -Input [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: [ca_address_sk#78 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: [ca_address_sk#175 ASC NULLS FIRST], false, 0 (190) SortMergeJoin [codegen id : 77] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#78] +Left keys [1]: [ss_addr_sk#133] +Right keys [1]: [ca_address_sk#175] Join condition: None (191) Project [codegen id : 77] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Output [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Input [18]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] (192) Exchange -Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: hashpartitioning(c_current_addr_sk#54, 5), ENSURE_REQUIREMENTS, [id=#144] +Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: hashpartitioning(c_current_addr_sk#156, 5), ENSURE_REQUIREMENTS, [id=#180] (193) Sort [codegen id : 78] -Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: [c_current_addr_sk#54 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: [c_current_addr_sk#156 ASC NULLS FIRST], false, 0 (194) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Output [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] (195) Sort [codegen id : 80] -Input [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Arguments: [ca_address_sk#145 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Arguments: [ca_address_sk#181 ASC NULLS FIRST], false, 0 (196) SortMergeJoin [codegen id : 84] -Left keys [1]: [c_current_addr_sk#54] -Right keys [1]: [ca_address_sk#145] +Left keys [1]: [c_current_addr_sk#156] +Right keys [1]: [ca_address_sk#181] Join condition: None (197) Project [codegen id : 84] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Output [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [21]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] (198) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#90] +Output [1]: [ib_income_band_sk#186] (199) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#73] -Right keys [1]: [ib_income_band_sk#90] +Left keys [1]: [hd_income_band_sk#171] +Right keys [1]: [ib_income_band_sk#186] Join condition: None (200) Project [codegen id : 84] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#90] +Output [18]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [20]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#186] (201) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#150] +Output [1]: [ib_income_band_sk#187] (202) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#142] -Right keys [1]: [ib_income_band_sk#150] +Left keys [1]: [hd_income_band_sk#173] +Right keys [1]: [ib_income_band_sk#187] Join condition: None (203) Project [codegen id : 84] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#150] +Output [17]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#187] (204) ReusedExchange [Reuses operator id: 124] -Output [2]: [i_item_sk#93, i_product_name#96] +Output [2]: [i_item_sk#188, i_product_name#189] (205) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#93] +Left keys [1]: [ss_item_sk#129] +Right keys [1]: [i_item_sk#188] Join condition: None (206) Project [codegen id : 84] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] +Output [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] +Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] (207) HashAggregate [codegen id : 84] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] -Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#151, sum#152, sum#153, sum#154] -Results [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] +Input [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] +Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#137)), partial_sum(UnscaledValue(ss_list_price#138)), partial_sum(UnscaledValue(ss_coupon_amt#139))] +Aggregate Attributes [4]: [count#190, sum#191, sum#192, sum#193] +Results [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] (208) Exchange -Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] -Arguments: hashpartitioning(i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, 5), ENSURE_REQUIREMENTS, [id=#159] +Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] +Arguments: hashpartitioning(i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, 5), ENSURE_REQUIREMENTS, [id=#198] (209) HashAggregate [codegen id : 85] -Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] -Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#160, sum(UnscaledValue(ss_wholesale_cost#9))#161, sum(UnscaledValue(ss_list_price#10))#162, sum(UnscaledValue(ss_coupon_amt#11))#163] -Results [8]: [i_item_sk#93 AS item_sk#164, s_store_name#47 AS store_name#165, s_zip#48 AS store_zip#166, d_year#44 AS syear#167, count(1)#160 AS cnt#168, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#161,17,2) AS s1#169, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#162,17,2) AS s2#170, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#163,17,2) AS s3#171] +Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] +Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#137)), sum(UnscaledValue(ss_list_price#138)), sum(UnscaledValue(ss_coupon_amt#139))] +Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#137))#200, sum(UnscaledValue(ss_list_price#138))#201, sum(UnscaledValue(ss_coupon_amt#139))#202] +Results [8]: [i_item_sk#188 AS item_sk#203, s_store_name#150 AS store_name#204, s_zip#151 AS store_zip#205, d_year#147 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#137))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#138))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#139))#202,17,2) AS s3#210] (210) Exchange -Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] -Arguments: hashpartitioning(item_sk#164, store_name#165, store_zip#166, 5), ENSURE_REQUIREMENTS, [id=#172] +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] (211) Sort [codegen id : 86] -Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] -Arguments: [item_sk#164 ASC NULLS FIRST, store_name#165 ASC NULLS FIRST, store_zip#166 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 (212) SortMergeJoin [codegen id : 87] Left keys [3]: [item_sk#112, store_name#113, store_zip#114] -Right keys [3]: [item_sk#164, store_name#165, store_zip#166] -Join condition: (cnt#168 <= cnt#124) +Right keys [3]: [item_sk#203, store_name#204, store_zip#205] +Join condition: (cnt#207 <= cnt#124) (213) Project [codegen id : 87] -Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] +Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] (214) Exchange -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#173] +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] (215) Sort [codegen id : 88] -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1152,11 +1152,11 @@ ReusedExchange (216) (216) ReusedExchange [Reuses operator id: 39] Output [2]: [d_date_sk#43, d_year#44] -Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#129 +Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#140 IN dynamicpruning#141 ReusedExchange (217) (217) ReusedExchange [Reuses operator id: 147] -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#146, d_year#147] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index 9cc3415e6e146..a866c8aa9b486 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt @@ -651,7 +651,7 @@ Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt# Output [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple,burlywood,indian,spring,floral,medium]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple ,burlywood ,indian ,spring ,floral ,medium ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct (105) ColumnarToRow [codegen id : 24] @@ -659,7 +659,7 @@ Input [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] (106) Filter [codegen id : 24] Input [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] -Condition : ((((((isnotnull(i_current_price#88) AND i_color#89 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#88 >= 64.00)) AND (i_current_price#88 <= 74.00)) AND (i_current_price#88 >= 65.00)) AND (i_current_price#88 <= 79.00)) AND isnotnull(i_item_sk#87)) +Condition : ((((((isnotnull(i_current_price#88) AND i_color#89 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#88 >= 64.00)) AND (i_current_price#88 <= 74.00)) AND (i_current_price#88 >= 65.00)) AND (i_current_price#88 <= 79.00)) AND isnotnull(i_item_sk#87)) (107) Project [codegen id : 24] Output [2]: [i_item_sk#87, i_product_name#90] @@ -701,323 +701,323 @@ Input [17]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_str Arguments: [item_sk#105 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, store_zip#107 ASC NULLS FIRST], false, 0 (115) Scan parquet default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#122)] +PartitionFilters: [isnotnull(ss_sold_date_sk#133), dynamicpruningexpression(ss_sold_date_sk#133 IN dynamicpruning#134)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (116) ColumnarToRow [codegen id : 27] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] (117) Filter [codegen id : 27] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Condition : (((((((isnotnull(ss_item_sk#122) AND isnotnull(ss_ticket_number#129)) AND isnotnull(ss_store_sk#127)) AND isnotnull(ss_customer_sk#123)) AND isnotnull(ss_cdemo_sk#124)) AND isnotnull(ss_promo_sk#128)) AND isnotnull(ss_hdemo_sk#125)) AND isnotnull(ss_addr_sk#126)) (118) BroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#123] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#135] (119) Scan parquet default.store_returns -Output [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Output [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct (120) ColumnarToRow -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] (121) Filter -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] -Condition : (isnotnull(sr_item_sk#15) AND isnotnull(sr_ticket_number#16)) +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Condition : (isnotnull(sr_item_sk#136) AND isnotnull(sr_ticket_number#137)) (122) Project -Output [2]: [sr_item_sk#15, sr_ticket_number#16] -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Output [2]: [sr_item_sk#136, sr_ticket_number#137] +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] (123) BroadcastHashJoin [codegen id : 28] -Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] -Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] +Left keys [2]: [cast(ss_item_sk#122 as bigint), cast(ss_ticket_number#129 as bigint)] +Right keys [2]: [sr_item_sk#136, sr_ticket_number#137] Join condition: None (124) Project [codegen id : 28] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, sr_item_sk#136, sr_ticket_number#137] (125) Exchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: hashpartitioning(ss_item_sk#122, 5), ENSURE_REQUIREMENTS, [id=#139] (126) Sort [codegen id : 29] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: [ss_item_sk#122 ASC NULLS FIRST], false, 0 (127) ReusedExchange [Reuses operator id: 28] -Output [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] +Output [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] (128) HashAggregate [codegen id : 35] -Input [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] -Keys [1]: [cs_item_sk#19] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#21)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#21))#128, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129] -Results [3]: [cs_item_sk#19, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#21))#128,17,2) AS sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] +Input [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] +Keys [1]: [cs_item_sk#140] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#144)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#144))#148, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149] +Results [3]: [cs_item_sk#140, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#144))#148,17,2) AS sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] (129) Filter [codegen id : 35] -Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] -Condition : (isnotnull(sum(cs_ext_list_price#21)#130) AND (cast(sum(cs_ext_list_price#21)#130 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131)), DecimalType(21,2), true))) +Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] +Condition : (isnotnull(sum(cs_ext_list_price#144)#150) AND (cast(sum(cs_ext_list_price#144)#150 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151)), DecimalType(21,2), true))) (130) Project [codegen id : 35] -Output [1]: [cs_item_sk#19] -Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] +Output [1]: [cs_item_sk#140] +Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] (131) Sort [codegen id : 35] -Input [1]: [cs_item_sk#19] -Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 +Input [1]: [cs_item_sk#140] +Arguments: [cs_item_sk#140 ASC NULLS FIRST], false, 0 (132) SortMergeJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#19] +Left keys [1]: [ss_item_sk#122] +Right keys [1]: [cs_item_sk#140] Join condition: None (133) Project [codegen id : 51] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, cs_item_sk#140] (134) Scan parquet default.date_dim -Output [2]: [d_date_sk#42, d_year#43] +Output [2]: [d_date_sk#152, d_year#153] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (135) ColumnarToRow [codegen id : 36] -Input [2]: [d_date_sk#42, d_year#43] +Input [2]: [d_date_sk#152, d_year#153] (136) Filter [codegen id : 36] -Input [2]: [d_date_sk#42, d_year#43] -Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2000)) AND isnotnull(d_date_sk#42)) +Input [2]: [d_date_sk#152, d_year#153] +Condition : ((isnotnull(d_year#153) AND (d_year#153 = 2000)) AND isnotnull(d_date_sk#152)) (137) BroadcastExchange -Input [2]: [d_date_sk#42, d_year#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#132] +Input [2]: [d_date_sk#152, d_year#153] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#154] (138) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#42] +Left keys [1]: [ss_sold_date_sk#133] +Right keys [1]: [d_date_sk#152] Join condition: None (139) Project [codegen id : 51] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#42, d_year#43] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153] +Input [13]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, d_date_sk#152, d_year#153] (140) ReusedExchange [Reuses operator id: 44] -Output [3]: [s_store_sk#45, s_store_name#46, s_zip#47] +Output [3]: [s_store_sk#155, s_store_name#156, s_zip#157] (141) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#45] +Left keys [1]: [ss_store_sk#127] +Right keys [1]: [s_store_sk#155] Join condition: None (142) Project [codegen id : 51] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_sk#45, s_store_name#46, s_zip#47] +Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157] +Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_sk#155, s_store_name#156, s_zip#157] (143) ReusedExchange [Reuses operator id: 50] -Output [6]: [c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] +Output [6]: [c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] (144) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#49] +Left keys [1]: [ss_customer_sk#123] +Right keys [1]: [c_customer_sk#158] Join condition: None (145) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] +Input [18]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] (146) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#133, d_year#134] +Output [2]: [d_date_sk#164, d_year#165] (147) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_sales_date_sk#54] -Right keys [1]: [d_date_sk#133] +Left keys [1]: [c_first_sales_date_sk#163] +Right keys [1]: [d_date_sk#164] Join condition: None (148) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54, d_date_sk#133, d_year#134] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163, d_date_sk#164, d_year#165] (149) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#135, d_year#136] +Output [2]: [d_date_sk#166, d_year#167] (150) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_shipto_date_sk#53] -Right keys [1]: [d_date_sk#135] +Left keys [1]: [c_first_shipto_date_sk#162] +Right keys [1]: [d_date_sk#166] Join condition: None (151) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134, d_date_sk#135, d_year#136] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165, d_date_sk#166, d_year#167] (152) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#61, cd_marital_status#62] +Output [2]: [cd_demo_sk#168, cd_marital_status#169] (153) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#61] +Left keys [1]: [ss_cdemo_sk#124] +Right keys [1]: [cd_demo_sk#168] Join condition: None (154) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_demo_sk#61, cd_marital_status#62] +Output [16]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_demo_sk#168, cd_marital_status#169] (155) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#137, cd_marital_status#138] +Output [2]: [cd_demo_sk#170, cd_marital_status#171] (156) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_cdemo_sk#50] -Right keys [1]: [cd_demo_sk#137] -Join condition: NOT (cd_marital_status#62 = cd_marital_status#138) +Left keys [1]: [c_current_cdemo_sk#159] +Right keys [1]: [cd_demo_sk#170] +Join condition: NOT (cd_marital_status#169 = cd_marital_status#171) (157) Project [codegen id : 51] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62, cd_demo_sk#137, cd_marital_status#138] +Output [14]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [18]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169, cd_demo_sk#170, cd_marital_status#171] (158) ReusedExchange [Reuses operator id: 74] -Output [1]: [p_promo_sk#66] +Output [1]: [p_promo_sk#172] (159) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#66] +Left keys [1]: [ss_promo_sk#128] +Right keys [1]: [p_promo_sk#172] Join condition: None (160) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, p_promo_sk#66] +Output [13]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, p_promo_sk#172] (161) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#68, hd_income_band_sk#69] +Output [2]: [hd_demo_sk#173, hd_income_band_sk#174] (162) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#68] +Left keys [1]: [ss_hdemo_sk#125] +Right keys [1]: [hd_demo_sk#173] Join condition: None (163) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_demo_sk#68, hd_income_band_sk#69] +Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174] +Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_demo_sk#173, hd_income_band_sk#174] (164) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] +Output [2]: [hd_demo_sk#175, hd_income_band_sk#176] (165) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_hdemo_sk#51] -Right keys [1]: [hd_demo_sk#139] +Left keys [1]: [c_current_hdemo_sk#160] +Right keys [1]: [hd_demo_sk#175] Join condition: None (166) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_demo_sk#139, hd_income_band_sk#140] +Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176] +Input [15]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_demo_sk#175, hd_income_band_sk#176] (167) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] +Output [5]: [ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] (168) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#73] +Left keys [1]: [ss_addr_sk#126] +Right keys [1]: [ca_address_sk#177] Join condition: None (169) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] +Output [16]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] +Input [18]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] (170) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Output [5]: [ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] (171) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_addr_sk#52] -Right keys [1]: [ca_address_sk#141] +Left keys [1]: [c_current_addr_sk#161] +Right keys [1]: [ca_address_sk#182] Join condition: None (172) Project [codegen id : 51] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Output [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [21]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] (173) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#84] +Output [1]: [ib_income_band_sk#187] (174) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#69] -Right keys [1]: [ib_income_band_sk#84] +Left keys [1]: [hd_income_band_sk#174] +Right keys [1]: [ib_income_band_sk#187] Join condition: None (175) Project [codegen id : 51] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#84] +Output [18]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [20]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#187] (176) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#146] +Output [1]: [ib_income_band_sk#188] (177) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#140] -Right keys [1]: [ib_income_band_sk#146] +Left keys [1]: [hd_income_band_sk#176] +Right keys [1]: [ib_income_band_sk#188] Join condition: None (178) Project [codegen id : 51] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#146] +Output [17]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#188] (179) ReusedExchange [Reuses operator id: 108] -Output [2]: [i_item_sk#87, i_product_name#90] +Output [2]: [i_item_sk#189, i_product_name#190] (180) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#87] +Left keys [1]: [ss_item_sk#122] +Right keys [1]: [i_item_sk#189] Join condition: None (181) Project [codegen id : 51] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] +Output [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] (182) HashAggregate [codegen id : 51] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] -Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#147, sum#148, sum#149, sum#150] -Results [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] +Input [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#130)), partial_sum(UnscaledValue(ss_list_price#131)), partial_sum(UnscaledValue(ss_coupon_amt#132))] +Aggregate Attributes [4]: [count#191, sum#192, sum#193, sum#194] +Results [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] (183) HashAggregate [codegen id : 51] -Input [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#155, sum(UnscaledValue(ss_wholesale_cost#9))#156, sum(UnscaledValue(ss_list_price#10))#157, sum(UnscaledValue(ss_coupon_amt#11))#158] -Results [8]: [i_item_sk#87 AS item_sk#159, s_store_name#46 AS store_name#160, s_zip#47 AS store_zip#161, d_year#43 AS syear#162, count(1)#155 AS cnt#163, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#156,17,2) AS s1#164, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#157,17,2) AS s2#165, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#158,17,2) AS s3#166] +Input [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] +Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#130)), sum(UnscaledValue(ss_list_price#131)), sum(UnscaledValue(ss_coupon_amt#132))] +Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#130))#200, sum(UnscaledValue(ss_list_price#131))#201, sum(UnscaledValue(ss_coupon_amt#132))#202] +Results [8]: [i_item_sk#189 AS item_sk#203, s_store_name#156 AS store_name#204, s_zip#157 AS store_zip#205, d_year#153 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#130))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#131))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#132))#202,17,2) AS s3#210] (184) Exchange -Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] -Arguments: hashpartitioning(item_sk#159, store_name#160, store_zip#161, 5), ENSURE_REQUIREMENTS, [id=#167] +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] (185) Sort [codegen id : 52] -Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] -Arguments: [item_sk#159 ASC NULLS FIRST, store_name#160 ASC NULLS FIRST, store_zip#161 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 (186) SortMergeJoin [codegen id : 53] Left keys [3]: [item_sk#105, store_name#106, store_zip#107] -Right keys [3]: [item_sk#159, store_name#160, store_zip#161] -Join condition: (cnt#163 <= cnt#117) +Right keys [3]: [item_sk#203, store_name#204, store_zip#205] +Join condition: (cnt#207 <= cnt#117) (187) Project [codegen id : 53] -Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] +Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] (188) Exchange -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#168] +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] (189) Sort [codegen id : 54] -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1028,11 +1028,11 @@ ReusedExchange (190) (190) ReusedExchange [Reuses operator id: 38] Output [2]: [d_date_sk#42, d_year#43] -Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#122 +Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#133 IN dynamicpruning#134 ReusedExchange (191) (191) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#42, d_year#43] +Output [2]: [d_date_sk#152, d_year#153] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt index 41c2ffcc75e7a..84e49ab9373e4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt @@ -130,135 +130,135 @@ Input [4]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17 Condition : isnotnull(ss_store_sk#15) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#18] (19) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#17] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#18] Join condition: None (20) Project [codegen id : 4] Output [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] -Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#6] +Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#18] (21) HashAggregate [codegen id : 4] Input [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] +Aggregate Attributes [1]: [sum#19] +Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] (22) Exchange -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] -Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] +Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#21] (23) HashAggregate [codegen id : 5] -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#21] -Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#21,17,2) AS revenue#22] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#22] +Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#22,17,2) AS revenue#23] (24) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#15, revenue#22] +Input [2]: [ss_store_sk#15, revenue#23] Keys [1]: [ss_store_sk#15] -Functions [1]: [partial_avg(revenue#22)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ss_store_sk#15, sum#25, count#26] +Functions [1]: [partial_avg(revenue#23)] +Aggregate Attributes [2]: [sum#24, count#25] +Results [3]: [ss_store_sk#15, sum#26, count#27] (25) Exchange -Input [3]: [ss_store_sk#15, sum#25, count#26] -Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ss_store_sk#15, sum#26, count#27] +Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#28] (26) HashAggregate [codegen id : 6] -Input [3]: [ss_store_sk#15, sum#25, count#26] +Input [3]: [ss_store_sk#15, sum#26, count#27] Keys [1]: [ss_store_sk#15] -Functions [1]: [avg(revenue#22)] -Aggregate Attributes [1]: [avg(revenue#22)#28] -Results [2]: [ss_store_sk#15, avg(revenue#22)#28 AS ave#29] +Functions [1]: [avg(revenue#23)] +Aggregate Attributes [1]: [avg(revenue#23)#29] +Results [2]: [ss_store_sk#15, avg(revenue#23)#29 AS ave#30] (27) BroadcastExchange -Input [2]: [ss_store_sk#15, ave#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] +Input [2]: [ss_store_sk#15, ave#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (28) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] Right keys [1]: [ss_store_sk#15] -Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#29)), DecimalType(23,7), true)) +Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#30)), DecimalType(23,7), true)) (29) Project [codegen id : 8] Output [3]: [ss_store_sk#2, ss_item_sk#1, revenue#13] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#29] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#30] (30) Scan parquet default.store -Output [2]: [s_store_sk#31, s_store_name#32] +Output [2]: [s_store_sk#32, s_store_name#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#31, s_store_name#32] +Input [2]: [s_store_sk#32, s_store_name#33] (32) Filter [codegen id : 7] -Input [2]: [s_store_sk#31, s_store_name#32] -Condition : isnotnull(s_store_sk#31) +Input [2]: [s_store_sk#32, s_store_name#33] +Condition : isnotnull(s_store_sk#32) (33) BroadcastExchange -Input [2]: [s_store_sk#31, s_store_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +Input [2]: [s_store_sk#32, s_store_name#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#31] +Right keys [1]: [s_store_sk#32] Join condition: None (35) Project [codegen id : 8] -Output [3]: [ss_item_sk#1, revenue#13, s_store_name#32] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#31, s_store_name#32] +Output [3]: [ss_item_sk#1, revenue#13, s_store_name#33] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#32, s_store_name#33] (36) Exchange -Input [3]: [ss_item_sk#1, revenue#13, s_store_name#32] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [ss_item_sk#1, revenue#13, s_store_name#33] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#35] (37) Sort [codegen id : 9] -Input [3]: [ss_item_sk#1, revenue#13, s_store_name#32] +Input [3]: [ss_item_sk#1, revenue#13, s_store_name#33] Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (38) Scan parquet default.item -Output [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Output [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 10] -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] (40) Filter [codegen id : 10] -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Condition : isnotnull(i_item_sk#35) +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Condition : isnotnull(i_item_sk#36) (41) Exchange -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Arguments: hashpartitioning(i_item_sk#35, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Arguments: hashpartitioning(i_item_sk#36, 5), ENSURE_REQUIREMENTS, [id=#41] (42) Sort [codegen id : 11] -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Arguments: [i_item_sk#35 ASC NULLS FIRST], false, 0 +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Arguments: [i_item_sk#36 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 12] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#35] +Right keys [1]: [i_item_sk#36] Join condition: None (44) Project [codegen id : 12] -Output [6]: [s_store_name#32, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Input [8]: [ss_item_sk#1, revenue#13, s_store_name#32, i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Output [6]: [s_store_name#33, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Input [8]: [ss_item_sk#1, revenue#13, s_store_name#33, i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] (45) TakeOrderedAndProject -Input [6]: [s_store_name#32, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Arguments: 100, [s_store_name#32 ASC NULLS FIRST, i_item_desc#36 ASC NULLS FIRST], [s_store_name#32, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [6]: [s_store_name#33, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Arguments: 100, [s_store_name#33 ASC NULLS FIRST, i_item_desc#37 ASC NULLS FIRST], [s_store_name#33, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt index 52de9873db590..45c7c051601c5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt @@ -181,65 +181,65 @@ Input [4]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26 Condition : isnotnull(ss_store_sk#24) (30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#27] (31) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#27] Join condition: None (32) Project [codegen id : 6] Output [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] -Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#8] +Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#27] (33) HashAggregate [codegen id : 6] Input [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#27] -Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] +Aggregate Attributes [1]: [sum#28] +Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] (34) Exchange -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] -Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] +Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#30] (35) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#30] -Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#30,17,2) AS revenue#31] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] +Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS revenue#32] (36) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#24, revenue#31] +Input [2]: [ss_store_sk#24, revenue#32] Keys [1]: [ss_store_sk#24] -Functions [1]: [partial_avg(revenue#31)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [3]: [ss_store_sk#24, sum#34, count#35] +Functions [1]: [partial_avg(revenue#32)] +Aggregate Attributes [2]: [sum#33, count#34] +Results [3]: [ss_store_sk#24, sum#35, count#36] (37) Exchange -Input [3]: [ss_store_sk#24, sum#34, count#35] -Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ss_store_sk#24, sum#35, count#36] +Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#37] (38) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#24, sum#34, count#35] +Input [3]: [ss_store_sk#24, sum#35, count#36] Keys [1]: [ss_store_sk#24] -Functions [1]: [avg(revenue#31)] -Aggregate Attributes [1]: [avg(revenue#31)#37] -Results [2]: [ss_store_sk#24, avg(revenue#31)#37 AS ave#38] +Functions [1]: [avg(revenue#32)] +Aggregate Attributes [1]: [avg(revenue#32)#38] +Results [2]: [ss_store_sk#24, avg(revenue#32)#38 AS ave#39] (39) BroadcastExchange -Input [2]: [ss_store_sk#24, ave#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Input [2]: [ss_store_sk#24, ave#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#24] -Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#38)), DecimalType(23,7), true)) +Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#39)), DecimalType(23,7), true)) (41) Project [codegen id : 9] Output [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#38] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#39] (42) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt index 51298b80bbbbe..8c342961cf970 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt @@ -221,70 +221,70 @@ Input [7]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_qu Condition : ((isnotnull(cs_warehouse_sk#176) AND isnotnull(cs_sold_time_sk#174)) AND isnotnull(cs_ship_mode_sk#175)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [sm_ship_mode_sk#9] +Output [1]: [sm_ship_mode_sk#181] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_mode_sk#175] -Right keys [1]: [sm_ship_mode_sk#9] +Right keys [1]: [sm_ship_mode_sk#181] Join condition: None (38) Project [codegen id : 11] Output [6]: [cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180] -Input [8]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, sm_ship_mode_sk#9] +Input [8]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, sm_ship_mode_sk#181] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [t_time_sk#12] +Output [1]: [t_time_sk#182] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_time_sk#174] -Right keys [1]: [t_time_sk#12] +Right keys [1]: [t_time_sk#182] Join condition: None (41) Project [codegen id : 11] Output [5]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180] -Input [7]: [cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, t_time_sk#12] +Input [7]: [cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, t_time_sk#182] (42) ReusedExchange [Reuses operator id: 21] -Output [3]: [d_date_sk#15, d_year#16, d_moy#17] +Output [3]: [d_date_sk#183, d_year#184, d_moy#185] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#180] -Right keys [1]: [d_date_sk#15] +Right keys [1]: [d_date_sk#183] Join condition: None (44) Project [codegen id : 11] -Output [6]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#16, d_moy#17] -Input [8]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, d_date_sk#15, d_year#16, d_moy#17] +Output [6]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#184, d_moy#185] +Input [8]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, d_date_sk#183, d_year#184, d_moy#185] (45) ReusedExchange [Reuses operator id: 27] -Output [7]: [w_warehouse_sk#19, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25] +Output [7]: [w_warehouse_sk#186, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192] (46) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_warehouse_sk#176] -Right keys [1]: [w_warehouse_sk#19] +Right keys [1]: [w_warehouse_sk#186] Join condition: None (47) Project [codegen id : 11] -Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, d_moy#17] -Input [13]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#16, d_moy#17, w_warehouse_sk#19, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25] +Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, d_moy#185] +Input [13]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#184, d_moy#185, w_warehouse_sk#186, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192] (48) HashAggregate [codegen id : 11] -Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, d_moy#17] -Keys [7]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16] -Functions [24]: [partial_sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228] -Results [55]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] +Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, d_moy#185] +Keys [7]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184] +Functions [24]: [partial_sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [48]: [sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240] +Results [55]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] (49) Exchange -Input [55]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] -Arguments: hashpartitioning(w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#277] +Input [55]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] +Arguments: hashpartitioning(w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, 5), ENSURE_REQUIREMENTS, [id=#289] (50) HashAggregate [codegen id : 12] -Input [55]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] -Keys [7]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16] -Functions [24]: [sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301] -Results [32]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, DHL,BARIAN AS ship_carriers#302, d_year#16 AS year#303, sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278 AS jan_sales#304, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279 AS feb_sales#305, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280 AS mar_sales#306, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281 AS apr_sales#307, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282 AS may_sales#308, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283 AS jun_sales#309, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284 AS jul_sales#310, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285 AS aug_sales#311, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286 AS sep_sales#312, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287 AS oct_sales#313, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288 AS nov_sales#314, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289 AS dec_sales#315, sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_net#316, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_net#317, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_net#318, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_net#319, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_net#320, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_net#321, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_net#322, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_net#323, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_net#324, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_net#325, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_net#326, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_net#327] +Input [55]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] +Keys [7]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184] +Functions [24]: [sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313] +Results [32]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, DHL,BARIAN AS ship_carriers#314, d_year#184 AS year#315, sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_sales#316, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_sales#317, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_sales#318, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_sales#319, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_sales#320, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_sales#321, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_sales#322, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_sales#323, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_sales#324, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_sales#325, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_sales#326, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_sales#327, sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302 AS jan_net#328, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303 AS feb_net#329, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304 AS mar_net#330, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305 AS apr_net#331, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306 AS may_net#332, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307 AS jun_net#333, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308 AS jul_net#334, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309 AS aug_net#335, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310 AS sep_net#336, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311 AS oct_net#337, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312 AS nov_net#338, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313 AS dec_net#339] (51) Union @@ -292,23 +292,23 @@ Results [32]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23 Input [32]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#150, feb_sales#151, mar_sales#152, apr_sales#153, may_sales#154, jun_sales#155, jul_sales#156, aug_sales#157, sep_sales#158, oct_sales#159, nov_sales#160, dec_sales#161, jan_net#162, feb_net#163, mar_net#164, apr_net#165, may_net#166, jun_net#167, jul_net#168, aug_net#169, sep_net#170, oct_net#171, nov_net#172, dec_net#173] Keys [8]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149] Functions [36]: [partial_sum(jan_sales#150), partial_sum(feb_sales#151), partial_sum(mar_sales#152), partial_sum(apr_sales#153), partial_sum(may_sales#154), partial_sum(jun_sales#155), partial_sum(jul_sales#156), partial_sum(aug_sales#157), partial_sum(sep_sales#158), partial_sum(oct_sales#159), partial_sum(nov_sales#160), partial_sum(dec_sales#161), partial_sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(jan_net#162), partial_sum(feb_net#163), partial_sum(mar_net#164), partial_sum(apr_net#165), partial_sum(may_net#166), partial_sum(jun_net#167), partial_sum(jul_net#168), partial_sum(aug_net#169), partial_sum(sep_net#170), partial_sum(oct_net#171), partial_sum(nov_net#172), partial_sum(dec_net#173)] -Aggregate Attributes [72]: [sum#328, isEmpty#329, sum#330, isEmpty#331, sum#332, isEmpty#333, sum#334, isEmpty#335, sum#336, isEmpty#337, sum#338, isEmpty#339, sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399] -Results [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] +Aggregate Attributes [72]: [sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411] +Results [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] (53) Exchange -Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] -Arguments: hashpartitioning(w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#472] +Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] +Arguments: hashpartitioning(w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#484] (54) HashAggregate [codegen id : 14] -Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] +Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] Keys [8]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149] Functions [36]: [sum(jan_sales#150), sum(feb_sales#151), sum(mar_sales#152), sum(apr_sales#153), sum(may_sales#154), sum(jun_sales#155), sum(jul_sales#156), sum(aug_sales#157), sum(sep_sales#158), sum(oct_sales#159), sum(nov_sales#160), sum(dec_sales#161), sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(jan_net#162), sum(feb_net#163), sum(mar_net#164), sum(apr_net#165), sum(may_net#166), sum(jun_net#167), sum(jul_net#168), sum(aug_net#169), sum(sep_net#170), sum(oct_net#171), sum(nov_net#172), sum(dec_net#173)] -Aggregate Attributes [36]: [sum(jan_sales#150)#473, sum(feb_sales#151)#474, sum(mar_sales#152)#475, sum(apr_sales#153)#476, sum(may_sales#154)#477, sum(jun_sales#155)#478, sum(jul_sales#156)#479, sum(aug_sales#157)#480, sum(sep_sales#158)#481, sum(oct_sales#159)#482, sum(nov_sales#160)#483, sum(dec_sales#161)#484, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496, sum(jan_net#162)#497, sum(feb_net#163)#498, sum(mar_net#164)#499, sum(apr_net#165)#500, sum(may_net#166)#501, sum(jun_net#167)#502, sum(jul_net#168)#503, sum(aug_net#169)#504, sum(sep_net#170)#505, sum(oct_net#171)#506, sum(nov_net#172)#507, sum(dec_net#173)#508] -Results [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum(jan_sales#150)#473 AS jan_sales#509, sum(feb_sales#151)#474 AS feb_sales#510, sum(mar_sales#152)#475 AS mar_sales#511, sum(apr_sales#153)#476 AS apr_sales#512, sum(may_sales#154)#477 AS may_sales#513, sum(jun_sales#155)#478 AS jun_sales#514, sum(jul_sales#156)#479 AS jul_sales#515, sum(aug_sales#157)#480 AS aug_sales#516, sum(sep_sales#158)#481 AS sep_sales#517, sum(oct_sales#159)#482 AS oct_sales#518, sum(nov_sales#160)#483 AS nov_sales#519, sum(dec_sales#161)#484 AS dec_sales#520, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485 AS jan_sales_per_sq_foot#521, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486 AS feb_sales_per_sq_foot#522, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487 AS mar_sales_per_sq_foot#523, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488 AS apr_sales_per_sq_foot#524, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489 AS may_sales_per_sq_foot#525, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490 AS jun_sales_per_sq_foot#526, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491 AS jul_sales_per_sq_foot#527, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492 AS aug_sales_per_sq_foot#528, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493 AS sep_sales_per_sq_foot#529, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494 AS oct_sales_per_sq_foot#530, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495 AS nov_sales_per_sq_foot#531, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496 AS dec_sales_per_sq_foot#532, sum(jan_net#162)#497 AS jan_net#533, sum(feb_net#163)#498 AS feb_net#534, sum(mar_net#164)#499 AS mar_net#535, sum(apr_net#165)#500 AS apr_net#536, sum(may_net#166)#501 AS may_net#537, sum(jun_net#167)#502 AS jun_net#538, sum(jul_net#168)#503 AS jul_net#539, sum(aug_net#169)#504 AS aug_net#540, sum(sep_net#170)#505 AS sep_net#541, sum(oct_net#171)#506 AS oct_net#542, sum(nov_net#172)#507 AS nov_net#543, sum(dec_net#173)#508 AS dec_net#544] +Aggregate Attributes [36]: [sum(jan_sales#150)#485, sum(feb_sales#151)#486, sum(mar_sales#152)#487, sum(apr_sales#153)#488, sum(may_sales#154)#489, sum(jun_sales#155)#490, sum(jul_sales#156)#491, sum(aug_sales#157)#492, sum(sep_sales#158)#493, sum(oct_sales#159)#494, sum(nov_sales#160)#495, sum(dec_sales#161)#496, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508, sum(jan_net#162)#509, sum(feb_net#163)#510, sum(mar_net#164)#511, sum(apr_net#165)#512, sum(may_net#166)#513, sum(jun_net#167)#514, sum(jul_net#168)#515, sum(aug_net#169)#516, sum(sep_net#170)#517, sum(oct_net#171)#518, sum(nov_net#172)#519, sum(dec_net#173)#520] +Results [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum(jan_sales#150)#485 AS jan_sales#521, sum(feb_sales#151)#486 AS feb_sales#522, sum(mar_sales#152)#487 AS mar_sales#523, sum(apr_sales#153)#488 AS apr_sales#524, sum(may_sales#154)#489 AS may_sales#525, sum(jun_sales#155)#490 AS jun_sales#526, sum(jul_sales#156)#491 AS jul_sales#527, sum(aug_sales#157)#492 AS aug_sales#528, sum(sep_sales#158)#493 AS sep_sales#529, sum(oct_sales#159)#494 AS oct_sales#530, sum(nov_sales#160)#495 AS nov_sales#531, sum(dec_sales#161)#496 AS dec_sales#532, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497 AS jan_sales_per_sq_foot#533, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498 AS feb_sales_per_sq_foot#534, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499 AS mar_sales_per_sq_foot#535, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500 AS apr_sales_per_sq_foot#536, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501 AS may_sales_per_sq_foot#537, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502 AS jun_sales_per_sq_foot#538, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503 AS jul_sales_per_sq_foot#539, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504 AS aug_sales_per_sq_foot#540, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505 AS sep_sales_per_sq_foot#541, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506 AS oct_sales_per_sq_foot#542, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507 AS nov_sales_per_sq_foot#543, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508 AS dec_sales_per_sq_foot#544, sum(jan_net#162)#509 AS jan_net#545, sum(feb_net#163)#510 AS feb_net#546, sum(mar_net#164)#511 AS mar_net#547, sum(apr_net#165)#512 AS apr_net#548, sum(may_net#166)#513 AS may_net#549, sum(jun_net#167)#514 AS jun_net#550, sum(jul_net#168)#515 AS jul_net#551, sum(aug_net#169)#516 AS aug_net#552, sum(sep_net#170)#517 AS sep_net#553, sum(oct_net#171)#518 AS oct_net#554, sum(nov_net#172)#519 AS nov_net#555, sum(dec_net#173)#520 AS dec_net#556] (55) TakeOrderedAndProject -Input [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, may_sales_per_sq_foot#525, jun_sales_per_sq_foot#526, jul_sales_per_sq_foot#527, aug_sales_per_sq_foot#528, sep_sales_per_sq_foot#529, oct_sales_per_sq_foot#530, nov_sales_per_sq_foot#531, dec_sales_per_sq_foot#532, jan_net#533, feb_net#534, mar_net#535, apr_net#536, may_net#537, jun_net#538, jul_net#539, aug_net#540, sep_net#541, oct_net#542, nov_net#543, dec_net#544] -Arguments: 100, [w_warehouse_name#20 ASC NULLS FIRST], [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, ... 20 more fields] +Input [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, may_sales_per_sq_foot#537, jun_sales_per_sq_foot#538, jul_sales_per_sq_foot#539, aug_sales_per_sq_foot#540, sep_sales_per_sq_foot#541, oct_sales_per_sq_foot#542, nov_sales_per_sq_foot#543, dec_sales_per_sq_foot#544, jan_net#545, feb_net#546, mar_net#547, apr_net#548, may_net#549, jun_net#550, jul_net#551, aug_net#552, sep_net#553, oct_net#554, nov_net#555, dec_net#556] +Arguments: 100, [w_warehouse_name#20 ASC NULLS FIRST], [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, ... 20 more fields] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt index 3d44b22396486..832965c1aaa31 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt @@ -221,70 +221,70 @@ Input [7]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_qu Condition : ((isnotnull(cs_warehouse_sk#176) AND isnotnull(cs_sold_time_sk#174)) AND isnotnull(cs_ship_mode_sk#175)) (36) ReusedExchange [Reuses operator id: 7] -Output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Output [7]: [w_warehouse_sk#181, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_warehouse_sk#176] -Right keys [1]: [w_warehouse_sk#9] +Right keys [1]: [w_warehouse_sk#181] Join condition: None (38) Project [codegen id : 11] -Output [12]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Output [12]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187] +Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_sk#181, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187] (39) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] +Output [3]: [d_date_sk#188, d_year#189, d_moy#190] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#180] -Right keys [1]: [d_date_sk#17] +Right keys [1]: [d_date_sk#188] Join condition: None (41) Project [codegen id : 11] -Output [13]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] -Input [15]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_date_sk#17, d_year#18, d_moy#19] +Output [13]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] +Input [15]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_date_sk#188, d_year#189, d_moy#190] (42) ReusedExchange [Reuses operator id: 20] -Output [1]: [t_time_sk#21] +Output [1]: [t_time_sk#191] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_time_sk#174] -Right keys [1]: [t_time_sk#21] +Right keys [1]: [t_time_sk#191] Join condition: None (44) Project [codegen id : 11] -Output [12]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] -Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19, t_time_sk#21] +Output [12]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] +Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190, t_time_sk#191] (45) ReusedExchange [Reuses operator id: 27] -Output [1]: [sm_ship_mode_sk#24] +Output [1]: [sm_ship_mode_sk#192] (46) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_mode_sk#175] -Right keys [1]: [sm_ship_mode_sk#24] +Right keys [1]: [sm_ship_mode_sk#192] Join condition: None (47) Project [codegen id : 11] -Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] -Input [13]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#24] +Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] +Input [13]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190, sm_ship_mode_sk#192] (48) HashAggregate [codegen id : 11] -Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18] -Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228] -Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] +Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] +Keys [7]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189] +Functions [24]: [partial_sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [48]: [sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240] +Results [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] (49) Exchange -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#277] +Input [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] +Arguments: hashpartitioning(w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, 5), ENSURE_REQUIREMENTS, [id=#289] (50) HashAggregate [codegen id : 12] -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18] -Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301] -Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, DHL,BARIAN AS ship_carriers#302, d_year#18 AS year#303, sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278 AS jan_sales#304, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279 AS feb_sales#305, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280 AS mar_sales#306, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281 AS apr_sales#307, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282 AS may_sales#308, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283 AS jun_sales#309, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284 AS jul_sales#310, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285 AS aug_sales#311, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286 AS sep_sales#312, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287 AS oct_sales#313, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288 AS nov_sales#314, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289 AS dec_sales#315, sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_net#316, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_net#317, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_net#318, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_net#319, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_net#320, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_net#321, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_net#322, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_net#323, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_net#324, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_net#325, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_net#326, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_net#327] +Input [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] +Keys [7]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189] +Functions [24]: [sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313] +Results [32]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, DHL,BARIAN AS ship_carriers#314, d_year#189 AS year#315, sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_sales#316, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_sales#317, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_sales#318, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_sales#319, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_sales#320, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_sales#321, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_sales#322, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_sales#323, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_sales#324, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_sales#325, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_sales#326, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_sales#327, sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302 AS jan_net#328, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303 AS feb_net#329, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304 AS mar_net#330, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305 AS apr_net#331, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306 AS may_net#332, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307 AS jun_net#333, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308 AS jul_net#334, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309 AS aug_net#335, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310 AS sep_net#336, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311 AS oct_net#337, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312 AS nov_net#338, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313 AS dec_net#339] (51) Union @@ -292,23 +292,23 @@ Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13 Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#150, feb_sales#151, mar_sales#152, apr_sales#153, may_sales#154, jun_sales#155, jul_sales#156, aug_sales#157, sep_sales#158, oct_sales#159, nov_sales#160, dec_sales#161, jan_net#162, feb_net#163, mar_net#164, apr_net#165, may_net#166, jun_net#167, jul_net#168, aug_net#169, sep_net#170, oct_net#171, nov_net#172, dec_net#173] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149] Functions [36]: [partial_sum(jan_sales#150), partial_sum(feb_sales#151), partial_sum(mar_sales#152), partial_sum(apr_sales#153), partial_sum(may_sales#154), partial_sum(jun_sales#155), partial_sum(jul_sales#156), partial_sum(aug_sales#157), partial_sum(sep_sales#158), partial_sum(oct_sales#159), partial_sum(nov_sales#160), partial_sum(dec_sales#161), partial_sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(jan_net#162), partial_sum(feb_net#163), partial_sum(mar_net#164), partial_sum(apr_net#165), partial_sum(may_net#166), partial_sum(jun_net#167), partial_sum(jul_net#168), partial_sum(aug_net#169), partial_sum(sep_net#170), partial_sum(oct_net#171), partial_sum(nov_net#172), partial_sum(dec_net#173)] -Aggregate Attributes [72]: [sum#328, isEmpty#329, sum#330, isEmpty#331, sum#332, isEmpty#333, sum#334, isEmpty#335, sum#336, isEmpty#337, sum#338, isEmpty#339, sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399] -Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] +Aggregate Attributes [72]: [sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411] +Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] (53) Exchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#472] +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#484] (54) HashAggregate [codegen id : 14] -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149] Functions [36]: [sum(jan_sales#150), sum(feb_sales#151), sum(mar_sales#152), sum(apr_sales#153), sum(may_sales#154), sum(jun_sales#155), sum(jul_sales#156), sum(aug_sales#157), sum(sep_sales#158), sum(oct_sales#159), sum(nov_sales#160), sum(dec_sales#161), sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(jan_net#162), sum(feb_net#163), sum(mar_net#164), sum(apr_net#165), sum(may_net#166), sum(jun_net#167), sum(jul_net#168), sum(aug_net#169), sum(sep_net#170), sum(oct_net#171), sum(nov_net#172), sum(dec_net#173)] -Aggregate Attributes [36]: [sum(jan_sales#150)#473, sum(feb_sales#151)#474, sum(mar_sales#152)#475, sum(apr_sales#153)#476, sum(may_sales#154)#477, sum(jun_sales#155)#478, sum(jul_sales#156)#479, sum(aug_sales#157)#480, sum(sep_sales#158)#481, sum(oct_sales#159)#482, sum(nov_sales#160)#483, sum(dec_sales#161)#484, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496, sum(jan_net#162)#497, sum(feb_net#163)#498, sum(mar_net#164)#499, sum(apr_net#165)#500, sum(may_net#166)#501, sum(jun_net#167)#502, sum(jul_net#168)#503, sum(aug_net#169)#504, sum(sep_net#170)#505, sum(oct_net#171)#506, sum(nov_net#172)#507, sum(dec_net#173)#508] -Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum(jan_sales#150)#473 AS jan_sales#509, sum(feb_sales#151)#474 AS feb_sales#510, sum(mar_sales#152)#475 AS mar_sales#511, sum(apr_sales#153)#476 AS apr_sales#512, sum(may_sales#154)#477 AS may_sales#513, sum(jun_sales#155)#478 AS jun_sales#514, sum(jul_sales#156)#479 AS jul_sales#515, sum(aug_sales#157)#480 AS aug_sales#516, sum(sep_sales#158)#481 AS sep_sales#517, sum(oct_sales#159)#482 AS oct_sales#518, sum(nov_sales#160)#483 AS nov_sales#519, sum(dec_sales#161)#484 AS dec_sales#520, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485 AS jan_sales_per_sq_foot#521, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486 AS feb_sales_per_sq_foot#522, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487 AS mar_sales_per_sq_foot#523, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488 AS apr_sales_per_sq_foot#524, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489 AS may_sales_per_sq_foot#525, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490 AS jun_sales_per_sq_foot#526, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491 AS jul_sales_per_sq_foot#527, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492 AS aug_sales_per_sq_foot#528, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493 AS sep_sales_per_sq_foot#529, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494 AS oct_sales_per_sq_foot#530, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495 AS nov_sales_per_sq_foot#531, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496 AS dec_sales_per_sq_foot#532, sum(jan_net#162)#497 AS jan_net#533, sum(feb_net#163)#498 AS feb_net#534, sum(mar_net#164)#499 AS mar_net#535, sum(apr_net#165)#500 AS apr_net#536, sum(may_net#166)#501 AS may_net#537, sum(jun_net#167)#502 AS jun_net#538, sum(jul_net#168)#503 AS jul_net#539, sum(aug_net#169)#504 AS aug_net#540, sum(sep_net#170)#505 AS sep_net#541, sum(oct_net#171)#506 AS oct_net#542, sum(nov_net#172)#507 AS nov_net#543, sum(dec_net#173)#508 AS dec_net#544] +Aggregate Attributes [36]: [sum(jan_sales#150)#485, sum(feb_sales#151)#486, sum(mar_sales#152)#487, sum(apr_sales#153)#488, sum(may_sales#154)#489, sum(jun_sales#155)#490, sum(jul_sales#156)#491, sum(aug_sales#157)#492, sum(sep_sales#158)#493, sum(oct_sales#159)#494, sum(nov_sales#160)#495, sum(dec_sales#161)#496, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508, sum(jan_net#162)#509, sum(feb_net#163)#510, sum(mar_net#164)#511, sum(apr_net#165)#512, sum(may_net#166)#513, sum(jun_net#167)#514, sum(jul_net#168)#515, sum(aug_net#169)#516, sum(sep_net#170)#517, sum(oct_net#171)#518, sum(nov_net#172)#519, sum(dec_net#173)#520] +Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum(jan_sales#150)#485 AS jan_sales#521, sum(feb_sales#151)#486 AS feb_sales#522, sum(mar_sales#152)#487 AS mar_sales#523, sum(apr_sales#153)#488 AS apr_sales#524, sum(may_sales#154)#489 AS may_sales#525, sum(jun_sales#155)#490 AS jun_sales#526, sum(jul_sales#156)#491 AS jul_sales#527, sum(aug_sales#157)#492 AS aug_sales#528, sum(sep_sales#158)#493 AS sep_sales#529, sum(oct_sales#159)#494 AS oct_sales#530, sum(nov_sales#160)#495 AS nov_sales#531, sum(dec_sales#161)#496 AS dec_sales#532, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497 AS jan_sales_per_sq_foot#533, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498 AS feb_sales_per_sq_foot#534, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499 AS mar_sales_per_sq_foot#535, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500 AS apr_sales_per_sq_foot#536, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501 AS may_sales_per_sq_foot#537, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502 AS jun_sales_per_sq_foot#538, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503 AS jul_sales_per_sq_foot#539, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504 AS aug_sales_per_sq_foot#540, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505 AS sep_sales_per_sq_foot#541, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506 AS oct_sales_per_sq_foot#542, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507 AS nov_sales_per_sq_foot#543, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508 AS dec_sales_per_sq_foot#544, sum(jan_net#162)#509 AS jan_net#545, sum(feb_net#163)#510 AS feb_net#546, sum(mar_net#164)#511 AS mar_net#547, sum(apr_net#165)#512 AS apr_net#548, sum(may_net#166)#513 AS may_net#549, sum(jun_net#167)#514 AS jun_net#550, sum(jul_net#168)#515 AS jul_net#551, sum(aug_net#169)#516 AS aug_net#552, sum(sep_net#170)#517 AS sep_net#553, sum(oct_net#171)#518 AS oct_net#554, sum(nov_net#172)#519 AS nov_net#555, sum(dec_net#173)#520 AS dec_net#556] (55) TakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, may_sales_per_sq_foot#525, jun_sales_per_sq_foot#526, jul_sales_per_sq_foot#527, aug_sales_per_sq_foot#528, sep_sales_per_sq_foot#529, oct_sales_per_sq_foot#530, nov_sales_per_sq_foot#531, dec_sales_per_sq_foot#532, jan_net#533, feb_net#534, mar_net#535, apr_net#536, may_net#537, jun_net#538, jul_net#539, aug_net#540, sep_net#541, oct_net#542, nov_net#543, dec_net#544] -Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, ... 20 more fields] +Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, may_sales_per_sq_foot#537, jun_sales_per_sq_foot#538, jul_sales_per_sq_foot#539, aug_sales_per_sq_foot#540, sep_sales_per_sq_foot#541, oct_sales_per_sq_foot#542, nov_sales_per_sq_foot#543, dec_sales_per_sq_foot#544, jan_net#545, feb_net#546, mar_net#547, apr_net#548, may_net#549, jun_net#550, jul_net#551, aug_net#552, sep_net#553, oct_net#554, nov_net#555, dec_net#556] +Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, ... 20 more fields] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt index 744fe4b5a594d..b6a609ec193b4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt @@ -230,55 +230,55 @@ Input [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_ Arguments: [ss_addr_sk#12 ASC NULLS FIRST], false, 0 (41) ReusedExchange [Reuses operator id: 9] -Output [2]: [ca_address_sk#6, ca_city#7] +Output [2]: [ca_address_sk#32, ca_city#33] (42) Sort [codegen id : 13] -Input [2]: [ca_address_sk#6, ca_city#7] -Arguments: [ca_address_sk#6 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#32, ca_city#33] +Arguments: [ca_address_sk#32 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 14] Left keys [1]: [ss_addr_sk#12] -Right keys [1]: [ca_address_sk#6] +Right keys [1]: [ca_address_sk#32] Join condition: None (44) Project [codegen id : 14] -Output [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#7] -Input [8]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_address_sk#6, ca_city#7] +Output [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#33] +Input [8]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_address_sk#32, ca_city#33] (45) HashAggregate [codegen id : 14] -Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#7] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] +Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#33] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#15)), partial_sum(UnscaledValue(ss_ext_list_price#16)), partial_sum(UnscaledValue(ss_ext_tax#17))] -Aggregate Attributes [3]: [sum#32, sum#33, sum#34] -Results [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#35, sum#36, sum#37] +Aggregate Attributes [3]: [sum#34, sum#35, sum#36] +Results [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33, sum#37, sum#38, sum#39] (46) HashAggregate [codegen id : 14] -Input [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#35, sum#36, sum#37] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] +Input [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33, sum#37, sum#38, sum#39] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#15)), sum(UnscaledValue(ss_ext_list_price#16)), sum(UnscaledValue(ss_ext_tax#17))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#15))#38, sum(UnscaledValue(ss_ext_list_price#16))#39, sum(UnscaledValue(ss_ext_tax#17))#40] -Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#7 AS bought_city#41, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#38,17,2) AS extended_price#42, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#16))#39,17,2) AS list_price#43, MakeDecimal(sum(UnscaledValue(ss_ext_tax#17))#40,17,2) AS extended_tax#44] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#15))#40, sum(UnscaledValue(ss_ext_list_price#16))#41, sum(UnscaledValue(ss_ext_tax#17))#42] +Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#33 AS bought_city#43, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#40,17,2) AS extended_price#44, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#16))#41,17,2) AS list_price#45, MakeDecimal(sum(UnscaledValue(ss_ext_tax#17))#42,17,2) AS extended_tax#46] (47) Exchange -Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#41, extended_price#42, list_price#43, extended_tax#44] -Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#43, extended_price#44, list_price#45, extended_tax#46] +Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#47] (48) Sort [codegen id : 15] -Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#41, extended_price#42, list_price#43, extended_tax#44] +Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#43, extended_price#44, list_price#45, extended_tax#46] Arguments: [ss_customer_sk#10 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 16] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#10] -Join condition: NOT (ca_city#7 = bought_city#41) +Join condition: NOT (ca_city#7 = bought_city#43) (50) Project [codegen id : 16] -Output [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#41, ss_ticket_number#14, extended_price#42, extended_tax#44, list_price#43] -Input [10]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#41, extended_price#42, list_price#43, extended_tax#44] +Output [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#43, ss_ticket_number#14, extended_price#44, extended_tax#46, list_price#45] +Input [10]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#43, extended_price#44, list_price#45, extended_tax#46] (51) TakeOrderedAndProject -Input [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#41, ss_ticket_number#14, extended_price#42, extended_tax#44, list_price#43] -Arguments: 100, [c_last_name#4 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#41, ss_ticket_number#14, extended_price#42, extended_tax#44, list_price#43] +Input [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#43, ss_ticket_number#14, extended_price#44, extended_tax#46, list_price#45] +Arguments: 100, [c_last_name#4 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#43, ss_ticket_number#14, extended_price#44, extended_tax#46, list_price#45] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt index 1a9fafda6250b..101e4f272f98a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt @@ -225,20 +225,20 @@ Output [8]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#3 Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_customer_sk#39, c_current_addr_sk#40, c_first_name#41, c_last_name#42] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#22, ca_city#23] +Output [2]: [ca_address_sk#44, ca_city#45] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#40] -Right keys [1]: [ca_address_sk#22] -Join condition: NOT (ca_city#23 = bought_city#35) +Right keys [1]: [ca_address_sk#44] +Join condition: NOT (ca_city#45 = bought_city#35) (42) Project [codegen id : 8] -Output [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#22, ca_city#23] +Output [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#44, ca_city#45] (43) TakeOrderedAndProject -Input [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt index 97d7a81baf2a7..a0532462eff64 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt @@ -139,20 +139,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#15] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#13] -Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] +Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] (23) Exchange Input [1]: [ws_bill_customer_sk#13] -Arguments: hashpartitioning(ws_bill_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#15] +Arguments: hashpartitioning(ws_bill_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#16] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#13] @@ -164,38 +164,38 @@ Right keys [1]: [ws_bill_customer_sk#13] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#19] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#16] -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] +Output [1]: [cs_ship_customer_sk#17] +Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] (31) Exchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: hashpartitioning(cs_ship_customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [1]: [cs_ship_customer_sk#17] +Arguments: hashpartitioning(cs_ship_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#20] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#16] -Arguments: [cs_ship_customer_sk#16 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#17] +Arguments: [cs_ship_customer_sk#17 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#16] +Right keys [1]: [cs_ship_customer_sk#17] Join condition: None (34) Project [codegen id : 13] @@ -203,84 +203,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (35) Scan parquet default.customer_address -Output [2]: [ca_address_sk#19, ca_state#20] +Output [2]: [ca_address_sk#21, ca_state#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 12] -Input [2]: [ca_address_sk#19, ca_state#20] +Input [2]: [ca_address_sk#21, ca_state#22] (37) Filter [codegen id : 12] -Input [2]: [ca_address_sk#19, ca_state#20] -Condition : (ca_state#20 IN (KY,GA,NM) AND isnotnull(ca_address_sk#19)) +Input [2]: [ca_address_sk#21, ca_state#22] +Condition : (ca_state#22 IN (KY,GA,NM) AND isnotnull(ca_address_sk#21)) (38) Project [codegen id : 12] -Output [1]: [ca_address_sk#19] -Input [2]: [ca_address_sk#19, ca_state#20] +Output [1]: [ca_address_sk#21] +Input [2]: [ca_address_sk#21, ca_state#22] (39) BroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [ca_address_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#19] +Right keys [1]: [ca_address_sk#21] Join condition: None (41) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21] (42) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (43) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (44) ColumnarToRow -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] (45) Filter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Condition : isnotnull(cd_demo_sk#23) +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Condition : isnotnull(cd_demo_sk#25) (46) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#25] Join condition: None (47) Project [codegen id : 14] -Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Output [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] (48) HashAggregate [codegen id : 14] -Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] +Aggregate Attributes [1]: [count#31] +Results [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] (49) Exchange -Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] -Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] +Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, 5), ENSURE_REQUIREMENTS, [id=#33] (50) HashAggregate [codegen id : 15] -Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] -Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] +Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#32 AS cnt1#33, cd_purchase_estimate#27, count(1)#32 AS cnt2#34, cd_credit_rating#28, count(1)#32 AS cnt3#35] +Aggregate Attributes [1]: [count(1)#34] +Results [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#34 AS cnt1#35, cd_purchase_estimate#29, count(1)#34 AS cnt2#36, cd_credit_rating#30, count(1)#34 AS cnt3#37] (51) TakeOrderedAndProject -Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#33, cd_purchase_estimate#27, cnt2#34, cd_credit_rating#28, cnt3#35] -Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#33, cd_purchase_estimate#27, cnt2#34, cd_credit_rating#28, cnt3#35] +Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#35, cd_purchase_estimate#29, cnt2#36, cd_credit_rating#30, cnt3#37] +Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#35, cd_purchase_estimate#29, cnt2#36, cd_credit_rating#30, cnt3#37] ===== Subqueries ===== @@ -293,6 +293,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index b5307abaf688c..d0b3c2231d997 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -122,20 +122,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#12] -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] @@ -143,34 +143,34 @@ Right keys [1]: [ws_bill_customer_sk#12] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#6)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#18] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#18] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#15] -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] +Output [1]: [cs_ship_customer_sk#16] +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#18] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Input [1]: [cs_ship_customer_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#15] +Right keys [1]: [cs_ship_customer_sk#16] Join condition: None (29) Project [codegen id : 9] @@ -178,84 +178,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (30) Scan parquet default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_state#19] +Input [2]: [ca_address_sk#20, ca_state#21] (32) Filter [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : (ca_state#21 IN (KY,GA,NM) AND isnotnull(ca_address_sk#20)) (33) Project [codegen id : 7] -Output [1]: [ca_address_sk#18] -Input [2]: [ca_address_sk#18, ca_state#19] +Output [1]: [ca_address_sk#20] +Input [2]: [ca_address_sk#20, ca_state#21] (34) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +Input [1]: [ca_address_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#18] +Right keys [1]: [ca_address_sk#20] Join condition: None (36) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] (37) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] (39) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -Condition : isnotnull(cd_demo_sk#21) +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Condition : isnotnull(cd_demo_sk#23) (40) BroadcastExchange -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#21] +Right keys [1]: [cd_demo_sk#23] Join condition: None (42) Project [codegen id : 9] -Output [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] (43) HashAggregate [codegen id : 9] -Input [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] +Aggregate Attributes [1]: [count#30] +Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#31] (44) Exchange -Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] -Arguments: hashpartitioning(cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#31] +Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, 5), ENSURE_REQUIREMENTS, [id=#32] (45) HashAggregate [codegen id : 10] -Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] -Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#31] +Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, count(1)#31 AS cnt1#32, cd_purchase_estimate#25, count(1)#31 AS cnt2#33, cd_credit_rating#26, count(1)#31 AS cnt3#34] +Aggregate Attributes [1]: [count(1)#33] +Results [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#33 AS cnt1#34, cd_purchase_estimate#27, count(1)#33 AS cnt2#35, cd_credit_rating#28, count(1)#33 AS cnt3#36] (46) TakeOrderedAndProject -Input [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#32, cd_purchase_estimate#25, cnt2#33, cd_credit_rating#26, cnt3#34] -Arguments: 100, [cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_education_status#24 ASC NULLS FIRST, cd_purchase_estimate#25 ASC NULLS FIRST, cd_credit_rating#26 ASC NULLS FIRST], [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#32, cd_purchase_estimate#25, cnt2#33, cd_credit_rating#26, cnt3#34] +Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#34, cd_purchase_estimate#27, cnt2#35, cd_credit_rating#28, cnt3#36] +Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#34, cd_purchase_estimate#27, cnt2#35, cd_credit_rating#28, cnt3#36] ===== Subqueries ===== @@ -268,6 +268,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt index ab2cc784a5f22..b138d059eaecb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnul Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt index 9c914d56f46a2..5f6002b84f411 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnul Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index d31dbc3498ead..ddce6bf2e14f4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -109,109 +109,109 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] (16) Filter [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_store_sk#11) (17) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#13] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] -Output [2]: [ss_store_sk#1, ss_net_profit#2] -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] +Output [2]: [ss_store_sk#11, ss_net_profit#12] +Input [4]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13, d_date_sk#14] (20) Scan parquet default.store -Output [2]: [s_store_sk#8, s_state#10] +Output [2]: [s_store_sk#15, s_state#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#8, s_state#10] +Input [2]: [s_store_sk#15, s_state#16] (22) Filter [codegen id : 3] -Input [2]: [s_store_sk#8, s_state#10] -Condition : isnotnull(s_store_sk#8) +Input [2]: [s_store_sk#15, s_state#16] +Condition : isnotnull(s_store_sk#15) (23) BroadcastExchange -Input [2]: [s_store_sk#8, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +Input [2]: [s_store_sk#15, s_state#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#8] +Left keys [1]: [ss_store_sk#11] +Right keys [1]: [s_store_sk#15] Join condition: None (25) Project [codegen id : 4] -Output [2]: [ss_net_profit#2, s_state#10] -Input [4]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_state#10] +Output [2]: [ss_net_profit#12, s_state#16] +Input [4]: [ss_store_sk#11, ss_net_profit#12, s_store_sk#15, s_state#16] (26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#2, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#12] -Results [2]: [s_state#10, sum#13] +Input [2]: [ss_net_profit#12, s_state#16] +Keys [1]: [s_state#16] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum#18] +Results [2]: [s_state#16, sum#19] (27) Exchange -Input [2]: [s_state#10, sum#13] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [2]: [s_state#16, sum#19] +Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, [id=#20] (28) HashAggregate [codegen id : 5] -Input [2]: [s_state#10, sum#13] -Keys [1]: [s_state#10] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#15] -Results [3]: [s_state#10 AS s_state#16, s_state#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#15,17,2) AS _w2#17] +Input [2]: [s_state#16, sum#19] +Keys [1]: [s_state#16] +Functions [1]: [sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#12))#21] +Results [3]: [s_state#16 AS s_state#22, s_state#16, MakeDecimal(sum(UnscaledValue(ss_net_profit#12))#21,17,2) AS _w2#23] (29) Exchange -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, [id=#24] (30) Sort [codegen id : 6] -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [s_state#10 ASC NULLS FIRST, _w2#17 DESC NULLS LAST], false, 0 +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: [s_state#16 ASC NULLS FIRST, _w2#23 DESC NULLS LAST], false, 0 (31) Window -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#10], [_w2#17 DESC NULLS LAST] +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: [rank(_w2#23) windowspecdefinition(s_state#16, _w2#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#25], [s_state#16], [_w2#23 DESC NULLS LAST] (32) Filter [codegen id : 7] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) +Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] +Condition : (isnotnull(ranking#25) AND (ranking#25 <= 5)) (33) Project [codegen id : 7] -Output [1]: [s_state#16] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] +Output [1]: [s_state#22] +Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] (34) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#20] +Input [1]: [s_state#22] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#26] (35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#10] -Right keys [1]: [s_state#16] +Right keys [1]: [s_state#22] Join condition: None (36) BroadcastExchange Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] @@ -224,45 +224,45 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_county#9, s_state#10 (39) Expand [codegen id : 9] Input [3]: [ss_net_profit#2, s_state#10, s_county#9] -Arguments: [List(ss_net_profit#2, s_state#10, s_county#9, 0), List(ss_net_profit#2, s_state#10, null, 1), List(ss_net_profit#2, null, null, 3)], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] +Arguments: [List(ss_net_profit#2, s_state#10, s_county#9, 0), List(ss_net_profit#2, s_state#10, null, 1), List(ss_net_profit#2, null, null, 3)], [ss_net_profit#2, s_state#28, s_county#29, spark_grouping_id#30] (40) HashAggregate [codegen id : 9] -Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] +Input [4]: [ss_net_profit#2, s_state#28, s_county#29, spark_grouping_id#30] +Keys [3]: [s_state#28, s_county#29, spark_grouping_id#30] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#25] -Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Aggregate Attributes [1]: [sum#31] +Results [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] (41) Exchange -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] +Arguments: hashpartitioning(s_state#28, s_county#29, spark_grouping_id#30, 5), ENSURE_REQUIREMENTS, [id=#33] (42) HashAggregate [codegen id : 10] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] +Input [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] +Keys [3]: [s_state#28, s_county#29, spark_grouping_id#30] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#28] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS total_sum#29, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS _w3#33] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#34] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS total_sum#35, s_state#28, s_county#29, (cast((shiftright(spark_grouping_id#30, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint)) AS lochierarchy#36, (cast((shiftright(spark_grouping_id#30, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint)) AS _w1#37, CASE WHEN (cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint) = 0) THEN s_state#28 END AS _w2#38, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS _w3#39] (43) Exchange -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: hashpartitioning(_w1#37, _w2#38, 5), ENSURE_REQUIREMENTS, [id=#40] (44) Sort [codegen id : 11] -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w3#33 DESC NULLS LAST], false, 0 +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: [_w1#37 ASC NULLS FIRST, _w2#38 ASC NULLS FIRST, _w3#39 DESC NULLS LAST], false, 0 (45) Window -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: [rank(_w3#33) windowspecdefinition(_w1#31, _w2#32, _w3#33 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#35], [_w1#31, _w2#32], [_w3#33 DESC NULLS LAST] +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: [rank(_w3#39) windowspecdefinition(_w1#37, _w2#38, _w3#39 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [_w1#37, _w2#38], [_w3#39 DESC NULLS LAST] (46) Project [codegen id : 12] -Output [5]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] -Input [8]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33, rank_within_parent#35] +Output [5]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] +Input [8]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39, rank_within_parent#41] (47) TakeOrderedAndProject -Input [5]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#35 ASC NULLS FIRST], [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] +Input [5]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] +Arguments: 100, [lochierarchy#36 DESC NULLS LAST, CASE WHEN (lochierarchy#36 = 0) THEN s_state#28 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] ===== Subqueries ===== @@ -273,6 +273,6 @@ ReusedExchange (48) (48) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#5] -Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index 26fec145f4211..dbdfc72f92624 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -109,109 +109,109 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] (16) Filter [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_store_sk#11) (17) Scan parquet default.store -Output [2]: [s_store_sk#8, s_state#10] +Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#8, s_state#10] +Input [2]: [s_store_sk#14, s_state#15] (19) Filter [codegen id : 2] -Input [2]: [s_store_sk#8, s_state#10] -Condition : isnotnull(s_store_sk#8) +Input [2]: [s_store_sk#14, s_state#15] +Condition : isnotnull(s_store_sk#14) (20) BroadcastExchange -Input [2]: [s_store_sk#8, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +Input [2]: [s_store_sk#14, s_state#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#16] (21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#8] +Left keys [1]: [ss_store_sk#11] +Right keys [1]: [s_store_sk#14] Join condition: None (22) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, ss_sold_date_sk#3, s_state#10] -Input [5]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, s_store_sk#8, s_state#10] +Output [3]: [ss_net_profit#12, ss_sold_date_sk#13, s_state#15] +Input [5]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13, s_store_sk#14, s_state#15] (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#17] (24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#13] +Right keys [1]: [d_date_sk#17] Join condition: None (25) Project [codegen id : 4] -Output [2]: [ss_net_profit#2, s_state#10] -Input [4]: [ss_net_profit#2, ss_sold_date_sk#3, s_state#10, d_date_sk#5] +Output [2]: [ss_net_profit#12, s_state#15] +Input [4]: [ss_net_profit#12, ss_sold_date_sk#13, s_state#15, d_date_sk#17] (26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#2, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#12] -Results [2]: [s_state#10, sum#13] +Input [2]: [ss_net_profit#12, s_state#15] +Keys [1]: [s_state#15] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum#18] +Results [2]: [s_state#15, sum#19] (27) Exchange -Input [2]: [s_state#10, sum#13] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [2]: [s_state#15, sum#19] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [id=#20] (28) HashAggregate [codegen id : 5] -Input [2]: [s_state#10, sum#13] -Keys [1]: [s_state#10] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#15] -Results [3]: [s_state#10 AS s_state#16, s_state#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#15,17,2) AS _w2#17] +Input [2]: [s_state#15, sum#19] +Keys [1]: [s_state#15] +Functions [1]: [sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#12))#21] +Results [3]: [s_state#15 AS s_state#22, s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#12))#21,17,2) AS _w2#23] (29) Exchange -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [id=#24] (30) Sort [codegen id : 6] -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [s_state#10 ASC NULLS FIRST, _w2#17 DESC NULLS LAST], false, 0 +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: [s_state#15 ASC NULLS FIRST, _w2#23 DESC NULLS LAST], false, 0 (31) Window -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#10], [_w2#17 DESC NULLS LAST] +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: [rank(_w2#23) windowspecdefinition(s_state#15, _w2#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#25], [s_state#15], [_w2#23 DESC NULLS LAST] (32) Filter [codegen id : 7] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) +Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] +Condition : (isnotnull(ranking#25) AND (ranking#25 <= 5)) (33) Project [codegen id : 7] -Output [1]: [s_state#16] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] +Output [1]: [s_state#22] +Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] (34) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#20] +Input [1]: [s_state#22] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#26] (35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#10] -Right keys [1]: [s_state#16] +Right keys [1]: [s_state#22] Join condition: None (36) BroadcastExchange Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] @@ -224,45 +224,45 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_county#9, s_state#10 (39) Expand [codegen id : 9] Input [3]: [ss_net_profit#2, s_state#10, s_county#9] -Arguments: [List(ss_net_profit#2, s_state#10, s_county#9, 0), List(ss_net_profit#2, s_state#10, null, 1), List(ss_net_profit#2, null, null, 3)], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] +Arguments: [List(ss_net_profit#2, s_state#10, s_county#9, 0), List(ss_net_profit#2, s_state#10, null, 1), List(ss_net_profit#2, null, null, 3)], [ss_net_profit#2, s_state#28, s_county#29, spark_grouping_id#30] (40) HashAggregate [codegen id : 9] -Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] +Input [4]: [ss_net_profit#2, s_state#28, s_county#29, spark_grouping_id#30] +Keys [3]: [s_state#28, s_county#29, spark_grouping_id#30] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#25] -Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Aggregate Attributes [1]: [sum#31] +Results [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] (41) Exchange -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] +Arguments: hashpartitioning(s_state#28, s_county#29, spark_grouping_id#30, 5), ENSURE_REQUIREMENTS, [id=#33] (42) HashAggregate [codegen id : 10] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] +Input [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] +Keys [3]: [s_state#28, s_county#29, spark_grouping_id#30] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#28] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS total_sum#29, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS _w3#33] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#34] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS total_sum#35, s_state#28, s_county#29, (cast((shiftright(spark_grouping_id#30, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint)) AS lochierarchy#36, (cast((shiftright(spark_grouping_id#30, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint)) AS _w1#37, CASE WHEN (cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint) = 0) THEN s_state#28 END AS _w2#38, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS _w3#39] (43) Exchange -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: hashpartitioning(_w1#37, _w2#38, 5), ENSURE_REQUIREMENTS, [id=#40] (44) Sort [codegen id : 11] -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w3#33 DESC NULLS LAST], false, 0 +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: [_w1#37 ASC NULLS FIRST, _w2#38 ASC NULLS FIRST, _w3#39 DESC NULLS LAST], false, 0 (45) Window -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: [rank(_w3#33) windowspecdefinition(_w1#31, _w2#32, _w3#33 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#35], [_w1#31, _w2#32], [_w3#33 DESC NULLS LAST] +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: [rank(_w3#39) windowspecdefinition(_w1#37, _w2#38, _w3#39 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [_w1#37, _w2#38], [_w3#39 DESC NULLS LAST] (46) Project [codegen id : 12] -Output [5]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] -Input [8]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33, rank_within_parent#35] +Output [5]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] +Input [8]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39, rank_within_parent#41] (47) TakeOrderedAndProject -Input [5]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#35 ASC NULLS FIRST], [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] +Input [5]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] +Arguments: 100, [lochierarchy#36 DESC NULLS LAST, CASE WHEN (lochierarchy#36 = 0) THEN s_state#28 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] ===== Subqueries ===== @@ -273,6 +273,6 @@ ReusedExchange (48) (48) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#5] -Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt index 3daa8b66851f7..274d0c02b5583 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt @@ -127,43 +127,43 @@ Input [4]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_da Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_sold_time_sk#18)) (19) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#22] (20) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#11] +Right keys [1]: [d_date_sk#22] Join condition: None (21) Project [codegen id : 5] -Output [3]: [cs_ext_sales_price#20 AS ext_price#22, cs_item_sk#19 AS sold_item_sk#23, cs_sold_time_sk#18 AS time_sk#24] -Input [5]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#11] +Output [3]: [cs_ext_sales_price#20 AS ext_price#23, cs_item_sk#19 AS sold_item_sk#24, cs_sold_time_sk#18 AS time_sk#25] +Input [5]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#22] (22) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 7] -Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] (24) Filter [codegen id : 7] -Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -Condition : (isnotnull(ss_item_sk#26) AND isnotnull(ss_sold_time_sk#25)) +Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Condition : (isnotnull(ss_item_sk#27) AND isnotnull(ss_sold_time_sk#26)) (25) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#30] (26) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ss_sold_date_sk#29] +Right keys [1]: [d_date_sk#30] Join condition: None (27) Project [codegen id : 7] -Output [3]: [ss_ext_sales_price#27 AS ext_price#29, ss_item_sk#26 AS sold_item_sk#30, ss_sold_time_sk#25 AS time_sk#31] -Input [5]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#11] +Output [3]: [ss_ext_sales_price#28 AS ext_price#31, ss_item_sk#27 AS sold_item_sk#32, ss_sold_time_sk#26 AS time_sk#33] +Input [5]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#30] (28) Union @@ -177,61 +177,61 @@ Output [4]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17] Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#15, sold_item_sk#16, time_sk#17] (31) Scan parquet default.time_dim -Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Output [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [Or(EqualTo(t_meal_time,breakfast),EqualTo(t_meal_time,dinner)), IsNotNull(t_time_sk)] +PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] (33) Filter [codegen id : 8] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Condition : (((t_meal_time#35 = breakfast) OR (t_meal_time#35 = dinner)) AND isnotnull(t_time_sk#32)) +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Condition : (((t_meal_time#37 = breakfast ) OR (t_meal_time#37 = dinner )) AND isnotnull(t_time_sk#34)) (34) Project [codegen id : 8] -Output [3]: [t_time_sk#32, t_hour#33, t_minute#34] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Output [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] (35) BroadcastExchange -Input [3]: [t_time_sk#32, t_hour#33, t_minute#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [time_sk#17] -Right keys [1]: [t_time_sk#32] +Right keys [1]: [t_time_sk#34] Join condition: None (37) Project [codegen id : 9] -Output [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#33, t_minute#34] -Input [7]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17, t_time_sk#32, t_hour#33, t_minute#34] +Output [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#35, t_minute#36] +Input [7]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17, t_time_sk#34, t_hour#35, t_minute#36] (38) HashAggregate [codegen id : 9] -Input [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#33, t_minute#34] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] +Input [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#35, t_minute#36] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] Functions [1]: [partial_sum(UnscaledValue(ext_price#15))] -Aggregate Attributes [1]: [sum#37] -Results [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] +Aggregate Attributes [1]: [sum#39] +Results [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] (39) Exchange -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] -Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] +Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, 5), ENSURE_REQUIREMENTS, [id=#41] (40) HashAggregate [codegen id : 10] -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] Functions [1]: [sum(UnscaledValue(ext_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#15))#40] -Results [5]: [i_brand_id#2 AS brand_id#41, i_brand#3 AS brand#42, t_hour#33, t_minute#34, MakeDecimal(sum(UnscaledValue(ext_price#15))#40,17,2) AS ext_price#43] +Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#15))#42] +Results [5]: [i_brand_id#2 AS brand_id#43, i_brand#3 AS brand#44, t_hour#35, t_minute#36, MakeDecimal(sum(UnscaledValue(ext_price#15))#42,17,2) AS ext_price#45] (41) Exchange -Input [5]: [brand_id#41, brand#42, t_hour#33, t_minute#34, ext_price#43] -Arguments: rangepartitioning(ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [5]: [brand_id#43, brand#44, t_hour#35, t_minute#36, ext_price#45] +Arguments: rangepartitioning(ext_price#45 DESC NULLS LAST, brand_id#43 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#46] (42) Sort [codegen id : 11] -Input [5]: [brand_id#41, brand#42, t_hour#33, t_minute#34, ext_price#43] -Arguments: [ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST], true, 0 +Input [5]: [brand_id#43, brand#44, t_hour#35, t_minute#36, ext_price#45] +Arguments: [ext_price#45 DESC NULLS LAST, brand_id#43 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -244,6 +244,6 @@ Output [1]: [d_date_sk#11] Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index 3daa8b66851f7..274d0c02b5583 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt @@ -127,43 +127,43 @@ Input [4]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_da Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_sold_time_sk#18)) (19) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#22] (20) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#11] +Right keys [1]: [d_date_sk#22] Join condition: None (21) Project [codegen id : 5] -Output [3]: [cs_ext_sales_price#20 AS ext_price#22, cs_item_sk#19 AS sold_item_sk#23, cs_sold_time_sk#18 AS time_sk#24] -Input [5]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#11] +Output [3]: [cs_ext_sales_price#20 AS ext_price#23, cs_item_sk#19 AS sold_item_sk#24, cs_sold_time_sk#18 AS time_sk#25] +Input [5]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#22] (22) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 7] -Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] (24) Filter [codegen id : 7] -Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -Condition : (isnotnull(ss_item_sk#26) AND isnotnull(ss_sold_time_sk#25)) +Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Condition : (isnotnull(ss_item_sk#27) AND isnotnull(ss_sold_time_sk#26)) (25) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#30] (26) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ss_sold_date_sk#29] +Right keys [1]: [d_date_sk#30] Join condition: None (27) Project [codegen id : 7] -Output [3]: [ss_ext_sales_price#27 AS ext_price#29, ss_item_sk#26 AS sold_item_sk#30, ss_sold_time_sk#25 AS time_sk#31] -Input [5]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#11] +Output [3]: [ss_ext_sales_price#28 AS ext_price#31, ss_item_sk#27 AS sold_item_sk#32, ss_sold_time_sk#26 AS time_sk#33] +Input [5]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#30] (28) Union @@ -177,61 +177,61 @@ Output [4]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17] Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#15, sold_item_sk#16, time_sk#17] (31) Scan parquet default.time_dim -Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Output [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [Or(EqualTo(t_meal_time,breakfast),EqualTo(t_meal_time,dinner)), IsNotNull(t_time_sk)] +PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] (33) Filter [codegen id : 8] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Condition : (((t_meal_time#35 = breakfast) OR (t_meal_time#35 = dinner)) AND isnotnull(t_time_sk#32)) +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Condition : (((t_meal_time#37 = breakfast ) OR (t_meal_time#37 = dinner )) AND isnotnull(t_time_sk#34)) (34) Project [codegen id : 8] -Output [3]: [t_time_sk#32, t_hour#33, t_minute#34] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Output [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] (35) BroadcastExchange -Input [3]: [t_time_sk#32, t_hour#33, t_minute#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [time_sk#17] -Right keys [1]: [t_time_sk#32] +Right keys [1]: [t_time_sk#34] Join condition: None (37) Project [codegen id : 9] -Output [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#33, t_minute#34] -Input [7]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17, t_time_sk#32, t_hour#33, t_minute#34] +Output [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#35, t_minute#36] +Input [7]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17, t_time_sk#34, t_hour#35, t_minute#36] (38) HashAggregate [codegen id : 9] -Input [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#33, t_minute#34] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] +Input [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#35, t_minute#36] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] Functions [1]: [partial_sum(UnscaledValue(ext_price#15))] -Aggregate Attributes [1]: [sum#37] -Results [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] +Aggregate Attributes [1]: [sum#39] +Results [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] (39) Exchange -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] -Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] +Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, 5), ENSURE_REQUIREMENTS, [id=#41] (40) HashAggregate [codegen id : 10] -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] Functions [1]: [sum(UnscaledValue(ext_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#15))#40] -Results [5]: [i_brand_id#2 AS brand_id#41, i_brand#3 AS brand#42, t_hour#33, t_minute#34, MakeDecimal(sum(UnscaledValue(ext_price#15))#40,17,2) AS ext_price#43] +Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#15))#42] +Results [5]: [i_brand_id#2 AS brand_id#43, i_brand#3 AS brand#44, t_hour#35, t_minute#36, MakeDecimal(sum(UnscaledValue(ext_price#15))#42,17,2) AS ext_price#45] (41) Exchange -Input [5]: [brand_id#41, brand#42, t_hour#33, t_minute#34, ext_price#43] -Arguments: rangepartitioning(ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [5]: [brand_id#43, brand#44, t_hour#35, t_minute#36, ext_price#45] +Arguments: rangepartitioning(ext_price#45 DESC NULLS LAST, brand_id#43 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#46] (42) Sort [codegen id : 11] -Input [5]: [brand_id#41, brand#42, t_hour#33, t_minute#34, ext_price#43] -Arguments: [ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST], true, 0 +Input [5]: [brand_id#43, brand#44, t_hour#35, t_minute#36, ext_price#45] +Arguments: [ext_price#45 DESC NULLS LAST, brand_id#43 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -244,6 +244,6 @@ Output [1]: [d_date_sk#11] Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt index 04172ab353f25..339dc5d95e4c4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt @@ -100,7 +100,7 @@ Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotn Output [2]: [hd_demo_sk#9, hd_buy_potential#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -108,7 +108,7 @@ Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (6) Filter [codegen id : 1] Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = >10000)) AND isnotnull(hd_demo_sk#9)) +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = >10000 )) AND isnotnull(hd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [hd_demo_sk#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt index fe49873e2e3f3..177b8e681608a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt @@ -207,7 +207,7 @@ Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_ Output [2]: [hd_demo_sk#24, hd_buy_potential#25] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] @@ -215,7 +215,7 @@ Input [2]: [hd_demo_sk#24, hd_buy_potential#25] (31) Filter [codegen id : 5] Input [2]: [hd_demo_sk#24, hd_buy_potential#25] -Condition : ((isnotnull(hd_buy_potential#25) AND (hd_buy_potential#25 = >10000)) AND isnotnull(hd_demo_sk#24)) +Condition : ((isnotnull(hd_buy_potential#25) AND (hd_buy_potential#25 = >10000 )) AND isnotnull(hd_demo_sk#24)) (32) Project [codegen id : 5] Output [1]: [hd_demo_sk#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt index e658e11ade4a0..aec6d66c98fdd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt @@ -121,7 +121,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -129,7 +129,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt index b45f0513ba576..646a8fbc11a3a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt index 74cbccc79fdc5..9b2ead7ea96f7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt @@ -199,285 +199,285 @@ Input [2]: [customer_id#18, year_total#19] Arguments: [customer_id#18 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] +Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] +Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] (27) Filter [codegen id : 10] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_customer_sk#1) +Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Condition : isnotnull(ss_customer_sk#21) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#25, d_year#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#5, d_year#6] +Input [2]: [d_date_sk#25, d_year#26] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#5, d_year#6] -Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2002)) AND d_year#6 IN (2001,2002)) AND isnotnull(d_date_sk#5)) +Input [2]: [d_date_sk#25, d_year#26] +Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) (31) BroadcastExchange -Input [2]: [d_date_sk#5, d_year#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +Input [2]: [d_date_sk#25, d_year#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#23] +Right keys [1]: [d_date_sk#25] Join condition: None (33) Project [codegen id : 10] -Output [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Input [5]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6] +Output [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Input [5]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] (34) Exchange -Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Arguments: hashpartitioning(ss_customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#28] (35) Sort [codegen id : 11] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Arguments: [ss_customer_sk#21 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] (37) Sort [codegen id : 13] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ss_customer_sk#21] +Right keys [1]: [c_customer_sk#29] Join condition: None (39) Project [codegen id : 14] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] -Input [7]: [ss_customer_sk#1, ss_net_paid#2, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] +Input [7]: [ss_customer_sk#21, ss_net_paid#22, d_year#26, c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] (40) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#2))] -Aggregate Attributes [1]: [sum#24] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] +Aggregate Attributes [1]: [sum#33] +Results [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] (41) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] +Arguments: hashpartitioning(c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, 5), ENSURE_REQUIREMENTS, [id=#35] (42) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ss_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#2))#27] -Results [4]: [c_customer_id#10 AS customer_id#28, c_first_name#11 AS customer_first_name#29, c_last_name#12 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#2))#27,17,2) AS year_total#31] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] +Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#22))#36] +Results [4]: [c_customer_id#30 AS customer_id#37, c_first_name#31 AS customer_first_name#38, c_last_name#32 AS customer_last_name#39, MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#36,17,2) AS year_total#40] (43) Exchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: hashpartitioning(customer_id#28, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] +Arguments: hashpartitioning(customer_id#37, 5), ENSURE_REQUIREMENTS, [id=#41] (44) Sort [codegen id : 16] -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#28 ASC NULLS FIRST], false, 0 +Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] +Arguments: [customer_id#37 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#28] +Right keys [1]: [customer_id#37] Join condition: None (46) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] (48) Filter [codegen id : 19] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_bill_customer_sk#33) +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_bill_customer_sk#42) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#45, d_year#46] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (51) Project [codegen id : 19] -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] +Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Input [5]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] (52) Exchange -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#47] (53) Sort [codegen id : 20] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Arguments: [ws_bill_customer_sk#42 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] (55) Sort [codegen id : 22] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Arguments: [c_customer_sk#48 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#33] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ws_bill_customer_sk#42] +Right keys [1]: [c_customer_sk#48] Join condition: None (57) Project [codegen id : 23] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] +Input [7]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46, c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] (58) HashAggregate [codegen id : 23] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum#37] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] +Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] +Aggregate Attributes [1]: [sum#52] +Results [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] (59) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] +Arguments: hashpartitioning(c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, 5), ENSURE_REQUIREMENTS, [id=#54] (60) HashAggregate [codegen id : 24] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#40] -Results [2]: [c_customer_id#10 AS customer_id#41, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#40,17,2) AS year_total#42] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] +Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] +Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#55] +Results [2]: [c_customer_id#49 AS customer_id#56, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#55,17,2) AS year_total#57] (61) Filter [codegen id : 24] -Input [2]: [customer_id#41, year_total#42] -Condition : (isnotnull(year_total#42) AND (year_total#42 > 0.00)) +Input [2]: [customer_id#56, year_total#57] +Condition : (isnotnull(year_total#57) AND (year_total#57 > 0.00)) (62) Project [codegen id : 24] -Output [2]: [customer_id#41 AS customer_id#43, year_total#42 AS year_total#44] -Input [2]: [customer_id#41, year_total#42] +Output [2]: [customer_id#56 AS customer_id#58, year_total#57 AS year_total#59] +Input [2]: [customer_id#56, year_total#57] (63) Exchange -Input [2]: [customer_id#43, year_total#44] -Arguments: hashpartitioning(customer_id#43, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [2]: [customer_id#58, year_total#59] +Arguments: hashpartitioning(customer_id#58, 5), ENSURE_REQUIREMENTS, [id=#60] (64) Sort [codegen id : 25] -Input [2]: [customer_id#43, year_total#44] -Arguments: [customer_id#43 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#58, year_total#59] +Arguments: [customer_id#58 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#43] +Right keys [1]: [customer_id#58] Join condition: None (66) Project [codegen id : 26] -Output [7]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44] -Input [8]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#43, year_total#44] +Output [7]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59] +Input [8]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, customer_id#58, year_total#59] (67) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] (69) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_bill_customer_sk#33) +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_bill_customer_sk#61) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#64, d_year#65] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#63] +Right keys [1]: [d_date_sk#64] Join condition: None (72) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] +Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Input [5]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65] (73) Exchange -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Arguments: hashpartitioning(ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#66] (74) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Arguments: [ws_bill_customer_sk#61 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] (76) Sort [codegen id : 31] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] +Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#33] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ws_bill_customer_sk#61] +Right keys [1]: [c_customer_sk#67] Join condition: None (78) Project [codegen id : 32] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] +Input [7]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65, c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] (79) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum#47] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] +Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#62))] +Aggregate Attributes [1]: [sum#71] +Results [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] (80) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] +Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#73] (81) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#50] -Results [2]: [c_customer_id#10 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#50,17,2) AS year_total#52] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] +Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] +Functions [1]: [sum(UnscaledValue(ws_net_paid#62))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#62))#74] +Results [2]: [c_customer_id#68 AS customer_id#75, MakeDecimal(sum(UnscaledValue(ws_net_paid#62))#74,17,2) AS year_total#76] (82) Exchange -Input [2]: [customer_id#51, year_total#52] -Arguments: hashpartitioning(customer_id#51, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [2]: [customer_id#75, year_total#76] +Arguments: hashpartitioning(customer_id#75, 5), ENSURE_REQUIREMENTS, [id=#77] (83) Sort [codegen id : 34] -Input [2]: [customer_id#51, year_total#52] -Arguments: [customer_id#51 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#75, year_total#76] +Arguments: [customer_id#75 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#51] -Join condition: (CASE WHEN (year_total#44 > 0.00) THEN CheckOverflow((promote_precision(year_total#52) / promote_precision(year_total#44)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#31) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#75] +Join condition: (CASE WHEN (year_total#59 > 0.00) THEN CheckOverflow((promote_precision(year_total#76) / promote_precision(year_total#59)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#40) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) (85) Project [codegen id : 35] -Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Input [9]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44, customer_id#51, year_total#52] +Output [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] +Input [9]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59, customer_id#75, year_total#76] (86) TakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: 100, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] +Input [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] +Arguments: 100, [customer_id#37 ASC NULLS FIRST, customer_id#37 ASC NULLS FIRST, customer_id#37 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39] ===== Subqueries ===== @@ -488,15 +488,15 @@ ReusedExchange (87) (87) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#5, d_year#6] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#21 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#25, d_year#26] -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#21 +Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt index db3a0a43be591..9fccc4c4ba66d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt @@ -171,282 +171,282 @@ Input [2]: [customer_id#17, year_total#18] Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) (20) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] (22) Filter [codegen id : 6] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Condition : (isnotnull(c_customer_sk#19) AND isnotnull(c_customer_id#20)) (23) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] (25) Filter [codegen id : 4] -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_customer_sk#5) +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_customer_sk#23) (26) BroadcastExchange -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#5] +Left keys [1]: [c_customer_sk#19] +Right keys [1]: [ss_customer_sk#23] Join condition: None (28) Project [codegen id : 6] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25] +Input [7]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#28, d_year#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#10, d_year#11] +Input [2]: [d_date_sk#28, d_year#29] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#10, d_year#11] -Condition : (((isnotnull(d_year#11) AND (d_year#11 = 2002)) AND d_year#11 IN (2001,2002)) AND isnotnull(d_date_sk#10)) +Input [2]: [d_date_sk#28, d_year#29] +Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) (32) BroadcastExchange -Input [2]: [d_date_sk#10, d_year#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Input [2]: [d_date_sk#28, d_year#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#25] +Right keys [1]: [d_date_sk#28] Join condition: None (34) Project [codegen id : 6] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] +Input [7]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#28, d_year#29] (35) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum#22] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] +Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] +Aggregate Attributes [1]: [sum#31] +Results [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] (36) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] +Arguments: hashpartitioning(c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, 5), ENSURE_REQUIREMENTS, [id=#33] (37) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#25] -Results [4]: [c_customer_id#2 AS customer_id#26, c_first_name#3 AS customer_first_name#27, c_last_name#4 AS customer_last_name#28, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#25,17,2) AS year_total#29] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] +Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] +Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#34] +Results [4]: [c_customer_id#20 AS customer_id#35, c_first_name#21 AS customer_first_name#36, c_last_name#22 AS customer_last_name#37, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#34,17,2) AS year_total#38] (38) BroadcastExchange -Input [4]: [customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] +Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#39] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#26] +Right keys [1]: [customer_id#35] Join condition: None (40) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] (42) Filter [codegen id : 10] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] +Condition : (isnotnull(c_customer_sk#40) AND isnotnull(c_customer_id#41)) (43) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] (45) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_bill_customer_sk#31) +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Condition : isnotnull(ws_bill_customer_sk#44) (46) BroadcastExchange -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#31] +Left keys [1]: [c_customer_sk#40] +Right keys [1]: [ws_bill_customer_sk#44] Join condition: None (48) Project [codegen id : 10] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46] +Input [7]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#48, d_year#49] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#46] +Right keys [1]: [d_date_sk#48] Join condition: None (51) Project [codegen id : 10] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] +Input [7]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46, d_date_sk#48, d_year#49] (52) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum#35] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] +Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#45))] +Aggregate Attributes [1]: [sum#50] +Results [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] (53) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] +Arguments: hashpartitioning(c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, 5), ENSURE_REQUIREMENTS, [id=#52] (54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#38] -Results [2]: [c_customer_id#2 AS customer_id#39, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#38,17,2) AS year_total#40] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] +Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] +Functions [1]: [sum(UnscaledValue(ws_net_paid#45))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#45))#53] +Results [2]: [c_customer_id#41 AS customer_id#54, MakeDecimal(sum(UnscaledValue(ws_net_paid#45))#53,17,2) AS year_total#55] (55) Filter [codegen id : 11] -Input [2]: [customer_id#39, year_total#40] -Condition : (isnotnull(year_total#40) AND (year_total#40 > 0.00)) +Input [2]: [customer_id#54, year_total#55] +Condition : (isnotnull(year_total#55) AND (year_total#55 > 0.00)) (56) Project [codegen id : 11] -Output [2]: [customer_id#39 AS customer_id#41, year_total#40 AS year_total#42] -Input [2]: [customer_id#39, year_total#40] +Output [2]: [customer_id#54 AS customer_id#56, year_total#55 AS year_total#57] +Input [2]: [customer_id#54, year_total#55] (57) BroadcastExchange -Input [2]: [customer_id#41, year_total#42] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#43] +Input [2]: [customer_id#56, year_total#57] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#58] (58) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#41] +Right keys [1]: [customer_id#56] Join condition: None (59) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42] -Input [8]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, customer_id#41, year_total#42] +Output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57] +Input [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, customer_id#56, year_total#57] (60) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] (62) Filter [codegen id : 14] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] +Condition : (isnotnull(c_customer_sk#59) AND isnotnull(c_customer_id#60)) (63) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] (65) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_bill_customer_sk#31) +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Condition : isnotnull(ws_bill_customer_sk#63) (66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#31] +Left keys [1]: [c_customer_sk#59] +Right keys [1]: [ws_bill_customer_sk#63] Join condition: None (68) Project [codegen id : 14] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65] +Input [7]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#67, d_year#68] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#65] +Right keys [1]: [d_date_sk#67] Join condition: None (71) Project [codegen id : 14] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] +Input [7]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65, d_date_sk#67, d_year#68] (72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum#45] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] +Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#64))] +Aggregate Attributes [1]: [sum#69] +Results [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] (73) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] +Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, 5), ENSURE_REQUIREMENTS, [id=#71] (74) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#48] -Results [2]: [c_customer_id#2 AS customer_id#49, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#48,17,2) AS year_total#50] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] +Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] +Functions [1]: [sum(UnscaledValue(ws_net_paid#64))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#64))#72] +Results [2]: [c_customer_id#60 AS customer_id#73, MakeDecimal(sum(UnscaledValue(ws_net_paid#64))#72,17,2) AS year_total#74] (75) BroadcastExchange -Input [2]: [customer_id#49, year_total#50] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#51] +Input [2]: [customer_id#73, year_total#74] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#75] (76) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#49] -Join condition: (CASE WHEN (year_total#42 > 0.00) THEN CheckOverflow((promote_precision(year_total#50) / promote_precision(year_total#42)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#29) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#73] +Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#57)), 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) (77) Project [codegen id : 16] -Output [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] -Input [9]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42, customer_id#49, year_total#50] +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#57, customer_id#73, year_total#74] (78) TakeOrderedAndProject -Input [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] -Arguments: 100, [customer_id#26 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST], [customer_id#26, customer_first_name#27, customer_last_name#28] +Input [3]: [customer_id#35, customer_first_name#36, customer_last_name#37] +Arguments: 100, [customer_id#35 ASC NULLS FIRST, customer_id#35 ASC NULLS FIRST, customer_id#35 ASC NULLS FIRST], [customer_id#35, customer_first_name#36, customer_last_name#37] ===== Subqueries ===== @@ -457,15 +457,15 @@ ReusedExchange (79) (79) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#10, d_year#11] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#19 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#28, d_year#29] -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#19 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#26 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt index 9472a8935eb8b..a06e41562f8ff 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt @@ -153,7 +153,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -161,7 +161,7 @@ Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_categor (6) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books)) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#31] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#36, d_year#37] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#36] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] -Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (42) Exchange -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_item_sk#47) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ws_item_sk#47] +Right keys [1]: [i_item_sk#52] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#57, d_year#58] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#57] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] (55) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] -Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (61) Exchange -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] +Right keys [2]: [wr_order_number#61, wr_item_sk#60] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] +Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#55, sum#56] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Aggregate Attributes [2]: [sum#69, sum#70] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] +Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Condition : isnotnull(cs_item_sk#79) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [cs_item_sk#79] +Right keys [1]: [i_item_sk#85] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] +Input [2]: [d_date_sk#90, d_year#91] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] -Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) +Input [2]: [d_date_sk#90, d_year#91] +Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) (83) BroadcastExchange -Input [2]: [d_date_sk#71, d_year#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [d_date_sk#90, d_year#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [cs_sold_date_sk#83] +Right keys [1]: [d_date_sk#90] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] (86) Exchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] -Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#2, cs_item_sk#1] -Right keys [2]: [cr_order_number#19, cr_item_sk#18] +Left keys [2]: [cs_order_number#80, cs_item_sk#79] +Right keys [2]: [cr_order_number#95, cr_item_sk#94] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Condition : isnotnull(ss_item_sk#26) +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Condition : isnotnull(ss_item_sk#98) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#98] +Right keys [1]: [i_item_sk#103] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#108, d_year#109] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ss_sold_date_sk#102] +Right keys [1]: [d_date_sk#108] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] (101) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] +Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] +Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Condition : isnotnull(ws_item_sk#116) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ws_item_sk#116] +Right keys [1]: [i_item_sk#121] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#126, d_year#127] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ws_sold_date_sk#120] +Right keys [1]: [d_date_sk#126] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] (116) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] +Right keys [2]: [wr_order_number#130, wr_item_sk#129] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] +Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Aggregate Attributes [2]: [sum#135, sum#136] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] (127) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] +Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] (129) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] (130) Sort [codegen id : 50] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 (131) 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#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] (133) TakeOrderedAndProject -Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] -Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt index 9472a8935eb8b..a06e41562f8ff 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt @@ -153,7 +153,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -161,7 +161,7 @@ Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_categor (6) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books)) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#31] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#36, d_year#37] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#36] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] -Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (42) Exchange -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_item_sk#47) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ws_item_sk#47] +Right keys [1]: [i_item_sk#52] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#57, d_year#58] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#57] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] (55) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] -Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (61) Exchange -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] +Right keys [2]: [wr_order_number#61, wr_item_sk#60] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] +Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#55, sum#56] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Aggregate Attributes [2]: [sum#69, sum#70] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] +Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Condition : isnotnull(cs_item_sk#79) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [cs_item_sk#79] +Right keys [1]: [i_item_sk#85] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] +Input [2]: [d_date_sk#90, d_year#91] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] -Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) +Input [2]: [d_date_sk#90, d_year#91] +Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) (83) BroadcastExchange -Input [2]: [d_date_sk#71, d_year#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [d_date_sk#90, d_year#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [cs_sold_date_sk#83] +Right keys [1]: [d_date_sk#90] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] (86) Exchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] -Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#2, cs_item_sk#1] -Right keys [2]: [cr_order_number#19, cr_item_sk#18] +Left keys [2]: [cs_order_number#80, cs_item_sk#79] +Right keys [2]: [cr_order_number#95, cr_item_sk#94] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Condition : isnotnull(ss_item_sk#26) +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Condition : isnotnull(ss_item_sk#98) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#98] +Right keys [1]: [i_item_sk#103] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#108, d_year#109] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ss_sold_date_sk#102] +Right keys [1]: [d_date_sk#108] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] (101) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] +Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] +Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Condition : isnotnull(ws_item_sk#116) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ws_item_sk#116] +Right keys [1]: [i_item_sk#121] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#126, d_year#127] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ws_sold_date_sk#120] +Right keys [1]: [d_date_sk#126] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] (116) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] +Right keys [2]: [wr_order_number#130, wr_item_sk#129] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] +Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Aggregate Attributes [2]: [sum#135, sum#136] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] (127) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] +Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] (129) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] (130) Sort [codegen id : 50] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 (131) 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#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] (133) TakeOrderedAndProject -Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] -Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt index 0092354e54be0..9284172139688 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt @@ -134,93 +134,93 @@ Input [4]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sol Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [id=#19] (20) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] (22) Filter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : isnotnull(d_date_sk#5) +Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] +Condition : isnotnull(d_date_sk#20) (23) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_sold_date_sk#18] -Right keys [1]: [d_date_sk#5] +Right keys [1]: [d_date_sk#20] Join condition: None (24) Project [codegen id : 5] -Output [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#6, d_qoy#7] -Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, d_date_sk#5, d_year#6, d_qoy#7] +Output [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#21, d_qoy#22] +Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, d_date_sk#20, d_year#21, d_qoy#22] (25) BroadcastExchange -Input [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +Input [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#21, d_qoy#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (26) Scan parquet default.item -Output [2]: [i_item_sk#9, i_category#10] +Output [2]: [i_item_sk#24, i_category#25] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (27) ColumnarToRow -Input [2]: [i_item_sk#9, i_category#10] +Input [2]: [i_item_sk#24, i_category#25] (28) Filter -Input [2]: [i_item_sk#9, i_category#10] -Condition : isnotnull(i_item_sk#9) +Input [2]: [i_item_sk#24, i_category#25] +Condition : isnotnull(i_item_sk#24) (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#15] -Right keys [1]: [i_item_sk#9] +Right keys [1]: [i_item_sk#24] Join condition: None (30) Project [codegen id : 6] -Output [6]: [web AS channel#21, ws_ship_customer_sk#16 AS col_name#22, d_year#6, d_qoy#7, i_category#10, ws_ext_sales_price#17 AS ext_sales_price#23] -Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#6, d_qoy#7, i_item_sk#9, i_category#10] +Output [6]: [web AS channel#26, ws_ship_customer_sk#16 AS col_name#27, d_year#21, d_qoy#22, i_category#25, ws_ext_sales_price#17 AS ext_sales_price#28] +Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#21, d_qoy#22, i_item_sk#24, i_category#25] (31) Scan parquet default.catalog_sales -Output [4]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +Output [4]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#27)] +PartitionFilters: [isnotnull(cs_sold_date_sk#32)] PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 9] -Input [4]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +Input [4]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32] (33) Filter [codegen id : 9] -Input [4]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] -Condition : (isnull(cs_ship_addr_sk#24) AND isnotnull(cs_item_sk#25)) +Input [4]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32] +Condition : (isnull(cs_ship_addr_sk#29) AND isnotnull(cs_item_sk#30)) (34) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] (35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [cs_sold_date_sk#32] +Right keys [1]: [d_date_sk#33] Join condition: None (36) Project [codegen id : 9] -Output [5]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_year#6, d_qoy#7] -Input [7]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#5, d_year#6, d_qoy#7] +Output [5]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, d_year#34, d_qoy#35] +Input [7]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32, d_date_sk#33, d_year#34, d_qoy#35] (37) ReusedExchange [Reuses operator id: 13] -Output [2]: [i_item_sk#9, i_category#10] +Output [2]: [i_item_sk#36, i_category#37] (38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#9] +Left keys [1]: [cs_item_sk#30] +Right keys [1]: [i_item_sk#36] Join condition: None (39) Project [codegen id : 9] -Output [6]: [catalog AS channel#28, cs_ship_addr_sk#24 AS col_name#29, d_year#6, d_qoy#7, i_category#10, cs_ext_sales_price#26 AS ext_sales_price#30] -Input [7]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_year#6, d_qoy#7, i_item_sk#9, i_category#10] +Output [6]: [catalog AS channel#38, cs_ship_addr_sk#29 AS col_name#39, d_year#34, d_qoy#35, i_category#37, cs_ext_sales_price#31 AS ext_sales_price#40] +Input [7]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, d_year#34, d_qoy#35, i_item_sk#36, i_category#37] (40) Union @@ -228,21 +228,21 @@ Input [7]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_year#6, Input [6]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, ext_sales_price#14] Keys [5]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10] Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count#31, sum#32] -Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#33, sum#34] +Aggregate Attributes [2]: [count#41, sum#42] +Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#43, sum#44] (42) Exchange -Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#33, sum#34] -Arguments: hashpartitioning(channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#43, sum#44] +Arguments: hashpartitioning(channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, 5), ENSURE_REQUIREMENTS, [id=#45] (43) HashAggregate [codegen id : 11] -Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#33, sum#34] +Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#43, sum#44] Keys [5]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count(1)#36, sum(UnscaledValue(ext_sales_price#14))#37] -Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count(1)#36 AS sales_cnt#38, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#37,17,2) AS sales_amt#39] +Aggregate Attributes [2]: [count(1)#46, sum(UnscaledValue(ext_sales_price#14))#47] +Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count(1)#46 AS sales_cnt#48, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#47,17,2) AS sales_amt#49] (44) TakeOrderedAndProject -Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#38, sales_amt#39] -Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#6 ASC NULLS FIRST, d_qoy#7 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#38, sales_amt#39] +Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#48, sales_amt#49] +Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#6 ASC NULLS FIRST, d_qoy#7 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#48, sales_amt#49] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt index 1250725d1782e..026c9396cd025 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt @@ -124,67 +124,67 @@ Input [4]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sol Condition : (isnull(ws_ship_customer_sk#16) AND isnotnull(ws_item_sk#15)) (19) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#5, i_category#6] +Output [2]: [i_item_sk#19, i_category#20] (20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#15] -Right keys [1]: [i_item_sk#5] +Right keys [1]: [i_item_sk#19] Join condition: None (21) Project [codegen id : 6] -Output [4]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#6] -Input [6]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_item_sk#5, i_category#6] +Output [4]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#20] +Input [6]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_item_sk#19, i_category#20] (22) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] +Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] (23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#18] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#21] Join condition: None (24) Project [codegen id : 6] -Output [6]: [web AS channel#19, ws_ship_customer_sk#16 AS col_name#20, d_year#9, d_qoy#10, i_category#6, ws_ext_sales_price#17 AS ext_sales_price#21] -Input [7]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#6, d_date_sk#8, d_year#9, d_qoy#10] +Output [6]: [web AS channel#24, ws_ship_customer_sk#16 AS col_name#25, d_year#22, d_qoy#23, i_category#20, ws_ext_sales_price#17 AS ext_sales_price#26] +Input [7]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#20, d_date_sk#21, d_year#22, d_qoy#23] (25) Scan parquet default.catalog_sales -Output [4]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25] +Output [4]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#25)] +PartitionFilters: [isnotnull(cs_sold_date_sk#30)] PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 9] -Input [4]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25] +Input [4]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30] (27) Filter [codegen id : 9] -Input [4]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25] -Condition : (isnull(cs_ship_addr_sk#22) AND isnotnull(cs_item_sk#23)) +Input [4]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30] +Condition : (isnull(cs_ship_addr_sk#27) AND isnotnull(cs_item_sk#28)) (28) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#5, i_category#6] +Output [2]: [i_item_sk#31, i_category#32] (29) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#23] -Right keys [1]: [i_item_sk#5] +Left keys [1]: [cs_item_sk#28] +Right keys [1]: [i_item_sk#31] Join condition: None (30) Project [codegen id : 9] -Output [4]: [cs_ship_addr_sk#22, cs_ext_sales_price#24, cs_sold_date_sk#25, i_category#6] -Input [6]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25, i_item_sk#5, i_category#6] +Output [4]: [cs_ship_addr_sk#27, cs_ext_sales_price#29, cs_sold_date_sk#30, i_category#32] +Input [6]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30, i_item_sk#31, i_category#32] (31) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] +Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#25] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#30] +Right keys [1]: [d_date_sk#33] Join condition: None (33) Project [codegen id : 9] -Output [6]: [catalog AS channel#26, cs_ship_addr_sk#22 AS col_name#27, d_year#9, d_qoy#10, i_category#6, cs_ext_sales_price#24 AS ext_sales_price#28] -Input [7]: [cs_ship_addr_sk#22, cs_ext_sales_price#24, cs_sold_date_sk#25, i_category#6, d_date_sk#8, d_year#9, d_qoy#10] +Output [6]: [catalog AS channel#36, cs_ship_addr_sk#27 AS col_name#37, d_year#34, d_qoy#35, i_category#32, cs_ext_sales_price#29 AS ext_sales_price#38] +Input [7]: [cs_ship_addr_sk#27, cs_ext_sales_price#29, cs_sold_date_sk#30, i_category#32, d_date_sk#33, d_year#34, d_qoy#35] (34) Union @@ -192,21 +192,21 @@ Input [7]: [cs_ship_addr_sk#22, cs_ext_sales_price#24, cs_sold_date_sk#25, i_cat Input [6]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, ext_sales_price#14] Keys [5]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6] Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count#29, sum#30] -Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] +Aggregate Attributes [2]: [count#39, sum#40] +Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#41, sum#42] (36) Exchange -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] -Arguments: hashpartitioning(channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#41, sum#42] +Arguments: hashpartitioning(channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, 5), ENSURE_REQUIREMENTS, [id=#43] (37) HashAggregate [codegen id : 11] -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] +Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#41, sum#42] Keys [5]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count(1)#34, sum(UnscaledValue(ext_sales_price#14))#35] -Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count(1)#34 AS sales_cnt#36, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#35,17,2) AS sales_amt#37] +Aggregate Attributes [2]: [count(1)#44, sum(UnscaledValue(ext_sales_price#14))#45] +Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count(1)#44 AS sales_cnt#46, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#45,17,2) AS sales_amt#47] (38) TakeOrderedAndProject -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#36, sales_amt#37] -Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#36, sales_amt#37] +Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#46, sales_amt#47] +Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#46, sales_amt#47] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt index a3626dfe3293d..4b2299ca2e749 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt @@ -209,38 +209,38 @@ Output [4]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_s Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25] (26) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#26] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Right keys [1]: [cast(d_date_sk#26 as bigint)] Join condition: None (28) Project [codegen id : 6] Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#6] +Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#26] (29) HashAggregate [codegen id : 6] Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] Keys [1]: [s_store_sk#25] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#26, sum#27] -Results [3]: [s_store_sk#25, sum#28, sum#29] +Aggregate Attributes [2]: [sum#27, sum#28] +Results [3]: [s_store_sk#25, sum#29, sum#30] (30) Exchange -Input [3]: [s_store_sk#25, sum#28, sum#29] -Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [s_store_sk#25, sum#29, sum#30] +Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#25, sum#28, sum#29] +Input [3]: [s_store_sk#25, sum#29, sum#30] Keys [1]: [s_store_sk#25] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] -Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] +Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] (32) BroadcastExchange -Input [3]: [s_store_sk#25, returns#33, profit_loss#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [3]: [s_store_sk#25, returns#34, profit_loss#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] @@ -248,269 +248,269 @@ Right keys [1]: [s_store_sk#25] Join condition: None (34) Project [codegen id : 8] -Output [5]: [sales#18, coalesce(returns#33, 0.00) AS returns#36, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#37, store channel AS channel#38, s_store_sk#9 AS id#39] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] +Output [5]: [sales#18, coalesce(returns#34, 0.00) AS returns#37, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#38, store channel AS channel#39, s_store_sk#9 AS id#40] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#34, profit_loss#35] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#45] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#43] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cs_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] +Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum#44, sum#45] -Results [3]: [cs_call_center_sk#40, sum#46, sum#47] +Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum#46, sum#47] +Results [3]: [cs_call_center_sk#41, sum#48, sum#49] (41) Exchange -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] (42) HashAggregate [codegen id : 14] -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] -Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] +Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] (43) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +Output [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#55), dynamicpruningexpression(cr_returned_date_sk#55 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#5)] ReadSchema: struct (44) ColumnarToRow [codegen id : 12] -Input [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +Input [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] (45) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#58] (46) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cr_returned_date_sk#55] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#57] +Right keys [1]: [d_date_sk#58] Join condition: None (47) Project [codegen id : 12] -Output [2]: [cr_return_amount#53, cr_net_loss#54] -Input [4]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55, d_date_sk#6] +Output [2]: [cr_return_amount#55, cr_net_loss#56] +Input [4]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57, d_date_sk#58] (48) HashAggregate [codegen id : 12] -Input [2]: [cr_return_amount#53, cr_net_loss#54] +Input [2]: [cr_return_amount#55, cr_net_loss#56] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#53)), partial_sum(UnscaledValue(cr_net_loss#54))] -Aggregate Attributes [2]: [sum#56, sum#57] -Results [2]: [sum#58, sum#59] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#55)), partial_sum(UnscaledValue(cr_net_loss#56))] +Aggregate Attributes [2]: [sum#59, sum#60] +Results [2]: [sum#61, sum#62] (49) Exchange -Input [2]: [sum#58, sum#59] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#60] +Input [2]: [sum#61, sum#62] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] (50) HashAggregate [codegen id : 13] -Input [2]: [sum#58, sum#59] +Input [2]: [sum#61, sum#62] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#53)), sum(UnscaledValue(cr_net_loss#54))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#53))#61, sum(UnscaledValue(cr_net_loss#54))#62] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#53))#61,17,2) AS returns#63, MakeDecimal(sum(UnscaledValue(cr_net_loss#54))#62,17,2) AS profit_loss#64] +Functions [2]: [sum(UnscaledValue(cr_return_amount#55)), sum(UnscaledValue(cr_net_loss#56))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#55))#64, sum(UnscaledValue(cr_net_loss#56))#65] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#55))#64,17,2) AS returns#66, MakeDecimal(sum(UnscaledValue(cr_net_loss#56))#65,17,2) AS profit_loss#67] (51) BroadcastExchange -Input [2]: [returns#63, profit_loss#64] -Arguments: IdentityBroadcastMode, [id=#65] +Input [2]: [returns#66, profit_loss#67] +Arguments: IdentityBroadcastMode, [id=#68] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [sales#51, returns#63, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#64 as decimal(18,2)))), DecimalType(18,2), true) AS profit#66, catalog channel AS channel#67, cs_call_center_sk#40 AS id#68] -Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#63, profit_loss#64] +Output [5]: [sales#53, returns#66, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#67 as decimal(18,2)))), DecimalType(18,2), true) AS profit#69, catalog channel AS channel#70, cs_call_center_sk#41 AS id#71] +Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#66, profit_loss#67] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] -Condition : isnotnull(ws_web_page_sk#69) +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Condition : isnotnull(ws_web_page_sk#72) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#76] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#72] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#75] +Right keys [1]: [d_date_sk#76] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] -Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] +Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] +Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#73] +Output [1]: [wp_web_page_sk#77] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#73] +Input [1]: [wp_web_page_sk#77] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#73] -Condition : isnotnull(wp_web_page_sk#73) +Input [1]: [wp_web_page_sk#77] +Condition : isnotnull(wp_web_page_sk#77) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [1]: [wp_web_page_sk#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#69] -Right keys [1]: [wp_web_page_sk#73] +Left keys [1]: [ws_web_page_sk#72] +Right keys [1]: [wp_web_page_sk#77] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum#75, sum#76] -Results [3]: [wp_web_page_sk#73, sum#77, sum#78] +Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum#79, sum#80] +Results [3]: [wp_web_page_sk#77, sum#81, sum#82] (67) Exchange -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] -Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] +Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : isnotnull(wr_web_page_sk#84) +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Condition : isnotnull(wr_web_page_sk#88) (72) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#88] +Output [1]: [wp_web_page_sk#92] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#84] -Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] +Left keys [1]: [wr_web_page_sk#88] +Right keys [1]: [cast(wp_web_page_sk#92 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [4]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] -Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] +Output [4]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] +Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] (75) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#93] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#87] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#91] +Right keys [1]: [cast(d_date_sk#93 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Input [5]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88, d_date_sk#6] +Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] +Input [5]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92, d_date_sk#93] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum#89, sum#90] -Results [3]: [wp_web_page_sk#88, sum#91, sum#92] +Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] +Keys [1]: [wp_web_page_sk#92] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum#94, sum#95] +Results [3]: [wp_web_page_sk#92, sum#96, sum#97] (79) Exchange -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [3]: [wp_web_page_sk#92, sum#96, sum#97] +Arguments: hashpartitioning(wp_web_page_sk#92, 5), ENSURE_REQUIREMENTS, [id=#98] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] -Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] +Input [3]: [wp_web_page_sk#92, sum#96, sum#97] +Keys [1]: [wp_web_page_sk#92] +Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] +Results [3]: [wp_web_page_sk#92, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] +Input [3]: [wp_web_page_sk#92, returns#101, profit_loss#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#73] -Right keys [1]: [wp_web_page_sk#88] +Left keys [1]: [wp_web_page_sk#77] +Right keys [1]: [wp_web_page_sk#92] Join condition: None (83) Project [codegen id : 22] -Output [5]: [sales#82, coalesce(returns#96, 0.00) AS returns#99, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#100, web channel AS channel#101, wp_web_page_sk#73 AS id#102] -Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] +Output [5]: [sales#86, coalesce(returns#101, 0.00) AS returns#104, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#105, web channel AS channel#106, wp_web_page_sk#77 AS id#107] +Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#92, returns#101, profit_loss#102] (84) Union (85) Expand [codegen id : 23] -Input [5]: [sales#18, returns#36, profit#37, channel#38, id#39] -Arguments: [List(sales#18, returns#36, profit#37, channel#38, id#39, 0), List(sales#18, returns#36, profit#37, channel#38, null, 1), List(sales#18, returns#36, profit#37, null, null, 3)], [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] +Input [5]: [sales#18, returns#37, profit#38, channel#39, id#40] +Arguments: [List(sales#18, returns#37, profit#38, channel#39, id#40, 0), List(sales#18, returns#37, profit#38, channel#39, null, 1), List(sales#18, returns#37, profit#38, null, null, 3)], [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] (86) HashAggregate [codegen id : 23] -Input [6]: [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] -Keys [3]: [channel#103, id#104, spark_grouping_id#105] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#36), partial_sum(profit#37)] -Aggregate Attributes [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Results [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Input [6]: [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] +Keys [3]: [channel#108, id#109, spark_grouping_id#110] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#37), partial_sum(profit#38)] +Aggregate Attributes [6]: [sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] +Results [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] (87) Exchange -Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Arguments: hashpartitioning(channel#103, id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, [id=#118] +Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Arguments: hashpartitioning(channel#108, id#109, spark_grouping_id#110, 5), ENSURE_REQUIREMENTS, [id=#123] (88) HashAggregate [codegen id : 24] -Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Keys [3]: [channel#103, id#104, spark_grouping_id#105] -Functions [3]: [sum(sales#18), sum(returns#36), sum(profit#37)] -Aggregate Attributes [3]: [sum(sales#18)#119, sum(returns#36)#120, sum(profit#37)#121] -Results [5]: [channel#103, id#104, sum(sales#18)#119 AS sales#122, sum(returns#36)#120 AS returns#123, sum(profit#37)#121 AS profit#124] +Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Keys [3]: [channel#108, id#109, spark_grouping_id#110] +Functions [3]: [sum(sales#18), sum(returns#37), sum(profit#38)] +Aggregate Attributes [3]: [sum(sales#18)#124, sum(returns#37)#125, sum(profit#38)#126] +Results [5]: [channel#108, id#109, sum(sales#18)#124 AS sales#127, sum(returns#37)#125 AS returns#128, sum(profit#38)#126 AS profit#129] (89) TakeOrderedAndProject -Input [5]: [channel#103, id#104, sales#122, returns#123, profit#124] -Arguments: 100, [channel#103 ASC NULLS FIRST, id#104 ASC NULLS FIRST], [channel#103, id#104, sales#122, returns#123, profit#124] +Input [5]: [channel#108, id#109, sales#127, returns#128, profit#129] +Arguments: 100, [channel#108 ASC NULLS FIRST, id#109 ASC NULLS FIRST], [channel#108, id#109, sales#127, returns#128, profit#129] ===== Subqueries ===== @@ -526,14 +526,14 @@ ReusedExchange (91) (91) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#26] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#55 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt index 7a277d5362e32..618da39637e23 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt @@ -197,320 +197,320 @@ Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_s Condition : isnotnull(sr_store_sk#20) (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#25] (24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None (25) Project [codegen id : 6] Output [3]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22] -Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#6] +Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#25] (26) ReusedExchange [Reuses operator id: 14] -Output [1]: [s_store_sk#25] +Output [1]: [s_store_sk#26] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_store_sk#20] -Right keys [1]: [cast(s_store_sk#25 as bigint)] +Right keys [1]: [cast(s_store_sk#26 as bigint)] Join condition: None (28) Project [codegen id : 6] -Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#25] +Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] +Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#26] (29) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Keys [1]: [s_store_sk#25] +Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] +Keys [1]: [s_store_sk#26] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#26, sum#27] -Results [3]: [s_store_sk#25, sum#28, sum#29] +Aggregate Attributes [2]: [sum#27, sum#28] +Results [3]: [s_store_sk#26, sum#29, sum#30] (30) Exchange -Input [3]: [s_store_sk#25, sum#28, sum#29] -Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [s_store_sk#26, sum#29, sum#30] +Arguments: hashpartitioning(s_store_sk#26, 5), ENSURE_REQUIREMENTS, [id=#31] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#25, sum#28, sum#29] -Keys [1]: [s_store_sk#25] +Input [3]: [s_store_sk#26, sum#29, sum#30] +Keys [1]: [s_store_sk#26] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] -Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] +Results [3]: [s_store_sk#26, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] (32) BroadcastExchange -Input [3]: [s_store_sk#25, returns#33, profit_loss#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [3]: [s_store_sk#26, returns#34, profit_loss#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] -Right keys [1]: [s_store_sk#25] +Right keys [1]: [s_store_sk#26] Join condition: None (34) Project [codegen id : 8] -Output [5]: [sales#18, coalesce(returns#33, 0.00) AS returns#36, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#37, store channel AS channel#38, s_store_sk#9 AS id#39] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] +Output [5]: [sales#18, coalesce(returns#34, 0.00) AS returns#37, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#38, store channel AS channel#39, s_store_sk#9 AS id#40] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#26, returns#34, profit_loss#35] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#45] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#43] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cs_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] +Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum#44, sum#45] -Results [3]: [cs_call_center_sk#40, sum#46, sum#47] +Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum#46, sum#47] +Results [3]: [cs_call_center_sk#41, sum#48, sum#49] (41) Exchange -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] -Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] +Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] (43) BroadcastExchange -Input [3]: [cs_call_center_sk#40, sales#51, profit#52] -Arguments: IdentityBroadcastMode, [id=#53] +Input [3]: [cs_call_center_sk#41, sales#53, profit#54] +Arguments: IdentityBroadcastMode, [id=#55] (44) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Output [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#58), dynamicpruningexpression(cr_returned_date_sk#58 IN dynamicpruning#5)] ReadSchema: struct (45) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Input [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] (46) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#59] (47) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#56] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#58] +Right keys [1]: [d_date_sk#59] Join condition: None (48) Project [codegen id : 13] -Output [2]: [cr_return_amount#54, cr_net_loss#55] -Input [4]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56, d_date_sk#6] +Output [2]: [cr_return_amount#56, cr_net_loss#57] +Input [4]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58, d_date_sk#59] (49) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#54, cr_net_loss#55] +Input [2]: [cr_return_amount#56, cr_net_loss#57] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#54)), partial_sum(UnscaledValue(cr_net_loss#55))] -Aggregate Attributes [2]: [sum#57, sum#58] -Results [2]: [sum#59, sum#60] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#56)), partial_sum(UnscaledValue(cr_net_loss#57))] +Aggregate Attributes [2]: [sum#60, sum#61] +Results [2]: [sum#62, sum#63] (50) Exchange -Input [2]: [sum#59, sum#60] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] +Input [2]: [sum#62, sum#63] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#64] (51) HashAggregate -Input [2]: [sum#59, sum#60] +Input [2]: [sum#62, sum#63] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#54)), sum(UnscaledValue(cr_net_loss#55))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#54))#62, sum(UnscaledValue(cr_net_loss#55))#63] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#54))#62,17,2) AS returns#64, MakeDecimal(sum(UnscaledValue(cr_net_loss#55))#63,17,2) AS profit_loss#65] +Functions [2]: [sum(UnscaledValue(cr_return_amount#56)), sum(UnscaledValue(cr_net_loss#57))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#56))#65, sum(UnscaledValue(cr_net_loss#57))#66] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#56))#65,17,2) AS returns#67, MakeDecimal(sum(UnscaledValue(cr_net_loss#57))#66,17,2) AS profit_loss#68] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [sales#51, returns#64, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#65 as decimal(18,2)))), DecimalType(18,2), true) AS profit#66, catalog channel AS channel#67, cs_call_center_sk#40 AS id#68] -Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#64, profit_loss#65] +Output [5]: [sales#53, returns#67, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#68 as decimal(18,2)))), DecimalType(18,2), true) AS profit#69, catalog channel AS channel#70, cs_call_center_sk#41 AS id#71] +Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#67, profit_loss#68] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] -Condition : isnotnull(ws_web_page_sk#69) +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Condition : isnotnull(ws_web_page_sk#72) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#76] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#72] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#75] +Right keys [1]: [d_date_sk#76] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] -Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] +Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] +Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#73] +Output [1]: [wp_web_page_sk#77] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#73] +Input [1]: [wp_web_page_sk#77] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#73] -Condition : isnotnull(wp_web_page_sk#73) +Input [1]: [wp_web_page_sk#77] +Condition : isnotnull(wp_web_page_sk#77) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [1]: [wp_web_page_sk#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#69] -Right keys [1]: [wp_web_page_sk#73] +Left keys [1]: [ws_web_page_sk#72] +Right keys [1]: [wp_web_page_sk#77] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum#75, sum#76] -Results [3]: [wp_web_page_sk#73, sum#77, sum#78] +Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum#79, sum#80] +Results [3]: [wp_web_page_sk#77, sum#81, sum#82] (67) Exchange -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] -Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] +Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : isnotnull(wr_web_page_sk#84) +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Condition : isnotnull(wr_web_page_sk#88) (72) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#92] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#87] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#91] +Right keys [1]: [cast(d_date_sk#92 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86] -Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, d_date_sk#6] +Output [3]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90] +Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, d_date_sk#92] (75) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#88] +Output [1]: [wp_web_page_sk#93] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#84] -Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] +Left keys [1]: [wr_web_page_sk#88] +Right keys [1]: [cast(wp_web_page_sk#93 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum#89, sum#90] -Results [3]: [wp_web_page_sk#88, sum#91, sum#92] +Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] +Keys [1]: [wp_web_page_sk#93] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum#94, sum#95] +Results [3]: [wp_web_page_sk#93, sum#96, sum#97] (79) Exchange -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [3]: [wp_web_page_sk#93, sum#96, sum#97] +Arguments: hashpartitioning(wp_web_page_sk#93, 5), ENSURE_REQUIREMENTS, [id=#98] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] -Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] +Input [3]: [wp_web_page_sk#93, sum#96, sum#97] +Keys [1]: [wp_web_page_sk#93] +Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] +Results [3]: [wp_web_page_sk#93, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] +Input [3]: [wp_web_page_sk#93, returns#101, profit_loss#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#73] -Right keys [1]: [wp_web_page_sk#88] +Left keys [1]: [wp_web_page_sk#77] +Right keys [1]: [wp_web_page_sk#93] Join condition: None (83) Project [codegen id : 22] -Output [5]: [sales#82, coalesce(returns#96, 0.00) AS returns#99, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#100, web channel AS channel#101, wp_web_page_sk#73 AS id#102] -Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] +Output [5]: [sales#86, coalesce(returns#101, 0.00) AS returns#104, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#105, web channel AS channel#106, wp_web_page_sk#77 AS id#107] +Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#93, returns#101, profit_loss#102] (84) Union (85) Expand [codegen id : 23] -Input [5]: [sales#18, returns#36, profit#37, channel#38, id#39] -Arguments: [List(sales#18, returns#36, profit#37, channel#38, id#39, 0), List(sales#18, returns#36, profit#37, channel#38, null, 1), List(sales#18, returns#36, profit#37, null, null, 3)], [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] +Input [5]: [sales#18, returns#37, profit#38, channel#39, id#40] +Arguments: [List(sales#18, returns#37, profit#38, channel#39, id#40, 0), List(sales#18, returns#37, profit#38, channel#39, null, 1), List(sales#18, returns#37, profit#38, null, null, 3)], [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] (86) HashAggregate [codegen id : 23] -Input [6]: [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] -Keys [3]: [channel#103, id#104, spark_grouping_id#105] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#36), partial_sum(profit#37)] -Aggregate Attributes [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Results [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Input [6]: [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] +Keys [3]: [channel#108, id#109, spark_grouping_id#110] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#37), partial_sum(profit#38)] +Aggregate Attributes [6]: [sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] +Results [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] (87) Exchange -Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Arguments: hashpartitioning(channel#103, id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, [id=#118] +Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Arguments: hashpartitioning(channel#108, id#109, spark_grouping_id#110, 5), ENSURE_REQUIREMENTS, [id=#123] (88) HashAggregate [codegen id : 24] -Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Keys [3]: [channel#103, id#104, spark_grouping_id#105] -Functions [3]: [sum(sales#18), sum(returns#36), sum(profit#37)] -Aggregate Attributes [3]: [sum(sales#18)#119, sum(returns#36)#120, sum(profit#37)#121] -Results [5]: [channel#103, id#104, sum(sales#18)#119 AS sales#122, sum(returns#36)#120 AS returns#123, sum(profit#37)#121 AS profit#124] +Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Keys [3]: [channel#108, id#109, spark_grouping_id#110] +Functions [3]: [sum(sales#18), sum(returns#37), sum(profit#38)] +Aggregate Attributes [3]: [sum(sales#18)#124, sum(returns#37)#125, sum(profit#38)#126] +Results [5]: [channel#108, id#109, sum(sales#18)#124 AS sales#127, sum(returns#37)#125 AS returns#128, sum(profit#38)#126 AS profit#129] (89) TakeOrderedAndProject -Input [5]: [channel#103, id#104, sales#122, returns#123, profit#124] -Arguments: 100, [channel#103 ASC NULLS FIRST, id#104 ASC NULLS FIRST], [channel#103, id#104, sales#122, returns#123, profit#124] +Input [5]: [channel#108, id#109, sales#127, returns#128, profit#129] +Arguments: 100, [channel#108 ASC NULLS FIRST, id#109 ASC NULLS FIRST], [channel#108, id#109, sales#127, returns#128, profit#129] ===== Subqueries ===== @@ -526,14 +526,14 @@ ReusedExchange (91) (91) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#25] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#58 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt index ec22a1048c116..b74247d7aef3f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt @@ -248,164 +248,164 @@ Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale Input [9]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#43, d_year#44] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#43] Join condition: None (41) Project [codegen id : 13] -Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] -Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#14, d_year#15] +Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] +Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#43, d_year#44] (42) HashAggregate [codegen id : 13] -Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] -Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] +Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [partial_sum(cs_quantity#34), partial_sum(UnscaledValue(cs_wholesale_cost#35)), partial_sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum#43, sum#44, sum#45] -Results [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] +Aggregate Attributes [3]: [sum#45, sum#46, sum#47] +Results [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] (43) Exchange -Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] -Arguments: hashpartitioning(d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] +Arguments: hashpartitioning(d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#51] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] -Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] +Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [sum(cs_quantity#34), sum(UnscaledValue(cs_wholesale_cost#35)), sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum(cs_quantity#34)#50, sum(UnscaledValue(cs_wholesale_cost#35))#51, sum(UnscaledValue(cs_sales_price#36))#52] -Results [6]: [d_year#15 AS cs_sold_year#53, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#54, sum(cs_quantity#34)#50 AS cs_qty#55, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#51,17,2) AS cs_wc#56, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#52,17,2) AS cs_sp#57] +Aggregate Attributes [3]: [sum(cs_quantity#34)#52, sum(UnscaledValue(cs_wholesale_cost#35))#53, sum(UnscaledValue(cs_sales_price#36))#54] +Results [6]: [d_year#44 AS cs_sold_year#55, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#56, sum(cs_quantity#34)#52 AS cs_qty#57, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#53,17,2) AS cs_wc#58, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#54,17,2) AS cs_sp#59] (45) Filter [codegen id : 14] -Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] -Condition : (coalesce(cs_qty#55, 0) > 0) +Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] +Condition : (coalesce(cs_qty#57, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] -Arguments: [cs_sold_year#53 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] +Arguments: [cs_sold_year#55 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#56 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54] +Right keys [3]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] (49) Scan parquet default.web_sales -Output [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Output [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] (51) Filter [codegen id : 16] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Condition : (isnotnull(ws_item_sk#58) AND isnotnull(ws_bill_customer_sk#59)) +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Condition : (isnotnull(ws_item_sk#60) AND isnotnull(ws_bill_customer_sk#61)) (52) Exchange -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Arguments: hashpartitioning(cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint), 5), ENSURE_REQUIREMENTS, [id=#65] +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Arguments: hashpartitioning(cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint), 5), ENSURE_REQUIREMENTS, [id=#67] (53) Sort [codegen id : 17] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Arguments: [cast(ws_order_number#60 as bigint) ASC NULLS FIRST, cast(ws_item_sk#58 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Arguments: [cast(ws_order_number#62 as bigint) ASC NULLS FIRST, cast(ws_item_sk#60 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.web_returns -Output [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Output [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] (56) Filter [codegen id : 18] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] -Condition : (isnotnull(wr_order_number#67) AND isnotnull(wr_item_sk#66)) +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Condition : (isnotnull(wr_order_number#69) AND isnotnull(wr_item_sk#68)) (57) Project [codegen id : 18] -Output [2]: [wr_item_sk#66, wr_order_number#67] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Output [2]: [wr_item_sk#68, wr_order_number#69] +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] (58) Exchange -Input [2]: [wr_item_sk#66, wr_order_number#67] -Arguments: hashpartitioning(wr_order_number#67, wr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [wr_item_sk#68, wr_order_number#69] +Arguments: hashpartitioning(wr_order_number#69, wr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] (59) Sort [codegen id : 19] -Input [2]: [wr_item_sk#66, wr_order_number#67] -Arguments: [wr_order_number#67 ASC NULLS FIRST, wr_item_sk#66 ASC NULLS FIRST], false, 0 +Input [2]: [wr_item_sk#68, wr_order_number#69] +Arguments: [wr_order_number#69 ASC NULLS FIRST, wr_item_sk#68 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint)] -Right keys [2]: [wr_order_number#67, wr_item_sk#66] +Left keys [2]: [cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint)] +Right keys [2]: [wr_order_number#69, wr_item_sk#68] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] -Condition : isnull(wr_order_number#67) +Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] +Condition : isnull(wr_order_number#69) (62) Project [codegen id : 21] -Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] +Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#72, d_year#73] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [ws_sold_date_sk#64] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#66] +Right keys [1]: [d_date_sk#72] Join condition: None (65) Project [codegen id : 21] -Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] -Input [8]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, d_date_sk#14, d_year#15] +Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] +Input [8]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, d_date_sk#72, d_year#73] (66) HashAggregate [codegen id : 21] -Input [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] -Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] -Functions [3]: [partial_sum(ws_quantity#61), partial_sum(UnscaledValue(ws_wholesale_cost#62)), partial_sum(UnscaledValue(ws_sales_price#63))] -Aggregate Attributes [3]: [sum#70, sum#71, sum#72] -Results [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] +Input [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] +Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] +Functions [3]: [partial_sum(ws_quantity#63), partial_sum(UnscaledValue(ws_wholesale_cost#64)), partial_sum(UnscaledValue(ws_sales_price#65))] +Aggregate Attributes [3]: [sum#74, sum#75, sum#76] +Results [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] (67) Exchange -Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] +Arguments: hashpartitioning(d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#80] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] -Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] -Functions [3]: [sum(ws_quantity#61), sum(UnscaledValue(ws_wholesale_cost#62)), sum(UnscaledValue(ws_sales_price#63))] -Aggregate Attributes [3]: [sum(ws_quantity#61)#77, sum(UnscaledValue(ws_wholesale_cost#62))#78, sum(UnscaledValue(ws_sales_price#63))#79] -Results [6]: [d_year#15 AS ws_sold_year#80, ws_item_sk#58, ws_bill_customer_sk#59 AS ws_customer_sk#81, sum(ws_quantity#61)#77 AS ws_qty#82, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#62))#78,17,2) AS ws_wc#83, MakeDecimal(sum(UnscaledValue(ws_sales_price#63))#79,17,2) AS ws_sp#84] +Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] +Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] +Functions [3]: [sum(ws_quantity#63), sum(UnscaledValue(ws_wholesale_cost#64)), sum(UnscaledValue(ws_sales_price#65))] +Aggregate Attributes [3]: [sum(ws_quantity#63)#81, sum(UnscaledValue(ws_wholesale_cost#64))#82, sum(UnscaledValue(ws_sales_price#65))#83] +Results [6]: [d_year#73 AS ws_sold_year#84, ws_item_sk#60, ws_bill_customer_sk#61 AS ws_customer_sk#85, sum(ws_quantity#63)#81 AS ws_qty#86, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#64))#82,17,2) AS ws_wc#87, MakeDecimal(sum(UnscaledValue(ws_sales_price#65))#83,17,2) AS ws_sp#88] (69) Filter [codegen id : 22] -Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] -Condition : (coalesce(ws_qty#82, 0) > 0) +Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] +Condition : (coalesce(ws_qty#86, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] -Arguments: [ws_sold_year#80 ASC NULLS FIRST, ws_item_sk#58 ASC NULLS FIRST, ws_customer_sk#81 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] +Arguments: [ws_sold_year#84 ASC NULLS FIRST, ws_item_sk#60 ASC NULLS FIRST, ws_customer_sk#85 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81] +Right keys [3]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85] Join condition: None (72) Project [codegen id : 23] -Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#82 + cs_qty#55), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#82, 0) + coalesce(cs_qty#55, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#83, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#56, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#84, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#57, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#82, cs_qty#55] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57, ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] +Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#86 + cs_qty#57), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#86, 0) + coalesce(cs_qty#57, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#87, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#58, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#88, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#59, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#86, cs_qty#57] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59, ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] (73) TakeOrderedAndProject -Input [12]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#82, cs_qty#55] -Arguments: 100, [ratio#85 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#82 + cs_qty#55), 1) as double)), 2) ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] +Input [12]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#86, cs_qty#57] +Arguments: 100, [ratio#89 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#86 + cs_qty#57), 1) as double)), 2) ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt index f5eff516f9b36..c7da3a1e23c62 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt @@ -248,164 +248,164 @@ Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale Input [9]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_order_number#33, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, wr_item_sk#39, wr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#43, d_year#44] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#37] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#43] Join condition: None (41) Project [codegen id : 13] -Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] -Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#14, d_year#15] +Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] +Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#43, d_year#44] (42) HashAggregate [codegen id : 13] -Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] -Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] +Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [partial_sum(ws_quantity#34), partial_sum(UnscaledValue(ws_wholesale_cost#35)), partial_sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum#43, sum#44, sum#45] -Results [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] +Aggregate Attributes [3]: [sum#45, sum#46, sum#47] +Results [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] (43) Exchange -Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] -Arguments: hashpartitioning(d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] +Arguments: hashpartitioning(d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#51] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] -Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] +Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [sum(ws_quantity#34), sum(UnscaledValue(ws_wholesale_cost#35)), sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum(ws_quantity#34)#50, sum(UnscaledValue(ws_wholesale_cost#35))#51, sum(UnscaledValue(ws_sales_price#36))#52] -Results [6]: [d_year#15 AS ws_sold_year#53, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#54, sum(ws_quantity#34)#50 AS ws_qty#55, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#51,17,2) AS ws_wc#56, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#52,17,2) AS ws_sp#57] +Aggregate Attributes [3]: [sum(ws_quantity#34)#52, sum(UnscaledValue(ws_wholesale_cost#35))#53, sum(UnscaledValue(ws_sales_price#36))#54] +Results [6]: [d_year#44 AS ws_sold_year#55, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#56, sum(ws_quantity#34)#52 AS ws_qty#57, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#53,17,2) AS ws_wc#58, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#54,17,2) AS ws_sp#59] (45) Filter [codegen id : 14] -Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] -Condition : (coalesce(ws_qty#55, 0) > 0) +Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] +Condition : (coalesce(ws_qty#57, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] -Arguments: [ws_sold_year#53 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] +Arguments: [ws_sold_year#55 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#56 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54] +Right keys [3]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] (49) Scan parquet default.catalog_sales -Output [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Output [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#66), dynamicpruningexpression(cs_sold_date_sk#66 IN dynamicpruning#8)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] (51) Filter [codegen id : 16] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Condition : (isnotnull(cs_item_sk#59) AND isnotnull(cs_bill_customer_sk#58)) +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Condition : (isnotnull(cs_item_sk#61) AND isnotnull(cs_bill_customer_sk#60)) (52) Exchange -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Arguments: hashpartitioning(cs_order_number#60, cs_item_sk#59, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Arguments: hashpartitioning(cs_order_number#62, cs_item_sk#61, 5), ENSURE_REQUIREMENTS, [id=#67] (53) Sort [codegen id : 17] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Arguments: [cs_order_number#60 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST], false, 0 +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Arguments: [cs_order_number#62 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST], false, 0 (54) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Output [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] (56) Filter [codegen id : 18] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] -Condition : (isnotnull(cr_order_number#67) AND isnotnull(cr_item_sk#66)) +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Condition : (isnotnull(cr_order_number#69) AND isnotnull(cr_item_sk#68)) (57) Project [codegen id : 18] -Output [2]: [cr_item_sk#66, cr_order_number#67] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Output [2]: [cr_item_sk#68, cr_order_number#69] +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] (58) Exchange -Input [2]: [cr_item_sk#66, cr_order_number#67] -Arguments: hashpartitioning(cr_order_number#67, cr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [cr_item_sk#68, cr_order_number#69] +Arguments: hashpartitioning(cr_order_number#69, cr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] (59) Sort [codegen id : 19] -Input [2]: [cr_item_sk#66, cr_order_number#67] -Arguments: [cr_order_number#67 ASC NULLS FIRST, cr_item_sk#66 ASC NULLS FIRST], false, 0 +Input [2]: [cr_item_sk#68, cr_order_number#69] +Arguments: [cr_order_number#69 ASC NULLS FIRST, cr_item_sk#68 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cs_order_number#60, cs_item_sk#59] -Right keys [2]: [cr_order_number#67, cr_item_sk#66] +Left keys [2]: [cs_order_number#62, cs_item_sk#61] +Right keys [2]: [cr_order_number#69, cr_item_sk#68] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] -Condition : isnull(cr_order_number#67) +Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] +Condition : isnull(cr_order_number#69) (62) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] +Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#72, d_year#73] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [cs_sold_date_sk#64] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [cs_sold_date_sk#66] +Right keys [1]: [d_date_sk#72] Join condition: None (65) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] -Input [8]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, d_date_sk#14, d_year#15] +Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] +Input [8]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, d_date_sk#72, d_year#73] (66) HashAggregate [codegen id : 21] -Input [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] -Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] -Functions [3]: [partial_sum(cs_quantity#61), partial_sum(UnscaledValue(cs_wholesale_cost#62)), partial_sum(UnscaledValue(cs_sales_price#63))] -Aggregate Attributes [3]: [sum#70, sum#71, sum#72] -Results [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] +Input [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] +Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] +Functions [3]: [partial_sum(cs_quantity#63), partial_sum(UnscaledValue(cs_wholesale_cost#64)), partial_sum(UnscaledValue(cs_sales_price#65))] +Aggregate Attributes [3]: [sum#74, sum#75, sum#76] +Results [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] (67) Exchange -Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] +Arguments: hashpartitioning(d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#80] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] -Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] -Functions [3]: [sum(cs_quantity#61), sum(UnscaledValue(cs_wholesale_cost#62)), sum(UnscaledValue(cs_sales_price#63))] -Aggregate Attributes [3]: [sum(cs_quantity#61)#77, sum(UnscaledValue(cs_wholesale_cost#62))#78, sum(UnscaledValue(cs_sales_price#63))#79] -Results [6]: [d_year#15 AS cs_sold_year#80, cs_item_sk#59, cs_bill_customer_sk#58 AS cs_customer_sk#81, sum(cs_quantity#61)#77 AS cs_qty#82, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#62))#78,17,2) AS cs_wc#83, MakeDecimal(sum(UnscaledValue(cs_sales_price#63))#79,17,2) AS cs_sp#84] +Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] +Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] +Functions [3]: [sum(cs_quantity#63), sum(UnscaledValue(cs_wholesale_cost#64)), sum(UnscaledValue(cs_sales_price#65))] +Aggregate Attributes [3]: [sum(cs_quantity#63)#81, sum(UnscaledValue(cs_wholesale_cost#64))#82, sum(UnscaledValue(cs_sales_price#65))#83] +Results [6]: [d_year#73 AS cs_sold_year#84, cs_item_sk#61, cs_bill_customer_sk#60 AS cs_customer_sk#85, sum(cs_quantity#63)#81 AS cs_qty#86, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#64))#82,17,2) AS cs_wc#87, MakeDecimal(sum(UnscaledValue(cs_sales_price#65))#83,17,2) AS cs_sp#88] (69) Filter [codegen id : 22] -Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] -Condition : (coalesce(cs_qty#82, 0) > 0) +Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] +Condition : (coalesce(cs_qty#86, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] -Arguments: [cs_sold_year#80 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST, cs_customer_sk#81 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] +Arguments: [cs_sold_year#84 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST, cs_customer_sk#85 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81] +Right keys [3]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85] Join condition: None (72) Project [codegen id : 23] -Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#55 + cs_qty#82), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#55, 0) + coalesce(cs_qty#82, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#56, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#83, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#57, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#84, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, cs_qty#82] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57, cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] +Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#57 + cs_qty#86), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#57, 0) + coalesce(cs_qty#86, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#58, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#87, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#59, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#88, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, cs_qty#86] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59, cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] (73) TakeOrderedAndProject -Input [12]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, cs_qty#82] -Arguments: 100, [ratio#85 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#55 + cs_qty#82), 1) as double)), 2) ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] +Input [12]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, cs_qty#86] +Arguments: 100, [ratio#89 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#57 + cs_qty#86), 1) as double)), 2) ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt index 48521c72281f4..d13b9623d1b34 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt @@ -149,157 +149,157 @@ Input [1]: [ca_zip#14] Condition : (substr(ca_zip#14, 1, 5) INSET (56910,69952,63792,39371,74351,11101,25003,97189,57834,73134,62377,51200,32754,22752,86379,14171,91110,40162,98569,28709,13394,66162,25733,25782,26065,18383,51949,87343,50298,83849,33786,64528,23470,67030,46136,25280,46820,77721,99076,18426,31880,17871,98235,45748,49156,18652,72013,51622,43848,78567,41248,13695,44165,67853,54917,53179,64034,10567,71791,68908,55565,59402,64147,85816,57855,61547,27700,68100,28810,58263,15723,83933,51103,58058,90578,82276,81096,81426,96451,77556,38607,76638,18906,62971,57047,48425,35576,11928,30625,83444,73520,51650,57647,60099,30122,94983,24128,10445,41368,26233,26859,21756,24676,19849,36420,38193,58470,39127,13595,87501,24317,15455,69399,98025,81019,48033,11376,39516,67875,92712,14867,38122,29741,42961,30469,51211,56458,15559,16021,33123,33282,33515,72823,54601,76698,56240,72175,60279,20004,68806,72325,28488,43933,50412,45200,22246,78668,79777,96765,67301,73273,49448,82636,23932,47305,29839,39192,18799,61265,37125,58943,64457,88424,24610,84935,89360,68893,30431,28898,10336,90257,59166,46081,26105,96888,36634,86284,35258,39972,22927,73241,53268,24206,27385,99543,31671,14663,30903,39861,24996,63089,88086,83921,21076,67897,66708,45721,60576,25103,52867,30450,36233,30010,96576,73171,56571,56575,64544,13955,78451,43285,18119,16725,83041,76107,79994,54364,35942,56691,19769,63435,34102,18845,22744,13354,75691,45549,23968,31387,83144,13375,15765,28577,88190,19736,73650,37930,25989,83926,94898,51798,39736,22437,55253,38415,71256,18376,42029,25858,44438,19515,38935,51649,71954,15882,18767,63193,25486,49130,37126,40604,34425,17043,12305,11634,26653,94167,36446,10516,67473,66864,72425,63981,18842,22461,42666,47770,69035,70372,28587,45266,15371,15798,45375,90225,16807,31016,68014,21337,19505,50016,10144,84093,21286,19430,34322,91068,94945,72305,24671,58048,65084,28545,21195,20548,22245,77191,96976,48583,76231,15734,61810,11356,68621,68786,98359,41367,26689,69913,76614,68101,88885,50308,79077,18270,28915,29178,53672,62878,10390,14922,68341,56529,41766,68309,56616,15126,61860,97789,11489,45692,41918,72151,72550,27156,36495,70738,17879,53535,17920,68880,78890,35850,14089,58078,65164,27068,26231,13376,57665,32213,77610,87816,21309,15146,86198,91137,55307,67467,40558,94627,82136,22351,89091,20260,23006,91393,47537,62496,98294,18840,71286,81312,31029,70466,35458,14060,22685,28286,25631,19512,40081,63837,14328,35474,22152,76232,51061,86057,17183) AND isnotnull(substr(ca_zip#14, 1, 5))) (22) Scan parquet default.customer_address -Output [2]: [ca_address_sk#15, ca_zip#14] +Output [2]: [ca_address_sk#15, ca_zip#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 5] -Input [2]: [ca_address_sk#15, ca_zip#14] +Input [2]: [ca_address_sk#15, ca_zip#16] (24) Filter [codegen id : 5] -Input [2]: [ca_address_sk#15, ca_zip#14] +Input [2]: [ca_address_sk#15, ca_zip#16] Condition : isnotnull(ca_address_sk#15) (25) Exchange -Input [2]: [ca_address_sk#15, ca_zip#14] -Arguments: hashpartitioning(ca_address_sk#15, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [2]: [ca_address_sk#15, ca_zip#16] +Arguments: hashpartitioning(ca_address_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] (26) Sort [codegen id : 6] -Input [2]: [ca_address_sk#15, ca_zip#14] +Input [2]: [ca_address_sk#15, ca_zip#16] Arguments: [ca_address_sk#15 ASC NULLS FIRST], false, 0 (27) Scan parquet default.customer -Output [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] +Output [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 7] -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] +Input [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] (29) Filter [codegen id : 7] -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Condition : ((isnotnull(c_preferred_cust_flag#18) AND (c_preferred_cust_flag#18 = Y)) AND isnotnull(c_current_addr_sk#17)) +Input [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] +Condition : ((isnotnull(c_preferred_cust_flag#19) AND (c_preferred_cust_flag#19 = Y)) AND isnotnull(c_current_addr_sk#18)) (30) Project [codegen id : 7] -Output [1]: [c_current_addr_sk#17] -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] +Output [1]: [c_current_addr_sk#18] +Input [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] (31) Exchange -Input [1]: [c_current_addr_sk#17] -Arguments: hashpartitioning(c_current_addr_sk#17, 5), ENSURE_REQUIREMENTS, [id=#19] +Input [1]: [c_current_addr_sk#18] +Arguments: hashpartitioning(c_current_addr_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] (32) Sort [codegen id : 8] -Input [1]: [c_current_addr_sk#17] -Arguments: [c_current_addr_sk#17 ASC NULLS FIRST], false, 0 +Input [1]: [c_current_addr_sk#18] +Arguments: [c_current_addr_sk#18 ASC NULLS FIRST], false, 0 (33) SortMergeJoin [codegen id : 9] Left keys [1]: [ca_address_sk#15] -Right keys [1]: [c_current_addr_sk#17] +Right keys [1]: [c_current_addr_sk#18] Join condition: None (34) Project [codegen id : 9] -Output [1]: [ca_zip#14] -Input [3]: [ca_address_sk#15, ca_zip#14, c_current_addr_sk#17] +Output [1]: [ca_zip#16] +Input [3]: [ca_address_sk#15, ca_zip#16, c_current_addr_sk#18] (35) HashAggregate [codegen id : 9] -Input [1]: [ca_zip#14] -Keys [1]: [ca_zip#14] +Input [1]: [ca_zip#16] +Keys [1]: [ca_zip#16] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#20] -Results [2]: [ca_zip#14, count#21] +Aggregate Attributes [1]: [count#21] +Results [2]: [ca_zip#16, count#22] (36) Exchange -Input [2]: [ca_zip#14, count#21] -Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [2]: [ca_zip#16, count#22] +Arguments: hashpartitioning(ca_zip#16, 5), ENSURE_REQUIREMENTS, [id=#23] (37) HashAggregate [codegen id : 10] -Input [2]: [ca_zip#14, count#21] -Keys [1]: [ca_zip#14] +Input [2]: [ca_zip#16, count#22] +Keys [1]: [ca_zip#16] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#23] -Results [2]: [substr(ca_zip#14, 1, 5) AS ca_zip#24, count(1)#23 AS count(1)#25] +Aggregate Attributes [1]: [count(1)#24] +Results [2]: [substr(ca_zip#16, 1, 5) AS ca_zip#25, count(1)#24 AS count(1)#26] (38) Filter [codegen id : 10] -Input [2]: [ca_zip#24, count(1)#25] -Condition : (count(1)#25 > 10) +Input [2]: [ca_zip#25, count(1)#26] +Condition : (count(1)#26 > 10) (39) Project [codegen id : 10] -Output [1]: [ca_zip#24] -Input [2]: [ca_zip#24, count(1)#25] +Output [1]: [ca_zip#25] +Input [2]: [ca_zip#25, count(1)#26] (40) BroadcastExchange -Input [1]: [ca_zip#24] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#26] +Input [1]: [ca_zip#25] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#27] (41) BroadcastHashJoin [codegen id : 11] Left keys [2]: [coalesce(substr(ca_zip#14, 1, 5), ), isnull(substr(ca_zip#14, 1, 5))] -Right keys [2]: [coalesce(ca_zip#24, ), isnull(ca_zip#24)] +Right keys [2]: [coalesce(ca_zip#25, ), isnull(ca_zip#25)] Join condition: None (42) Project [codegen id : 11] -Output [1]: [substr(ca_zip#14, 1, 5) AS ca_zip#27] +Output [1]: [substr(ca_zip#14, 1, 5) AS ca_zip#28] Input [1]: [ca_zip#14] (43) HashAggregate [codegen id : 11] -Input [1]: [ca_zip#27] -Keys [1]: [ca_zip#27] +Input [1]: [ca_zip#28] +Keys [1]: [ca_zip#28] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#27] +Results [1]: [ca_zip#28] (44) Exchange -Input [1]: [ca_zip#27] -Arguments: hashpartitioning(ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [1]: [ca_zip#28] +Arguments: hashpartitioning(ca_zip#28, 5), ENSURE_REQUIREMENTS, [id=#29] (45) HashAggregate [codegen id : 12] -Input [1]: [ca_zip#27] -Keys [1]: [ca_zip#27] +Input [1]: [ca_zip#28] +Keys [1]: [ca_zip#28] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#27] +Results [1]: [ca_zip#28] (46) Exchange -Input [1]: [ca_zip#27] -Arguments: hashpartitioning(substr(ca_zip#27, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#29] +Input [1]: [ca_zip#28] +Arguments: hashpartitioning(substr(ca_zip#28, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#30] (47) Sort [codegen id : 13] -Input [1]: [ca_zip#27] -Arguments: [substr(ca_zip#27, 1, 2) ASC NULLS FIRST], false, 0 +Input [1]: [ca_zip#28] +Arguments: [substr(ca_zip#28, 1, 2) ASC NULLS FIRST], false, 0 (48) SortMergeJoin [codegen id : 14] Left keys [1]: [substr(s_zip#11, 1, 2)] -Right keys [1]: [substr(ca_zip#27, 1, 2)] +Right keys [1]: [substr(ca_zip#28, 1, 2)] Join condition: None (49) Project [codegen id : 14] Output [2]: [ss_net_profit#2, s_store_name#10] -Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#27] +Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#28] (50) HashAggregate [codegen id : 14] Input [2]: [ss_net_profit#2, s_store_name#10] Keys [1]: [s_store_name#10] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#30] -Results [2]: [s_store_name#10, sum#31] +Aggregate Attributes [1]: [sum#31] +Results [2]: [s_store_name#10, sum#32] (51) Exchange -Input [2]: [s_store_name#10, sum#31] -Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [2]: [s_store_name#10, sum#32] +Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#33] (52) HashAggregate [codegen id : 15] -Input [2]: [s_store_name#10, sum#31] +Input [2]: [s_store_name#10, sum#32] Keys [1]: [s_store_name#10] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#33] -Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#33,17,2) AS sum(ss_net_profit)#34] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#34] +Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS sum(ss_net_profit)#35] (53) TakeOrderedAndProject -Input [2]: [s_store_name#10, sum(ss_net_profit)#34] -Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#34] +Input [2]: [s_store_name#10, sum(ss_net_profit)#35] +Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#35] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt index 09e1189003ed6..dbb5e1f606a8d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt @@ -135,141 +135,141 @@ Input [1]: [ca_zip#13] Condition : (substr(ca_zip#13, 1, 5) INSET (56910,69952,63792,39371,74351,11101,25003,97189,57834,73134,62377,51200,32754,22752,86379,14171,91110,40162,98569,28709,13394,66162,25733,25782,26065,18383,51949,87343,50298,83849,33786,64528,23470,67030,46136,25280,46820,77721,99076,18426,31880,17871,98235,45748,49156,18652,72013,51622,43848,78567,41248,13695,44165,67853,54917,53179,64034,10567,71791,68908,55565,59402,64147,85816,57855,61547,27700,68100,28810,58263,15723,83933,51103,58058,90578,82276,81096,81426,96451,77556,38607,76638,18906,62971,57047,48425,35576,11928,30625,83444,73520,51650,57647,60099,30122,94983,24128,10445,41368,26233,26859,21756,24676,19849,36420,38193,58470,39127,13595,87501,24317,15455,69399,98025,81019,48033,11376,39516,67875,92712,14867,38122,29741,42961,30469,51211,56458,15559,16021,33123,33282,33515,72823,54601,76698,56240,72175,60279,20004,68806,72325,28488,43933,50412,45200,22246,78668,79777,96765,67301,73273,49448,82636,23932,47305,29839,39192,18799,61265,37125,58943,64457,88424,24610,84935,89360,68893,30431,28898,10336,90257,59166,46081,26105,96888,36634,86284,35258,39972,22927,73241,53268,24206,27385,99543,31671,14663,30903,39861,24996,63089,88086,83921,21076,67897,66708,45721,60576,25103,52867,30450,36233,30010,96576,73171,56571,56575,64544,13955,78451,43285,18119,16725,83041,76107,79994,54364,35942,56691,19769,63435,34102,18845,22744,13354,75691,45549,23968,31387,83144,13375,15765,28577,88190,19736,73650,37930,25989,83926,94898,51798,39736,22437,55253,38415,71256,18376,42029,25858,44438,19515,38935,51649,71954,15882,18767,63193,25486,49130,37126,40604,34425,17043,12305,11634,26653,94167,36446,10516,67473,66864,72425,63981,18842,22461,42666,47770,69035,70372,28587,45266,15371,15798,45375,90225,16807,31016,68014,21337,19505,50016,10144,84093,21286,19430,34322,91068,94945,72305,24671,58048,65084,28545,21195,20548,22245,77191,96976,48583,76231,15734,61810,11356,68621,68786,98359,41367,26689,69913,76614,68101,88885,50308,79077,18270,28915,29178,53672,62878,10390,14922,68341,56529,41766,68309,56616,15126,61860,97789,11489,45692,41918,72151,72550,27156,36495,70738,17879,53535,17920,68880,78890,35850,14089,58078,65164,27068,26231,13376,57665,32213,77610,87816,21309,15146,86198,91137,55307,67467,40558,94627,82136,22351,89091,20260,23006,91393,47537,62496,98294,18840,71286,81312,31029,70466,35458,14060,22685,28286,25631,19512,40081,63837,14328,35474,22152,76232,51061,86057,17183) AND isnotnull(substr(ca_zip#13, 1, 5))) (20) Scan parquet default.customer_address -Output [2]: [ca_address_sk#14, ca_zip#13] +Output [2]: [ca_address_sk#14, ca_zip#15] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#14, ca_zip#13] +Input [2]: [ca_address_sk#14, ca_zip#15] (22) Filter [codegen id : 4] -Input [2]: [ca_address_sk#14, ca_zip#13] +Input [2]: [ca_address_sk#14, ca_zip#15] Condition : isnotnull(ca_address_sk#14) (23) Scan parquet default.customer -Output [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Output [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 3] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] (25) Filter [codegen id : 3] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -Condition : ((isnotnull(c_preferred_cust_flag#16) AND (c_preferred_cust_flag#16 = Y)) AND isnotnull(c_current_addr_sk#15)) +Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Condition : ((isnotnull(c_preferred_cust_flag#17) AND (c_preferred_cust_flag#17 = Y)) AND isnotnull(c_current_addr_sk#16)) (26) Project [codegen id : 3] -Output [1]: [c_current_addr_sk#15] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Output [1]: [c_current_addr_sk#16] +Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] (27) BroadcastExchange -Input [1]: [c_current_addr_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Input [1]: [c_current_addr_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (28) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ca_address_sk#14] -Right keys [1]: [c_current_addr_sk#15] +Right keys [1]: [c_current_addr_sk#16] Join condition: None (29) Project [codegen id : 4] -Output [1]: [ca_zip#13] -Input [3]: [ca_address_sk#14, ca_zip#13, c_current_addr_sk#15] +Output [1]: [ca_zip#15] +Input [3]: [ca_address_sk#14, ca_zip#15, c_current_addr_sk#16] (30) HashAggregate [codegen id : 4] -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] +Input [1]: [ca_zip#15] +Keys [1]: [ca_zip#15] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#18] -Results [2]: [ca_zip#13, count#19] +Aggregate Attributes [1]: [count#19] +Results [2]: [ca_zip#15, count#20] (31) Exchange -Input [2]: [ca_zip#13, count#19] -Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [2]: [ca_zip#15, count#20] +Arguments: hashpartitioning(ca_zip#15, 5), ENSURE_REQUIREMENTS, [id=#21] (32) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#13, count#19] -Keys [1]: [ca_zip#13] +Input [2]: [ca_zip#15, count#20] +Keys [1]: [ca_zip#15] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#21] -Results [2]: [substr(ca_zip#13, 1, 5) AS ca_zip#22, count(1)#21 AS count(1)#23] +Aggregate Attributes [1]: [count(1)#22] +Results [2]: [substr(ca_zip#15, 1, 5) AS ca_zip#23, count(1)#22 AS count(1)#24] (33) Filter [codegen id : 5] -Input [2]: [ca_zip#22, count(1)#23] -Condition : (count(1)#23 > 10) +Input [2]: [ca_zip#23, count(1)#24] +Condition : (count(1)#24 > 10) (34) Project [codegen id : 5] -Output [1]: [ca_zip#22] -Input [2]: [ca_zip#22, count(1)#23] +Output [1]: [ca_zip#23] +Input [2]: [ca_zip#23, count(1)#24] (35) BroadcastExchange -Input [1]: [ca_zip#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#24] +Input [1]: [ca_zip#23] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#25] (36) BroadcastHashJoin [codegen id : 6] Left keys [2]: [coalesce(substr(ca_zip#13, 1, 5), ), isnull(substr(ca_zip#13, 1, 5))] -Right keys [2]: [coalesce(ca_zip#22, ), isnull(ca_zip#22)] +Right keys [2]: [coalesce(ca_zip#23, ), isnull(ca_zip#23)] Join condition: None (37) Project [codegen id : 6] -Output [1]: [substr(ca_zip#13, 1, 5) AS ca_zip#25] +Output [1]: [substr(ca_zip#13, 1, 5) AS ca_zip#26] Input [1]: [ca_zip#13] (38) HashAggregate [codegen id : 6] -Input [1]: [ca_zip#25] -Keys [1]: [ca_zip#25] +Input [1]: [ca_zip#26] +Keys [1]: [ca_zip#26] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#25] +Results [1]: [ca_zip#26] (39) Exchange -Input [1]: [ca_zip#25] -Arguments: hashpartitioning(ca_zip#25, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [1]: [ca_zip#26] +Arguments: hashpartitioning(ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#27] (40) HashAggregate [codegen id : 7] -Input [1]: [ca_zip#25] -Keys [1]: [ca_zip#25] +Input [1]: [ca_zip#26] +Keys [1]: [ca_zip#26] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#25] +Results [1]: [ca_zip#26] (41) BroadcastExchange -Input [1]: [ca_zip#25] -Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [id=#27] +Input [1]: [ca_zip#26] +Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [id=#28] (42) BroadcastHashJoin [codegen id : 8] Left keys [1]: [substr(s_zip#11, 1, 2)] -Right keys [1]: [substr(ca_zip#25, 1, 2)] +Right keys [1]: [substr(ca_zip#26, 1, 2)] Join condition: None (43) Project [codegen id : 8] Output [2]: [ss_net_profit#2, s_store_name#10] -Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#25] +Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#26] (44) HashAggregate [codegen id : 8] Input [2]: [ss_net_profit#2, s_store_name#10] Keys [1]: [s_store_name#10] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [s_store_name#10, sum#29] +Aggregate Attributes [1]: [sum#29] +Results [2]: [s_store_name#10, sum#30] (45) Exchange -Input [2]: [s_store_name#10, sum#29] -Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [s_store_name#10, sum#30] +Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#31] (46) HashAggregate [codegen id : 9] -Input [2]: [s_store_name#10, sum#29] +Input [2]: [s_store_name#10, sum#30] Keys [1]: [s_store_name#10] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] -Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS sum(ss_net_profit)#32] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#32] +Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#32,17,2) AS sum(ss_net_profit)#33] (47) TakeOrderedAndProject -Input [2]: [s_store_name#10, sum(ss_net_profit)#32] -Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#32] +Input [2]: [s_store_name#10, sum(ss_net_profit)#33] +Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#33] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt index 6e86260f0aba8..bdb1a52a18f2d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt @@ -244,7 +244,7 @@ Input [2]: [d_date_sk#22, d_date#23] (30) Filter [codegen id : 7] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11192)) AND (d_date#23 <= 11222)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-22)) AND isnotnull(d_date_sk#22)) (31) Project [codegen id : 7] Output [1]: [d_date_sk#22] @@ -367,252 +367,252 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#16] +Output [1]: [i_item_sk#61] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#16] +Right keys [1]: [i_item_sk#61] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#16] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#61] (60) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#19] +Output [1]: [p_promo_sk#62] (61) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#19] +Right keys [1]: [p_promo_sk#62] Join condition: None (62) Project [codegen id : 19] Output [6]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#19] +Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#62] (63) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#63] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#63] Join condition: None (65) Project [codegen id : 19] Output [5]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#22] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#63] (66) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 18] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] (68) Filter [codegen id : 18] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Condition : isnotnull(cp_catalog_page_sk#61) +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Condition : isnotnull(cp_catalog_page_sk#64) (69) BroadcastExchange -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#61] +Right keys [1]: [cp_catalog_page_sk#64] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#64, cp_catalog_page_id#65] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Keys [1]: [cp_catalog_page_id#62] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] +Keys [1]: [cp_catalog_page_id#65] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Results [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] (73) Exchange -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [id=#77] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [cp_catalog_page_id#62] +Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Keys [1]: [cp_catalog_page_id#65] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#80, catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#62) AS id#82] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] +Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#81, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#82, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#83, catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#65) AS id#85] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) (78) Exchange -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] -Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (84) Exchange -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] -Right keys [2]: [wr_item_sk#91, wr_order_number#92] +Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] +Right keys [2]: [wr_item_sk#94, wr_order_number#95] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#16] +Output [1]: [i_item_sk#100] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#83] -Right keys [1]: [i_item_sk#16] +Left keys [1]: [ws_item_sk#86] +Right keys [1]: [i_item_sk#100] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, i_item_sk#16] +Output [7]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, i_item_sk#100] (91) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#19] +Output [1]: [p_promo_sk#101] (92) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#85] -Right keys [1]: [p_promo_sk#19] +Left keys [1]: [ws_promo_sk#88] +Right keys [1]: [p_promo_sk#101] Join condition: None (93) Project [codegen id : 29] -Output [6]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [8]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, p_promo_sk#19] +Output [6]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [8]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, p_promo_sk#101] (94) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#102] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#89] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [ws_sold_date_sk#92] +Right keys [1]: [d_date_sk#102] Join condition: None (96) Project [codegen id : 29] -Output [5]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] -Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#22] +Output [5]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] +Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#102] (97) Scan parquet default.web_site -Output [2]: [web_site_sk#97, web_site_id#98] +Output [2]: [web_site_sk#103, web_site_id#104] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (98) ColumnarToRow [codegen id : 28] -Input [2]: [web_site_sk#97, web_site_id#98] +Input [2]: [web_site_sk#103, web_site_id#104] (99) Filter [codegen id : 28] -Input [2]: [web_site_sk#97, web_site_id#98] -Condition : isnotnull(web_site_sk#97) +Input [2]: [web_site_sk#103, web_site_id#104] +Condition : isnotnull(web_site_sk#103) (100) BroadcastExchange -Input [2]: [web_site_sk#97, web_site_id#98] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Input [2]: [web_site_sk#103, web_site_id#104] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#105] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#84] -Right keys [1]: [web_site_sk#97] +Left keys [1]: [ws_web_site_sk#87] +Right keys [1]: [web_site_sk#103] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] +Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] +Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#103, web_site_id#104] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Keys [1]: [web_site_id#98] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] +Keys [1]: [web_site_id#104] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Results [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] (104) Exchange -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] +Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, [id=#116] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Keys [1]: [web_site_id#98] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#114, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#115, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#116, web channel AS channel#117, concat(web_site, web_site_id#98) AS id#118] +Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Keys [1]: [web_site_id#104] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] +Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#120, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#121, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#122, web channel AS channel#123, concat(web_site, web_site_id#104) AS id#124] (106) Union (107) Expand [codegen id : 31] Input [5]: [sales#42, returns#43, profit#44, channel#45, id#46] -Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] +Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] (108) HashAggregate [codegen id : 31] -Input [6]: [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] -Keys [3]: [channel#119, id#120, spark_grouping_id#121] +Input [6]: [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] +Keys [3]: [channel#125, id#126, spark_grouping_id#127] Functions [3]: [partial_sum(sales#42), partial_sum(returns#43), partial_sum(profit#44)] -Aggregate Attributes [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Results [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Aggregate Attributes [6]: [sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Results [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] (109) Exchange -Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] -Arguments: hashpartitioning(channel#119, id#120, spark_grouping_id#121, 5), ENSURE_REQUIREMENTS, [id=#134] +Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Arguments: hashpartitioning(channel#125, id#126, spark_grouping_id#127, 5), ENSURE_REQUIREMENTS, [id=#140] (110) HashAggregate [codegen id : 32] -Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] -Keys [3]: [channel#119, id#120, spark_grouping_id#121] +Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Keys [3]: [channel#125, id#126, spark_grouping_id#127] Functions [3]: [sum(sales#42), sum(returns#43), sum(profit#44)] -Aggregate Attributes [3]: [sum(sales#42)#135, sum(returns#43)#136, sum(profit#44)#137] -Results [5]: [channel#119, id#120, sum(sales#42)#135 AS sales#138, sum(returns#43)#136 AS returns#139, sum(profit#44)#137 AS profit#140] +Aggregate Attributes [3]: [sum(sales#42)#141, sum(returns#43)#142, sum(profit#44)#143] +Results [5]: [channel#125, id#126, sum(sales#42)#141 AS sales#144, sum(returns#43)#142 AS returns#145, sum(profit#44)#143 AS profit#146] (111) TakeOrderedAndProject -Input [5]: [channel#119, id#120, sales#138, returns#139, profit#140] -Arguments: 100, [channel#119 ASC NULLS FIRST, id#120 ASC NULLS FIRST], [channel#119, id#120, sales#138, returns#139, profit#140] +Input [5]: [channel#125, id#126, sales#144, returns#145, profit#146] +Arguments: 100, [channel#125 ASC NULLS FIRST, id#126 ASC NULLS FIRST], [channel#125, id#126, sales#144, returns#145, profit#146] ===== Subqueries ===== @@ -625,6 +625,6 @@ Output [1]: [d_date_sk#22] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt index 87abdc02b21a7..aa15d27d4e562 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt @@ -182,7 +182,7 @@ Input [2]: [d_date_sk#16, d_date#17] (16) Filter [codegen id : 5] Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 11192)) AND (d_date#17 <= 11222)) AND isnotnull(d_date_sk#16)) +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-08-23)) AND (d_date#17 <= 2000-09-22)) AND isnotnull(d_date_sk#16)) (17) Project [codegen id : 5] Output [1]: [d_date_sk#16] @@ -367,252 +367,252 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#61] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#61] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#16] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#61] (60) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] (62) Filter [codegen id : 16] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Condition : isnotnull(cp_catalog_page_sk#61) +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Condition : isnotnull(cp_catalog_page_sk#62) (63) BroadcastExchange -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#61] +Right keys [1]: [cp_catalog_page_sk#62] Join condition: None (65) Project [codegen id : 19] -Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#62, cp_catalog_page_id#63] (66) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#22] +Output [1]: [i_item_sk#65] (67) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#22] +Right keys [1]: [i_item_sk#65] Join condition: None (68) Project [codegen id : 19] -Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, i_item_sk#22] +Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, i_item_sk#65] (69) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#25] +Output [1]: [p_promo_sk#66] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#25] +Right keys [1]: [p_promo_sk#66] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, p_promo_sk#25] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, p_promo_sk#66] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Keys [1]: [cp_catalog_page_id#62] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Keys [1]: [cp_catalog_page_id#63] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Results [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] (73) Exchange -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Arguments: hashpartitioning(cp_catalog_page_id#63, 5), ENSURE_REQUIREMENTS, [id=#77] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [cp_catalog_page_id#62] +Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Keys [1]: [cp_catalog_page_id#63] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#80, catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#62) AS id#82] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] +Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#81, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#82, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#83, catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#63) AS id#85] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) (78) Exchange -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] -Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (84) Exchange -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] -Right keys [2]: [wr_item_sk#91, wr_order_number#92] +Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] +Right keys [2]: [wr_item_sk#94, wr_order_number#95] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#100] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#89] -Right keys [1]: [d_date_sk#16] +Left keys [1]: [ws_sold_date_sk#92] +Right keys [1]: [d_date_sk#100] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#16] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#100] (91) Scan parquet default.web_site -Output [2]: [web_site_sk#97, web_site_id#98] +Output [2]: [web_site_sk#101, web_site_id#102] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 26] -Input [2]: [web_site_sk#97, web_site_id#98] +Input [2]: [web_site_sk#101, web_site_id#102] (93) Filter [codegen id : 26] -Input [2]: [web_site_sk#97, web_site_id#98] -Condition : isnotnull(web_site_sk#97) +Input [2]: [web_site_sk#101, web_site_id#102] +Condition : isnotnull(web_site_sk#101) (94) BroadcastExchange -Input [2]: [web_site_sk#97, web_site_id#98] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Input [2]: [web_site_sk#101, web_site_id#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#84] -Right keys [1]: [web_site_sk#97] +Left keys [1]: [ws_web_site_sk#87] +Right keys [1]: [web_site_sk#101] Join condition: None (96) Project [codegen id : 29] -Output [7]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] +Output [7]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#101, web_site_id#102] (97) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#22] +Output [1]: [i_item_sk#104] (98) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#83] -Right keys [1]: [i_item_sk#22] +Left keys [1]: [ws_item_sk#86] +Right keys [1]: [i_item_sk#104] Join condition: None (99) Project [codegen id : 29] -Output [6]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [8]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, i_item_sk#22] +Output [6]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [8]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, i_item_sk#104] (100) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#25] +Output [1]: [p_promo_sk#105] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#85] -Right keys [1]: [p_promo_sk#25] +Left keys [1]: [ws_promo_sk#88] +Right keys [1]: [p_promo_sk#105] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [7]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, p_promo_sk#25] +Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [7]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, p_promo_sk#105] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Keys [1]: [web_site_id#98] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Keys [1]: [web_site_id#102] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Results [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] (104) Exchange -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] +Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Arguments: hashpartitioning(web_site_id#102, 5), ENSURE_REQUIREMENTS, [id=#116] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Keys [1]: [web_site_id#98] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#114, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#115, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#116, web channel AS channel#117, concat(web_site, web_site_id#98) AS id#118] +Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Keys [1]: [web_site_id#102] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] +Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#120, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#121, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#122, web channel AS channel#123, concat(web_site, web_site_id#102) AS id#124] (106) Union (107) Expand [codegen id : 31] Input [5]: [sales#42, returns#43, profit#44, channel#45, id#46] -Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] +Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] (108) HashAggregate [codegen id : 31] -Input [6]: [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] -Keys [3]: [channel#119, id#120, spark_grouping_id#121] +Input [6]: [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] +Keys [3]: [channel#125, id#126, spark_grouping_id#127] Functions [3]: [partial_sum(sales#42), partial_sum(returns#43), partial_sum(profit#44)] -Aggregate Attributes [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Results [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Aggregate Attributes [6]: [sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Results [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] (109) Exchange -Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] -Arguments: hashpartitioning(channel#119, id#120, spark_grouping_id#121, 5), ENSURE_REQUIREMENTS, [id=#134] +Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Arguments: hashpartitioning(channel#125, id#126, spark_grouping_id#127, 5), ENSURE_REQUIREMENTS, [id=#140] (110) HashAggregate [codegen id : 32] -Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] -Keys [3]: [channel#119, id#120, spark_grouping_id#121] +Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Keys [3]: [channel#125, id#126, spark_grouping_id#127] Functions [3]: [sum(sales#42), sum(returns#43), sum(profit#44)] -Aggregate Attributes [3]: [sum(sales#42)#135, sum(returns#43)#136, sum(profit#44)#137] -Results [5]: [channel#119, id#120, sum(sales#42)#135 AS sales#138, sum(returns#43)#136 AS returns#139, sum(profit#44)#137 AS profit#140] +Aggregate Attributes [3]: [sum(sales#42)#141, sum(returns#43)#142, sum(profit#44)#143] +Results [5]: [channel#125, id#126, sum(sales#42)#141 AS sales#144, sum(returns#43)#142 AS returns#145, sum(profit#44)#143 AS profit#146] (111) TakeOrderedAndProject -Input [5]: [channel#119, id#120, sales#138, returns#139, profit#140] -Arguments: 100, [channel#119 ASC NULLS FIRST, id#120 ASC NULLS FIRST], [channel#119, id#120, sales#138, returns#139, profit#140] +Input [5]: [channel#125, id#126, sales#144, returns#145, profit#146] +Arguments: 100, [channel#125 ASC NULLS FIRST, id#126 ASC NULLS FIRST], [channel#125, id#126, sales#144, returns#145, profit#146] ===== Subqueries ===== @@ -625,6 +625,6 @@ Output [1]: [d_date_sk#16] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#8 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 167e142598ae8..d2d70bef8c3ee 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 @@ -170,178 +170,178 @@ Input [3]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19] Condition : isnotnull(ctr_total_return#19) (24) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Output [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(cr_returned_date_sk#23 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 8] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Input [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] (26) Filter [codegen id : 8] -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) +Input [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] +Condition : isnotnull(cr_returning_addr_sk#21) (27) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#24] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None (29) Project [codegen id : 8] -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] +Output [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] +Input [5]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23, d_date_sk#24] (30) Exchange -Input [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Arguments: hashpartitioning(cr_returning_addr_sk#2, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] +Arguments: hashpartitioning(cr_returning_addr_sk#21, 5), ENSURE_REQUIREMENTS, [id=#25] (31) Sort [codegen id : 9] -Input [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Arguments: [cr_returning_addr_sk#2 ASC NULLS FIRST], false, 0 +Input [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] +Arguments: [cr_returning_addr_sk#21 ASC NULLS FIRST], false, 0 (32) ReusedExchange [Reuses operator id: 16] -Output [2]: [ca_address_sk#10, ca_state#11] +Output [2]: [ca_address_sk#26, ca_state#27] (33) Sort [codegen id : 11] -Input [2]: [ca_address_sk#10, ca_state#11] -Arguments: [ca_address_sk#10 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#26, ca_state#27] +Arguments: [ca_address_sk#26 ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 12] -Left keys [1]: [cr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [cr_returning_addr_sk#21] +Right keys [1]: [ca_address_sk#26] Join condition: None (35) Project [codegen id : 12] -Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#11] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#10, ca_state#11] +Output [3]: [cr_returning_customer_sk#20, cr_return_amt_inc_tax#22, ca_state#27] +Input [5]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, ca_address_sk#26, ca_state#27] (36) HashAggregate [codegen id : 12] -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#11] -Keys [2]: [cr_returning_customer_sk#1, ca_state#11] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum#21] -Results [3]: [cr_returning_customer_sk#1, ca_state#11, sum#22] +Input [3]: [cr_returning_customer_sk#20, cr_return_amt_inc_tax#22, ca_state#27] +Keys [2]: [cr_returning_customer_sk#20, ca_state#27] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#22))] +Aggregate Attributes [1]: [sum#28] +Results [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] (37) Exchange -Input [3]: [cr_returning_customer_sk#1, ca_state#11, sum#22] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#11, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] +Arguments: hashpartitioning(cr_returning_customer_sk#20, ca_state#27, 5), ENSURE_REQUIREMENTS, [id=#30] (38) HashAggregate [codegen id : 13] -Input [3]: [cr_returning_customer_sk#1, ca_state#11, sum#22] -Keys [2]: [cr_returning_customer_sk#1, ca_state#11] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#24] -Results [2]: [ca_state#11 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#24,17,2) AS ctr_total_return#19] +Input [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] +Keys [2]: [cr_returning_customer_sk#20, ca_state#27] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#22))#31] +Results [2]: [ca_state#27 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#22))#31,17,2) AS ctr_total_return#19] (39) HashAggregate [codegen id : 13] Input [2]: [ctr_state#18, ctr_total_return#19] Keys [1]: [ctr_state#18] Functions [1]: [partial_avg(ctr_total_return#19)] -Aggregate Attributes [2]: [sum#25, count#26] -Results [3]: [ctr_state#18, sum#27, count#28] +Aggregate Attributes [2]: [sum#32, count#33] +Results [3]: [ctr_state#18, sum#34, count#35] (40) Exchange -Input [3]: [ctr_state#18, sum#27, count#28] -Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ctr_state#18, sum#34, count#35] +Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#36] (41) HashAggregate [codegen id : 14] -Input [3]: [ctr_state#18, sum#27, count#28] +Input [3]: [ctr_state#18, sum#34, count#35] Keys [1]: [ctr_state#18] Functions [1]: [avg(ctr_total_return#19)] -Aggregate Attributes [1]: [avg(ctr_total_return#19)#30] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#18 AS ctr_state#18#32] +Aggregate Attributes [1]: [avg(ctr_total_return#19)#37] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#37) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#38, ctr_state#18 AS ctr_state#18#39] (42) Filter [codegen id : 14] -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) +Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#38) (43) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#33] +Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#40] (44) BroadcastHashJoin [codegen id : 15] Left keys [1]: [ctr_state#18] -Right keys [1]: [ctr_state#18#32] -Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) +Right keys [1]: [ctr_state#18#39] +Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#38) (45) Project [codegen id : 15] Output [2]: [ctr_customer_sk#17, ctr_total_return#19] -Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] +Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] (46) Exchange Input [2]: [ctr_customer_sk#17, ctr_total_return#19] -Arguments: hashpartitioning(ctr_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#34] +Arguments: hashpartitioning(ctr_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#41] (47) Sort [codegen id : 16] Input [2]: [ctr_customer_sk#17, ctr_total_return#19] Arguments: [ctr_customer_sk#17 ASC NULLS FIRST], false, 0 (48) Scan parquet default.customer -Output [6]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40] +Output [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 18] -Input [6]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40] +Input [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] (50) Filter [codegen id : 18] -Input [6]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40] -Condition : (isnotnull(c_customer_sk#35) AND isnotnull(c_current_addr_sk#37)) +Input [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] +Condition : (isnotnull(c_customer_sk#42) AND isnotnull(c_current_addr_sk#44)) (51) Scan parquet default.customer_address -Output [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Output [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] 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 (52) ColumnarToRow [codegen id : 17] -Input [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] (53) Filter [codegen id : 17] -Input [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Condition : ((isnotnull(ca_state#11) AND (ca_state#11 = GA)) AND isnotnull(ca_address_sk#10)) +Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Condition : ((isnotnull(ca_state#55) AND (ca_state#55 = GA)) AND isnotnull(ca_address_sk#48)) (54) BroadcastExchange -Input [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] +Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#60] (55) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#37] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [c_current_addr_sk#44] +Right keys [1]: [ca_address_sk#48] Join condition: None (56) Project [codegen id : 18] -Output [16]: [c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Input [18]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40, ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Output [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Input [18]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47, ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] (57) Exchange -Input [16]: [c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Arguments: hashpartitioning(c_customer_sk#35, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Arguments: hashpartitioning(c_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#61] (58) Sort [codegen id : 19] -Input [16]: [c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Arguments: [c_customer_sk#35 ASC NULLS FIRST], false, 0 +Input [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Arguments: [c_customer_sk#42 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 20] Left keys [1]: [ctr_customer_sk#17] -Right keys [1]: [c_customer_sk#35] +Right keys [1]: [c_customer_sk#42] Join condition: None (60) Project [codegen id : 20] -Output [16]: [c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#19] -Input [18]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Output [16]: [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] +Input [18]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] (61) TakeOrderedAndProject -Input [16]: [c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#19] -Arguments: 100, [c_customer_id#36 ASC NULLS FIRST, c_salutation#38 ASC NULLS FIRST, c_first_name#39 ASC NULLS FIRST, c_last_name#40 ASC NULLS FIRST, ca_street_number#41 ASC NULLS FIRST, ca_street_name#42 ASC NULLS FIRST, ca_street_type#43 ASC NULLS FIRST, ca_suite_number#44 ASC NULLS FIRST, ca_city#45 ASC NULLS FIRST, ca_county#46 ASC NULLS FIRST, ca_state#11 ASC NULLS FIRST, ca_zip#47 ASC NULLS FIRST, ca_country#48 ASC NULLS FIRST, ca_gmt_offset#49 ASC NULLS FIRST, ca_location_type#50 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#19] +Input [16]: [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] +Arguments: 100, [c_customer_id#43 ASC NULLS FIRST, c_salutation#45 ASC NULLS FIRST, c_first_name#46 ASC NULLS FIRST, c_last_name#47 ASC NULLS FIRST, ca_street_number#49 ASC NULLS FIRST, ca_street_name#50 ASC NULLS FIRST, ca_street_type#51 ASC NULLS FIRST, ca_suite_number#52 ASC NULLS FIRST, ca_city#53 ASC NULLS FIRST, ca_county#54 ASC NULLS FIRST, ca_state#55 ASC NULLS FIRST, ca_zip#56 ASC NULLS FIRST, ca_country#57 ASC NULLS FIRST, ca_gmt_offset#58 ASC NULLS FIRST, ca_location_type#59 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] ===== Subqueries ===== @@ -352,6 +352,6 @@ ReusedExchange (62) (62) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#23 IN dynamicpruning#5 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 6a8f9e5264d49..04371a7f43d2a 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 @@ -149,154 +149,154 @@ Input [3]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18] Condition : isnotnull(ctr_total_return#18) (21) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Output [4]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#22), dynamicpruningexpression(cr_returned_date_sk#22 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct (22) ColumnarToRow [codegen id : 6] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Input [4]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22] (23) Filter [codegen id : 6] -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) +Input [4]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22] +Condition : isnotnull(cr_returning_addr_sk#20) (24) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#23] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#22] +Right keys [1]: [d_date_sk#23] Join condition: None (26) Project [codegen id : 6] -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] +Output [3]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21] +Input [5]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22, d_date_sk#23] (27) ReusedExchange [Reuses operator id: 14] -Output [2]: [ca_address_sk#9, ca_state#10] +Output [2]: [ca_address_sk#24, ca_state#25] (28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#9] +Left keys [1]: [cr_returning_addr_sk#20] +Right keys [1]: [ca_address_sk#24] Join condition: None (29) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#9, ca_state#10] +Output [3]: [cr_returning_customer_sk#19, cr_return_amt_inc_tax#21, ca_state#25] +Input [5]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, ca_address_sk#24, ca_state#25] (30) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [cr_returning_customer_sk#1, ca_state#10, sum#20] +Input [3]: [cr_returning_customer_sk#19, cr_return_amt_inc_tax#21, ca_state#25] +Keys [2]: [cr_returning_customer_sk#19, ca_state#25] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#21))] +Aggregate Attributes [1]: [sum#26] +Results [3]: [cr_returning_customer_sk#19, ca_state#25, sum#27] (31) Exchange -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#20] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [3]: [cr_returning_customer_sk#19, ca_state#25, sum#27] +Arguments: hashpartitioning(cr_returning_customer_sk#19, ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#28] (32) HashAggregate [codegen id : 7] -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#20] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#22] -Results [2]: [ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#22,17,2) AS ctr_total_return#18] +Input [3]: [cr_returning_customer_sk#19, ca_state#25, sum#27] +Keys [2]: [cr_returning_customer_sk#19, ca_state#25] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#21))#29] +Results [2]: [ca_state#25 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#21))#29,17,2) AS ctr_total_return#18] (33) HashAggregate [codegen id : 7] Input [2]: [ctr_state#17, ctr_total_return#18] Keys [1]: [ctr_state#17] Functions [1]: [partial_avg(ctr_total_return#18)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ctr_state#17, sum#25, count#26] +Aggregate Attributes [2]: [sum#30, count#31] +Results [3]: [ctr_state#17, sum#32, count#33] (34) Exchange -Input [3]: [ctr_state#17, sum#25, count#26] -Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ctr_state#17, sum#32, count#33] +Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#34] (35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#17, sum#25, count#26] +Input [3]: [ctr_state#17, sum#32, count#33] Keys [1]: [ctr_state#17] Functions [1]: [avg(ctr_total_return#18)] -Aggregate Attributes [1]: [avg(ctr_total_return#18)#28] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#28) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#29, ctr_state#17 AS ctr_state#17#30] +Aggregate Attributes [1]: [avg(ctr_total_return#18)#35] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#35) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#36, ctr_state#17 AS ctr_state#17#37] (36) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) +Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#36) (37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#31] +Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#38] (38) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#17] -Right keys [1]: [ctr_state#17#30] -Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) +Right keys [1]: [ctr_state#17#37] +Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#36) (39) Project [codegen id : 11] Output [2]: [ctr_customer_sk#16, ctr_total_return#18] -Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] +Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] (40) Scan parquet default.customer -Output [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +Output [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +Input [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] (42) Filter [codegen id : 9] -Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34)) +Input [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] +Condition : (isnotnull(c_customer_sk#39) AND isnotnull(c_current_addr_sk#41)) (43) BroadcastExchange -Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] +Input [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] (44) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#16] -Right keys [1]: [c_customer_sk#32] +Right keys [1]: [c_customer_sk#39] Join condition: None (45) Project [codegen id : 11] -Output [6]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] -Input [8]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +Output [6]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] +Input [8]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] (46) Scan parquet default.customer_address -Output [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] +Output [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] 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 (47) ColumnarToRow [codegen id : 10] -Input [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] +Input [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] (48) Filter [codegen id : 10] -Input [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] -Condition : ((isnotnull(ca_state#10) AND (ca_state#10 = GA)) AND isnotnull(ca_address_sk#9)) +Input [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] +Condition : ((isnotnull(ca_state#53) AND (ca_state#53 = GA)) AND isnotnull(ca_address_sk#46)) (49) BroadcastExchange -Input [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Input [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#34] -Right keys [1]: [ca_address_sk#9] +Left keys [1]: [c_current_addr_sk#41] +Right keys [1]: [ca_address_sk#46] Join condition: None (51) Project [codegen id : 11] -Output [16]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48, ctr_total_return#18] -Input [18]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] +Output [16]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57, ctr_total_return#18] +Input [18]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] (52) TakeOrderedAndProject -Input [16]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48, ctr_total_return#18] -Arguments: 100, [c_customer_id#33 ASC NULLS FIRST, c_salutation#35 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, c_last_name#37 ASC NULLS FIRST, ca_street_number#39 ASC NULLS FIRST, ca_street_name#40 ASC NULLS FIRST, ca_street_type#41 ASC NULLS FIRST, ca_suite_number#42 ASC NULLS FIRST, ca_city#43 ASC NULLS FIRST, ca_county#44 ASC NULLS FIRST, ca_state#10 ASC NULLS FIRST, ca_zip#45 ASC NULLS FIRST, ca_country#46 ASC NULLS FIRST, ca_gmt_offset#47 ASC NULLS FIRST, ca_location_type#48 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48, ctr_total_return#18] +Input [16]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57, ctr_total_return#18] +Arguments: 100, [c_customer_id#40 ASC NULLS FIRST, c_salutation#42 ASC NULLS FIRST, c_first_name#43 ASC NULLS FIRST, c_last_name#44 ASC NULLS FIRST, ca_street_number#47 ASC NULLS FIRST, ca_street_name#48 ASC NULLS FIRST, ca_street_type#49 ASC NULLS FIRST, ca_suite_number#50 ASC NULLS FIRST, ca_city#51 ASC NULLS FIRST, ca_county#52 ASC NULLS FIRST, ca_state#53 ASC NULLS FIRST, ca_zip#54 ASC NULLS FIRST, ca_country#55 ASC NULLS FIRST, ca_gmt_offset#56 ASC NULLS FIRST, ca_location_type#57 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57, ctr_total_return#18] ===== Subqueries ===== @@ -307,6 +307,6 @@ ReusedExchange (53) (53) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 21 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 21 Hosting Expression = cr_returned_date_sk#22 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt index 8b09336f86535..6725e273a3acc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt @@ -95,7 +95,7 @@ Input [2]: [d_date_sk#11, d_date#12] (14) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 11102)) AND (d_date#12 <= 11162)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) (15) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt index 340fcf5fa31fd..a03333f7623cc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt @@ -92,7 +92,7 @@ Input [2]: [d_date_sk#11, d_date#12] (14) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 11102)) AND (d_date#12 <= 11162)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) (15) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt index 0073fd978d748..7f7c9ac72707e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt @@ -245,7 +245,7 @@ Arguments: [wr_refunded_cdemo_sk#13 ASC NULLS FIRST, wr_returning_cdemo_sk#15 AS Output [3]: [cd_demo_sk#33, cd_marital_status#34, cd_education_status#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct (43) ColumnarToRow [codegen id : 11] @@ -253,7 +253,7 @@ Input [3]: [cd_demo_sk#33, cd_marital_status#34, cd_education_status#35] (44) Filter [codegen id : 11] Input [3]: [cd_demo_sk#33, cd_marital_status#34, cd_education_status#35] -Condition : (((isnotnull(cd_demo_sk#33) AND isnotnull(cd_marital_status#34)) AND isnotnull(cd_education_status#35)) AND ((((cd_marital_status#34 = M) AND (cd_education_status#35 = Advanced Degree)) OR ((cd_marital_status#34 = S) AND (cd_education_status#35 = College))) OR ((cd_marital_status#34 = W) AND (cd_education_status#35 = 2 yr Degree)))) +Condition : (((isnotnull(cd_demo_sk#33) AND isnotnull(cd_marital_status#34)) AND isnotnull(cd_education_status#35)) AND ((((cd_marital_status#34 = M) AND (cd_education_status#35 = Advanced Degree )) OR ((cd_marital_status#34 = S) AND (cd_education_status#35 = College ))) OR ((cd_marital_status#34 = W) AND (cd_education_status#35 = 2 yr Degree )))) (45) BroadcastExchange Input [3]: [cd_demo_sk#33, cd_marital_status#34, cd_education_status#35] @@ -293,7 +293,7 @@ Arguments: [cast(cd_demo_sk#33 as bigint) ASC NULLS FIRST, cast(cd_demo_sk#37 as (53) SortMergeJoin [codegen id : 14] Left keys [2]: [wr_refunded_cdemo_sk#13, wr_returning_cdemo_sk#15] Right keys [2]: [cast(cd_demo_sk#33 as bigint), cast(cd_demo_sk#37 as bigint)] -Join condition: ((((((cd_marital_status#34 = M) AND (cd_education_status#35 = Advanced Degree)) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#34 = S) AND (cd_education_status#35 = College)) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#34 = W) AND (cd_education_status#35 = 2 yr Degree)) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) +Join condition: ((((((cd_marital_status#34 = M) AND (cd_education_status#35 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#34 = S) AND (cd_education_status#35 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#34 = W) AND (cd_education_status#35 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) (54) Project [codegen id : 14] Output [4]: [ws_quantity#4, wr_fee#18, wr_refunded_cash#19, r_reason_desc#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt index c528cf8f91a55..74a9aa3dec985 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt @@ -130,7 +130,7 @@ Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, Output [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] @@ -138,7 +138,7 @@ Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] (19) Filter [codegen id : 3] Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] -Condition : (((isnotnull(cd_demo_sk#21) AND isnotnull(cd_marital_status#22)) AND isnotnull(cd_education_status#23)) AND ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) OR ((cd_marital_status#22 = S) AND (cd_education_status#23 = College))) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)))) +Condition : (((isnotnull(cd_demo_sk#21) AND isnotnull(cd_marital_status#22)) AND isnotnull(cd_education_status#23)) AND ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) OR ((cd_marital_status#22 = S) AND (cd_education_status#23 = College ))) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )))) (20) BroadcastExchange Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] @@ -147,7 +147,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (21) BroadcastHashJoin [codegen id : 8] Left keys [1]: [wr_refunded_cdemo_sk#11] Right keys [1]: [cast(cd_demo_sk#21 as bigint)] -Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College)) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) +Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) (22) Project [codegen id : 8] Output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_fee#16, wr_refunded_cash#17, cd_marital_status#22, cd_education_status#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt index 888fd8e520796..92895cb566fd2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt @@ -177,11 +177,11 @@ Results [3]: [c_last_name#11, c_first_name#10, d_date#5] (23) Exchange Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#14] (24) Sort [codegen id : 8] Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 0) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 (25) Scan parquet default.catalog_sales Output [2]: [cs_bill_customer_sk#15, cs_sold_date_sk#16] @@ -254,15 +254,15 @@ Results [3]: [c_last_name#22, c_first_name#21, d_date#18] (40) Exchange Input [3]: [c_last_name#22, c_first_name#21, d_date#18] -Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 0), isnull(d_date#18), 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18), 5), ENSURE_REQUIREMENTS, [id=#24] (41) Sort [codegen id : 16] Input [3]: [c_last_name#22, c_first_name#21, d_date#18] -Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 0) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 1970-01-01) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 (42) SortMergeJoin -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 0), isnull(d_date#18)] +Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18)] Join condition: None (43) HashAggregate [codegen id : 17] @@ -285,11 +285,11 @@ Results [3]: [c_last_name#11, c_first_name#10, d_date#5] (46) Exchange Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#26] +Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#26] (47) Sort [codegen id : 19] Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 0) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 (48) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#27, ws_sold_date_sk#28] @@ -362,15 +362,15 @@ Results [3]: [c_last_name#34, c_first_name#33, d_date#30] (63) Exchange Input [3]: [c_last_name#34, c_first_name#33, d_date#30] -Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 0), isnull(d_date#30), 5), ENSURE_REQUIREMENTS, [id=#36] +Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30), 5), ENSURE_REQUIREMENTS, [id=#36] (64) Sort [codegen id : 27] Input [3]: [c_last_name#34, c_first_name#33, d_date#30] -Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 0) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 1970-01-01) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 (65) SortMergeJoin -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 0), isnull(d_date#30)] +Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30)] Join condition: None (66) HashAggregate [codegen id : 28] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt index bc62b67957935..27e16b75638a8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt @@ -205,11 +205,11 @@ Results [3]: [c_last_name#19, c_first_name#18, d_date#16] (32) BroadcastExchange Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 0), isnull(input[2, date, true])),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [id=#21] (33) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 0), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 0), isnull(d_date#16)] +Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)] Join condition: None (34) HashAggregate [codegen id : 12] @@ -285,11 +285,11 @@ Results [3]: [c_last_name#28, c_first_name#27, d_date#25] (48) BroadcastExchange Input [3]: [c_last_name#28, c_first_name#27, d_date#25] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 0), isnull(input[2, date, true])),false), [id=#30] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [id=#30] (49) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 0), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#28, ), isnull(c_last_name#28), coalesce(c_first_name#27, ), isnull(c_first_name#27), coalesce(d_date#25, 0), isnull(d_date#25)] +Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#28, ), isnull(c_last_name#28), coalesce(c_first_name#27, ), isnull(c_first_name#27), coalesce(d_date#25, 1970-01-01), isnull(d_date#25)] Join condition: None (50) HashAggregate [codegen id : 12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt index f9d73c29db3d8..e72928545d080 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt @@ -313,687 +313,687 @@ Aggregate Attributes [1]: [count(1)#19] Results [1]: [count(1)#19 AS h8_30_to_9#20] (29) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] (31) Filter [codegen id : 8] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Condition : ((isnotnull(ss_hdemo_sk#22) AND isnotnull(ss_sold_time_sk#21)) AND isnotnull(ss_store_sk#23)) (32) Project [codegen id : 8] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] (33) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#25, t_hour#26, t_minute#27] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#25, t_hour#26, t_minute#27] (35) Filter [codegen id : 5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 9)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#25, t_hour#26, t_minute#27] +Condition : ((((isnotnull(t_hour#26) AND isnotnull(t_minute#27)) AND (t_hour#26 = 9)) AND (t_minute#27 < 30)) AND isnotnull(t_time_sk#25)) (36) Project [codegen id : 5] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#25] +Input [3]: [t_time_sk#25, t_hour#26, t_minute#27] (37) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [t_time_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (38) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#21] +Right keys [1]: [t_time_sk#25] Join condition: None (39) Project [codegen id : 8] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#22, ss_store_sk#23] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, t_time_sk#25] (40) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#29] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#23] +Right keys [1]: [s_store_sk#29] Join condition: None (42) Project [codegen id : 8] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#22] +Input [3]: [ss_hdemo_sk#22, ss_store_sk#23, s_store_sk#29] (43) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#30] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#22] +Right keys [1]: [hd_demo_sk#30] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#22, hd_demo_sk#30] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#22] -Results [1]: [count#23] +Aggregate Attributes [1]: [count#31] +Results [1]: [count#32] (47) Exchange -Input [1]: [count#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [1]: [count#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] (48) HashAggregate [codegen id : 9] -Input [1]: [count#23] +Input [1]: [count#32] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#25] -Results [1]: [count(1)#25 AS h9_to_9_30#26] +Aggregate Attributes [1]: [count(1)#34] +Results [1]: [count(1)#34 AS h9_to_9_30#35] (49) BroadcastExchange -Input [1]: [h9_to_9_30#26] -Arguments: IdentityBroadcastMode, [id=#27] +Input [1]: [h9_to_9_30#35] +Arguments: IdentityBroadcastMode, [id=#36] (50) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (51) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 13] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] (53) Filter [codegen id : 13] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) (54) Project [codegen id : 13] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] (55) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#41, t_hour#42, t_minute#43] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 10] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#41, t_hour#42, t_minute#43] (57) Filter [codegen id : 10] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 9)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#41, t_hour#42, t_minute#43] +Condition : ((((isnotnull(t_hour#42) AND isnotnull(t_minute#43)) AND (t_hour#42 = 9)) AND (t_minute#43 >= 30)) AND isnotnull(t_time_sk#41)) (58) Project [codegen id : 10] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#41] +Input [3]: [t_time_sk#41, t_hour#42, t_minute#43] (59) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [t_time_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (60) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#37] +Right keys [1]: [t_time_sk#41] Join condition: None (61) Project [codegen id : 13] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#38, ss_store_sk#39] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, t_time_sk#41] (62) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#45] (63) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#39] +Right keys [1]: [s_store_sk#45] Join condition: None (64) Project [codegen id : 13] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#38] +Input [3]: [ss_hdemo_sk#38, ss_store_sk#39, s_store_sk#45] (65) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#46] (66) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#38] +Right keys [1]: [hd_demo_sk#46] Join condition: None (67) Project [codegen id : 13] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#38, hd_demo_sk#46] (68) HashAggregate [codegen id : 13] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [1]: [count#30] +Aggregate Attributes [1]: [count#47] +Results [1]: [count#48] (69) Exchange -Input [1]: [count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] +Input [1]: [count#48] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] (70) HashAggregate [codegen id : 14] -Input [1]: [count#30] +Input [1]: [count#48] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [1]: [count(1)#32 AS h9_30_to_10#33] +Aggregate Attributes [1]: [count(1)#50] +Results [1]: [count(1)#50 AS h9_30_to_10#51] (71) BroadcastExchange -Input [1]: [h9_30_to_10#33] -Arguments: IdentityBroadcastMode, [id=#34] +Input [1]: [h9_30_to_10#51] +Arguments: IdentityBroadcastMode, [id=#52] (72) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (73) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 18] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] (75) Filter [codegen id : 18] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Condition : ((isnotnull(ss_hdemo_sk#54) AND isnotnull(ss_sold_time_sk#53)) AND isnotnull(ss_store_sk#55)) (76) Project [codegen id : 18] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] (77) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#57, t_hour#58, t_minute#59] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (78) ColumnarToRow [codegen id : 15] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] (79) Filter [codegen id : 15] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 10)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Condition : ((((isnotnull(t_hour#58) AND isnotnull(t_minute#59)) AND (t_hour#58 = 10)) AND (t_minute#59 < 30)) AND isnotnull(t_time_sk#57)) (80) Project [codegen id : 15] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#57] +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] (81) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [1]: [t_time_sk#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] (82) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#53] +Right keys [1]: [t_time_sk#57] Join condition: None (83) Project [codegen id : 18] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#54, ss_store_sk#55] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, t_time_sk#57] (84) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#61] (85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#55] +Right keys [1]: [s_store_sk#61] Join condition: None (86) Project [codegen id : 18] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#54] +Input [3]: [ss_hdemo_sk#54, ss_store_sk#55, s_store_sk#61] (87) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#62] (88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#54] +Right keys [1]: [hd_demo_sk#62] Join condition: None (89) Project [codegen id : 18] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#54, hd_demo_sk#62] (90) HashAggregate [codegen id : 18] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#36] -Results [1]: [count#37] +Aggregate Attributes [1]: [count#63] +Results [1]: [count#64] (91) Exchange -Input [1]: [count#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#38] +Input [1]: [count#64] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] (92) HashAggregate [codegen id : 19] -Input [1]: [count#37] +Input [1]: [count#64] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#39] -Results [1]: [count(1)#39 AS h10_to_10_30#40] +Aggregate Attributes [1]: [count(1)#66] +Results [1]: [count(1)#66 AS h10_to_10_30#67] (93) BroadcastExchange -Input [1]: [h10_to_10_30#40] -Arguments: IdentityBroadcastMode, [id=#41] +Input [1]: [h10_to_10_30#67] +Arguments: IdentityBroadcastMode, [id=#68] (94) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (95) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (96) ColumnarToRow [codegen id : 23] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] (97) Filter [codegen id : 23] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Condition : ((isnotnull(ss_hdemo_sk#70) AND isnotnull(ss_sold_time_sk#69)) AND isnotnull(ss_store_sk#71)) (98) Project [codegen id : 23] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] (99) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#73, t_hour#74, t_minute#75] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (100) ColumnarToRow [codegen id : 20] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#73, t_hour#74, t_minute#75] (101) Filter [codegen id : 20] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 10)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#73, t_hour#74, t_minute#75] +Condition : ((((isnotnull(t_hour#74) AND isnotnull(t_minute#75)) AND (t_hour#74 = 10)) AND (t_minute#75 >= 30)) AND isnotnull(t_time_sk#73)) (102) Project [codegen id : 20] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#73] +Input [3]: [t_time_sk#73, t_hour#74, t_minute#75] (103) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [t_time_sk#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] (104) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#69] +Right keys [1]: [t_time_sk#73] Join condition: None (105) Project [codegen id : 23] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#70, ss_store_sk#71] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, t_time_sk#73] (106) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#77] (107) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#71] +Right keys [1]: [s_store_sk#77] Join condition: None (108) Project [codegen id : 23] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#70] +Input [3]: [ss_hdemo_sk#70, ss_store_sk#71, s_store_sk#77] (109) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#78] (110) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#70] +Right keys [1]: [hd_demo_sk#78] Join condition: None (111) Project [codegen id : 23] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#70, hd_demo_sk#78] (112) HashAggregate [codegen id : 23] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#43] -Results [1]: [count#44] +Aggregate Attributes [1]: [count#79] +Results [1]: [count#80] (113) Exchange -Input [1]: [count#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45] +Input [1]: [count#80] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] (114) HashAggregate [codegen id : 24] -Input [1]: [count#44] +Input [1]: [count#80] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#46] -Results [1]: [count(1)#46 AS h10_30_to_11#47] +Aggregate Attributes [1]: [count(1)#82] +Results [1]: [count(1)#82 AS h10_30_to_11#83] (115) BroadcastExchange -Input [1]: [h10_30_to_11#47] -Arguments: IdentityBroadcastMode, [id=#48] +Input [1]: [h10_30_to_11#83] +Arguments: IdentityBroadcastMode, [id=#84] (116) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (117) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (118) ColumnarToRow [codegen id : 28] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] (119) Filter [codegen id : 28] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Condition : ((isnotnull(ss_hdemo_sk#86) AND isnotnull(ss_sold_time_sk#85)) AND isnotnull(ss_store_sk#87)) (120) Project [codegen id : 28] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] (121) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#89, t_hour#90, t_minute#91] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (122) ColumnarToRow [codegen id : 25] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#89, t_hour#90, t_minute#91] (123) Filter [codegen id : 25] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 11)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#89, t_hour#90, t_minute#91] +Condition : ((((isnotnull(t_hour#90) AND isnotnull(t_minute#91)) AND (t_hour#90 = 11)) AND (t_minute#91 < 30)) AND isnotnull(t_time_sk#89)) (124) Project [codegen id : 25] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#89] +Input [3]: [t_time_sk#89, t_hour#90, t_minute#91] (125) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] +Input [1]: [t_time_sk#89] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#92] (126) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#85] +Right keys [1]: [t_time_sk#89] Join condition: None (127) Project [codegen id : 28] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#86, ss_store_sk#87] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, t_time_sk#89] (128) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#93] (129) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#87] +Right keys [1]: [s_store_sk#93] Join condition: None (130) Project [codegen id : 28] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#86] +Input [3]: [ss_hdemo_sk#86, ss_store_sk#87, s_store_sk#93] (131) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#94] (132) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#86] +Right keys [1]: [hd_demo_sk#94] Join condition: None (133) Project [codegen id : 28] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#86, hd_demo_sk#94] (134) HashAggregate [codegen id : 28] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#50] -Results [1]: [count#51] +Aggregate Attributes [1]: [count#95] +Results [1]: [count#96] (135) Exchange -Input [1]: [count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] +Input [1]: [count#96] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#97] (136) HashAggregate [codegen id : 29] -Input [1]: [count#51] +Input [1]: [count#96] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#53] -Results [1]: [count(1)#53 AS h11_to_11_30#54] +Aggregate Attributes [1]: [count(1)#98] +Results [1]: [count(1)#98 AS h11_to_11_30#99] (137) BroadcastExchange -Input [1]: [h11_to_11_30#54] -Arguments: IdentityBroadcastMode, [id=#55] +Input [1]: [h11_to_11_30#99] +Arguments: IdentityBroadcastMode, [id=#100] (138) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (139) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (140) ColumnarToRow [codegen id : 33] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] (141) Filter [codegen id : 33] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Condition : ((isnotnull(ss_hdemo_sk#102) AND isnotnull(ss_sold_time_sk#101)) AND isnotnull(ss_store_sk#103)) (142) Project [codegen id : 33] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] (143) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#105, t_hour#106, t_minute#107] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (144) ColumnarToRow [codegen id : 30] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#105, t_hour#106, t_minute#107] (145) Filter [codegen id : 30] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 11)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#105, t_hour#106, t_minute#107] +Condition : ((((isnotnull(t_hour#106) AND isnotnull(t_minute#107)) AND (t_hour#106 = 11)) AND (t_minute#107 >= 30)) AND isnotnull(t_time_sk#105)) (146) Project [codegen id : 30] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#105] +Input [3]: [t_time_sk#105, t_hour#106, t_minute#107] (147) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +Input [1]: [t_time_sk#105] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#108] (148) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#101] +Right keys [1]: [t_time_sk#105] Join condition: None (149) Project [codegen id : 33] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#102, ss_store_sk#103] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, t_time_sk#105] (150) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#109] (151) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#103] +Right keys [1]: [s_store_sk#109] Join condition: None (152) Project [codegen id : 33] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#102] +Input [3]: [ss_hdemo_sk#102, ss_store_sk#103, s_store_sk#109] (153) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#110] (154) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#102] +Right keys [1]: [hd_demo_sk#110] Join condition: None (155) Project [codegen id : 33] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#102, hd_demo_sk#110] (156) HashAggregate [codegen id : 33] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#57] -Results [1]: [count#58] +Aggregate Attributes [1]: [count#111] +Results [1]: [count#112] (157) Exchange -Input [1]: [count#58] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#59] +Input [1]: [count#112] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] (158) HashAggregate [codegen id : 34] -Input [1]: [count#58] +Input [1]: [count#112] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#60] -Results [1]: [count(1)#60 AS h11_30_to_12#61] +Aggregate Attributes [1]: [count(1)#114] +Results [1]: [count(1)#114 AS h11_30_to_12#115] (159) BroadcastExchange -Input [1]: [h11_30_to_12#61] -Arguments: IdentityBroadcastMode, [id=#62] +Input [1]: [h11_30_to_12#115] +Arguments: IdentityBroadcastMode, [id=#116] (160) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (161) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (162) ColumnarToRow [codegen id : 38] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] (163) Filter [codegen id : 38] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Condition : ((isnotnull(ss_hdemo_sk#118) AND isnotnull(ss_sold_time_sk#117)) AND isnotnull(ss_store_sk#119)) (164) Project [codegen id : 38] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] (165) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#121, t_hour#122, t_minute#123] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (166) ColumnarToRow [codegen id : 35] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#121, t_hour#122, t_minute#123] (167) Filter [codegen id : 35] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 12)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#121, t_hour#122, t_minute#123] +Condition : ((((isnotnull(t_hour#122) AND isnotnull(t_minute#123)) AND (t_hour#122 = 12)) AND (t_minute#123 < 30)) AND isnotnull(t_time_sk#121)) (168) Project [codegen id : 35] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#121] +Input [3]: [t_time_sk#121, t_hour#122, t_minute#123] (169) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] +Input [1]: [t_time_sk#121] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#124] (170) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#117] +Right keys [1]: [t_time_sk#121] Join condition: None (171) Project [codegen id : 38] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#118, ss_store_sk#119] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, t_time_sk#121] (172) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#125] (173) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#119] +Right keys [1]: [s_store_sk#125] Join condition: None (174) Project [codegen id : 38] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#118] +Input [3]: [ss_hdemo_sk#118, ss_store_sk#119, s_store_sk#125] (175) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#126] (176) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#118] +Right keys [1]: [hd_demo_sk#126] Join condition: None (177) Project [codegen id : 38] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#118, hd_demo_sk#126] (178) HashAggregate [codegen id : 38] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#64] -Results [1]: [count#65] +Aggregate Attributes [1]: [count#127] +Results [1]: [count#128] (179) Exchange -Input [1]: [count#65] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] +Input [1]: [count#128] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] (180) HashAggregate [codegen id : 39] -Input [1]: [count#65] +Input [1]: [count#128] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#67] -Results [1]: [count(1)#67 AS h12_to_12_30#68] +Aggregate Attributes [1]: [count(1)#130] +Results [1]: [count(1)#130 AS h12_to_12_30#131] (181) BroadcastExchange -Input [1]: [h12_to_12_30#68] -Arguments: IdentityBroadcastMode, [id=#69] +Input [1]: [h12_to_12_30#131] +Arguments: IdentityBroadcastMode, [id=#132] (182) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt index 453906c2b5f34..9f56c71154a66 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt @@ -313,687 +313,687 @@ Aggregate Attributes [1]: [count(1)#19] Results [1]: [count(1)#19 AS h8_30_to_9#20] (29) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] (31) Filter [codegen id : 8] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Condition : ((isnotnull(ss_hdemo_sk#22) AND isnotnull(ss_sold_time_sk#21)) AND isnotnull(ss_store_sk#23)) (32) Project [codegen id : 8] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] (33) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#25] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#22] +Right keys [1]: [hd_demo_sk#25] Join condition: None (35) Project [codegen id : 8] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#21, ss_store_sk#23] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, hd_demo_sk#25] (36) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#26, t_hour#27, t_minute#28] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#26, t_hour#27, t_minute#28] (38) Filter [codegen id : 6] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 9)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#26, t_hour#27, t_minute#28] +Condition : ((((isnotnull(t_hour#27) AND isnotnull(t_minute#28)) AND (t_hour#27 = 9)) AND (t_minute#28 < 30)) AND isnotnull(t_time_sk#26)) (39) Project [codegen id : 6] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#26] +Input [3]: [t_time_sk#26, t_hour#27, t_minute#28] (40) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [t_time_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#21] +Right keys [1]: [t_time_sk#26] Join condition: None (42) Project [codegen id : 8] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#23] +Input [3]: [ss_sold_time_sk#21, ss_store_sk#23, t_time_sk#26] (43) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#30] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#23] +Right keys [1]: [s_store_sk#30] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#23, s_store_sk#30] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#22] -Results [1]: [count#23] +Aggregate Attributes [1]: [count#31] +Results [1]: [count#32] (47) Exchange -Input [1]: [count#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [1]: [count#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] (48) HashAggregate [codegen id : 9] -Input [1]: [count#23] +Input [1]: [count#32] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#25] -Results [1]: [count(1)#25 AS h9_to_9_30#26] +Aggregate Attributes [1]: [count(1)#34] +Results [1]: [count(1)#34 AS h9_to_9_30#35] (49) BroadcastExchange -Input [1]: [h9_to_9_30#26] -Arguments: IdentityBroadcastMode, [id=#27] +Input [1]: [h9_to_9_30#35] +Arguments: IdentityBroadcastMode, [id=#36] (50) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (51) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 13] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] (53) Filter [codegen id : 13] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) (54) Project [codegen id : 13] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] (55) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#41] (56) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#38] +Right keys [1]: [hd_demo_sk#41] Join condition: None (57) Project [codegen id : 13] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#37, ss_store_sk#39] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] (58) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 11] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] (60) Filter [codegen id : 11] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 9)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] +Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 9)) AND (t_minute#44 >= 30)) AND isnotnull(t_time_sk#42)) (61) Project [codegen id : 11] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#42] +Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] (62) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [t_time_sk#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#45] (63) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#37] +Right keys [1]: [t_time_sk#42] Join condition: None (64) Project [codegen id : 13] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#39] +Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] (65) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#46] (66) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#39] +Right keys [1]: [s_store_sk#46] Join condition: None (67) Project [codegen id : 13] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#39, s_store_sk#46] (68) HashAggregate [codegen id : 13] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [1]: [count#30] +Aggregate Attributes [1]: [count#47] +Results [1]: [count#48] (69) Exchange -Input [1]: [count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] +Input [1]: [count#48] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] (70) HashAggregate [codegen id : 14] -Input [1]: [count#30] +Input [1]: [count#48] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [1]: [count(1)#32 AS h9_30_to_10#33] +Aggregate Attributes [1]: [count(1)#50] +Results [1]: [count(1)#50 AS h9_30_to_10#51] (71) BroadcastExchange -Input [1]: [h9_30_to_10#33] -Arguments: IdentityBroadcastMode, [id=#34] +Input [1]: [h9_30_to_10#51] +Arguments: IdentityBroadcastMode, [id=#52] (72) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (73) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 18] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] (75) Filter [codegen id : 18] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Condition : ((isnotnull(ss_hdemo_sk#54) AND isnotnull(ss_sold_time_sk#53)) AND isnotnull(ss_store_sk#55)) (76) Project [codegen id : 18] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] (77) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#57] (78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#54] +Right keys [1]: [hd_demo_sk#57] Join condition: None (79) Project [codegen id : 18] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#53, ss_store_sk#55] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, hd_demo_sk#57] (80) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#58, t_hour#59, t_minute#60] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (81) ColumnarToRow [codegen id : 16] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#58, t_hour#59, t_minute#60] (82) Filter [codegen id : 16] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 10)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#58, t_hour#59, t_minute#60] +Condition : ((((isnotnull(t_hour#59) AND isnotnull(t_minute#60)) AND (t_hour#59 = 10)) AND (t_minute#60 < 30)) AND isnotnull(t_time_sk#58)) (83) Project [codegen id : 16] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#58] +Input [3]: [t_time_sk#58, t_hour#59, t_minute#60] (84) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [1]: [t_time_sk#58] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] (85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#53] +Right keys [1]: [t_time_sk#58] Join condition: None (86) Project [codegen id : 18] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#55] +Input [3]: [ss_sold_time_sk#53, ss_store_sk#55, t_time_sk#58] (87) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#62] (88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#55] +Right keys [1]: [s_store_sk#62] Join condition: None (89) Project [codegen id : 18] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#55, s_store_sk#62] (90) HashAggregate [codegen id : 18] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#36] -Results [1]: [count#37] +Aggregate Attributes [1]: [count#63] +Results [1]: [count#64] (91) Exchange -Input [1]: [count#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#38] +Input [1]: [count#64] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] (92) HashAggregate [codegen id : 19] -Input [1]: [count#37] +Input [1]: [count#64] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#39] -Results [1]: [count(1)#39 AS h10_to_10_30#40] +Aggregate Attributes [1]: [count(1)#66] +Results [1]: [count(1)#66 AS h10_to_10_30#67] (93) BroadcastExchange -Input [1]: [h10_to_10_30#40] -Arguments: IdentityBroadcastMode, [id=#41] +Input [1]: [h10_to_10_30#67] +Arguments: IdentityBroadcastMode, [id=#68] (94) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (95) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (96) ColumnarToRow [codegen id : 23] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] (97) Filter [codegen id : 23] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Condition : ((isnotnull(ss_hdemo_sk#70) AND isnotnull(ss_sold_time_sk#69)) AND isnotnull(ss_store_sk#71)) (98) Project [codegen id : 23] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] (99) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#73] (100) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#70] +Right keys [1]: [hd_demo_sk#73] Join condition: None (101) Project [codegen id : 23] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#69, ss_store_sk#71] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, hd_demo_sk#73] (102) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#74, t_hour#75, t_minute#76] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (103) ColumnarToRow [codegen id : 21] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] (104) Filter [codegen id : 21] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 10)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] +Condition : ((((isnotnull(t_hour#75) AND isnotnull(t_minute#76)) AND (t_hour#75 = 10)) AND (t_minute#76 >= 30)) AND isnotnull(t_time_sk#74)) (105) Project [codegen id : 21] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#74] +Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] (106) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [t_time_sk#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] (107) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#69] +Right keys [1]: [t_time_sk#74] Join condition: None (108) Project [codegen id : 23] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#71] +Input [3]: [ss_sold_time_sk#69, ss_store_sk#71, t_time_sk#74] (109) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#78] (110) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#71] +Right keys [1]: [s_store_sk#78] Join condition: None (111) Project [codegen id : 23] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#71, s_store_sk#78] (112) HashAggregate [codegen id : 23] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#43] -Results [1]: [count#44] +Aggregate Attributes [1]: [count#79] +Results [1]: [count#80] (113) Exchange -Input [1]: [count#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45] +Input [1]: [count#80] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] (114) HashAggregate [codegen id : 24] -Input [1]: [count#44] +Input [1]: [count#80] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#46] -Results [1]: [count(1)#46 AS h10_30_to_11#47] +Aggregate Attributes [1]: [count(1)#82] +Results [1]: [count(1)#82 AS h10_30_to_11#83] (115) BroadcastExchange -Input [1]: [h10_30_to_11#47] -Arguments: IdentityBroadcastMode, [id=#48] +Input [1]: [h10_30_to_11#83] +Arguments: IdentityBroadcastMode, [id=#84] (116) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (117) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (118) ColumnarToRow [codegen id : 28] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] (119) Filter [codegen id : 28] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Condition : ((isnotnull(ss_hdemo_sk#86) AND isnotnull(ss_sold_time_sk#85)) AND isnotnull(ss_store_sk#87)) (120) Project [codegen id : 28] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] (121) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#89] (122) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#86] +Right keys [1]: [hd_demo_sk#89] Join condition: None (123) Project [codegen id : 28] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#85, ss_store_sk#87] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, hd_demo_sk#89] (124) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#90, t_hour#91, t_minute#92] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (125) ColumnarToRow [codegen id : 26] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#90, t_hour#91, t_minute#92] (126) Filter [codegen id : 26] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 11)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#90, t_hour#91, t_minute#92] +Condition : ((((isnotnull(t_hour#91) AND isnotnull(t_minute#92)) AND (t_hour#91 = 11)) AND (t_minute#92 < 30)) AND isnotnull(t_time_sk#90)) (127) Project [codegen id : 26] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#90] +Input [3]: [t_time_sk#90, t_hour#91, t_minute#92] (128) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] +Input [1]: [t_time_sk#90] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] (129) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#85] +Right keys [1]: [t_time_sk#90] Join condition: None (130) Project [codegen id : 28] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#87] +Input [3]: [ss_sold_time_sk#85, ss_store_sk#87, t_time_sk#90] (131) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#94] (132) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#87] +Right keys [1]: [s_store_sk#94] Join condition: None (133) Project [codegen id : 28] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#87, s_store_sk#94] (134) HashAggregate [codegen id : 28] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#50] -Results [1]: [count#51] +Aggregate Attributes [1]: [count#95] +Results [1]: [count#96] (135) Exchange -Input [1]: [count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] +Input [1]: [count#96] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#97] (136) HashAggregate [codegen id : 29] -Input [1]: [count#51] +Input [1]: [count#96] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#53] -Results [1]: [count(1)#53 AS h11_to_11_30#54] +Aggregate Attributes [1]: [count(1)#98] +Results [1]: [count(1)#98 AS h11_to_11_30#99] (137) BroadcastExchange -Input [1]: [h11_to_11_30#54] -Arguments: IdentityBroadcastMode, [id=#55] +Input [1]: [h11_to_11_30#99] +Arguments: IdentityBroadcastMode, [id=#100] (138) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (139) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (140) ColumnarToRow [codegen id : 33] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] (141) Filter [codegen id : 33] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Condition : ((isnotnull(ss_hdemo_sk#102) AND isnotnull(ss_sold_time_sk#101)) AND isnotnull(ss_store_sk#103)) (142) Project [codegen id : 33] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] (143) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#105] (144) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#102] +Right keys [1]: [hd_demo_sk#105] Join condition: None (145) Project [codegen id : 33] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#101, ss_store_sk#103] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, hd_demo_sk#105] (146) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#106, t_hour#107, t_minute#108] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (147) ColumnarToRow [codegen id : 31] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#106, t_hour#107, t_minute#108] (148) Filter [codegen id : 31] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 11)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#106, t_hour#107, t_minute#108] +Condition : ((((isnotnull(t_hour#107) AND isnotnull(t_minute#108)) AND (t_hour#107 = 11)) AND (t_minute#108 >= 30)) AND isnotnull(t_time_sk#106)) (149) Project [codegen id : 31] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#106] +Input [3]: [t_time_sk#106, t_hour#107, t_minute#108] (150) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +Input [1]: [t_time_sk#106] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#109] (151) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#101] +Right keys [1]: [t_time_sk#106] Join condition: None (152) Project [codegen id : 33] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#103] +Input [3]: [ss_sold_time_sk#101, ss_store_sk#103, t_time_sk#106] (153) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#110] (154) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#103] +Right keys [1]: [s_store_sk#110] Join condition: None (155) Project [codegen id : 33] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#103, s_store_sk#110] (156) HashAggregate [codegen id : 33] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#57] -Results [1]: [count#58] +Aggregate Attributes [1]: [count#111] +Results [1]: [count#112] (157) Exchange -Input [1]: [count#58] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#59] +Input [1]: [count#112] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] (158) HashAggregate [codegen id : 34] -Input [1]: [count#58] +Input [1]: [count#112] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#60] -Results [1]: [count(1)#60 AS h11_30_to_12#61] +Aggregate Attributes [1]: [count(1)#114] +Results [1]: [count(1)#114 AS h11_30_to_12#115] (159) BroadcastExchange -Input [1]: [h11_30_to_12#61] -Arguments: IdentityBroadcastMode, [id=#62] +Input [1]: [h11_30_to_12#115] +Arguments: IdentityBroadcastMode, [id=#116] (160) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (161) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (162) ColumnarToRow [codegen id : 38] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] (163) Filter [codegen id : 38] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Condition : ((isnotnull(ss_hdemo_sk#118) AND isnotnull(ss_sold_time_sk#117)) AND isnotnull(ss_store_sk#119)) (164) Project [codegen id : 38] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] (165) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#121] (166) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#118] +Right keys [1]: [hd_demo_sk#121] Join condition: None (167) Project [codegen id : 38] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#117, ss_store_sk#119] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, hd_demo_sk#121] (168) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#122, t_hour#123, t_minute#124] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (169) ColumnarToRow [codegen id : 36] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#122, t_hour#123, t_minute#124] (170) Filter [codegen id : 36] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 12)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#122, t_hour#123, t_minute#124] +Condition : ((((isnotnull(t_hour#123) AND isnotnull(t_minute#124)) AND (t_hour#123 = 12)) AND (t_minute#124 < 30)) AND isnotnull(t_time_sk#122)) (171) Project [codegen id : 36] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#122] +Input [3]: [t_time_sk#122, t_hour#123, t_minute#124] (172) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] +Input [1]: [t_time_sk#122] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#125] (173) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#117] +Right keys [1]: [t_time_sk#122] Join condition: None (174) Project [codegen id : 38] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#119] +Input [3]: [ss_sold_time_sk#117, ss_store_sk#119, t_time_sk#122] (175) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#126] (176) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#119] +Right keys [1]: [s_store_sk#126] Join condition: None (177) Project [codegen id : 38] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#119, s_store_sk#126] (178) HashAggregate [codegen id : 38] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#64] -Results [1]: [count#65] +Aggregate Attributes [1]: [count#127] +Results [1]: [count#128] (179) Exchange -Input [1]: [count#65] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] +Input [1]: [count#128] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] (180) HashAggregate [codegen id : 39] -Input [1]: [count#65] +Input [1]: [count#128] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#67] -Results [1]: [count(1)#67 AS h12_to_12_30#68] +Aggregate Attributes [1]: [count(1)#130] +Results [1]: [count(1)#130 AS h12_to_12_30#131] (181) BroadcastExchange -Input [1]: [h12_to_12_30#68] -Arguments: IdentityBroadcastMode, [id=#69] +Input [1]: [h12_to_12_30#131] +Arguments: IdentityBroadcastMode, [id=#132] (182) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt index c9ebf0e9c9426..b3d0c07cd80fe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject (31) Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(In(i_category, [Books,Electronics,Sports]),In(i_class, [computers,stereo,football])),And(In(i_category, [Men,Jewelry,Women]),In(i_class, [shirts,birdal,dresses]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(In(i_category, [Books ,Electronics ,Sports ]),In(i_class, [computers ,stereo ,football ])),And(In(i_category, [Men ,Jewelry ,Women ]),In(i_class, [shirts ,birdal ,dresses ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -44,7 +44,7 @@ Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] (3) Filter [codegen id : 1] Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((i_category#4 IN (Books,Electronics,Sports) AND i_class#3 IN (computers,stereo,football)) OR (i_category#4 IN (Men,Jewelry,Women) AND i_class#3 IN (shirts,birdal,dresses))) AND isnotnull(i_item_sk#1)) +Condition : (((i_category#4 IN (Books ,Electronics ,Sports ) AND i_class#3 IN (computers ,stereo ,football )) OR (i_category#4 IN (Men ,Jewelry ,Women ) AND i_class#3 IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) (4) BroadcastExchange Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] @@ -164,7 +164,7 @@ Arguments: [avg(_w0#24) windowspecdefinition(i_category#4, i_brand#2, s_store_na (29) Filter [codegen id : 7] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, _w0#24, avg_monthly_sales#26] -Condition : ((isnotnull(avg_monthly_sales#26) AND NOT (avg_monthly_sales#26 = 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_monthly_sales#26) AND (NOT (avg_monthly_sales#26 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (30) Project [codegen id : 7] Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, avg_monthly_sales#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt index cbf6273e28568..f61c8e6945003 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject (31) Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(In(i_category, [Books,Electronics,Sports]),In(i_class, [computers,stereo,football])),And(In(i_category, [Men,Jewelry,Women]),In(i_class, [shirts,birdal,dresses]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(In(i_category, [Books ,Electronics ,Sports ]),In(i_class, [computers ,stereo ,football ])),And(In(i_category, [Men ,Jewelry ,Women ]),In(i_class, [shirts ,birdal ,dresses ]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -44,7 +44,7 @@ Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] (3) Filter [codegen id : 4] Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((i_category#4 IN (Books,Electronics,Sports) AND i_class#3 IN (computers,stereo,football)) OR (i_category#4 IN (Men,Jewelry,Women) AND i_class#3 IN (shirts,birdal,dresses))) AND isnotnull(i_item_sk#1)) +Condition : (((i_category#4 IN (Books ,Electronics ,Sports ) AND i_class#3 IN (computers ,stereo ,football )) OR (i_category#4 IN (Men ,Jewelry ,Women ) AND i_class#3 IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) (4) Scan parquet default.store_sales Output [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] @@ -164,7 +164,7 @@ Arguments: [avg(_w0#24) windowspecdefinition(i_category#4, i_brand#2, s_store_na (29) Filter [codegen id : 7] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, _w0#24, avg_monthly_sales#26] -Condition : ((isnotnull(avg_monthly_sales#26) AND NOT (avg_monthly_sales#26 = 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +Condition : (isnotnull(avg_monthly_sales#26) AND (NOT (avg_monthly_sales#26 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) (30) Project [codegen id : 7] Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, avg_monthly_sales#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt index 65c66eb083f55..8736c9861a5ce 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt @@ -82,40 +82,40 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (12) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (13) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] (14) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Condition : ((isnotnull(ss_quantity#44) AND (ss_quantity#44 >= 1)) AND (ss_quantity#44 <= 20)) (15) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#45] +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] (16) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#45] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#45, count#46] -Results [2]: [sum#47, count#48] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#45))] +Aggregate Attributes [2]: [sum#47, count#48] +Results [2]: [sum#49, count#50] (17) Exchange -Input [2]: [sum#47, count#48] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] +Input [2]: [sum#49, count#50] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#51] (18) HashAggregate [codegen id : 2] -Input [2]: [sum#47, count#48] +Input [2]: [sum#49, count#50] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#50] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#50 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#51] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))#52] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#45))#52 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#53] Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#6, [id=#7] * HashAggregate (25) @@ -128,40 +128,40 @@ Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (19) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (20) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] (21) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Condition : ((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 1)) AND (ss_quantity#54 <= 20)) (22) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#55] +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] (23) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#55] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#53, count#54] -Results [2]: [sum#55, count#56] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [2]: [sum#57, count#58] +Results [2]: [sum#59, count#60] (24) Exchange -Input [2]: [sum#55, count#56] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#57] +Input [2]: [sum#59, count#60] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] (25) HashAggregate [codegen id : 2] -Input [2]: [sum#55, count#56] +Input [2]: [sum#59, count#60] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#58] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#58 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#59] +Functions [1]: [avg(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#55))#62] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#55))#62 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#63] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#9, [id=#10] * HashAggregate (32) @@ -174,40 +174,40 @@ Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (26) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#64, ss_sold_date_sk#65] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (27) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] (28) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +Condition : ((isnotnull(ss_quantity#64) AND (ss_quantity#64 >= 21)) AND (ss_quantity#64 <= 40)) (29) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] (30) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#60] -Results [1]: [count#61] +Aggregate Attributes [1]: [count#66] +Results [1]: [count#67] (31) Exchange -Input [1]: [count#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] +Input [1]: [count#67] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#68] (32) HashAggregate [codegen id : 2] -Input [1]: [count#61] +Input [1]: [count#67] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#63] -Results [1]: [count(1)#63 AS count(1)#64] +Aggregate Attributes [1]: [count(1)#69] +Results [1]: [count(1)#69 AS count(1)#70] Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] * HashAggregate (39) @@ -220,40 +220,40 @@ Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (33) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (34) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] (35) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Condition : ((isnotnull(ss_quantity#71) AND (ss_quantity#71 >= 21)) AND (ss_quantity#71 <= 40)) (36) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#72] +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] (37) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#72] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#65, count#66] -Results [2]: [sum#67, count#68] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#72))] +Aggregate Attributes [2]: [sum#74, count#75] +Results [2]: [sum#76, count#77] (38) Exchange -Input [2]: [sum#67, count#68] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [sum#76, count#77] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] (39) HashAggregate [codegen id : 2] -Input [2]: [sum#67, count#68] +Input [2]: [sum#76, count#77] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#70] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#70 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#71] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))#79] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#72))#79 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#80] Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#13, [id=#14] * HashAggregate (46) @@ -266,40 +266,40 @@ Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (40) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (41) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] (42) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Condition : ((isnotnull(ss_quantity#81) AND (ss_quantity#81 >= 21)) AND (ss_quantity#81 <= 40)) (43) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#82] +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] (44) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#82] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#72, count#73] -Results [2]: [sum#74, count#75] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#82))] +Aggregate Attributes [2]: [sum#84, count#85] +Results [2]: [sum#86, count#87] (45) Exchange -Input [2]: [sum#74, count#75] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] +Input [2]: [sum#86, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] (46) HashAggregate [codegen id : 2] -Input [2]: [sum#74, count#75] +Input [2]: [sum#86, count#87] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#77] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#77 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#78] +Functions [1]: [avg(UnscaledValue(ss_net_paid#82))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#82))#89] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#82))#89 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#90] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#16, [id=#17] * HashAggregate (53) @@ -312,40 +312,40 @@ Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (47) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#91, ss_sold_date_sk#92] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (48) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] (49) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +Condition : ((isnotnull(ss_quantity#91) AND (ss_quantity#91 >= 41)) AND (ss_quantity#91 <= 60)) (50) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] (51) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#79] -Results [1]: [count#80] +Aggregate Attributes [1]: [count#93] +Results [1]: [count#94] (52) Exchange -Input [1]: [count#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +Input [1]: [count#94] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] (53) HashAggregate [codegen id : 2] -Input [1]: [count#80] +Input [1]: [count#94] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#82] -Results [1]: [count(1)#82 AS count(1)#83] +Aggregate Attributes [1]: [count(1)#96] +Results [1]: [count(1)#96 AS count(1)#97] Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#18, [id=#19] * HashAggregate (60) @@ -358,40 +358,40 @@ Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (54) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (55) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] (56) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Condition : ((isnotnull(ss_quantity#98) AND (ss_quantity#98 >= 41)) AND (ss_quantity#98 <= 60)) (57) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#99] +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] (58) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#99] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#84, count#85] -Results [2]: [sum#86, count#87] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#99))] +Aggregate Attributes [2]: [sum#101, count#102] +Results [2]: [sum#103, count#104] (59) Exchange -Input [2]: [sum#86, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] +Input [2]: [sum#103, count#104] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] (60) HashAggregate [codegen id : 2] -Input [2]: [sum#86, count#87] +Input [2]: [sum#103, count#104] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#89] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#89 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#90] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))#106] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#99))#106 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#107] Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#20, [id=#21] * HashAggregate (67) @@ -404,40 +404,40 @@ Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (61) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (62) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] (63) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Condition : ((isnotnull(ss_quantity#108) AND (ss_quantity#108 >= 41)) AND (ss_quantity#108 <= 60)) (64) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#109] +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] (65) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#109] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#91, count#92] -Results [2]: [sum#93, count#94] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#109))] +Aggregate Attributes [2]: [sum#111, count#112] +Results [2]: [sum#113, count#114] (66) Exchange -Input [2]: [sum#93, count#94] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] +Input [2]: [sum#113, count#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] (67) HashAggregate [codegen id : 2] -Input [2]: [sum#93, count#94] +Input [2]: [sum#113, count#114] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#96] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#96 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#97] +Functions [1]: [avg(UnscaledValue(ss_net_paid#109))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#109))#116] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#109))#116 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#117] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#23, [id=#24] * HashAggregate (74) @@ -450,40 +450,40 @@ Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (68) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#118, ss_sold_date_sk#119] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (69) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] (70) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +Condition : ((isnotnull(ss_quantity#118) AND (ss_quantity#118 >= 61)) AND (ss_quantity#118 <= 80)) (71) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] (72) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#98] -Results [1]: [count#99] +Aggregate Attributes [1]: [count#120] +Results [1]: [count#121] (73) Exchange -Input [1]: [count#99] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#100] +Input [1]: [count#121] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#122] (74) HashAggregate [codegen id : 2] -Input [1]: [count#99] +Input [1]: [count#121] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#101] -Results [1]: [count(1)#101 AS count(1)#102] +Aggregate Attributes [1]: [count(1)#123] +Results [1]: [count(1)#123 AS count(1)#124] Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#25, [id=#26] * HashAggregate (81) @@ -496,40 +496,40 @@ Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (75) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (76) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] (77) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Condition : ((isnotnull(ss_quantity#125) AND (ss_quantity#125 >= 61)) AND (ss_quantity#125 <= 80)) (78) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#126] +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] (79) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#126] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#103, count#104] -Results [2]: [sum#105, count#106] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#126))] +Aggregate Attributes [2]: [sum#128, count#129] +Results [2]: [sum#130, count#131] (80) Exchange -Input [2]: [sum#105, count#106] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#107] +Input [2]: [sum#130, count#131] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#132] (81) HashAggregate [codegen id : 2] -Input [2]: [sum#105, count#106] +Input [2]: [sum#130, count#131] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#108] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#108 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#109] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))#133] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#126))#133 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#134] Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#27, [id=#28] * HashAggregate (88) @@ -542,40 +542,40 @@ Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (82) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (83) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] (84) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Condition : ((isnotnull(ss_quantity#135) AND (ss_quantity#135 >= 61)) AND (ss_quantity#135 <= 80)) (85) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#136] +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] (86) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#136] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#110, count#111] -Results [2]: [sum#112, count#113] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#136))] +Aggregate Attributes [2]: [sum#138, count#139] +Results [2]: [sum#140, count#141] (87) Exchange -Input [2]: [sum#112, count#113] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] +Input [2]: [sum#140, count#141] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#142] (88) HashAggregate [codegen id : 2] -Input [2]: [sum#112, count#113] +Input [2]: [sum#140, count#141] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#115] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#115 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#116] +Functions [1]: [avg(UnscaledValue(ss_net_paid#136))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#136))#143] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#136))#143 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#144] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#30, [id=#31] * HashAggregate (95) @@ -588,40 +588,40 @@ Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (89) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#145, ss_sold_date_sk#146] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (90) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] (91) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] +Condition : ((isnotnull(ss_quantity#145) AND (ss_quantity#145 >= 81)) AND (ss_quantity#145 <= 100)) (92) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] (93) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#117] -Results [1]: [count#118] +Aggregate Attributes [1]: [count#147] +Results [1]: [count#148] (94) Exchange -Input [1]: [count#118] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#119] +Input [1]: [count#148] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#149] (95) HashAggregate [codegen id : 2] -Input [1]: [count#118] +Input [1]: [count#148] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#120] -Results [1]: [count(1)#120 AS count(1)#121] +Aggregate Attributes [1]: [count(1)#150] +Results [1]: [count(1)#150 AS count(1)#151] Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#32, [id=#33] * HashAggregate (102) @@ -634,40 +634,40 @@ Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (96) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (97) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] (98) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Condition : ((isnotnull(ss_quantity#152) AND (ss_quantity#152 >= 81)) AND (ss_quantity#152 <= 100)) (99) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#153] +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] (100) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#153] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#122, count#123] -Results [2]: [sum#124, count#125] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#153))] +Aggregate Attributes [2]: [sum#155, count#156] +Results [2]: [sum#157, count#158] (101) Exchange -Input [2]: [sum#124, count#125] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#126] +Input [2]: [sum#157, count#158] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#159] (102) HashAggregate [codegen id : 2] -Input [2]: [sum#124, count#125] +Input [2]: [sum#157, count#158] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#127] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#127 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#128] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))#160] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#153))#160 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#161] Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#34, [id=#35] * HashAggregate (109) @@ -680,39 +680,39 @@ Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (103) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] (105) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Condition : ((isnotnull(ss_quantity#162) AND (ss_quantity#162 >= 81)) AND (ss_quantity#162 <= 100)) (106) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#163] +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] (107) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#163] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#129, count#130] -Results [2]: [sum#131, count#132] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#163))] +Aggregate Attributes [2]: [sum#165, count#166] +Results [2]: [sum#167, count#168] (108) Exchange -Input [2]: [sum#131, count#132] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#133] +Input [2]: [sum#167, count#168] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#169] (109) HashAggregate [codegen id : 2] -Input [2]: [sum#131, count#132] +Input [2]: [sum#167, count#168] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#134] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#134 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#135] +Functions [1]: [avg(UnscaledValue(ss_net_paid#163))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#163))#170] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#163))#170 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#171] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt index 65c66eb083f55..8736c9861a5ce 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt @@ -82,40 +82,40 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (12) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (13) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] (14) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Condition : ((isnotnull(ss_quantity#44) AND (ss_quantity#44 >= 1)) AND (ss_quantity#44 <= 20)) (15) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#45] +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] (16) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#45] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#45, count#46] -Results [2]: [sum#47, count#48] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#45))] +Aggregate Attributes [2]: [sum#47, count#48] +Results [2]: [sum#49, count#50] (17) Exchange -Input [2]: [sum#47, count#48] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] +Input [2]: [sum#49, count#50] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#51] (18) HashAggregate [codegen id : 2] -Input [2]: [sum#47, count#48] +Input [2]: [sum#49, count#50] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#50] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#50 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#51] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))#52] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#45))#52 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#53] Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#6, [id=#7] * HashAggregate (25) @@ -128,40 +128,40 @@ Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (19) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (20) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] (21) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Condition : ((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 1)) AND (ss_quantity#54 <= 20)) (22) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#55] +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] (23) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#55] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#53, count#54] -Results [2]: [sum#55, count#56] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [2]: [sum#57, count#58] +Results [2]: [sum#59, count#60] (24) Exchange -Input [2]: [sum#55, count#56] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#57] +Input [2]: [sum#59, count#60] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] (25) HashAggregate [codegen id : 2] -Input [2]: [sum#55, count#56] +Input [2]: [sum#59, count#60] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#58] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#58 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#59] +Functions [1]: [avg(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#55))#62] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#55))#62 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#63] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#9, [id=#10] * HashAggregate (32) @@ -174,40 +174,40 @@ Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (26) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#64, ss_sold_date_sk#65] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (27) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] (28) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +Condition : ((isnotnull(ss_quantity#64) AND (ss_quantity#64 >= 21)) AND (ss_quantity#64 <= 40)) (29) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] (30) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#60] -Results [1]: [count#61] +Aggregate Attributes [1]: [count#66] +Results [1]: [count#67] (31) Exchange -Input [1]: [count#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] +Input [1]: [count#67] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#68] (32) HashAggregate [codegen id : 2] -Input [1]: [count#61] +Input [1]: [count#67] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#63] -Results [1]: [count(1)#63 AS count(1)#64] +Aggregate Attributes [1]: [count(1)#69] +Results [1]: [count(1)#69 AS count(1)#70] Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] * HashAggregate (39) @@ -220,40 +220,40 @@ Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (33) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (34) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] (35) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Condition : ((isnotnull(ss_quantity#71) AND (ss_quantity#71 >= 21)) AND (ss_quantity#71 <= 40)) (36) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#72] +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] (37) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#72] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#65, count#66] -Results [2]: [sum#67, count#68] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#72))] +Aggregate Attributes [2]: [sum#74, count#75] +Results [2]: [sum#76, count#77] (38) Exchange -Input [2]: [sum#67, count#68] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [sum#76, count#77] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] (39) HashAggregate [codegen id : 2] -Input [2]: [sum#67, count#68] +Input [2]: [sum#76, count#77] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#70] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#70 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#71] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))#79] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#72))#79 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#80] Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#13, [id=#14] * HashAggregate (46) @@ -266,40 +266,40 @@ Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (40) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (41) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] (42) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Condition : ((isnotnull(ss_quantity#81) AND (ss_quantity#81 >= 21)) AND (ss_quantity#81 <= 40)) (43) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#82] +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] (44) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#82] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#72, count#73] -Results [2]: [sum#74, count#75] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#82))] +Aggregate Attributes [2]: [sum#84, count#85] +Results [2]: [sum#86, count#87] (45) Exchange -Input [2]: [sum#74, count#75] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] +Input [2]: [sum#86, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] (46) HashAggregate [codegen id : 2] -Input [2]: [sum#74, count#75] +Input [2]: [sum#86, count#87] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#77] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#77 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#78] +Functions [1]: [avg(UnscaledValue(ss_net_paid#82))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#82))#89] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#82))#89 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#90] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#16, [id=#17] * HashAggregate (53) @@ -312,40 +312,40 @@ Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (47) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#91, ss_sold_date_sk#92] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (48) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] (49) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +Condition : ((isnotnull(ss_quantity#91) AND (ss_quantity#91 >= 41)) AND (ss_quantity#91 <= 60)) (50) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] (51) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#79] -Results [1]: [count#80] +Aggregate Attributes [1]: [count#93] +Results [1]: [count#94] (52) Exchange -Input [1]: [count#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +Input [1]: [count#94] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] (53) HashAggregate [codegen id : 2] -Input [1]: [count#80] +Input [1]: [count#94] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#82] -Results [1]: [count(1)#82 AS count(1)#83] +Aggregate Attributes [1]: [count(1)#96] +Results [1]: [count(1)#96 AS count(1)#97] Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#18, [id=#19] * HashAggregate (60) @@ -358,40 +358,40 @@ Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (54) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (55) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] (56) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Condition : ((isnotnull(ss_quantity#98) AND (ss_quantity#98 >= 41)) AND (ss_quantity#98 <= 60)) (57) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#99] +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] (58) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#99] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#84, count#85] -Results [2]: [sum#86, count#87] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#99))] +Aggregate Attributes [2]: [sum#101, count#102] +Results [2]: [sum#103, count#104] (59) Exchange -Input [2]: [sum#86, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] +Input [2]: [sum#103, count#104] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] (60) HashAggregate [codegen id : 2] -Input [2]: [sum#86, count#87] +Input [2]: [sum#103, count#104] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#89] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#89 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#90] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))#106] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#99))#106 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#107] Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#20, [id=#21] * HashAggregate (67) @@ -404,40 +404,40 @@ Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (61) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (62) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] (63) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Condition : ((isnotnull(ss_quantity#108) AND (ss_quantity#108 >= 41)) AND (ss_quantity#108 <= 60)) (64) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#109] +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] (65) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#109] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#91, count#92] -Results [2]: [sum#93, count#94] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#109))] +Aggregate Attributes [2]: [sum#111, count#112] +Results [2]: [sum#113, count#114] (66) Exchange -Input [2]: [sum#93, count#94] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] +Input [2]: [sum#113, count#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] (67) HashAggregate [codegen id : 2] -Input [2]: [sum#93, count#94] +Input [2]: [sum#113, count#114] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#96] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#96 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#97] +Functions [1]: [avg(UnscaledValue(ss_net_paid#109))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#109))#116] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#109))#116 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#117] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#23, [id=#24] * HashAggregate (74) @@ -450,40 +450,40 @@ Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (68) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#118, ss_sold_date_sk#119] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (69) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] (70) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +Condition : ((isnotnull(ss_quantity#118) AND (ss_quantity#118 >= 61)) AND (ss_quantity#118 <= 80)) (71) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] (72) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#98] -Results [1]: [count#99] +Aggregate Attributes [1]: [count#120] +Results [1]: [count#121] (73) Exchange -Input [1]: [count#99] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#100] +Input [1]: [count#121] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#122] (74) HashAggregate [codegen id : 2] -Input [1]: [count#99] +Input [1]: [count#121] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#101] -Results [1]: [count(1)#101 AS count(1)#102] +Aggregate Attributes [1]: [count(1)#123] +Results [1]: [count(1)#123 AS count(1)#124] Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#25, [id=#26] * HashAggregate (81) @@ -496,40 +496,40 @@ Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (75) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (76) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] (77) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Condition : ((isnotnull(ss_quantity#125) AND (ss_quantity#125 >= 61)) AND (ss_quantity#125 <= 80)) (78) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#126] +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] (79) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#126] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#103, count#104] -Results [2]: [sum#105, count#106] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#126))] +Aggregate Attributes [2]: [sum#128, count#129] +Results [2]: [sum#130, count#131] (80) Exchange -Input [2]: [sum#105, count#106] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#107] +Input [2]: [sum#130, count#131] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#132] (81) HashAggregate [codegen id : 2] -Input [2]: [sum#105, count#106] +Input [2]: [sum#130, count#131] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#108] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#108 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#109] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))#133] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#126))#133 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#134] Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#27, [id=#28] * HashAggregate (88) @@ -542,40 +542,40 @@ Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (82) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (83) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] (84) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Condition : ((isnotnull(ss_quantity#135) AND (ss_quantity#135 >= 61)) AND (ss_quantity#135 <= 80)) (85) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#136] +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] (86) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#136] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#110, count#111] -Results [2]: [sum#112, count#113] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#136))] +Aggregate Attributes [2]: [sum#138, count#139] +Results [2]: [sum#140, count#141] (87) Exchange -Input [2]: [sum#112, count#113] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] +Input [2]: [sum#140, count#141] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#142] (88) HashAggregate [codegen id : 2] -Input [2]: [sum#112, count#113] +Input [2]: [sum#140, count#141] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#115] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#115 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#116] +Functions [1]: [avg(UnscaledValue(ss_net_paid#136))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#136))#143] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#136))#143 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#144] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#30, [id=#31] * HashAggregate (95) @@ -588,40 +588,40 @@ Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (89) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#145, ss_sold_date_sk#146] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (90) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] (91) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] +Condition : ((isnotnull(ss_quantity#145) AND (ss_quantity#145 >= 81)) AND (ss_quantity#145 <= 100)) (92) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] (93) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#117] -Results [1]: [count#118] +Aggregate Attributes [1]: [count#147] +Results [1]: [count#148] (94) Exchange -Input [1]: [count#118] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#119] +Input [1]: [count#148] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#149] (95) HashAggregate [codegen id : 2] -Input [1]: [count#118] +Input [1]: [count#148] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#120] -Results [1]: [count(1)#120 AS count(1)#121] +Aggregate Attributes [1]: [count(1)#150] +Results [1]: [count(1)#150 AS count(1)#151] Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#32, [id=#33] * HashAggregate (102) @@ -634,40 +634,40 @@ Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (96) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (97) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] (98) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Condition : ((isnotnull(ss_quantity#152) AND (ss_quantity#152 >= 81)) AND (ss_quantity#152 <= 100)) (99) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#153] +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] (100) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#153] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#122, count#123] -Results [2]: [sum#124, count#125] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#153))] +Aggregate Attributes [2]: [sum#155, count#156] +Results [2]: [sum#157, count#158] (101) Exchange -Input [2]: [sum#124, count#125] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#126] +Input [2]: [sum#157, count#158] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#159] (102) HashAggregate [codegen id : 2] -Input [2]: [sum#124, count#125] +Input [2]: [sum#157, count#158] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#127] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#127 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#128] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))#160] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#153))#160 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#161] Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#34, [id=#35] * HashAggregate (109) @@ -680,39 +680,39 @@ Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (103) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] (105) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Condition : ((isnotnull(ss_quantity#162) AND (ss_quantity#162 >= 81)) AND (ss_quantity#162 <= 100)) (106) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#163] +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] (107) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#163] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#129, count#130] -Results [2]: [sum#131, count#132] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#163))] +Aggregate Attributes [2]: [sum#165, count#166] +Results [2]: [sum#167, count#168] (108) Exchange -Input [2]: [sum#131, count#132] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#133] +Input [2]: [sum#167, count#168] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#169] (109) HashAggregate [codegen id : 2] -Input [2]: [sum#131, count#132] +Input [2]: [sum#167, count#168] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#134] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#134 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#135] +Functions [1]: [avg(UnscaledValue(ss_net_paid#163))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#163))#170] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#163))#170 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#171] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt index a74b232e8c667..5226fce4ef512 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt @@ -183,108 +183,108 @@ Aggregate Attributes [1]: [count(1)#17] Results [1]: [count(1)#17 AS amc#18] (29) Scan parquet default.web_sales -Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Output [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] (31) Filter [codegen id : 8] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Condition : ((isnotnull(ws_ship_hdemo_sk#20) AND isnotnull(ws_sold_time_sk#19)) AND isnotnull(ws_web_page_sk#21)) (32) Project [codegen id : 8] -Output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Output [3]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] (33) ReusedExchange [Reuses operator id: 9] -Output [1]: [wp_web_page_sk#5] +Output [1]: [wp_web_page_sk#23] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#3] -Right keys [1]: [wp_web_page_sk#5] +Left keys [1]: [ws_web_page_sk#21] +Right keys [1]: [wp_web_page_sk#23] Join condition: None (35) Project [codegen id : 8] -Output [2]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, wp_web_page_sk#5] +Output [2]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, wp_web_page_sk#23] (36) ReusedExchange [Reuses operator id: 16] -Output [1]: [hd_demo_sk#8] +Output [1]: [hd_demo_sk#24] (37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_ship_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#8] +Left keys [1]: [ws_ship_hdemo_sk#20] +Right keys [1]: [hd_demo_sk#24] Join condition: None (38) Project [codegen id : 8] -Output [1]: [ws_sold_time_sk#1] -Input [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, hd_demo_sk#8] +Output [1]: [ws_sold_time_sk#19] +Input [3]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, hd_demo_sk#24] (39) Scan parquet default.time_dim -Output [2]: [t_time_sk#11, t_hour#12] +Output [2]: [t_time_sk#25, t_hour#26] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 7] -Input [2]: [t_time_sk#11, t_hour#12] +Input [2]: [t_time_sk#25, t_hour#26] (41) Filter [codegen id : 7] -Input [2]: [t_time_sk#11, t_hour#12] -Condition : (((isnotnull(t_hour#12) AND (t_hour#12 >= 19)) AND (t_hour#12 <= 20)) AND isnotnull(t_time_sk#11)) +Input [2]: [t_time_sk#25, t_hour#26] +Condition : (((isnotnull(t_hour#26) AND (t_hour#26 >= 19)) AND (t_hour#26 <= 20)) AND isnotnull(t_time_sk#25)) (42) Project [codegen id : 7] -Output [1]: [t_time_sk#11] -Input [2]: [t_time_sk#11, t_hour#12] +Output [1]: [t_time_sk#25] +Input [2]: [t_time_sk#25, t_hour#26] (43) BroadcastExchange -Input [1]: [t_time_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [t_time_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_time_sk#1] -Right keys [1]: [t_time_sk#11] +Left keys [1]: [ws_sold_time_sk#19] +Right keys [1]: [t_time_sk#25] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ws_sold_time_sk#1, t_time_sk#11] +Input [2]: [ws_sold_time_sk#19, t_time_sk#25] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#20] -Results [1]: [count#21] +Aggregate Attributes [1]: [count#28] +Results [1]: [count#29] (47) Exchange -Input [1]: [count#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [count#29] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#30] (48) HashAggregate [codegen id : 9] -Input [1]: [count#21] +Input [1]: [count#29] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#23] -Results [1]: [count(1)#23 AS pmc#24] +Aggregate Attributes [1]: [count(1)#31] +Results [1]: [count(1)#31 AS pmc#32] (49) BroadcastExchange -Input [1]: [pmc#24] -Arguments: IdentityBroadcastMode, [id=#25] +Input [1]: [pmc#32] +Arguments: IdentityBroadcastMode, [id=#33] (50) BroadcastNestedLoopJoin [codegen id : 10] Join condition: None (51) Project [codegen id : 10] -Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#24 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#26] -Input [2]: [amc#18, pmc#24] +Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#32 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#34] +Input [2]: [amc#18, pmc#32] (52) Sort [codegen id : 10] -Input [1]: [am_pm_ratio#26] -Arguments: [am_pm_ratio#26 ASC NULLS FIRST], true, 0 +Input [1]: [am_pm_ratio#34] +Arguments: [am_pm_ratio#34 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt index 070b497ac4d18..e369a027040d3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt @@ -183,108 +183,108 @@ Aggregate Attributes [1]: [count(1)#17] Results [1]: [count(1)#17 AS amc#18] (29) Scan parquet default.web_sales -Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Output [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] (31) Filter [codegen id : 8] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Condition : ((isnotnull(ws_ship_hdemo_sk#20) AND isnotnull(ws_sold_time_sk#19)) AND isnotnull(ws_web_page_sk#21)) (32) Project [codegen id : 8] -Output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Output [3]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] (33) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#23] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_ship_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ws_ship_hdemo_sk#20] +Right keys [1]: [hd_demo_sk#23] Join condition: None (35) Project [codegen id : 8] -Output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] +Output [2]: [ws_sold_time_sk#19, ws_web_page_sk#21] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, hd_demo_sk#23] (36) Scan parquet default.time_dim -Output [2]: [t_time_sk#8, t_hour#9] +Output [2]: [t_time_sk#24, t_hour#25] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [2]: [t_time_sk#8, t_hour#9] +Input [2]: [t_time_sk#24, t_hour#25] (38) Filter [codegen id : 6] -Input [2]: [t_time_sk#8, t_hour#9] -Condition : (((isnotnull(t_hour#9) AND (t_hour#9 >= 19)) AND (t_hour#9 <= 20)) AND isnotnull(t_time_sk#8)) +Input [2]: [t_time_sk#24, t_hour#25] +Condition : (((isnotnull(t_hour#25) AND (t_hour#25 >= 19)) AND (t_hour#25 <= 20)) AND isnotnull(t_time_sk#24)) (39) Project [codegen id : 6] -Output [1]: [t_time_sk#8] -Input [2]: [t_time_sk#8, t_hour#9] +Output [1]: [t_time_sk#24] +Input [2]: [t_time_sk#24, t_hour#25] (40) BroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [t_time_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_time_sk#1] -Right keys [1]: [t_time_sk#8] +Left keys [1]: [ws_sold_time_sk#19] +Right keys [1]: [t_time_sk#24] Join condition: None (42) Project [codegen id : 8] -Output [1]: [ws_web_page_sk#3] -Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#8] +Output [1]: [ws_web_page_sk#21] +Input [3]: [ws_sold_time_sk#19, ws_web_page_sk#21, t_time_sk#24] (43) ReusedExchange [Reuses operator id: 23] -Output [1]: [wp_web_page_sk#11] +Output [1]: [wp_web_page_sk#27] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#3] -Right keys [1]: [wp_web_page_sk#11] +Left keys [1]: [ws_web_page_sk#21] +Right keys [1]: [wp_web_page_sk#27] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ws_web_page_sk#3, wp_web_page_sk#11] +Input [2]: [ws_web_page_sk#21, wp_web_page_sk#27] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#20] -Results [1]: [count#21] +Aggregate Attributes [1]: [count#28] +Results [1]: [count#29] (47) Exchange -Input [1]: [count#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [count#29] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#30] (48) HashAggregate [codegen id : 9] -Input [1]: [count#21] +Input [1]: [count#29] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#23] -Results [1]: [count(1)#23 AS pmc#24] +Aggregate Attributes [1]: [count(1)#31] +Results [1]: [count(1)#31 AS pmc#32] (49) BroadcastExchange -Input [1]: [pmc#24] -Arguments: IdentityBroadcastMode, [id=#25] +Input [1]: [pmc#32] +Arguments: IdentityBroadcastMode, [id=#33] (50) BroadcastNestedLoopJoin [codegen id : 10] Join condition: None (51) Project [codegen id : 10] -Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#24 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#26] -Input [2]: [amc#18, pmc#24] +Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#32 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#34] +Input [2]: [amc#18, pmc#32] (52) Sort [codegen id : 10] -Input [1]: [am_pm_ratio#26] -Arguments: [am_pm_ratio#26 ASC NULLS FIRST], true, 0 +Input [1]: [am_pm_ratio#34] +Arguments: [am_pm_ratio#34 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt index e38c455eb6be2..3554ce5dcb117 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt @@ -187,7 +187,7 @@ Input [5]: [cr_call_center_sk#2, cr_net_loss#3, c_current_cdemo_sk#12, c_current Output [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] +PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown )),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree ))), IsNotNull(cd_demo_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 5] @@ -195,7 +195,7 @@ Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] (33) Filter [codegen id : 5] Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] -Condition : ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Unknown)) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = Advanced Degree))) AND isnotnull(cd_demo_sk#21)) +Condition : ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Unknown )) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = Advanced Degree ))) AND isnotnull(cd_demo_sk#21)) (34) BroadcastExchange Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt index c14a8421bc3b0..fc0da4adf7b16 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt @@ -183,7 +183,7 @@ Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_curre Output [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] +PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown )),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree ))), IsNotNull(cd_demo_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 5] @@ -191,7 +191,7 @@ Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] (32) Filter [codegen id : 5] Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] -Condition : ((((cd_marital_status#24 = M) AND (cd_education_status#25 = Unknown)) OR ((cd_marital_status#24 = W) AND (cd_education_status#25 = Advanced Degree))) AND isnotnull(cd_demo_sk#23)) +Condition : ((((cd_marital_status#24 = M) AND (cd_education_status#25 = Unknown )) OR ((cd_marital_status#24 = W) AND (cd_education_status#25 = Advanced Degree ))) AND isnotnull(cd_demo_sk#23)) (33) BroadcastExchange Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt index 8ae9d897a7d79..94e76ced527a0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt @@ -84,7 +84,7 @@ Input [2]: [d_date_sk#8, d_date#9] (11) Filter [codegen id : 2] Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) (12) Project [codegen id : 2] Output [1]: [d_date_sk#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index bae7dd2f3a240..46953fe184f28 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -108,7 +108,7 @@ Input [2]: [d_date_sk#11, d_date#12] (16) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10983)) AND (d_date#12 <= 11073)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) (17) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt index aec81c0d5c7aa..2466a4d38f21e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt @@ -47,7 +47,7 @@ Input [5]: [sr_item_sk#1, sr_reason_sk#2, sr_ticket_number#3, sr_return_quantity Output [2]: [r_reason_sk#6, r_reason_desc#7] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28), IsNotNull(r_reason_sk)] +PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -55,7 +55,7 @@ Input [2]: [r_reason_sk#6, r_reason_desc#7] (7) Filter [codegen id : 1] Input [2]: [r_reason_sk#6, r_reason_desc#7] -Condition : ((isnotnull(r_reason_desc#7) AND (r_reason_desc#7 = reason 28)) AND isnotnull(r_reason_sk#6)) +Condition : ((isnotnull(r_reason_desc#7) AND (r_reason_desc#7 = reason 28 )) AND isnotnull(r_reason_sk#6)) (8) Project [codegen id : 1] Output [1]: [r_reason_sk#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index 1875c5418a890..1f3470198cd20 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -85,7 +85,7 @@ Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, s Output [2]: [r_reason_sk#14, r_reason_desc#15] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28), IsNotNull(r_reason_sk)] +PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 5] @@ -93,7 +93,7 @@ Input [2]: [r_reason_sk#14, r_reason_desc#15] (16) Filter [codegen id : 5] Input [2]: [r_reason_sk#14, r_reason_desc#15] -Condition : ((isnotnull(r_reason_desc#15) AND (r_reason_desc#15 = reason 28)) AND isnotnull(r_reason_sk#14)) +Condition : ((isnotnull(r_reason_desc#15) AND (r_reason_desc#15 = reason 28 )) AND isnotnull(r_reason_sk#14)) (17) Project [codegen id : 5] Output [1]: [r_reason_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index bee7110ecd6dd..8a5b567285939 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -77,30 +77,30 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet default.web_sales -Output [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Output [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Input [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] (9) Project [codegen id : 3] -Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#10, ws_order_number#5 AS ws_order_number#5#11] -Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Output [2]: [ws_warehouse_sk#10, ws_order_number#11] +Input [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] (10) Exchange -Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] -Arguments: hashpartitioning(ws_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [2]: [ws_warehouse_sk#10, ws_order_number#11] +Arguments: hashpartitioning(ws_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 4] -Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] -Arguments: [ws_order_number#5#11 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#10, ws_order_number#11] +Arguments: [ws_order_number#11 ASC NULLS FIRST], false, 0 (12) SortMergeJoin Left keys [1]: [ws_order_number#5] -Right keys [1]: [ws_order_number#5#11] -Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#10) +Right keys [1]: [ws_order_number#11] +Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#10) (13) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -108,168 +108,168 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (14) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#13] +Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#14] (15) Sort [codegen id : 6] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: [cast(ws_order_number#5 as bigint) ASC NULLS FIRST], false, 0 (16) Scan parquet default.web_returns -Output [2]: [wr_order_number#14, wr_returned_date_sk#15] +Output [2]: [wr_order_number#15, wr_returned_date_sk#16] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] ReadSchema: struct (17) ColumnarToRow [codegen id : 7] -Input [2]: [wr_order_number#14, wr_returned_date_sk#15] +Input [2]: [wr_order_number#15, wr_returned_date_sk#16] (18) Project [codegen id : 7] -Output [1]: [wr_order_number#14] -Input [2]: [wr_order_number#14, wr_returned_date_sk#15] +Output [1]: [wr_order_number#15] +Input [2]: [wr_order_number#15, wr_returned_date_sk#16] (19) Exchange -Input [1]: [wr_order_number#14] -Arguments: hashpartitioning(wr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [1]: [wr_order_number#15] +Arguments: hashpartitioning(wr_order_number#15, 5), ENSURE_REQUIREMENTS, [id=#17] (20) Sort [codegen id : 8] -Input [1]: [wr_order_number#14] -Arguments: [wr_order_number#14 ASC NULLS FIRST], false, 0 +Input [1]: [wr_order_number#15] +Arguments: [wr_order_number#15 ASC NULLS FIRST], false, 0 (21) SortMergeJoin Left keys [1]: [cast(ws_order_number#5 as bigint)] -Right keys [1]: [wr_order_number#14] +Right keys [1]: [wr_order_number#15] Join condition: None (22) Scan parquet default.customer_address -Output [2]: [ca_address_sk#17, ca_state#18] +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 (23) ColumnarToRow [codegen id : 9] -Input [2]: [ca_address_sk#17, ca_state#18] +Input [2]: [ca_address_sk#18, ca_state#19] (24) Filter [codegen id : 9] -Input [2]: [ca_address_sk#17, ca_state#18] -Condition : ((isnotnull(ca_state#18) AND (ca_state#18 = IL)) AND isnotnull(ca_address_sk#17)) +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)) (25) Project [codegen id : 9] -Output [1]: [ca_address_sk#17] -Input [2]: [ca_address_sk#17, ca_state#18] +Output [1]: [ca_address_sk#18] +Input [2]: [ca_address_sk#18, ca_state#19] (26) BroadcastExchange -Input [1]: [ca_address_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [ca_address_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (27) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#17] +Right keys [1]: [ca_address_sk#18] Join condition: None (28) Project [codegen id : 12] Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#17] +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#18] (29) Scan parquet default.web_site -Output [2]: [web_site_sk#20, web_company_name#21] +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)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 10] -Input [2]: [web_site_sk#20, web_company_name#21] +Input [2]: [web_site_sk#21, web_company_name#22] (31) Filter [codegen id : 10] -Input [2]: [web_site_sk#20, web_company_name#21] -Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri)) AND isnotnull(web_site_sk#20)) +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)) (32) Project [codegen id : 10] -Output [1]: [web_site_sk#20] -Input [2]: [web_site_sk#20, web_company_name#21] +Output [1]: [web_site_sk#21] +Input [2]: [web_site_sk#21, web_company_name#22] (33) BroadcastExchange -Input [1]: [web_site_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [web_site_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#20] +Right keys [1]: [web_site_sk#21] Join condition: None (35) Project [codegen id : 12] Output [4]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#20] +Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#21] (36) Scan parquet default.date_dim -Output [2]: [d_date_sk#23, d_date#24] +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 (37) ColumnarToRow [codegen id : 11] -Input [2]: [d_date_sk#23, d_date#24] +Input [2]: [d_date_sk#24, d_date#25] (38) Filter [codegen id : 11] -Input [2]: [d_date_sk#23, d_date#24] -Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 10623)) AND (d_date#24 <= 10683)) AND isnotnull(d_date_sk#23)) +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)) (39) Project [codegen id : 11] -Output [1]: [d_date_sk#23] -Input [2]: [d_date_sk#23, d_date#24] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_date#25] (40) BroadcastExchange -Input [1]: [d_date_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None (42) Project [codegen id : 12] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [5]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#23] +Input [5]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#24] (43) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] -Results [3]: [ws_order_number#5, sum#28, sum#29] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28] +Results [3]: [ws_order_number#5, sum#29, sum#30] (44) Exchange -Input [3]: [ws_order_number#5, sum#28, sum#29] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [ws_order_number#5, sum#29, sum#30] +Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#31] (45) HashAggregate [codegen id : 13] -Input [3]: [ws_order_number#5, sum#28, sum#29] +Input [3]: [ws_order_number#5, sum#29, sum#30] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] -Results [3]: [ws_order_number#5, sum#28, sum#29] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28] +Results [3]: [ws_order_number#5, sum#29, sum#30] (46) HashAggregate [codegen id : 13] -Input [3]: [ws_order_number#5, sum#28, sum#29] +Input [3]: [ws_order_number#5, sum#29, sum#30] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] -Results [3]: [sum#28, sum#29, count#32] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28, count(ws_order_number#5)#32] +Results [3]: [sum#29, sum#30, count#33] (47) Exchange -Input [3]: [sum#28, sum#29, count#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [sum#29, sum#30, count#33] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#34] (48) HashAggregate [codegen id : 14] -Input [3]: [sum#28, sum#29, count#32] +Input [3]: [sum#29, sum#30, count#33] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] -Results [3]: [count(ws_order_number#5)#31 AS order count #34, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#26,17,2) AS total shipping cost #35, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#27,17,2) AS total net profit #36] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28, count(ws_order_number#5)#32] +Results [3]: [count(ws_order_number#5)#32 AS order count #35, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#27,17,2) AS total shipping cost #36, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#28,17,2) AS total net profit #37] (49) Sort [codegen id : 14] -Input [3]: [order count #34, total shipping cost #35, total net profit #36] -Arguments: [order count #34 ASC NULLS FIRST], true, 0 +Input [3]: [order count #35, total shipping cost #36, total net profit #37] +Arguments: [order count #35 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index efa09c2f625cd..042d4a3e347a2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -77,30 +77,30 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet default.web_sales -Output [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Output [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Input [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] (9) Project [codegen id : 3] -Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#10, ws_order_number#5 AS ws_order_number#5#11] -Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Output [2]: [ws_warehouse_sk#10, ws_order_number#11] +Input [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] (10) Exchange -Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] -Arguments: hashpartitioning(ws_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [2]: [ws_warehouse_sk#10, ws_order_number#11] +Arguments: hashpartitioning(ws_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 4] -Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] -Arguments: [ws_order_number#5#11 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#10, ws_order_number#11] +Arguments: [ws_order_number#11 ASC NULLS FIRST], false, 0 (12) SortMergeJoin Left keys [1]: [ws_order_number#5] -Right keys [1]: [ws_order_number#5#11] -Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#10) +Right keys [1]: [ws_order_number#11] +Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#10) (13) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -108,168 +108,168 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (14) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#13] +Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#14] (15) Sort [codegen id : 6] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: [cast(ws_order_number#5 as bigint) ASC NULLS FIRST], false, 0 (16) Scan parquet default.web_returns -Output [2]: [wr_order_number#14, wr_returned_date_sk#15] +Output [2]: [wr_order_number#15, wr_returned_date_sk#16] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] ReadSchema: struct (17) ColumnarToRow [codegen id : 7] -Input [2]: [wr_order_number#14, wr_returned_date_sk#15] +Input [2]: [wr_order_number#15, wr_returned_date_sk#16] (18) Project [codegen id : 7] -Output [1]: [wr_order_number#14] -Input [2]: [wr_order_number#14, wr_returned_date_sk#15] +Output [1]: [wr_order_number#15] +Input [2]: [wr_order_number#15, wr_returned_date_sk#16] (19) Exchange -Input [1]: [wr_order_number#14] -Arguments: hashpartitioning(wr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [1]: [wr_order_number#15] +Arguments: hashpartitioning(wr_order_number#15, 5), ENSURE_REQUIREMENTS, [id=#17] (20) Sort [codegen id : 8] -Input [1]: [wr_order_number#14] -Arguments: [wr_order_number#14 ASC NULLS FIRST], false, 0 +Input [1]: [wr_order_number#15] +Arguments: [wr_order_number#15 ASC NULLS FIRST], false, 0 (21) SortMergeJoin Left keys [1]: [cast(ws_order_number#5 as bigint)] -Right keys [1]: [wr_order_number#14] +Right keys [1]: [wr_order_number#15] Join condition: None (22) Scan parquet default.date_dim -Output [2]: [d_date_sk#17, d_date#18] +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 (23) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#17, d_date#18] +Input [2]: [d_date_sk#18, d_date#19] (24) Filter [codegen id : 9] -Input [2]: [d_date_sk#17, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 10623)) AND (d_date#18 <= 10683)) AND isnotnull(d_date_sk#17)) +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)) (25) Project [codegen id : 9] -Output [1]: [d_date_sk#17] -Input [2]: [d_date_sk#17, d_date#18] +Output [1]: [d_date_sk#18] +Input [2]: [d_date_sk#18, d_date#19] (26) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [d_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (27) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#17] +Right keys [1]: [d_date_sk#18] Join condition: None (28) Project [codegen id : 12] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#17] +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#18] (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +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 (30) ColumnarToRow [codegen id : 10] -Input [2]: [ca_address_sk#20, ca_state#21] +Input [2]: [ca_address_sk#21, ca_state#22] (31) Filter [codegen id : 10] -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : ((isnotnull(ca_state#21) AND (ca_state#21 = IL)) AND isnotnull(ca_address_sk#20)) +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)) (32) Project [codegen id : 10] -Output [1]: [ca_address_sk#20] -Input [2]: [ca_address_sk#20, ca_state#21] +Output [1]: [ca_address_sk#21] +Input [2]: [ca_address_sk#21, ca_state#22] (33) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [ca_address_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#21] Join condition: None (35) Project [codegen id : 12] Output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#20] +Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#21] (36) Scan parquet default.web_site -Output [2]: [web_site_sk#23, web_company_name#24] +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)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 11] -Input [2]: [web_site_sk#23, web_company_name#24] +Input [2]: [web_site_sk#24, web_company_name#25] (38) Filter [codegen id : 11] -Input [2]: [web_site_sk#23, web_company_name#24] -Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri)) AND isnotnull(web_site_sk#23)) +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)) (39) Project [codegen id : 11] -Output [1]: [web_site_sk#23] -Input [2]: [web_site_sk#23, web_company_name#24] +Output [1]: [web_site_sk#24] +Input [2]: [web_site_sk#24, web_company_name#25] (40) BroadcastExchange -Input [1]: [web_site_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] +Input [1]: [web_site_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#23] +Right keys [1]: [web_site_sk#24] Join condition: None (42) Project [codegen id : 12] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#23] +Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#24] (43) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] -Results [3]: [ws_order_number#5, sum#28, sum#29] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28] +Results [3]: [ws_order_number#5, sum#29, sum#30] (44) Exchange -Input [3]: [ws_order_number#5, sum#28, sum#29] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [ws_order_number#5, sum#29, sum#30] +Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#31] (45) HashAggregate [codegen id : 13] -Input [3]: [ws_order_number#5, sum#28, sum#29] +Input [3]: [ws_order_number#5, sum#29, sum#30] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] -Results [3]: [ws_order_number#5, sum#28, sum#29] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28] +Results [3]: [ws_order_number#5, sum#29, sum#30] (46) HashAggregate [codegen id : 13] -Input [3]: [ws_order_number#5, sum#28, sum#29] +Input [3]: [ws_order_number#5, sum#29, sum#30] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] -Results [3]: [sum#28, sum#29, count#32] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28, count(ws_order_number#5)#32] +Results [3]: [sum#29, sum#30, count#33] (47) Exchange -Input [3]: [sum#28, sum#29, count#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [sum#29, sum#30, count#33] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#34] (48) HashAggregate [codegen id : 14] -Input [3]: [sum#28, sum#29, count#32] +Input [3]: [sum#29, sum#30, count#33] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] -Results [3]: [count(ws_order_number#5)#31 AS order count #34, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#26,17,2) AS total shipping cost #35, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#27,17,2) AS total net profit #36] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28, count(ws_order_number#5)#32] +Results [3]: [count(ws_order_number#5)#32 AS order count #35, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#27,17,2) AS total shipping cost #36, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#28,17,2) AS total net profit #37] (49) Sort [codegen id : 14] -Input [3]: [order count #34, total shipping cost #35, total net profit #36] -Arguments: [order count #34 ASC NULLS FIRST], true, 0 +Input [3]: [order count #35, total shipping cost #36, total net profit #37] +Arguments: [order count #35 ASC NULLS FIRST], true, 0 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 725939ef24f1b..ee82855f8c010 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 @@ -92,261 +92,261 @@ Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (7) Scan parquet default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] (9) Filter [codegen id : 3] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] -Condition : (isnotnull(ws_order_number#4) AND isnotnull(ws_warehouse_sk#9)) +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] +Condition : (isnotnull(ws_order_number#10) AND isnotnull(ws_warehouse_sk#9)) (10) Project [codegen id : 3] -Output [2]: [ws_warehouse_sk#9, ws_order_number#4] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Output [2]: [ws_warehouse_sk#9, ws_order_number#10] +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] (11) Exchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#10] +Input [2]: [ws_warehouse_sk#9, ws_order_number#10] +Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, [id=#12] (12) Sort [codegen id : 4] -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: [ws_order_number#4 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 (13) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] +Output [2]: [ws_warehouse_sk#13, ws_order_number#14] (14) Sort [codegen id : 6] -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#12 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 (15) SortMergeJoin [codegen id : 7] -Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#12] -Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#11) +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) (16) Project [codegen id : 7] -Output [1]: [ws_order_number#4 AS ws_order_number#4#13] -Input [4]: [ws_warehouse_sk#9, ws_order_number#4, ws_warehouse_sk#11, ws_order_number#12] +Output [1]: [ws_order_number#10] +Input [4]: [ws_warehouse_sk#9, ws_order_number#10, ws_warehouse_sk#13, ws_order_number#14] (17) SortMergeJoin Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#4#13] +Right keys [1]: [ws_order_number#10] Join condition: None (18) Exchange Input [6]: [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] -Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#15] (19) Sort [codegen id : 8] Input [6]: [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] Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 (20) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#9, ws_order_number#4] +Output [2]: [ws_warehouse_sk#16, ws_order_number#17] (21) Sort [codegen id : 10] -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#16, ws_order_number#17] +Arguments: [ws_order_number#17 ASC NULLS FIRST], false, 0 (22) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#15, ws_order_number#16] +Output [2]: [ws_warehouse_sk#18, ws_order_number#19] (23) Sort [codegen id : 12] -Input [2]: [ws_warehouse_sk#15, ws_order_number#16] -Arguments: [ws_order_number#16 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#18, ws_order_number#19] +Arguments: [ws_order_number#19 ASC NULLS FIRST], false, 0 (24) SortMergeJoin [codegen id : 13] -Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#16] -Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#15) +Left keys [1]: [ws_order_number#17] +Right keys [1]: [ws_order_number#19] +Join condition: NOT (ws_warehouse_sk#16 = ws_warehouse_sk#18) (25) Project [codegen id : 13] -Output [1]: [ws_order_number#4] -Input [4]: [ws_warehouse_sk#9, ws_order_number#4, ws_warehouse_sk#15, ws_order_number#16] +Output [1]: [ws_order_number#17] +Input [4]: [ws_warehouse_sk#16, ws_order_number#17, ws_warehouse_sk#18, ws_order_number#19] (26) Exchange -Input [1]: [ws_order_number#4] -Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#17] +Input [1]: [ws_order_number#17] +Arguments: hashpartitioning(cast(ws_order_number#17 as bigint), 5), ENSURE_REQUIREMENTS, [id=#20] (27) Sort [codegen id : 14] -Input [1]: [ws_order_number#4] -Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 +Input [1]: [ws_order_number#17] +Arguments: [cast(ws_order_number#17 as bigint) ASC NULLS FIRST], false, 0 (28) Scan parquet default.web_returns -Output [2]: [wr_order_number#18, wr_returned_date_sk#19] +Output [2]: [wr_order_number#21, wr_returned_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct (29) ColumnarToRow [codegen id : 15] -Input [2]: [wr_order_number#18, wr_returned_date_sk#19] +Input [2]: [wr_order_number#21, wr_returned_date_sk#22] (30) Filter [codegen id : 15] -Input [2]: [wr_order_number#18, wr_returned_date_sk#19] -Condition : isnotnull(wr_order_number#18) +Input [2]: [wr_order_number#21, wr_returned_date_sk#22] +Condition : isnotnull(wr_order_number#21) (31) Project [codegen id : 15] -Output [1]: [wr_order_number#18] -Input [2]: [wr_order_number#18, wr_returned_date_sk#19] +Output [1]: [wr_order_number#21] +Input [2]: [wr_order_number#21, wr_returned_date_sk#22] (32) Exchange -Input [1]: [wr_order_number#18] -Arguments: hashpartitioning(wr_order_number#18, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [wr_order_number#21] +Arguments: hashpartitioning(wr_order_number#21, 5), ENSURE_REQUIREMENTS, [id=#23] (33) Sort [codegen id : 16] -Input [1]: [wr_order_number#18] -Arguments: [wr_order_number#18 ASC NULLS FIRST], false, 0 +Input [1]: [wr_order_number#21] +Arguments: [wr_order_number#21 ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 17] -Left keys [1]: [cast(ws_order_number#4 as bigint)] -Right keys [1]: [wr_order_number#18] +Left keys [1]: [cast(ws_order_number#17 as bigint)] +Right keys [1]: [wr_order_number#21] Join condition: None (35) Project [codegen id : 17] -Output [1]: [wr_order_number#18] -Input [2]: [ws_order_number#4, wr_order_number#18] +Output [1]: [wr_order_number#21] +Input [2]: [ws_order_number#17, wr_order_number#21] (36) SortMergeJoin Left keys [1]: [cast(ws_order_number#4 as bigint)] -Right keys [1]: [wr_order_number#18] +Right keys [1]: [wr_order_number#21] Join condition: None (37) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] +Output [2]: [ca_address_sk#24, ca_state#25] 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 (38) ColumnarToRow [codegen id : 18] -Input [2]: [ca_address_sk#21, ca_state#22] +Input [2]: [ca_address_sk#24, ca_state#25] (39) Filter [codegen id : 18] -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)) +Input [2]: [ca_address_sk#24, ca_state#25] +Condition : ((isnotnull(ca_state#25) AND (ca_state#25 = IL)) AND isnotnull(ca_address_sk#24)) (40) Project [codegen id : 18] -Output [1]: [ca_address_sk#21] -Input [2]: [ca_address_sk#21, ca_state#22] +Output [1]: [ca_address_sk#24] +Input [2]: [ca_address_sk#24, ca_state#25] (41) BroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [ca_address_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (42) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#24] Join condition: None (43) Project [codegen id : 21] 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#21] +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#24] (44) Scan parquet default.web_site -Output [2]: [web_site_sk#24, web_company_name#25] +Output [2]: [web_site_sk#27, web_company_name#28] 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)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct (45) ColumnarToRow [codegen id : 19] -Input [2]: [web_site_sk#24, web_company_name#25] +Input [2]: [web_site_sk#27, web_company_name#28] (46) Filter [codegen id : 19] -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)) +Input [2]: [web_site_sk#27, web_company_name#28] +Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri )) AND isnotnull(web_site_sk#27)) (47) Project [codegen id : 19] -Output [1]: [web_site_sk#24] -Input [2]: [web_site_sk#24, web_company_name#25] +Output [1]: [web_site_sk#27] +Input [2]: [web_site_sk#27, web_company_name#28] (48) BroadcastExchange -Input [1]: [web_site_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [web_site_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (49) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#24] +Right keys [1]: [web_site_sk#27] Join condition: None (50) Project [codegen id : 21] 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#24] +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#27] (51) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_date#28] +Output [2]: [d_date_sk#30, d_date#31] 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 (52) ColumnarToRow [codegen id : 20] -Input [2]: [d_date_sk#27, d_date#28] +Input [2]: [d_date_sk#30, d_date#31] (53) Filter [codegen id : 20] -Input [2]: [d_date_sk#27, d_date#28] -Condition : (((isnotnull(d_date#28) AND (d_date#28 >= 10623)) AND (d_date#28 <= 10683)) AND isnotnull(d_date_sk#27)) +Input [2]: [d_date_sk#30, d_date#31] +Condition : (((isnotnull(d_date#31) AND (d_date#31 >= 1999-02-01)) AND (d_date#31 <= 1999-04-02)) AND isnotnull(d_date_sk#30)) (54) Project [codegen id : 20] -Output [1]: [d_date_sk#27] -Input [2]: [d_date_sk#27, d_date#28] +Output [1]: [d_date_sk#30] +Input [2]: [d_date_sk#30, d_date#31] (55) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] +Input [1]: [d_date_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] (56) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#30] Join condition: None (57) Project [codegen id : 21] 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#27] +Input [5]: [ws_ship_date_sk#1, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#30] (58) HashAggregate [codegen id : 21] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31] -Results [3]: [ws_order_number#4, sum#32, sum#33] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34] +Results [3]: [ws_order_number#4, sum#35, sum#36] (59) Exchange -Input [3]: [ws_order_number#4, sum#32, sum#33] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [ws_order_number#4, sum#35, sum#36] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#37] (60) HashAggregate [codegen id : 22] -Input [3]: [ws_order_number#4, sum#32, sum#33] +Input [3]: [ws_order_number#4, sum#35, sum#36] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31] -Results [3]: [ws_order_number#4, sum#32, sum#33] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34] +Results [3]: [ws_order_number#4, sum#35, sum#36] (61) HashAggregate [codegen id : 22] -Input [3]: [ws_order_number#4, sum#32, sum#33] +Input [3]: [ws_order_number#4, sum#35, sum#36] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31, count(ws_order_number#4)#35] -Results [3]: [sum#32, sum#33, count#36] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34, count(ws_order_number#4)#38] +Results [3]: [sum#35, sum#36, count#39] (62) Exchange -Input [3]: [sum#32, sum#33, count#36] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [sum#35, sum#36, count#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] (63) HashAggregate [codegen id : 23] -Input [3]: [sum#32, sum#33, count#36] +Input [3]: [sum#35, sum#36, count#39] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31, 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))#30,17,2) AS total shipping cost #39, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#31,17,2) AS total net profit #40] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34, count(ws_order_number#4)#38] +Results [3]: [count(ws_order_number#4)#38 AS order count #41, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#33,17,2) AS total shipping cost #42, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#34,17,2) AS total net profit #43] (64) Sort [codegen id : 23] -Input [3]: [order count #38, total shipping cost #39, total net profit #40] -Arguments: [order count #38 ASC NULLS FIRST], true, 0 +Input [3]: [order count #41, total shipping cost #42, total net profit #43] +Arguments: [order count #41 ASC NULLS FIRST], true, 0 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 9f15375b5cfc0..73f7f88aeb68d 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 @@ -92,261 +92,261 @@ Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (7) Scan parquet default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] (9) Filter [codegen id : 3] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] -Condition : (isnotnull(ws_order_number#4) AND isnotnull(ws_warehouse_sk#9)) +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] +Condition : (isnotnull(ws_order_number#10) AND isnotnull(ws_warehouse_sk#9)) (10) Project [codegen id : 3] -Output [2]: [ws_warehouse_sk#9, ws_order_number#4] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Output [2]: [ws_warehouse_sk#9, ws_order_number#10] +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] (11) Exchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#10] +Input [2]: [ws_warehouse_sk#9, ws_order_number#10] +Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, [id=#12] (12) Sort [codegen id : 4] -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: [ws_order_number#4 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 (13) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] +Output [2]: [ws_warehouse_sk#13, ws_order_number#14] (14) Sort [codegen id : 6] -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#12 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 (15) SortMergeJoin [codegen id : 7] -Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#12] -Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#11) +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) (16) Project [codegen id : 7] -Output [1]: [ws_order_number#4 AS ws_order_number#4#13] -Input [4]: [ws_warehouse_sk#9, ws_order_number#4, ws_warehouse_sk#11, ws_order_number#12] +Output [1]: [ws_order_number#10] +Input [4]: [ws_warehouse_sk#9, ws_order_number#10, ws_warehouse_sk#13, ws_order_number#14] (17) SortMergeJoin Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#4#13] +Right keys [1]: [ws_order_number#10] Join condition: None (18) Exchange Input [6]: [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] -Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#15] (19) Sort [codegen id : 8] Input [6]: [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] Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 (20) Scan parquet default.web_returns -Output [2]: [wr_order_number#15, wr_returned_date_sk#16] +Output [2]: [wr_order_number#16, wr_returned_date_sk#17] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct (21) ColumnarToRow [codegen id : 9] -Input [2]: [wr_order_number#15, wr_returned_date_sk#16] +Input [2]: [wr_order_number#16, wr_returned_date_sk#17] (22) Filter [codegen id : 9] -Input [2]: [wr_order_number#15, wr_returned_date_sk#16] -Condition : isnotnull(wr_order_number#15) +Input [2]: [wr_order_number#16, wr_returned_date_sk#17] +Condition : isnotnull(wr_order_number#16) (23) Project [codegen id : 9] -Output [1]: [wr_order_number#15] -Input [2]: [wr_order_number#15, wr_returned_date_sk#16] +Output [1]: [wr_order_number#16] +Input [2]: [wr_order_number#16, wr_returned_date_sk#17] (24) Exchange -Input [1]: [wr_order_number#15] -Arguments: hashpartitioning(wr_order_number#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [1]: [wr_order_number#16] +Arguments: hashpartitioning(wr_order_number#16, 5), ENSURE_REQUIREMENTS, [id=#18] (25) Sort [codegen id : 10] -Input [1]: [wr_order_number#15] -Arguments: [wr_order_number#15 ASC NULLS FIRST], false, 0 +Input [1]: [wr_order_number#16] +Arguments: [wr_order_number#16 ASC NULLS FIRST], false, 0 (26) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#9, ws_order_number#4] +Output [2]: [ws_warehouse_sk#19, ws_order_number#20] (27) Sort [codegen id : 12] -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#19, ws_order_number#20] +Arguments: [ws_order_number#20 ASC NULLS FIRST], false, 0 (28) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#18, ws_order_number#19] +Output [2]: [ws_warehouse_sk#21, ws_order_number#22] (29) Sort [codegen id : 14] -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#21, ws_order_number#22] +Arguments: [ws_order_number#22 ASC NULLS FIRST], false, 0 (30) SortMergeJoin [codegen id : 15] -Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#19] -Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#18) +Left keys [1]: [ws_order_number#20] +Right keys [1]: [ws_order_number#22] +Join condition: NOT (ws_warehouse_sk#19 = ws_warehouse_sk#21) (31) Project [codegen id : 15] -Output [1]: [ws_order_number#4] -Input [4]: [ws_warehouse_sk#9, ws_order_number#4, ws_warehouse_sk#18, ws_order_number#19] +Output [1]: [ws_order_number#20] +Input [4]: [ws_warehouse_sk#19, ws_order_number#20, ws_warehouse_sk#21, ws_order_number#22] (32) Exchange -Input [1]: [ws_order_number#4] -Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [ws_order_number#20] +Arguments: hashpartitioning(cast(ws_order_number#20 as bigint), 5), ENSURE_REQUIREMENTS, [id=#23] (33) Sort [codegen id : 16] -Input [1]: [ws_order_number#4] -Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 +Input [1]: [ws_order_number#20] +Arguments: [cast(ws_order_number#20 as bigint) ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 17] -Left keys [1]: [wr_order_number#15] -Right keys [1]: [cast(ws_order_number#4 as bigint)] +Left keys [1]: [wr_order_number#16] +Right keys [1]: [cast(ws_order_number#20 as bigint)] Join condition: None (35) Project [codegen id : 17] -Output [1]: [wr_order_number#15] -Input [2]: [wr_order_number#15, ws_order_number#4] +Output [1]: [wr_order_number#16] +Input [2]: [wr_order_number#16, ws_order_number#20] (36) SortMergeJoin Left keys [1]: [cast(ws_order_number#4 as bigint)] -Right keys [1]: [wr_order_number#15] +Right keys [1]: [wr_order_number#16] Join condition: None (37) Scan parquet default.date_dim -Output [2]: [d_date_sk#21, d_date#22] +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 (38) ColumnarToRow [codegen id : 18] -Input [2]: [d_date_sk#21, d_date#22] +Input [2]: [d_date_sk#24, d_date#25] (39) Filter [codegen id : 18] -Input [2]: [d_date_sk#21, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 10623)) AND (d_date#22 <= 10683)) AND isnotnull(d_date_sk#21)) +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)) (40) Project [codegen id : 18] -Output [1]: [d_date_sk#21] -Input [2]: [d_date_sk#21, d_date#22] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_date#25] (41) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (42) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#21] +Right keys [1]: [d_date_sk#24] Join condition: None (43) Project [codegen id : 21] 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#21] +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#24] (44) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#27, ca_state#28] 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 (45) ColumnarToRow [codegen id : 19] -Input [2]: [ca_address_sk#24, ca_state#25] +Input [2]: [ca_address_sk#27, ca_state#28] (46) Filter [codegen id : 19] -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : ((isnotnull(ca_state#25) AND (ca_state#25 = IL)) AND isnotnull(ca_address_sk#24)) +Input [2]: [ca_address_sk#27, ca_state#28] +Condition : ((isnotnull(ca_state#28) AND (ca_state#28 = IL)) AND isnotnull(ca_address_sk#27)) (47) Project [codegen id : 19] -Output [1]: [ca_address_sk#24] -Input [2]: [ca_address_sk#24, ca_state#25] +Output [1]: [ca_address_sk#27] +Input [2]: [ca_address_sk#27, ca_state#28] (48) BroadcastExchange -Input [1]: [ca_address_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [ca_address_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (49) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#24] +Right keys [1]: [ca_address_sk#27] Join condition: None (50) Project [codegen id : 21] 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#24] +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#27] (51) Scan parquet default.web_site -Output [2]: [web_site_sk#27, web_company_name#28] +Output [2]: [web_site_sk#30, web_company_name#31] 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)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 20] -Input [2]: [web_site_sk#27, web_company_name#28] +Input [2]: [web_site_sk#30, web_company_name#31] (53) Filter [codegen id : 20] -Input [2]: [web_site_sk#27, web_company_name#28] -Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri)) AND isnotnull(web_site_sk#27)) +Input [2]: [web_site_sk#30, web_company_name#31] +Condition : ((isnotnull(web_company_name#31) AND (web_company_name#31 = pri )) AND isnotnull(web_site_sk#30)) (54) Project [codegen id : 20] -Output [1]: [web_site_sk#27] -Input [2]: [web_site_sk#27, web_company_name#28] +Output [1]: [web_site_sk#30] +Input [2]: [web_site_sk#30, web_company_name#31] (55) BroadcastExchange -Input [1]: [web_site_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] +Input [1]: [web_site_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] (56) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#27] +Right keys [1]: [web_site_sk#30] Join condition: None (57) Project [codegen id : 21] 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#27] +Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#30] (58) HashAggregate [codegen id : 21] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31] -Results [3]: [ws_order_number#4, sum#32, sum#33] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34] +Results [3]: [ws_order_number#4, sum#35, sum#36] (59) Exchange -Input [3]: [ws_order_number#4, sum#32, sum#33] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [ws_order_number#4, sum#35, sum#36] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#37] (60) HashAggregate [codegen id : 22] -Input [3]: [ws_order_number#4, sum#32, sum#33] +Input [3]: [ws_order_number#4, sum#35, sum#36] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31] -Results [3]: [ws_order_number#4, sum#32, sum#33] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34] +Results [3]: [ws_order_number#4, sum#35, sum#36] (61) HashAggregate [codegen id : 22] -Input [3]: [ws_order_number#4, sum#32, sum#33] +Input [3]: [ws_order_number#4, sum#35, sum#36] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31, count(ws_order_number#4)#35] -Results [3]: [sum#32, sum#33, count#36] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34, count(ws_order_number#4)#38] +Results [3]: [sum#35, sum#36, count#39] (62) Exchange -Input [3]: [sum#32, sum#33, count#36] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [sum#35, sum#36, count#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] (63) HashAggregate [codegen id : 23] -Input [3]: [sum#32, sum#33, count#36] +Input [3]: [sum#35, sum#36, count#39] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31, 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))#30,17,2) AS total shipping cost #39, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#31,17,2) AS total net profit #40] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34, count(ws_order_number#4)#38] +Results [3]: [count(ws_order_number#4)#38 AS order count #41, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#33,17,2) AS total shipping cost #42, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#34,17,2) AS total net profit #43] (64) Sort [codegen id : 23] -Input [3]: [order count #38, total shipping cost #39, total net profit #40] -Arguments: [order count #38 ASC NULLS FIRST], true, 0 +Input [3]: [order count #41, total shipping cost #42, total net profit #43] +Arguments: [order count #41 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt index 245357ef5560c..c01f3465ed693 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt @@ -102,16 +102,16 @@ ReadSchema: struct Input [3]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13] (16) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#14] (17) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#5] +Right keys [1]: [d_date_sk#14] Join condition: None (18) Project [codegen id : 5] Output [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#5] +Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#14] (19) HashAggregate [codegen id : 5] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] @@ -122,45 +122,45 @@ Results [2]: [cs_bill_customer_sk#11, cs_item_sk#12] (20) Exchange Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#15] (21) HashAggregate [codegen id : 6] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Keys [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Functions: [] Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#11 AS customer_sk#15, cs_item_sk#12 AS item_sk#16] +Results [2]: [cs_bill_customer_sk#11 AS customer_sk#16, cs_item_sk#12 AS item_sk#17] (22) Sort [codegen id : 6] -Input [2]: [customer_sk#15, item_sk#16] -Arguments: [customer_sk#15 ASC NULLS FIRST, item_sk#16 ASC NULLS FIRST], false, 0 +Input [2]: [customer_sk#16, item_sk#17] +Arguments: [customer_sk#16 ASC NULLS FIRST, item_sk#17 ASC NULLS FIRST], false, 0 (23) SortMergeJoin Left keys [2]: [customer_sk#9, item_sk#10] -Right keys [2]: [customer_sk#15, item_sk#16] +Right keys [2]: [customer_sk#16, item_sk#17] Join condition: None (24) Project [codegen id : 7] -Output [2]: [customer_sk#9, customer_sk#15] -Input [4]: [customer_sk#9, item_sk#10, customer_sk#15, item_sk#16] +Output [2]: [customer_sk#9, customer_sk#16] +Input [4]: [customer_sk#9, item_sk#10, customer_sk#16, item_sk#17] (25) HashAggregate [codegen id : 7] -Input [2]: [customer_sk#9, customer_sk#15] +Input [2]: [customer_sk#9, customer_sk#16] Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum#17, sum#18, sum#19] -Results [3]: [sum#20, sum#21, sum#22] +Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum#18, sum#19, sum#20] +Results [3]: [sum#21, sum#22, sum#23] (26) Exchange -Input [3]: [sum#20, sum#21, sum#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [sum#21, sum#22, sum#23] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] (27) HashAggregate [codegen id : 8] -Input [3]: [sum#20, sum#21, sum#22] +Input [3]: [sum#21, sum#22, sum#23] Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26] -Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24 AS store_only#27, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25 AS catalog_only#28, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26 AS store_and_catalog#29] +Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27] +Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25 AS store_only#28, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26 AS catalog_only#29, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27 AS store_and_catalog#30] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt index 245357ef5560c..c01f3465ed693 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt @@ -102,16 +102,16 @@ ReadSchema: struct Input [3]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13] (16) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#14] (17) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#5] +Right keys [1]: [d_date_sk#14] Join condition: None (18) Project [codegen id : 5] Output [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#5] +Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#14] (19) HashAggregate [codegen id : 5] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] @@ -122,45 +122,45 @@ Results [2]: [cs_bill_customer_sk#11, cs_item_sk#12] (20) Exchange Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#15] (21) HashAggregate [codegen id : 6] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Keys [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Functions: [] Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#11 AS customer_sk#15, cs_item_sk#12 AS item_sk#16] +Results [2]: [cs_bill_customer_sk#11 AS customer_sk#16, cs_item_sk#12 AS item_sk#17] (22) Sort [codegen id : 6] -Input [2]: [customer_sk#15, item_sk#16] -Arguments: [customer_sk#15 ASC NULLS FIRST, item_sk#16 ASC NULLS FIRST], false, 0 +Input [2]: [customer_sk#16, item_sk#17] +Arguments: [customer_sk#16 ASC NULLS FIRST, item_sk#17 ASC NULLS FIRST], false, 0 (23) SortMergeJoin Left keys [2]: [customer_sk#9, item_sk#10] -Right keys [2]: [customer_sk#15, item_sk#16] +Right keys [2]: [customer_sk#16, item_sk#17] Join condition: None (24) Project [codegen id : 7] -Output [2]: [customer_sk#9, customer_sk#15] -Input [4]: [customer_sk#9, item_sk#10, customer_sk#15, item_sk#16] +Output [2]: [customer_sk#9, customer_sk#16] +Input [4]: [customer_sk#9, item_sk#10, customer_sk#16, item_sk#17] (25) HashAggregate [codegen id : 7] -Input [2]: [customer_sk#9, customer_sk#15] +Input [2]: [customer_sk#9, customer_sk#16] Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum#17, sum#18, sum#19] -Results [3]: [sum#20, sum#21, sum#22] +Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum#18, sum#19, sum#20] +Results [3]: [sum#21, sum#22, sum#23] (26) Exchange -Input [3]: [sum#20, sum#21, sum#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [sum#21, sum#22, sum#23] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] (27) HashAggregate [codegen id : 8] -Input [3]: [sum#20, sum#21, sum#22] +Input [3]: [sum#21, sum#22, sum#23] Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26] -Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24 AS store_only#27, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25 AS catalog_only#28, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26 AS store_and_catalog#29] +Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27] +Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25 AS store_only#28, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26 AS catalog_only#29, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27 AS store_and_catalog#30] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt index 7b8e4ff5516a7..98c1bc5671cd9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt @@ -57,7 +57,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -88,7 +88,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -96,7 +96,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt index 3457f398b7dea..1507b5ccbc0ae 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt @@ -46,7 +46,7 @@ Condition : isnotnull(ss_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -54,7 +54,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -81,7 +81,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt index 58d28b28dae10..2e710b7eda4b3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt @@ -137,52 +137,52 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#15] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (22) Project [codegen id : 7] -Output [1]: [ws_bill_customer_sk#13 AS customer_sk#15] -Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] +Output [1]: [ws_bill_customer_sk#13 AS customer_sk#16] +Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] (23) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] ReadSchema: struct (24) ColumnarToRow [codegen id : 9] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (25) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#19] (26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (27) Project [codegen id : 9] -Output [1]: [cs_ship_customer_sk#16 AS customer_sk#18] -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] +Output [1]: [cs_ship_customer_sk#17 AS customer_sk#20] +Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] (28) Union (29) Exchange -Input [1]: [customer_sk#15] -Arguments: hashpartitioning(customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#19] +Input [1]: [customer_sk#16] +Arguments: hashpartitioning(customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#21] (30) Sort [codegen id : 10] -Input [1]: [customer_sk#15] -Arguments: [customer_sk#15 ASC NULLS FIRST], false, 0 +Input [1]: [customer_sk#16] +Arguments: [customer_sk#16 ASC NULLS FIRST], false, 0 (31) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#15] +Right keys [1]: [customer_sk#16] Join condition: None (32) Project [codegen id : 12] @@ -190,84 +190,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (33) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] +Output [2]: [ca_address_sk#22, ca_county#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -Input [2]: [ca_address_sk#20, ca_county#21] +Input [2]: [ca_address_sk#22, ca_county#23] (35) Filter [codegen id : 11] -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) +Input [2]: [ca_address_sk#22, ca_county#23] +Condition : (ca_county#23 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#22)) (36) Project [codegen id : 11] -Output [1]: [ca_address_sk#20] -Input [2]: [ca_address_sk#20, ca_county#21] +Output [1]: [ca_address_sk#22] +Input [2]: [ca_address_sk#22, ca_county#23] (37) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [ca_address_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (38) BroadcastHashJoin [codegen id : 12] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#22] Join condition: None (39) Project [codegen id : 12] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#22] (40) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (41) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Output [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (42) ColumnarToRow -Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] (43) Filter -Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Condition : isnotnull(cd_demo_sk#24) +Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Condition : isnotnull(cd_demo_sk#26) (44) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#24] +Right keys [1]: [cd_demo_sk#26] Join condition: None (45) Project [codegen id : 13] -Output [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Output [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] (46) HashAggregate [codegen id : 13] -Input [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] +Aggregate Attributes [1]: [count#35] +Results [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] (47) Exchange -Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] -Arguments: hashpartitioning(cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] +Arguments: hashpartitioning(cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, 5), ENSURE_REQUIREMENTS, [id=#37] (48) HashAggregate [codegen id : 14] -Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] -Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] +Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#36] -Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count(1)#36 AS cnt1#37, cd_purchase_estimate#28, count(1)#36 AS cnt2#38, cd_credit_rating#29, count(1)#36 AS cnt3#39, cd_dep_count#30, count(1)#36 AS cnt4#40, cd_dep_employed_count#31, count(1)#36 AS cnt5#41, cd_dep_college_count#32, count(1)#36 AS cnt6#42] +Aggregate Attributes [1]: [count(1)#38] +Results [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, count(1)#38 AS cnt1#39, cd_purchase_estimate#30, count(1)#38 AS cnt2#40, cd_credit_rating#31, count(1)#38 AS cnt3#41, cd_dep_count#32, count(1)#38 AS cnt4#42, cd_dep_employed_count#33, count(1)#38 AS cnt5#43, cd_dep_college_count#34, count(1)#38 AS cnt6#44] (49) TakeOrderedAndProject -Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] -Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] +Input [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#39, cd_purchase_estimate#30, cnt2#40, cd_credit_rating#31, cnt3#41, cd_dep_count#32, cnt4#42, cd_dep_employed_count#33, cnt5#43, cd_dep_college_count#34, cnt6#44] +Arguments: 100, [cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_education_status#29 ASC NULLS FIRST, cd_purchase_estimate#30 ASC NULLS FIRST, cd_credit_rating#31 ASC NULLS FIRST, cd_dep_count#32 ASC NULLS FIRST, cd_dep_employed_count#33 ASC NULLS FIRST, cd_dep_college_count#34 ASC NULLS FIRST], [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#39, cd_purchase_estimate#30, cnt2#40, cd_credit_rating#31, cnt3#41, cd_dep_count#32, cnt4#42, cd_dep_employed_count#33, cnt5#43, cd_dep_college_count#34, cnt6#44] ===== Subqueries ===== @@ -280,6 +280,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index eac6acc3543ca..62054fe96dd5c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -121,48 +121,48 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#12 AS customer_sk#14] -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] +Output [1]: [ws_bill_customer_sk#12 AS customer_sk#15] +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] (20) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#6)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] (22) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#18] (23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#18] Join condition: None (24) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#15 AS customer_sk#17] -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] +Output [1]: [cs_ship_customer_sk#16 AS customer_sk#19] +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#18] (25) Union (26) BroadcastExchange -Input [1]: [customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Input [1]: [customer_sk#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (27) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#14] +Right keys [1]: [customer_sk#15] Join condition: None (28) Project [codegen id : 9] @@ -170,84 +170,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#19, ca_county#20] +Output [2]: [ca_address_sk#21, ca_county#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#19, ca_county#20] +Input [2]: [ca_address_sk#21, ca_county#22] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#19, ca_county#20] -Condition : (ca_county#20 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#19)) +Input [2]: [ca_address_sk#21, ca_county#22] +Condition : (ca_county#22 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#21)) (32) Project [codegen id : 7] -Output [1]: [ca_address_sk#19] -Input [2]: [ca_address_sk#19, ca_county#20] +Output [1]: [ca_address_sk#21] +Input [2]: [ca_address_sk#21, ca_county#22] (33) BroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [ca_address_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (34) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#19] +Right keys [1]: [ca_address_sk#21] Join condition: None (35) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21] (36) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] (38) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#22) +Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Condition : isnotnull(cd_demo_sk#24) (39) BroadcastExchange -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#22] +Right keys [1]: [cd_demo_sk#24] Join condition: None (41) Project [codegen id : 9] -Output [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] (42) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#32] -Results [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] +Aggregate Attributes [1]: [count#34] +Results [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#35] (43) Exchange -Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] -Arguments: hashpartitioning(cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#35] +Arguments: hashpartitioning(cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#36] (44) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] -Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#35] +Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(1)#35 AS cnt1#36, cd_purchase_estimate#26, count(1)#35 AS cnt2#37, cd_credit_rating#27, count(1)#35 AS cnt3#38, cd_dep_count#28, count(1)#35 AS cnt4#39, cd_dep_employed_count#29, count(1)#35 AS cnt5#40, cd_dep_college_count#30, count(1)#35 AS cnt6#41] +Aggregate Attributes [1]: [count(1)#37] +Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count(1)#37 AS cnt1#38, cd_purchase_estimate#28, count(1)#37 AS cnt2#39, cd_credit_rating#29, count(1)#37 AS cnt3#40, cd_dep_count#30, count(1)#37 AS cnt4#41, cd_dep_employed_count#31, count(1)#37 AS cnt5#42, cd_dep_college_count#32, count(1)#37 AS cnt6#43] (45) TakeOrderedAndProject -Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] -Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] +Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#38, cd_purchase_estimate#28, cnt2#39, cd_credit_rating#29, cnt3#40, cd_dep_count#30, cnt4#41, cd_dep_employed_count#31, cnt5#42, cd_dep_college_count#32, cnt6#43] +Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#38, cd_purchase_estimate#28, cnt2#39, cd_credit_rating#29, cnt3#40, cd_dep_count#30, cnt4#41, cd_dep_employed_count#31, cnt5#42, cd_dep_college_count#32, cnt6#43] ===== Subqueries ===== @@ -260,6 +260,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#6 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 6eb3be35a05bc..453a35d8d5175 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 @@ -199,285 +199,285 @@ Input [2]: [customer_id#23, year_total#24] Arguments: [customer_id#23 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] (27) Filter [codegen id : 10] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_customer_sk#1) +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] +Condition : isnotnull(ss_customer_sk#26) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#31, d_year#32] 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 (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#6, d_year#7] +Input [2]: [d_date_sk#31, d_year#32] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#31, d_year#32] +Condition : ((isnotnull(d_year#32) AND (d_year#32 = 2002)) AND isnotnull(d_date_sk#31)) (31) BroadcastExchange -Input [2]: [d_date_sk#6, d_year#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [2]: [d_date_sk#31, d_year#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ss_sold_date_sk#29] +Right keys [1]: [d_date_sk#31] Join condition: None (33) Project [codegen id : 10] -Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4, d_date_sk#6, d_year#7] +Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Input [6]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29, d_date_sk#31, d_year#32] (34) Exchange -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#34] (35) Sort [codegen id : 11] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] (37) Sort [codegen id : 13] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Arguments: [c_customer_sk#35 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#35] Join condition: None (39) Project [codegen id : 14] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Input [12]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Input [12]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32, c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] (40) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#29] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] +Input [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#43] +Results [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] (41) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] +Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, 5), ENSURE_REQUIREMENTS, [id=#45] (42) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32] -Results [5]: [c_customer_id#11 AS customer_id#33, c_first_name#12 AS customer_first_name#34, c_last_name#13 AS customer_last_name#35, c_email_address#17 AS customer_email_address#36, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32,18,2) AS year_total#37] +Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] +Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46] +Results [5]: [c_customer_id#36 AS customer_id#47, c_first_name#37 AS customer_first_name#48, c_last_name#38 AS customer_last_name#49, c_email_address#42 AS customer_email_address#50, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46,18,2) AS year_total#51] (43) Exchange -Input [5]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37] -Arguments: hashpartitioning(customer_id#33, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] +Arguments: hashpartitioning(customer_id#47, 5), ENSURE_REQUIREMENTS, [id=#52] (44) Sort [codegen id : 16] -Input [5]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37] -Arguments: [customer_id#33 ASC NULLS FIRST], false, 0 +Input [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] +Arguments: [customer_id#47 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#33] +Right keys [1]: [customer_id#47] Join condition: None (46) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] +Output [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#56), dynamicpruningexpression(ws_sold_date_sk#56 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] +Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56] (48) Filter [codegen id : 19] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] -Condition : isnotnull(ws_bill_customer_sk#39) +Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56] +Condition : isnotnull(ws_bill_customer_sk#53) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#57, d_year#58] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#56] +Right keys [1]: [d_date_sk#57] Join condition: None (51) Project [codegen id : 19] -Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Input [6]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42, d_date_sk#6, d_year#7] +Output [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] +Input [6]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56, d_date_sk#57, d_year#58] (52) Exchange -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Arguments: hashpartitioning(ws_bill_customer_sk#39, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] +Arguments: hashpartitioning(ws_bill_customer_sk#53, 5), ENSURE_REQUIREMENTS, [id=#59] (53) Sort [codegen id : 20] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Arguments: [ws_bill_customer_sk#39 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] +Arguments: [ws_bill_customer_sk#53 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +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] (55) Sort [codegen id : 22] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +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] +Arguments: [c_customer_sk#60 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#39] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ws_bill_customer_sk#53] +Right keys [1]: [c_customer_sk#60] Join condition: None (57) Project [codegen id : 23] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Input [12]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [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, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] +Input [12]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58, 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] (58) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#44] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#45] +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, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] +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#58] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#68] +Results [9]: [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#58, sum#69] (59) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#45] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [9]: [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#58, sum#69] +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#58, 5), ENSURE_REQUIREMENTS, [id=#70] (60) HashAggregate [codegen id : 24] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#45] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#47] -Results [2]: [c_customer_id#11 AS customer_id#48, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#47,18,2) AS year_total#49] +Input [9]: [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#58, sum#69] +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#58] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))#71] +Results [2]: [c_customer_id#61 AS customer_id#72, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))#71,18,2) AS year_total#73] (61) Filter [codegen id : 24] -Input [2]: [customer_id#48, year_total#49] -Condition : (isnotnull(year_total#49) AND (year_total#49 > 0.00)) +Input [2]: [customer_id#72, year_total#73] +Condition : (isnotnull(year_total#73) AND (year_total#73 > 0.00)) (62) Project [codegen id : 24] -Output [2]: [customer_id#48 AS customer_id#50, year_total#49 AS year_total#51] -Input [2]: [customer_id#48, year_total#49] +Output [2]: [customer_id#72 AS customer_id#74, year_total#73 AS year_total#75] +Input [2]: [customer_id#72, year_total#73] (63) Exchange -Input [2]: [customer_id#50, year_total#51] -Arguments: hashpartitioning(customer_id#50, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [2]: [customer_id#74, year_total#75] +Arguments: hashpartitioning(customer_id#74, 5), ENSURE_REQUIREMENTS, [id=#76] (64) Sort [codegen id : 25] -Input [2]: [customer_id#50, year_total#51] -Arguments: [customer_id#50 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#74, year_total#75] +Arguments: [customer_id#74 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#50] +Right keys [1]: [customer_id#74] Join condition: None (66) Project [codegen id : 26] -Output [8]: [customer_id#23, year_total#24, customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37, year_total#51] -Input [9]: [customer_id#23, year_total#24, customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37, customer_id#50, year_total#51] +Output [8]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75] +Input [9]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, customer_id#74, year_total#75] (67) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] +Output [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] +Input [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80] (69) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] -Condition : isnotnull(ws_bill_customer_sk#39) +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) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#81, d_year#82] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#80] +Right keys [1]: [d_date_sk#81] Join condition: None (72) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Input [6]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42, d_date_sk#6, d_year#7] +Output [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] +Input [6]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80, d_date_sk#81, d_year#82] (73) Exchange -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Arguments: hashpartitioning(ws_bill_customer_sk#39, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] +Arguments: hashpartitioning(ws_bill_customer_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] (74) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Arguments: [ws_bill_customer_sk#39 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] +Arguments: [ws_bill_customer_sk#77 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [8]: [c_customer_sk#84, c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91] (76) Sort [codegen id : 31] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#84, c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91] +Arguments: [c_customer_sk#84 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#39] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ws_bill_customer_sk#77] +Right keys [1]: [c_customer_sk#84] Join condition: None (78) Project [codegen id : 32] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Input [12]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [10]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] +Input [12]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82, c_customer_sk#84, c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91] (79) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#54] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#55] +Input [10]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] +Keys [8]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82] +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#92] +Results [9]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, sum#93] (80) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#55] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [9]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, sum#93] +Arguments: hashpartitioning(c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, 5), ENSURE_REQUIREMENTS, [id=#94] (81) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#55] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#57] -Results [2]: [c_customer_id#11 AS customer_id#58, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#57,18,2) AS year_total#59] +Input [9]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, sum#93] +Keys [8]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82] +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)))#95] +Results [2]: [c_customer_id#85 AS customer_id#96, 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)))#95,18,2) AS year_total#97] (82) Exchange -Input [2]: [customer_id#58, year_total#59] -Arguments: hashpartitioning(customer_id#58, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [2]: [customer_id#96, year_total#97] +Arguments: hashpartitioning(customer_id#96, 5), ENSURE_REQUIREMENTS, [id=#98] (83) Sort [codegen id : 34] -Input [2]: [customer_id#58, year_total#59] -Arguments: [customer_id#58 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#96, year_total#97] +Arguments: [customer_id#96 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#58] -Join condition: (CASE WHEN (year_total#51 > 0.00) THEN CheckOverflow((promote_precision(year_total#59) / promote_precision(year_total#51)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#37) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE 0E-20 END) +Right keys [1]: [customer_id#96] +Join condition: (CASE WHEN (year_total#75 > 0.00) THEN CheckOverflow((promote_precision(year_total#97) / promote_precision(year_total#75)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#51) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE 0E-20 END) (85) Project [codegen id : 35] -Output [4]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36] -Input [10]: [customer_id#23, year_total#24, customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37, year_total#51, customer_id#58, year_total#59] +Output [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] +Input [10]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75, customer_id#96, year_total#97] (86) TakeOrderedAndProject -Input [4]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36] -Arguments: 100, [customer_id#33 ASC NULLS FIRST, customer_first_name#34 ASC NULLS FIRST, customer_last_name#35 ASC NULLS FIRST, customer_email_address#36 ASC NULLS FIRST], [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36] +Input [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] +Arguments: 100, [customer_id#47 ASC NULLS FIRST, customer_first_name#48 ASC NULLS FIRST, customer_last_name#49 ASC NULLS FIRST, customer_email_address#50 ASC NULLS FIRST], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] ===== Subqueries ===== @@ -488,15 +488,15 @@ ReusedExchange (87) (87) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#6, d_year#7] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#26 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#30 ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#31, d_year#32] -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#56 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#26 +Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#30 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 8f7b2faaa1220..17d16033bb5be 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 @@ -171,282 +171,282 @@ Input [2]: [customer_id#22, year_total#23] Condition : (isnotnull(year_total#23) AND (year_total#23 > 0.00)) (20) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] (22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_customer_id#25)) (23) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Output [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ss_sold_date_sk#35), dynamicpruningexpression(ss_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] (25) Filter [codegen id : 4] -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_customer_sk#9) +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Condition : isnotnull(ss_customer_sk#32) (26) BroadcastExchange -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#9] +Left keys [1]: [c_customer_sk#24] +Right keys [1]: [ss_customer_sk#32] Join condition: None (28) Project [codegen id : 6] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Input [12]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#38, d_year#39] 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 (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#15, d_year#16] +Input [2]: [d_date_sk#38, d_year#39] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#15, d_year#16] -Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2002)) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#38, d_year#39] +Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) (32) BroadcastExchange -Input [2]: [d_date_sk#15, d_year#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +Input [2]: [d_date_sk#38, d_year#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#35] +Right keys [1]: [d_date_sk#38] Join condition: None (34) Project [codegen id : 6] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#15, d_year#16] +Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] +Input [12]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#38, d_year#39] (35) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#27] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] +Input [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] +Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#41] +Results [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] (36) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] +Arguments: hashpartitioning(c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, 5), ENSURE_REQUIREMENTS, [id=#43] (37) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30] -Results [5]: [c_customer_id#2 AS customer_id#31, c_first_name#3 AS customer_first_name#32, c_last_name#4 AS customer_last_name#33, c_email_address#8 AS customer_email_address#34, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30,18,2) AS year_total#35] +Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] +Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44] +Results [5]: [c_customer_id#25 AS customer_id#45, c_first_name#26 AS customer_first_name#46, c_last_name#27 AS customer_last_name#47, c_email_address#31 AS customer_email_address#48, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44,18,2) AS year_total#49] (38) BroadcastExchange -Input [5]: [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#36] +Input [5]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#31] +Right keys [1]: [customer_id#45] Join condition: None (40) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] (42) Filter [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] +Condition : (isnotnull(c_customer_sk#51) AND isnotnull(c_customer_id#52)) (43) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Input [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] (45) Filter [codegen id : 8] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_customer_sk#37) +Input [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] +Condition : isnotnull(ws_bill_customer_sk#59) (46) BroadcastExchange -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#41] +Input [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#37] +Left keys [1]: [c_customer_sk#51] +Right keys [1]: [ws_bill_customer_sk#59] Join condition: None (48) Project [codegen id : 10] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] +Input [12]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#64, d_year#65] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ws_sold_date_sk#62] +Right keys [1]: [d_date_sk#64] Join condition: None (51) Project [codegen id : 10] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#15, d_year#16] +Output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, d_year#65] +Input [12]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62, d_date_sk#64, d_year#65] (52) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#42] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#43] +Input [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, d_year#65] +Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#66] +Results [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, sum#67] (53) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#43] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, sum#67] +Arguments: hashpartitioning(c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#68] (54) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#43] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45] -Results [2]: [c_customer_id#2 AS customer_id#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45,18,2) AS year_total#47] +Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, sum#67] +Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))#69] +Results [2]: [c_customer_id#52 AS customer_id#70, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))#69,18,2) AS year_total#71] (55) Filter [codegen id : 11] -Input [2]: [customer_id#46, year_total#47] -Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) +Input [2]: [customer_id#70, year_total#71] +Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.00)) (56) Project [codegen id : 11] -Output [2]: [customer_id#46 AS customer_id#48, year_total#47 AS year_total#49] -Input [2]: [customer_id#46, year_total#47] +Output [2]: [customer_id#70 AS customer_id#72, year_total#71 AS year_total#73] +Input [2]: [customer_id#70, year_total#71] (57) BroadcastExchange -Input [2]: [customer_id#48, year_total#49] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] +Input [2]: [customer_id#72, year_total#73] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#74] (58) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#48] +Right keys [1]: [customer_id#72] Join condition: None (59) Project [codegen id : 16] -Output [8]: [customer_id#22, year_total#23, customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35, year_total#49] -Input [9]: [customer_id#22, year_total#23, customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35, customer_id#48, year_total#49] +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#73] +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#72, year_total#73] (60) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] (62) Filter [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] +Condition : (isnotnull(c_customer_sk#75) AND isnotnull(c_customer_id#76)) (63) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] (65) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_customer_sk#37) +Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Condition : isnotnull(ws_bill_customer_sk#83) (66) BroadcastExchange -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] +Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#87] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#37] +Left keys [1]: [c_customer_sk#75] +Right keys [1]: [ws_bill_customer_sk#83] Join condition: None (68) Project [codegen id : 14] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [10]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Input [12]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#88, d_year#89] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ws_sold_date_sk#86] +Right keys [1]: [d_date_sk#88] Join condition: None (71) Project [codegen id : 14] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#15, d_year#16] +Output [10]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] +Input [12]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86, d_date_sk#88, d_year#89] (72) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#52] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#53] +Input [10]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] +Keys [8]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#90] +Results [9]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, sum#91] (73) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#53] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [9]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, sum#91] +Arguments: hashpartitioning(c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, 5), ENSURE_REQUIREMENTS, [id=#92] (74) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#53] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55] -Results [2]: [c_customer_id#2 AS customer_id#56, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55,18,2) AS year_total#57] +Input [9]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, sum#91] +Keys [8]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))#93] +Results [2]: [c_customer_id#76 AS customer_id#94, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))#93,18,2) AS year_total#95] (75) BroadcastExchange -Input [2]: [customer_id#56, year_total#57] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#58] +Input [2]: [customer_id#94, year_total#95] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#96] (76) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#56] -Join condition: (CASE WHEN (year_total#49 > 0.00) THEN CheckOverflow((promote_precision(year_total#57) / promote_precision(year_total#49)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#35) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE 0E-20 END) +Right keys [1]: [customer_id#94] +Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#95) / promote_precision(year_total#73)), 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) (77) Project [codegen id : 16] -Output [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34] -Input [10]: [customer_id#22, year_total#23, customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35, year_total#49, customer_id#56, year_total#57] +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#73, customer_id#94, year_total#95] (78) TakeOrderedAndProject -Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34] -Arguments: 100, [customer_id#31 ASC NULLS FIRST, customer_first_name#32 ASC NULLS FIRST, customer_last_name#33 ASC NULLS FIRST, customer_email_address#34 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34] +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] ===== Subqueries ===== @@ -457,15 +457,15 @@ ReusedExchange (79) (79) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#15, d_year#16] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#24 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#38, d_year#39] -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#24 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#36 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt index 15c445b6b1325..dd85594160c8f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt @@ -55,7 +55,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -86,7 +86,7 @@ Arguments: [ws_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -94,7 +94,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt index 774081bc5c3f6..f6dd412f5c4b1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt @@ -44,7 +44,7 @@ Condition : isnotnull(ws_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -52,7 +52,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -79,7 +79,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt index b19d4f3aeae0f..25baf22f2b4b5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt @@ -155,499 +155,499 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (10) ColumnarToRow [codegen id : 10] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] (11) Filter [codegen id : 10] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_item_sk#11) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#14, d_year#15] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#14, d_year#15] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1998)) AND (d_year#15 <= 2000)) AND isnotnull(d_date_sk#14)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#14] +Input [2]: [d_date_sk#14, d_year#15] (16) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (17) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#14] Join condition: None (18) Project [codegen id : 10] -Output [1]: [ss_item_sk#1] -Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] +Output [1]: [ss_item_sk#11] +Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] (19) Scan parquet default.item -Output [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Condition : (((isnotnull(i_item_sk#15) AND isnotnull(i_brand_id#16)) AND isnotnull(i_class_id#17)) AND isnotnull(i_category_id#18)) +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) (22) Exchange -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: hashpartitioning(coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] (23) Sort [codegen id : 5] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: [coalesce(i_brand_id#16, 0) ASC NULLS FIRST, isnull(i_brand_id#16) ASC NULLS FIRST, coalesce(i_class_id#17, 0) ASC NULLS FIRST, isnull(i_class_id#17) ASC NULLS FIRST, coalesce(i_category_id#18, 0) ASC NULLS FIRST, isnull(i_category_id#18) ASC NULLS FIRST], false, 0 +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 (24) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] +Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 8] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] (26) Filter [codegen id : 8] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#22) (27) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#24] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None (29) Project [codegen id : 8] -Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#22] +Output [1]: [cs_item_sk#22] +Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] (30) Scan parquet default.item -Output [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] (32) Filter [codegen id : 7] -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -Condition : isnotnull(i_item_sk#23) +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Condition : isnotnull(i_item_sk#25) (33) BroadcastExchange -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#23] +Left keys [1]: [cs_item_sk#22] +Right keys [1]: [i_item_sk#25] Join condition: None (35) Project [codegen id : 8] -Output [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Input [5]: [cs_item_sk#20, i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] (36) Exchange -Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: hashpartitioning(coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26), 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] (37) Sort [codegen id : 9] -Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: [coalesce(i_brand_id#24, 0) ASC NULLS FIRST, isnull(i_brand_id#24) ASC NULLS FIRST, coalesce(i_class_id#25, 0) ASC NULLS FIRST, isnull(i_class_id#25) ASC NULLS FIRST, coalesce(i_category_id#26, 0) ASC NULLS FIRST, isnull(i_category_id#26) ASC NULLS FIRST], false, 0 +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 (38) SortMergeJoin -Left keys [6]: [coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18)] -Right keys [6]: [coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26)] +Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] Join condition: None (39) BroadcastExchange -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] +Left keys [1]: [ss_item_sk#11] +Right keys [1]: [i_item_sk#17] Join condition: None (41) Project [codegen id : 10] -Output [3]: [i_brand_id#16 AS brand_id#30, i_class_id#17 AS class_id#31, i_category_id#18 AS category_id#32] -Input [5]: [ss_item_sk#1, i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] +Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (42) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#32, class_id#33, category_id#34] (43) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] (44) HashAggregate [codegen id : 11] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#32, class_id#33, category_id#34] (45) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32), 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] (46) Sort [codegen id : 12] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: [coalesce(brand_id#30, 0) ASC NULLS FIRST, isnull(brand_id#30) ASC NULLS FIRST, coalesce(class_id#31, 0) ASC NULLS FIRST, isnull(class_id#31) ASC NULLS FIRST, coalesce(category_id#32, 0) ASC NULLS FIRST, isnull(category_id#32) ASC NULLS FIRST], false, 0 +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 15] -Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] (49) Filter [codegen id : 15] -Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#35) +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_item_sk#37) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#37] +Output [1]: [d_date_sk#39] (51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#37] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#39] Join condition: None (52) Project [codegen id : 15] -Output [1]: [ws_item_sk#35] -Input [3]: [ws_item_sk#35, ws_sold_date_sk#36, d_date_sk#37] +Output [1]: [ws_item_sk#37] +Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] (53) ReusedExchange [Reuses operator id: 33] -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] (54) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [i_item_sk#38] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#40] Join condition: None (55) Project [codegen id : 15] -Output [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Input [5]: [ws_item_sk#35, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] (56) Exchange -Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: hashpartitioning(coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41), 5), ENSURE_REQUIREMENTS, [id=#42] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] (57) Sort [codegen id : 16] -Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: [coalesce(i_brand_id#39, 0) ASC NULLS FIRST, isnull(i_brand_id#39) ASC NULLS FIRST, coalesce(i_class_id#40, 0) ASC NULLS FIRST, isnull(i_class_id#40) ASC NULLS FIRST, coalesce(i_category_id#41, 0) ASC NULLS FIRST, isnull(i_category_id#41) ASC NULLS FIRST], false, 0 +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] -Right keys [6]: [coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41)] +Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] +Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] Join condition: None (59) HashAggregate [codegen id : 17] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#32, class_id#33, category_id#34] (60) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] (61) HashAggregate [codegen id : 18] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#32, class_id#33, category_id#34] (62) BroadcastExchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#44] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] (63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#30, class_id#31, category_id#32] +Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join condition: None (64) Project [codegen id : 19] -Output [1]: [i_item_sk#7 AS ss_item_sk#45] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#30, class_id#31, category_id#32] +Output [1]: [i_item_sk#7 AS ss_item_sk#47] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] (65) Exchange -Input [1]: [ss_item_sk#45] -Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [1]: [ss_item_sk#47] +Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] (66) Sort [codegen id : 20] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#47] Join condition: None (68) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#47] +Output [2]: [d_date_sk#49, d_week_seq#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 21] -Input [2]: [d_date_sk#12, d_week_seq#47] +Input [2]: [d_date_sk#49, d_week_seq#50] (70) Filter [codegen id : 21] -Input [2]: [d_date_sk#12, d_week_seq#47] -Condition : ((isnotnull(d_week_seq#47) AND (d_week_seq#47 = Subquery scalar-subquery#48, [id=#49])) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#49, d_week_seq#50] +Condition : ((isnotnull(d_week_seq#50) AND (d_week_seq#50 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#49)) (71) Project [codegen id : 21] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#47] +Output [1]: [d_date_sk#49] +Input [2]: [d_date_sk#49, d_week_seq#50] (72) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] +Input [1]: [d_date_sk#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] (73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#49] Join condition: None (74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] (75) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (76) ColumnarToRow [codegen id : 22] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] (77) Filter [codegen id : 22] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Condition : (((isnotnull(i_item_sk#54) AND isnotnull(i_brand_id#55)) AND isnotnull(i_class_id#56)) AND isnotnull(i_category_id#57)) (78) Exchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: hashpartitioning(i_item_sk#54, 5), ENSURE_REQUIREMENTS, [id=#58] (79) Sort [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: [i_item_sk#54 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#47] (81) Sort [codegen id : 41] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#45] +Left keys [1]: [i_item_sk#54] +Right keys [1]: [ss_item_sk#47] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#59] (84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#54] Join condition: None (85) Project [codegen id : 42] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] (86) HashAggregate [codegen id : 42] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] +Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#53, isEmpty#54, count#55] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] +Aggregate Attributes [3]: [sum#60, isEmpty#61, count#62] +Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] +Arguments: hashpartitioning(i_brand_id#55, i_class_id#56, i_category_id#57, 5), ENSURE_REQUIREMENTS, [id=#66] (88) HashAggregate [codegen id : 86] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] +Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60, count(1)#61] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sales#62, count(1)#61 AS number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67, count(1)#68] +Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#69, count(1)#68 AS number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] (89) Filter [codegen id : 86] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 as decimal(32,6)) > cast(Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(32,6)) > cast(Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) (90) Project [codegen id : 86] -Output [6]: [store AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] +Output [6]: [store AS channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] (91) Scan parquet default.store_sales -Output [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Output [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#71), dynamicpruningexpression(ss_sold_date_sk#71 IN dynamicpruning#72)] +PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 43] -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] (93) Filter [codegen id : 43] -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] -Condition : isnotnull(ss_item_sk#68) +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Condition : isnotnull(ss_item_sk#75) (94) Exchange -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] -Arguments: hashpartitioning(ss_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Arguments: hashpartitioning(ss_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#80] (95) Sort [codegen id : 44] -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] -Arguments: [ss_item_sk#68 ASC NULLS FIRST], false, 0 +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Arguments: [ss_item_sk#75 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#47] (97) Sort [codegen id : 62] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ss_item_sk#68] -Right keys [1]: [ss_item_sk#45] +Left keys [1]: [ss_item_sk#75] +Right keys [1]: [ss_item_sk#47] Join condition: None (99) Scan parquet default.date_dim -Output [2]: [d_date_sk#74, d_week_seq#75] +Output [2]: [d_date_sk#81, d_week_seq#82] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (100) ColumnarToRow [codegen id : 63] -Input [2]: [d_date_sk#74, d_week_seq#75] +Input [2]: [d_date_sk#81, d_week_seq#82] (101) Filter [codegen id : 63] -Input [2]: [d_date_sk#74, d_week_seq#75] -Condition : ((isnotnull(d_week_seq#75) AND (d_week_seq#75 = Subquery scalar-subquery#76, [id=#77])) AND isnotnull(d_date_sk#74)) +Input [2]: [d_date_sk#81, d_week_seq#82] +Condition : ((isnotnull(d_week_seq#82) AND (d_week_seq#82 = Subquery scalar-subquery#83, [id=#84])) AND isnotnull(d_date_sk#81)) (102) Project [codegen id : 63] -Output [1]: [d_date_sk#74] -Input [2]: [d_date_sk#74, d_week_seq#75] +Output [1]: [d_date_sk#81] +Input [2]: [d_date_sk#81, d_week_seq#82] (103) BroadcastExchange -Input [1]: [d_date_sk#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] +Input [1]: [d_date_sk#81] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#85] (104) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_sold_date_sk#71] -Right keys [1]: [d_date_sk#74] +Left keys [1]: [ss_sold_date_sk#78] +Right keys [1]: [d_date_sk#81] Join condition: None (105) Project [codegen id : 84] -Output [3]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70] -Input [5]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71, d_date_sk#74] +Output [3]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77] +Input [5]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#81] (106) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] +Output [4]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] (107) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#68] -Right keys [1]: [i_item_sk#79] +Left keys [1]: [ss_item_sk#75] +Right keys [1]: [i_item_sk#86] Join condition: None (108) Project [codegen id : 84] -Output [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#80, i_class_id#81, i_category_id#82] -Input [7]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] +Output [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] +Input [7]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] (109) HashAggregate [codegen id : 84] -Input [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#80, i_class_id#81, i_category_id#82] -Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] -Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] +Input [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] +Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] +Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] (110) Exchange -Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] -Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] +Arguments: hashpartitioning(i_brand_id#87, i_class_id#88, i_category_id#89, 5), ENSURE_REQUIREMENTS, [id=#96] (111) HashAggregate [codegen id : 85] -Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] -Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90, count(1)#91] -Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sales#92, count(1)#91 AS number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] +Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] +Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#99, count(1)#98 AS number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] (112) Filter [codegen id : 85] -Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) (113) Project [codegen id : 85] -Output [6]: [store AS channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] -Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] +Output [6]: [store AS channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] (114) BroadcastExchange -Input [6]: [channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#96] +Input [6]: [channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#103] (115) BroadcastHashJoin [codegen id : 86] -Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] +Left keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] +Right keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] Join condition: None (116) TakeOrderedAndProject -Input [12]: [channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] -Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Input [12]: [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Arguments: 100, [i_brand_id#55 ASC NULLS FIRST, i_class_id#56 ASC NULLS FIRST, i_category_id#57 ASC NULLS FIRST], [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#65, [id=#66] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#72, [id=#73] * HashAggregate (139) +- Exchange (138) +- * HashAggregate (137) @@ -674,140 +674,140 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (117) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#97)] +PartitionFilters: [isnotnull(ss_sold_date_sk#106), dynamicpruningexpression(ss_sold_date_sk#106 IN dynamicpruning#107)] ReadSchema: struct (118) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] (119) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#108, d_year#109] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (120) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#108, d_year#109] (121) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#108, d_year#109] +Condition : (((isnotnull(d_year#109) AND (d_year#109 >= 1998)) AND (d_year#109 <= 2000)) AND isnotnull(d_date_sk#108)) (122) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#108] +Input [2]: [d_date_sk#108, d_year#109] (123) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] +Input [1]: [d_date_sk#108] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#110] (124) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#106] +Right keys [1]: [d_date_sk#108] Join condition: None (125) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#99, ss_list_price#3 AS list_price#100] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#104 AS quantity#111, ss_list_price#105 AS list_price#112] +Input [4]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106, d_date_sk#108] (126) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21] +Output [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#97)] +PartitionFilters: [isnotnull(cs_sold_date_sk#115), dynamicpruningexpression(cs_sold_date_sk#115 IN dynamicpruning#107)] ReadSchema: struct (127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21] +Input [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] (128) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#103] +Output [1]: [d_date_sk#116] (129) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#103] +Left keys [1]: [cs_sold_date_sk#115] +Right keys [1]: [d_date_sk#116] Join condition: None (130) Project [codegen id : 4] -Output [2]: [cs_quantity#101 AS quantity#104, cs_list_price#102 AS list_price#105] -Input [4]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21, d_date_sk#103] +Output [2]: [cs_quantity#113 AS quantity#117, cs_list_price#114 AS list_price#118] +Input [4]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115, d_date_sk#116] (131) Scan parquet default.web_sales -Output [3]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36] +Output [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#97)] +PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#107)] ReadSchema: struct (132) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36] +Input [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] (133) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#108] +Output [1]: [d_date_sk#122] (134) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#108] +Left keys [1]: [ws_sold_date_sk#121] +Right keys [1]: [d_date_sk#122] Join condition: None (135) Project [codegen id : 6] -Output [2]: [ws_quantity#106 AS quantity#109, ws_list_price#107 AS list_price#110] -Input [4]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36, d_date_sk#108] +Output [2]: [ws_quantity#119 AS quantity#123, ws_list_price#120 AS list_price#124] +Input [4]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121, d_date_sk#122] (136) Union (137) HashAggregate [codegen id : 7] -Input [2]: [quantity#99, list_price#100] +Input [2]: [quantity#111, list_price#112] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#111, count#112] -Results [2]: [sum#113, count#114] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#125, count#126] +Results [2]: [sum#127, count#128] (138) Exchange -Input [2]: [sum#113, count#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] +Input [2]: [sum#127, count#128] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] (139) HashAggregate [codegen id : 8] -Input [2]: [sum#113, count#114] +Input [2]: [sum#127, count#128] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#116] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#116 AS average_sales#117] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130 AS average_sales#131] -Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#97 +Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#106 IN dynamicpruning#107 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#108] -Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#97 +Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#115 IN dynamicpruning#107 -Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#97 +Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#107 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (141) (141) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#49] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 ReusedExchange (142) (142) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#14] -Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 -Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#51, [id=#52] * Project (146) +- * Filter (145) +- * ColumnarToRow (144) @@ -815,33 +815,33 @@ Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquer (143) Scan parquet default.date_dim -Output [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Output [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] (145) Filter [codegen id : 1] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#13 = 1999)) AND (d_moy#118 = 12)) AND (d_dom#119 = 16)) +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Condition : (((((isnotnull(d_year#133) AND isnotnull(d_moy#134)) AND isnotnull(d_dom#135)) AND (d_year#133 = 1999)) AND (d_moy#134 = 12)) AND (d_dom#135 = 16)) (146) Project [codegen id : 1] -Output [1]: [d_week_seq#47] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Output [1]: [d_week_seq#132] +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] -Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#65, [id=#66] +Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] -Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#71 IN dynamicpruning#72 +Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#79 ReusedExchange (147) (147) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#74] +Output [1]: [d_date_sk#81] -Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#76, [id=#77] +Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#83, [id=#84] * Project (151) +- * Filter (150) +- * ColumnarToRow (149) @@ -849,21 +849,21 @@ Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subqu (148) Scan parquet default.date_dim -Output [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Output [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (149) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] (150) Filter [codegen id : 1] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#13 = 1998)) AND (d_moy#118 = 12)) AND (d_dom#119 = 16)) +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Condition : (((((isnotnull(d_year#137) AND isnotnull(d_moy#138)) AND isnotnull(d_dom#139)) AND (d_year#137 = 1998)) AND (d_moy#138 = 12)) AND (d_dom#139 = 16)) (151) Project [codegen id : 1] -Output [1]: [d_week_seq#47] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Output [1]: [d_week_seq#136] +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 4c62de561c263..bdafb17d69d45 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -131,443 +131,443 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_item_sk#10) (10) Scan parquet default.item -Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (11) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] (12) Filter [codegen id : 4] -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14)) +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) (13) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] (15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#15) +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Condition : isnotnull(cs_item_sk#17) (16) Scan parquet default.item -Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] (18) Filter [codegen id : 1] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Condition : isnotnull(i_item_sk#17) +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Condition : isnotnull(i_item_sk#19) (19) BroadcastExchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] (20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#15] -Right keys [1]: [i_item_sk#17] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#19] Join condition: None (21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20] -Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] (22) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_year#23] +Output [2]: [d_date_sk#24, d_year#25] 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 (23) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#22, d_year#23] +Input [2]: [d_date_sk#24, d_year#25] (24) Filter [codegen id : 2] -Input [2]: [d_date_sk#22, d_year#23] -Condition : (((isnotnull(d_year#23) AND (d_year#23 >= 1998)) AND (d_year#23 <= 2000)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) (25) Project [codegen id : 2] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_year#23] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_year#25] (26) 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#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (27) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] Join condition: None (28) Project [codegen id : 3] -Output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] -Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#22] +Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] (29) BroadcastExchange -Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] (30) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)] -Right keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join condition: None (31) BroadcastExchange -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] (32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#11] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] Join condition: None (33) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] (34) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#29] (35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#29] Join condition: None (36) Project [codegen id : 6] -Output [3]: [i_brand_id#12 AS brand_id#28, i_class_id#13 AS class_id#29, i_category_id#14 AS category_id#30] -Input [5]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#27] +Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#30, class_id#31, category_id#32] (38) Exchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#30, class_id#31, category_id#32] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_item_sk#32) +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_item_sk#34) (43) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#34] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#36] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37] -Input [6]: [ws_item_sk#32, ws_sold_date_sk#33, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] +Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] (46) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#38] +Output [1]: [d_date_sk#40] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#40] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] -Input [5]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] +Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] (49) BroadcastExchange -Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#39] +Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] -Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#30, class_id#31, category_id#32] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#30, class_id#31, category_id#32] (53) BroadcastExchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#40] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#28, class_id#29, category_id#30] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#41] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] +Output [1]: [i_item_sk#6 AS ss_item_sk#43] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] (56) BroadcastExchange -Input [1]: [ss_item_sk#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [ss_item_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#41] +Right keys [1]: [ss_item_sk#43] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Condition : (((isnotnull(i_item_sk#45) AND isnotnull(i_brand_id#46)) AND isnotnull(i_class_id#47)) AND isnotnull(i_category_id#48)) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#43] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#41] +Left keys [1]: [i_item_sk#45] +Right keys [1]: [ss_item_sk#43] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#45] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (66) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_week_seq#44] +Output [2]: [d_date_sk#50, d_week_seq#51] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [2]: [d_date_sk#27, d_week_seq#44] +Input [2]: [d_date_sk#50, d_week_seq#51] (68) Filter [codegen id : 24] -Input [2]: [d_date_sk#27, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#27)) +Input [2]: [d_date_sk#50, d_week_seq#51] +Condition : ((isnotnull(d_week_seq#51) AND (d_week_seq#51 = Subquery scalar-subquery#52, [id=#53])) AND isnotnull(d_date_sk#50)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#27] -Input [2]: [d_date_sk#27, d_week_seq#44] +Output [1]: [d_date_sk#50] +Input [2]: [d_date_sk#50, d_week_seq#51] (70) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] +Input [1]: [d_date_sk#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#50] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#27] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#48, isEmpty#49, count#50] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] +Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] +Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#61] (75) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55, count(1)#56] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sales#57, count(1)#56 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] (76) Filter [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59 as decimal(32,6)) > cast(Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) (77) Project [codegen id : 52] -Output [6]: [store AS channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] +Output [6]: [store AS channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] (78) Scan parquet default.store_sales -Output [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] +Output [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] +PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] +Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] (80) Filter [codegen id : 50] -Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] -Condition : isnotnull(ss_item_sk#63) +Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +Condition : isnotnull(ss_item_sk#70) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#43] (82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#63] -Right keys [1]: [ss_item_sk#41] +Left keys [1]: [ss_item_sk#70] +Right keys [1]: [ss_item_sk#43] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] +Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] (84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#63] -Right keys [1]: [i_item_sk#68] +Left keys [1]: [ss_item_sk#70] +Right keys [1]: [i_item_sk#75] Join condition: None (85) Project [codegen id : 50] -Output [6]: [ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_brand_id#69, i_class_id#70, i_category_id#71] -Input [8]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] +Output [6]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] +Input [8]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#72, d_week_seq#73] +Output [2]: [d_date_sk#79, d_week_seq#80] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#72, d_week_seq#73] +Input [2]: [d_date_sk#79, d_week_seq#80] (88) Filter [codegen id : 49] -Input [2]: [d_date_sk#72, d_week_seq#73] -Condition : ((isnotnull(d_week_seq#73) AND (d_week_seq#73 = Subquery scalar-subquery#74, [id=#75])) AND isnotnull(d_date_sk#72)) +Input [2]: [d_date_sk#79, d_week_seq#80] +Condition : ((isnotnull(d_week_seq#80) AND (d_week_seq#80 = Subquery scalar-subquery#81, [id=#82])) AND isnotnull(d_date_sk#79)) (89) Project [codegen id : 49] -Output [1]: [d_date_sk#72] -Input [2]: [d_date_sk#72, d_week_seq#73] +Output [1]: [d_date_sk#79] +Input [2]: [d_date_sk#79, d_week_seq#80] (90) BroadcastExchange -Input [1]: [d_date_sk#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] +Input [1]: [d_date_sk#79] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#83] (91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#66] -Right keys [1]: [d_date_sk#72] +Left keys [1]: [ss_sold_date_sk#73] +Right keys [1]: [d_date_sk#79] Join condition: None (92) Project [codegen id : 50] -Output [5]: [ss_quantity#64, ss_list_price#65, i_brand_id#69, i_class_id#70, i_category_id#71] -Input [7]: [ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_brand_id#69, i_class_id#70, i_category_id#71, d_date_sk#72] +Output [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] +Input [7]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78, d_date_sk#79] (93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#64, ss_list_price#65, i_brand_id#69, i_class_id#70, i_category_id#71] -Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#77, isEmpty#78, count#79] -Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] +Input [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] +Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] +Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] (94) Exchange -Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] -Arguments: hashpartitioning(i_brand_id#69, i_class_id#70, i_category_id#71, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] +Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, [id=#90] (95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] -Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84, count(1)#85] -Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sales#86, count(1)#85 AS number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] +Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] +Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] (96) Filter [codegen id : 51] -Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) (97) Project [codegen id : 51] -Output [6]: [store AS channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] -Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] +Output [6]: [store AS channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] (98) BroadcastExchange -Input [6]: [channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#90] +Input [6]: [channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#97] (99) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] +Left keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Right keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] Join condition: None (100) TakeOrderedAndProject -Input [12]: [channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] -Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] +Input [12]: [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Arguments: 100, [i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#60, [id=#61] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#67, [id=#68] * HashAggregate (123) +- Exchange (122) +- * HashAggregate (121) @@ -594,140 +594,140 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#91)] +PartitionFilters: [isnotnull(ss_sold_date_sk#100), dynamicpruningexpression(ss_sold_date_sk#100 IN dynamicpruning#101)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] (103) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_year#92] +Output [2]: [d_date_sk#102, d_year#103] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#92] +Input [2]: [d_date_sk#102, d_year#103] (105) Filter [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#92] -Condition : (((isnotnull(d_year#92) AND (d_year#92 >= 1998)) AND (d_year#92 <= 2000)) AND isnotnull(d_date_sk#27)) +Input [2]: [d_date_sk#102, d_year#103] +Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 <= 2000)) AND isnotnull(d_date_sk#102)) (106) Project [codegen id : 1] -Output [1]: [d_date_sk#27] -Input [2]: [d_date_sk#27, d_year#92] +Output [1]: [d_date_sk#102] +Input [2]: [d_date_sk#102, d_year#103] (107) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] +Input [1]: [d_date_sk#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#104] (108) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Left keys [1]: [ss_sold_date_sk#100] +Right keys [1]: [d_date_sk#102] Join condition: None (109) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#94, ss_list_price#3 AS list_price#95] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#27] +Output [2]: [ss_quantity#98 AS quantity#105, ss_list_price#99 AS list_price#106] +Input [4]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100, d_date_sk#102] (110) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16] +Output [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#91)] +PartitionFilters: [isnotnull(cs_sold_date_sk#109), dynamicpruningexpression(cs_sold_date_sk#109 IN dynamicpruning#101)] ReadSchema: struct (111) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16] +Input [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] (112) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#98] +Output [1]: [d_date_sk#110] (113) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#98] +Left keys [1]: [cs_sold_date_sk#109] +Right keys [1]: [d_date_sk#110] Join condition: None (114) Project [codegen id : 4] -Output [2]: [cs_quantity#96 AS quantity#99, cs_list_price#97 AS list_price#100] -Input [4]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16, d_date_sk#98] +Output [2]: [cs_quantity#107 AS quantity#111, cs_list_price#108 AS list_price#112] +Input [4]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109, d_date_sk#110] (115) Scan parquet default.web_sales -Output [3]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33] +Output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#91)] +PartitionFilters: [isnotnull(ws_sold_date_sk#115), dynamicpruningexpression(ws_sold_date_sk#115 IN dynamicpruning#101)] ReadSchema: struct (116) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33] +Input [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] (117) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#103] +Output [1]: [d_date_sk#116] (118) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#103] +Left keys [1]: [ws_sold_date_sk#115] +Right keys [1]: [d_date_sk#116] Join condition: None (119) Project [codegen id : 6] -Output [2]: [ws_quantity#101 AS quantity#104, ws_list_price#102 AS list_price#105] -Input [4]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33, d_date_sk#103] +Output [2]: [ws_quantity#113 AS quantity#117, ws_list_price#114 AS list_price#118] +Input [4]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115, d_date_sk#116] (120) Union (121) HashAggregate [codegen id : 7] -Input [2]: [quantity#94, list_price#95] +Input [2]: [quantity#105, list_price#106] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#106, count#107] -Results [2]: [sum#108, count#109] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#119, count#120] +Results [2]: [sum#121, count#122] (122) Exchange -Input [2]: [sum#108, count#109] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] +Input [2]: [sum#121, count#122] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#123] (123) HashAggregate [codegen id : 8] -Input [2]: [sum#108, count#109] +Input [2]: [sum#121, count#122] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))#111] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))#111 AS average_sales#112] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124 AS average_sales#125] -Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#91 +Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#100 IN dynamicpruning#101 ReusedExchange (124) (124) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#102] -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#91 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#109 IN dynamicpruning#101 -Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#91 +Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#115 IN dynamicpruning#101 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#50] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 ReusedExchange (126) (126) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#29] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] * Project (130) +- * Filter (129) +- * ColumnarToRow (128) @@ -735,33 +735,33 @@ Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer (127) Scan parquet default.date_dim -Output [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Output [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (128) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] (129) Filter [codegen id : 1] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] -Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#92 = 1999)) AND (d_moy#113 = 12)) AND (d_dom#114 = 16)) +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Condition : (((((isnotnull(d_year#127) AND isnotnull(d_moy#128)) AND isnotnull(d_dom#129)) AND (d_year#127 = 1999)) AND (d_moy#128 = 12)) AND (d_dom#129 = 16)) (130) Project [codegen id : 1] -Output [1]: [d_week_seq#44] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Output [1]: [d_week_seq#126] +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] -Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#60, [id=#61] +Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] -Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67 +Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 ReusedExchange (131) (131) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#72] +Output [1]: [d_date_sk#79] -Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#74, [id=#75] +Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#81, [id=#82] * Project (135) +- * Filter (134) +- * ColumnarToRow (133) @@ -769,21 +769,21 @@ Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subque (132) Scan parquet default.date_dim -Output [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Output [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (133) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] (134) Filter [codegen id : 1] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] -Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#92 = 1998)) AND (d_moy#113 = 12)) AND (d_dom#114 = 16)) +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Condition : (((((isnotnull(d_year#131) AND isnotnull(d_moy#132)) AND isnotnull(d_dom#133)) AND (d_year#131 = 1998)) AND (d_moy#132 = 12)) AND (d_dom#133 = 16)) (135) Project [codegen id : 1] -Output [1]: [d_week_seq#44] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Output [1]: [d_week_seq#130] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] 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 26141bdb911ca..43ebe6e0b4d9b 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 @@ -252,964 +252,964 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (10) ColumnarToRow [codegen id : 10] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] (11) Filter [codegen id : 10] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_item_sk#11) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#14, d_year#15] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#14, d_year#15] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1999)) AND (d_year#15 <= 2001)) AND isnotnull(d_date_sk#14)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#14] +Input [2]: [d_date_sk#14, d_year#15] (16) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (17) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#14] Join condition: None (18) Project [codegen id : 10] -Output [1]: [ss_item_sk#1] -Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] +Output [1]: [ss_item_sk#11] +Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] (19) Scan parquet default.item -Output [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Condition : (((isnotnull(i_item_sk#15) AND isnotnull(i_brand_id#16)) AND isnotnull(i_class_id#17)) AND isnotnull(i_category_id#18)) +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) (22) Exchange -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: hashpartitioning(coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] (23) Sort [codegen id : 5] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: [coalesce(i_brand_id#16, 0) ASC NULLS FIRST, isnull(i_brand_id#16) ASC NULLS FIRST, coalesce(i_class_id#17, 0) ASC NULLS FIRST, isnull(i_class_id#17) ASC NULLS FIRST, coalesce(i_category_id#18, 0) ASC NULLS FIRST, isnull(i_category_id#18) ASC NULLS FIRST], false, 0 +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 (24) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] +Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 8] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] (26) Filter [codegen id : 8] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#22) (27) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#24] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None (29) Project [codegen id : 8] -Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#22] +Output [1]: [cs_item_sk#22] +Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] (30) Scan parquet default.item -Output [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] (32) Filter [codegen id : 7] -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -Condition : isnotnull(i_item_sk#23) +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Condition : isnotnull(i_item_sk#25) (33) BroadcastExchange -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#23] +Left keys [1]: [cs_item_sk#22] +Right keys [1]: [i_item_sk#25] Join condition: None (35) Project [codegen id : 8] -Output [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Input [5]: [cs_item_sk#20, i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] (36) Exchange -Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: hashpartitioning(coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26), 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] (37) Sort [codegen id : 9] -Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: [coalesce(i_brand_id#24, 0) ASC NULLS FIRST, isnull(i_brand_id#24) ASC NULLS FIRST, coalesce(i_class_id#25, 0) ASC NULLS FIRST, isnull(i_class_id#25) ASC NULLS FIRST, coalesce(i_category_id#26, 0) ASC NULLS FIRST, isnull(i_category_id#26) ASC NULLS FIRST], false, 0 +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 (38) SortMergeJoin -Left keys [6]: [coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18)] -Right keys [6]: [coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26)] +Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] Join condition: None (39) BroadcastExchange -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] +Left keys [1]: [ss_item_sk#11] +Right keys [1]: [i_item_sk#17] Join condition: None (41) Project [codegen id : 10] -Output [3]: [i_brand_id#16 AS brand_id#30, i_class_id#17 AS class_id#31, i_category_id#18 AS category_id#32] -Input [5]: [ss_item_sk#1, i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] +Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (42) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#32, class_id#33, category_id#34] (43) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] (44) HashAggregate [codegen id : 11] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#32, class_id#33, category_id#34] (45) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32), 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] (46) Sort [codegen id : 12] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: [coalesce(brand_id#30, 0) ASC NULLS FIRST, isnull(brand_id#30) ASC NULLS FIRST, coalesce(class_id#31, 0) ASC NULLS FIRST, isnull(class_id#31) ASC NULLS FIRST, coalesce(category_id#32, 0) ASC NULLS FIRST, isnull(category_id#32) ASC NULLS FIRST], false, 0 +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 15] -Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] (49) Filter [codegen id : 15] -Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#35) +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_item_sk#37) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#37] +Output [1]: [d_date_sk#39] (51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#37] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#39] Join condition: None (52) Project [codegen id : 15] -Output [1]: [ws_item_sk#35] -Input [3]: [ws_item_sk#35, ws_sold_date_sk#36, d_date_sk#37] +Output [1]: [ws_item_sk#37] +Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] (53) ReusedExchange [Reuses operator id: 33] -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] (54) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [i_item_sk#38] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#40] Join condition: None (55) Project [codegen id : 15] -Output [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Input [5]: [ws_item_sk#35, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] (56) Exchange -Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: hashpartitioning(coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41), 5), ENSURE_REQUIREMENTS, [id=#42] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] (57) Sort [codegen id : 16] -Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: [coalesce(i_brand_id#39, 0) ASC NULLS FIRST, isnull(i_brand_id#39) ASC NULLS FIRST, coalesce(i_class_id#40, 0) ASC NULLS FIRST, isnull(i_class_id#40) ASC NULLS FIRST, coalesce(i_category_id#41, 0) ASC NULLS FIRST, isnull(i_category_id#41) ASC NULLS FIRST], false, 0 +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] -Right keys [6]: [coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41)] +Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] +Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] Join condition: None (59) HashAggregate [codegen id : 17] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#32, class_id#33, category_id#34] (60) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] (61) HashAggregate [codegen id : 18] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#32, class_id#33, category_id#34] (62) BroadcastExchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#44] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] (63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#30, class_id#31, category_id#32] +Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join condition: None (64) Project [codegen id : 19] -Output [1]: [i_item_sk#7 AS ss_item_sk#45] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#30, class_id#31, category_id#32] +Output [1]: [i_item_sk#7 AS ss_item_sk#47] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] (65) Exchange -Input [1]: [ss_item_sk#45] -Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [1]: [ss_item_sk#47] +Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] (66) Sort [codegen id : 20] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#47] Join condition: None (68) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#47] +Output [3]: [d_date_sk#49, d_year#50, d_moy#51] 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 (69) ColumnarToRow [codegen id : 21] -Input [3]: [d_date_sk#12, d_year#13, d_moy#47] +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] (70) Filter [codegen id : 21] -Input [3]: [d_date_sk#12, d_year#13, d_moy#47] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#47)) AND (d_year#13 = 2000)) AND (d_moy#47 = 11)) AND isnotnull(d_date_sk#12)) +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] +Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2000)) AND (d_moy#51 = 11)) AND isnotnull(d_date_sk#49)) (71) Project [codegen id : 21] -Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#47] +Output [1]: [d_date_sk#49] +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] (72) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] +Input [1]: [d_date_sk#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] (73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#49] Join condition: None (74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] (75) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 22] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] (77) Filter [codegen id : 22] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Condition : isnotnull(i_item_sk#53) (78) Exchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: hashpartitioning(i_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#57] (79) Sort [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: [i_item_sk#53 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#47] (81) Sort [codegen id : 41] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#45] +Left keys [1]: [i_item_sk#53] +Right keys [1]: [ss_item_sk#47] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] (84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#53] Join condition: None (85) Project [codegen id : 42] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] (86) HashAggregate [codegen id : 42] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] +Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#51, isEmpty#52, count#53] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] +Aggregate Attributes [3]: [sum#59, isEmpty#60, count#61] +Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] +Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#65] (88) HashAggregate [codegen id : 43] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] +Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66, count(1)#67] +Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] (89) Filter [codegen id : 43] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(32,6)) > cast(Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (90) Project [codegen id : 43] -Output [6]: [store AS channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] +Output [6]: [store AS channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] (91) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Output [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#77), dynamicpruningexpression(cs_sold_date_sk#77 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 44] -Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] (93) Filter [codegen id : 44] -Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Condition : isnotnull(cs_item_sk#74) (94) Exchange -Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] -Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Arguments: hashpartitioning(cs_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#78] (95) Sort [codegen id : 45] -Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] -Arguments: [cs_item_sk#20 ASC NULLS FIRST], false, 0 +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Arguments: [cs_item_sk#74 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#47] (97) Sort [codegen id : 63] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [ss_item_sk#45] +Left keys [1]: [cs_item_sk#74] +Right keys [1]: [ss_item_sk#47] Join condition: None (99) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#69] +Output [1]: [d_date_sk#79] (100) BroadcastHashJoin [codegen id : 85] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#69] +Left keys [1]: [cs_sold_date_sk#77] +Right keys [1]: [d_date_sk#79] Join condition: None (101) Project [codegen id : 85] -Output [3]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67] -Input [5]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21, d_date_sk#69] +Output [3]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76] +Input [5]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77, d_date_sk#79] (102) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73] +Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] (103) BroadcastHashJoin [codegen id : 85] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#70] +Left keys [1]: [cs_item_sk#74] +Right keys [1]: [i_item_sk#80] Join condition: None (104) Project [codegen id : 85] -Output [5]: [cs_quantity#66, cs_list_price#67, i_brand_id#71, i_class_id#72, i_category_id#73] -Input [7]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73] +Output [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] +Input [7]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] (105) HashAggregate [codegen id : 85] -Input [5]: [cs_quantity#66, cs_list_price#67, i_brand_id#71, i_class_id#72, i_category_id#73] -Keys [3]: [i_brand_id#71, i_class_id#72, i_category_id#73] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#74, isEmpty#75, count#76] -Results [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] +Input [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] +Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] +Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] (106) Exchange -Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] -Arguments: hashpartitioning(i_brand_id#71, i_class_id#72, i_category_id#73, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, [id=#90] (107) HashAggregate [codegen id : 86] -Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] -Keys [3]: [i_brand_id#71, i_class_id#72, i_category_id#73] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81, count(1)#82] -Results [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sales#83, count(1)#82 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] +Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] (108) Filter [codegen id : 86] -Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (109) Project [codegen id : 86] -Output [6]: [catalog AS channel#86, i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84] -Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] +Output [6]: [catalog AS channel#96, i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] (110) Scan parquet default.web_sales -Output [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Output [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#100), dynamicpruningexpression(ws_sold_date_sk#100 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (111) ColumnarToRow [codegen id : 87] -Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] (112) Filter [codegen id : 87] -Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#35) +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Condition : isnotnull(ws_item_sk#97) (113) Exchange -Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] -Arguments: hashpartitioning(ws_item_sk#35, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Arguments: hashpartitioning(ws_item_sk#97, 5), ENSURE_REQUIREMENTS, [id=#101] (114) Sort [codegen id : 88] -Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] -Arguments: [ws_item_sk#35 ASC NULLS FIRST], false, 0 +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Arguments: [ws_item_sk#97 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#47] (116) Sort [codegen id : 106] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (117) SortMergeJoin -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [ss_item_sk#45] +Left keys [1]: [ws_item_sk#97] +Right keys [1]: [ss_item_sk#47] Join condition: None (118) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#90] +Output [1]: [d_date_sk#102] (119) BroadcastHashJoin [codegen id : 128] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#90] +Left keys [1]: [ws_sold_date_sk#100] +Right keys [1]: [d_date_sk#102] Join condition: None (120) Project [codegen id : 128] -Output [3]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88] -Input [5]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36, d_date_sk#90] +Output [3]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99] +Input [5]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100, d_date_sk#102] (121) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94] +Output [4]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] (122) BroadcastHashJoin [codegen id : 128] -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [i_item_sk#91] +Left keys [1]: [ws_item_sk#97] +Right keys [1]: [i_item_sk#103] Join condition: None (123) Project [codegen id : 128] -Output [5]: [ws_quantity#87, ws_list_price#88, i_brand_id#92, i_class_id#93, i_category_id#94] -Input [7]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94] +Output [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] +Input [7]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] (124) HashAggregate [codegen id : 128] -Input [5]: [ws_quantity#87, ws_list_price#88, i_brand_id#92, i_class_id#93, i_category_id#94] -Keys [3]: [i_brand_id#92, i_class_id#93, i_category_id#94] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#95, isEmpty#96, count#97] -Results [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] +Input [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] +Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#107, isEmpty#108, count#109] +Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] (125) Exchange -Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] -Arguments: hashpartitioning(i_brand_id#92, i_class_id#93, i_category_id#94, 5), ENSURE_REQUIREMENTS, [id=#101] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +Arguments: hashpartitioning(i_brand_id#104, i_class_id#105, i_category_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] (126) HashAggregate [codegen id : 129] -Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] -Keys [3]: [i_brand_id#92, i_class_id#93, i_category_id#94] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102, count(1)#103] -Results [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sales#104, count(1)#103 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114, count(1)#115] +Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sales#116, count(1)#115 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] (127) Filter [codegen id : 129] -Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (128) Project [codegen id : 129] -Output [6]: [web AS channel#107, i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105] -Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] +Output [6]: [web AS channel#119, i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] (129) Union (130) HashAggregate [codegen id : 130] -Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61] -Keys [4]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] -Aggregate Attributes [3]: [sum#108, isEmpty#109, sum#110] -Results [7]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum#111, isEmpty#112, sum#113] +Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69] +Keys [4]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#120, isEmpty#121, sum#122] +Results [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] (131) Exchange -Input [7]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum#111, isEmpty#112, sum#113] -Arguments: hashpartitioning(channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#114] +Input [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] +Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#126] (132) HashAggregate [codegen id : 131] -Input [7]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum#111, isEmpty#112, sum#113] -Keys [4]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#60), sum(number_sales#61)] -Aggregate Attributes [2]: [sum(sales#60)#115, sum(number_sales#61)#116] -Results [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum(sales#60)#115 AS sum_sales#117, sum(number_sales#61)#116 AS number_sales#118] +Input [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] +Keys [4]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#127, sum(number_sales#69)#128] +Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum(sales#68)#127 AS sum_sales#129, sum(number_sales#69)#128 AS number_sales#130] (133) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sum#122, isEmpty#123, count#124] +Output [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, count#136] (134) HashAggregate [codegen id : 174] -Input [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sum#122, isEmpty#123, count#124] -Keys [3]: [i_brand_id#119, i_class_id#120, i_category_id#121] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#127, count(1)#128] -Results [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#127 AS sales#60, count(1)#128 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#127 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129] +Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, count#136] +Keys [3]: [i_brand_id#131, i_class_id#132, i_category_id#133] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139, count(1)#140] +Results [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sales#68, count(1)#140 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] (135) Filter [codegen id : 174] -Input [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (136) Project [codegen id : 174] -Output [6]: [store AS channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sales#60, number_sales#61] -Input [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129] +Output [6]: [store AS channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69] +Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] (137) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sum#133, isEmpty#134, count#135] +Output [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum#145, isEmpty#146, count#147] (138) HashAggregate [codegen id : 217] -Input [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sum#133, isEmpty#134, count#135] -Keys [3]: [i_brand_id#130, i_class_id#131, i_category_id#132] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#138, count(1)#139] -Results [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sales#83, count(1)#139 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140] +Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum#145, isEmpty#146, count#147] +Keys [3]: [i_brand_id#142, i_class_id#143, i_category_id#144] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150, count(1)#151] +Results [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sales#93, count(1)#151 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] (139) Filter [codegen id : 217] -Input [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (140) Project [codegen id : 217] -Output [6]: [catalog AS channel#86, i_brand_id#130, i_class_id#131, i_category_id#132, sales#83, number_sales#84] -Input [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140] +Output [6]: [catalog AS channel#96, i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94] +Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] (141) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sum#144, isEmpty#145, count#146] +Output [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum#156, isEmpty#157, count#158] (142) HashAggregate [codegen id : 260] -Input [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sum#144, isEmpty#145, count#146] -Keys [3]: [i_brand_id#141, i_class_id#142, i_category_id#143] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#149, count(1)#150] -Results [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#149 AS sales#104, count(1)#150 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151] +Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum#156, isEmpty#157, count#158] +Keys [3]: [i_brand_id#153, i_class_id#154, i_category_id#155] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161, count(1)#162] +Results [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161 AS sales#116, count(1)#162 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] (143) Filter [codegen id : 260] -Input [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (144) Project [codegen id : 260] -Output [6]: [web AS channel#107, i_brand_id#141, i_class_id#142, i_category_id#143, sales#104, number_sales#105] -Input [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151] +Output [6]: [web AS channel#119, i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117] +Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] (145) Union (146) HashAggregate [codegen id : 261] -Input [6]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sales#60, number_sales#61] -Keys [4]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121] -Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] -Aggregate Attributes [3]: [sum#152, isEmpty#153, sum#154] -Results [7]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sum#155, isEmpty#156, sum#157] +Input [6]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69] +Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#164, isEmpty#165, sum#166] +Results [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] (147) Exchange -Input [7]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sum#155, isEmpty#156, sum#157] -Arguments: hashpartitioning(channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, 5), ENSURE_REQUIREMENTS, [id=#158] +Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] +Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, 5), ENSURE_REQUIREMENTS, [id=#170] (148) HashAggregate [codegen id : 262] -Input [7]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sum#155, isEmpty#156, sum#157] -Keys [4]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121] -Functions [2]: [sum(sales#60), sum(number_sales#61)] -Aggregate Attributes [2]: [sum(sales#60)#159, sum(number_sales#61)#160] -Results [5]: [channel#65, i_brand_id#119, i_class_id#120, sum(sales#60)#159 AS sum_sales#117, sum(number_sales#61)#160 AS number_sales#118] +Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] +Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#171, sum(number_sales#69)#172] +Results [5]: [channel#73, i_brand_id#131, i_class_id#132, sum(sales#68)#171 AS sum_sales#129, sum(number_sales#69)#172 AS number_sales#130] (149) HashAggregate [codegen id : 262] -Input [5]: [channel#65, i_brand_id#119, i_class_id#120, sum_sales#117, number_sales#118] -Keys [3]: [channel#65, i_brand_id#119, i_class_id#120] -Functions [2]: [partial_sum(sum_sales#117), partial_sum(number_sales#118)] -Aggregate Attributes [3]: [sum#161, isEmpty#162, sum#163] -Results [6]: [channel#65, i_brand_id#119, i_class_id#120, sum#164, isEmpty#165, sum#166] +Input [5]: [channel#73, i_brand_id#131, i_class_id#132, sum_sales#129, number_sales#130] +Keys [3]: [channel#73, i_brand_id#131, i_class_id#132] +Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] +Aggregate Attributes [3]: [sum#173, isEmpty#174, sum#175] +Results [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] (150) Exchange -Input [6]: [channel#65, i_brand_id#119, i_class_id#120, sum#164, isEmpty#165, sum#166] -Arguments: hashpartitioning(channel#65, i_brand_id#119, i_class_id#120, 5), ENSURE_REQUIREMENTS, [id=#167] +Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] +Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, 5), ENSURE_REQUIREMENTS, [id=#179] (151) HashAggregate [codegen id : 263] -Input [6]: [channel#65, i_brand_id#119, i_class_id#120, sum#164, isEmpty#165, sum#166] -Keys [3]: [channel#65, i_brand_id#119, i_class_id#120] -Functions [2]: [sum(sum_sales#117), sum(number_sales#118)] -Aggregate Attributes [2]: [sum(sum_sales#117)#168, sum(number_sales#118)#169] -Results [6]: [channel#65, i_brand_id#119, i_class_id#120, null AS i_category_id#170, sum(sum_sales#117)#168 AS sum(sum_sales)#171, sum(number_sales#118)#169 AS sum(number_sales)#172] +Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] +Keys [3]: [channel#73, i_brand_id#131, i_class_id#132] +Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] +Aggregate Attributes [2]: [sum(sum_sales#129)#180, sum(number_sales#130)#181] +Results [6]: [channel#73, i_brand_id#131, i_class_id#132, null AS i_category_id#182, sum(sum_sales#129)#180 AS sum(sum_sales)#183, sum(number_sales#130)#181 AS sum(number_sales)#184] (152) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sum#176, isEmpty#177, count#178] +Output [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum#188, isEmpty#189, count#190] (153) HashAggregate [codegen id : 306] -Input [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sum#176, isEmpty#177, count#178] -Keys [3]: [i_brand_id#173, i_class_id#174, i_category_id#175] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#181, count(1)#182] -Results [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sales#60, count(1)#182 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183] +Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum#188, isEmpty#189, count#190] +Keys [3]: [i_brand_id#185, i_class_id#186, i_category_id#187] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193, count(1)#194] +Results [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193 AS sales#68, count(1)#194 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] (154) Filter [codegen id : 306] -Input [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (155) Project [codegen id : 306] -Output [6]: [store AS channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sales#60, number_sales#61] -Input [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183] +Output [6]: [store AS channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69] +Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] (156) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sum#187, isEmpty#188, count#189] +Output [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum#199, isEmpty#200, count#201] (157) HashAggregate [codegen id : 349] -Input [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sum#187, isEmpty#188, count#189] -Keys [3]: [i_brand_id#184, i_class_id#185, i_category_id#186] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#192, count(1)#193] -Results [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sales#83, count(1)#193 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194] +Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum#199, isEmpty#200, count#201] +Keys [3]: [i_brand_id#196, i_class_id#197, i_category_id#198] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204, count(1)#205] +Results [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204 AS sales#93, count(1)#205 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] (158) Filter [codegen id : 349] -Input [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (159) Project [codegen id : 349] -Output [6]: [catalog AS channel#86, i_brand_id#184, i_class_id#185, i_category_id#186, sales#83, number_sales#84] -Input [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194] +Output [6]: [catalog AS channel#96, i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94] +Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] (160) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sum#198, isEmpty#199, count#200] +Output [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum#210, isEmpty#211, count#212] (161) HashAggregate [codegen id : 392] -Input [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sum#198, isEmpty#199, count#200] -Keys [3]: [i_brand_id#195, i_class_id#196, i_category_id#197] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#203, count(1)#204] -Results [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#203 AS sales#104, count(1)#204 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#203 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205] +Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum#210, isEmpty#211, count#212] +Keys [3]: [i_brand_id#207, i_class_id#208, i_category_id#209] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215, count(1)#216] +Results [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sales#116, count(1)#216 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] (162) Filter [codegen id : 392] -Input [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (163) Project [codegen id : 392] -Output [6]: [web AS channel#107, i_brand_id#195, i_class_id#196, i_category_id#197, sales#104, number_sales#105] -Input [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205] +Output [6]: [web AS channel#119, i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117] +Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] (164) Union (165) HashAggregate [codegen id : 393] -Input [6]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sales#60, number_sales#61] -Keys [4]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175] -Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] -Aggregate Attributes [3]: [sum#206, isEmpty#207, sum#208] -Results [7]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sum#209, isEmpty#210, sum#211] +Input [6]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69] +Keys [4]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#218, isEmpty#219, sum#220] +Results [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] (166) Exchange -Input [7]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sum#209, isEmpty#210, sum#211] -Arguments: hashpartitioning(channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, 5), ENSURE_REQUIREMENTS, [id=#212] +Input [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] +Arguments: hashpartitioning(channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, 5), ENSURE_REQUIREMENTS, [id=#224] (167) HashAggregate [codegen id : 394] -Input [7]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sum#209, isEmpty#210, sum#211] -Keys [4]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175] -Functions [2]: [sum(sales#60), sum(number_sales#61)] -Aggregate Attributes [2]: [sum(sales#60)#213, sum(number_sales#61)#214] -Results [4]: [channel#65, i_brand_id#173, sum(sales#60)#213 AS sum_sales#117, sum(number_sales#61)#214 AS number_sales#118] +Input [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] +Keys [4]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#225, sum(number_sales#69)#226] +Results [4]: [channel#73, i_brand_id#185, sum(sales#68)#225 AS sum_sales#129, sum(number_sales#69)#226 AS number_sales#130] (168) HashAggregate [codegen id : 394] -Input [4]: [channel#65, i_brand_id#173, sum_sales#117, number_sales#118] -Keys [2]: [channel#65, i_brand_id#173] -Functions [2]: [partial_sum(sum_sales#117), partial_sum(number_sales#118)] -Aggregate Attributes [3]: [sum#215, isEmpty#216, sum#217] -Results [5]: [channel#65, i_brand_id#173, sum#218, isEmpty#219, sum#220] +Input [4]: [channel#73, i_brand_id#185, sum_sales#129, number_sales#130] +Keys [2]: [channel#73, i_brand_id#185] +Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] +Aggregate Attributes [3]: [sum#227, isEmpty#228, sum#229] +Results [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] (169) Exchange -Input [5]: [channel#65, i_brand_id#173, sum#218, isEmpty#219, sum#220] -Arguments: hashpartitioning(channel#65, i_brand_id#173, 5), ENSURE_REQUIREMENTS, [id=#221] +Input [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] +Arguments: hashpartitioning(channel#73, i_brand_id#185, 5), ENSURE_REQUIREMENTS, [id=#233] (170) HashAggregate [codegen id : 395] -Input [5]: [channel#65, i_brand_id#173, sum#218, isEmpty#219, sum#220] -Keys [2]: [channel#65, i_brand_id#173] -Functions [2]: [sum(sum_sales#117), sum(number_sales#118)] -Aggregate Attributes [2]: [sum(sum_sales#117)#222, sum(number_sales#118)#223] -Results [6]: [channel#65, i_brand_id#173, null AS i_class_id#224, null AS i_category_id#225, sum(sum_sales#117)#222 AS sum(sum_sales)#226, sum(number_sales#118)#223 AS sum(number_sales)#227] +Input [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] +Keys [2]: [channel#73, i_brand_id#185] +Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] +Aggregate Attributes [2]: [sum(sum_sales#129)#234, sum(number_sales#130)#235] +Results [6]: [channel#73, i_brand_id#185, null AS i_class_id#236, null AS i_category_id#237, sum(sum_sales#129)#234 AS sum(sum_sales)#238, sum(number_sales#130)#235 AS sum(number_sales)#239] (171) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sum#231, isEmpty#232, count#233] +Output [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum#243, isEmpty#244, count#245] (172) HashAggregate [codegen id : 438] -Input [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sum#231, isEmpty#232, count#233] -Keys [3]: [i_brand_id#228, i_class_id#229, i_category_id#230] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#236, count(1)#237] -Results [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#236 AS sales#60, count(1)#237 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#236 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238] +Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum#243, isEmpty#244, count#245] +Keys [3]: [i_brand_id#240, i_class_id#241, i_category_id#242] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248, count(1)#249] +Results [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248 AS sales#68, count(1)#249 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] (173) Filter [codegen id : 438] -Input [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (174) Project [codegen id : 438] -Output [6]: [store AS channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sales#60, number_sales#61] -Input [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238] +Output [6]: [store AS channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69] +Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] (175) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sum#242, isEmpty#243, count#244] +Output [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum#254, isEmpty#255, count#256] (176) HashAggregate [codegen id : 481] -Input [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sum#242, isEmpty#243, count#244] -Keys [3]: [i_brand_id#239, i_class_id#240, i_category_id#241] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#247, count(1)#248] -Results [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#247 AS sales#83, count(1)#248 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#247 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249] +Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum#254, isEmpty#255, count#256] +Keys [3]: [i_brand_id#251, i_class_id#252, i_category_id#253] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259, count(1)#260] +Results [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259 AS sales#93, count(1)#260 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] (177) Filter [codegen id : 481] -Input [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (178) Project [codegen id : 481] -Output [6]: [catalog AS channel#86, i_brand_id#239, i_class_id#240, i_category_id#241, sales#83, number_sales#84] -Input [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249] +Output [6]: [catalog AS channel#96, i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94] +Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] (179) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sum#253, isEmpty#254, count#255] +Output [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum#265, isEmpty#266, count#267] (180) HashAggregate [codegen id : 524] -Input [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sum#253, isEmpty#254, count#255] -Keys [3]: [i_brand_id#250, i_class_id#251, i_category_id#252] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#258, count(1)#259] -Results [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#258 AS sales#104, count(1)#259 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#258 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260] +Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum#265, isEmpty#266, count#267] +Keys [3]: [i_brand_id#262, i_class_id#263, i_category_id#264] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270, count(1)#271] +Results [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270 AS sales#116, count(1)#271 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] (181) Filter [codegen id : 524] -Input [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (182) Project [codegen id : 524] -Output [6]: [web AS channel#107, i_brand_id#250, i_class_id#251, i_category_id#252, sales#104, number_sales#105] -Input [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260] +Output [6]: [web AS channel#119, i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117] +Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] (183) Union (184) HashAggregate [codegen id : 525] -Input [6]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sales#60, number_sales#61] -Keys [4]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230] -Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] -Aggregate Attributes [3]: [sum#261, isEmpty#262, sum#263] -Results [7]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sum#264, isEmpty#265, sum#266] +Input [6]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69] +Keys [4]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#273, isEmpty#274, sum#275] +Results [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] (185) Exchange -Input [7]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sum#264, isEmpty#265, sum#266] -Arguments: hashpartitioning(channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, 5), ENSURE_REQUIREMENTS, [id=#267] +Input [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] +Arguments: hashpartitioning(channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, 5), ENSURE_REQUIREMENTS, [id=#279] (186) HashAggregate [codegen id : 526] -Input [7]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sum#264, isEmpty#265, sum#266] -Keys [4]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230] -Functions [2]: [sum(sales#60), sum(number_sales#61)] -Aggregate Attributes [2]: [sum(sales#60)#268, sum(number_sales#61)#269] -Results [3]: [channel#65, sum(sales#60)#268 AS sum_sales#117, sum(number_sales#61)#269 AS number_sales#118] +Input [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] +Keys [4]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#280, sum(number_sales#69)#281] +Results [3]: [channel#73, sum(sales#68)#280 AS sum_sales#129, sum(number_sales#69)#281 AS number_sales#130] (187) HashAggregate [codegen id : 526] -Input [3]: [channel#65, sum_sales#117, number_sales#118] -Keys [1]: [channel#65] -Functions [2]: [partial_sum(sum_sales#117), partial_sum(number_sales#118)] -Aggregate Attributes [3]: [sum#270, isEmpty#271, sum#272] -Results [4]: [channel#65, sum#273, isEmpty#274, sum#275] +Input [3]: [channel#73, sum_sales#129, number_sales#130] +Keys [1]: [channel#73] +Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] +Aggregate Attributes [3]: [sum#282, isEmpty#283, sum#284] +Results [4]: [channel#73, sum#285, isEmpty#286, sum#287] (188) Exchange -Input [4]: [channel#65, sum#273, isEmpty#274, sum#275] -Arguments: hashpartitioning(channel#65, 5), ENSURE_REQUIREMENTS, [id=#276] +Input [4]: [channel#73, sum#285, isEmpty#286, sum#287] +Arguments: hashpartitioning(channel#73, 5), ENSURE_REQUIREMENTS, [id=#288] (189) HashAggregate [codegen id : 527] -Input [4]: [channel#65, sum#273, isEmpty#274, sum#275] -Keys [1]: [channel#65] -Functions [2]: [sum(sum_sales#117), sum(number_sales#118)] -Aggregate Attributes [2]: [sum(sum_sales#117)#277, sum(number_sales#118)#278] -Results [6]: [channel#65, null AS i_brand_id#279, null AS i_class_id#280, null AS i_category_id#281, sum(sum_sales#117)#277 AS sum(sum_sales)#282, sum(number_sales#118)#278 AS sum(number_sales)#283] +Input [4]: [channel#73, sum#285, isEmpty#286, sum#287] +Keys [1]: [channel#73] +Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] +Aggregate Attributes [2]: [sum(sum_sales#129)#289, sum(number_sales#130)#290] +Results [6]: [channel#73, null AS i_brand_id#291, null AS i_class_id#292, null AS i_category_id#293, sum(sum_sales#129)#289 AS sum(sum_sales)#294, sum(number_sales#130)#290 AS sum(number_sales)#295] (190) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sum#287, isEmpty#288, count#289] +Output [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum#299, isEmpty#300, count#301] (191) HashAggregate [codegen id : 570] -Input [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sum#287, isEmpty#288, count#289] -Keys [3]: [i_brand_id#284, i_class_id#285, i_category_id#286] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#292, count(1)#293] -Results [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#292 AS sales#60, count(1)#293 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#292 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294] +Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum#299, isEmpty#300, count#301] +Keys [3]: [i_brand_id#296, i_class_id#297, i_category_id#298] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304, count(1)#305] +Results [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304 AS sales#68, count(1)#305 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] (192) Filter [codegen id : 570] -Input [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (193) Project [codegen id : 570] -Output [6]: [store AS channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sales#60, number_sales#61] -Input [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294] +Output [6]: [store AS channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69] +Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] (194) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sum#298, isEmpty#299, count#300] +Output [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum#310, isEmpty#311, count#312] (195) HashAggregate [codegen id : 613] -Input [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sum#298, isEmpty#299, count#300] -Keys [3]: [i_brand_id#295, i_class_id#296, i_category_id#297] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#303, count(1)#304] -Results [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#303 AS sales#83, count(1)#304 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#303 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305] +Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum#310, isEmpty#311, count#312] +Keys [3]: [i_brand_id#307, i_class_id#308, i_category_id#309] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315, count(1)#316] +Results [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315 AS sales#93, count(1)#316 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] (196) Filter [codegen id : 613] -Input [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (197) Project [codegen id : 613] -Output [6]: [catalog AS channel#86, i_brand_id#295, i_class_id#296, i_category_id#297, sales#83, number_sales#84] -Input [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305] +Output [6]: [catalog AS channel#96, i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94] +Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] (198) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sum#309, isEmpty#310, count#311] +Output [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum#321, isEmpty#322, count#323] (199) HashAggregate [codegen id : 656] -Input [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sum#309, isEmpty#310, count#311] -Keys [3]: [i_brand_id#306, i_class_id#307, i_category_id#308] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#314, count(1)#315] -Results [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#314 AS sales#104, count(1)#315 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#314 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316] +Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum#321, isEmpty#322, count#323] +Keys [3]: [i_brand_id#318, i_class_id#319, i_category_id#320] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326, count(1)#327] +Results [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326 AS sales#116, count(1)#327 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] (200) Filter [codegen id : 656] -Input [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) (201) Project [codegen id : 656] -Output [6]: [web AS channel#107, i_brand_id#306, i_class_id#307, i_category_id#308, sales#104, number_sales#105] -Input [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316] +Output [6]: [web AS channel#119, i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117] +Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] (202) Union (203) HashAggregate [codegen id : 657] -Input [6]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sales#60, number_sales#61] -Keys [4]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286] -Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] -Aggregate Attributes [3]: [sum#317, isEmpty#318, sum#319] -Results [7]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sum#320, isEmpty#321, sum#322] +Input [6]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69] +Keys [4]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#329, isEmpty#330, sum#331] +Results [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] (204) Exchange -Input [7]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sum#320, isEmpty#321, sum#322] -Arguments: hashpartitioning(channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, 5), ENSURE_REQUIREMENTS, [id=#323] +Input [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] +Arguments: hashpartitioning(channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, 5), ENSURE_REQUIREMENTS, [id=#335] (205) HashAggregate [codegen id : 658] -Input [7]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sum#320, isEmpty#321, sum#322] -Keys [4]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286] -Functions [2]: [sum(sales#60), sum(number_sales#61)] -Aggregate Attributes [2]: [sum(sales#60)#324, sum(number_sales#61)#325] -Results [2]: [sum(sales#60)#324 AS sum_sales#117, sum(number_sales#61)#325 AS number_sales#118] +Input [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] +Keys [4]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#336, sum(number_sales#69)#337] +Results [2]: [sum(sales#68)#336 AS sum_sales#129, sum(number_sales#69)#337 AS number_sales#130] (206) HashAggregate [codegen id : 658] -Input [2]: [sum_sales#117, number_sales#118] +Input [2]: [sum_sales#129, number_sales#130] Keys: [] -Functions [2]: [partial_sum(sum_sales#117), partial_sum(number_sales#118)] -Aggregate Attributes [3]: [sum#326, isEmpty#327, sum#328] -Results [3]: [sum#329, isEmpty#330, sum#331] +Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] +Aggregate Attributes [3]: [sum#338, isEmpty#339, sum#340] +Results [3]: [sum#341, isEmpty#342, sum#343] (207) Exchange -Input [3]: [sum#329, isEmpty#330, sum#331] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#332] +Input [3]: [sum#341, isEmpty#342, sum#343] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#344] (208) HashAggregate [codegen id : 659] -Input [3]: [sum#329, isEmpty#330, sum#331] +Input [3]: [sum#341, isEmpty#342, sum#343] Keys: [] -Functions [2]: [sum(sum_sales#117), sum(number_sales#118)] -Aggregate Attributes [2]: [sum(sum_sales#117)#333, sum(number_sales#118)#334] -Results [6]: [null AS channel#335, null AS i_brand_id#336, null AS i_class_id#337, null AS i_category_id#338, sum(sum_sales#117)#333 AS sum(sum_sales)#339, sum(number_sales#118)#334 AS sum(number_sales)#340] +Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] +Aggregate Attributes [2]: [sum(sum_sales#129)#345, sum(number_sales#130)#346] +Results [6]: [null AS channel#347, null AS i_brand_id#348, null AS i_class_id#349, null AS i_category_id#350, sum(sum_sales#129)#345 AS sum(sum_sales)#351, sum(number_sales#130)#346 AS sum(number_sales)#352] (209) Union (210) HashAggregate [codegen id : 660] -Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] -Keys [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] (211) Exchange -Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] -Arguments: hashpartitioning(channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118, 5), ENSURE_REQUIREMENTS, [id=#341] +Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130, 5), ENSURE_REQUIREMENTS, [id=#353] (212) HashAggregate [codegen id : 661] -Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] -Keys [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] (213) TakeOrderedAndProject -Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] -Arguments: 100, [channel#65 ASC NULLS FIRST, i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Arguments: 100, [channel#73 ASC NULLS FIRST, i_brand_id#54 ASC NULLS FIRST, i_class_id#55 ASC NULLS FIRST, i_category_id#56 ASC NULLS FIRST], [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#63, [id=#64] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#71, [id=#72] * HashAggregate (240) +- Exchange (239) +- * HashAggregate (238) @@ -1240,193 +1240,193 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (214) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#342)] +PartitionFilters: [isnotnull(ss_sold_date_sk#356), dynamicpruningexpression(ss_sold_date_sk#356 IN dynamicpruning#357)] ReadSchema: struct (215) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356] (216) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#358, d_year#359] 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 (217) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#358, d_year#359] (218) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#358, d_year#359] +Condition : (((isnotnull(d_year#359) AND (d_year#359 >= 1999)) AND (d_year#359 <= 2001)) AND isnotnull(d_date_sk#358)) (219) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#358] +Input [2]: [d_date_sk#358, d_year#359] (220) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#343] +Input [1]: [d_date_sk#358] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#360] (221) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#356] +Right keys [1]: [d_date_sk#358] Join condition: None (222) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#344, ss_list_price#3 AS list_price#345] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#354 AS quantity#361, ss_list_price#355 AS list_price#362] +Input [4]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356, d_date_sk#358] (223) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Output [3]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#346)] +PartitionFilters: [isnotnull(cs_sold_date_sk#365), dynamicpruningexpression(cs_sold_date_sk#365 IN dynamicpruning#366)] ReadSchema: struct (224) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Input [3]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365] (225) Scan parquet default.date_dim -Output [2]: [d_date_sk#347, d_year#348] +Output [2]: [d_date_sk#367, d_year#368] 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 (226) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#347, d_year#348] +Input [2]: [d_date_sk#367, d_year#368] (227) Filter [codegen id : 3] -Input [2]: [d_date_sk#347, d_year#348] -Condition : (((isnotnull(d_year#348) AND (d_year#348 >= 1998)) AND (d_year#348 <= 2000)) AND isnotnull(d_date_sk#347)) +Input [2]: [d_date_sk#367, d_year#368] +Condition : (((isnotnull(d_year#368) AND (d_year#368 >= 1998)) AND (d_year#368 <= 2000)) AND isnotnull(d_date_sk#367)) (228) Project [codegen id : 3] -Output [1]: [d_date_sk#347] -Input [2]: [d_date_sk#347, d_year#348] +Output [1]: [d_date_sk#367] +Input [2]: [d_date_sk#367, d_year#368] (229) BroadcastExchange -Input [1]: [d_date_sk#347] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#349] +Input [1]: [d_date_sk#367] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#369] (230) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#347] +Left keys [1]: [cs_sold_date_sk#365] +Right keys [1]: [d_date_sk#367] Join condition: None (231) Project [codegen id : 4] -Output [2]: [cs_quantity#66 AS quantity#350, cs_list_price#67 AS list_price#351] -Input [4]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21, d_date_sk#347] +Output [2]: [cs_quantity#363 AS quantity#370, cs_list_price#364 AS list_price#371] +Input [4]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365, d_date_sk#367] (232) Scan parquet default.web_sales -Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Output [3]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#346)] +PartitionFilters: [isnotnull(ws_sold_date_sk#374), dynamicpruningexpression(ws_sold_date_sk#374 IN dynamicpruning#366)] ReadSchema: struct (233) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Input [3]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374] (234) ReusedExchange [Reuses operator id: 229] -Output [1]: [d_date_sk#352] +Output [1]: [d_date_sk#375] (235) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#352] +Left keys [1]: [ws_sold_date_sk#374] +Right keys [1]: [d_date_sk#375] Join condition: None (236) Project [codegen id : 6] -Output [2]: [ws_quantity#87 AS quantity#353, ws_list_price#88 AS list_price#354] -Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36, d_date_sk#352] +Output [2]: [ws_quantity#372 AS quantity#376, ws_list_price#373 AS list_price#377] +Input [4]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374, d_date_sk#375] (237) Union (238) HashAggregate [codegen id : 7] -Input [2]: [quantity#344, list_price#345] +Input [2]: [quantity#361, list_price#362] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#344 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#345 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#355, count#356] -Results [2]: [sum#357, count#358] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#378, count#379] +Results [2]: [sum#380, count#381] (239) Exchange -Input [2]: [sum#357, count#358] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#359] +Input [2]: [sum#380, count#381] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#382] (240) HashAggregate [codegen id : 8] -Input [2]: [sum#357, count#358] +Input [2]: [sum#380, count#381] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#344 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#345 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#344 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#345 as decimal(12,2)))), DecimalType(18,2), true))#360] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#344 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#345 as decimal(12,2)))), DecimalType(18,2), true))#360 AS average_sales#361] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))#383] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))#383 AS average_sales#384] -Subquery:2 Hosting operator id = 214 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#342 +Subquery:2 Hosting operator id = 214 Hosting Expression = ss_sold_date_sk#356 IN dynamicpruning#357 ReusedExchange (241) (241) ReusedExchange [Reuses operator id: 220] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#358] -Subquery:3 Hosting operator id = 223 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#346 +Subquery:3 Hosting operator id = 223 Hosting Expression = cs_sold_date_sk#365 IN dynamicpruning#366 ReusedExchange (242) (242) ReusedExchange [Reuses operator id: 229] -Output [1]: [d_date_sk#347] +Output [1]: [d_date_sk#367] -Subquery:4 Hosting operator id = 232 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#346 +Subquery:4 Hosting operator id = 232 Hosting Expression = ws_sold_date_sk#374 IN dynamicpruning#366 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (243) (243) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#49] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 ReusedExchange (244) (244) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#14] -Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 -Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#77 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#5 -Subquery:13 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:13 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:14 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:14 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:15 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:15 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:16 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:16 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:17 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:17 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:18 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:18 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:19 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:19 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:20 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:20 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:21 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:21 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:22 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:22 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:23 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:23 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:24 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:24 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] 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 4bd1e57d994ab..755973f79e493 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -225,896 +225,896 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_item_sk#10) (10) Scan parquet default.item -Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (11) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] (12) Filter [codegen id : 4] -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14)) +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) (13) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] (15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#15) +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Condition : isnotnull(cs_item_sk#17) (16) Scan parquet default.item -Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] (18) Filter [codegen id : 1] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Condition : isnotnull(i_item_sk#17) +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Condition : isnotnull(i_item_sk#19) (19) BroadcastExchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] (20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#15] -Right keys [1]: [i_item_sk#17] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#19] Join condition: None (21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20] -Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] (22) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_year#23] +Output [2]: [d_date_sk#24, d_year#25] 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 (23) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#22, d_year#23] +Input [2]: [d_date_sk#24, d_year#25] (24) Filter [codegen id : 2] -Input [2]: [d_date_sk#22, d_year#23] -Condition : (((isnotnull(d_year#23) AND (d_year#23 >= 1999)) AND (d_year#23 <= 2001)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) (25) Project [codegen id : 2] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_year#23] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_year#25] (26) 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#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (27) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] Join condition: None (28) Project [codegen id : 3] -Output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] -Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#22] +Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] (29) BroadcastExchange -Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] (30) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)] -Right keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join condition: None (31) BroadcastExchange -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] (32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#11] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] Join condition: None (33) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] (34) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#29] (35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#29] Join condition: None (36) Project [codegen id : 6] -Output [3]: [i_brand_id#12 AS brand_id#28, i_class_id#13 AS class_id#29, i_category_id#14 AS category_id#30] -Input [5]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#27] +Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#30, class_id#31, category_id#32] (38) Exchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#30, class_id#31, category_id#32] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_item_sk#32) +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_item_sk#34) (43) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#34] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#36] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37] -Input [6]: [ws_item_sk#32, ws_sold_date_sk#33, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] +Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] (46) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#38] +Output [1]: [d_date_sk#40] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#40] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] -Input [5]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] +Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] (49) BroadcastExchange -Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#39] +Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] -Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#30, class_id#31, category_id#32] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#30, class_id#31, category_id#32] (53) BroadcastExchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#40] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#28, class_id#29, category_id#30] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#41] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] +Output [1]: [i_item_sk#6 AS ss_item_sk#43] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] (56) BroadcastExchange -Input [1]: [ss_item_sk#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [ss_item_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#41] +Right keys [1]: [ss_item_sk#43] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Condition : isnotnull(i_item_sk#45) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#43] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#41] +Left keys [1]: [i_item_sk#45] +Right keys [1]: [ss_item_sk#43] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#45] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (66) Scan parquet default.date_dim -Output [3]: [d_date_sk#27, d_year#44, d_moy#45] +Output [3]: [d_date_sk#50, d_year#51, d_moy#52] 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 (67) ColumnarToRow [codegen id : 24] -Input [3]: [d_date_sk#27, d_year#44, d_moy#45] +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] (68) Filter [codegen id : 24] -Input [3]: [d_date_sk#27, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#27)) +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] +Condition : ((((isnotnull(d_year#51) AND isnotnull(d_moy#52)) AND (d_year#51 = 2000)) AND (d_moy#52 = 11)) AND isnotnull(d_date_sk#50)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#27] -Input [3]: [d_date_sk#27, d_year#44, d_moy#45] +Output [1]: [d_date_sk#50] +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] (70) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Input [1]: [d_date_sk#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#50] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#27] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#47, isEmpty#48, count#49] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] +Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] +Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#60] (75) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] (76) Filter [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [store AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] +Output [6]: [store AS channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] (78) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Output [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] (80) Filter [codegen id : 51] -Input [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#15) +Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +Condition : isnotnull(cs_item_sk#69) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#43] (82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#15] -Right keys [1]: [ss_item_sk#41] +Left keys [1]: [cs_item_sk#69] +Right keys [1]: [ss_item_sk#43] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +Output [4]: [i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] (84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#15] -Right keys [1]: [i_item_sk#64] +Left keys [1]: [cs_item_sk#69] +Right keys [1]: [i_item_sk#73] Join condition: None (85) Project [codegen id : 51] -Output [6]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [8]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +Output [6]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [8]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] (86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#68] +Output [1]: [d_date_sk#77] (87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#68] +Left keys [1]: [cs_sold_date_sk#72] +Right keys [1]: [d_date_sk#77] Join condition: None (88) Project [codegen id : 51] -Output [5]: [cs_quantity#62, cs_list_price#63, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [7]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_brand_id#65, i_class_id#66, i_category_id#67, d_date_sk#68] +Output [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [7]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76, d_date_sk#77] (89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#62, cs_list_price#63, i_brand_id#65, i_class_id#66, i_category_id#67] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] -Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] +Input [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] +Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#78, isEmpty#79, count#80] +Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] (90) Exchange -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Arguments: hashpartitioning(i_brand_id#65, i_class_id#66, i_category_id#67, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Arguments: hashpartitioning(i_brand_id#74, i_class_id#75, i_category_id#76, 5), ENSURE_REQUIREMENTS, [id=#84] (91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76, count(1)#77] -Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sales#78, count(1)#77 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85, count(1)#86] +Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sales#87, count(1)#86 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] (92) Filter [codegen id : 52] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [catalog AS channel#81, i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] +Output [6]: [catalog AS channel#90, i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] (94) Scan parquet default.web_sales -Output [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Output [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] (96) Filter [codegen id : 77] -Input [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] -Condition : isnotnull(ws_item_sk#32) +Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Condition : isnotnull(ws_item_sk#91) (97) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#43] (98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [ss_item_sk#41] +Left keys [1]: [ws_item_sk#91] +Right keys [1]: [ss_item_sk#43] Join condition: None (99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] +Output [4]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] (100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#84] +Left keys [1]: [ws_item_sk#91] +Right keys [1]: [i_item_sk#95] Join condition: None (101) Project [codegen id : 77] -Output [6]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_brand_id#85, i_class_id#86, i_category_id#87] -Input [8]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] +Output [6]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [8]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] (102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#88] +Output [1]: [d_date_sk#99] (103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#88] +Left keys [1]: [ws_sold_date_sk#94] +Right keys [1]: [d_date_sk#99] Join condition: None (104) Project [codegen id : 77] -Output [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#85, i_class_id#86, i_category_id#87] -Input [7]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_brand_id#85, i_class_id#86, i_category_id#87, d_date_sk#88] +Output [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [7]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98, d_date_sk#99] (105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#85, i_class_id#86, i_category_id#87] -Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#89, isEmpty#90, count#91] -Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Input [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] +Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#100, isEmpty#101, count#102] +Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] (106) Exchange -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] -Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, 5), ENSURE_REQUIREMENTS, [id=#95] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Arguments: hashpartitioning(i_brand_id#96, i_class_id#97, i_category_id#98, 5), ENSURE_REQUIREMENTS, [id=#106] (107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] -Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96, count(1)#97] -Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sales#98, count(1)#97 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] +Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#109, count(1)#108 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] (108) Filter [codegen id : 78] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [web AS channel#101, i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] +Output [6]: [web AS channel#112, i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] (110) Union (111) HashAggregate [codegen id : 79] -Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57] -Keys [4]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] -Aggregate Attributes [3]: [sum#102, isEmpty#103, sum#104] -Results [7]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum#105, isEmpty#106, sum#107] +Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64] +Keys [4]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#113, isEmpty#114, sum#115] +Results [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] (112) Exchange -Input [7]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum#105, isEmpty#106, sum#107] -Arguments: hashpartitioning(channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#108] +Input [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] +Arguments: hashpartitioning(channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#119] (113) HashAggregate [codegen id : 80] -Input [7]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum#105, isEmpty#106, sum#107] -Keys [4]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#56), sum(number_sales#57)] -Aggregate Attributes [2]: [sum(sales#56)#109, sum(number_sales#57)#110] -Results [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#56)#109 AS sum_sales#111, sum(number_sales#57)#110 AS number_sales#112] +Input [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] +Keys [4]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#120, sum(number_sales#64)#121] +Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum(sales#63)#120 AS sum_sales#122, sum(number_sales#64)#121 AS number_sales#123] (114) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sum#116, isEmpty#117, count#118] +Output [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] (115) HashAggregate [codegen id : 106] -Input [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sum#116, isEmpty#117, count#118] -Keys [3]: [i_brand_id#113, i_class_id#114, i_category_id#115] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#121, count(1)#122] -Results [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#121 AS sales#56, count(1)#122 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#121 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123] +Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] +Keys [3]: [i_brand_id#124, i_class_id#125, i_category_id#126] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132, count(1)#133] +Results [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sales#63, count(1)#133 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] (116) Filter [codegen id : 106] -Input [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (117) Project [codegen id : 106] -Output [6]: [store AS channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sales#56, number_sales#57] -Input [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123] +Output [6]: [store AS channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64] +Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] (118) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] +Output [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] (119) HashAggregate [codegen id : 132] -Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] -Keys [3]: [i_brand_id#124, i_class_id#125, i_category_id#126] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132, count(1)#133] -Results [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sales#78, count(1)#133 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] +Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] +Keys [3]: [i_brand_id#135, i_class_id#136, i_category_id#137] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143, count(1)#144] +Results [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sales#87, count(1)#144 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] (120) Filter [codegen id : 132] -Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (121) Project [codegen id : 132] -Output [6]: [catalog AS channel#81, i_brand_id#124, i_class_id#125, i_category_id#126, sales#78, number_sales#79] -Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] +Output [6]: [catalog AS channel#90, i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88] +Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] (122) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] +Output [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum#149, isEmpty#150, count#151] (123) HashAggregate [codegen id : 158] -Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] -Keys [3]: [i_brand_id#135, i_class_id#136, i_category_id#137] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143, count(1)#144] -Results [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sales#98, count(1)#144 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] +Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum#149, isEmpty#150, count#151] +Keys [3]: [i_brand_id#146, i_class_id#147, i_category_id#148] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154, count(1)#155] +Results [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154 AS sales#109, count(1)#155 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] (124) Filter [codegen id : 158] -Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (125) Project [codegen id : 158] -Output [6]: [web AS channel#101, i_brand_id#135, i_class_id#136, i_category_id#137, sales#98, number_sales#99] -Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] +Output [6]: [web AS channel#112, i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110] +Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] (126) Union (127) HashAggregate [codegen id : 159] -Input [6]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sales#56, number_sales#57] -Keys [4]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115] -Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] -Aggregate Attributes [3]: [sum#146, isEmpty#147, sum#148] -Results [7]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sum#149, isEmpty#150, sum#151] +Input [6]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64] +Keys [4]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#157, isEmpty#158, sum#159] +Results [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] (128) Exchange -Input [7]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sum#149, isEmpty#150, sum#151] -Arguments: hashpartitioning(channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, 5), ENSURE_REQUIREMENTS, [id=#152] +Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] +Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, 5), ENSURE_REQUIREMENTS, [id=#163] (129) HashAggregate [codegen id : 160] -Input [7]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sum#149, isEmpty#150, sum#151] -Keys [4]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115] -Functions [2]: [sum(sales#56), sum(number_sales#57)] -Aggregate Attributes [2]: [sum(sales#56)#153, sum(number_sales#57)#154] -Results [5]: [channel#61, i_brand_id#113, i_class_id#114, sum(sales#56)#153 AS sum_sales#111, sum(number_sales#57)#154 AS number_sales#112] +Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] +Keys [4]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#164, sum(number_sales#64)#165] +Results [5]: [channel#68, i_brand_id#124, i_class_id#125, sum(sales#63)#164 AS sum_sales#122, sum(number_sales#64)#165 AS number_sales#123] (130) HashAggregate [codegen id : 160] -Input [5]: [channel#61, i_brand_id#113, i_class_id#114, sum_sales#111, number_sales#112] -Keys [3]: [channel#61, i_brand_id#113, i_class_id#114] -Functions [2]: [partial_sum(sum_sales#111), partial_sum(number_sales#112)] -Aggregate Attributes [3]: [sum#155, isEmpty#156, sum#157] -Results [6]: [channel#61, i_brand_id#113, i_class_id#114, sum#158, isEmpty#159, sum#160] +Input [5]: [channel#68, i_brand_id#124, i_class_id#125, sum_sales#122, number_sales#123] +Keys [3]: [channel#68, i_brand_id#124, i_class_id#125] +Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] +Aggregate Attributes [3]: [sum#166, isEmpty#167, sum#168] +Results [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] (131) Exchange -Input [6]: [channel#61, i_brand_id#113, i_class_id#114, sum#158, isEmpty#159, sum#160] -Arguments: hashpartitioning(channel#61, i_brand_id#113, i_class_id#114, 5), ENSURE_REQUIREMENTS, [id=#161] +Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] +Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, 5), ENSURE_REQUIREMENTS, [id=#172] (132) HashAggregate [codegen id : 161] -Input [6]: [channel#61, i_brand_id#113, i_class_id#114, sum#158, isEmpty#159, sum#160] -Keys [3]: [channel#61, i_brand_id#113, i_class_id#114] -Functions [2]: [sum(sum_sales#111), sum(number_sales#112)] -Aggregate Attributes [2]: [sum(sum_sales#111)#162, sum(number_sales#112)#163] -Results [6]: [channel#61, i_brand_id#113, i_class_id#114, null AS i_category_id#164, sum(sum_sales#111)#162 AS sum(sum_sales)#165, sum(number_sales#112)#163 AS sum(number_sales)#166] +Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] +Keys [3]: [channel#68, i_brand_id#124, i_class_id#125] +Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] +Aggregate Attributes [2]: [sum(sum_sales#122)#173, sum(number_sales#123)#174] +Results [6]: [channel#68, i_brand_id#124, i_class_id#125, null AS i_category_id#175, sum(sum_sales#122)#173 AS sum(sum_sales)#176, sum(number_sales#123)#174 AS sum(number_sales)#177] (133) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sum#170, isEmpty#171, count#172] +Output [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] (134) HashAggregate [codegen id : 187] -Input [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sum#170, isEmpty#171, count#172] -Keys [3]: [i_brand_id#167, i_class_id#168, i_category_id#169] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#175, count(1)#176] -Results [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#175 AS sales#56, count(1)#176 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#175 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177] +Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] +Keys [3]: [i_brand_id#178, i_class_id#179, i_category_id#180] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186, count(1)#187] +Results [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sales#63, count(1)#187 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] (135) Filter [codegen id : 187] -Input [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (136) Project [codegen id : 187] -Output [6]: [store AS channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sales#56, number_sales#57] -Input [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177] +Output [6]: [store AS channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64] +Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] (137) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] +Output [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] (138) HashAggregate [codegen id : 213] -Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] -Keys [3]: [i_brand_id#178, i_class_id#179, i_category_id#180] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186, count(1)#187] -Results [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sales#78, count(1)#187 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] +Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] +Keys [3]: [i_brand_id#189, i_class_id#190, i_category_id#191] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197, count(1)#198] +Results [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sales#87, count(1)#198 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] (139) Filter [codegen id : 213] -Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (140) Project [codegen id : 213] -Output [6]: [catalog AS channel#81, i_brand_id#178, i_class_id#179, i_category_id#180, sales#78, number_sales#79] -Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] +Output [6]: [catalog AS channel#90, i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88] +Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] (141) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] +Output [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum#203, isEmpty#204, count#205] (142) HashAggregate [codegen id : 239] -Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] -Keys [3]: [i_brand_id#189, i_class_id#190, i_category_id#191] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197, count(1)#198] -Results [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sales#98, count(1)#198 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] +Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum#203, isEmpty#204, count#205] +Keys [3]: [i_brand_id#200, i_class_id#201, i_category_id#202] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208, count(1)#209] +Results [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208 AS sales#109, count(1)#209 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] (143) Filter [codegen id : 239] -Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (144) Project [codegen id : 239] -Output [6]: [web AS channel#101, i_brand_id#189, i_class_id#190, i_category_id#191, sales#98, number_sales#99] -Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] +Output [6]: [web AS channel#112, i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110] +Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] (145) Union (146) HashAggregate [codegen id : 240] -Input [6]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sales#56, number_sales#57] -Keys [4]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169] -Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] -Aggregate Attributes [3]: [sum#200, isEmpty#201, sum#202] -Results [7]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sum#203, isEmpty#204, sum#205] +Input [6]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64] +Keys [4]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#211, isEmpty#212, sum#213] +Results [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] (147) Exchange -Input [7]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sum#203, isEmpty#204, sum#205] -Arguments: hashpartitioning(channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, 5), ENSURE_REQUIREMENTS, [id=#206] +Input [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] +Arguments: hashpartitioning(channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, 5), ENSURE_REQUIREMENTS, [id=#217] (148) HashAggregate [codegen id : 241] -Input [7]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sum#203, isEmpty#204, sum#205] -Keys [4]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169] -Functions [2]: [sum(sales#56), sum(number_sales#57)] -Aggregate Attributes [2]: [sum(sales#56)#207, sum(number_sales#57)#208] -Results [4]: [channel#61, i_brand_id#167, sum(sales#56)#207 AS sum_sales#111, sum(number_sales#57)#208 AS number_sales#112] +Input [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] +Keys [4]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#218, sum(number_sales#64)#219] +Results [4]: [channel#68, i_brand_id#178, sum(sales#63)#218 AS sum_sales#122, sum(number_sales#64)#219 AS number_sales#123] (149) HashAggregate [codegen id : 241] -Input [4]: [channel#61, i_brand_id#167, sum_sales#111, number_sales#112] -Keys [2]: [channel#61, i_brand_id#167] -Functions [2]: [partial_sum(sum_sales#111), partial_sum(number_sales#112)] -Aggregate Attributes [3]: [sum#209, isEmpty#210, sum#211] -Results [5]: [channel#61, i_brand_id#167, sum#212, isEmpty#213, sum#214] +Input [4]: [channel#68, i_brand_id#178, sum_sales#122, number_sales#123] +Keys [2]: [channel#68, i_brand_id#178] +Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] +Aggregate Attributes [3]: [sum#220, isEmpty#221, sum#222] +Results [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] (150) Exchange -Input [5]: [channel#61, i_brand_id#167, sum#212, isEmpty#213, sum#214] -Arguments: hashpartitioning(channel#61, i_brand_id#167, 5), ENSURE_REQUIREMENTS, [id=#215] +Input [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] +Arguments: hashpartitioning(channel#68, i_brand_id#178, 5), ENSURE_REQUIREMENTS, [id=#226] (151) HashAggregate [codegen id : 242] -Input [5]: [channel#61, i_brand_id#167, sum#212, isEmpty#213, sum#214] -Keys [2]: [channel#61, i_brand_id#167] -Functions [2]: [sum(sum_sales#111), sum(number_sales#112)] -Aggregate Attributes [2]: [sum(sum_sales#111)#216, sum(number_sales#112)#217] -Results [6]: [channel#61, i_brand_id#167, null AS i_class_id#218, null AS i_category_id#219, sum(sum_sales#111)#216 AS sum(sum_sales)#220, sum(number_sales#112)#217 AS sum(number_sales)#221] +Input [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] +Keys [2]: [channel#68, i_brand_id#178] +Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] +Aggregate Attributes [2]: [sum(sum_sales#122)#227, sum(number_sales#123)#228] +Results [6]: [channel#68, i_brand_id#178, null AS i_class_id#229, null AS i_category_id#230, sum(sum_sales#122)#227 AS sum(sum_sales)#231, sum(number_sales#123)#228 AS sum(number_sales)#232] (152) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sum#225, isEmpty#226, count#227] +Output [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] (153) HashAggregate [codegen id : 268] -Input [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sum#225, isEmpty#226, count#227] -Keys [3]: [i_brand_id#222, i_class_id#223, i_category_id#224] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#230, count(1)#231] -Results [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sales#56, count(1)#231 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232] +Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] +Keys [3]: [i_brand_id#233, i_class_id#234, i_category_id#235] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241, count(1)#242] +Results [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sales#63, count(1)#242 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] (154) Filter [codegen id : 268] -Input [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (155) Project [codegen id : 268] -Output [6]: [store AS channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sales#56, number_sales#57] -Input [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232] +Output [6]: [store AS channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64] +Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] (156) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] +Output [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] (157) HashAggregate [codegen id : 294] -Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] -Keys [3]: [i_brand_id#233, i_class_id#234, i_category_id#235] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241, count(1)#242] -Results [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sales#78, count(1)#242 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] +Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] +Keys [3]: [i_brand_id#244, i_class_id#245, i_category_id#246] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252, count(1)#253] +Results [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sales#87, count(1)#253 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] (158) Filter [codegen id : 294] -Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (159) Project [codegen id : 294] -Output [6]: [catalog AS channel#81, i_brand_id#233, i_class_id#234, i_category_id#235, sales#78, number_sales#79] -Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] +Output [6]: [catalog AS channel#90, i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88] +Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] (160) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] +Output [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#260] (161) HashAggregate [codegen id : 320] -Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] -Keys [3]: [i_brand_id#244, i_class_id#245, i_category_id#246] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252, count(1)#253] -Results [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sales#98, count(1)#253 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] +Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#260] +Keys [3]: [i_brand_id#255, i_class_id#256, i_category_id#257] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263, count(1)#264] +Results [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263 AS sales#109, count(1)#264 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] (162) Filter [codegen id : 320] -Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (163) Project [codegen id : 320] -Output [6]: [web AS channel#101, i_brand_id#244, i_class_id#245, i_category_id#246, sales#98, number_sales#99] -Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] +Output [6]: [web AS channel#112, i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110] +Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] (164) Union (165) HashAggregate [codegen id : 321] -Input [6]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sales#56, number_sales#57] -Keys [4]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224] -Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] -Aggregate Attributes [3]: [sum#255, isEmpty#256, sum#257] -Results [7]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sum#258, isEmpty#259, sum#260] +Input [6]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64] +Keys [4]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#266, isEmpty#267, sum#268] +Results [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] (166) Exchange -Input [7]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sum#258, isEmpty#259, sum#260] -Arguments: hashpartitioning(channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, 5), ENSURE_REQUIREMENTS, [id=#261] +Input [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] +Arguments: hashpartitioning(channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, 5), ENSURE_REQUIREMENTS, [id=#272] (167) HashAggregate [codegen id : 322] -Input [7]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sum#258, isEmpty#259, sum#260] -Keys [4]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224] -Functions [2]: [sum(sales#56), sum(number_sales#57)] -Aggregate Attributes [2]: [sum(sales#56)#262, sum(number_sales#57)#263] -Results [3]: [channel#61, sum(sales#56)#262 AS sum_sales#111, sum(number_sales#57)#263 AS number_sales#112] +Input [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] +Keys [4]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#273, sum(number_sales#64)#274] +Results [3]: [channel#68, sum(sales#63)#273 AS sum_sales#122, sum(number_sales#64)#274 AS number_sales#123] (168) HashAggregate [codegen id : 322] -Input [3]: [channel#61, sum_sales#111, number_sales#112] -Keys [1]: [channel#61] -Functions [2]: [partial_sum(sum_sales#111), partial_sum(number_sales#112)] -Aggregate Attributes [3]: [sum#264, isEmpty#265, sum#266] -Results [4]: [channel#61, sum#267, isEmpty#268, sum#269] +Input [3]: [channel#68, sum_sales#122, number_sales#123] +Keys [1]: [channel#68] +Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] +Aggregate Attributes [3]: [sum#275, isEmpty#276, sum#277] +Results [4]: [channel#68, sum#278, isEmpty#279, sum#280] (169) Exchange -Input [4]: [channel#61, sum#267, isEmpty#268, sum#269] -Arguments: hashpartitioning(channel#61, 5), ENSURE_REQUIREMENTS, [id=#270] +Input [4]: [channel#68, sum#278, isEmpty#279, sum#280] +Arguments: hashpartitioning(channel#68, 5), ENSURE_REQUIREMENTS, [id=#281] (170) HashAggregate [codegen id : 323] -Input [4]: [channel#61, sum#267, isEmpty#268, sum#269] -Keys [1]: [channel#61] -Functions [2]: [sum(sum_sales#111), sum(number_sales#112)] -Aggregate Attributes [2]: [sum(sum_sales#111)#271, sum(number_sales#112)#272] -Results [6]: [channel#61, null AS i_brand_id#273, null AS i_class_id#274, null AS i_category_id#275, sum(sum_sales#111)#271 AS sum(sum_sales)#276, sum(number_sales#112)#272 AS sum(number_sales)#277] +Input [4]: [channel#68, sum#278, isEmpty#279, sum#280] +Keys [1]: [channel#68] +Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] +Aggregate Attributes [2]: [sum(sum_sales#122)#282, sum(number_sales#123)#283] +Results [6]: [channel#68, null AS i_brand_id#284, null AS i_class_id#285, null AS i_category_id#286, sum(sum_sales#122)#282 AS sum(sum_sales)#287, sum(number_sales#123)#283 AS sum(number_sales)#288] (171) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sum#281, isEmpty#282, count#283] +Output [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] (172) HashAggregate [codegen id : 349] -Input [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sum#281, isEmpty#282, count#283] -Keys [3]: [i_brand_id#278, i_class_id#279, i_category_id#280] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#286, count(1)#287] -Results [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#286 AS sales#56, count(1)#287 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#286 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288] +Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] +Keys [3]: [i_brand_id#289, i_class_id#290, i_category_id#291] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297, count(1)#298] +Results [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sales#63, count(1)#298 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] (173) Filter [codegen id : 349] -Input [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (174) Project [codegen id : 349] -Output [6]: [store AS channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sales#56, number_sales#57] -Input [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288] +Output [6]: [store AS channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64] +Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] (175) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] +Output [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] (176) HashAggregate [codegen id : 375] -Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] -Keys [3]: [i_brand_id#289, i_class_id#290, i_category_id#291] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297, count(1)#298] -Results [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sales#78, count(1)#298 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] +Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] +Keys [3]: [i_brand_id#300, i_class_id#301, i_category_id#302] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308, count(1)#309] +Results [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sales#87, count(1)#309 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] (177) Filter [codegen id : 375] -Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (178) Project [codegen id : 375] -Output [6]: [catalog AS channel#81, i_brand_id#289, i_class_id#290, i_category_id#291, sales#78, number_sales#79] -Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] +Output [6]: [catalog AS channel#90, i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88] +Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] (179) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] +Output [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum#314, isEmpty#315, count#316] (180) HashAggregate [codegen id : 401] -Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] -Keys [3]: [i_brand_id#300, i_class_id#301, i_category_id#302] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308, count(1)#309] -Results [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sales#98, count(1)#309 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] +Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum#314, isEmpty#315, count#316] +Keys [3]: [i_brand_id#311, i_class_id#312, i_category_id#313] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319, count(1)#320] +Results [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319 AS sales#109, count(1)#320 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] (181) Filter [codegen id : 401] -Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (182) Project [codegen id : 401] -Output [6]: [web AS channel#101, i_brand_id#300, i_class_id#301, i_category_id#302, sales#98, number_sales#99] -Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] +Output [6]: [web AS channel#112, i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110] +Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] (183) Union (184) HashAggregate [codegen id : 402] -Input [6]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sales#56, number_sales#57] -Keys [4]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280] -Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] -Aggregate Attributes [3]: [sum#311, isEmpty#312, sum#313] -Results [7]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sum#314, isEmpty#315, sum#316] +Input [6]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64] +Keys [4]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#322, isEmpty#323, sum#324] +Results [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] (185) Exchange -Input [7]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sum#314, isEmpty#315, sum#316] -Arguments: hashpartitioning(channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, 5), ENSURE_REQUIREMENTS, [id=#317] +Input [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] +Arguments: hashpartitioning(channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, 5), ENSURE_REQUIREMENTS, [id=#328] (186) HashAggregate [codegen id : 403] -Input [7]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sum#314, isEmpty#315, sum#316] -Keys [4]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280] -Functions [2]: [sum(sales#56), sum(number_sales#57)] -Aggregate Attributes [2]: [sum(sales#56)#318, sum(number_sales#57)#319] -Results [2]: [sum(sales#56)#318 AS sum_sales#111, sum(number_sales#57)#319 AS number_sales#112] +Input [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] +Keys [4]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#329, sum(number_sales#64)#330] +Results [2]: [sum(sales#63)#329 AS sum_sales#122, sum(number_sales#64)#330 AS number_sales#123] (187) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#111, number_sales#112] +Input [2]: [sum_sales#122, number_sales#123] Keys: [] -Functions [2]: [partial_sum(sum_sales#111), partial_sum(number_sales#112)] -Aggregate Attributes [3]: [sum#320, isEmpty#321, sum#322] -Results [3]: [sum#323, isEmpty#324, sum#325] +Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] +Aggregate Attributes [3]: [sum#331, isEmpty#332, sum#333] +Results [3]: [sum#334, isEmpty#335, sum#336] (188) Exchange -Input [3]: [sum#323, isEmpty#324, sum#325] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#326] +Input [3]: [sum#334, isEmpty#335, sum#336] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#337] (189) HashAggregate [codegen id : 404] -Input [3]: [sum#323, isEmpty#324, sum#325] +Input [3]: [sum#334, isEmpty#335, sum#336] Keys: [] -Functions [2]: [sum(sum_sales#111), sum(number_sales#112)] -Aggregate Attributes [2]: [sum(sum_sales#111)#327, sum(number_sales#112)#328] -Results [6]: [null AS channel#329, null AS i_brand_id#330, null AS i_class_id#331, null AS i_category_id#332, sum(sum_sales#111)#327 AS sum(sum_sales)#333, sum(number_sales#112)#328 AS sum(number_sales)#334] +Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] +Aggregate Attributes [2]: [sum(sum_sales#122)#338, sum(number_sales#123)#339] +Results [6]: [null AS channel#340, null AS i_brand_id#341, null AS i_class_id#342, null AS i_category_id#343, sum(sum_sales#122)#338 AS sum(sum_sales)#344, sum(number_sales#123)#339 AS sum(number_sales)#345] (190) Union (191) HashAggregate [codegen id : 405] -Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] -Keys [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Keys [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] (192) Exchange -Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] -Arguments: hashpartitioning(channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112, 5), ENSURE_REQUIREMENTS, [id=#335] +Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Arguments: hashpartitioning(channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123, 5), ENSURE_REQUIREMENTS, [id=#346] (193) HashAggregate [codegen id : 406] -Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] -Keys [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Keys [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] (194) TakeOrderedAndProject -Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] -Arguments: 100, [channel#61 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Arguments: 100, [channel#68 ASC NULLS FIRST, i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#59, [id=#60] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#66, [id=#67] * HashAggregate (221) +- Exchange (220) +- * HashAggregate (219) @@ -1145,193 +1145,193 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (195) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#336)] +PartitionFilters: [isnotnull(ss_sold_date_sk#349), dynamicpruningexpression(ss_sold_date_sk#349 IN dynamicpruning#350)] ReadSchema: struct (196) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349] (197) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_year#44] +Output [2]: [d_date_sk#351, d_year#352] 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 (198) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#44] +Input [2]: [d_date_sk#351, d_year#352] (199) Filter [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#44] -Condition : (((isnotnull(d_year#44) AND (d_year#44 >= 1999)) AND (d_year#44 <= 2001)) AND isnotnull(d_date_sk#27)) +Input [2]: [d_date_sk#351, d_year#352] +Condition : (((isnotnull(d_year#352) AND (d_year#352 >= 1999)) AND (d_year#352 <= 2001)) AND isnotnull(d_date_sk#351)) (200) Project [codegen id : 1] -Output [1]: [d_date_sk#27] -Input [2]: [d_date_sk#27, d_year#44] +Output [1]: [d_date_sk#351] +Input [2]: [d_date_sk#351, d_year#352] (201) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#337] +Input [1]: [d_date_sk#351] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#353] (202) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Left keys [1]: [ss_sold_date_sk#349] +Right keys [1]: [d_date_sk#351] Join condition: None (203) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#338, ss_list_price#3 AS list_price#339] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#27] +Output [2]: [ss_quantity#347 AS quantity#354, ss_list_price#348 AS list_price#355] +Input [4]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349, d_date_sk#351] (204) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Output [3]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#340)] +PartitionFilters: [isnotnull(cs_sold_date_sk#358), dynamicpruningexpression(cs_sold_date_sk#358 IN dynamicpruning#359)] ReadSchema: struct (205) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Input [3]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358] (206) Scan parquet default.date_dim -Output [2]: [d_date_sk#341, d_year#342] +Output [2]: [d_date_sk#360, d_year#361] 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 (207) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#341, d_year#342] +Input [2]: [d_date_sk#360, d_year#361] (208) Filter [codegen id : 3] -Input [2]: [d_date_sk#341, d_year#342] -Condition : (((isnotnull(d_year#342) AND (d_year#342 >= 1998)) AND (d_year#342 <= 2000)) AND isnotnull(d_date_sk#341)) +Input [2]: [d_date_sk#360, d_year#361] +Condition : (((isnotnull(d_year#361) AND (d_year#361 >= 1998)) AND (d_year#361 <= 2000)) AND isnotnull(d_date_sk#360)) (209) Project [codegen id : 3] -Output [1]: [d_date_sk#341] -Input [2]: [d_date_sk#341, d_year#342] +Output [1]: [d_date_sk#360] +Input [2]: [d_date_sk#360, d_year#361] (210) BroadcastExchange -Input [1]: [d_date_sk#341] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#343] +Input [1]: [d_date_sk#360] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#362] (211) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#341] +Left keys [1]: [cs_sold_date_sk#358] +Right keys [1]: [d_date_sk#360] Join condition: None (212) Project [codegen id : 4] -Output [2]: [cs_quantity#62 AS quantity#344, cs_list_price#63 AS list_price#345] -Input [4]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, d_date_sk#341] +Output [2]: [cs_quantity#356 AS quantity#363, cs_list_price#357 AS list_price#364] +Input [4]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358, d_date_sk#360] (213) Scan parquet default.web_sales -Output [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Output [3]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#340)] +PartitionFilters: [isnotnull(ws_sold_date_sk#367), dynamicpruningexpression(ws_sold_date_sk#367 IN dynamicpruning#359)] ReadSchema: struct (214) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Input [3]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367] (215) ReusedExchange [Reuses operator id: 210] -Output [1]: [d_date_sk#346] +Output [1]: [d_date_sk#368] (216) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#346] +Left keys [1]: [ws_sold_date_sk#367] +Right keys [1]: [d_date_sk#368] Join condition: None (217) Project [codegen id : 6] -Output [2]: [ws_quantity#82 AS quantity#347, ws_list_price#83 AS list_price#348] -Input [4]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, d_date_sk#346] +Output [2]: [ws_quantity#365 AS quantity#369, ws_list_price#366 AS list_price#370] +Input [4]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367, d_date_sk#368] (218) Union (219) HashAggregate [codegen id : 7] -Input [2]: [quantity#338, list_price#339] +Input [2]: [quantity#354, list_price#355] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#338 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#339 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#349, count#350] -Results [2]: [sum#351, count#352] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#371, count#372] +Results [2]: [sum#373, count#374] (220) Exchange -Input [2]: [sum#351, count#352] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#353] +Input [2]: [sum#373, count#374] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#375] (221) HashAggregate [codegen id : 8] -Input [2]: [sum#351, count#352] +Input [2]: [sum#373, count#374] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#338 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#339 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#338 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#339 as decimal(12,2)))), DecimalType(18,2), true))#354] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#338 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#339 as decimal(12,2)))), DecimalType(18,2), true))#354 AS average_sales#355] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))#376] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))#376 AS average_sales#377] -Subquery:2 Hosting operator id = 195 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#336 +Subquery:2 Hosting operator id = 195 Hosting Expression = ss_sold_date_sk#349 IN dynamicpruning#350 ReusedExchange (222) (222) ReusedExchange [Reuses operator id: 201] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#351] -Subquery:3 Hosting operator id = 204 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#340 +Subquery:3 Hosting operator id = 204 Hosting Expression = cs_sold_date_sk#358 IN dynamicpruning#359 ReusedExchange (223) (223) ReusedExchange [Reuses operator id: 210] -Output [1]: [d_date_sk#341] +Output [1]: [d_date_sk#360] -Subquery:4 Hosting operator id = 213 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#340 +Subquery:4 Hosting operator id = 213 Hosting Expression = ws_sold_date_sk#367 IN dynamicpruning#359 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (224) (224) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#50] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 ReusedExchange (225) (225) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#29] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#5 -Subquery:13 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:13 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:14 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:14 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:15 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:15 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:16 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:16 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:17 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:17 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:18 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:18 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:19 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:19 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:20 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:20 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:21 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:21 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:22 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:22 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:23 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:23 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:24 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:24 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] 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 15f74a7802b27..56fa48006d894 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 @@ -393,484 +393,484 @@ Aggregate Attributes [7]: [avg(agg1#37)#73, avg(agg2#38)#74, avg(agg3#39)#75, av Results [11]: [i_item_id#20, ca_country#34, ca_state#33, ca_county#32, avg(agg1#37)#73 AS agg1#80, avg(agg2#38)#74 AS agg2#81, avg(agg3#39)#75 AS agg3#82, avg(agg4#40)#76 AS agg4#83, avg(agg5#41)#77 AS agg5#84, avg(agg6#42)#78 AS agg6#85, avg(agg7#43)#79 AS agg7#86] (52) ReusedExchange [Reuses operator id: unknown] -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#20] +Output [8]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94] (53) Sort [codegen id : 19] -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#20] -Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94] +Arguments: [cs_bill_customer_sk#87 ASC NULLS FIRST], false, 0 (54) Scan parquet default.customer -Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 21] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] (56) Filter [codegen id : 21] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] +Condition : (((c_birth_month#98 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#95)) AND isnotnull(c_current_cdemo_sk#96)) AND isnotnull(c_current_addr_sk#97)) (57) Project [codegen id : 21] -Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [4]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#99] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] (58) Scan parquet default.customer_address -Output [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Output [3]: [ca_address_sk#100, ca_state#101, ca_country#102] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] (60) Filter [codegen id : 20] -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] -Condition : (ca_state#33 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#31)) +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Condition : (ca_state#101 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#100)) (61) BroadcastExchange -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#87] +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] (62) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [c_current_addr_sk#25] -Right keys [1]: [ca_address_sk#31] +Left keys [1]: [c_current_addr_sk#97] +Right keys [1]: [ca_address_sk#100] Join condition: None (63) Project [codegen id : 21] -Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34] -Input [7]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31, ca_state#33, ca_country#34] +Output [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Input [7]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#99, ca_address_sk#100, ca_state#101, ca_country#102] (64) Exchange -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34] -Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: hashpartitioning(c_current_cdemo_sk#96, 5), ENSURE_REQUIREMENTS, [id=#104] (65) Sort [codegen id : 22] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34] -Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: [c_current_cdemo_sk#96 ASC NULLS FIRST], false, 0 (66) ReusedExchange [Reuses operator id: 35] -Output [1]: [cd_demo_sk#89] +Output [1]: [cd_demo_sk#105] (67) Sort [codegen id : 24] -Input [1]: [cd_demo_sk#89] -Arguments: [cd_demo_sk#89 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#105] +Arguments: [cd_demo_sk#105 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 25] -Left keys [1]: [c_current_cdemo_sk#24] -Right keys [1]: [cd_demo_sk#89] +Left keys [1]: [c_current_cdemo_sk#96] +Right keys [1]: [cd_demo_sk#105] Join condition: None (69) Project [codegen id : 25] -Output [4]: [c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] -Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34, cd_demo_sk#89] +Output [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Input [6]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102, cd_demo_sk#105] (70) Exchange -Input [4]: [c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: hashpartitioning(c_customer_sk#95, 5), ENSURE_REQUIREMENTS, [id=#106] (71) Sort [codegen id : 26] -Input [4]: [c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] -Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: [c_customer_sk#95 ASC NULLS FIRST], false, 0 (72) SortMergeJoin [codegen id : 27] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#23] +Left keys [1]: [cs_bill_customer_sk#87] +Right keys [1]: [c_customer_sk#95] Join condition: None (73) Project [codegen id : 27] -Output [10]: [i_item_id#20, ca_country#34, ca_state#33, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] -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#20, c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] +Output [10]: [i_item_id#94, ca_country#102, ca_state#101, cast(cs_quantity#88 as decimal(12,2)) AS agg1#37, cast(cs_list_price#89 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#91 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#90 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#92 as decimal(12,2)) AS agg5#41, cast(c_birth_year#99 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#93 as decimal(12,2)) AS agg7#43] +Input [12]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94, c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] (74) HashAggregate [codegen id : 27] -Input [10]: [i_item_id#20, ca_country#34, ca_state#33, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] -Keys [3]: [i_item_id#20, ca_country#34, ca_state#33] +Input [10]: [i_item_id#94, ca_country#102, ca_state#101, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [3]: [i_item_id#94, ca_country#102, ca_state#101] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100, sum#101, count#102, sum#103, count#104] -Results [17]: [i_item_id#20, ca_country#34, ca_state#33, 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] +Aggregate Attributes [14]: [sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] +Results [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] (75) Exchange -Input [17]: [i_item_id#20, ca_country#34, ca_state#33, 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] -Arguments: hashpartitioning(i_item_id#20, ca_country#34, ca_state#33, 5), ENSURE_REQUIREMENTS, [id=#119] +Input [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] +Arguments: hashpartitioning(i_item_id#94, ca_country#102, ca_state#101, 5), ENSURE_REQUIREMENTS, [id=#135] (76) HashAggregate [codegen id : 28] -Input [17]: [i_item_id#20, ca_country#34, ca_state#33, 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] -Keys [3]: [i_item_id#20, ca_country#34, ca_state#33] +Input [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] +Keys [3]: [i_item_id#94, ca_country#102, ca_state#101] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#120, avg(agg2#38)#121, avg(agg3#39)#122, avg(agg4#40)#123, avg(agg5#41)#124, avg(agg6#42)#125, avg(agg7#43)#126] -Results [11]: [i_item_id#20, ca_country#34, ca_state#33, null AS county#127, avg(agg1#37)#120 AS agg1#128, avg(agg2#38)#121 AS agg2#129, avg(agg3#39)#122 AS agg3#130, avg(agg4#40)#123 AS agg4#131, avg(agg5#41)#124 AS agg5#132, avg(agg6#42)#125 AS agg6#133, avg(agg7#43)#126 AS agg7#134] +Aggregate Attributes [7]: [avg(agg1#37)#136, avg(agg2#38)#137, avg(agg3#39)#138, avg(agg4#40)#139, avg(agg5#41)#140, avg(agg6#42)#141, avg(agg7#43)#142] +Results [11]: [i_item_id#94, ca_country#102, ca_state#101, null AS county#143, avg(agg1#37)#136 AS agg1#144, avg(agg2#38)#137 AS agg2#145, avg(agg3#39)#138 AS agg3#146, avg(agg4#40)#139 AS agg4#147, avg(agg5#41)#140 AS agg5#148, avg(agg6#42)#141 AS agg6#149, avg(agg7#43)#142 AS agg7#150] (77) ReusedExchange [Reuses operator id: unknown] -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#20] +Output [8]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158] (78) Sort [codegen id : 33] -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#20] -Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158] +Arguments: [cs_bill_customer_sk#151 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer -Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (80) ColumnarToRow [codegen id : 35] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] (81) Filter [codegen id : 35] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] +Condition : (((c_birth_month#162 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#159)) AND isnotnull(c_current_cdemo_sk#160)) AND isnotnull(c_current_addr_sk#161)) (82) Project [codegen id : 35] -Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_year#163] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] (83) Scan parquet default.customer_address -Output [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Output [3]: [ca_address_sk#164, ca_state#165, ca_country#166] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (84) ColumnarToRow [codegen id : 34] -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] (85) Filter [codegen id : 34] -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] -Condition : (ca_state#33 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#31)) +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] +Condition : (ca_state#165 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#164)) (86) Project [codegen id : 34] -Output [2]: [ca_address_sk#31, ca_country#34] -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Output [2]: [ca_address_sk#164, ca_country#166] +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] (87) BroadcastExchange -Input [2]: [ca_address_sk#31, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#135] +Input [2]: [ca_address_sk#164, ca_country#166] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#167] (88) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [c_current_addr_sk#25] -Right keys [1]: [ca_address_sk#31] +Left keys [1]: [c_current_addr_sk#161] +Right keys [1]: [ca_address_sk#164] Join condition: None (89) Project [codegen id : 35] -Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34] -Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31, ca_country#34] +Output [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Input [6]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_year#163, ca_address_sk#164, ca_country#166] (90) Exchange -Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34] -Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#136] +Input [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Arguments: hashpartitioning(c_current_cdemo_sk#160, 5), ENSURE_REQUIREMENTS, [id=#168] (91) Sort [codegen id : 36] -Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34] -Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Arguments: [c_current_cdemo_sk#160 ASC NULLS FIRST], false, 0 (92) ReusedExchange [Reuses operator id: 35] -Output [1]: [cd_demo_sk#137] +Output [1]: [cd_demo_sk#169] (93) Sort [codegen id : 38] -Input [1]: [cd_demo_sk#137] -Arguments: [cd_demo_sk#137 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#169] +Arguments: [cd_demo_sk#169 ASC NULLS FIRST], false, 0 (94) SortMergeJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#24] -Right keys [1]: [cd_demo_sk#137] +Left keys [1]: [c_current_cdemo_sk#160] +Right keys [1]: [cd_demo_sk#169] Join condition: None (95) Project [codegen id : 39] -Output [3]: [c_customer_sk#23, c_birth_year#27, ca_country#34] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34, cd_demo_sk#137] +Output [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166, cd_demo_sk#169] (96) Exchange -Input [3]: [c_customer_sk#23, c_birth_year#27, ca_country#34] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#138] +Input [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Arguments: hashpartitioning(c_customer_sk#159, 5), ENSURE_REQUIREMENTS, [id=#170] (97) Sort [codegen id : 40] -Input [3]: [c_customer_sk#23, c_birth_year#27, ca_country#34] -Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Arguments: [c_customer_sk#159 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 41] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#23] +Left keys [1]: [cs_bill_customer_sk#151] +Right keys [1]: [c_customer_sk#159] Join condition: None (99) Project [codegen id : 41] -Output [9]: [i_item_id#20, ca_country#34, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] -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#20, c_customer_sk#23, c_birth_year#27, ca_country#34] +Output [9]: [i_item_id#158, ca_country#166, cast(cs_quantity#152 as decimal(12,2)) AS agg1#37, cast(cs_list_price#153 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#155 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#154 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#156 as decimal(12,2)) AS agg5#41, cast(c_birth_year#163 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#157 as decimal(12,2)) AS agg7#43] +Input [11]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158, c_customer_sk#159, c_birth_year#163, ca_country#166] (100) HashAggregate [codegen id : 41] -Input [9]: [i_item_id#20, ca_country#34, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] -Keys [2]: [i_item_id#20, ca_country#34] +Input [9]: [i_item_id#158, ca_country#166, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [2]: [i_item_id#158, ca_country#166] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#139, count#140, sum#141, count#142, sum#143, count#144, sum#145, count#146, sum#147, count#148, sum#149, count#150, sum#151, count#152] -Results [16]: [i_item_id#20, ca_country#34, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] +Aggregate Attributes [14]: [sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178, sum#179, count#180, sum#181, count#182, sum#183, count#184] +Results [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] (101) Exchange -Input [16]: [i_item_id#20, ca_country#34, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] -Arguments: hashpartitioning(i_item_id#20, ca_country#34, 5), ENSURE_REQUIREMENTS, [id=#167] +Input [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] +Arguments: hashpartitioning(i_item_id#158, ca_country#166, 5), ENSURE_REQUIREMENTS, [id=#199] (102) HashAggregate [codegen id : 42] -Input [16]: [i_item_id#20, ca_country#34, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] -Keys [2]: [i_item_id#20, ca_country#34] +Input [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] +Keys [2]: [i_item_id#158, ca_country#166] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#168, avg(agg2#38)#169, avg(agg3#39)#170, avg(agg4#40)#171, avg(agg5#41)#172, avg(agg6#42)#173, avg(agg7#43)#174] -Results [11]: [i_item_id#20, ca_country#34, null AS ca_state#175, null AS county#176, avg(agg1#37)#168 AS agg1#177, avg(agg2#38)#169 AS agg2#178, avg(agg3#39)#170 AS agg3#179, avg(agg4#40)#171 AS agg4#180, avg(agg5#41)#172 AS agg5#181, avg(agg6#42)#173 AS agg6#182, avg(agg7#43)#174 AS agg7#183] +Aggregate Attributes [7]: [avg(agg1#37)#200, avg(agg2#38)#201, avg(agg3#39)#202, avg(agg4#40)#203, avg(agg5#41)#204, avg(agg6#42)#205, avg(agg7#43)#206] +Results [11]: [i_item_id#158, ca_country#166, null AS ca_state#207, null AS county#208, avg(agg1#37)#200 AS agg1#209, avg(agg2#38)#201 AS agg2#210, avg(agg3#39)#202 AS agg3#211, avg(agg4#40)#203 AS agg4#212, avg(agg5#41)#204 AS agg5#213, avg(agg6#42)#205 AS agg6#214, avg(agg7#43)#206 AS agg7#215] (103) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#224), dynamicpruningexpression(cs_sold_date_sk#224 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 49] -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] +Input [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] (105) Filter [codegen id : 49] -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)) +Input [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] +Condition : ((isnotnull(cs_bill_cdemo_sk#217) AND isnotnull(cs_bill_customer_sk#216)) AND isnotnull(cs_item_sk#218)) (106) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#225, cd_dep_count#226] (107) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#217] +Right keys [1]: [cd_demo_sk#225] Join condition: None (108) Project [codegen id : 49] -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] +Output [9]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_dep_count#226] +Input [11]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_demo_sk#225, cd_dep_count#226] (109) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#227] (110) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#16] +Left keys [1]: [cs_sold_date_sk#224] +Right keys [1]: [d_date_sk#227] Join condition: None (111) Project [codegen id : 49] -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] +Output [8]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226] +Input [10]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_dep_count#226, d_date_sk#227] (112) Scan parquet default.customer -Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (113) ColumnarToRow [codegen id : 46] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] (114) Filter [codegen id : 46] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] +Condition : (((c_birth_month#231 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#228)) AND isnotnull(c_current_cdemo_sk#229)) AND isnotnull(c_current_addr_sk#230)) (115) Project [codegen id : 46] -Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [4]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_year#232] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] (116) Scan parquet default.customer_address -Output [2]: [ca_address_sk#31, ca_state#33] +Output [2]: [ca_address_sk#233, ca_state#234] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (117) ColumnarToRow [codegen id : 45] -Input [2]: [ca_address_sk#31, ca_state#33] +Input [2]: [ca_address_sk#233, ca_state#234] (118) Filter [codegen id : 45] -Input [2]: [ca_address_sk#31, ca_state#33] -Condition : (ca_state#33 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#31)) +Input [2]: [ca_address_sk#233, ca_state#234] +Condition : (ca_state#234 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#233)) (119) Project [codegen id : 45] -Output [1]: [ca_address_sk#31] -Input [2]: [ca_address_sk#31, ca_state#33] +Output [1]: [ca_address_sk#233] +Input [2]: [ca_address_sk#233, ca_state#234] (120) BroadcastExchange -Input [1]: [ca_address_sk#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#184] +Input [1]: [ca_address_sk#233] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#235] (121) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [c_current_addr_sk#25] -Right keys [1]: [ca_address_sk#31] +Left keys [1]: [c_current_addr_sk#230] +Right keys [1]: [ca_address_sk#233] Join condition: None (122) Project [codegen id : 46] -Output [3]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31] +Output [3]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_year#232, ca_address_sk#233] (123) BroadcastExchange -Input [3]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#185] +Input [3]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#236] (124) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#186] +Output [1]: [cd_demo_sk#237] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (125) ColumnarToRow -Input [1]: [cd_demo_sk#186] +Input [1]: [cd_demo_sk#237] (126) Filter -Input [1]: [cd_demo_sk#186] -Condition : isnotnull(cd_demo_sk#186) +Input [1]: [cd_demo_sk#237] +Condition : isnotnull(cd_demo_sk#237) (127) BroadcastHashJoin [codegen id : 47] -Left keys [1]: [c_current_cdemo_sk#24] -Right keys [1]: [cd_demo_sk#186] +Left keys [1]: [c_current_cdemo_sk#229] +Right keys [1]: [cd_demo_sk#237] Join condition: None (128) Project [codegen id : 47] -Output [2]: [c_customer_sk#23, c_birth_year#27] -Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, cd_demo_sk#186] +Output [2]: [c_customer_sk#228, c_birth_year#232] +Input [4]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232, cd_demo_sk#237] (129) BroadcastExchange -Input [2]: [c_customer_sk#23, c_birth_year#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#187] +Input [2]: [c_customer_sk#228, c_birth_year#232] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#238] (130) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#23] +Left keys [1]: [cs_bill_customer_sk#216] +Right keys [1]: [c_customer_sk#228] Join condition: None (131) Project [codegen id : 49] -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#27] -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#23, c_birth_year#27] +Output [8]: [cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_birth_year#232] +Input [10]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_customer_sk#228, c_birth_year#232] (132) ReusedExchange [Reuses operator id: 21] -Output [2]: [i_item_sk#19, i_item_id#20] +Output [2]: [i_item_sk#239, i_item_id#240] (133) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#19] +Left keys [1]: [cs_item_sk#218] +Right keys [1]: [i_item_sk#239] Join condition: None (134) Project [codegen id : 49] -Output [8]: [i_item_id#20, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] -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#27, i_item_sk#19, i_item_id#20] +Output [8]: [i_item_id#240, cast(cs_quantity#219 as decimal(12,2)) AS agg1#37, cast(cs_list_price#220 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#222 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#221 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#223 as decimal(12,2)) AS agg5#41, cast(c_birth_year#232 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#226 as decimal(12,2)) AS agg7#43] +Input [10]: [cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_birth_year#232, i_item_sk#239, i_item_id#240] (135) HashAggregate [codegen id : 49] -Input [8]: [i_item_id#20, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] -Keys [1]: [i_item_id#20] +Input [8]: [i_item_id#240, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [1]: [i_item_id#240] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201] -Results [15]: [i_item_id#20, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215] +Aggregate Attributes [14]: [sum#241, count#242, sum#243, count#244, sum#245, count#246, sum#247, count#248, sum#249, count#250, sum#251, count#252, sum#253, count#254] +Results [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] (136) Exchange -Input [15]: [i_item_id#20, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215] -Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, [id=#216] +Input [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] +Arguments: hashpartitioning(i_item_id#240, 5), ENSURE_REQUIREMENTS, [id=#269] (137) HashAggregate [codegen id : 50] -Input [15]: [i_item_id#20, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215] -Keys [1]: [i_item_id#20] +Input [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] +Keys [1]: [i_item_id#240] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#217, avg(agg2#38)#218, avg(agg3#39)#219, avg(agg4#40)#220, avg(agg5#41)#221, avg(agg6#42)#222, avg(agg7#43)#223] -Results [11]: [i_item_id#20, null AS ca_country#224, null AS ca_state#225, null AS county#226, avg(agg1#37)#217 AS agg1#227, avg(agg2#38)#218 AS agg2#228, avg(agg3#39)#219 AS agg3#229, avg(agg4#40)#220 AS agg4#230, avg(agg5#41)#221 AS agg5#231, avg(agg6#42)#222 AS agg6#232, avg(agg7#43)#223 AS agg7#233] +Aggregate Attributes [7]: [avg(agg1#37)#270, avg(agg2#38)#271, avg(agg3#39)#272, avg(agg4#40)#273, avg(agg5#41)#274, avg(agg6#42)#275, avg(agg7#43)#276] +Results [11]: [i_item_id#240, null AS ca_country#277, null AS ca_state#278, null AS county#279, avg(agg1#37)#270 AS agg1#280, avg(agg2#38)#271 AS agg2#281, avg(agg3#39)#272 AS agg3#282, avg(agg4#40)#273 AS agg4#283, avg(agg5#41)#274 AS agg5#284, avg(agg6#42)#275 AS agg6#285, avg(agg7#43)#276 AS agg7#286] (138) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#295), dynamicpruningexpression(cs_sold_date_sk#295 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (139) ColumnarToRow [codegen id : 57] -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] +Input [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] (140) Filter [codegen id : 57] -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)) +Input [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] +Condition : ((isnotnull(cs_bill_cdemo_sk#288) AND isnotnull(cs_bill_customer_sk#287)) AND isnotnull(cs_item_sk#289)) (141) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#296, cd_dep_count#297] (142) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#288] +Right keys [1]: [cd_demo_sk#296] Join condition: None (143) Project [codegen id : 57] -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] +Output [9]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_dep_count#297] +Input [11]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_demo_sk#296, cd_dep_count#297] (144) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#298] (145) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#16] +Left keys [1]: [cs_sold_date_sk#295] +Right keys [1]: [d_date_sk#298] Join condition: None (146) Project [codegen id : 57] -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] +Output [8]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297] +Input [10]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_dep_count#297, d_date_sk#298] (147) Scan parquet default.item -Output [1]: [i_item_sk#19] +Output [1]: [i_item_sk#299] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (148) ColumnarToRow [codegen id : 53] -Input [1]: [i_item_sk#19] +Input [1]: [i_item_sk#299] (149) Filter [codegen id : 53] -Input [1]: [i_item_sk#19] -Condition : isnotnull(i_item_sk#19) +Input [1]: [i_item_sk#299] +Condition : isnotnull(i_item_sk#299) (150) BroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#234] +Input [1]: [i_item_sk#299] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#300] (151) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#19] +Left keys [1]: [cs_item_sk#289] +Right keys [1]: [i_item_sk#299] Join condition: None (152) Project [codegen id : 57] -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#19] +Output [7]: [cs_bill_customer_sk#287, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297] +Input [9]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297, i_item_sk#299] (153) ReusedExchange [Reuses operator id: 129] -Output [2]: [c_customer_sk#23, c_birth_year#27] +Output [2]: [c_customer_sk#301, c_birth_year#302] (154) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#23] +Left keys [1]: [cs_bill_customer_sk#287] +Right keys [1]: [c_customer_sk#301] Join condition: None (155) Project [codegen id : 57] -Output [7]: [cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] -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#23, c_birth_year#27] +Output [7]: [cast(cs_quantity#290 as decimal(12,2)) AS agg1#37, cast(cs_list_price#291 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#293 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#292 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#294 as decimal(12,2)) AS agg5#41, cast(c_birth_year#302 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#297 as decimal(12,2)) AS agg7#43] +Input [9]: [cs_bill_customer_sk#287, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297, c_customer_sk#301, c_birth_year#302] (156) HashAggregate [codegen id : 57] Input [7]: [agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] Keys: [] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#235, count#236, sum#237, count#238, sum#239, count#240, sum#241, count#242, sum#243, count#244, sum#245, count#246, sum#247, count#248] -Results [14]: [sum#249, count#250, sum#251, count#252, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262] +Aggregate Attributes [14]: [sum#303, count#304, sum#305, count#306, sum#307, count#308, sum#309, count#310, sum#311, count#312, sum#313, count#314, sum#315, count#316] +Results [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] (157) Exchange -Input [14]: [sum#249, count#250, sum#251, count#252, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#263] +Input [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#331] (158) HashAggregate [codegen id : 58] -Input [14]: [sum#249, count#250, sum#251, count#252, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262] +Input [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] Keys: [] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#264, avg(agg2#38)#265, avg(agg3#39)#266, avg(agg4#40)#267, avg(agg5#41)#268, avg(agg6#42)#269, avg(agg7#43)#270] -Results [11]: [null AS i_item_id#271, null AS ca_country#272, null AS ca_state#273, null AS county#274, avg(agg1#37)#264 AS agg1#275, avg(agg2#38)#265 AS agg2#276, avg(agg3#39)#266 AS agg3#277, avg(agg4#40)#267 AS agg4#278, avg(agg5#41)#268 AS agg5#279, avg(agg6#42)#269 AS agg6#280, avg(agg7#43)#270 AS agg7#281] +Aggregate Attributes [7]: [avg(agg1#37)#332, avg(agg2#38)#333, avg(agg3#39)#334, avg(agg4#40)#335, avg(agg5#41)#336, avg(agg6#42)#337, avg(agg7#43)#338] +Results [11]: [null AS i_item_id#339, null AS ca_country#340, null AS ca_state#341, null AS county#342, avg(agg1#37)#332 AS agg1#343, avg(agg2#38)#333 AS agg2#344, avg(agg3#39)#334 AS agg3#345, avg(agg4#40)#335 AS agg4#346, avg(agg5#41)#336 AS agg5#347, avg(agg6#42)#337 AS agg6#348, avg(agg7#43)#338 AS agg7#349] (159) Union @@ -887,8 +887,8 @@ ReusedExchange (161) (161) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#16] -Subquery:2 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#224 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 138 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 138 Hosting Expression = cs_sold_date_sk#295 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 0cb9a331b406a..8e3c5958d2831 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 @@ -366,492 +366,492 @@ Aggregate Attributes [7]: [avg(agg1#35)#71, avg(agg2#36)#72, avg(agg3#37)#73, av Results [11]: [i_item_id#33, ca_country#27, ca_state#26, ca_county#25, 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] (46) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#93), dynamicpruningexpression(cs_sold_date_sk#93 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 15] -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] +Input [9]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93] (48) Filter [codegen id : 15] -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)) +Input [9]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93] +Condition : ((isnotnull(cs_bill_cdemo_sk#86) AND isnotnull(cs_bill_customer_sk#85)) AND isnotnull(cs_item_sk#87)) (49) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#94, cd_dep_count#95] (50) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#86] +Right keys [1]: [cd_demo_sk#94] Join condition: None (51) Project [codegen id : 15] -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] +Output [9]: [cs_bill_customer_sk#85, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95] +Input [11]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_demo_sk#94, cd_dep_count#95] (52) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] +Output [4]: [c_customer_sk#96, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99] (53) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#16] +Left keys [1]: [cs_bill_customer_sk#85] +Right keys [1]: [c_customer_sk#96] Join condition: None (54) Project [codegen id : 15] -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] +Output [11]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99] +Input [13]: [cs_bill_customer_sk#85, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_customer_sk#96, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99] (55) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#85] +Output [1]: [cd_demo_sk#100] (56) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_cdemo_sk#17] -Right keys [1]: [cd_demo_sk#85] +Left keys [1]: [c_current_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#100] Join condition: None (57) Project [codegen id : 15] -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#85] +Output [10]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_addr_sk#98, c_birth_year#99] +Input [12]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99, cd_demo_sk#100] (58) Scan parquet default.customer_address -Output [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Output [3]: [ca_address_sk#101, ca_state#102, ca_country#103] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 12] -Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Input [3]: [ca_address_sk#101, ca_state#102, ca_country#103] (60) Filter [codegen id : 12] -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)) +Input [3]: [ca_address_sk#101, ca_state#102, ca_country#103] +Condition : (ca_state#102 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#101)) (61) BroadcastExchange -Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#86] +Input [3]: [ca_address_sk#101, ca_state#102, ca_country#103] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#104] (62) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_addr_sk#18] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [c_current_addr_sk#98] +Right keys [1]: [ca_address_sk#101] Join condition: None (63) Project [codegen id : 15] -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] +Output [11]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103] +Input [13]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_addr_sk#98, c_birth_year#99, ca_address_sk#101, ca_state#102, ca_country#103] (64) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#105] (65) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [cs_sold_date_sk#93] +Right keys [1]: [d_date_sk#105] Join condition: None (66) Project [codegen id : 15] -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] +Output [10]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103] +Input [12]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103, d_date_sk#105] (67) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#32, i_item_id#33] +Output [2]: [i_item_sk#106, i_item_id#107] (68) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [cs_item_sk#87] +Right keys [1]: [i_item_sk#106] Join condition: None (69) Project [codegen id : 15] -Output [10]: [i_item_id#33, ca_country#27, ca_state#26, 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#20 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] -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#32, i_item_id#33] +Output [10]: [i_item_id#107, ca_country#103, ca_state#102, cast(cs_quantity#88 as decimal(12,2)) AS agg1#35, cast(cs_list_price#89 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#91 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#90 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#92 as decimal(12,2)) AS agg5#39, cast(c_birth_year#99 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#95 as decimal(12,2)) AS agg7#41] +Input [12]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103, i_item_sk#106, i_item_id#107] (70) HashAggregate [codegen id : 15] -Input [10]: [i_item_id#33, ca_country#27, ca_state#26, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [3]: [i_item_id#33, ca_country#27, ca_state#26] +Input [10]: [i_item_id#107, ca_country#103, ca_state#102, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [3]: [i_item_id#107, ca_country#103, ca_state#102] 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#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100] -Results [17]: [i_item_id#33, ca_country#27, ca_state#26, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] +Aggregate Attributes [14]: [sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117, sum#118, count#119, sum#120, count#121] +Results [17]: [i_item_id#107, ca_country#103, ca_state#102, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] (71) Exchange -Input [17]: [i_item_id#33, ca_country#27, ca_state#26, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] -Arguments: hashpartitioning(i_item_id#33, ca_country#27, ca_state#26, 5), ENSURE_REQUIREMENTS, [id=#115] +Input [17]: [i_item_id#107, ca_country#103, ca_state#102, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] +Arguments: hashpartitioning(i_item_id#107, ca_country#103, ca_state#102, 5), ENSURE_REQUIREMENTS, [id=#136] (72) HashAggregate [codegen id : 16] -Input [17]: [i_item_id#33, ca_country#27, ca_state#26, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] -Keys [3]: [i_item_id#33, ca_country#27, ca_state#26] +Input [17]: [i_item_id#107, ca_country#103, ca_state#102, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] +Keys [3]: [i_item_id#107, ca_country#103, ca_state#102] 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)#116, avg(agg2#36)#117, avg(agg3#37)#118, avg(agg4#38)#119, avg(agg5#39)#120, avg(agg6#40)#121, avg(agg7#41)#122] -Results [11]: [i_item_id#33, ca_country#27, ca_state#26, null AS county#123, avg(agg1#35)#116 AS agg1#124, avg(agg2#36)#117 AS agg2#125, avg(agg3#37)#118 AS agg3#126, avg(agg4#38)#119 AS agg4#127, avg(agg5#39)#120 AS agg5#128, avg(agg6#40)#121 AS agg6#129, avg(agg7#41)#122 AS agg7#130] +Aggregate Attributes [7]: [avg(agg1#35)#137, avg(agg2#36)#138, avg(agg3#37)#139, avg(agg4#38)#140, avg(agg5#39)#141, avg(agg6#40)#142, avg(agg7#41)#143] +Results [11]: [i_item_id#107, ca_country#103, ca_state#102, null AS county#144, avg(agg1#35)#137 AS agg1#145, avg(agg2#36)#138 AS agg2#146, avg(agg3#37)#139 AS agg3#147, avg(agg4#38)#140 AS agg4#148, avg(agg5#39)#141 AS agg5#149, avg(agg6#40)#142 AS agg6#150, avg(agg7#41)#143 AS agg7#151] (73) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#160), dynamicpruningexpression(cs_sold_date_sk#160 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 23] -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] +Input [9]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160] (75) Filter [codegen id : 23] -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)) +Input [9]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160] +Condition : ((isnotnull(cs_bill_cdemo_sk#153) AND isnotnull(cs_bill_customer_sk#152)) AND isnotnull(cs_item_sk#154)) (76) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#161, cd_dep_count#162] (77) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#153] +Right keys [1]: [cd_demo_sk#161] Join condition: None (78) Project [codegen id : 23] -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] +Output [9]: [cs_bill_customer_sk#152, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162] +Input [11]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_demo_sk#161, cd_dep_count#162] (79) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] +Output [4]: [c_customer_sk#163, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] (80) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#16] +Left keys [1]: [cs_bill_customer_sk#152] +Right keys [1]: [c_customer_sk#163] Join condition: None (81) Project [codegen id : 23] -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] +Output [11]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] +Input [13]: [cs_bill_customer_sk#152, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_customer_sk#163, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] (82) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#131] +Output [1]: [cd_demo_sk#167] (83) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_cdemo_sk#17] -Right keys [1]: [cd_demo_sk#131] +Left keys [1]: [c_current_cdemo_sk#164] +Right keys [1]: [cd_demo_sk#167] Join condition: None (84) Project [codegen id : 23] -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#131] +Output [10]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_addr_sk#165, c_birth_year#166] +Input [12]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166, cd_demo_sk#167] (85) Scan parquet default.customer_address -Output [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Output [3]: [ca_address_sk#168, ca_state#169, ca_country#170] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (86) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] (87) Filter [codegen id : 20] -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)) +Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] +Condition : (ca_state#169 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#168)) (88) Project [codegen id : 20] -Output [2]: [ca_address_sk#24, ca_country#27] -Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Output [2]: [ca_address_sk#168, ca_country#170] +Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] (89) BroadcastExchange -Input [2]: [ca_address_sk#24, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#132] +Input [2]: [ca_address_sk#168, ca_country#170] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#171] (90) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_addr_sk#18] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [c_current_addr_sk#165] +Right keys [1]: [ca_address_sk#168] Join condition: None (91) Project [codegen id : 23] -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] +Output [10]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_birth_year#166, ca_country#170] +Input [12]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_addr_sk#165, c_birth_year#166, ca_address_sk#168, ca_country#170] (92) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#172] (93) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [cs_sold_date_sk#160] +Right keys [1]: [d_date_sk#172] Join condition: None (94) Project [codegen id : 23] -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] +Output [9]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cd_dep_count#162, c_birth_year#166, ca_country#170] +Input [11]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_birth_year#166, ca_country#170, d_date_sk#172] (95) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#32, i_item_id#33] +Output [2]: [i_item_sk#173, i_item_id#174] (96) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [cs_item_sk#154] +Right keys [1]: [i_item_sk#173] Join condition: None (97) Project [codegen id : 23] -Output [9]: [i_item_id#33, ca_country#27, 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#20 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] -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#32, i_item_id#33] +Output [9]: [i_item_id#174, ca_country#170, cast(cs_quantity#155 as decimal(12,2)) AS agg1#35, cast(cs_list_price#156 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#158 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#157 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#159 as decimal(12,2)) AS agg5#39, cast(c_birth_year#166 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#162 as decimal(12,2)) AS agg7#41] +Input [11]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cd_dep_count#162, c_birth_year#166, ca_country#170, i_item_sk#173, i_item_id#174] (98) HashAggregate [codegen id : 23] -Input [9]: [i_item_id#33, ca_country#27, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [2]: [i_item_id#33, ca_country#27] +Input [9]: [i_item_id#174, ca_country#170, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [2]: [i_item_id#174, ca_country#170] 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#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140, sum#141, count#142, sum#143, count#144, sum#145, count#146] -Results [16]: [i_item_id#33, ca_country#27, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160] +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 [16]: [i_item_id#174, ca_country#170, 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] (99) Exchange -Input [16]: [i_item_id#33, ca_country#27, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160] -Arguments: hashpartitioning(i_item_id#33, ca_country#27, 5), ENSURE_REQUIREMENTS, [id=#161] +Input [16]: [i_item_id#174, ca_country#170, 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#174, ca_country#170, 5), ENSURE_REQUIREMENTS, [id=#203] (100) HashAggregate [codegen id : 24] -Input [16]: [i_item_id#33, ca_country#27, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160] -Keys [2]: [i_item_id#33, ca_country#27] +Input [16]: [i_item_id#174, ca_country#170, 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 [2]: [i_item_id#174, ca_country#170] 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)#162, avg(agg2#36)#163, avg(agg3#37)#164, avg(agg4#38)#165, avg(agg5#39)#166, avg(agg6#40)#167, avg(agg7#41)#168] -Results [11]: [i_item_id#33, ca_country#27, null AS ca_state#169, null AS county#170, avg(agg1#35)#162 AS agg1#171, avg(agg2#36)#163 AS agg2#172, avg(agg3#37)#164 AS agg3#173, avg(agg4#38)#165 AS agg4#174, avg(agg5#39)#166 AS agg5#175, avg(agg6#40)#167 AS agg6#176, avg(agg7#41)#168 AS agg7#177] +Aggregate Attributes [7]: [avg(agg1#35)#204, avg(agg2#36)#205, avg(agg3#37)#206, avg(agg4#38)#207, avg(agg5#39)#208, avg(agg6#40)#209, avg(agg7#41)#210] +Results [11]: [i_item_id#174, ca_country#170, null AS ca_state#211, null AS county#212, avg(agg1#35)#204 AS agg1#213, avg(agg2#36)#205 AS agg2#214, avg(agg3#37)#206 AS agg3#215, avg(agg4#38)#207 AS agg4#216, avg(agg5#39)#208 AS agg5#217, avg(agg6#40)#209 AS agg6#218, avg(agg7#41)#210 AS agg7#219] (101) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#228), dynamicpruningexpression(cs_sold_date_sk#228 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 31] -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] +Input [9]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228] (103) Filter [codegen id : 31] -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)) +Input [9]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228] +Condition : ((isnotnull(cs_bill_cdemo_sk#221) AND isnotnull(cs_bill_customer_sk#220)) AND isnotnull(cs_item_sk#222)) (104) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#229, cd_dep_count#230] (105) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#221] +Right keys [1]: [cd_demo_sk#229] Join condition: None (106) Project [codegen id : 31] -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] +Output [9]: [cs_bill_customer_sk#220, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230] +Input [11]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_demo_sk#229, cd_dep_count#230] (107) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] +Output [4]: [c_customer_sk#231, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234] (108) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#16] +Left keys [1]: [cs_bill_customer_sk#220] +Right keys [1]: [c_customer_sk#231] Join condition: None (109) Project [codegen id : 31] -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] +Output [11]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234] +Input [13]: [cs_bill_customer_sk#220, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_customer_sk#231, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234] (110) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#178] +Output [1]: [cd_demo_sk#235] (111) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_cdemo_sk#17] -Right keys [1]: [cd_demo_sk#178] +Left keys [1]: [c_current_cdemo_sk#232] +Right keys [1]: [cd_demo_sk#235] Join condition: None (112) Project [codegen id : 31] -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#178] +Output [10]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_addr_sk#233, c_birth_year#234] +Input [12]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234, cd_demo_sk#235] (113) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#26] +Output [2]: [ca_address_sk#236, ca_state#237] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (114) ColumnarToRow [codegen id : 28] -Input [2]: [ca_address_sk#24, ca_state#26] +Input [2]: [ca_address_sk#236, ca_state#237] (115) Filter [codegen id : 28] -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)) +Input [2]: [ca_address_sk#236, ca_state#237] +Condition : (ca_state#237 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#236)) (116) Project [codegen id : 28] -Output [1]: [ca_address_sk#24] -Input [2]: [ca_address_sk#24, ca_state#26] +Output [1]: [ca_address_sk#236] +Input [2]: [ca_address_sk#236, ca_state#237] (117) BroadcastExchange -Input [1]: [ca_address_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#179] +Input [1]: [ca_address_sk#236] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#238] (118) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_addr_sk#18] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [c_current_addr_sk#233] +Right keys [1]: [ca_address_sk#236] Join condition: None (119) Project [codegen id : 31] -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] +Output [9]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_birth_year#234] +Input [11]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_addr_sk#233, c_birth_year#234, ca_address_sk#236] (120) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#239] (121) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [cs_sold_date_sk#228] +Right keys [1]: [d_date_sk#239] Join condition: None (122) Project [codegen id : 31] -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] +Output [8]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cd_dep_count#230, c_birth_year#234] +Input [10]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_birth_year#234, d_date_sk#239] (123) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#32, i_item_id#33] +Output [2]: [i_item_sk#240, i_item_id#241] (124) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [cs_item_sk#222] +Right keys [1]: [i_item_sk#240] Join condition: None (125) Project [codegen id : 31] -Output [8]: [i_item_id#33, 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#20 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#20, i_item_sk#32, i_item_id#33] +Output [8]: [i_item_id#241, cast(cs_quantity#223 as decimal(12,2)) AS agg1#35, cast(cs_list_price#224 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#226 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#225 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#227 as decimal(12,2)) AS agg5#39, cast(c_birth_year#234 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#230 as decimal(12,2)) AS agg7#41] +Input [10]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cd_dep_count#230, c_birth_year#234, i_item_sk#240, i_item_id#241] (126) HashAggregate [codegen id : 31] -Input [8]: [i_item_id#33, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [1]: [i_item_id#33] +Input [8]: [i_item_id#241, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [1]: [i_item_id#241] 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#180, count#181, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193] -Results [15]: [i_item_id#33, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] +Aggregate Attributes [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] +Results [15]: [i_item_id#241, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] (127) Exchange -Input [15]: [i_item_id#33, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] -Arguments: hashpartitioning(i_item_id#33, 5), ENSURE_REQUIREMENTS, [id=#208] +Input [15]: [i_item_id#241, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] +Arguments: hashpartitioning(i_item_id#241, 5), ENSURE_REQUIREMENTS, [id=#270] (128) HashAggregate [codegen id : 32] -Input [15]: [i_item_id#33, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] -Keys [1]: [i_item_id#33] +Input [15]: [i_item_id#241, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] +Keys [1]: [i_item_id#241] 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)#209, avg(agg2#36)#210, avg(agg3#37)#211, avg(agg4#38)#212, avg(agg5#39)#213, avg(agg6#40)#214, avg(agg7#41)#215] -Results [11]: [i_item_id#33, null AS ca_country#216, null AS ca_state#217, null AS county#218, avg(agg1#35)#209 AS agg1#219, avg(agg2#36)#210 AS agg2#220, avg(agg3#37)#211 AS agg3#221, avg(agg4#38)#212 AS agg4#222, avg(agg5#39)#213 AS agg5#223, avg(agg6#40)#214 AS agg6#224, avg(agg7#41)#215 AS agg7#225] +Aggregate Attributes [7]: [avg(agg1#35)#271, avg(agg2#36)#272, avg(agg3#37)#273, avg(agg4#38)#274, avg(agg5#39)#275, avg(agg6#40)#276, avg(agg7#41)#277] +Results [11]: [i_item_id#241, null AS ca_country#278, null AS ca_state#279, null AS county#280, avg(agg1#35)#271 AS agg1#281, avg(agg2#36)#272 AS agg2#282, avg(agg3#37)#273 AS agg3#283, avg(agg4#38)#274 AS agg4#284, avg(agg5#39)#275 AS agg5#285, avg(agg6#40)#276 AS agg6#286, avg(agg7#41)#277 AS agg7#287] (129) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#296), dynamicpruningexpression(cs_sold_date_sk#296 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (130) ColumnarToRow [codegen id : 39] -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] +Input [9]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296] (131) Filter [codegen id : 39] -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)) +Input [9]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296] +Condition : ((isnotnull(cs_bill_cdemo_sk#289) AND isnotnull(cs_bill_customer_sk#288)) AND isnotnull(cs_item_sk#290)) (132) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#297, cd_dep_count#298] (133) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#289] +Right keys [1]: [cd_demo_sk#297] Join condition: None (134) Project [codegen id : 39] -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] +Output [9]: [cs_bill_customer_sk#288, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298] +Input [11]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_demo_sk#297, cd_dep_count#298] (135) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] +Output [4]: [c_customer_sk#299, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302] (136) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#16] +Left keys [1]: [cs_bill_customer_sk#288] +Right keys [1]: [c_customer_sk#299] Join condition: None (137) Project [codegen id : 39] -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] +Output [11]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302] +Input [13]: [cs_bill_customer_sk#288, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_customer_sk#299, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302] (138) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#226] +Output [1]: [cd_demo_sk#303] (139) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#17] -Right keys [1]: [cd_demo_sk#226] +Left keys [1]: [c_current_cdemo_sk#300] +Right keys [1]: [cd_demo_sk#303] Join condition: None (140) Project [codegen id : 39] -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#226] +Output [10]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_addr_sk#301, c_birth_year#302] +Input [12]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302, cd_demo_sk#303] (141) ReusedExchange [Reuses operator id: 117] -Output [1]: [ca_address_sk#24] +Output [1]: [ca_address_sk#304] (142) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_addr_sk#18] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [c_current_addr_sk#301] +Right keys [1]: [ca_address_sk#304] Join condition: None (143) Project [codegen id : 39] -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] +Output [9]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_birth_year#302] +Input [11]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_addr_sk#301, c_birth_year#302, ca_address_sk#304] (144) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#305] (145) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [cs_sold_date_sk#296] +Right keys [1]: [d_date_sk#305] Join condition: None (146) Project [codegen id : 39] -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] +Output [8]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cd_dep_count#298, c_birth_year#302] +Input [10]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_birth_year#302, d_date_sk#305] (147) Scan parquet default.item -Output [1]: [i_item_sk#32] +Output [1]: [i_item_sk#306] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (148) ColumnarToRow [codegen id : 38] -Input [1]: [i_item_sk#32] +Input [1]: [i_item_sk#306] (149) Filter [codegen id : 38] -Input [1]: [i_item_sk#32] -Condition : isnotnull(i_item_sk#32) +Input [1]: [i_item_sk#306] +Condition : isnotnull(i_item_sk#306) (150) BroadcastExchange -Input [1]: [i_item_sk#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#227] +Input [1]: [i_item_sk#306] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#307] (151) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [cs_item_sk#290] +Right keys [1]: [i_item_sk#306] Join condition: None (152) Project [codegen id : 39] -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#20 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] -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#32] +Output [7]: [cast(cs_quantity#291 as decimal(12,2)) AS agg1#35, cast(cs_list_price#292 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#294 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#293 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#295 as decimal(12,2)) AS agg5#39, cast(c_birth_year#302 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#298 as decimal(12,2)) AS agg7#41] +Input [9]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cd_dep_count#298, c_birth_year#302, i_item_sk#306] (153) HashAggregate [codegen id : 39] 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#228, count#229, sum#230, count#231, sum#232, count#233, sum#234, count#235, sum#236, count#237, sum#238, count#239, sum#240, count#241] -Results [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] +Aggregate Attributes [14]: [sum#308, count#309, sum#310, count#311, sum#312, count#313, sum#314, count#315, sum#316, count#317, sum#318, count#319, sum#320, count#321] +Results [14]: [sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335] (154) Exchange -Input [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#256] +Input [14]: [sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#336] (155) HashAggregate [codegen id : 40] -Input [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] +Input [14]: [sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335] 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)#257, avg(agg2#36)#258, avg(agg3#37)#259, avg(agg4#38)#260, avg(agg5#39)#261, avg(agg6#40)#262, avg(agg7#41)#263] -Results [11]: [null AS i_item_id#264, null AS ca_country#265, null AS ca_state#266, null AS county#267, avg(agg1#35)#257 AS agg1#268, avg(agg2#36)#258 AS agg2#269, avg(agg3#37)#259 AS agg3#270, avg(agg4#38)#260 AS agg4#271, avg(agg5#39)#261 AS agg5#272, avg(agg6#40)#262 AS agg6#273, avg(agg7#41)#263 AS agg7#274] +Aggregate Attributes [7]: [avg(agg1#35)#337, avg(agg2#36)#338, avg(agg3#37)#339, avg(agg4#38)#340, avg(agg5#39)#341, avg(agg6#40)#342, avg(agg7#41)#343] +Results [11]: [null AS i_item_id#344, null AS ca_country#345, null AS ca_state#346, null AS county#347, avg(agg1#35)#337 AS agg1#348, avg(agg2#36)#338 AS agg2#349, avg(agg3#37)#339 AS agg3#350, avg(agg4#38)#340 AS agg4#351, avg(agg5#39)#341 AS agg5#352, avg(agg6#40)#342 AS agg6#353, avg(agg7#41)#343 AS agg7#354] (156) Union @@ -868,12 +868,12 @@ ReusedExchange (158) (158) ReusedExchange [Reuses operator id: 34] Output [1]: [d_date_sk#29] -Subquery:2 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#93 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#160 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 101 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 101 Hosting Expression = cs_sold_date_sk#228 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 129 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 129 Hosting Expression = cs_sold_date_sk#296 IN dynamicpruning#10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt index 8bc09f9457fe7..a74ad3e4b8b77 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt @@ -55,7 +55,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -86,7 +86,7 @@ Arguments: [cs_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -94,7 +94,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt index 49b35aacfcf2f..c6a345be29c9d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt @@ -44,7 +44,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -52,7 +52,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -79,7 +79,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] 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 014b89cecd475..f5a7a9135cf29 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 @@ -198,116 +198,116 @@ Aggregate Attributes [1]: [avg(qoh#24)#29] Results [5]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, avg(qoh#24)#29 AS qoh#30] (31) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#31, count#32] +Output [6]: [i_product_name#31, i_brand#32, i_class#33, i_category#34, sum#35, count#36] (32) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#31, count#32] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#33] -Results [4]: [i_product_name#16, i_brand#13, i_class#14, avg(inv_quantity_on_hand#3)#33 AS qoh#24] +Input [6]: [i_product_name#31, i_brand#32, i_class#33, i_category#34, sum#35, count#36] +Keys [4]: [i_product_name#31, i_brand#32, i_class#33, i_category#34] +Functions [1]: [avg(inv_quantity_on_hand#37)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#37)#38] +Results [4]: [i_product_name#31, i_brand#32, i_class#33, avg(inv_quantity_on_hand#37)#38 AS qoh#24] (33) HashAggregate [codegen id : 16] -Input [4]: [i_product_name#16, i_brand#13, i_class#14, qoh#24] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] +Input [4]: [i_product_name#31, i_brand#32, i_class#33, qoh#24] +Keys [3]: [i_product_name#31, i_brand#32, i_class#33] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#34, count#35] -Results [5]: [i_product_name#16, i_brand#13, i_class#14, sum#36, count#37] +Aggregate Attributes [2]: [sum#39, count#40] +Results [5]: [i_product_name#31, i_brand#32, i_class#33, sum#41, count#42] (34) Exchange -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#36, count#37] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [5]: [i_product_name#31, i_brand#32, i_class#33, sum#41, count#42] +Arguments: hashpartitioning(i_product_name#31, i_brand#32, i_class#33, 5), ENSURE_REQUIREMENTS, [id=#43] (35) HashAggregate [codegen id : 17] -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#36, count#37] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] +Input [5]: [i_product_name#31, i_brand#32, i_class#33, sum#41, count#42] +Keys [3]: [i_product_name#31, i_brand#32, i_class#33] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#39] -Results [5]: [i_product_name#16, i_brand#13, i_class#14, null AS i_category#40, avg(qoh#24)#39 AS qoh#41] +Aggregate Attributes [1]: [avg(qoh#24)#44] +Results [5]: [i_product_name#31, i_brand#32, i_class#33, null AS i_category#45, avg(qoh#24)#44 AS qoh#46] (36) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#42, count#43] +Output [6]: [i_product_name#47, i_brand#48, i_class#49, i_category#50, sum#51, count#52] (37) HashAggregate [codegen id : 25] -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#42, count#43] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#44] -Results [3]: [i_product_name#16, i_brand#13, avg(inv_quantity_on_hand#3)#44 AS qoh#24] +Input [6]: [i_product_name#47, i_brand#48, i_class#49, i_category#50, sum#51, count#52] +Keys [4]: [i_product_name#47, i_brand#48, i_class#49, i_category#50] +Functions [1]: [avg(inv_quantity_on_hand#53)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#53)#54] +Results [3]: [i_product_name#47, i_brand#48, avg(inv_quantity_on_hand#53)#54 AS qoh#24] (38) HashAggregate [codegen id : 25] -Input [3]: [i_product_name#16, i_brand#13, qoh#24] -Keys [2]: [i_product_name#16, i_brand#13] +Input [3]: [i_product_name#47, i_brand#48, qoh#24] +Keys [2]: [i_product_name#47, i_brand#48] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#45, count#46] -Results [4]: [i_product_name#16, i_brand#13, sum#47, count#48] +Aggregate Attributes [2]: [sum#55, count#56] +Results [4]: [i_product_name#47, i_brand#48, sum#57, count#58] (39) Exchange -Input [4]: [i_product_name#16, i_brand#13, sum#47, count#48] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [i_product_name#47, i_brand#48, sum#57, count#58] +Arguments: hashpartitioning(i_product_name#47, i_brand#48, 5), ENSURE_REQUIREMENTS, [id=#59] (40) HashAggregate [codegen id : 26] -Input [4]: [i_product_name#16, i_brand#13, sum#47, count#48] -Keys [2]: [i_product_name#16, i_brand#13] +Input [4]: [i_product_name#47, i_brand#48, sum#57, count#58] +Keys [2]: [i_product_name#47, i_brand#48] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#50] -Results [5]: [i_product_name#16, i_brand#13, null AS i_class#51, null AS i_category#52, avg(qoh#24)#50 AS qoh#53] +Aggregate Attributes [1]: [avg(qoh#24)#60] +Results [5]: [i_product_name#47, i_brand#48, null AS i_class#61, null AS i_category#62, avg(qoh#24)#60 AS qoh#63] (41) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#54, count#55] +Output [6]: [i_product_name#64, i_brand#65, i_class#66, i_category#67, sum#68, count#69] (42) HashAggregate [codegen id : 34] -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#54, count#55] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#56] -Results [2]: [i_product_name#16, avg(inv_quantity_on_hand#3)#56 AS qoh#24] +Input [6]: [i_product_name#64, i_brand#65, i_class#66, i_category#67, sum#68, count#69] +Keys [4]: [i_product_name#64, i_brand#65, i_class#66, i_category#67] +Functions [1]: [avg(inv_quantity_on_hand#70)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#70)#71] +Results [2]: [i_product_name#64, avg(inv_quantity_on_hand#70)#71 AS qoh#24] (43) HashAggregate [codegen id : 34] -Input [2]: [i_product_name#16, qoh#24] -Keys [1]: [i_product_name#16] +Input [2]: [i_product_name#64, qoh#24] +Keys [1]: [i_product_name#64] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#57, count#58] -Results [3]: [i_product_name#16, sum#59, count#60] +Aggregate Attributes [2]: [sum#72, count#73] +Results [3]: [i_product_name#64, sum#74, count#75] (44) Exchange -Input [3]: [i_product_name#16, sum#59, count#60] -Arguments: hashpartitioning(i_product_name#16, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [3]: [i_product_name#64, sum#74, count#75] +Arguments: hashpartitioning(i_product_name#64, 5), ENSURE_REQUIREMENTS, [id=#76] (45) HashAggregate [codegen id : 35] -Input [3]: [i_product_name#16, sum#59, count#60] -Keys [1]: [i_product_name#16] +Input [3]: [i_product_name#64, sum#74, count#75] +Keys [1]: [i_product_name#64] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#62] -Results [5]: [i_product_name#16, null AS i_brand#63, null AS i_class#64, null AS i_category#65, avg(qoh#24)#62 AS qoh#66] +Aggregate Attributes [1]: [avg(qoh#24)#77] +Results [5]: [i_product_name#64, null AS i_brand#78, null AS i_class#79, null AS i_category#80, avg(qoh#24)#77 AS qoh#81] (46) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#67, count#68] +Output [6]: [i_product_name#82, i_brand#83, i_class#84, i_category#85, sum#86, count#87] (47) HashAggregate [codegen id : 43] -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#67, count#68] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#69] -Results [1]: [avg(inv_quantity_on_hand#3)#69 AS qoh#24] +Input [6]: [i_product_name#82, i_brand#83, i_class#84, i_category#85, sum#86, count#87] +Keys [4]: [i_product_name#82, i_brand#83, i_class#84, i_category#85] +Functions [1]: [avg(inv_quantity_on_hand#88)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#88)#89] +Results [1]: [avg(inv_quantity_on_hand#88)#89 AS qoh#24] (48) HashAggregate [codegen id : 43] Input [1]: [qoh#24] Keys: [] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#70, count#71] -Results [2]: [sum#72, count#73] +Aggregate Attributes [2]: [sum#90, count#91] +Results [2]: [sum#92, count#93] (49) Exchange -Input [2]: [sum#72, count#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#74] +Input [2]: [sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] (50) HashAggregate [codegen id : 44] -Input [2]: [sum#72, count#73] +Input [2]: [sum#92, count#93] Keys: [] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#75] -Results [5]: [null AS i_product_name#76, null AS i_brand#77, null AS i_class#78, null AS i_category#79, avg(qoh#24)#75 AS qoh#80] +Aggregate Attributes [1]: [avg(qoh#24)#95] +Results [5]: [null AS i_product_name#96, null AS i_brand#97, null AS i_class#98, null AS i_category#99, avg(qoh#24)#95 AS qoh#100] (51) Union 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 74dd076851c1d..527d2eb5e86a4 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 @@ -183,116 +183,116 @@ Aggregate Attributes [1]: [avg(qoh#23)#28] Results [5]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, avg(qoh#23)#28 AS qoh#29] (28) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#30, count#31] +Output [6]: [i_product_name#30, i_brand#31, i_class#32, i_category#33, sum#34, count#35] (29) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#30, count#31] -Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#32] -Results [4]: [i_product_name#13, i_brand#10, i_class#11, avg(inv_quantity_on_hand#3)#32 AS qoh#23] +Input [6]: [i_product_name#30, i_brand#31, i_class#32, i_category#33, sum#34, count#35] +Keys [4]: [i_product_name#30, i_brand#31, i_class#32, i_category#33] +Functions [1]: [avg(inv_quantity_on_hand#36)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#36)#37] +Results [4]: [i_product_name#30, i_brand#31, i_class#32, avg(inv_quantity_on_hand#36)#37 AS qoh#23] (30) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#13, i_brand#10, i_class#11, qoh#23] -Keys [3]: [i_product_name#13, i_brand#10, i_class#11] +Input [4]: [i_product_name#30, i_brand#31, i_class#32, qoh#23] +Keys [3]: [i_product_name#30, i_brand#31, i_class#32] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#33, count#34] -Results [5]: [i_product_name#13, i_brand#10, i_class#11, sum#35, count#36] +Aggregate Attributes [2]: [sum#38, count#39] +Results [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] (31) Exchange -Input [5]: [i_product_name#13, i_brand#10, i_class#11, sum#35, count#36] -Arguments: hashpartitioning(i_product_name#13, i_brand#10, i_class#11, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] +Arguments: hashpartitioning(i_product_name#30, i_brand#31, i_class#32, 5), ENSURE_REQUIREMENTS, [id=#42] (32) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#13, i_brand#10, i_class#11, sum#35, count#36] -Keys [3]: [i_product_name#13, i_brand#10, i_class#11] +Input [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] +Keys [3]: [i_product_name#30, i_brand#31, i_class#32] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#38] -Results [5]: [i_product_name#13, i_brand#10, i_class#11, null AS i_category#39, avg(qoh#23)#38 AS qoh#40] +Aggregate Attributes [1]: [avg(qoh#23)#43] +Results [5]: [i_product_name#30, i_brand#31, i_class#32, null AS i_category#44, avg(qoh#23)#43 AS qoh#45] (33) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#41, count#42] +Output [6]: [i_product_name#46, i_brand#47, i_class#48, i_category#49, sum#50, count#51] (34) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#41, count#42] -Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#43] -Results [3]: [i_product_name#13, i_brand#10, avg(inv_quantity_on_hand#3)#43 AS qoh#23] +Input [6]: [i_product_name#46, i_brand#47, i_class#48, i_category#49, sum#50, count#51] +Keys [4]: [i_product_name#46, i_brand#47, i_class#48, i_category#49] +Functions [1]: [avg(inv_quantity_on_hand#52)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#52)#53] +Results [3]: [i_product_name#46, i_brand#47, avg(inv_quantity_on_hand#52)#53 AS qoh#23] (35) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#13, i_brand#10, qoh#23] -Keys [2]: [i_product_name#13, i_brand#10] +Input [3]: [i_product_name#46, i_brand#47, qoh#23] +Keys [2]: [i_product_name#46, i_brand#47] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#44, count#45] -Results [4]: [i_product_name#13, i_brand#10, sum#46, count#47] +Aggregate Attributes [2]: [sum#54, count#55] +Results [4]: [i_product_name#46, i_brand#47, sum#56, count#57] (36) Exchange -Input [4]: [i_product_name#13, i_brand#10, sum#46, count#47] -Arguments: hashpartitioning(i_product_name#13, i_brand#10, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [4]: [i_product_name#46, i_brand#47, sum#56, count#57] +Arguments: hashpartitioning(i_product_name#46, i_brand#47, 5), ENSURE_REQUIREMENTS, [id=#58] (37) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#13, i_brand#10, sum#46, count#47] -Keys [2]: [i_product_name#13, i_brand#10] +Input [4]: [i_product_name#46, i_brand#47, sum#56, count#57] +Keys [2]: [i_product_name#46, i_brand#47] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#49] -Results [5]: [i_product_name#13, i_brand#10, null AS i_class#50, null AS i_category#51, avg(qoh#23)#49 AS qoh#52] +Aggregate Attributes [1]: [avg(qoh#23)#59] +Results [5]: [i_product_name#46, i_brand#47, null AS i_class#60, null AS i_category#61, avg(qoh#23)#59 AS qoh#62] (38) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#53, count#54] +Output [6]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, sum#67, count#68] (39) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#53, count#54] -Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#55] -Results [2]: [i_product_name#13, avg(inv_quantity_on_hand#3)#55 AS qoh#23] +Input [6]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, sum#67, count#68] +Keys [4]: [i_product_name#63, i_brand#64, i_class#65, i_category#66] +Functions [1]: [avg(inv_quantity_on_hand#69)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#69)#70] +Results [2]: [i_product_name#63, avg(inv_quantity_on_hand#69)#70 AS qoh#23] (40) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#13, qoh#23] -Keys [1]: [i_product_name#13] +Input [2]: [i_product_name#63, qoh#23] +Keys [1]: [i_product_name#63] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#56, count#57] -Results [3]: [i_product_name#13, sum#58, count#59] +Aggregate Attributes [2]: [sum#71, count#72] +Results [3]: [i_product_name#63, sum#73, count#74] (41) Exchange -Input [3]: [i_product_name#13, sum#58, count#59] -Arguments: hashpartitioning(i_product_name#13, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [3]: [i_product_name#63, sum#73, count#74] +Arguments: hashpartitioning(i_product_name#63, 5), ENSURE_REQUIREMENTS, [id=#75] (42) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#13, sum#58, count#59] -Keys [1]: [i_product_name#13] +Input [3]: [i_product_name#63, sum#73, count#74] +Keys [1]: [i_product_name#63] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#61] -Results [5]: [i_product_name#13, null AS i_brand#62, null AS i_class#63, null AS i_category#64, avg(qoh#23)#61 AS qoh#65] +Aggregate Attributes [1]: [avg(qoh#23)#76] +Results [5]: [i_product_name#63, null AS i_brand#77, null AS i_class#78, null AS i_category#79, avg(qoh#23)#76 AS qoh#80] (43) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#66, count#67] +Output [6]: [i_product_name#81, i_brand#82, i_class#83, i_category#84, sum#85, count#86] (44) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#66, count#67] -Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#68] -Results [1]: [avg(inv_quantity_on_hand#3)#68 AS qoh#23] +Input [6]: [i_product_name#81, i_brand#82, i_class#83, i_category#84, sum#85, count#86] +Keys [4]: [i_product_name#81, i_brand#82, i_class#83, i_category#84] +Functions [1]: [avg(inv_quantity_on_hand#87)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#87)#88] +Results [1]: [avg(inv_quantity_on_hand#87)#88 AS qoh#23] (45) HashAggregate [codegen id : 28] Input [1]: [qoh#23] Keys: [] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#69, count#70] -Results [2]: [sum#71, count#72] +Aggregate Attributes [2]: [sum#89, count#90] +Results [2]: [sum#91, count#92] (46) Exchange -Input [2]: [sum#71, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#73] +Input [2]: [sum#91, count#92] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] (47) HashAggregate [codegen id : 29] -Input [2]: [sum#71, count#72] +Input [2]: [sum#91, count#92] Keys: [] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#74] -Results [5]: [null AS i_product_name#75, null AS i_brand#76, null AS i_class#77, null AS i_category#78, avg(qoh#23)#74 AS qoh#79] +Aggregate Attributes [1]: [avg(qoh#23)#94] +Results [5]: [null AS i_product_name#95, null AS i_brand#96, null AS i_class#97, null AS i_category#98, avg(qoh#23)#94 AS qoh#99] (48) Union 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 d4d0015a9fec0..1c0ed5c8c188b 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 @@ -157,7 +157,7 @@ Input [12]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk# 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_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 4] @@ -165,7 +165,7 @@ Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, (26) Filter [codegen id : 4] 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_color#27) AND (i_color#27 = pale)) AND isnotnull(i_item_sk#24)) +Condition : ((isnotnull(i_color#27) AND (i_color#27 = pale )) AND isnotnull(i_item_sk#24)) (27) BroadcastExchange Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] @@ -329,224 +329,224 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (50) Scan parquet default.store -Output [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] +Output [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (51) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] +Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] (52) Filter [codegen id : 1] -Input [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] -Condition : (((isnotnull(s_market_id#3) AND (s_market_id#3 = 8)) AND isnotnull(s_store_sk#1)) AND isnotnull(s_zip#5)) +Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] +Condition : (((isnotnull(s_market_id#54) AND (s_market_id#54 = 8)) AND isnotnull(s_store_sk#52)) AND isnotnull(s_zip#56)) (53) Project [codegen id : 1] -Output [4]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5] -Input [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] +Output [4]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56] +Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] (54) BroadcastExchange -Input [4]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5] -Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#52] +Input [4]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56] +Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#57] (55) Scan parquet default.customer_address -Output [4]: [ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] +Output [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (56) ColumnarToRow -Input [4]: [ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] +Input [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] (57) Filter -Input [4]: [ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] -Condition : ((isnotnull(ca_address_sk#7) AND isnotnull(ca_country#10)) AND isnotnull(ca_zip#9)) +Input [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] +Condition : ((isnotnull(ca_address_sk#58) AND isnotnull(ca_country#61)) AND isnotnull(ca_zip#60)) (58) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [s_zip#5] -Right keys [1]: [ca_zip#9] +Left keys [1]: [s_zip#56] +Right keys [1]: [ca_zip#60] Join condition: None (59) Project [codegen id : 2] -Output [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10] -Input [8]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5, ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] +Output [6]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61] +Input [8]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56, ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] (60) BroadcastExchange -Input [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10] -Arguments: HashedRelationBroadcastMode(List(input[3, int, true], upper(input[5, string, true])),false), [id=#53] +Input [6]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61] +Arguments: HashedRelationBroadcastMode(List(input[3, int, true], upper(input[5, string, true])),false), [id=#62] (61) Scan parquet default.customer -Output [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] +Output [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct (62) ColumnarToRow -Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] +Input [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] (63) Filter -Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] -Condition : ((isnotnull(c_customer_sk#12) AND isnotnull(c_current_addr_sk#13)) AND isnotnull(c_birth_country#16)) +Input [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] +Condition : ((isnotnull(c_customer_sk#63) AND isnotnull(c_current_addr_sk#64)) AND isnotnull(c_birth_country#67)) (64) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ca_address_sk#7, upper(ca_country#10)] -Right keys [2]: [c_current_addr_sk#13, c_birth_country#16] +Left keys [2]: [ca_address_sk#58, upper(ca_country#61)] +Right keys [2]: [c_current_addr_sk#64, c_birth_country#67] Join condition: None (65) Project [codegen id : 3] -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] -Input [11]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10, c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] +Output [7]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66] +Input [11]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61, c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] (66) BroadcastExchange -Input [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] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#54] +Input [7]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#68] (67) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (68) ColumnarToRow -Input [6]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22, ss_sold_date_sk#23] +Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] (69) Filter -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)) +Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] +Condition : (((isnotnull(ss_ticket_number#72) AND isnotnull(ss_item_sk#69)) AND isnotnull(ss_store_sk#71)) AND isnotnull(ss_customer_sk#70)) (70) Project -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] +Output [5]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73] +Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] (71) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [s_store_sk#1, c_customer_sk#12] -Right keys [2]: [ss_store_sk#20, ss_customer_sk#19] +Left keys [2]: [s_store_sk#52, c_customer_sk#63] +Right keys [2]: [ss_store_sk#71, ss_customer_sk#70] Join condition: None (72) Project [codegen id : 4] -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] +Output [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] +Input [12]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73] (73) Exchange -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=#55] +Input [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] +Arguments: hashpartitioning(ss_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] (74) Sort [codegen id : 5] -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 +Input [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] +Arguments: [ss_item_sk#69 ASC NULLS FIRST], false, 0 (75) Scan parquet default.item -Output [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Output [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] (77) Filter [codegen id : 6] -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) +Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Condition : isnotnull(i_item_sk#76) (78) Exchange -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=#56] +Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Arguments: hashpartitioning(i_item_sk#76, 5), ENSURE_REQUIREMENTS, [id=#82] (79) Sort [codegen id : 7] -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 +Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Arguments: [i_item_sk#76 ASC NULLS FIRST], false, 0 (80) SortMergeJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#18] -Right keys [1]: [i_item_sk#24] +Left keys [1]: [ss_item_sk#69] +Right keys [1]: [i_item_sk#76] Join condition: None (81) Project [codegen id : 8] -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] +Output [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Input [14]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] (82) Exchange -Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] -Arguments: hashpartitioning(cast(ss_ticket_number#21 as bigint), cast(ss_item_sk#18 as bigint), 5), ENSURE_REQUIREMENTS, [id=#57] +Input [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Arguments: hashpartitioning(cast(ss_ticket_number#72 as bigint), cast(ss_item_sk#69 as bigint), 5), ENSURE_REQUIREMENTS, [id=#83] (83) Sort [codegen id : 9] -Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] -Arguments: [cast(ss_ticket_number#21 as bigint) ASC NULLS FIRST, cast(ss_item_sk#18 as bigint) ASC NULLS FIRST], false, 0 +Input [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Arguments: [cast(ss_ticket_number#72 as bigint) ASC NULLS FIRST, cast(ss_item_sk#69 as bigint) ASC NULLS FIRST], false, 0 (84) Scan parquet default.store_returns -Output [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Output [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (85) ColumnarToRow [codegen id : 10] -Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] (86) Filter [codegen id : 10] -Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] -Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) +Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] +Condition : (isnotnull(sr_ticket_number#85) AND isnotnull(sr_item_sk#84)) (87) Project [codegen id : 10] -Output [2]: [sr_item_sk#32, sr_ticket_number#33] -Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Output [2]: [sr_item_sk#84, sr_ticket_number#85] +Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] (88) Exchange -Input [2]: [sr_item_sk#32, sr_ticket_number#33] -Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [sr_item_sk#84, sr_ticket_number#85] +Arguments: hashpartitioning(sr_ticket_number#85, sr_item_sk#84, 5), ENSURE_REQUIREMENTS, [id=#87] (89) Sort [codegen id : 11] -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 +Input [2]: [sr_item_sk#84, sr_ticket_number#85] +Arguments: [sr_ticket_number#85 ASC NULLS FIRST, sr_item_sk#84 ASC NULLS FIRST], false, 0 (90) SortMergeJoin [codegen id : 12] -Left keys [2]: [cast(ss_ticket_number#21 as bigint), cast(ss_item_sk#18 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Left keys [2]: [cast(ss_ticket_number#72 as bigint), cast(ss_item_sk#69 as bigint)] +Right keys [2]: [sr_ticket_number#85, sr_item_sk#84] Join condition: None (91) Project [codegen id : 12] -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] +Output [11]: [ss_net_paid#73, s_store_name#53, s_state#55, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, c_first_name#65, c_last_name#66, ca_state#59] +Input [15]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, sr_item_sk#84, sr_ticket_number#85] (92) HashAggregate [codegen id : 12] -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#59] -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#60] +Input [11]: [ss_net_paid#73, s_store_name#53, s_state#55, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, c_first_name#65, c_last_name#66, ca_state#59] +Keys [10]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#73))] +Aggregate Attributes [1]: [sum#88] +Results [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] (93) Exchange -Input [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#60] -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=#61] +Input [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] +Arguments: hashpartitioning(c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, 5), ENSURE_REQUIREMENTS, [id=#90] (94) HashAggregate [codegen id : 13] -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#60] -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))#62] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#62,17,2) AS netpaid#40] +Input [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] +Keys [10]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78] +Functions [1]: [sum(UnscaledValue(ss_net_paid#73))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#73))#91] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#73))#91,17,2) AS netpaid#40] (95) HashAggregate [codegen id : 13] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#63, count#64] -Results [2]: [sum#65, count#66] +Aggregate Attributes [2]: [sum#92, count#93] +Results [2]: [sum#94, count#95] (96) Exchange -Input [2]: [sum#65, count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] +Input [2]: [sum#94, count#95] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#96] (97) HashAggregate [codegen id : 14] -Input [2]: [sum#65, count#66] +Input [2]: [sum#94, count#95] Keys: [] Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#68] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#68)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#69] +Aggregate Attributes [1]: [avg(netpaid#40)#97] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#97)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#98] 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 6835ff84f5bc7..7259a3d829613 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 @@ -146,7 +146,7 @@ Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_stor 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_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] @@ -154,7 +154,7 @@ Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, (24) Filter [codegen id : 6] 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_color#21) AND (i_color#21 = pale)) AND isnotnull(i_item_sk#18)) +Condition : ((isnotnull(i_color#21) AND (i_color#21 = pale )) AND isnotnull(i_item_sk#18)) (25) BroadcastExchange Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] @@ -326,212 +326,212 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (50) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 1] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] (52) Filter [codegen id : 1] -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)) +Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] +Condition : (((isnotnull(ss_ticket_number#55) AND isnotnull(ss_item_sk#52)) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_customer_sk#53)) (53) Project [codegen id : 1] -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] +Output [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] +Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] (54) Exchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#52] +Input [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] +Arguments: hashpartitioning(cast(ss_ticket_number#55 as bigint), cast(ss_item_sk#52 as bigint), 5), ENSURE_REQUIREMENTS, [id=#58] (55) Sort [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +Input [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] +Arguments: [cast(ss_ticket_number#55 as bigint) ASC NULLS FIRST, cast(ss_item_sk#52 as bigint) ASC NULLS FIRST], false, 0 (56) Scan parquet default.store_returns -Output [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] (58) Filter [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] -Condition : (isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#8)) +Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] +Condition : (isnotnull(sr_ticket_number#60) AND isnotnull(sr_item_sk#59)) (59) Project [codegen id : 3] -Output [2]: [sr_item_sk#8, sr_ticket_number#9] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [2]: [sr_item_sk#59, sr_ticket_number#60] +Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] (60) Exchange -Input [2]: [sr_item_sk#8, sr_ticket_number#9] -Arguments: hashpartitioning(sr_ticket_number#9, sr_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [2]: [sr_item_sk#59, sr_ticket_number#60] +Arguments: hashpartitioning(sr_ticket_number#60, sr_item_sk#59, 5), ENSURE_REQUIREMENTS, [id=#62] (61) Sort [codegen id : 4] -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 +Input [2]: [sr_item_sk#59, sr_ticket_number#60] +Arguments: [sr_ticket_number#60 ASC NULLS FIRST, sr_item_sk#59 ASC NULLS FIRST], false, 0 (62) SortMergeJoin [codegen id : 9] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] +Left keys [2]: [cast(ss_ticket_number#55 as bigint), cast(ss_item_sk#52 as bigint)] +Right keys [2]: [sr_ticket_number#60, sr_item_sk#59] Join condition: None (63) Project [codegen id : 9] -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] +Output [4]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_net_paid#56] +Input [7]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, sr_item_sk#59, sr_ticket_number#60] (64) Scan parquet default.store -Output [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (65) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] (66) Filter [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] -Condition : (((isnotnull(s_market_id#14) AND (s_market_id#14 = 8)) AND isnotnull(s_store_sk#12)) AND isnotnull(s_zip#16)) +Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] +Condition : (((isnotnull(s_market_id#65) AND (s_market_id#65 = 8)) AND isnotnull(s_store_sk#63)) AND isnotnull(s_zip#67)) (67) Project [codegen id : 5] -Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [4]: [s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] +Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] (68) BroadcastExchange -Input [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [4]: [s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#68] (69) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] +Left keys [1]: [ss_store_sk#54] +Right keys [1]: [s_store_sk#63] Join condition: None (70) Project [codegen id : 9] -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] +Output [6]: [ss_item_sk#52, ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67] +Input [8]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_net_paid#56, s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] (71) Scan parquet default.item -Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (73) Filter [codegen id : 6] -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) +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Condition : isnotnull(i_item_sk#69) (74) BroadcastExchange -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=#55] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] (75) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [ss_item_sk#52] +Right keys [1]: [i_item_sk#69] Join condition: None (76) Project [codegen id : 9] -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] +Output [10]: [ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [12]: [ss_item_sk#52, ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (77) Scan parquet default.customer -Output [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] +Output [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct (78) ColumnarToRow [codegen id : 7] -Input [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] +Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] (79) Filter [codegen id : 7] -Input [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : ((isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#26)) AND isnotnull(c_birth_country#29)) +Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +Condition : ((isnotnull(c_customer_sk#76) AND isnotnull(c_current_addr_sk#77)) AND isnotnull(c_birth_country#80)) (80) BroadcastExchange -Input [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#56] +Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] (81) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#25] +Left keys [1]: [ss_customer_sk#53] +Right keys [1]: [c_customer_sk#76] Join condition: None (82) Project [codegen id : 9] -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] +Output [13]: [ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +Input [15]: [ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] (83) Scan parquet default.customer_address -Output [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] +Output [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (84) ColumnarToRow [codegen id : 8] -Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] +Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] (85) Filter [codegen id : 8] -Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] -Condition : ((isnotnull(ca_address_sk#31) AND isnotnull(ca_country#34)) AND isnotnull(ca_zip#33)) +Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] +Condition : ((isnotnull(ca_address_sk#82) AND isnotnull(ca_country#85)) AND isnotnull(ca_zip#84)) (86) BroadcastExchange -Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [id=#57] +Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] +Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [id=#86] (87) BroadcastHashJoin [codegen id : 9] -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] +Left keys [3]: [c_current_addr_sk#77, c_birth_country#80, s_zip#67] +Right keys [3]: [ca_address_sk#82, upper(ca_country#85), ca_zip#84] Join condition: None (88) Project [codegen id : 9] -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] +Output [11]: [ss_net_paid#56, s_store_name#64, s_state#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#78, c_last_name#79, ca_state#83] +Input [17]: [ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80, ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] (89) HashAggregate [codegen id : 9] -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#58] -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#59] +Input [11]: [ss_net_paid#56, s_store_name#64, s_state#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#78, c_last_name#79, ca_state#83] +Keys [10]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#56))] +Aggregate Attributes [1]: [sum#87] +Results [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] (90) Exchange -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#59] -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=#60] +Input [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] +Arguments: hashpartitioning(c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#89] (91) HashAggregate [codegen id : 10] -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#59] -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))#61] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#61,17,2) AS netpaid#40] +Input [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] +Keys [10]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [sum(UnscaledValue(ss_net_paid#56))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#56))#90] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#56))#90,17,2) AS netpaid#40] (92) HashAggregate [codegen id : 10] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#62, count#63] -Results [2]: [sum#64, count#65] +Aggregate Attributes [2]: [sum#91, count#92] +Results [2]: [sum#93, count#94] (93) Exchange -Input [2]: [sum#64, count#65] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] +Input [2]: [sum#93, count#94] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] (94) HashAggregate [codegen id : 11] -Input [2]: [sum#64, count#65] +Input [2]: [sum#93, count#94] Keys: [] Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#67] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#67)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#68] +Aggregate Attributes [1]: [avg(netpaid#40)#96] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#96)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#97] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt index 6c48953be5aeb..1fb9533024b2f 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#61), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -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] +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] (35) Filter [codegen id : 11] -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)) +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#62] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#55] +Right keys [1]: [cd_demo_sk#62] Join condition: None (38) Project [codegen id : 11] -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] +Output [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, cd_demo_sk#62] (39) Scan parquet default.store -Output [2]: [s_store_sk#18, s_state#19] +Output [2]: [s_store_sk#63, s_state#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 8] -Input [2]: [s_store_sk#18, s_state#19] +Input [2]: [s_store_sk#63, s_state#64] (41) Filter [codegen id : 8] -Input [2]: [s_store_sk#18, s_state#19] -Condition : ((isnotnull(s_state#19) AND (s_state#19 = TN)) AND isnotnull(s_store_sk#18)) +Input [2]: [s_store_sk#63, s_state#64] +Condition : ((isnotnull(s_state#64) AND (s_state#64 = TN)) AND isnotnull(s_store_sk#63)) (42) Project [codegen id : 8] -Output [1]: [s_store_sk#18] -Input [2]: [s_store_sk#18, s_state#19] +Output [1]: [s_store_sk#63] +Input [2]: [s_store_sk#63, s_state#64] (43) BroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [s_store_sk#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] (44) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#56] +Right keys [1]: [s_store_sk#63] Join condition: None (45) Project [codegen id : 11] -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#18] +Output [6]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Input [8]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, s_store_sk#63] (46) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#66] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#61] +Right keys [1]: [d_date_sk#66] Join condition: None (48) Project [codegen id : 11] -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] +Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#66] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#21, i_item_id#22] +Output [2]: [i_item_sk#67, i_item_id#68] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#67] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] +Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] +Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#22] +Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#68] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] +Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] (53) Exchange -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [1]: [i_item_id#22] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Keys [1]: [i_item_id#68] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] -Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] +Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] +Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] (55) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 17] -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] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] (57) Filter [codegen id : 17] -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)) +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#104] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#104] Join condition: None (60) Project [codegen id : 17] -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] +Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] (61) ReusedExchange [Reuses operator id: 43] -Output [1]: [s_store_sk#18] +Output [1]: [s_store_sk#105] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#98] +Right keys [1]: [s_store_sk#105] Join condition: None (63) Project [codegen id : 17] -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#18] +Output [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, s_store_sk#105] (64) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#106] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#103] +Right keys [1]: [d_date_sk#106] Join condition: None (66) Project [codegen id : 17] -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] +Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [7]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#106] (67) Scan parquet default.item -Output [1]: [i_item_sk#21] +Output [1]: [i_item_sk#107] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#21] +Input [1]: [i_item_sk#107] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#21] -Condition : isnotnull(i_item_sk#21) +Input [1]: [i_item_sk#107] +Condition : isnotnull(i_item_sk#107) (70) BroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] +Input [1]: [i_item_sk#107] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#96] +Right keys [1]: [i_item_sk#107] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] +Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] +Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] -Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] +Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] (74) Exchange -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] -Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] +Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] +Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#15] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 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 04c79a5413ad9..f05f4e9f2e43e 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#61), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -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] +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] (35) Filter [codegen id : 11] -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)) +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#62] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#55] +Right keys [1]: [cd_demo_sk#62] Join condition: None (38) Project [codegen id : 11] -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] +Output [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, cd_demo_sk#62] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#63] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#61] +Right keys [1]: [d_date_sk#63] Join condition: None (41) Project [codegen id : 11] -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] +Output [6]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [8]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#63] (42) Scan parquet default.store -Output [2]: [s_store_sk#18, s_state#19] +Output [2]: [s_store_sk#64, s_state#65] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (43) ColumnarToRow [codegen id : 9] -Input [2]: [s_store_sk#18, s_state#19] +Input [2]: [s_store_sk#64, s_state#65] (44) Filter [codegen id : 9] -Input [2]: [s_store_sk#18, s_state#19] -Condition : ((isnotnull(s_state#19) AND (s_state#19 = TN)) AND isnotnull(s_store_sk#18)) +Input [2]: [s_store_sk#64, s_state#65] +Condition : ((isnotnull(s_state#65) AND (s_state#65 = TN)) AND isnotnull(s_store_sk#64)) (45) Project [codegen id : 9] -Output [1]: [s_store_sk#18] -Input [2]: [s_store_sk#18, s_state#19] +Output [1]: [s_store_sk#64] +Input [2]: [s_store_sk#64, s_state#65] (46) BroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [s_store_sk#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#66] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#56] +Right keys [1]: [s_store_sk#64] Join condition: None (48) Project [codegen id : 11] -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#18] +Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, s_store_sk#64] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#21, i_item_id#22] +Output [2]: [i_item_sk#67, i_item_id#68] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#67] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] +Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] +Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#22] +Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#68] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] +Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] (53) Exchange -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [1]: [i_item_id#22] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Keys [1]: [i_item_id#68] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] -Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] +Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] +Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] (55) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 17] -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] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] (57) Filter [codegen id : 17] -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)) +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#104] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#104] Join condition: None (60) Project [codegen id : 17] -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] +Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] (61) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#105] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#103] +Right keys [1]: [d_date_sk#105] Join condition: None (63) Project [codegen id : 17] -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] +Output [6]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#105] (64) ReusedExchange [Reuses operator id: 46] -Output [1]: [s_store_sk#18] +Output [1]: [s_store_sk#106] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#98] +Right keys [1]: [s_store_sk#106] Join condition: None (66) Project [codegen id : 17] -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#18] +Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#106] (67) Scan parquet default.item -Output [1]: [i_item_sk#21] +Output [1]: [i_item_sk#107] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#21] +Input [1]: [i_item_sk#107] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#21] -Condition : isnotnull(i_item_sk#21) +Input [1]: [i_item_sk#107] +Condition : isnotnull(i_item_sk#107) (70) BroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] +Input [1]: [i_item_sk#107] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#96] +Right keys [1]: [i_item_sk#107] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] +Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] +Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] -Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] +Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] (74) Exchange -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] -Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] +Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] +Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#15] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt index 104340f2fd6db..f6151f4d79ec1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt @@ -121,7 +121,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -129,7 +129,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt index f106fba199355..e4d14f842f5a9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt index 0a861f4ce8e6d..f7fff421817c0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt @@ -145,20 +145,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#17] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#10] +Right keys [1]: [d_date_sk#17] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] -Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] +Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] (23) Exchange Input [1]: [ws_bill_customer_sk#15] -Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] @@ -170,38 +170,38 @@ Right keys [1]: [ws_bill_customer_sk#15] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Output [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#21] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [cs_sold_date_sk#20] +Right keys [1]: [d_date_sk#21] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#18] -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] +Output [1]: [cs_ship_customer_sk#19] +Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] (31) Exchange -Input [1]: [cs_ship_customer_sk#18] -Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [cs_ship_customer_sk#19] +Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#18] -Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#19] +Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#19] Join condition: None (34) Filter [codegen id : 12] @@ -214,103 +214,103 @@ Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2 (36) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#21] +Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#23] (37) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 (38) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_state#23] +Output [2]: [ca_address_sk#24, ca_state#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 14] -Input [2]: [ca_address_sk#22, ca_state#23] +Input [2]: [ca_address_sk#24, ca_state#25] (40) Filter [codegen id : 14] -Input [2]: [ca_address_sk#22, ca_state#23] -Condition : isnotnull(ca_address_sk#22) +Input [2]: [ca_address_sk#24, ca_state#25] +Condition : isnotnull(ca_address_sk#24) (41) Exchange -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: hashpartitioning(ca_address_sk#22, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] (42) Sort [codegen id : 15] -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#24] Join condition: None (44) Project [codegen id : 16] -Output [2]: [c_current_cdemo_sk#4, ca_state#23] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] +Output [2]: [c_current_cdemo_sk#4, ca_state#25] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24, ca_state#25] (45) Exchange -Input [2]: [c_current_cdemo_sk#4, ca_state#23] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [2]: [c_current_cdemo_sk#4, ca_state#25] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#27] (46) Sort [codegen id : 17] -Input [2]: [c_current_cdemo_sk#4, ca_state#23] +Input [2]: [c_current_cdemo_sk#4, ca_state#25] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 (47) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 18] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (49) Filter [codegen id : 18] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Condition : isnotnull(cd_demo_sk#26) +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Condition : isnotnull(cd_demo_sk#28) (50) Exchange -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: hashpartitioning(cd_demo_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#34] (51) Sort [codegen id : 19] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: [cd_demo_sk#26 ASC NULLS FIRST], false, 0 +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#26] +Right keys [1]: [cd_demo_sk#28] Join condition: None (53) Project [codegen id : 20] -Output [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [8]: [c_current_cdemo_sk#4, ca_state#25, cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (54) HashAggregate [codegen id : 20] -Input [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#29), partial_max(cd_dep_count#29), partial_sum(cd_dep_count#29), partial_avg(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_sum(cd_dep_employed_count#30), partial_avg(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_sum(cd_dep_college_count#31)] -Aggregate Attributes [13]: [count#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45] -Results [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] +Input [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#31), partial_max(cd_dep_count#31), partial_sum(cd_dep_count#31), partial_avg(cd_dep_employed_count#32), partial_max(cd_dep_employed_count#32), partial_sum(cd_dep_employed_count#32), partial_avg(cd_dep_college_count#33), partial_max(cd_dep_college_count#33), partial_sum(cd_dep_college_count#33)] +Aggregate Attributes [13]: [count#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47] +Results [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] (55) Exchange -Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] -Arguments: hashpartitioning(ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] +Arguments: hashpartitioning(ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#61] (56) HashAggregate [codegen id : 21] -Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] -Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [count(1), avg(cd_dep_count#29), max(cd_dep_count#29), sum(cd_dep_count#29), avg(cd_dep_employed_count#30), max(cd_dep_employed_count#30), sum(cd_dep_employed_count#30), avg(cd_dep_college_count#31), max(cd_dep_college_count#31), sum(cd_dep_college_count#31)] -Aggregate Attributes [10]: [count(1)#60, avg(cd_dep_count#29)#61, max(cd_dep_count#29)#62, sum(cd_dep_count#29)#63, avg(cd_dep_employed_count#30)#64, max(cd_dep_employed_count#30)#65, sum(cd_dep_employed_count#30)#66, avg(cd_dep_college_count#31)#67, max(cd_dep_college_count#31)#68, sum(cd_dep_college_count#31)#69] -Results [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, count(1)#60 AS cnt1#70, avg(cd_dep_count#29)#61 AS avg(cd_dep_count)#71, max(cd_dep_count#29)#62 AS max(cd_dep_count)#72, sum(cd_dep_count#29)#63 AS sum(cd_dep_count)#73, cd_dep_employed_count#30, count(1)#60 AS cnt2#74, avg(cd_dep_employed_count#30)#64 AS avg(cd_dep_employed_count)#75, max(cd_dep_employed_count#30)#65 AS max(cd_dep_employed_count)#76, sum(cd_dep_employed_count#30)#66 AS sum(cd_dep_employed_count)#77, cd_dep_college_count#31, count(1)#60 AS cnt3#78, avg(cd_dep_college_count#31)#67 AS avg(cd_dep_college_count)#79, max(cd_dep_college_count#31)#68 AS max(cd_dep_college_count)#80, sum(cd_dep_college_count#31)#69 AS sum(cd_dep_college_count)#81] +Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] +Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Functions [10]: [count(1), avg(cd_dep_count#31), max(cd_dep_count#31), sum(cd_dep_count#31), avg(cd_dep_employed_count#32), max(cd_dep_employed_count#32), sum(cd_dep_employed_count#32), avg(cd_dep_college_count#33), max(cd_dep_college_count#33), sum(cd_dep_college_count#33)] +Aggregate Attributes [10]: [count(1)#62, avg(cd_dep_count#31)#63, max(cd_dep_count#31)#64, sum(cd_dep_count#31)#65, avg(cd_dep_employed_count#32)#66, max(cd_dep_employed_count#32)#67, sum(cd_dep_employed_count#32)#68, avg(cd_dep_college_count#33)#69, max(cd_dep_college_count#33)#70, sum(cd_dep_college_count#33)#71] +Results [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, count(1)#62 AS cnt1#72, avg(cd_dep_count#31)#63 AS avg(cd_dep_count)#73, max(cd_dep_count#31)#64 AS max(cd_dep_count)#74, sum(cd_dep_count#31)#65 AS sum(cd_dep_count)#75, cd_dep_employed_count#32, count(1)#62 AS cnt2#76, avg(cd_dep_employed_count#32)#66 AS avg(cd_dep_employed_count)#77, max(cd_dep_employed_count#32)#67 AS max(cd_dep_employed_count)#78, sum(cd_dep_employed_count#32)#68 AS sum(cd_dep_employed_count)#79, cd_dep_college_count#33, count(1)#62 AS cnt3#80, avg(cd_dep_college_count#33)#69 AS avg(cd_dep_college_count)#81, max(cd_dep_college_count#33)#70 AS max(cd_dep_college_count)#82, sum(cd_dep_college_count#33)#71 AS sum(cd_dep_college_count)#83] (57) TakeOrderedAndProject -Input [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#70, avg(cd_dep_count)#71, max(cd_dep_count)#72, sum(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, avg(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, sum(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, avg(cd_dep_college_count)#79, max(cd_dep_college_count)#80, sum(cd_dep_college_count)#81] -Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#70, avg(cd_dep_count)#71, max(cd_dep_count)#72, sum(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, avg(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, sum(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, avg(cd_dep_college_count)#79, max(cd_dep_college_count)#80, sum(cd_dep_college_count)#81] +Input [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cnt1#72, avg(cd_dep_count)#73, max(cd_dep_count)#74, sum(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, avg(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, sum(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, avg(cd_dep_college_count)#81, max(cd_dep_college_count)#82, sum(cd_dep_college_count)#83] +Arguments: 100, [ca_state#25 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cnt1#72, avg(cd_dep_count)#73, max(cd_dep_count)#74, sum(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, avg(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, sum(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, avg(cd_dep_college_count)#81, max(cd_dep_college_count)#82, sum(cd_dep_college_count)#83] ===== Subqueries ===== @@ -323,6 +323,6 @@ Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index 12c8230c7e3bc..642bf989159d3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -122,20 +122,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#16] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] -Right keys [1]: [d_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] +Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#16] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] @@ -143,34 +143,34 @@ Right keys [1]: [ws_bill_customer_sk#14] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#8)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#20] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#9] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#20] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#17] -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] +Output [1]: [cs_ship_customer_sk#18] +Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#20] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [cs_ship_customer_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#17] +Right keys [1]: [cs_ship_customer_sk#18] Join condition: None (29) Filter [codegen id : 9] @@ -182,80 +182,80 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +Output [2]: [ca_address_sk#22, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_state#21] +Input [2]: [ca_address_sk#22, ca_state#23] (33) Filter [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) +Input [2]: [ca_address_sk#22, ca_state#23] +Condition : isnotnull(ca_address_sk#22) (34) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +Input [2]: [ca_address_sk#22, ca_state#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] (35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#22] Join condition: None (36) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#21] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] +Output [2]: [c_current_cdemo_sk#4, ca_state#23] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] (37) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (39) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#25) (40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#25] Join condition: None (42) Project [codegen id : 9] -Output [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Output [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (43) HashAggregate [codegen id : 9] -Input [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] -Aggregate Attributes [13]: [count#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42] -Results [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] +Input [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#28), partial_max(cd_dep_count#28), partial_sum(cd_dep_count#28), partial_avg(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_sum(cd_dep_employed_count#29), partial_avg(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_sum(cd_dep_college_count#30)] +Aggregate Attributes [13]: [count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] +Results [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] (44) Exchange -Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] -Arguments: hashpartitioning(ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] +Arguments: hashpartitioning(ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] (45) HashAggregate [codegen id : 10] -Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] -Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] -Aggregate Attributes [10]: [count(1)#57, avg(cd_dep_count#26)#58, max(cd_dep_count#26)#59, sum(cd_dep_count#26)#60, avg(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, sum(cd_dep_employed_count#27)#63, avg(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, sum(cd_dep_college_count#28)#66] -Results [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, count(1)#57 AS cnt1#67, avg(cd_dep_count#26)#58 AS avg(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, sum(cd_dep_count#26)#60 AS sum(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, avg(cd_dep_employed_count#27)#61 AS avg(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, sum(cd_dep_employed_count#27)#63 AS sum(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, avg(cd_dep_college_count#28)#64 AS avg(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, sum(cd_dep_college_count#28)#66 AS sum(cd_dep_college_count)#78] +Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] +Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Functions [10]: [count(1), avg(cd_dep_count#28), max(cd_dep_count#28), sum(cd_dep_count#28), avg(cd_dep_employed_count#29), max(cd_dep_employed_count#29), sum(cd_dep_employed_count#29), avg(cd_dep_college_count#30), max(cd_dep_college_count#30), sum(cd_dep_college_count#30)] +Aggregate Attributes [10]: [count(1)#59, avg(cd_dep_count#28)#60, max(cd_dep_count#28)#61, sum(cd_dep_count#28)#62, avg(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, sum(cd_dep_employed_count#29)#65, avg(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, sum(cd_dep_college_count#30)#68] +Results [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, count(1)#59 AS cnt1#69, avg(cd_dep_count#28)#60 AS avg(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, sum(cd_dep_count#28)#62 AS sum(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, avg(cd_dep_employed_count#29)#63 AS avg(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, sum(cd_dep_employed_count#29)#65 AS sum(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, avg(cd_dep_college_count#30)#66 AS avg(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, sum(cd_dep_college_count#30)#68 AS sum(cd_dep_college_count)#80] (46) TakeOrderedAndProject -Input [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] -Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] +Input [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] +Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] ===== Subqueries ===== @@ -268,6 +268,6 @@ Output [1]: [d_date_sk#9] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt index 1c1b8ad01abfa..df75bec72326d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt @@ -142,52 +142,52 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#15] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (22) Project [codegen id : 7] -Output [1]: [ws_bill_customer_sk#13 AS customsk#15] -Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] +Output [1]: [ws_bill_customer_sk#13 AS customsk#16] +Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] (23) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] ReadSchema: struct (24) ColumnarToRow [codegen id : 9] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (25) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#19] (26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (27) Project [codegen id : 9] -Output [1]: [cs_ship_customer_sk#16 AS customsk#18] -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] +Output [1]: [cs_ship_customer_sk#17 AS customsk#20] +Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] (28) Union (29) Exchange -Input [1]: [customsk#15] -Arguments: hashpartitioning(customsk#15, 5), ENSURE_REQUIREMENTS, [id=#19] +Input [1]: [customsk#16] +Arguments: hashpartitioning(customsk#16, 5), ENSURE_REQUIREMENTS, [id=#21] (30) Sort [codegen id : 10] -Input [1]: [customsk#15] -Arguments: [customsk#15 ASC NULLS FIRST], false, 0 +Input [1]: [customsk#16] +Arguments: [customsk#16 ASC NULLS FIRST], false, 0 (31) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#15] +Right keys [1]: [customsk#16] Join condition: None (32) Project [codegen id : 11] @@ -196,103 +196,103 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (33) Exchange Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: hashpartitioning(c_current_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#20] +Arguments: hashpartitioning(c_current_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#22] (34) Sort [codegen id : 12] Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: [c_current_addr_sk#3 ASC NULLS FIRST], false, 0 (35) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] +Output [2]: [ca_address_sk#23, ca_state#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 13] -Input [2]: [ca_address_sk#21, ca_state#22] +Input [2]: [ca_address_sk#23, ca_state#24] (37) Filter [codegen id : 13] -Input [2]: [ca_address_sk#21, ca_state#22] -Condition : isnotnull(ca_address_sk#21) +Input [2]: [ca_address_sk#23, ca_state#24] +Condition : isnotnull(ca_address_sk#23) (38) Exchange -Input [2]: [ca_address_sk#21, ca_state#22] -Arguments: hashpartitioning(ca_address_sk#21, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [2]: [ca_address_sk#23, ca_state#24] +Arguments: hashpartitioning(ca_address_sk#23, 5), ENSURE_REQUIREMENTS, [id=#25] (39) Sort [codegen id : 14] -Input [2]: [ca_address_sk#21, ca_state#22] -Arguments: [ca_address_sk#21 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#23, ca_state#24] +Arguments: [ca_address_sk#23 ASC NULLS FIRST], false, 0 (40) SortMergeJoin [codegen id : 15] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#23] Join condition: None (41) Project [codegen id : 15] -Output [2]: [c_current_cdemo_sk#2, ca_state#22] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21, ca_state#22] +Output [2]: [c_current_cdemo_sk#2, ca_state#24] +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#23, ca_state#24] (42) Exchange -Input [2]: [c_current_cdemo_sk#2, ca_state#22] -Arguments: hashpartitioning(c_current_cdemo_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [2]: [c_current_cdemo_sk#2, ca_state#24] +Arguments: hashpartitioning(c_current_cdemo_sk#2, 5), ENSURE_REQUIREMENTS, [id=#26] (43) Sort [codegen id : 16] -Input [2]: [c_current_cdemo_sk#2, ca_state#22] +Input [2]: [c_current_cdemo_sk#2, ca_state#24] Arguments: [c_current_cdemo_sk#2 ASC NULLS FIRST], false, 0 (44) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (45) ColumnarToRow [codegen id : 17] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] (46) Filter [codegen id : 17] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#25) +Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Condition : isnotnull(cd_demo_sk#27) (47) Exchange -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: hashpartitioning(cd_demo_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Arguments: hashpartitioning(cd_demo_sk#27, 5), ENSURE_REQUIREMENTS, [id=#33] (48) Sort [codegen id : 18] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#25 ASC NULLS FIRST], false, 0 +Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Arguments: [cd_demo_sk#27 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 19] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#27] Join condition: None (50) Project [codegen id : 19] -Output [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [8]: [c_current_cdemo_sk#2, ca_state#24, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] (51) HashAggregate [codegen id : 19] -Input [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#28), partial_max(cd_dep_count#28), partial_sum(cd_dep_count#28), partial_avg(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_sum(cd_dep_employed_count#29), partial_avg(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_sum(cd_dep_college_count#30)] -Aggregate Attributes [13]: [count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] -Results [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] +Input [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Keys [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#30), partial_max(cd_dep_count#30), partial_sum(cd_dep_count#30), partial_avg(cd_dep_employed_count#31), partial_max(cd_dep_employed_count#31), partial_sum(cd_dep_employed_count#31), partial_avg(cd_dep_college_count#32), partial_max(cd_dep_college_count#32), partial_sum(cd_dep_college_count#32)] +Aggregate Attributes [13]: [count#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42, sum#43, count#44, max#45, sum#46] +Results [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] (52) Exchange -Input [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] -Arguments: hashpartitioning(ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] +Arguments: hashpartitioning(ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#60] (53) HashAggregate [codegen id : 20] -Input [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] -Keys [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [10]: [count(1), avg(cd_dep_count#28), max(cd_dep_count#28), sum(cd_dep_count#28), avg(cd_dep_employed_count#29), max(cd_dep_employed_count#29), sum(cd_dep_employed_count#29), avg(cd_dep_college_count#30), max(cd_dep_college_count#30), sum(cd_dep_college_count#30)] -Aggregate Attributes [10]: [count(1)#59, avg(cd_dep_count#28)#60, max(cd_dep_count#28)#61, sum(cd_dep_count#28)#62, avg(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, sum(cd_dep_employed_count#29)#65, avg(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, sum(cd_dep_college_count#30)#68] -Results [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, count(1)#59 AS cnt1#69, avg(cd_dep_count#28)#60 AS avg(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, sum(cd_dep_count#28)#62 AS sum(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, avg(cd_dep_employed_count#29)#63 AS avg(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, sum(cd_dep_employed_count#29)#65 AS sum(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, avg(cd_dep_college_count#30)#66 AS avg(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, sum(cd_dep_college_count#30)#68 AS sum(cd_dep_college_count)#80] +Input [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] +Keys [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Functions [10]: [count(1), avg(cd_dep_count#30), max(cd_dep_count#30), sum(cd_dep_count#30), avg(cd_dep_employed_count#31), max(cd_dep_employed_count#31), sum(cd_dep_employed_count#31), avg(cd_dep_college_count#32), max(cd_dep_college_count#32), sum(cd_dep_college_count#32)] +Aggregate Attributes [10]: [count(1)#61, avg(cd_dep_count#30)#62, max(cd_dep_count#30)#63, sum(cd_dep_count#30)#64, avg(cd_dep_employed_count#31)#65, max(cd_dep_employed_count#31)#66, sum(cd_dep_employed_count#31)#67, avg(cd_dep_college_count#32)#68, max(cd_dep_college_count#32)#69, sum(cd_dep_college_count#32)#70] +Results [18]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, count(1)#61 AS cnt1#71, avg(cd_dep_count#30)#62 AS avg(cd_dep_count)#72, max(cd_dep_count#30)#63 AS max(cd_dep_count)#73, sum(cd_dep_count#30)#64 AS sum(cd_dep_count)#74, cd_dep_employed_count#31, count(1)#61 AS cnt2#75, avg(cd_dep_employed_count#31)#65 AS avg(cd_dep_employed_count)#76, max(cd_dep_employed_count#31)#66 AS max(cd_dep_employed_count)#77, sum(cd_dep_employed_count#31)#67 AS sum(cd_dep_employed_count)#78, cd_dep_college_count#32, count(1)#61 AS cnt3#79, avg(cd_dep_college_count#32)#68 AS avg(cd_dep_college_count)#80, max(cd_dep_college_count#32)#69 AS max(cd_dep_college_count)#81, sum(cd_dep_college_count#32)#70 AS sum(cd_dep_college_count)#82] (54) TakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] -Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] +Input [18]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cnt1#71, avg(cd_dep_count)#72, max(cd_dep_count)#73, sum(cd_dep_count)#74, cd_dep_employed_count#31, cnt2#75, avg(cd_dep_employed_count)#76, max(cd_dep_employed_count)#77, sum(cd_dep_employed_count)#78, cd_dep_college_count#32, cnt3#79, avg(cd_dep_college_count)#80, max(cd_dep_college_count)#81, sum(cd_dep_college_count)#82] +Arguments: 100, [ca_state#24 ASC NULLS FIRST, cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cnt1#71, avg(cd_dep_count)#72, max(cd_dep_count)#73, sum(cd_dep_count)#74, cd_dep_employed_count#31, cnt2#75, avg(cd_dep_employed_count)#76, max(cd_dep_employed_count)#77, sum(cd_dep_employed_count)#78, cd_dep_college_count#32, cnt3#79, avg(cd_dep_college_count)#80, max(cd_dep_college_count)#81, sum(cd_dep_college_count)#82] ===== Subqueries ===== @@ -305,6 +305,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index ffb07d398de71..57710f427f0f8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -120,48 +120,48 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#12 AS customsk#14] -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] +Output [1]: [ws_bill_customer_sk#12 AS customsk#15] +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] (20) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#6)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] (22) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#18] (23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#18] Join condition: None (24) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#15 AS customsk#17] -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] +Output [1]: [cs_ship_customer_sk#16 AS customsk#19] +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#18] (25) Union (26) BroadcastExchange -Input [1]: [customsk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Input [1]: [customsk#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (27) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#14] +Right keys [1]: [customsk#15] Join condition: None (28) Project [codegen id : 9] @@ -169,80 +169,80 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#19, ca_state#20] +Output [2]: [ca_address_sk#21, ca_state#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#19, ca_state#20] +Input [2]: [ca_address_sk#21, ca_state#22] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#19, ca_state#20] -Condition : isnotnull(ca_address_sk#19) +Input [2]: [ca_address_sk#21, ca_state#22] +Condition : isnotnull(ca_address_sk#21) (32) BroadcastExchange -Input [2]: [ca_address_sk#19, ca_state#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Input [2]: [ca_address_sk#21, ca_state#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] (33) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#19] +Right keys [1]: [ca_address_sk#21] Join condition: None (34) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, ca_state#20] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19, ca_state#20] +Output [2]: [c_current_cdemo_sk#2, ca_state#22] +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21, ca_state#22] (35) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Output [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] (37) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Condition : isnotnull(cd_demo_sk#22) +Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Condition : isnotnull(cd_demo_sk#24) (38) BroadcastExchange -Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] +Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (39) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#22] +Right keys [1]: [cd_demo_sk#24] Join condition: None (40) Project [codegen id : 9] -Output [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Input [8]: [c_current_cdemo_sk#2, ca_state#20, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Output [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] (41) HashAggregate [codegen id : 9] -Input [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_dep_count#25), partial_sum(cd_dep_count#25), partial_avg(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_sum(cd_dep_employed_count#26), partial_avg(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_sum(cd_dep_college_count#27)] -Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] -Results [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +Input [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Keys [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#27), partial_max(cd_dep_count#27), partial_sum(cd_dep_count#27), partial_avg(cd_dep_employed_count#28), partial_max(cd_dep_employed_count#28), partial_sum(cd_dep_employed_count#28), partial_avg(cd_dep_college_count#29), partial_max(cd_dep_college_count#29), partial_sum(cd_dep_college_count#29)] +Aggregate Attributes [13]: [count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] +Results [19]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] (42) Exchange -Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Arguments: hashpartitioning(ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [19]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] +Arguments: hashpartitioning(ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, 5), ENSURE_REQUIREMENTS, [id=#57] (43) HashAggregate [codegen id : 10] -Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_dep_count#25), avg(cd_dep_employed_count#26), max(cd_dep_employed_count#26), sum(cd_dep_employed_count#26), avg(cd_dep_college_count#27), max(cd_dep_college_count#27), sum(cd_dep_college_count#27)] -Aggregate Attributes [10]: [count(1)#56, avg(cd_dep_count#25)#57, max(cd_dep_count#25)#58, sum(cd_dep_count#25)#59, avg(cd_dep_employed_count#26)#60, max(cd_dep_employed_count#26)#61, sum(cd_dep_employed_count#26)#62, avg(cd_dep_college_count#27)#63, max(cd_dep_college_count#27)#64, sum(cd_dep_college_count#27)#65] -Results [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, count(1)#56 AS cnt1#66, avg(cd_dep_count#25)#57 AS avg(cd_dep_count)#67, max(cd_dep_count#25)#58 AS max(cd_dep_count)#68, sum(cd_dep_count#25)#59 AS sum(cd_dep_count)#69, cd_dep_employed_count#26, count(1)#56 AS cnt2#70, avg(cd_dep_employed_count#26)#60 AS avg(cd_dep_employed_count)#71, max(cd_dep_employed_count#26)#61 AS max(cd_dep_employed_count)#72, sum(cd_dep_employed_count#26)#62 AS sum(cd_dep_employed_count)#73, cd_dep_college_count#27, count(1)#56 AS cnt3#74, avg(cd_dep_college_count#27)#63 AS avg(cd_dep_college_count)#75, max(cd_dep_college_count#27)#64 AS max(cd_dep_college_count)#76, sum(cd_dep_college_count#27)#65 AS sum(cd_dep_college_count)#77] +Input [19]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] +Keys [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Functions [10]: [count(1), avg(cd_dep_count#27), max(cd_dep_count#27), sum(cd_dep_count#27), avg(cd_dep_employed_count#28), max(cd_dep_employed_count#28), sum(cd_dep_employed_count#28), avg(cd_dep_college_count#29), max(cd_dep_college_count#29), sum(cd_dep_college_count#29)] +Aggregate Attributes [10]: [count(1)#58, avg(cd_dep_count#27)#59, max(cd_dep_count#27)#60, sum(cd_dep_count#27)#61, avg(cd_dep_employed_count#28)#62, max(cd_dep_employed_count#28)#63, sum(cd_dep_employed_count#28)#64, avg(cd_dep_college_count#29)#65, max(cd_dep_college_count#29)#66, sum(cd_dep_college_count#29)#67] +Results [18]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, count(1)#58 AS cnt1#68, avg(cd_dep_count#27)#59 AS avg(cd_dep_count)#69, max(cd_dep_count#27)#60 AS max(cd_dep_count)#70, sum(cd_dep_count#27)#61 AS sum(cd_dep_count)#71, cd_dep_employed_count#28, count(1)#58 AS cnt2#72, avg(cd_dep_employed_count#28)#62 AS avg(cd_dep_employed_count)#73, max(cd_dep_employed_count#28)#63 AS max(cd_dep_employed_count)#74, sum(cd_dep_employed_count#28)#64 AS sum(cd_dep_employed_count)#75, cd_dep_college_count#29, count(1)#58 AS cnt3#76, avg(cd_dep_college_count#29)#65 AS avg(cd_dep_college_count)#77, max(cd_dep_college_count#29)#66 AS max(cd_dep_college_count)#78, sum(cd_dep_college_count#29)#67 AS sum(cd_dep_college_count)#79] (44) TakeOrderedAndProject -Input [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] -Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] +Input [18]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#28, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#29, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79] +Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_dep_count#27 ASC NULLS FIRST, cd_dep_employed_count#28 ASC NULLS FIRST, cd_dep_college_count#29 ASC NULLS FIRST], [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#28, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#29, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79] ===== Subqueries ===== @@ -255,6 +255,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#6 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 9f594df6754ba..79037befbd6b5 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 @@ -169,60 +169,60 @@ Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#22, sum(UnscaledV Results [6]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#22,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#23,17,2))), DecimalType(37,20), true) as decimal(38,20)) AS gross_margin#24, i_category#15, i_class#14, 0 AS t_category#25, 0 AS t_class#26, 0 AS lochierarchy#27] (27) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#15, i_class#14, sum#28, sum#29] +Output [4]: [i_category#28, i_class#29, sum#30, sum#31] (28) HashAggregate [codegen id : 10] -Input [4]: [i_category#15, i_class#14, sum#28, sum#29] -Keys [2]: [i_category#15, i_class#14] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#30, sum(UnscaledValue(ss_ext_sales_price#3))#31] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#30,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#31,17,2) AS ss_ext_sales_price#33, i_category#15] +Input [4]: [i_category#28, i_class#29, sum#30, sum#31] +Keys [2]: [i_category#28, i_class#29] +Functions [2]: [sum(UnscaledValue(ss_net_profit#32)), sum(UnscaledValue(ss_ext_sales_price#33))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#32))#34, sum(UnscaledValue(ss_ext_sales_price#33))#35] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#32))#34,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#33))#35,17,2) AS ss_ext_sales_price#37, i_category#28] (29) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#32, ss_ext_sales_price#33, i_category#15] -Keys [1]: [i_category#15] -Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] -Aggregate Attributes [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] -Results [5]: [i_category#15, sum#38, isEmpty#39, sum#40, isEmpty#41] +Input [3]: [ss_net_profit#36, ss_ext_sales_price#37, i_category#28] +Keys [1]: [i_category#28] +Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] +Aggregate Attributes [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] +Results [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] (30) Exchange -Input [5]: [i_category#15, sum#38, isEmpty#39, sum#40, isEmpty#41] -Arguments: hashpartitioning(i_category#15, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, [id=#46] (31) HashAggregate [codegen id : 11] -Input [5]: [i_category#15, sum#38, isEmpty#39, sum#40, isEmpty#41] -Keys [1]: [i_category#15] -Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] -Aggregate Attributes [2]: [sum(ss_net_profit#32)#43, sum(ss_ext_sales_price#33)#44] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#43) / promote_precision(sum(ss_ext_sales_price#33)#44)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#45, i_category#15, null AS i_class#46, 0 AS t_category#47, 1 AS t_class#48, 1 AS lochierarchy#49] +Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] +Keys [1]: [i_category#28] +Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] +Aggregate Attributes [2]: [sum(ss_net_profit#36)#47, sum(ss_ext_sales_price#37)#48] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#47) / promote_precision(sum(ss_ext_sales_price#37)#48)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#49, i_category#28, null AS i_class#50, 0 AS t_category#51, 1 AS t_class#52, 1 AS lochierarchy#53] (32) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#15, i_class#14, sum#50, sum#51] +Output [4]: [i_category#54, i_class#55, sum#56, sum#57] (33) HashAggregate [codegen id : 16] -Input [4]: [i_category#15, i_class#14, sum#50, sum#51] -Keys [2]: [i_category#15, i_class#14] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#52, sum(UnscaledValue(ss_ext_sales_price#3))#53] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#52,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#53,17,2) AS ss_ext_sales_price#33] +Input [4]: [i_category#54, i_class#55, sum#56, sum#57] +Keys [2]: [i_category#54, i_class#55] +Functions [2]: [sum(UnscaledValue(ss_net_profit#58)), sum(UnscaledValue(ss_ext_sales_price#59))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#58))#60, sum(UnscaledValue(ss_ext_sales_price#59))#61] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#58))#60,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#59))#61,17,2) AS ss_ext_sales_price#37] (34) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#32, ss_ext_sales_price#33] +Input [2]: [ss_net_profit#36, ss_ext_sales_price#37] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] -Aggregate Attributes [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] -Results [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] +Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] +Aggregate Attributes [4]: [sum#62, isEmpty#63, sum#64, isEmpty#65] +Results [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] (35) Exchange -Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] +Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#70] (36) HashAggregate [codegen id : 17] -Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] +Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] Keys: [] -Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] -Aggregate Attributes [2]: [sum(ss_net_profit#32)#63, sum(ss_ext_sales_price#33)#64] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#63) / promote_precision(sum(ss_ext_sales_price#33)#64)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#65, null AS i_category#66, null AS i_class#67, 1 AS t_category#68, 1 AS t_class#69, 2 AS lochierarchy#70] +Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] +Aggregate Attributes [2]: [sum(ss_net_profit#36)#71, sum(ss_ext_sales_price#37)#72] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#71) / promote_precision(sum(ss_ext_sales_price#37)#72)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#73, null AS i_category#74, null AS i_class#75, 1 AS t_category#76, 1 AS t_class#77, 2 AS lochierarchy#78] (37) Union @@ -235,34 +235,34 @@ Results [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class (39) Exchange Input [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27] -Arguments: hashpartitioning(gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#71] +Arguments: hashpartitioning(gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#79] (40) HashAggregate [codegen id : 19] Input [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27] Keys [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27] Functions: [] Aggregate Attributes: [] -Results [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#15 END AS _w0#72] +Results [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#15 END AS _w0#80] (41) Exchange -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72] -Arguments: hashpartitioning(lochierarchy#27, _w0#72, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80] +Arguments: hashpartitioning(lochierarchy#27, _w0#80, 5), ENSURE_REQUIREMENTS, [id=#81] (42) Sort [codegen id : 20] -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72] -Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80] +Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#80 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 (43) Window -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72] -Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#72, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#74], [lochierarchy#27, _w0#72], [gross_margin#24 ASC NULLS FIRST] +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80] +Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#80, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#82], [lochierarchy#27, _w0#80], [gross_margin#24 ASC NULLS FIRST] (44) Project [codegen id : 21] -Output [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#74] -Input [6]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72, rank_within_parent#74] +Output [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#82] +Input [6]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80, rank_within_parent#82] (45) TakeOrderedAndProject -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#74] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#74 ASC NULLS FIRST], [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#74] +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#82] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#82 ASC NULLS FIRST], [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#82] ===== Subqueries ===== 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 5e58913557f22..bc2931129868b 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 @@ -169,60 +169,60 @@ Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#22, sum(UnscaledV Results [6]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#22,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#23,17,2))), DecimalType(37,20), true) as decimal(38,20)) AS gross_margin#24, i_category#12, i_class#11, 0 AS t_category#25, 0 AS t_class#26, 0 AS lochierarchy#27] (27) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#12, i_class#11, sum#28, sum#29] +Output [4]: [i_category#28, i_class#29, sum#30, sum#31] (28) HashAggregate [codegen id : 10] -Input [4]: [i_category#12, i_class#11, sum#28, sum#29] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#30, sum(UnscaledValue(ss_ext_sales_price#3))#31] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#30,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#31,17,2) AS ss_ext_sales_price#33, i_category#12] +Input [4]: [i_category#28, i_class#29, sum#30, sum#31] +Keys [2]: [i_category#28, i_class#29] +Functions [2]: [sum(UnscaledValue(ss_net_profit#32)), sum(UnscaledValue(ss_ext_sales_price#33))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#32))#34, sum(UnscaledValue(ss_ext_sales_price#33))#35] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#32))#34,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#33))#35,17,2) AS ss_ext_sales_price#37, i_category#28] (29) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#32, ss_ext_sales_price#33, i_category#12] -Keys [1]: [i_category#12] -Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] -Aggregate Attributes [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] -Results [5]: [i_category#12, sum#38, isEmpty#39, sum#40, isEmpty#41] +Input [3]: [ss_net_profit#36, ss_ext_sales_price#37, i_category#28] +Keys [1]: [i_category#28] +Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] +Aggregate Attributes [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] +Results [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] (30) Exchange -Input [5]: [i_category#12, sum#38, isEmpty#39, sum#40, isEmpty#41] -Arguments: hashpartitioning(i_category#12, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, [id=#46] (31) HashAggregate [codegen id : 11] -Input [5]: [i_category#12, sum#38, isEmpty#39, sum#40, isEmpty#41] -Keys [1]: [i_category#12] -Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] -Aggregate Attributes [2]: [sum(ss_net_profit#32)#43, sum(ss_ext_sales_price#33)#44] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#43) / promote_precision(sum(ss_ext_sales_price#33)#44)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#45, i_category#12, null AS i_class#46, 0 AS t_category#47, 1 AS t_class#48, 1 AS lochierarchy#49] +Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] +Keys [1]: [i_category#28] +Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] +Aggregate Attributes [2]: [sum(ss_net_profit#36)#47, sum(ss_ext_sales_price#37)#48] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#47) / promote_precision(sum(ss_ext_sales_price#37)#48)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#49, i_category#28, null AS i_class#50, 0 AS t_category#51, 1 AS t_class#52, 1 AS lochierarchy#53] (32) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#12, i_class#11, sum#50, sum#51] +Output [4]: [i_category#54, i_class#55, sum#56, sum#57] (33) HashAggregate [codegen id : 16] -Input [4]: [i_category#12, i_class#11, sum#50, sum#51] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#52, sum(UnscaledValue(ss_ext_sales_price#3))#53] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#52,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#53,17,2) AS ss_ext_sales_price#33] +Input [4]: [i_category#54, i_class#55, sum#56, sum#57] +Keys [2]: [i_category#54, i_class#55] +Functions [2]: [sum(UnscaledValue(ss_net_profit#58)), sum(UnscaledValue(ss_ext_sales_price#59))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#58))#60, sum(UnscaledValue(ss_ext_sales_price#59))#61] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#58))#60,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#59))#61,17,2) AS ss_ext_sales_price#37] (34) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#32, ss_ext_sales_price#33] +Input [2]: [ss_net_profit#36, ss_ext_sales_price#37] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] -Aggregate Attributes [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] -Results [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] +Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] +Aggregate Attributes [4]: [sum#62, isEmpty#63, sum#64, isEmpty#65] +Results [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] (35) Exchange -Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] +Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#70] (36) HashAggregate [codegen id : 17] -Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] +Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] Keys: [] -Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] -Aggregate Attributes [2]: [sum(ss_net_profit#32)#63, sum(ss_ext_sales_price#33)#64] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#63) / promote_precision(sum(ss_ext_sales_price#33)#64)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#65, null AS i_category#66, null AS i_class#67, 1 AS t_category#68, 1 AS t_class#69, 2 AS lochierarchy#70] +Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] +Aggregate Attributes [2]: [sum(ss_net_profit#36)#71, sum(ss_ext_sales_price#37)#72] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#71) / promote_precision(sum(ss_ext_sales_price#37)#72)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#73, null AS i_category#74, null AS i_class#75, 1 AS t_category#76, 1 AS t_class#77, 2 AS lochierarchy#78] (37) Union @@ -235,34 +235,34 @@ Results [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class (39) Exchange Input [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27] -Arguments: hashpartitioning(gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#71] +Arguments: hashpartitioning(gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#79] (40) HashAggregate [codegen id : 19] Input [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27] Keys [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27] Functions: [] Aggregate Attributes: [] -Results [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#12 END AS _w0#72] +Results [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#12 END AS _w0#80] (41) Exchange -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72] -Arguments: hashpartitioning(lochierarchy#27, _w0#72, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80] +Arguments: hashpartitioning(lochierarchy#27, _w0#80, 5), ENSURE_REQUIREMENTS, [id=#81] (42) Sort [codegen id : 20] -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72] -Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80] +Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#80 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 (43) Window -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72] -Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#72, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#74], [lochierarchy#27, _w0#72], [gross_margin#24 ASC NULLS FIRST] +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80] +Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#80, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#82], [lochierarchy#27, _w0#80], [gross_margin#24 ASC NULLS FIRST] (44) Project [codegen id : 21] -Output [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#74] -Input [6]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72, rank_within_parent#74] +Output [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#82] +Input [6]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80, rank_within_parent#82] (45) TakeOrderedAndProject -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#74] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#74 ASC NULLS FIRST], [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#74] +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#82] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#82 ASC NULLS FIRST], [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#82] ===== Subqueries ===== 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 a0737e88bd760..20d80b4503755 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 @@ -213,7 +213,11 @@ Arguments: [rank(d_year#7, d_moy#8) windowspecdefinition(i_category#17, i_brand# (35) Filter [codegen id : 12] Input [9]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28] +<<<<<<< HEAD Condition : ((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#26)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#26 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +======= +Condition : (((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#26)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#26 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#28)) +>>>>>>> regen (36) Exchange Input [9]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28] @@ -229,22 +233,23 @@ Output [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_ye (39) HashAggregate [codegen id : 21] Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum#36] Keys [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35] -Functions [1]: [sum(UnscaledValue(ss_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#3))#37] -Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#3))#37,17,2) AS sum_sales#38] +Functions [1]: [sum(UnscaledValue(ss_sales_price#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#37))#38] +Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#37))#38,17,2) AS sum_sales#39] (40) Exchange -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#40] (41) Sort [codegen id : 22] -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST], false, 0 (42) Window -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +<<<<<<< HEAD (43) Project [codegen id : 23] Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38, rn#40] @@ -256,12 +261,30 @@ Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_compan (45) Sort [codegen id : 24] Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#40 + 1) ASC NULLS FIRST], false, 0 +======= +(43) Filter [codegen id : 23] +Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] +Condition : isnotnull(rn#41) + +(44) Project [codegen id : 23] +Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] + +(45) Exchange +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1), 5), ENSURE_REQUIREMENTS, [id=#42] + +(46) Sort [codegen id : 24] +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#41 + 1) ASC NULLS FIRST], false, 0 +>>>>>>> regen (46) SortMergeJoin [codegen id : 25] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1)] +Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1)] Join condition: None +<<<<<<< HEAD (47) Project [codegen id : 25] Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38] Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] @@ -288,12 +311,45 @@ Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_compan (53) Sort [codegen id : 36] Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (rn#49 - 1) ASC NULLS FIRST], false, 0 +======= +(48) Project [codegen id : 25] +Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39] +Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] + +(49) ReusedExchange [Reuses operator id: 40] +Output [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] + +(50) Sort [codegen id : 34] +Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] +Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST], false, 0 + +(51) Window +Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] +Arguments: [rank(d_year#47, d_moy#48) windowspecdefinition(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#50], [i_category#43, i_brand#44, s_store_name#45, s_company_name#46], [d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST] + +(52) Filter [codegen id : 35] +Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] +Condition : isnotnull(rn#50) + +(53) Project [codegen id : 35] +Output [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] + +(54) Exchange +Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Arguments: hashpartitioning(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1), 5), ENSURE_REQUIREMENTS, [id=#51] + +(55) Sort [codegen id : 36] +Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, (rn#50 - 1) ASC NULLS FIRST], false, 0 +>>>>>>> regen (54) SortMergeJoin [codegen id : 37] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1)] +Right keys [5]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1)] Join condition: None +<<<<<<< HEAD (55) Project [codegen id : 37] Output [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#38 AS psum#51, sum_sales#48 AS nsum#52] Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] @@ -301,6 +357,15 @@ Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_ye (56) TakeOrderedAndProject Input [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] +======= +(57) Project [codegen id : 37] +Output [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#39 AS psum#52, sum_sales#49 AS nsum#53] +Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39, i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] + +(58) TakeOrderedAndProject +Input [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] +>>>>>>> regen ===== Subqueries ===== 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 b0a231fc01379..9017cd21a3873 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 @@ -194,7 +194,11 @@ Arguments: [rank(d_year#11, d_moy#12) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [9]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27] +<<<<<<< HEAD Condition : ((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#25)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +======= +Condition : (((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#25)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#27)) +>>>>>>> regen (33) ReusedExchange [Reuses operator id: unknown] Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34] @@ -202,22 +206,23 @@ Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_ye (34) HashAggregate [codegen id : 13] 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#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#35] -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#6))#35,17,2) AS sum_sales#36] +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] (35) 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#36] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#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#37] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] (36) Sort [codegen id : 14] -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36] +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: [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 (37) 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#36] -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#38], [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#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] +<<<<<<< HEAD (38) Project [codegen id : 15] Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] @@ -225,12 +230,26 @@ Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_yea (39) BroadcastExchange Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] 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=#39] +======= +(38) Filter [codegen id : 15] +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] +Condition : isnotnull(rn#39) + +(39) Project [codegen id : 15] +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] + +(40) BroadcastExchange +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] + 1)),false), [id=#40] +>>>>>>> regen (40) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#38 + 1)] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] Join condition: None +<<<<<<< HEAD (41) Project [codegen id : 23] Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] @@ -253,12 +272,41 @@ Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_yea (46) BroadcastExchange Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] 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=#48] +======= +(42) Project [codegen id : 23] +Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] +Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] + +(43) ReusedExchange [Reuses operator id: 35] +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] + +(44) Sort [codegen id : 21] +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 + +(45) 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] + +(46) Filter [codegen id : 22] +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] +Condition : isnotnull(rn#48) + +(47) Project [codegen id : 22] +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] + +(48) BroadcastExchange +Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] - 1)),false), [id=#49] +>>>>>>> regen (47) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -Right keys [5]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#47 - 1)] +Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] Join condition: None +<<<<<<< HEAD (48) Project [codegen id : 23] Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#49, sum_sales#46 AS nsum#50] Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] @@ -266,6 +314,15 @@ Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year (49) TakeOrderedAndProject Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, 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#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#22, psum#49, nsum#50] +======= +(50) Project [codegen id : 23] +Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] +Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] + +(51) TakeOrderedAndProject +Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, 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#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#22, psum#50, nsum#51] +>>>>>>> regen ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt index 57e57951851aa..a1c7e4bfb35dc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt @@ -250,225 +250,225 @@ Output [5]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_net_profit#47, cs_sold_date_sk#48] (36) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#49] (37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_sold_date_sk#48] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#49] Join condition: None (38) Project [codegen id : 12] Output [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#8] +Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#49] (39) Exchange Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#49] +Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#50] (40) Sort [codegen id : 13] Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] Arguments: [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST], false, 0 (41) Scan parquet default.catalog_returns -Output [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Output [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (42) ColumnarToRow [codegen id : 14] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] (43) Filter [codegen id : 14] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] -Condition : (((isnotnull(cr_return_amount#53) AND (cr_return_amount#53 > 10000.00)) AND isnotnull(cr_order_number#51)) AND isnotnull(cr_item_sk#50)) +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Condition : (((isnotnull(cr_return_amount#54) AND (cr_return_amount#54 > 10000.00)) AND isnotnull(cr_order_number#52)) AND isnotnull(cr_item_sk#51)) (44) Project [codegen id : 14] -Output [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Output [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] (45) Exchange -Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Arguments: hashpartitioning(cr_order_number#52, cr_item_sk#51, 5), ENSURE_REQUIREMENTS, [id=#56] (46) Sort [codegen id : 15] -Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Arguments: [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Arguments: [cr_order_number#52 ASC NULLS FIRST, cr_item_sk#51 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 16] Left keys [2]: [cs_order_number#44, cs_item_sk#43] -Right keys [2]: [cr_order_number#51, cr_item_sk#50] +Right keys [2]: [cr_order_number#52, cr_item_sk#51] Join condition: None (48) Project [codegen id : 16] -Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] -Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] +Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] (49) HashAggregate [codegen id : 16] -Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] +Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] Keys [1]: [cs_item_sk#43] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#52, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#56, sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] -Results [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#53, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] +Results [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] (50) Exchange -Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] -Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#69] (51) HashAggregate [codegen id : 17] -Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] Keys [1]: [cs_item_sk#43] -Functions [4]: [sum(coalesce(cr_return_quantity#52, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#52, 0))#69, sum(coalesce(cs_quantity#45, 0))#70, sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72] -Results [3]: [cs_item_sk#43 AS item#73, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#52, 0))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#70 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#74, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#75] +Functions [4]: [sum(coalesce(cr_return_quantity#53, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#53, 0))#70, sum(coalesce(cs_quantity#45, 0))#71, sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73] +Results [3]: [cs_item_sk#43 AS item#74, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#53, 0))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#71 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#75, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#76] (52) Exchange -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#77] (53) Sort [codegen id : 18] -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: [return_ratio#74 ASC NULLS FIRST], false, 0 +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: [return_ratio#75 ASC NULLS FIRST], false, 0 (54) Window -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: [rank(return_ratio#74) windowspecdefinition(return_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#77], [return_ratio#74 ASC NULLS FIRST] +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: [rank(return_ratio#75) windowspecdefinition(return_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#78], [return_ratio#75 ASC NULLS FIRST] (55) Sort [codegen id : 19] -Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] -Arguments: [currency_ratio#75 ASC NULLS FIRST], false, 0 +Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] +Arguments: [currency_ratio#76 ASC NULLS FIRST], false, 0 (56) Window -Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] -Arguments: [rank(currency_ratio#75) windowspecdefinition(currency_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#78], [currency_ratio#75 ASC NULLS FIRST] +Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] +Arguments: [rank(currency_ratio#76) windowspecdefinition(currency_ratio#76 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#79], [currency_ratio#76 ASC NULLS FIRST] (57) Filter [codegen id : 20] -Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] -Condition : ((return_rank#77 <= 10) OR (currency_rank#78 <= 10)) +Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] +Condition : ((return_rank#78 <= 10) OR (currency_rank#79 <= 10)) (58) Project [codegen id : 20] -Output [5]: [catalog AS channel#79, item#73, return_ratio#74, return_rank#77, currency_rank#78] -Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] +Output [5]: [catalog AS channel#80, item#74, return_ratio#75, return_rank#78, currency_rank#79] +Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] (59) Scan parquet default.store_sales -Output [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Output [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_sold_date_sk#85 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 22] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] (61) Filter [codegen id : 22] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] -Condition : (((((((isnotnull(ss_net_profit#84) AND isnotnull(ss_net_paid#83)) AND isnotnull(ss_quantity#82)) AND (ss_net_profit#84 > 1.00)) AND (ss_net_paid#83 > 0.00)) AND (ss_quantity#82 > 0)) AND isnotnull(ss_ticket_number#81)) AND isnotnull(ss_item_sk#80)) +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Condition : (((((((isnotnull(ss_net_profit#85) AND isnotnull(ss_net_paid#84)) AND isnotnull(ss_quantity#83)) AND (ss_net_profit#85 > 1.00)) AND (ss_net_paid#84 > 0.00)) AND (ss_quantity#83 > 0)) AND isnotnull(ss_ticket_number#82)) AND isnotnull(ss_item_sk#81)) (62) Project [codegen id : 22] -Output [5]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Output [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] (63) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#87] (64) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ss_sold_date_sk#85] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ss_sold_date_sk#86] +Right keys [1]: [d_date_sk#87] Join condition: None (65) Project [codegen id : 22] -Output [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85, d_date_sk#8] +Output [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86, d_date_sk#87] (66) Exchange -Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Arguments: hashpartitioning(cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint), 5), ENSURE_REQUIREMENTS, [id=#86] +Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Arguments: hashpartitioning(cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint), 5), ENSURE_REQUIREMENTS, [id=#88] (67) Sort [codegen id : 23] -Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Arguments: [cast(ss_ticket_number#81 as bigint) ASC NULLS FIRST, cast(ss_item_sk#80 as bigint) ASC NULLS FIRST], false, 0 +Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Arguments: [cast(ss_ticket_number#82 as bigint) ASC NULLS FIRST, cast(ss_item_sk#81 as bigint) ASC NULLS FIRST], false, 0 (68) Scan parquet default.store_returns -Output [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Output [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 24] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] (70) Filter [codegen id : 24] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] -Condition : (((isnotnull(sr_return_amt#90) AND (sr_return_amt#90 > 10000.00)) AND isnotnull(sr_ticket_number#88)) AND isnotnull(sr_item_sk#87)) +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Condition : (((isnotnull(sr_return_amt#92) AND (sr_return_amt#92 > 10000.00)) AND isnotnull(sr_ticket_number#90)) AND isnotnull(sr_item_sk#89)) (71) Project [codegen id : 24] -Output [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Output [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] (72) Exchange -Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Arguments: hashpartitioning(sr_ticket_number#88, sr_item_sk#87, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Arguments: hashpartitioning(sr_ticket_number#90, sr_item_sk#89, 5), ENSURE_REQUIREMENTS, [id=#94] (73) Sort [codegen id : 25] -Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Arguments: [sr_ticket_number#88 ASC NULLS FIRST, sr_item_sk#87 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Arguments: [sr_ticket_number#90 ASC NULLS FIRST, sr_item_sk#89 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 26] -Left keys [2]: [cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint)] -Right keys [2]: [sr_ticket_number#88, sr_item_sk#87] +Left keys [2]: [cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint)] +Right keys [2]: [sr_ticket_number#90, sr_item_sk#89] Join condition: None (75) Project [codegen id : 26] -Output [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] -Input [8]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Output [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] +Input [8]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] (76) HashAggregate [codegen id : 26] -Input [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] -Keys [1]: [ss_item_sk#80] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#89, 0)), partial_sum(coalesce(ss_quantity#82, 0)), partial_sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#93, sum#94, sum#95, isEmpty#96, sum#97, isEmpty#98] -Results [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Input [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] +Keys [1]: [ss_item_sk#81] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#91, 0)), partial_sum(coalesce(ss_quantity#83, 0)), partial_sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#95, sum#96, sum#97, isEmpty#98, sum#99, isEmpty#100] +Results [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] (77) Exchange -Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Arguments: hashpartitioning(ss_item_sk#80, 5), ENSURE_REQUIREMENTS, [id=#105] +Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] +Arguments: hashpartitioning(ss_item_sk#81, 5), ENSURE_REQUIREMENTS, [id=#107] (78) HashAggregate [codegen id : 27] -Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Keys [1]: [ss_item_sk#80] -Functions [4]: [sum(coalesce(sr_return_quantity#89, 0)), sum(coalesce(ss_quantity#82, 0)), sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#89, 0))#106, sum(coalesce(ss_quantity#82, 0))#107, sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108, sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109] -Results [3]: [ss_item_sk#80 AS item#110, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#89, 0))#106 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#82, 0))#107 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#111, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#112] +Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] +Keys [1]: [ss_item_sk#81] +Functions [4]: [sum(coalesce(sr_return_quantity#91, 0)), sum(coalesce(ss_quantity#83, 0)), sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#91, 0))#108, sum(coalesce(ss_quantity#83, 0))#109, sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110, sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111] +Results [3]: [ss_item_sk#81 AS item#112, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#91, 0))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#83, 0))#109 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#113, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#114] (79) Exchange -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] (80) Sort [codegen id : 28] -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: [return_ratio#111 ASC NULLS FIRST], false, 0 +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: [return_ratio#113 ASC NULLS FIRST], false, 0 (81) Window -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: [rank(return_ratio#111) windowspecdefinition(return_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#114], [return_ratio#111 ASC NULLS FIRST] +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: [rank(return_ratio#113) windowspecdefinition(return_ratio#113 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#116], [return_ratio#113 ASC NULLS FIRST] (82) Sort [codegen id : 29] -Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] -Arguments: [currency_ratio#112 ASC NULLS FIRST], false, 0 +Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] +Arguments: [currency_ratio#114 ASC NULLS FIRST], false, 0 (83) Window -Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] -Arguments: [rank(currency_ratio#112) windowspecdefinition(currency_ratio#112 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#115], [currency_ratio#112 ASC NULLS FIRST] +Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] +Arguments: [rank(currency_ratio#114) windowspecdefinition(currency_ratio#114 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#117], [currency_ratio#114 ASC NULLS FIRST] (84) Filter [codegen id : 30] -Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] -Condition : ((return_rank#114 <= 10) OR (currency_rank#115 <= 10)) +Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] +Condition : ((return_rank#116 <= 10) OR (currency_rank#117 <= 10)) (85) Project [codegen id : 30] -Output [5]: [store AS channel#116, item#110, return_ratio#111, return_rank#114, currency_rank#115] -Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] +Output [5]: [store AS channel#118, item#112, return_ratio#113, return_rank#116, currency_rank#117] +Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] (86) Union @@ -481,7 +481,7 @@ Results [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_ran (88) Exchange Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] -Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#117] +Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#119] (89) HashAggregate [codegen id : 32] Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] @@ -505,6 +505,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#85 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index ada8c0415c672..74b789e4a7a65 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -260,170 +260,170 @@ Output [6]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, Input [9]: [cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_item_sk#49, cr_order_number#50, cr_return_quantity#51, cr_return_amount#52] (40) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#54] (41) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#47] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#54] Join condition: None (42) Project [codegen id : 10] Output [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] -Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#14] +Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#54] (43) HashAggregate [codegen id : 10] Input [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] Keys [1]: [cs_item_sk#42] Functions [4]: [partial_sum(coalesce(cr_return_quantity#51, 0)), partial_sum(coalesce(cs_quantity#44, 0)), partial_sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Results [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] +Aggregate Attributes [6]: [sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Results [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] (44) Exchange -Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] -Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#67] (45) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] +Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] Keys [1]: [cs_item_sk#42] Functions [4]: [sum(coalesce(cr_return_quantity#51, 0)), sum(coalesce(cs_quantity#44, 0)), sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#67, sum(coalesce(cs_quantity#44, 0))#68, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70] -Results [3]: [cs_item_sk#42 AS item#71, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#67 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#68 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#72, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#73] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#68, sum(coalesce(cs_quantity#44, 0))#69, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71] +Results [3]: [cs_item_sk#42 AS item#72, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#68 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#69 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#73, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#74] (46) Exchange -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#74] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#75] (47) Sort [codegen id : 12] -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: [return_ratio#72 ASC NULLS FIRST], false, 0 +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [return_ratio#73 ASC NULLS FIRST], false, 0 (48) Window -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: [rank(return_ratio#72) windowspecdefinition(return_ratio#72 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#72 ASC NULLS FIRST] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#76], [return_ratio#73 ASC NULLS FIRST] (49) Sort [codegen id : 13] -Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] -Arguments: [currency_ratio#73 ASC NULLS FIRST], false, 0 +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 (50) Window -Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] -Arguments: [rank(currency_ratio#73) windowspecdefinition(currency_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#73 ASC NULLS FIRST] +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#77], [currency_ratio#74 ASC NULLS FIRST] (51) Filter [codegen id : 14] -Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] +Condition : ((return_rank#76 <= 10) OR (currency_rank#77 <= 10)) (52) Project [codegen id : 14] -Output [5]: [catalog AS channel#77, item#71, return_ratio#72, return_rank#75, currency_rank#76] -Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] +Output [5]: [catalog AS channel#78, item#72, return_ratio#73, return_rank#76, currency_rank#77] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] (53) Scan parquet default.store_sales -Output [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Output [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#83), dynamicpruningexpression(ss_sold_date_sk#83 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#84), dynamicpruningexpression(ss_sold_date_sk#84 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 15] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] (55) Filter [codegen id : 15] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] -Condition : (((((((isnotnull(ss_net_profit#82) AND isnotnull(ss_net_paid#81)) AND isnotnull(ss_quantity#80)) AND (ss_net_profit#82 > 1.00)) AND (ss_net_paid#81 > 0.00)) AND (ss_quantity#80 > 0)) AND isnotnull(ss_ticket_number#79)) AND isnotnull(ss_item_sk#78)) +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Condition : (((((((isnotnull(ss_net_profit#83) AND isnotnull(ss_net_paid#82)) AND isnotnull(ss_quantity#81)) AND (ss_net_profit#83 > 1.00)) AND (ss_net_paid#82 > 0.00)) AND (ss_quantity#81 > 0)) AND isnotnull(ss_ticket_number#80)) AND isnotnull(ss_item_sk#79)) (56) Project [codegen id : 15] -Output [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Output [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] (57) BroadcastExchange -Input [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#84] +Input [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#85] (58) Scan parquet default.store_returns -Output [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Output [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (59) ColumnarToRow -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] (60) Filter -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] -Condition : (((isnotnull(sr_return_amt#88) AND (sr_return_amt#88 > 10000.00)) AND isnotnull(sr_ticket_number#86)) AND isnotnull(sr_item_sk#85)) +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Condition : (((isnotnull(sr_return_amt#89) AND (sr_return_amt#89 > 10000.00)) AND isnotnull(sr_ticket_number#87)) AND isnotnull(sr_item_sk#86)) (61) Project -Output [4]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Output [4]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] (62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [cast(ss_ticket_number#79 as bigint), cast(ss_item_sk#78 as bigint)] -Right keys [2]: [sr_ticket_number#86, sr_item_sk#85] +Left keys [2]: [cast(ss_ticket_number#80 as bigint), cast(ss_item_sk#79 as bigint)] +Right keys [2]: [sr_ticket_number#87, sr_item_sk#86] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88] -Input [9]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] +Output [6]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89] +Input [9]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] (64) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#91] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#83] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ss_sold_date_sk#84] +Right keys [1]: [d_date_sk#91] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] -Input [7]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88, d_date_sk#14] +Output [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] +Input [7]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89, d_date_sk#91] (67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] -Keys [1]: [ss_item_sk#78] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#87, 0)), partial_sum(coalesce(ss_quantity#80, 0)), partial_sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Results [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Input [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] +Keys [1]: [ss_item_sk#79] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#88, 0)), partial_sum(coalesce(ss_quantity#81, 0)), partial_sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] +Results [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] (68) Exchange -Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Arguments: hashpartitioning(ss_item_sk#78, 5), ENSURE_REQUIREMENTS, [id=#102] +Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Arguments: hashpartitioning(ss_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#104] (69) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Keys [1]: [ss_item_sk#78] -Functions [4]: [sum(coalesce(sr_return_quantity#87, 0)), sum(coalesce(ss_quantity#80, 0)), sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#87, 0))#103, sum(coalesce(ss_quantity#80, 0))#104, sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105, sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106] -Results [3]: [ss_item_sk#78 AS item#107, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#87, 0))#103 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#80, 0))#104 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#108, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#109] +Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Keys [1]: [ss_item_sk#79] +Functions [4]: [sum(coalesce(sr_return_quantity#88, 0)), sum(coalesce(ss_quantity#81, 0)), sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#88, 0))#105, sum(coalesce(ss_quantity#81, 0))#106, sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107, sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108] +Results [3]: [ss_item_sk#79 AS item#109, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#88, 0))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#81, 0))#106 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#110, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#111] (70) Exchange -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#112] (71) Sort [codegen id : 19] -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: [return_ratio#108 ASC NULLS FIRST], false, 0 +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: [return_ratio#110 ASC NULLS FIRST], false, 0 (72) Window -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: [rank(return_ratio#108) windowspecdefinition(return_ratio#108 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#111], [return_ratio#108 ASC NULLS FIRST] +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: [rank(return_ratio#110) windowspecdefinition(return_ratio#110 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#113], [return_ratio#110 ASC NULLS FIRST] (73) Sort [codegen id : 20] -Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] -Arguments: [currency_ratio#109 ASC NULLS FIRST], false, 0 +Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] +Arguments: [currency_ratio#111 ASC NULLS FIRST], false, 0 (74) Window -Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] -Arguments: [rank(currency_ratio#109) windowspecdefinition(currency_ratio#109 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#112], [currency_ratio#109 ASC NULLS FIRST] +Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] +Arguments: [rank(currency_ratio#111) windowspecdefinition(currency_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#114], [currency_ratio#111 ASC NULLS FIRST] (75) Filter [codegen id : 21] -Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] -Condition : ((return_rank#111 <= 10) OR (currency_rank#112 <= 10)) +Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] +Condition : ((return_rank#113 <= 10) OR (currency_rank#114 <= 10)) (76) Project [codegen id : 21] -Output [5]: [store AS channel#113, item#107, return_ratio#108, return_rank#111, currency_rank#112] -Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] +Output [5]: [store AS channel#115, item#109, return_ratio#110, return_rank#113, currency_rank#114] +Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] (77) Union @@ -436,7 +436,7 @@ Results [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_ran (79) Exchange Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] -Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#114] +Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#116] (80) HashAggregate [codegen id : 23] Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] @@ -460,6 +460,6 @@ Output [1]: [d_date_sk#14] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#83 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#84 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt index bcf47f68f2e59..957e2d9a015b4 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 @@ -163,6 +163,7 @@ Arguments: hashpartitioning(item_sk#13, 5), ENSURE_REQUIREMENTS, [id=#17] Input [4]: [item_sk#13, d_date#6, sumws#14, rk#16] Arguments: [item_sk#13 ASC NULLS FIRST], false, 0 +<<<<<<< HEAD (20) ReusedExchange [Reuses operator id: unknown] Output [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] @@ -184,31 +185,64 @@ Arguments: hashpartitioning(item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#22] (25) Sort [codegen id : 12] Input [3]: [item_sk#18, sumws#20, rk#21] +======= +(21) ReusedExchange [Reuses operator id: unknown] +Output [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] + +(22) Sort [codegen id : 10] +Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] +Arguments: [ws_item_sk#21 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST], false, 0 + +(23) Window +Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] +Arguments: [row_number() windowspecdefinition(ws_item_sk#21, d_date#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#22], [ws_item_sk#21], [d_date#19 ASC NULLS FIRST] + +(24) Filter [codegen id : 11] +Input [5]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21, rk#22] +Condition : isnotnull(rk#22) + +(25) Project [codegen id : 11] +Output [3]: [item_sk#18, sumws#20, rk#22] +Input [5]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21, rk#22] + +(26) Exchange +Input [3]: [item_sk#18, sumws#20, rk#22] +Arguments: hashpartitioning(item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#23] + +(27) Sort [codegen id : 12] +Input [3]: [item_sk#18, sumws#20, rk#22] +>>>>>>> regen Arguments: [item_sk#18 ASC NULLS FIRST], false, 0 (26) SortMergeJoin [codegen id : 13] Left keys [1]: [item_sk#13] Right keys [1]: [item_sk#18] -Join condition: (rk#16 >= rk#21) +Join condition: (rk#16 >= rk#22) (27) Project [codegen id : 13] Output [4]: [item_sk#13, d_date#6, sumws#14, sumws#20] -Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#18, sumws#20, rk#21] +Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#18, sumws#20, rk#22] (28) HashAggregate [codegen id : 13] Input [4]: [item_sk#13, d_date#6, sumws#14, sumws#20] Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [partial_sum(sumws#20)] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +Aggregate Attributes [2]: [sum#24, isEmpty#25] +Results [5]: [item_sk#13, d_date#6, sumws#14, sum#26, isEmpty#27] +<<<<<<< HEAD (29) HashAggregate [codegen id : 13] Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +======= +(31) HashAggregate [codegen id : 13] +Input [5]: [item_sk#13, d_date#6, sumws#14, sum#26, isEmpty#27] +>>>>>>> regen Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [sum(sumws#20)] -Aggregate Attributes [1]: [sum(sumws#20)#27] -Results [3]: [item_sk#13, d_date#6, sum(sumws#20)#27 AS cume_sales#28] +Aggregate Attributes [1]: [sum(sumws#20)#28] +Results [3]: [item_sk#13, d_date#6, sum(sumws#20)#28 AS cume_sales#29] +<<<<<<< HEAD (30) Exchange Input [3]: [item_sk#13, d_date#6, cume_sales#28] Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#29] @@ -219,12 +253,25 @@ Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 (32) Scan parquet default.store_sales Output [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +======= +(32) Exchange +Input [3]: [item_sk#13, d_date#6, cume_sales#29] +Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#30] + +(33) Sort [codegen id : 14] +Input [3]: [item_sk#13, d_date#6, cume_sales#29] +Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 + +(34) Scan parquet default.store_sales +Output [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] +>>>>>>> regen Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#33), dynamicpruningexpression(ss_sold_date_sk#33 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct +<<<<<<< HEAD (33) ColumnarToRow [codegen id : 16] Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] @@ -339,12 +386,137 @@ Arguments: hashpartitioning(item_sk#39, d_date#34, 5), ENSURE_REQUIREMENTS, [id= (58) Sort [codegen id : 28] Input [3]: [item_sk#39, d_date#34, cume_sales#54] Arguments: [item_sk#39 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 +======= +(35) ColumnarToRow [codegen id : 16] +Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] + +(36) Filter [codegen id : 16] +Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] +Condition : isnotnull(ss_item_sk#31) + +(37) ReusedExchange [Reuses operator id: 8] +Output [2]: [d_date_sk#34, d_date#35] + +(38) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ss_sold_date_sk#33] +Right keys [1]: [d_date_sk#34] +Join condition: None + +(39) Project [codegen id : 16] +Output [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] +Input [5]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33, d_date_sk#34, d_date#35] + +(40) HashAggregate [codegen id : 16] +Input [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] +Keys [2]: [ss_item_sk#31, d_date#35] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#32))] +Aggregate Attributes [1]: [sum#36] +Results [3]: [ss_item_sk#31, d_date#35, sum#37] + +(41) Exchange +Input [3]: [ss_item_sk#31, d_date#35, sum#37] +Arguments: hashpartitioning(ss_item_sk#31, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#38] + +(42) HashAggregate [codegen id : 17] +Input [3]: [ss_item_sk#31, d_date#35, sum#37] +Keys [2]: [ss_item_sk#31, d_date#35] +Functions [1]: [sum(UnscaledValue(ss_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#32))#39] +Results [4]: [ss_item_sk#31 AS item_sk#40, d_date#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#32))#39,17,2) AS sumss#41, ss_item_sk#31] + +(43) Exchange +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#42] + +(44) Sort [codegen id : 18] +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 + +(45) Window +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: [row_number() windowspecdefinition(ss_item_sk#31, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [ss_item_sk#31], [d_date#35 ASC NULLS FIRST] + +(46) Filter [codegen id : 19] +Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] +Condition : isnotnull(rk#43) + +(47) Project [codegen id : 19] +Output [4]: [item_sk#40, d_date#35, sumss#41, rk#43] +Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] + +(48) Exchange +Input [4]: [item_sk#40, d_date#35, sumss#41, rk#43] +Arguments: hashpartitioning(item_sk#40, 5), ENSURE_REQUIREMENTS, [id=#44] + +(49) Sort [codegen id : 20] +Input [4]: [item_sk#40, d_date#35, sumss#41, rk#43] +Arguments: [item_sk#40 ASC NULLS FIRST], false, 0 + +(50) ReusedExchange [Reuses operator id: unknown] +Output [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] + +(51) Sort [codegen id : 24] +Input [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] +Arguments: [ss_item_sk#48 ASC NULLS FIRST, d_date#46 ASC NULLS FIRST], false, 0 + +(52) Window +Input [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] +Arguments: [row_number() windowspecdefinition(ss_item_sk#48, d_date#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [ss_item_sk#48], [d_date#46 ASC NULLS FIRST] + +(53) Filter [codegen id : 25] +Input [5]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48, rk#49] +Condition : isnotnull(rk#49) + +(54) Project [codegen id : 25] +Output [3]: [item_sk#45, sumss#47, rk#49] +Input [5]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48, rk#49] + +(55) Exchange +Input [3]: [item_sk#45, sumss#47, rk#49] +Arguments: hashpartitioning(item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#50] + +(56) Sort [codegen id : 26] +Input [3]: [item_sk#45, sumss#47, rk#49] +Arguments: [item_sk#45 ASC NULLS FIRST], false, 0 + +(57) SortMergeJoin [codegen id : 27] +Left keys [1]: [item_sk#40] +Right keys [1]: [item_sk#45] +Join condition: (rk#43 >= rk#49) + +(58) Project [codegen id : 27] +Output [4]: [item_sk#40, d_date#35, sumss#41, sumss#47] +Input [7]: [item_sk#40, d_date#35, sumss#41, rk#43, item_sk#45, sumss#47, rk#49] + +(59) HashAggregate [codegen id : 27] +Input [4]: [item_sk#40, d_date#35, sumss#41, sumss#47] +Keys [3]: [item_sk#40, d_date#35, sumss#41] +Functions [1]: [partial_sum(sumss#47)] +Aggregate Attributes [2]: [sum#51, isEmpty#52] +Results [5]: [item_sk#40, d_date#35, sumss#41, sum#53, isEmpty#54] + +(60) HashAggregate [codegen id : 27] +Input [5]: [item_sk#40, d_date#35, sumss#41, sum#53, isEmpty#54] +Keys [3]: [item_sk#40, d_date#35, sumss#41] +Functions [1]: [sum(sumss#47)] +Aggregate Attributes [1]: [sum(sumss#47)#55] +Results [3]: [item_sk#40, d_date#35, sum(sumss#47)#55 AS cume_sales#56] + +(61) Exchange +Input [3]: [item_sk#40, d_date#35, cume_sales#56] +Arguments: hashpartitioning(item_sk#40, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#57] + +(62) Sort [codegen id : 28] +Input [3]: [item_sk#40, d_date#35, cume_sales#56] +Arguments: [item_sk#40 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 +>>>>>>> regen (59) SortMergeJoin Left keys [2]: [item_sk#13, d_date#6] -Right keys [2]: [item_sk#39, d_date#34] +Right keys [2]: [item_sk#40, d_date#35] Join condition: None +<<<<<<< HEAD (60) Filter [codegen id : 29] Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END) @@ -410,6 +582,81 @@ Condition : ((isnotnull(web_cumulative#73) AND isnotnull(store_cumulative#74)) A (74) 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] +======= +(64) Filter [codegen id : 29] +Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END) + +(65) Project [codegen id : 29] +Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END AS item_sk#58, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#35 END AS d_date#59, cume_sales#29 AS web_sales#60, cume_sales#56 AS store_sales#61] +Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] + +(66) Exchange +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: hashpartitioning(item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#62] + +(67) Sort [codegen id : 30] +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], false, 0 + +(68) Window +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: [row_number() windowspecdefinition(item_sk#58, d_date#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#63], [item_sk#58], [d_date#59 ASC NULLS FIRST] + +(69) Filter [codegen id : 31] +Input [5]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63] +Condition : isnotnull(rk#63) + +(70) ReusedExchange [Reuses operator id: unknown] +Output [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] + +(71) Sort [codegen id : 61] +Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] +Arguments: [item_sk#64 ASC NULLS FIRST, d_date#65 ASC NULLS FIRST], false, 0 + +(72) Window +Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] +Arguments: [row_number() windowspecdefinition(item_sk#64, d_date#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#68], [item_sk#64], [d_date#65 ASC NULLS FIRST] + +(73) Filter [codegen id : 62] +Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] +Condition : isnotnull(rk#68) + +(74) Project [codegen id : 62] +Output [4]: [item_sk#64, web_sales#66, store_sales#67, rk#68] +Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] + +(75) SortMergeJoin [codegen id : 63] +Left keys [1]: [item_sk#58] +Right keys [1]: [item_sk#64] +Join condition: (rk#63 >= rk#68) + +(76) Project [codegen id : 63] +Output [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] +Input [9]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63, item_sk#64, web_sales#66, store_sales#67, rk#68] + +(77) HashAggregate [codegen id : 63] +Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] +Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Functions [2]: [partial_max(web_sales#66), partial_max(store_sales#67)] +Aggregate Attributes [2]: [max#69, max#70] +Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#71, max#72] + +(78) HashAggregate [codegen id : 63] +Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#71, max#72] +Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Functions [2]: [max(web_sales#66), max(store_sales#67)] +Aggregate Attributes [2]: [max(web_sales#66)#73, max(store_sales#67)#74] +Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max(web_sales#66)#73 AS web_cumulative#75, max(store_sales#67)#74 AS store_cumulative#76] + +(79) Filter [codegen id : 63] +Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] +Condition : ((isnotnull(web_cumulative#75) AND isnotnull(store_cumulative#76)) AND (web_cumulative#75 > store_cumulative#76)) + +(80) TakeOrderedAndProject +Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] +Arguments: 100, [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] +>>>>>>> regen ===== Subqueries ===== @@ -420,6 +667,10 @@ ReusedExchange (75) (75) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#5, d_date#6] +<<<<<<< HEAD Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#4 +======= +Subquery:2 Hosting operator id = 34 Hosting Expression = ss_sold_date_sk#33 IN dynamicpruning#4 +>>>>>>> regen 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 8aaaf48a8162c..471fe83a95901 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 @@ -152,6 +152,7 @@ Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS F Output [4]: [item_sk#13, d_date#6, sumws#14, rk#16] Input [5]: [item_sk#13, d_date#6, sumws#14, ws_item_sk#1, rk#16] +<<<<<<< HEAD (18) ReusedExchange [Reuses operator id: unknown] Output [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] @@ -170,34 +171,68 @@ Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1, rk#20] (22) BroadcastExchange Input [3]: [item_sk#17, sumws#19, rk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +======= +(19) ReusedExchange [Reuses operator id: unknown] +Output [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] + +(20) Sort [codegen id : 8] +Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] +Arguments: [ws_item_sk#20 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 + +(21) Window +Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] +Arguments: [row_number() windowspecdefinition(ws_item_sk#20, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#21], [ws_item_sk#20], [d_date#18 ASC NULLS FIRST] + +(22) Filter [codegen id : 9] +Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20, rk#21] +Condition : isnotnull(rk#21) + +(23) Project [codegen id : 9] +Output [3]: [item_sk#17, sumws#19, rk#21] +Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20, rk#21] + +(24) BroadcastExchange +Input [3]: [item_sk#17, sumws#19, rk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +>>>>>>> regen (23) BroadcastHashJoin [codegen id : 10] Left keys [1]: [item_sk#13] Right keys [1]: [item_sk#17] -Join condition: (rk#16 >= rk#20) +Join condition: (rk#16 >= rk#21) (24) Project [codegen id : 10] Output [4]: [item_sk#13, d_date#6, sumws#14, sumws#19] -Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#17, sumws#19, rk#20] +Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#17, sumws#19, rk#21] (25) HashAggregate [codegen id : 10] Input [4]: [item_sk#13, d_date#6, sumws#14, sumws#19] Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [partial_sum(sumws#19)] -Aggregate Attributes [2]: [sum#22, isEmpty#23] -Results [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] +Aggregate Attributes [2]: [sum#23, isEmpty#24] +Results [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +<<<<<<< HEAD (26) Exchange Input [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] Arguments: hashpartitioning(item_sk#13, d_date#6, sumws#14, 5), ENSURE_REQUIREMENTS, [id=#26] (27) HashAggregate [codegen id : 11] Input [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] +======= +(28) Exchange +Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +Arguments: hashpartitioning(item_sk#13, d_date#6, sumws#14, 5), ENSURE_REQUIREMENTS, [id=#27] + +(29) HashAggregate [codegen id : 11] +Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +>>>>>>> regen Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [sum(sumws#19)] -Aggregate Attributes [1]: [sum(sumws#19)#27] -Results [3]: [item_sk#13, d_date#6, sum(sumws#19)#27 AS cume_sales#28] +Aggregate Attributes [1]: [sum(sumws#19)#28] +Results [3]: [item_sk#13, d_date#6, sum(sumws#19)#28 AS cume_sales#29] +<<<<<<< HEAD (28) Exchange Input [3]: [item_sk#13, d_date#6, cume_sales#28] Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#29] @@ -208,12 +243,25 @@ Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 (30) Scan parquet default.store_sales Output [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +======= +(30) Exchange +Input [3]: [item_sk#13, d_date#6, cume_sales#29] +Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#30] + +(31) Sort [codegen id : 12] +Input [3]: [item_sk#13, d_date#6, cume_sales#29] +Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 + +(32) Scan parquet default.store_sales +Output [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] +>>>>>>> regen Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#33), dynamicpruningexpression(ss_sold_date_sk#33 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct +<<<<<<< HEAD (31) ColumnarToRow [codegen id : 14] Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] @@ -320,12 +368,129 @@ Arguments: hashpartitioning(item_sk#39, d_date#34, 5), ENSURE_REQUIREMENTS, [id= (54) Sort [codegen id : 24] Input [3]: [item_sk#39, d_date#34, cume_sales#54] Arguments: [item_sk#39 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 +======= +(33) ColumnarToRow [codegen id : 14] +Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] + +(34) Filter [codegen id : 14] +Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] +Condition : isnotnull(ss_item_sk#31) + +(35) ReusedExchange [Reuses operator id: 8] +Output [2]: [d_date_sk#34, d_date#35] + +(36) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ss_sold_date_sk#33] +Right keys [1]: [d_date_sk#34] +Join condition: None + +(37) Project [codegen id : 14] +Output [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] +Input [5]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33, d_date_sk#34, d_date#35] + +(38) HashAggregate [codegen id : 14] +Input [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] +Keys [2]: [ss_item_sk#31, d_date#35] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#32))] +Aggregate Attributes [1]: [sum#36] +Results [3]: [ss_item_sk#31, d_date#35, sum#37] + +(39) Exchange +Input [3]: [ss_item_sk#31, d_date#35, sum#37] +Arguments: hashpartitioning(ss_item_sk#31, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#38] + +(40) HashAggregate [codegen id : 15] +Input [3]: [ss_item_sk#31, d_date#35, sum#37] +Keys [2]: [ss_item_sk#31, d_date#35] +Functions [1]: [sum(UnscaledValue(ss_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#32))#39] +Results [4]: [ss_item_sk#31 AS item_sk#40, d_date#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#32))#39,17,2) AS sumss#41, ss_item_sk#31] + +(41) Exchange +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#42] + +(42) Sort [codegen id : 16] +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 + +(43) Window +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: [row_number() windowspecdefinition(ss_item_sk#31, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [ss_item_sk#31], [d_date#35 ASC NULLS FIRST] + +(44) Filter [codegen id : 22] +Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] +Condition : isnotnull(rk#43) + +(45) Project [codegen id : 22] +Output [4]: [item_sk#40, d_date#35, sumss#41, rk#43] +Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] + +(46) ReusedExchange [Reuses operator id: unknown] +Output [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] + +(47) Sort [codegen id : 20] +Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], false, 0 + +(48) Window +Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] +Arguments: [row_number() windowspecdefinition(ss_item_sk#47, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [ss_item_sk#47], [d_date#45 ASC NULLS FIRST] + +(49) Filter [codegen id : 21] +Input [5]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47, rk#48] +Condition : isnotnull(rk#48) + +(50) Project [codegen id : 21] +Output [3]: [item_sk#44, sumss#46, rk#48] +Input [5]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47, rk#48] + +(51) BroadcastExchange +Input [3]: [item_sk#44, sumss#46, rk#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] + +(52) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [item_sk#40] +Right keys [1]: [item_sk#44] +Join condition: (rk#43 >= rk#48) + +(53) Project [codegen id : 22] +Output [4]: [item_sk#40, d_date#35, sumss#41, sumss#46] +Input [7]: [item_sk#40, d_date#35, sumss#41, rk#43, item_sk#44, sumss#46, rk#48] + +(54) HashAggregate [codegen id : 22] +Input [4]: [item_sk#40, d_date#35, sumss#41, sumss#46] +Keys [3]: [item_sk#40, d_date#35, sumss#41] +Functions [1]: [partial_sum(sumss#46)] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] + +(55) Exchange +Input [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] +Arguments: hashpartitioning(item_sk#40, d_date#35, sumss#41, 5), ENSURE_REQUIREMENTS, [id=#54] + +(56) HashAggregate [codegen id : 23] +Input [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] +Keys [3]: [item_sk#40, d_date#35, sumss#41] +Functions [1]: [sum(sumss#46)] +Aggregate Attributes [1]: [sum(sumss#46)#55] +Results [3]: [item_sk#40, d_date#35, sum(sumss#46)#55 AS cume_sales#56] + +(57) Exchange +Input [3]: [item_sk#40, d_date#35, cume_sales#56] +Arguments: hashpartitioning(item_sk#40, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#57] + +(58) Sort [codegen id : 24] +Input [3]: [item_sk#40, d_date#35, cume_sales#56] +Arguments: [item_sk#40 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 +>>>>>>> regen (55) SortMergeJoin Left keys [2]: [item_sk#13, d_date#6] -Right keys [2]: [item_sk#39, d_date#34] +Right keys [2]: [item_sk#40, d_date#35] Join condition: None +<<<<<<< HEAD (56) Filter [codegen id : 25] Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END) @@ -395,6 +560,85 @@ Condition : ((isnotnull(web_cumulative#74) AND isnotnull(store_cumulative#75)) A (71) 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] +======= +(60) Filter [codegen id : 25] +Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END) + +(61) Project [codegen id : 25] +Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END AS item_sk#58, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#35 END AS d_date#59, cume_sales#29 AS web_sales#60, cume_sales#56 AS store_sales#61] +Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] + +(62) Exchange +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: hashpartitioning(item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#62] + +(63) Sort [codegen id : 26] +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], false, 0 + +(64) Window +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: [row_number() windowspecdefinition(item_sk#58, d_date#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#63], [item_sk#58], [d_date#59 ASC NULLS FIRST] + +(65) Filter [codegen id : 54] +Input [5]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63] +Condition : isnotnull(rk#63) + +(66) ReusedExchange [Reuses operator id: unknown] +Output [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] + +(67) Sort [codegen id : 52] +Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] +Arguments: [item_sk#64 ASC NULLS FIRST, d_date#65 ASC NULLS FIRST], false, 0 + +(68) Window +Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] +Arguments: [row_number() windowspecdefinition(item_sk#64, d_date#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#68], [item_sk#64], [d_date#65 ASC NULLS FIRST] + +(69) Filter [codegen id : 53] +Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] +Condition : isnotnull(rk#68) + +(70) Project [codegen id : 53] +Output [4]: [item_sk#64, web_sales#66, store_sales#67, rk#68] +Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] + +(71) BroadcastExchange +Input [4]: [item_sk#64, web_sales#66, store_sales#67, rk#68] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] + +(72) BroadcastHashJoin [codegen id : 54] +Left keys [1]: [item_sk#58] +Right keys [1]: [item_sk#64] +Join condition: (rk#63 >= rk#68) + +(73) Project [codegen id : 54] +Output [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] +Input [9]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63, item_sk#64, web_sales#66, store_sales#67, rk#68] + +(74) HashAggregate [codegen id : 54] +Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] +Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Functions [2]: [partial_max(web_sales#66), partial_max(store_sales#67)] +Aggregate Attributes [2]: [max#70, max#71] +Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#72, max#73] + +(75) HashAggregate [codegen id : 54] +Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#72, max#73] +Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Functions [2]: [max(web_sales#66), max(store_sales#67)] +Aggregate Attributes [2]: [max(web_sales#66)#74, max(store_sales#67)#75] +Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max(web_sales#66)#74 AS web_cumulative#76, max(store_sales#67)#75 AS store_cumulative#77] + +(76) Filter [codegen id : 54] +Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] +Condition : ((isnotnull(web_cumulative#76) AND isnotnull(store_cumulative#77)) AND (web_cumulative#76 > store_cumulative#77)) + +(77) TakeOrderedAndProject +Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] +Arguments: 100, [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] +>>>>>>> regen ===== Subqueries ===== @@ -405,6 +649,10 @@ ReusedExchange (72) (72) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#5, d_date#6] +<<<<<<< HEAD Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#4 +======= +Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#33 IN dynamicpruning#4 +>>>>>>> regen 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 ed0399519f579..8d2776513f90b 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 @@ -213,7 +213,11 @@ Arguments: [rank(d_year#7, d_moy#8) windowspecdefinition(i_category#16, i_brand# (35) Filter [codegen id : 12] Input [8]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27] +<<<<<<< HEAD Condition : ((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#25)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +======= +Condition : (((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#25)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#27)) +>>>>>>> regen (36) Exchange Input [8]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27] @@ -229,22 +233,23 @@ Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] (39) HashAggregate [codegen id : 21] Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] Keys [5]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33] -Functions [1]: [sum(UnscaledValue(cs_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#3))#35] -Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#3))#35,17,2) AS sum_sales#36] +Functions [1]: [sum(UnscaledValue(cs_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#35))#36] +Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#35))#36,17,2) AS sum_sales#37] (40) Exchange -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] (41) Sort [codegen id : 22] -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 (42) Window -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] -Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +<<<<<<< HEAD (43) Project [codegen id : 23] Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] @@ -256,12 +261,30 @@ Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#38 + 1), (45) Sort [codegen id : 24] Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#38 + 1) ASC NULLS FIRST], false, 0 +======= +(43) Filter [codegen id : 23] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] +Condition : isnotnull(rn#39) + +(44) Project [codegen id : 23] +Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] + +(45) Exchange +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] + +(46) Sort [codegen id : 24] +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 +>>>>>>> regen (46) SortMergeJoin [codegen id : 25] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#38 + 1)] +Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#39 + 1)] Join condition: None +<<<<<<< HEAD (47) Project [codegen id : 25] Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] @@ -288,12 +311,45 @@ Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (rn#46 - 1), (53) Sort [codegen id : 36] Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 +======= +(48) Project [codegen id : 25] +Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] +Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] + +(49) ReusedExchange [Reuses operator id: 40] +Output [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] + +(50) Sort [codegen id : 34] +Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 + +(51) Window +Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] +Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#41, i_brand#42, cc_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#41, i_brand#42, cc_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] + +(52) Filter [codegen id : 35] +Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] +Condition : isnotnull(rn#47) + +(53) Project [codegen id : 35] +Output [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] + +(54) Exchange +Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Arguments: hashpartitioning(i_category#41, i_brand#42, cc_name#43, (rn#47 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] + +(55) Sort [codegen id : 36] +Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, (rn#47 - 1) ASC NULLS FIRST], false, 0 +>>>>>>> regen (54) SortMergeJoin [codegen id : 37] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (rn#46 - 1)] +Right keys [4]: [i_category#41, i_brand#42, cc_name#43, (rn#47 - 1)] Join condition: None +<<<<<<< HEAD (55) Project [codegen id : 37] Output [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#48, sum_sales#45 AS nsum#49] Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] @@ -301,6 +357,15 @@ Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales (56) TakeOrderedAndProject Input [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 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#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] +======= +(57) Project [codegen id : 37] +Output [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#49, sum_sales#46 AS nsum#50] +Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] + +(58) TakeOrderedAndProject +Input [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] +>>>>>>> regen ===== Subqueries ===== 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 c347bbbe3b4ad..f1956e47f453e 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 @@ -194,7 +194,11 @@ Arguments: [rank(d_year#11, d_moy#12) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [8]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26] +<<<<<<< HEAD Condition : ((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#24)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +======= +Condition : (((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#24)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#26)) +>>>>>>> regen (33) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] @@ -202,22 +206,23 @@ Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] (34) HashAggregate [codegen id : 13] 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#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#33] -Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#33,17,2) AS sum_sales#34] +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] (35) Exchange -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#35] +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] (36) Sort [codegen id : 14] -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35] 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 (37) Window -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] -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#36], [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#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] +<<<<<<< HEAD (38) Project [codegen id : 15] Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34, rn#36] @@ -225,12 +230,26 @@ Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sale (39) BroadcastExchange Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#37] +======= +(38) Filter [codegen id : 15] +Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] +Condition : isnotnull(rn#37) + +(39) Project [codegen id : 15] +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] + +(40) BroadcastExchange +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] + 1)),false), [id=#38] +>>>>>>> regen (40) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#36 + 1)] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] Join condition: None +<<<<<<< HEAD (41) Project [codegen id : 23] Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34] Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] @@ -253,12 +272,41 @@ Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sale (46) BroadcastExchange Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#45] +======= +(42) Project [codegen id : 23] +Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35] +Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] + +(43) ReusedExchange [Reuses operator id: 35] +Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] + +(44) Sort [codegen id : 21] +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 + +(45) 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] + +(46) Filter [codegen id : 22] +Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] +Condition : isnotnull(rn#45) + +(47) Project [codegen id : 22] +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] + +(48) BroadcastExchange +Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] - 1)),false), [id=#46] +>>>>>>> regen (47) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -Right keys [4]: [i_category#38, i_brand#39, cc_name#40, (rn#44 - 1)] +Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] Join condition: None +<<<<<<< HEAD (48) Project [codegen id : 23] Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#34 AS psum#46, sum_sales#43 AS nsum#47] Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34, i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] @@ -266,6 +314,15 @@ Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales (49) TakeOrderedAndProject Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, 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#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#21, psum#46, nsum#47] +======= +(50) Project [codegen id : 23] +Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] +Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] + +(51) TakeOrderedAndProject +Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, 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#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#21, psum#47, nsum#48] +>>>>>>> regen ===== Subqueries ===== 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 e04148fad545e..ef8088e415cc5 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 @@ -173,7 +173,7 @@ Input [2]: [d_date_sk#25, d_date#26] (18) Filter [codegen id : 4] Input [2]: [d_date_sk#25, d_date#26] -Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 10442)) AND (d_date#26 <= 10456)) AND isnotnull(d_date_sk#25)) +Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 1998-08-04)) AND (d_date#26 <= 1998-08-18)) AND isnotnull(d_date_sk#25)) (19) Project [codegen id : 4] Output [1]: [d_date_sk#25] @@ -278,165 +278,165 @@ Output [6]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, Input [8]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] (41) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#70] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#25] +Right keys [1]: [d_date_sk#70] Join condition: None (43) Project [codegen id : 11] Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#25] +Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#70] (44) HashAggregate [codegen id : 11] Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] -Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] +Results [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] (45) Exchange -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] +Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#79] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] -Results [5]: [catalog channel AS channel#83, concat(catalog_page, cp_catalog_page_id#68) AS id#84, MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#85, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS returns#86, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#87] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] +Results [5]: [catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#68) AS id#85, MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS returns#87, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#88] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] -Condition : isnotnull(ws_web_site_sk#88) +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : isnotnull(ws_web_site_sk#89) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] (53) Exchange -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: hashpartitioning(wr_item_sk#98, wr_order_number#99, 5), ENSURE_REQUIREMENTS, [id=#103] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: hashpartitioning(wr_item_sk#99, wr_order_number#100, 5), ENSURE_REQUIREMENTS, [id=#104] (54) Sort [codegen id : 15] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: [wr_item_sk#98 ASC NULLS FIRST, wr_order_number#99 ASC NULLS FIRST], false, 0 +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: [wr_item_sk#99 ASC NULLS FIRST, wr_order_number#100 ASC NULLS FIRST], false, 0 (55) Scan parquet default.web_sales -Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 16] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (57) Filter [codegen id : 16] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) (58) Project [codegen id : 16] -Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (59) Exchange -Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Arguments: hashpartitioning(cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint), 5), ENSURE_REQUIREMENTS, [id=#106] +Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Arguments: hashpartitioning(cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint), 5), ENSURE_REQUIREMENTS, [id=#109] (60) Sort [codegen id : 17] -Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Arguments: [cast(ws_item_sk#104 as bigint) ASC NULLS FIRST, cast(ws_order_number#105 as bigint) ASC NULLS FIRST], false, 0 +Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Arguments: [cast(ws_item_sk#105 as bigint) ASC NULLS FIRST, cast(ws_order_number#107 as bigint) ASC NULLS FIRST], false, 0 (61) SortMergeJoin [codegen id : 18] -Left keys [2]: [wr_item_sk#98, wr_order_number#99] -Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] +Left keys [2]: [wr_item_sk#99, wr_order_number#100] +Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] Join condition: None (62) Project [codegen id : 18] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#107, wr_returned_date_sk#102 AS date_sk#108, 0.00 AS sales_price#109, 0.00 AS profit#110, wr_return_amt#100 AS return_amt#111, wr_net_loss#101 AS net_loss#112] -Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#110, wr_returned_date_sk#103 AS date_sk#111, 0.00 AS sales_price#112, 0.00 AS profit#113, wr_return_amt#101 AS return_amt#114, wr_net_loss#102 AS net_loss#115] +Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] (63) Union (64) Scan parquet default.web_site -Output [2]: [web_site_sk#113, web_site_id#114] +Output [2]: [web_site_sk#116, web_site_id#117] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 19] -Input [2]: [web_site_sk#113, web_site_id#114] +Input [2]: [web_site_sk#116, web_site_id#117] (66) Filter [codegen id : 19] -Input [2]: [web_site_sk#113, web_site_id#114] -Condition : isnotnull(web_site_sk#113) +Input [2]: [web_site_sk#116, web_site_id#117] +Condition : isnotnull(web_site_sk#116) (67) BroadcastExchange -Input [2]: [web_site_sk#113, web_site_id#114] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] +Input [2]: [web_site_sk#116, web_site_id#117] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] (68) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [wsr_web_site_sk#92] -Right keys [1]: [web_site_sk#113] +Left keys [1]: [wsr_web_site_sk#93] +Right keys [1]: [web_site_sk#116] Join condition: None (69) Project [codegen id : 21] -Output [6]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Input [8]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#113, web_site_id#114] +Output [6]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [8]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] (70) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#119] (71) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [date_sk#93] -Right keys [1]: [cast(d_date_sk#25 as bigint)] +Left keys [1]: [date_sk#94] +Right keys [1]: [cast(d_date_sk#119 as bigint)] Join condition: None (72) Project [codegen id : 21] -Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Input [7]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114, d_date_sk#25] +Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [7]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117, d_date_sk#119] (73) HashAggregate [codegen id : 21] -Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Keys [1]: [web_site_id#114] -Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] -Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] +Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Keys [1]: [web_site_id#117] +Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum#120, sum#121, sum#122, sum#123] +Results [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] (74) Exchange -Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] +Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#128] (75) HashAggregate [codegen id : 22] -Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -Keys [1]: [web_site_id#114] -Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#125, sum(UnscaledValue(return_amt#96))#126, sum(UnscaledValue(profit#95))#127, sum(UnscaledValue(net_loss#97))#128] -Results [5]: [web channel AS channel#129, concat(web_site, web_site_id#114) AS id#130, MakeDecimal(sum(UnscaledValue(sales_price#94))#125,17,2) AS sales#131, MakeDecimal(sum(UnscaledValue(return_amt#96))#126,17,2) AS returns#132, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#133] +Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] +Keys [1]: [web_site_id#117] +Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#129, sum(UnscaledValue(return_amt#97))#130, sum(UnscaledValue(profit#96))#131, sum(UnscaledValue(net_loss#98))#132] +Results [5]: [web channel AS channel#133, concat(web_site, web_site_id#117) AS id#134, MakeDecimal(sum(UnscaledValue(sales_price#95))#129,17,2) AS sales#135, MakeDecimal(sum(UnscaledValue(return_amt#97))#130,17,2) AS returns#136, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#131,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#132,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#137] (76) Union @@ -444,99 +444,99 @@ Results [5]: [web channel AS channel#129, concat(web_site, web_site_id#114) AS i Input [5]: [channel#41, id#42, sales#43, returns#44, profit#45] Keys [2]: [channel#41, id#42] Functions [3]: [partial_sum(sales#43), partial_sum(returns#44), partial_sum(profit#45)] -Aggregate Attributes [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Results [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Aggregate Attributes [6]: [sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Results [8]: [channel#41, id#42, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] (78) Exchange -Input [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#146] +Input [8]: [channel#41, id#42, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] +Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#150] (79) HashAggregate [codegen id : 24] -Input [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Input [8]: [channel#41, id#42, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#45)] -Aggregate Attributes [3]: [sum(sales#43)#147, sum(returns#44)#148, sum(profit#45)#149] -Results [5]: [channel#41, id#42, cast(sum(sales#43)#147 as decimal(37,2)) AS sales#150, cast(sum(returns#44)#148 as decimal(37,2)) AS returns#151, cast(sum(profit#45)#149 as decimal(38,2)) AS profit#152] +Aggregate Attributes [3]: [sum(sales#43)#151, sum(returns#44)#152, sum(profit#45)#153] +Results [5]: [channel#41, id#42, cast(sum(sales#43)#151 as decimal(37,2)) AS sales#154, cast(sum(returns#44)#152 as decimal(37,2)) AS returns#155, cast(sum(profit#45)#153 as decimal(38,2)) AS profit#156] (80) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Output [8]: [channel#41, id#42, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] (81) HashAggregate [codegen id : 48] -Input [8]: [channel#41, id#42, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Input [8]: [channel#41, id#42, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#159)] -Aggregate Attributes [3]: [sum(sales#43)#160, sum(returns#44)#161, sum(profit#159)#162] -Results [4]: [channel#41, sum(sales#43)#160 AS sales#163, sum(returns#44)#161 AS returns#164, sum(profit#159)#162 AS profit#165] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#163)] +Aggregate Attributes [3]: [sum(sales#43)#164, sum(returns#44)#165, sum(profit#163)#166] +Results [4]: [channel#41, sum(sales#43)#164 AS sales#167, sum(returns#44)#165 AS returns#168, sum(profit#163)#166 AS profit#169] (82) HashAggregate [codegen id : 48] -Input [4]: [channel#41, sales#163, returns#164, profit#165] +Input [4]: [channel#41, sales#167, returns#168, profit#169] Keys [1]: [channel#41] -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#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] +Functions [3]: [partial_sum(sales#167), partial_sum(returns#168), partial_sum(profit#169)] +Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Results [7]: [channel#41, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] (83) Exchange -Input [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] -Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#178] +Input [7]: [channel#41, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#182] (84) HashAggregate [codegen id : 49] -Input [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] +Input [7]: [channel#41, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Keys [1]: [channel#41] -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#41, 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#167), sum(returns#168), sum(profit#169)] +Aggregate Attributes [3]: [sum(sales#167)#183, sum(returns#168)#184, sum(profit#169)#185] +Results [5]: [channel#41, null AS id#186, sum(sales#167)#183 AS sum(sales)#187, sum(returns#168)#184 AS sum(returns)#188, sum(profit#169)#185 AS sum(profit)#189] (85) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] +Output [8]: [channel#41, id#42, sum#190, isEmpty#191, sum#192, isEmpty#193, sum#194, isEmpty#195] (86) HashAggregate [codegen id : 73] -Input [8]: [channel#41, id#42, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] +Input [8]: [channel#41, id#42, sum#190, isEmpty#191, sum#192, isEmpty#193, sum#194, isEmpty#195] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#192)] -Aggregate Attributes [3]: [sum(sales#43)#193, sum(returns#44)#194, sum(profit#192)#195] -Results [3]: [sum(sales#43)#193 AS sales#163, sum(returns#44)#194 AS returns#164, sum(profit#192)#195 AS profit#165] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#196)] +Aggregate Attributes [3]: [sum(sales#43)#197, sum(returns#44)#198, sum(profit#196)#199] +Results [3]: [sum(sales#43)#197 AS sales#167, sum(returns#44)#198 AS returns#168, sum(profit#196)#199 AS profit#169] (87) HashAggregate [codegen id : 73] -Input [3]: [sales#163, returns#164, profit#165] +Input [3]: [sales#167, returns#168, profit#169] Keys: [] -Functions [3]: [partial_sum(sales#163), partial_sum(returns#164), partial_sum(profit#165)] -Aggregate Attributes [6]: [sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201] -Results [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] +Functions [3]: [partial_sum(sales#167), partial_sum(returns#168), partial_sum(profit#169)] +Aggregate Attributes [6]: [sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205] +Results [6]: [sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211] (88) Exchange -Input [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#208] +Input [6]: [sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#212] (89) HashAggregate [codegen id : 74] -Input [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] +Input [6]: [sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211] Keys: [] -Functions [3]: [sum(sales#163), sum(returns#164), sum(profit#165)] -Aggregate Attributes [3]: [sum(sales#163)#209, sum(returns#164)#210, sum(profit#165)#211] -Results [5]: [null AS channel#212, null AS id#213, sum(sales#163)#209 AS sum(sales)#214, sum(returns#164)#210 AS sum(returns)#215, sum(profit#165)#211 AS sum(profit)#216] +Functions [3]: [sum(sales#167), sum(returns#168), sum(profit#169)] +Aggregate Attributes [3]: [sum(sales#167)#213, sum(returns#168)#214, sum(profit#169)#215] +Results [5]: [null AS channel#216, null AS id#217, sum(sales#167)#213 AS sum(sales)#218, sum(returns#168)#214 AS sum(returns)#219, sum(profit#169)#215 AS sum(profit)#220] (90) Union (91) HashAggregate [codegen id : 75] -Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -Keys [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Keys [5]: [channel#41, id#42, sales#154, returns#155, profit#156] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Results [5]: [channel#41, id#42, sales#154, returns#155, profit#156] (92) Exchange -Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -Arguments: hashpartitioning(channel#41, id#42, sales#150, returns#151, profit#152, 5), ENSURE_REQUIREMENTS, [id=#217] +Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Arguments: hashpartitioning(channel#41, id#42, sales#154, returns#155, profit#156, 5), ENSURE_REQUIREMENTS, [id=#221] (93) HashAggregate [codegen id : 76] -Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -Keys [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Keys [5]: [channel#41, id#42, sales#154, returns#155, profit#156] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Results [5]: [channel#41, id#42, sales#154, returns#155, profit#156] (94) TakeOrderedAndProject -Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#150, returns#151, profit#152] +Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#154, returns#155, profit#156] ===== Subqueries ===== @@ -554,12 +554,12 @@ ReusedExchange (96) (96) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#70] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 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 36b1ff63b2065..c6b64d59abf82 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 @@ -143,7 +143,7 @@ Input [2]: [d_date_sk#22, d_date#23] (12) Filter [codegen id : 3] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 10442)) AND (d_date#23 <= 10456)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) (13) Project [codegen id : 3] Output [1]: [d_date_sk#22] @@ -248,180 +248,180 @@ Input [4]: [cr_catalog_page_sk#57, cr_return_amount#58, cr_net_loss#59, cr_retur (34) Union (35) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#67] (36) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#67] Join condition: None (37) Project [codegen id : 11] Output [5]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56] -Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#22] +Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#67] (38) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Output [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] (40) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] -Condition : isnotnull(cp_catalog_page_sk#67) +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Condition : isnotnull(cp_catalog_page_sk#68) (41) BroadcastExchange -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#69] +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#70] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [page_sk#51] -Right keys [1]: [cp_catalog_page_sk#67] +Right keys [1]: [cp_catalog_page_sk#68] Join condition: None (43) Project [codegen id : 11] -Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] +Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] +Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#68, cp_catalog_page_id#69] (44) HashAggregate [codegen id : 11] -Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Keys [1]: [cp_catalog_page_id#68] +Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] +Keys [1]: [cp_catalog_page_id#69] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] -Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] +Results [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] (45) Exchange -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] +Arguments: hashpartitioning(cp_catalog_page_id#69, 5), ENSURE_REQUIREMENTS, [id=#79] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Keys [1]: [cp_catalog_page_id#68] +Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] +Keys [1]: [cp_catalog_page_id#69] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] -Results [5]: [catalog channel AS channel#83, concat(catalog_page, cp_catalog_page_id#68) AS id#84, MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#85, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS returns#86, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#87] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] +Results [5]: [catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#69) AS id#85, MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS returns#87, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#88] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] -Condition : isnotnull(ws_web_site_sk#88) +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : isnotnull(ws_web_site_sk#89) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] (53) BroadcastExchange -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#103] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#104] (54) Scan parquet default.web_sales -Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (55) ColumnarToRow -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (56) Filter -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) (57) Project -Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#98, wr_order_number#99] -Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] +Left keys [2]: [wr_item_sk#99, wr_order_number#100] +Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] Join condition: None (59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#106, wr_returned_date_sk#102 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#100 AS return_amt#110, wr_net_loss#101 AS net_loss#111] -Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#109, wr_returned_date_sk#103 AS date_sk#110, 0.00 AS sales_price#111, 0.00 AS profit#112, wr_return_amt#101 AS return_amt#113, wr_net_loss#102 AS net_loss#114] +Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] (60) Union (61) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#115] (62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#93] -Right keys [1]: [cast(d_date_sk#22 as bigint)] +Left keys [1]: [date_sk#94] +Right keys [1]: [cast(d_date_sk#115 as bigint)] Join condition: None (63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97] -Input [7]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, d_date_sk#22] +Output [5]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98] +Input [7]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, d_date_sk#115] (64) Scan parquet default.web_site -Output [2]: [web_site_sk#112, web_site_id#113] +Output [2]: [web_site_sk#116, web_site_id#117] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#112, web_site_id#113] +Input [2]: [web_site_sk#116, web_site_id#117] (66) Filter [codegen id : 17] -Input [2]: [web_site_sk#112, web_site_id#113] -Condition : isnotnull(web_site_sk#112) +Input [2]: [web_site_sk#116, web_site_id#117] +Condition : isnotnull(web_site_sk#116) (67) BroadcastExchange -Input [2]: [web_site_sk#112, web_site_id#113] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#114] +Input [2]: [web_site_sk#116, web_site_id#117] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] (68) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#92] -Right keys [1]: [web_site_sk#112] +Left keys [1]: [wsr_web_site_sk#93] +Right keys [1]: [web_site_sk#116] Join condition: None (69) Project [codegen id : 18] -Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] -Input [7]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#112, web_site_id#113] +Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [7]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] (70) HashAggregate [codegen id : 18] -Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] -Keys [1]: [web_site_id#113] -Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] -Results [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] +Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Keys [1]: [web_site_id#117] +Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum#119, sum#120, sum#121, sum#122] +Results [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] (71) Exchange -Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -Arguments: hashpartitioning(web_site_id#113, 5), ENSURE_REQUIREMENTS, [id=#123] +Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] +Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#127] (72) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -Keys [1]: [web_site_id#113] -Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#124, sum(UnscaledValue(return_amt#96))#125, sum(UnscaledValue(profit#95))#126, sum(UnscaledValue(net_loss#97))#127] -Results [5]: [web channel AS channel#128, concat(web_site, web_site_id#113) AS id#129, MakeDecimal(sum(UnscaledValue(sales_price#94))#124,17,2) AS sales#130, MakeDecimal(sum(UnscaledValue(return_amt#96))#125,17,2) AS returns#131, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#126,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#127,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#132] +Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] +Keys [1]: [web_site_id#117] +Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#128, sum(UnscaledValue(return_amt#97))#129, sum(UnscaledValue(profit#96))#130, sum(UnscaledValue(net_loss#98))#131] +Results [5]: [web channel AS channel#132, concat(web_site, web_site_id#117) AS id#133, MakeDecimal(sum(UnscaledValue(sales_price#95))#128,17,2) AS sales#134, MakeDecimal(sum(UnscaledValue(return_amt#97))#129,17,2) AS returns#135, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#130,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#131,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#136] (73) Union @@ -429,99 +429,99 @@ Results [5]: [web channel AS channel#128, concat(web_site, web_site_id#113) AS i Input [5]: [channel#41, id#42, sales#43, returns#44, profit#45] Keys [2]: [channel#41, id#42] Functions [3]: [partial_sum(sales#43), partial_sum(returns#44), partial_sum(profit#45)] -Aggregate Attributes [6]: [sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -Results [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] +Results [8]: [channel#41, id#42, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] (75) Exchange -Input [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#145] +Input [8]: [channel#41, id#42, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#149] (76) HashAggregate [codegen id : 21] -Input [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Input [8]: [channel#41, id#42, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#45)] -Aggregate Attributes [3]: [sum(sales#43)#146, sum(returns#44)#147, sum(profit#45)#148] -Results [5]: [channel#41, id#42, cast(sum(sales#43)#146 as decimal(37,2)) AS sales#149, cast(sum(returns#44)#147 as decimal(37,2)) AS returns#150, cast(sum(profit#45)#148 as decimal(38,2)) AS profit#151] +Aggregate Attributes [3]: [sum(sales#43)#150, sum(returns#44)#151, sum(profit#45)#152] +Results [5]: [channel#41, id#42, cast(sum(sales#43)#150 as decimal(37,2)) AS sales#153, cast(sum(returns#44)#151 as decimal(37,2)) AS returns#154, cast(sum(profit#45)#152 as decimal(38,2)) AS profit#155] (77) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] +Output [8]: [channel#41, id#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] (78) HashAggregate [codegen id : 42] -Input [8]: [channel#41, id#42, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] +Input [8]: [channel#41, id#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#158)] -Aggregate Attributes [3]: [sum(sales#43)#159, sum(returns#44)#160, sum(profit#158)#161] -Results [4]: [channel#41, sum(sales#43)#159 AS sales#162, sum(returns#44)#160 AS returns#163, sum(profit#158)#161 AS profit#164] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#162)] +Aggregate Attributes [3]: [sum(sales#43)#163, sum(returns#44)#164, sum(profit#162)#165] +Results [4]: [channel#41, sum(sales#43)#163 AS sales#166, sum(returns#44)#164 AS returns#167, sum(profit#162)#165 AS profit#168] (79) HashAggregate [codegen id : 42] -Input [4]: [channel#41, sales#162, returns#163, profit#164] +Input [4]: [channel#41, sales#166, returns#167, profit#168] Keys [1]: [channel#41] -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#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Functions [3]: [partial_sum(sales#166), partial_sum(returns#167), partial_sum(profit#168)] +Aggregate Attributes [6]: [sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174] +Results [7]: [channel#41, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] (80) Exchange -Input [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#177] +Input [7]: [channel#41, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] +Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#181] (81) HashAggregate [codegen id : 43] -Input [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Input [7]: [channel#41, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] Keys [1]: [channel#41] -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#41, 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#166), sum(returns#167), sum(profit#168)] +Aggregate Attributes [3]: [sum(sales#166)#182, sum(returns#167)#183, sum(profit#168)#184] +Results [5]: [channel#41, null AS id#185, sum(sales#166)#182 AS sum(sales)#186, sum(returns#167)#183 AS sum(returns)#187, sum(profit#168)#184 AS sum(profit)#188] (82) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Output [8]: [channel#41, id#42, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194] (83) HashAggregate [codegen id : 64] -Input [8]: [channel#41, id#42, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Input [8]: [channel#41, id#42, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#191)] -Aggregate Attributes [3]: [sum(sales#43)#192, sum(returns#44)#193, sum(profit#191)#194] -Results [3]: [sum(sales#43)#192 AS sales#162, sum(returns#44)#193 AS returns#163, sum(profit#191)#194 AS profit#164] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#195)] +Aggregate Attributes [3]: [sum(sales#43)#196, sum(returns#44)#197, sum(profit#195)#198] +Results [3]: [sum(sales#43)#196 AS sales#166, sum(returns#44)#197 AS returns#167, sum(profit#195)#198 AS profit#168] (84) HashAggregate [codegen id : 64] -Input [3]: [sales#162, returns#163, profit#164] +Input [3]: [sales#166, returns#167, profit#168] Keys: [] -Functions [3]: [partial_sum(sales#162), partial_sum(returns#163), partial_sum(profit#164)] -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#166), partial_sum(returns#167), partial_sum(profit#168)] +Aggregate Attributes [6]: [sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204] +Results [6]: [sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210] (85) Exchange -Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#207] +Input [6]: [sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#211] (86) HashAggregate [codegen id : 65] -Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] +Input [6]: [sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210] Keys: [] -Functions [3]: [sum(sales#162), sum(returns#163), sum(profit#164)] -Aggregate Attributes [3]: [sum(sales#162)#208, sum(returns#163)#209, sum(profit#164)#210] -Results [5]: [null AS channel#211, null AS id#212, sum(sales#162)#208 AS sum(sales)#213, sum(returns#163)#209 AS sum(returns)#214, sum(profit#164)#210 AS sum(profit)#215] +Functions [3]: [sum(sales#166), sum(returns#167), sum(profit#168)] +Aggregate Attributes [3]: [sum(sales#166)#212, sum(returns#167)#213, sum(profit#168)#214] +Results [5]: [null AS channel#215, null AS id#216, sum(sales#166)#212 AS sum(sales)#217, sum(returns#167)#213 AS sum(returns)#218, sum(profit#168)#214 AS sum(profit)#219] (87) Union (88) HashAggregate [codegen id : 66] -Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -Keys [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Keys [5]: [channel#41, id#42, sales#153, returns#154, profit#155] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Results [5]: [channel#41, id#42, sales#153, returns#154, profit#155] (89) Exchange -Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -Arguments: hashpartitioning(channel#41, id#42, sales#149, returns#150, profit#151, 5), ENSURE_REQUIREMENTS, [id=#216] +Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Arguments: hashpartitioning(channel#41, id#42, sales#153, returns#154, profit#155, 5), ENSURE_REQUIREMENTS, [id=#220] (90) HashAggregate [codegen id : 67] -Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -Keys [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Keys [5]: [channel#41, id#42, sales#153, returns#154, profit#155] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Results [5]: [channel#41, id#42, sales#153, returns#154, profit#155] (91) TakeOrderedAndProject -Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#149, returns#150, profit#151] +Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#153, returns#154, profit#155] ===== Subqueries ===== @@ -539,12 +539,12 @@ ReusedExchange (93) (93) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#67] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 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 c096a28386361..3847146fa8fa8 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 @@ -741,7 +741,7 @@ Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt# Output [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple,burlywood,indian,spring,floral,medium]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple ,burlywood ,indian ,spring ,floral ,medium ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct (121) ColumnarToRow [codegen id : 40] @@ -749,7 +749,7 @@ Input [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] (122) Filter [codegen id : 40] Input [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] -Condition : ((((((isnotnull(i_current_price#94) AND i_color#95 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#94 >= 64.00)) AND (i_current_price#94 <= 74.00)) AND (i_current_price#94 >= 65.00)) AND (i_current_price#94 <= 79.00)) AND isnotnull(i_item_sk#93)) +Condition : ((((((isnotnull(i_current_price#94) AND i_color#95 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#94 >= 64.00)) AND (i_current_price#94 <= 74.00)) AND (i_current_price#94 >= 65.00)) AND (i_current_price#94 <= 79.00)) AND isnotnull(i_item_sk#93)) (123) Project [codegen id : 40] Output [2]: [i_item_sk#93, i_product_name#96] @@ -795,353 +795,353 @@ Input [17]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_str Arguments: [item_sk#112 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, store_zip#114 ASC NULLS FIRST], false, 0 (132) Scan parquet default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#129)] +PartitionFilters: [isnotnull(ss_sold_date_sk#140), dynamicpruningexpression(ss_sold_date_sk#140 IN dynamicpruning#141)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (133) ColumnarToRow [codegen id : 44] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] (134) Filter [codegen id : 44] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Condition : (((((((isnotnull(ss_item_sk#129) AND isnotnull(ss_ticket_number#136)) AND isnotnull(ss_store_sk#134)) AND isnotnull(ss_customer_sk#130)) AND isnotnull(ss_cdemo_sk#131)) AND isnotnull(ss_promo_sk#135)) AND isnotnull(ss_hdemo_sk#132)) AND isnotnull(ss_addr_sk#133)) (135) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint), 5), ENSURE_REQUIREMENTS, [id=#130] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Arguments: hashpartitioning(cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint), 5), ENSURE_REQUIREMENTS, [id=#142] (136) Sort [codegen id : 45] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [cast(ss_item_sk#1 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#8 as bigint) ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Arguments: [cast(ss_item_sk#129 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#136 as bigint) ASC NULLS FIRST], false, 0 (137) ReusedExchange [Reuses operator id: 10] -Output [2]: [sr_item_sk#15, sr_ticket_number#16] +Output [2]: [sr_item_sk#143, sr_ticket_number#144] (138) Sort [codegen id : 47] -Input [2]: [sr_item_sk#15, sr_ticket_number#16] -Arguments: [sr_item_sk#15 ASC NULLS FIRST, sr_ticket_number#16 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#143, sr_ticket_number#144] +Arguments: [sr_item_sk#143 ASC NULLS FIRST, sr_ticket_number#144 ASC NULLS FIRST], false, 0 (139) SortMergeJoin [codegen id : 56] -Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] -Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] +Left keys [2]: [cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint)] +Right keys [2]: [sr_item_sk#143, sr_ticket_number#144] Join condition: None (140) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, sr_item_sk#143, sr_ticket_number#144] (141) ReusedExchange [Reuses operator id: 33] -Output [1]: [cs_item_sk#19] +Output [1]: [cs_item_sk#145] (142) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#19] +Left keys [1]: [ss_item_sk#129] +Right keys [1]: [cs_item_sk#145] Join condition: None (143) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, cs_item_sk#145] (144) Scan parquet default.date_dim -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#146, d_year#147] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (145) ColumnarToRow [codegen id : 54] -Input [2]: [d_date_sk#43, d_year#44] +Input [2]: [d_date_sk#146, d_year#147] (146) Filter [codegen id : 54] -Input [2]: [d_date_sk#43, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#43)) +Input [2]: [d_date_sk#146, d_year#147] +Condition : ((isnotnull(d_year#147) AND (d_year#147 = 2000)) AND isnotnull(d_date_sk#146)) (147) BroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#131] +Input [2]: [d_date_sk#146, d_year#147] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#148] (148) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#43] +Left keys [1]: [ss_sold_date_sk#140] +Right keys [1]: [d_date_sk#146] Join condition: None (149) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#43, d_year#44] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147] +Input [13]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, d_date_sk#146, d_year#147] (150) ReusedExchange [Reuses operator id: 45] -Output [3]: [s_store_sk#46, s_store_name#47, s_zip#48] +Output [3]: [s_store_sk#149, s_store_name#150, s_zip#151] (151) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#46] +Left keys [1]: [ss_store_sk#134] +Right keys [1]: [s_store_sk#149] Join condition: None (152) Project [codegen id : 56] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_sk#46, s_store_name#47, s_zip#48] +Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_sk#149, s_store_name#150, s_zip#151] (153) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#132] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Arguments: hashpartitioning(ss_customer_sk#130, 5), ENSURE_REQUIREMENTS, [id=#152] (154) Sort [codegen id : 57] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Arguments: [ss_customer_sk#130 ASC NULLS FIRST], false, 0 (155) ReusedExchange [Reuses operator id: 53] -Output [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Output [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] (156) Sort [codegen id : 59] -Input [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] -Arguments: [c_customer_sk#51 ASC NULLS FIRST], false, 0 +Input [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Arguments: [c_customer_sk#153 ASC NULLS FIRST], false, 0 (157) SortMergeJoin [codegen id : 62] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#51] +Left keys [1]: [ss_customer_sk#130] +Right keys [1]: [c_customer_sk#153] Join condition: None (158) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Input [18]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] (159) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#133, d_year#134] +Output [2]: [d_date_sk#159, d_year#160] (160) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_sales_date_sk#56] -Right keys [1]: [d_date_sk#133] +Left keys [1]: [c_first_sales_date_sk#158] +Right keys [1]: [d_date_sk#159] Join condition: None (161) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56, d_date_sk#133, d_year#134] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158, d_date_sk#159, d_year#160] (162) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#135, d_year#136] +Output [2]: [d_date_sk#161, d_year#162] (163) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_shipto_date_sk#55] -Right keys [1]: [d_date_sk#135] +Left keys [1]: [c_first_shipto_date_sk#157] +Right keys [1]: [d_date_sk#161] Join condition: None (164) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134, d_date_sk#135, d_year#136] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160, d_date_sk#161, d_year#162] (165) Exchange -Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Arguments: hashpartitioning(ss_cdemo_sk#3, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Arguments: hashpartitioning(ss_cdemo_sk#131, 5), ENSURE_REQUIREMENTS, [id=#163] (166) Sort [codegen id : 63] -Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Arguments: [ss_cdemo_sk#3 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Arguments: [ss_cdemo_sk#131 ASC NULLS FIRST], false, 0 (167) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#64, cd_marital_status#65] +Output [2]: [cd_demo_sk#164, cd_marital_status#165] (168) Sort [codegen id : 65] -Input [2]: [cd_demo_sk#64, cd_marital_status#65] -Arguments: [cd_demo_sk#64 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#164, cd_marital_status#165] +Arguments: [cd_demo_sk#164 ASC NULLS FIRST], false, 0 (169) SortMergeJoin [codegen id : 66] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#64] +Left keys [1]: [ss_cdemo_sk#131] +Right keys [1]: [cd_demo_sk#164] Join condition: None (170) Project [codegen id : 66] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_demo_sk#64, cd_marital_status#65] +Output [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_demo_sk#164, cd_marital_status#165] (171) Exchange -Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Arguments: hashpartitioning(c_current_cdemo_sk#52, 5), ENSURE_REQUIREMENTS, [id=#138] +Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Arguments: hashpartitioning(c_current_cdemo_sk#154, 5), ENSURE_REQUIREMENTS, [id=#166] (172) Sort [codegen id : 67] -Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Arguments: [c_current_cdemo_sk#52 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Arguments: [c_current_cdemo_sk#154 ASC NULLS FIRST], false, 0 (173) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#139, cd_marital_status#140] +Output [2]: [cd_demo_sk#167, cd_marital_status#168] (174) Sort [codegen id : 69] -Input [2]: [cd_demo_sk#139, cd_marital_status#140] -Arguments: [cd_demo_sk#139 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#167, cd_marital_status#168] +Arguments: [cd_demo_sk#167 ASC NULLS FIRST], false, 0 (175) SortMergeJoin [codegen id : 73] -Left keys [1]: [c_current_cdemo_sk#52] -Right keys [1]: [cd_demo_sk#139] -Join condition: NOT (cd_marital_status#65 = cd_marital_status#140) +Left keys [1]: [c_current_cdemo_sk#154] +Right keys [1]: [cd_demo_sk#167] +Join condition: NOT (cd_marital_status#165 = cd_marital_status#168) (176) Project [codegen id : 73] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65, cd_demo_sk#139, cd_marital_status#140] +Output [14]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [18]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165, cd_demo_sk#167, cd_marital_status#168] (177) ReusedExchange [Reuses operator id: 84] -Output [1]: [p_promo_sk#70] +Output [1]: [p_promo_sk#169] (178) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#70] +Left keys [1]: [ss_promo_sk#135] +Right keys [1]: [p_promo_sk#169] Join condition: None (179) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, p_promo_sk#70] +Output [13]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, p_promo_sk#169] (180) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#72, hd_income_band_sk#73] +Output [2]: [hd_demo_sk#170, hd_income_band_sk#171] (181) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#72] +Left keys [1]: [ss_hdemo_sk#132] +Right keys [1]: [hd_demo_sk#170] Join condition: None (182) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_demo_sk#72, hd_income_band_sk#73] +Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171] +Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_demo_sk#170, hd_income_band_sk#171] (183) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] +Output [2]: [hd_demo_sk#172, hd_income_band_sk#173] (184) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [c_current_hdemo_sk#53] -Right keys [1]: [hd_demo_sk#141] +Left keys [1]: [c_current_hdemo_sk#155] +Right keys [1]: [hd_demo_sk#172] Join condition: None (185) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_demo_sk#141, hd_income_band_sk#142] +Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Input [15]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_demo_sk#172, hd_income_band_sk#173] (186) Exchange -Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Arguments: hashpartitioning(ss_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#143] +Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Arguments: hashpartitioning(ss_addr_sk#133, 5), ENSURE_REQUIREMENTS, [id=#174] (187) Sort [codegen id : 74] -Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Arguments: [ss_addr_sk#5 ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Arguments: [ss_addr_sk#133 ASC NULLS FIRST], false, 0 (188) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Output [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] (189) Sort [codegen id : 76] -Input [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: [ca_address_sk#78 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: [ca_address_sk#175 ASC NULLS FIRST], false, 0 (190) SortMergeJoin [codegen id : 77] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#78] +Left keys [1]: [ss_addr_sk#133] +Right keys [1]: [ca_address_sk#175] Join condition: None (191) Project [codegen id : 77] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Output [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Input [18]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] (192) Exchange -Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: hashpartitioning(c_current_addr_sk#54, 5), ENSURE_REQUIREMENTS, [id=#144] +Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: hashpartitioning(c_current_addr_sk#156, 5), ENSURE_REQUIREMENTS, [id=#180] (193) Sort [codegen id : 78] -Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: [c_current_addr_sk#54 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: [c_current_addr_sk#156 ASC NULLS FIRST], false, 0 (194) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Output [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] (195) Sort [codegen id : 80] -Input [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Arguments: [ca_address_sk#145 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Arguments: [ca_address_sk#181 ASC NULLS FIRST], false, 0 (196) SortMergeJoin [codegen id : 84] -Left keys [1]: [c_current_addr_sk#54] -Right keys [1]: [ca_address_sk#145] +Left keys [1]: [c_current_addr_sk#156] +Right keys [1]: [ca_address_sk#181] Join condition: None (197) Project [codegen id : 84] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Output [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [21]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] (198) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#90] +Output [1]: [ib_income_band_sk#186] (199) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#73] -Right keys [1]: [ib_income_band_sk#90] +Left keys [1]: [hd_income_band_sk#171] +Right keys [1]: [ib_income_band_sk#186] Join condition: None (200) Project [codegen id : 84] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#90] +Output [18]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [20]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#186] (201) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#150] +Output [1]: [ib_income_band_sk#187] (202) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#142] -Right keys [1]: [ib_income_band_sk#150] +Left keys [1]: [hd_income_band_sk#173] +Right keys [1]: [ib_income_band_sk#187] Join condition: None (203) Project [codegen id : 84] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#150] +Output [17]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#187] (204) ReusedExchange [Reuses operator id: 124] -Output [2]: [i_item_sk#93, i_product_name#96] +Output [2]: [i_item_sk#188, i_product_name#189] (205) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#93] +Left keys [1]: [ss_item_sk#129] +Right keys [1]: [i_item_sk#188] Join condition: None (206) Project [codegen id : 84] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] +Output [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] +Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] (207) HashAggregate [codegen id : 84] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] -Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#151, sum#152, sum#153, sum#154] -Results [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] +Input [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] +Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#137)), partial_sum(UnscaledValue(ss_list_price#138)), partial_sum(UnscaledValue(ss_coupon_amt#139))] +Aggregate Attributes [4]: [count#190, sum#191, sum#192, sum#193] +Results [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] (208) Exchange -Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] -Arguments: hashpartitioning(i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, 5), ENSURE_REQUIREMENTS, [id=#159] +Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] +Arguments: hashpartitioning(i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, 5), ENSURE_REQUIREMENTS, [id=#198] (209) HashAggregate [codegen id : 85] -Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] -Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#160, sum(UnscaledValue(ss_wholesale_cost#9))#161, sum(UnscaledValue(ss_list_price#10))#162, sum(UnscaledValue(ss_coupon_amt#11))#163] -Results [8]: [i_item_sk#93 AS item_sk#164, s_store_name#47 AS store_name#165, s_zip#48 AS store_zip#166, d_year#44 AS syear#167, count(1)#160 AS cnt#168, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#161,17,2) AS s1#169, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#162,17,2) AS s2#170, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#163,17,2) AS s3#171] +Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] +Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#137)), sum(UnscaledValue(ss_list_price#138)), sum(UnscaledValue(ss_coupon_amt#139))] +Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#137))#200, sum(UnscaledValue(ss_list_price#138))#201, sum(UnscaledValue(ss_coupon_amt#139))#202] +Results [8]: [i_item_sk#188 AS item_sk#203, s_store_name#150 AS store_name#204, s_zip#151 AS store_zip#205, d_year#147 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#137))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#138))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#139))#202,17,2) AS s3#210] (210) Exchange -Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] -Arguments: hashpartitioning(item_sk#164, store_name#165, store_zip#166, 5), ENSURE_REQUIREMENTS, [id=#172] +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] (211) Sort [codegen id : 86] -Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] -Arguments: [item_sk#164 ASC NULLS FIRST, store_name#165 ASC NULLS FIRST, store_zip#166 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 (212) SortMergeJoin [codegen id : 87] Left keys [3]: [item_sk#112, store_name#113, store_zip#114] -Right keys [3]: [item_sk#164, store_name#165, store_zip#166] -Join condition: (cnt#168 <= cnt#124) +Right keys [3]: [item_sk#203, store_name#204, store_zip#205] +Join condition: (cnt#207 <= cnt#124) (213) Project [codegen id : 87] -Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] +Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] (214) Exchange -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#169 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#173] +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#208 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] (215) Sort [codegen id : 88] -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#169 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#208 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1152,11 +1152,11 @@ ReusedExchange (216) (216) ReusedExchange [Reuses operator id: 39] Output [2]: [d_date_sk#43, d_year#44] -Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#129 +Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#140 IN dynamicpruning#141 ReusedExchange (217) (217) ReusedExchange [Reuses operator id: 147] -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#146, d_year#147] 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 9424b7f1e2225..0115f4ab8a63a 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 @@ -651,7 +651,7 @@ Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt# Output [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple,burlywood,indian,spring,floral,medium]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple ,burlywood ,indian ,spring ,floral ,medium ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct (105) ColumnarToRow [codegen id : 24] @@ -659,7 +659,7 @@ Input [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] (106) Filter [codegen id : 24] Input [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] -Condition : ((((((isnotnull(i_current_price#88) AND i_color#89 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#88 >= 64.00)) AND (i_current_price#88 <= 74.00)) AND (i_current_price#88 >= 65.00)) AND (i_current_price#88 <= 79.00)) AND isnotnull(i_item_sk#87)) +Condition : ((((((isnotnull(i_current_price#88) AND i_color#89 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#88 >= 64.00)) AND (i_current_price#88 <= 74.00)) AND (i_current_price#88 >= 65.00)) AND (i_current_price#88 <= 79.00)) AND isnotnull(i_item_sk#87)) (107) Project [codegen id : 24] Output [2]: [i_item_sk#87, i_product_name#90] @@ -701,323 +701,323 @@ Input [17]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_str Arguments: [item_sk#105 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, store_zip#107 ASC NULLS FIRST], false, 0 (115) Scan parquet default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#122)] +PartitionFilters: [isnotnull(ss_sold_date_sk#133), dynamicpruningexpression(ss_sold_date_sk#133 IN dynamicpruning#134)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (116) ColumnarToRow [codegen id : 27] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] (117) Filter [codegen id : 27] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Condition : (((((((isnotnull(ss_item_sk#122) AND isnotnull(ss_ticket_number#129)) AND isnotnull(ss_store_sk#127)) AND isnotnull(ss_customer_sk#123)) AND isnotnull(ss_cdemo_sk#124)) AND isnotnull(ss_promo_sk#128)) AND isnotnull(ss_hdemo_sk#125)) AND isnotnull(ss_addr_sk#126)) (118) BroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#123] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#135] (119) Scan parquet default.store_returns -Output [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Output [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct (120) ColumnarToRow -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] (121) Filter -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] -Condition : (isnotnull(sr_item_sk#15) AND isnotnull(sr_ticket_number#16)) +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Condition : (isnotnull(sr_item_sk#136) AND isnotnull(sr_ticket_number#137)) (122) Project -Output [2]: [sr_item_sk#15, sr_ticket_number#16] -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Output [2]: [sr_item_sk#136, sr_ticket_number#137] +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] (123) BroadcastHashJoin [codegen id : 28] -Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] -Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] +Left keys [2]: [cast(ss_item_sk#122 as bigint), cast(ss_ticket_number#129 as bigint)] +Right keys [2]: [sr_item_sk#136, sr_ticket_number#137] Join condition: None (124) Project [codegen id : 28] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, sr_item_sk#136, sr_ticket_number#137] (125) Exchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: hashpartitioning(ss_item_sk#122, 5), ENSURE_REQUIREMENTS, [id=#139] (126) Sort [codegen id : 29] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: [ss_item_sk#122 ASC NULLS FIRST], false, 0 (127) ReusedExchange [Reuses operator id: 28] -Output [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] +Output [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] (128) HashAggregate [codegen id : 35] -Input [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] -Keys [1]: [cs_item_sk#19] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#21)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#21))#128, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129] -Results [3]: [cs_item_sk#19, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#21))#128,17,2) AS sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] +Input [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] +Keys [1]: [cs_item_sk#140] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#144)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#144))#148, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149] +Results [3]: [cs_item_sk#140, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#144))#148,17,2) AS sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] (129) Filter [codegen id : 35] -Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] -Condition : (isnotnull(sum(cs_ext_list_price#21)#130) AND (cast(sum(cs_ext_list_price#21)#130 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131)), DecimalType(21,2), true))) +Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] +Condition : (isnotnull(sum(cs_ext_list_price#144)#150) AND (cast(sum(cs_ext_list_price#144)#150 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151)), DecimalType(21,2), true))) (130) Project [codegen id : 35] -Output [1]: [cs_item_sk#19] -Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] +Output [1]: [cs_item_sk#140] +Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] (131) Sort [codegen id : 35] -Input [1]: [cs_item_sk#19] -Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 +Input [1]: [cs_item_sk#140] +Arguments: [cs_item_sk#140 ASC NULLS FIRST], false, 0 (132) SortMergeJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#19] +Left keys [1]: [ss_item_sk#122] +Right keys [1]: [cs_item_sk#140] Join condition: None (133) Project [codegen id : 51] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, cs_item_sk#140] (134) Scan parquet default.date_dim -Output [2]: [d_date_sk#42, d_year#43] +Output [2]: [d_date_sk#152, d_year#153] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (135) ColumnarToRow [codegen id : 36] -Input [2]: [d_date_sk#42, d_year#43] +Input [2]: [d_date_sk#152, d_year#153] (136) Filter [codegen id : 36] -Input [2]: [d_date_sk#42, d_year#43] -Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2000)) AND isnotnull(d_date_sk#42)) +Input [2]: [d_date_sk#152, d_year#153] +Condition : ((isnotnull(d_year#153) AND (d_year#153 = 2000)) AND isnotnull(d_date_sk#152)) (137) BroadcastExchange -Input [2]: [d_date_sk#42, d_year#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#132] +Input [2]: [d_date_sk#152, d_year#153] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#154] (138) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#42] +Left keys [1]: [ss_sold_date_sk#133] +Right keys [1]: [d_date_sk#152] Join condition: None (139) Project [codegen id : 51] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#42, d_year#43] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153] +Input [13]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, d_date_sk#152, d_year#153] (140) ReusedExchange [Reuses operator id: 44] -Output [3]: [s_store_sk#45, s_store_name#46, s_zip#47] +Output [3]: [s_store_sk#155, s_store_name#156, s_zip#157] (141) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#45] +Left keys [1]: [ss_store_sk#127] +Right keys [1]: [s_store_sk#155] Join condition: None (142) Project [codegen id : 51] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_sk#45, s_store_name#46, s_zip#47] +Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157] +Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_sk#155, s_store_name#156, s_zip#157] (143) ReusedExchange [Reuses operator id: 50] -Output [6]: [c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] +Output [6]: [c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] (144) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#49] +Left keys [1]: [ss_customer_sk#123] +Right keys [1]: [c_customer_sk#158] Join condition: None (145) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] +Input [18]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] (146) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#133, d_year#134] +Output [2]: [d_date_sk#164, d_year#165] (147) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_sales_date_sk#54] -Right keys [1]: [d_date_sk#133] +Left keys [1]: [c_first_sales_date_sk#163] +Right keys [1]: [d_date_sk#164] Join condition: None (148) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54, d_date_sk#133, d_year#134] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163, d_date_sk#164, d_year#165] (149) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#135, d_year#136] +Output [2]: [d_date_sk#166, d_year#167] (150) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_shipto_date_sk#53] -Right keys [1]: [d_date_sk#135] +Left keys [1]: [c_first_shipto_date_sk#162] +Right keys [1]: [d_date_sk#166] Join condition: None (151) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134, d_date_sk#135, d_year#136] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165, d_date_sk#166, d_year#167] (152) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#61, cd_marital_status#62] +Output [2]: [cd_demo_sk#168, cd_marital_status#169] (153) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#61] +Left keys [1]: [ss_cdemo_sk#124] +Right keys [1]: [cd_demo_sk#168] Join condition: None (154) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_demo_sk#61, cd_marital_status#62] +Output [16]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_demo_sk#168, cd_marital_status#169] (155) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#137, cd_marital_status#138] +Output [2]: [cd_demo_sk#170, cd_marital_status#171] (156) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_cdemo_sk#50] -Right keys [1]: [cd_demo_sk#137] -Join condition: NOT (cd_marital_status#62 = cd_marital_status#138) +Left keys [1]: [c_current_cdemo_sk#159] +Right keys [1]: [cd_demo_sk#170] +Join condition: NOT (cd_marital_status#169 = cd_marital_status#171) (157) Project [codegen id : 51] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62, cd_demo_sk#137, cd_marital_status#138] +Output [14]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [18]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169, cd_demo_sk#170, cd_marital_status#171] (158) ReusedExchange [Reuses operator id: 74] -Output [1]: [p_promo_sk#66] +Output [1]: [p_promo_sk#172] (159) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#66] +Left keys [1]: [ss_promo_sk#128] +Right keys [1]: [p_promo_sk#172] Join condition: None (160) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, p_promo_sk#66] +Output [13]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, p_promo_sk#172] (161) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#68, hd_income_band_sk#69] +Output [2]: [hd_demo_sk#173, hd_income_band_sk#174] (162) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#68] +Left keys [1]: [ss_hdemo_sk#125] +Right keys [1]: [hd_demo_sk#173] Join condition: None (163) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_demo_sk#68, hd_income_band_sk#69] +Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174] +Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_demo_sk#173, hd_income_band_sk#174] (164) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] +Output [2]: [hd_demo_sk#175, hd_income_band_sk#176] (165) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_hdemo_sk#51] -Right keys [1]: [hd_demo_sk#139] +Left keys [1]: [c_current_hdemo_sk#160] +Right keys [1]: [hd_demo_sk#175] Join condition: None (166) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_demo_sk#139, hd_income_band_sk#140] +Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176] +Input [15]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_demo_sk#175, hd_income_band_sk#176] (167) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] +Output [5]: [ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] (168) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#73] +Left keys [1]: [ss_addr_sk#126] +Right keys [1]: [ca_address_sk#177] Join condition: None (169) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] +Output [16]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] +Input [18]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] (170) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Output [5]: [ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] (171) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_addr_sk#52] -Right keys [1]: [ca_address_sk#141] +Left keys [1]: [c_current_addr_sk#161] +Right keys [1]: [ca_address_sk#182] Join condition: None (172) Project [codegen id : 51] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Output [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [21]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] (173) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#84] +Output [1]: [ib_income_band_sk#187] (174) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#69] -Right keys [1]: [ib_income_band_sk#84] +Left keys [1]: [hd_income_band_sk#174] +Right keys [1]: [ib_income_band_sk#187] Join condition: None (175) Project [codegen id : 51] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#84] +Output [18]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [20]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#187] (176) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#146] +Output [1]: [ib_income_band_sk#188] (177) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#140] -Right keys [1]: [ib_income_band_sk#146] +Left keys [1]: [hd_income_band_sk#176] +Right keys [1]: [ib_income_band_sk#188] Join condition: None (178) Project [codegen id : 51] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#146] +Output [17]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#188] (179) ReusedExchange [Reuses operator id: 108] -Output [2]: [i_item_sk#87, i_product_name#90] +Output [2]: [i_item_sk#189, i_product_name#190] (180) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#87] +Left keys [1]: [ss_item_sk#122] +Right keys [1]: [i_item_sk#189] Join condition: None (181) Project [codegen id : 51] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] +Output [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] (182) HashAggregate [codegen id : 51] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] -Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#147, sum#148, sum#149, sum#150] -Results [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] +Input [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#130)), partial_sum(UnscaledValue(ss_list_price#131)), partial_sum(UnscaledValue(ss_coupon_amt#132))] +Aggregate Attributes [4]: [count#191, sum#192, sum#193, sum#194] +Results [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] (183) HashAggregate [codegen id : 51] -Input [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#155, sum(UnscaledValue(ss_wholesale_cost#9))#156, sum(UnscaledValue(ss_list_price#10))#157, sum(UnscaledValue(ss_coupon_amt#11))#158] -Results [8]: [i_item_sk#87 AS item_sk#159, s_store_name#46 AS store_name#160, s_zip#47 AS store_zip#161, d_year#43 AS syear#162, count(1)#155 AS cnt#163, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#156,17,2) AS s1#164, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#157,17,2) AS s2#165, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#158,17,2) AS s3#166] +Input [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] +Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#130)), sum(UnscaledValue(ss_list_price#131)), sum(UnscaledValue(ss_coupon_amt#132))] +Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#130))#200, sum(UnscaledValue(ss_list_price#131))#201, sum(UnscaledValue(ss_coupon_amt#132))#202] +Results [8]: [i_item_sk#189 AS item_sk#203, s_store_name#156 AS store_name#204, s_zip#157 AS store_zip#205, d_year#153 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#130))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#131))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#132))#202,17,2) AS s3#210] (184) Exchange -Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] -Arguments: hashpartitioning(item_sk#159, store_name#160, store_zip#161, 5), ENSURE_REQUIREMENTS, [id=#167] +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] (185) Sort [codegen id : 52] -Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] -Arguments: [item_sk#159 ASC NULLS FIRST, store_name#160 ASC NULLS FIRST, store_zip#161 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 (186) SortMergeJoin [codegen id : 53] Left keys [3]: [item_sk#105, store_name#106, store_zip#107] -Right keys [3]: [item_sk#159, store_name#160, store_zip#161] -Join condition: (cnt#163 <= cnt#117) +Right keys [3]: [item_sk#203, store_name#204, store_zip#205] +Join condition: (cnt#207 <= cnt#117) (187) Project [codegen id : 53] -Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] +Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] (188) Exchange -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#164 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#168] +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#208 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] (189) Sort [codegen id : 54] -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#164 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#208 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1028,11 +1028,11 @@ ReusedExchange (190) (190) ReusedExchange [Reuses operator id: 38] Output [2]: [d_date_sk#42, d_year#43] -Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#122 +Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#133 IN dynamicpruning#134 ReusedExchange (191) (191) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#42, d_year#43] +Output [2]: [d_date_sk#152, d_year#153] 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 6d9adf1d38106..13bcda00cb11d 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 @@ -206,234 +206,234 @@ Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 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))#28 as decimal(38,2)) AS sumsales#29] (29) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#30, isEmpty#31] +Output [10]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, s_store_id#37, sum#38, isEmpty#39] (30) HashAggregate [codegen id : 16] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#30, isEmpty#31] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#32] -Results [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#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))#32 AS sumsales#33] +Input [10]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, s_store_id#37, sum#38, isEmpty#39] +Keys [8]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, s_store_id#37] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#40 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#41 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#40 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#41 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#42] +Results [8]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#40 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#41 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#42 AS sumsales#43] (31) HashAggregate [codegen id : 16] -Input [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, sumsales#33] -Keys [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#34, isEmpty#35] -Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, sum#36, isEmpty#37] +Input [8]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sumsales#43] +Keys [7]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#44, isEmpty#45] +Results [9]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sum#46, isEmpty#47] (32) Exchange -Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, sum#36, isEmpty#37] -Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [9]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sum#46, isEmpty#47] +Arguments: hashpartitioning(i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, 5), ENSURE_REQUIREMENTS, [id=#48] (33) HashAggregate [codegen id : 17] -Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, sum#36, isEmpty#37] -Keys [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#39] -Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, null AS s_store_id#40, sum(sumsales#33)#39 AS sumsales#41] +Input [9]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sum#46, isEmpty#47] +Keys [7]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#49] +Results [9]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, null AS s_store_id#50, sum(sumsales#43)#49 AS sumsales#51] (34) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#42, isEmpty#43] +Output [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] (35) HashAggregate [codegen id : 25] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#42, isEmpty#43] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#44] -Results [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, 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))#44 AS sumsales#33] +Input [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] +Keys [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#62 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#63 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#62 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#63 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#64] +Results [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#62 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#63 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#64 AS sumsales#43] (36) HashAggregate [codegen id : 25] -Input [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, sumsales#33] -Keys [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#45, isEmpty#46] -Results [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, sum#47, isEmpty#48] +Input [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sumsales#43] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] (37) Exchange -Input [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, sum#47, isEmpty#48] -Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Arguments: hashpartitioning(i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, 5), ENSURE_REQUIREMENTS, [id=#69] (38) HashAggregate [codegen id : 26] -Input [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, sum#47, isEmpty#48] -Keys [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#50] -Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, null AS d_moy#51, null AS s_store_id#52, sum(sumsales#33)#50 AS sumsales#53] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#70] +Results [9]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, null AS d_moy#71, null AS s_store_id#72, sum(sumsales#43)#70 AS sumsales#73] (39) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#54, isEmpty#55] +Output [10]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81, sum#82, isEmpty#83] (40) HashAggregate [codegen id : 34] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#54, isEmpty#55] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#56] -Results [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#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))#56 AS sumsales#33] +Input [10]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81, sum#82, isEmpty#83] +Keys [8]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#84 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#85 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#84 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#86] +Results [6]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#84 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#86 AS sumsales#43] (41) HashAggregate [codegen id : 34] -Input [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, sumsales#33] -Keys [5]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#57, isEmpty#58] -Results [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, sum#59, isEmpty#60] +Input [6]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sumsales#43] +Keys [5]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#87, isEmpty#88] +Results [7]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum#89, isEmpty#90] (42) Exchange -Input [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, sum#59, isEmpty#60] -Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [7]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum#89, isEmpty#90] +Arguments: hashpartitioning(i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#91] (43) HashAggregate [codegen id : 35] -Input [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, sum#59, isEmpty#60] -Keys [5]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#62] -Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, null AS d_qoy#63, null AS d_moy#64, null AS s_store_id#65, sum(sumsales#33)#62 AS sumsales#66] +Input [7]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum#89, isEmpty#90] +Keys [5]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#92] +Results [9]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, null AS d_qoy#93, null AS d_moy#94, null AS s_store_id#95, sum(sumsales#43)#92 AS sumsales#96] (44) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#67, isEmpty#68] +Output [10]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, d_year#101, d_qoy#102, d_moy#103, s_store_id#104, sum#105, isEmpty#106] (45) HashAggregate [codegen id : 43] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#67, isEmpty#68] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#69] -Results [5]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, 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))#69 AS sumsales#33] +Input [10]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, d_year#101, d_qoy#102, d_moy#103, s_store_id#104, sum#105, isEmpty#106] +Keys [8]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, d_year#101, d_qoy#102, d_moy#103, s_store_id#104] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#107 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#108 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#107 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#108 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#109] +Results [5]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#107 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#108 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#109 AS sumsales#43] (46) HashAggregate [codegen id : 43] -Input [5]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, sumsales#33] -Keys [4]: [i_category#20, i_class#19, i_brand#18, i_product_name#21] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#70, isEmpty#71] -Results [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, sum#72, isEmpty#73] +Input [5]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sumsales#43] +Keys [4]: [i_category#97, i_class#98, i_brand#99, i_product_name#100] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#110, isEmpty#111] +Results [6]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sum#112, isEmpty#113] (47) Exchange -Input [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, sum#72, isEmpty#73] -Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, i_product_name#21, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [6]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sum#112, isEmpty#113] +Arguments: hashpartitioning(i_category#97, i_class#98, i_brand#99, i_product_name#100, 5), ENSURE_REQUIREMENTS, [id=#114] (48) HashAggregate [codegen id : 44] -Input [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, sum#72, isEmpty#73] -Keys [4]: [i_category#20, i_class#19, i_brand#18, i_product_name#21] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#75] -Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, null AS d_year#76, null AS d_qoy#77, null AS d_moy#78, null AS s_store_id#79, sum(sumsales#33)#75 AS sumsales#80] +Input [6]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sum#112, isEmpty#113] +Keys [4]: [i_category#97, i_class#98, i_brand#99, i_product_name#100] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#115] +Results [9]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, null AS d_year#116, null AS d_qoy#117, null AS d_moy#118, null AS s_store_id#119, sum(sumsales#43)#115 AS sumsales#120] (49) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#81, isEmpty#82] +Output [10]: [i_category#121, i_class#122, i_brand#123, i_product_name#124, d_year#125, d_qoy#126, d_moy#127, s_store_id#128, sum#129, isEmpty#130] (50) HashAggregate [codegen id : 52] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#81, isEmpty#82] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#83] -Results [4]: [i_category#20, i_class#19, i_brand#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))#83 AS sumsales#33] +Input [10]: [i_category#121, i_class#122, i_brand#123, i_product_name#124, d_year#125, d_qoy#126, d_moy#127, s_store_id#128, sum#129, isEmpty#130] +Keys [8]: [i_category#121, i_class#122, i_brand#123, i_product_name#124, d_year#125, d_qoy#126, d_moy#127, s_store_id#128] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#131 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#132 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#131 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#132 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#133] +Results [4]: [i_category#121, i_class#122, i_brand#123, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#131 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#132 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#133 AS sumsales#43] (51) HashAggregate [codegen id : 52] -Input [4]: [i_category#20, i_class#19, i_brand#18, sumsales#33] -Keys [3]: [i_category#20, i_class#19, i_brand#18] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#84, isEmpty#85] -Results [5]: [i_category#20, i_class#19, i_brand#18, sum#86, isEmpty#87] +Input [4]: [i_category#121, i_class#122, i_brand#123, sumsales#43] +Keys [3]: [i_category#121, i_class#122, i_brand#123] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#134, isEmpty#135] +Results [5]: [i_category#121, i_class#122, i_brand#123, sum#136, isEmpty#137] (52) Exchange -Input [5]: [i_category#20, i_class#19, i_brand#18, sum#86, isEmpty#87] -Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [5]: [i_category#121, i_class#122, i_brand#123, sum#136, isEmpty#137] +Arguments: hashpartitioning(i_category#121, i_class#122, i_brand#123, 5), ENSURE_REQUIREMENTS, [id=#138] (53) HashAggregate [codegen id : 53] -Input [5]: [i_category#20, i_class#19, i_brand#18, sum#86, isEmpty#87] -Keys [3]: [i_category#20, i_class#19, i_brand#18] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#89] -Results [9]: [i_category#20, i_class#19, i_brand#18, null AS i_product_name#90, null AS d_year#91, null AS d_qoy#92, null AS d_moy#93, null AS s_store_id#94, sum(sumsales#33)#89 AS sumsales#95] +Input [5]: [i_category#121, i_class#122, i_brand#123, sum#136, isEmpty#137] +Keys [3]: [i_category#121, i_class#122, i_brand#123] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#139] +Results [9]: [i_category#121, i_class#122, i_brand#123, null AS i_product_name#140, null AS d_year#141, null AS d_qoy#142, null AS d_moy#143, null AS s_store_id#144, sum(sumsales#43)#139 AS sumsales#145] (54) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#96, isEmpty#97] +Output [10]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sum#154, isEmpty#155] (55) HashAggregate [codegen id : 61] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#96, isEmpty#97] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#98] -Results [3]: [i_category#20, i_class#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))#98 AS sumsales#33] +Input [10]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sum#154, isEmpty#155] +Keys [8]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#156 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#157 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#156 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#157 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#158] +Results [3]: [i_category#146, i_class#147, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#156 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#157 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#158 AS sumsales#43] (56) HashAggregate [codegen id : 61] -Input [3]: [i_category#20, i_class#19, sumsales#33] -Keys [2]: [i_category#20, i_class#19] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#99, isEmpty#100] -Results [4]: [i_category#20, i_class#19, sum#101, isEmpty#102] +Input [3]: [i_category#146, i_class#147, sumsales#43] +Keys [2]: [i_category#146, i_class#147] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#159, isEmpty#160] +Results [4]: [i_category#146, i_class#147, sum#161, isEmpty#162] (57) Exchange -Input [4]: [i_category#20, i_class#19, sum#101, isEmpty#102] -Arguments: hashpartitioning(i_category#20, i_class#19, 5), ENSURE_REQUIREMENTS, [id=#103] +Input [4]: [i_category#146, i_class#147, sum#161, isEmpty#162] +Arguments: hashpartitioning(i_category#146, i_class#147, 5), ENSURE_REQUIREMENTS, [id=#163] (58) HashAggregate [codegen id : 62] -Input [4]: [i_category#20, i_class#19, sum#101, isEmpty#102] -Keys [2]: [i_category#20, i_class#19] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#104] -Results [9]: [i_category#20, i_class#19, null AS i_brand#105, null AS i_product_name#106, null AS d_year#107, null AS d_qoy#108, null AS d_moy#109, null AS s_store_id#110, sum(sumsales#33)#104 AS sumsales#111] +Input [4]: [i_category#146, i_class#147, sum#161, isEmpty#162] +Keys [2]: [i_category#146, i_class#147] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#164] +Results [9]: [i_category#146, i_class#147, null AS i_brand#165, null AS i_product_name#166, null AS d_year#167, null AS d_qoy#168, null AS d_moy#169, null AS s_store_id#170, sum(sumsales#43)#164 AS sumsales#171] (59) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#112, isEmpty#113] +Output [10]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179, sum#180, isEmpty#181] (60) HashAggregate [codegen id : 70] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#112, isEmpty#113] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#114] -Results [2]: [i_category#20, 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))#114 AS sumsales#33] +Input [10]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179, sum#180, isEmpty#181] +Keys [8]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#182 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#183 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#182 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#183 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#184] +Results [2]: [i_category#172, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#182 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#183 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#184 AS sumsales#43] (61) HashAggregate [codegen id : 70] -Input [2]: [i_category#20, sumsales#33] -Keys [1]: [i_category#20] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#115, isEmpty#116] -Results [3]: [i_category#20, sum#117, isEmpty#118] +Input [2]: [i_category#172, sumsales#43] +Keys [1]: [i_category#172] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#185, isEmpty#186] +Results [3]: [i_category#172, sum#187, isEmpty#188] (62) Exchange -Input [3]: [i_category#20, sum#117, isEmpty#118] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#119] +Input [3]: [i_category#172, sum#187, isEmpty#188] +Arguments: hashpartitioning(i_category#172, 5), ENSURE_REQUIREMENTS, [id=#189] (63) HashAggregate [codegen id : 71] -Input [3]: [i_category#20, sum#117, isEmpty#118] -Keys [1]: [i_category#20] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#120] -Results [9]: [i_category#20, null AS i_class#121, null AS i_brand#122, null AS i_product_name#123, null AS d_year#124, null AS d_qoy#125, null AS d_moy#126, null AS s_store_id#127, sum(sumsales#33)#120 AS sumsales#128] +Input [3]: [i_category#172, sum#187, isEmpty#188] +Keys [1]: [i_category#172] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#190] +Results [9]: [i_category#172, null AS i_class#191, null AS i_brand#192, null AS i_product_name#193, null AS d_year#194, null AS d_qoy#195, null AS d_moy#196, null AS s_store_id#197, sum(sumsales#43)#190 AS sumsales#198] (64) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#129, isEmpty#130] +Output [10]: [i_category#199, i_class#200, i_brand#201, i_product_name#202, d_year#203, d_qoy#204, d_moy#205, s_store_id#206, sum#207, isEmpty#208] (65) HashAggregate [codegen id : 79] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#129, isEmpty#130] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#131] -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))#131 AS sumsales#33] +Input [10]: [i_category#199, i_class#200, i_brand#201, i_product_name#202, d_year#203, d_qoy#204, d_moy#205, s_store_id#206, sum#207, isEmpty#208] +Keys [8]: [i_category#199, i_class#200, i_brand#201, i_product_name#202, d_year#203, d_qoy#204, d_moy#205, s_store_id#206] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#209 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#210 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#209 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#210 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#211] +Results [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#209 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#210 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#211 AS sumsales#43] (66) HashAggregate [codegen id : 79] -Input [1]: [sumsales#33] +Input [1]: [sumsales#43] Keys: [] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#132, isEmpty#133] -Results [2]: [sum#134, isEmpty#135] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#212, isEmpty#213] +Results [2]: [sum#214, isEmpty#215] (67) Exchange -Input [2]: [sum#134, isEmpty#135] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#136] +Input [2]: [sum#214, isEmpty#215] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#216] (68) HashAggregate [codegen id : 80] -Input [2]: [sum#134, isEmpty#135] +Input [2]: [sum#214, isEmpty#215] Keys: [] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#137] -Results [9]: [null AS i_category#138, null AS i_class#139, null AS i_brand#140, null AS i_product_name#141, null AS d_year#142, null AS d_qoy#143, null AS d_moy#144, null AS s_store_id#145, sum(sumsales#33)#137 AS sumsales#146] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#217] +Results [9]: [null AS i_category#218, null AS i_class#219, null AS i_brand#220, null AS i_product_name#221, null AS d_year#222, null AS d_qoy#223, null AS d_moy#224, null AS s_store_id#225, sum(sumsales#43)#217 AS sumsales#226] (69) Union (70) Exchange Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#147] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#227] (71) Sort [codegen id : 81] Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29] @@ -441,15 +441,15 @@ Arguments: [i_category#20 ASC NULLS FIRST, sumsales#29 DESC NULLS LAST], false, (72) Window Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29] -Arguments: [rank(sumsales#29) windowspecdefinition(i_category#20, sumsales#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#148], [i_category#20], [sumsales#29 DESC NULLS LAST] +Arguments: [rank(sumsales#29) windowspecdefinition(i_category#20, sumsales#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#228], [i_category#20], [sumsales#29 DESC NULLS LAST] (73) Filter [codegen id : 82] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] -Condition : (isnotnull(rk#148) AND (rk#148 <= 100)) +Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#228] +Condition : (isnotnull(rk#228) AND (rk#228 <= 100)) (74) TakeOrderedAndProject -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] -Arguments: 100, [i_category#20 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#18 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#14 ASC NULLS FIRST, sumsales#29 ASC NULLS FIRST, rk#148 ASC NULLS FIRST], [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] +Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#228] +Arguments: 100, [i_category#20 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#18 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#14 ASC NULLS FIRST, sumsales#29 ASC NULLS FIRST, rk#228 ASC NULLS FIRST], [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#228] ===== Subqueries ===== 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 ae6b3ff4d7542..ffa5e96fad67e 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 @@ -191,234 +191,234 @@ Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 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))#27 as decimal(38,2)) AS sumsales#28] (26) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#29, isEmpty#30] +Output [10]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, s_store_id#36, sum#37, isEmpty#38] (27) HashAggregate [codegen id : 10] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#29, isEmpty#30] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#31] -Results [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#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))#31 AS sumsales#32] +Input [10]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, s_store_id#36, sum#37, isEmpty#38] +Keys [8]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, s_store_id#36] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#39 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#40 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#39 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#40 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#41] +Results [8]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#39 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#40 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#41 AS sumsales#42] (28) HashAggregate [codegen id : 10] -Input [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, sumsales#32] -Keys [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, sum#35, isEmpty#36] +Input [8]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sumsales#42] +Keys [7]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#43, isEmpty#44] +Results [9]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sum#45, isEmpty#46] (29) Exchange -Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, sum#35, isEmpty#36] -Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [9]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sum#45, isEmpty#46] +Arguments: hashpartitioning(i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, 5), ENSURE_REQUIREMENTS, [id=#47] (30) HashAggregate [codegen id : 11] -Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, sum#35, isEmpty#36] -Keys [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#38] -Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, null AS s_store_id#39, sum(sumsales#32)#38 AS sumsales#40] +Input [9]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sum#45, isEmpty#46] +Keys [7]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#48] +Results [9]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, null AS s_store_id#49, sum(sumsales#42)#48 AS sumsales#50] (31) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#41, isEmpty#42] +Output [10]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, d_moy#57, s_store_id#58, sum#59, isEmpty#60] (32) HashAggregate [codegen id : 16] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#41, isEmpty#42] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#43] -Results [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, 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))#43 AS sumsales#32] +Input [10]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, d_moy#57, s_store_id#58, sum#59, isEmpty#60] +Keys [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, d_moy#57, s_store_id#58] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#61 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#62 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#61 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#62 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#63] +Results [7]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#61 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#62 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#63 AS sumsales#42] (33) HashAggregate [codegen id : 16] -Input [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, sumsales#32] -Keys [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#44, isEmpty#45] -Results [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, sum#46, isEmpty#47] +Input [7]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sumsales#42] +Keys [6]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#64, isEmpty#65] +Results [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sum#66, isEmpty#67] (34) Exchange -Input [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, sum#46, isEmpty#47] -Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sum#66, isEmpty#67] +Arguments: hashpartitioning(i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, 5), ENSURE_REQUIREMENTS, [id=#68] (35) HashAggregate [codegen id : 17] -Input [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, sum#46, isEmpty#47] -Keys [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#49] -Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, null AS d_moy#50, null AS s_store_id#51, sum(sumsales#32)#49 AS sumsales#52] +Input [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sum#66, isEmpty#67] +Keys [6]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#69] +Results [9]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, null AS d_moy#70, null AS s_store_id#71, sum(sumsales#42)#69 AS sumsales#72] (36) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#53, isEmpty#54] +Output [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] (37) HashAggregate [codegen id : 22] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#53, isEmpty#54] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#55] -Results [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#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))#55 AS sumsales#32] +Input [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] +Keys [8]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#83 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#84 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#83 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#85] +Results [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#83 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#85 AS sumsales#42] (38) HashAggregate [codegen id : 22] -Input [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, sumsales#32] -Keys [5]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#56, isEmpty#57] -Results [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, sum#58, isEmpty#59] +Input [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sumsales#42] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#86, isEmpty#87] +Results [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] (39) Exchange -Input [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, sum#58, isEmpty#59] -Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Arguments: hashpartitioning(i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, 5), ENSURE_REQUIREMENTS, [id=#90] (40) HashAggregate [codegen id : 23] -Input [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, sum#58, isEmpty#59] -Keys [5]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#61] -Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, null AS d_qoy#62, null AS d_moy#63, null AS s_store_id#64, sum(sumsales#32)#61 AS sumsales#65] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#91] +Results [9]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, null AS d_qoy#92, null AS d_moy#93, null AS s_store_id#94, sum(sumsales#42)#91 AS sumsales#95] (41) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#66, isEmpty#67] +Output [10]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, d_year#100, d_qoy#101, d_moy#102, s_store_id#103, sum#104, isEmpty#105] (42) HashAggregate [codegen id : 28] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#66, isEmpty#67] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#68] -Results [5]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, 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))#68 AS sumsales#32] +Input [10]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, d_year#100, d_qoy#101, d_moy#102, s_store_id#103, sum#104, isEmpty#105] +Keys [8]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, d_year#100, d_qoy#101, d_moy#102, s_store_id#103] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#106 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#107 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#106 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#107 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#108] +Results [5]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#106 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#107 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#108 AS sumsales#42] (43) HashAggregate [codegen id : 28] -Input [5]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, sumsales#32] -Keys [4]: [i_category#19, i_class#18, i_brand#17, i_product_name#20] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#69, isEmpty#70] -Results [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, sum#71, isEmpty#72] +Input [5]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sumsales#42] +Keys [4]: [i_category#96, i_class#97, i_brand#98, i_product_name#99] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#109, isEmpty#110] +Results [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] (44) Exchange -Input [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, sum#71, isEmpty#72] -Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, i_product_name#20, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] +Arguments: hashpartitioning(i_category#96, i_class#97, i_brand#98, i_product_name#99, 5), ENSURE_REQUIREMENTS, [id=#113] (45) HashAggregate [codegen id : 29] -Input [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, sum#71, isEmpty#72] -Keys [4]: [i_category#19, i_class#18, i_brand#17, i_product_name#20] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#74] -Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, null AS d_year#75, null AS d_qoy#76, null AS d_moy#77, null AS s_store_id#78, sum(sumsales#32)#74 AS sumsales#79] +Input [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] +Keys [4]: [i_category#96, i_class#97, i_brand#98, i_product_name#99] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#114] +Results [9]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, null AS d_year#115, null AS d_qoy#116, null AS d_moy#117, null AS s_store_id#118, sum(sumsales#42)#114 AS sumsales#119] (46) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#80, isEmpty#81] +Output [10]: [i_category#120, i_class#121, i_brand#122, i_product_name#123, d_year#124, d_qoy#125, d_moy#126, s_store_id#127, sum#128, isEmpty#129] (47) HashAggregate [codegen id : 34] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#80, isEmpty#81] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#82] -Results [4]: [i_category#19, i_class#18, i_brand#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))#82 AS sumsales#32] +Input [10]: [i_category#120, i_class#121, i_brand#122, i_product_name#123, d_year#124, d_qoy#125, d_moy#126, s_store_id#127, sum#128, isEmpty#129] +Keys [8]: [i_category#120, i_class#121, i_brand#122, i_product_name#123, d_year#124, d_qoy#125, d_moy#126, s_store_id#127] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#130 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#131 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#130 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#131 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#132] +Results [4]: [i_category#120, i_class#121, i_brand#122, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#130 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#131 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#132 AS sumsales#42] (48) HashAggregate [codegen id : 34] -Input [4]: [i_category#19, i_class#18, i_brand#17, sumsales#32] -Keys [3]: [i_category#19, i_class#18, i_brand#17] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#83, isEmpty#84] -Results [5]: [i_category#19, i_class#18, i_brand#17, sum#85, isEmpty#86] +Input [4]: [i_category#120, i_class#121, i_brand#122, sumsales#42] +Keys [3]: [i_category#120, i_class#121, i_brand#122] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#133, isEmpty#134] +Results [5]: [i_category#120, i_class#121, i_brand#122, sum#135, isEmpty#136] (49) Exchange -Input [5]: [i_category#19, i_class#18, i_brand#17, sum#85, isEmpty#86] -Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, 5), ENSURE_REQUIREMENTS, [id=#87] +Input [5]: [i_category#120, i_class#121, i_brand#122, sum#135, isEmpty#136] +Arguments: hashpartitioning(i_category#120, i_class#121, i_brand#122, 5), ENSURE_REQUIREMENTS, [id=#137] (50) HashAggregate [codegen id : 35] -Input [5]: [i_category#19, i_class#18, i_brand#17, sum#85, isEmpty#86] -Keys [3]: [i_category#19, i_class#18, i_brand#17] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#88] -Results [9]: [i_category#19, i_class#18, i_brand#17, null AS i_product_name#89, null AS d_year#90, null AS d_qoy#91, null AS d_moy#92, null AS s_store_id#93, sum(sumsales#32)#88 AS sumsales#94] +Input [5]: [i_category#120, i_class#121, i_brand#122, sum#135, isEmpty#136] +Keys [3]: [i_category#120, i_class#121, i_brand#122] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#138] +Results [9]: [i_category#120, i_class#121, i_brand#122, null AS i_product_name#139, null AS d_year#140, null AS d_qoy#141, null AS d_moy#142, null AS s_store_id#143, sum(sumsales#42)#138 AS sumsales#144] (51) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#95, isEmpty#96] +Output [10]: [i_category#145, i_class#146, i_brand#147, i_product_name#148, d_year#149, d_qoy#150, d_moy#151, s_store_id#152, sum#153, isEmpty#154] (52) HashAggregate [codegen id : 40] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#95, isEmpty#96] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#97] -Results [3]: [i_category#19, i_class#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))#97 AS sumsales#32] +Input [10]: [i_category#145, i_class#146, i_brand#147, i_product_name#148, d_year#149, d_qoy#150, d_moy#151, s_store_id#152, sum#153, isEmpty#154] +Keys [8]: [i_category#145, i_class#146, i_brand#147, i_product_name#148, d_year#149, d_qoy#150, d_moy#151, s_store_id#152] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#155 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#156 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#155 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#156 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#157] +Results [3]: [i_category#145, i_class#146, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#155 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#156 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#157 AS sumsales#42] (53) HashAggregate [codegen id : 40] -Input [3]: [i_category#19, i_class#18, sumsales#32] -Keys [2]: [i_category#19, i_class#18] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#98, isEmpty#99] -Results [4]: [i_category#19, i_class#18, sum#100, isEmpty#101] +Input [3]: [i_category#145, i_class#146, sumsales#42] +Keys [2]: [i_category#145, i_class#146] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#158, isEmpty#159] +Results [4]: [i_category#145, i_class#146, sum#160, isEmpty#161] (54) Exchange -Input [4]: [i_category#19, i_class#18, sum#100, isEmpty#101] -Arguments: hashpartitioning(i_category#19, i_class#18, 5), ENSURE_REQUIREMENTS, [id=#102] +Input [4]: [i_category#145, i_class#146, sum#160, isEmpty#161] +Arguments: hashpartitioning(i_category#145, i_class#146, 5), ENSURE_REQUIREMENTS, [id=#162] (55) HashAggregate [codegen id : 41] -Input [4]: [i_category#19, i_class#18, sum#100, isEmpty#101] -Keys [2]: [i_category#19, i_class#18] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#103] -Results [9]: [i_category#19, i_class#18, null AS i_brand#104, null AS i_product_name#105, null AS d_year#106, null AS d_qoy#107, null AS d_moy#108, null AS s_store_id#109, sum(sumsales#32)#103 AS sumsales#110] +Input [4]: [i_category#145, i_class#146, sum#160, isEmpty#161] +Keys [2]: [i_category#145, i_class#146] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#163] +Results [9]: [i_category#145, i_class#146, null AS i_brand#164, null AS i_product_name#165, null AS d_year#166, null AS d_qoy#167, null AS d_moy#168, null AS s_store_id#169, sum(sumsales#42)#163 AS sumsales#170] (56) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#111, isEmpty#112] +Output [10]: [i_category#171, i_class#172, i_brand#173, i_product_name#174, d_year#175, d_qoy#176, d_moy#177, s_store_id#178, sum#179, isEmpty#180] (57) HashAggregate [codegen id : 46] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#111, isEmpty#112] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#113] -Results [2]: [i_category#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))#113 AS sumsales#32] +Input [10]: [i_category#171, i_class#172, i_brand#173, i_product_name#174, d_year#175, d_qoy#176, d_moy#177, s_store_id#178, sum#179, isEmpty#180] +Keys [8]: [i_category#171, i_class#172, i_brand#173, i_product_name#174, d_year#175, d_qoy#176, d_moy#177, s_store_id#178] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#181 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#182 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#181 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#182 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#183] +Results [2]: [i_category#171, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#181 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#182 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#183 AS sumsales#42] (58) HashAggregate [codegen id : 46] -Input [2]: [i_category#19, sumsales#32] -Keys [1]: [i_category#19] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#114, isEmpty#115] -Results [3]: [i_category#19, sum#116, isEmpty#117] +Input [2]: [i_category#171, sumsales#42] +Keys [1]: [i_category#171] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#184, isEmpty#185] +Results [3]: [i_category#171, sum#186, isEmpty#187] (59) Exchange -Input [3]: [i_category#19, sum#116, isEmpty#117] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, [id=#118] +Input [3]: [i_category#171, sum#186, isEmpty#187] +Arguments: hashpartitioning(i_category#171, 5), ENSURE_REQUIREMENTS, [id=#188] (60) HashAggregate [codegen id : 47] -Input [3]: [i_category#19, sum#116, isEmpty#117] -Keys [1]: [i_category#19] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#119] -Results [9]: [i_category#19, null AS i_class#120, null AS i_brand#121, null AS i_product_name#122, null AS d_year#123, null AS d_qoy#124, null AS d_moy#125, null AS s_store_id#126, sum(sumsales#32)#119 AS sumsales#127] +Input [3]: [i_category#171, sum#186, isEmpty#187] +Keys [1]: [i_category#171] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#189] +Results [9]: [i_category#171, null AS i_class#190, null AS i_brand#191, null AS i_product_name#192, null AS d_year#193, null AS d_qoy#194, null AS d_moy#195, null AS s_store_id#196, sum(sumsales#42)#189 AS sumsales#197] (61) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#128, isEmpty#129] +Output [10]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205, sum#206, isEmpty#207] (62) HashAggregate [codegen id : 52] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#128, isEmpty#129] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#130] -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))#130 AS sumsales#32] +Input [10]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205, sum#206, isEmpty#207] +Keys [8]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#208 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#209 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#208 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#209 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#210] +Results [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#208 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#209 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#210 AS sumsales#42] (63) HashAggregate [codegen id : 52] -Input [1]: [sumsales#32] +Input [1]: [sumsales#42] Keys: [] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#131, isEmpty#132] -Results [2]: [sum#133, isEmpty#134] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#211, isEmpty#212] +Results [2]: [sum#213, isEmpty#214] (64) Exchange -Input [2]: [sum#133, isEmpty#134] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#135] +Input [2]: [sum#213, isEmpty#214] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#215] (65) HashAggregate [codegen id : 53] -Input [2]: [sum#133, isEmpty#134] +Input [2]: [sum#213, isEmpty#214] Keys: [] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#136] -Results [9]: [null AS i_category#137, null AS i_class#138, null AS i_brand#139, null AS i_product_name#140, null AS d_year#141, null AS d_qoy#142, null AS d_moy#143, null AS s_store_id#144, sum(sumsales#32)#136 AS sumsales#145] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#216] +Results [9]: [null AS i_category#217, null AS i_class#218, null AS i_brand#219, null AS i_product_name#220, null AS d_year#221, null AS d_qoy#222, null AS d_moy#223, null AS s_store_id#224, sum(sumsales#42)#216 AS sumsales#225] (66) Union (67) Exchange Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, [id=#146] +Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, [id=#226] (68) Sort [codegen id : 54] Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28] @@ -426,15 +426,15 @@ Arguments: [i_category#19 ASC NULLS FIRST, sumsales#28 DESC NULLS LAST], false, (69) Window Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28] -Arguments: [rank(sumsales#28) windowspecdefinition(i_category#19, sumsales#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#147], [i_category#19], [sumsales#28 DESC NULLS LAST] +Arguments: [rank(sumsales#28) windowspecdefinition(i_category#19, sumsales#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#227], [i_category#19], [sumsales#28 DESC NULLS LAST] (70) Filter [codegen id : 55] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] -Condition : (isnotnull(rk#147) AND (rk#147 <= 100)) +Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#227] +Condition : (isnotnull(rk#227) AND (rk#227 <= 100)) (71) TakeOrderedAndProject -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] -Arguments: 100, [i_category#19 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_product_name#20 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#14 ASC NULLS FIRST, sumsales#28 ASC NULLS FIRST, rk#147 ASC NULLS FIRST], [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] +Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#227] +Arguments: 100, [i_category#19 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_product_name#20 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#14 ASC NULLS FIRST, sumsales#28 ASC NULLS FIRST, rk#227 ASC NULLS FIRST], [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#227] ===== Subqueries ===== 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 3c65529504320..d99363440b2bf 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 @@ -122,109 +122,109 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] (16) Filter [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_store_sk#11) (17) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#13] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] -Output [2]: [ss_store_sk#1, ss_net_profit#2] -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] +Output [2]: [ss_store_sk#11, ss_net_profit#12] +Input [4]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13, d_date_sk#14] (20) Scan parquet default.store -Output [2]: [s_store_sk#8, s_state#10] +Output [2]: [s_store_sk#15, s_state#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#8, s_state#10] +Input [2]: [s_store_sk#15, s_state#16] (22) Filter [codegen id : 3] -Input [2]: [s_store_sk#8, s_state#10] -Condition : isnotnull(s_store_sk#8) +Input [2]: [s_store_sk#15, s_state#16] +Condition : isnotnull(s_store_sk#15) (23) BroadcastExchange -Input [2]: [s_store_sk#8, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +Input [2]: [s_store_sk#15, s_state#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#8] +Left keys [1]: [ss_store_sk#11] +Right keys [1]: [s_store_sk#15] Join condition: None (25) Project [codegen id : 4] -Output [2]: [ss_net_profit#2, s_state#10] -Input [4]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_state#10] +Output [2]: [ss_net_profit#12, s_state#16] +Input [4]: [ss_store_sk#11, ss_net_profit#12, s_store_sk#15, s_state#16] (26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#2, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#12] -Results [2]: [s_state#10, sum#13] +Input [2]: [ss_net_profit#12, s_state#16] +Keys [1]: [s_state#16] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum#18] +Results [2]: [s_state#16, sum#19] (27) Exchange -Input [2]: [s_state#10, sum#13] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [2]: [s_state#16, sum#19] +Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, [id=#20] (28) HashAggregate [codegen id : 5] -Input [2]: [s_state#10, sum#13] -Keys [1]: [s_state#10] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#15] -Results [3]: [s_state#10 AS s_state#16, s_state#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#15,17,2) AS _w2#17] +Input [2]: [s_state#16, sum#19] +Keys [1]: [s_state#16] +Functions [1]: [sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#12))#21] +Results [3]: [s_state#16 AS s_state#22, s_state#16, MakeDecimal(sum(UnscaledValue(ss_net_profit#12))#21,17,2) AS _w2#23] (29) Exchange -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, [id=#24] (30) Sort [codegen id : 6] -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [s_state#10 ASC NULLS FIRST, _w2#17 DESC NULLS LAST], false, 0 +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: [s_state#16 ASC NULLS FIRST, _w2#23 DESC NULLS LAST], false, 0 (31) Window -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#10], [_w2#17 DESC NULLS LAST] +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: [rank(_w2#23) windowspecdefinition(s_state#16, _w2#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#25], [s_state#16], [_w2#23 DESC NULLS LAST] (32) Filter [codegen id : 7] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) +Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] +Condition : (isnotnull(ranking#25) AND (ranking#25 <= 5)) (33) Project [codegen id : 7] -Output [1]: [s_state#16] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] +Output [1]: [s_state#22] +Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] (34) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#20] +Input [1]: [s_state#22] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#26] (35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#10] -Right keys [1]: [s_state#16] +Right keys [1]: [s_state#22] Join condition: None (36) BroadcastExchange Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] @@ -239,115 +239,115 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_county#9, s_state#10 Input [3]: [ss_net_profit#2, s_county#9, s_state#10] Keys [2]: [s_state#10, s_county#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#10, s_county#9, sum#23] +Aggregate Attributes [1]: [sum#28] +Results [3]: [s_state#10, s_county#9, sum#29] (40) Exchange -Input [3]: [s_state#10, s_county#9, sum#23] -Arguments: hashpartitioning(s_state#10, s_county#9, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [s_state#10, s_county#9, sum#29] +Arguments: hashpartitioning(s_state#10, s_county#9, 5), ENSURE_REQUIREMENTS, [id=#30] (41) HashAggregate [codegen id : 10] -Input [3]: [s_state#10, s_county#9, sum#23] +Input [3]: [s_state#10, s_county#9, sum#29] Keys [2]: [s_state#10, s_county#9] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) as decimal(27,2)) AS total_sum#26, s_state#10, s_county#9, 0 AS g_state#27, 0 AS g_county#28, 0 AS lochierarchy#29] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) as decimal(27,2)) AS total_sum#32, s_state#10, s_county#9, 0 AS g_state#33, 0 AS g_county#34, 0 AS lochierarchy#35] (42) ReusedExchange [Reuses operator id: unknown] -Output [3]: [s_state#10, s_county#9, sum#30] +Output [3]: [s_state#36, s_county#37, sum#38] (43) HashAggregate [codegen id : 20] -Input [3]: [s_state#10, s_county#9, sum#30] -Keys [2]: [s_state#10, s_county#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS total_sum#32, s_state#10] +Input [3]: [s_state#36, s_county#37, sum#38] +Keys [2]: [s_state#36, s_county#37] +Functions [1]: [sum(UnscaledValue(ss_net_profit#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#39))#40] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#39))#40,17,2) AS total_sum#41, s_state#36] (44) HashAggregate [codegen id : 20] -Input [2]: [total_sum#32, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(total_sum#32)] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [3]: [s_state#10, sum#35, isEmpty#36] +Input [2]: [total_sum#41, s_state#36] +Keys [1]: [s_state#36] +Functions [1]: [partial_sum(total_sum#41)] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [s_state#36, sum#44, isEmpty#45] (45) Exchange -Input [3]: [s_state#10, sum#35, isEmpty#36] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [s_state#36, sum#44, isEmpty#45] +Arguments: hashpartitioning(s_state#36, 5), ENSURE_REQUIREMENTS, [id=#46] (46) HashAggregate [codegen id : 21] -Input [3]: [s_state#10, sum#35, isEmpty#36] -Keys [1]: [s_state#10] -Functions [1]: [sum(total_sum#32)] -Aggregate Attributes [1]: [sum(total_sum#32)#38] -Results [6]: [sum(total_sum#32)#38 AS total_sum#39, s_state#10, null AS s_county#40, 0 AS g_state#41, 1 AS g_county#42, 1 AS lochierarchy#43] +Input [3]: [s_state#36, sum#44, isEmpty#45] +Keys [1]: [s_state#36] +Functions [1]: [sum(total_sum#41)] +Aggregate Attributes [1]: [sum(total_sum#41)#47] +Results [6]: [sum(total_sum#41)#47 AS total_sum#48, s_state#36, null AS s_county#49, 0 AS g_state#50, 1 AS g_county#51, 1 AS lochierarchy#52] (47) ReusedExchange [Reuses operator id: unknown] -Output [3]: [s_state#10, s_county#9, sum#44] +Output [3]: [s_state#53, s_county#54, sum#55] (48) HashAggregate [codegen id : 31] -Input [3]: [s_state#10, s_county#9, sum#44] -Keys [2]: [s_state#10, s_county#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#45] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#45,17,2) AS total_sum#32] +Input [3]: [s_state#53, s_county#54, sum#55] +Keys [2]: [s_state#53, s_county#54] +Functions [1]: [sum(UnscaledValue(ss_net_profit#56))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#56))#57] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#56))#57,17,2) AS total_sum#41] (49) HashAggregate [codegen id : 31] -Input [1]: [total_sum#32] +Input [1]: [total_sum#41] Keys: [] -Functions [1]: [partial_sum(total_sum#32)] -Aggregate Attributes [2]: [sum#46, isEmpty#47] -Results [2]: [sum#48, isEmpty#49] +Functions [1]: [partial_sum(total_sum#41)] +Aggregate Attributes [2]: [sum#58, isEmpty#59] +Results [2]: [sum#60, isEmpty#61] (50) Exchange -Input [2]: [sum#48, isEmpty#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [sum#60, isEmpty#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] (51) HashAggregate [codegen id : 32] -Input [2]: [sum#48, isEmpty#49] +Input [2]: [sum#60, isEmpty#61] Keys: [] -Functions [1]: [sum(total_sum#32)] -Aggregate Attributes [1]: [sum(total_sum#32)#51] -Results [6]: [sum(total_sum#32)#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] +Functions [1]: [sum(total_sum#41)] +Aggregate Attributes [1]: [sum(total_sum#41)#63] +Results [6]: [sum(total_sum#41)#63 AS total_sum#64, null AS s_state#65, null AS s_county#66, 1 AS g_state#67, 1 AS g_county#68, 2 AS lochierarchy#69] (52) Union (53) HashAggregate [codegen id : 33] -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Keys [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Keys [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Results [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] (54) Exchange -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Arguments: hashpartitioning(total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Arguments: hashpartitioning(total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35, 5), ENSURE_REQUIREMENTS, [id=#70] (55) HashAggregate [codegen id : 34] -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Keys [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Keys [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, CASE WHEN (g_county#28 = 0) THEN s_state#10 END AS _w0#59] +Results [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, CASE WHEN (g_county#34 = 0) THEN s_state#10 END AS _w0#71] (56) Exchange -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: hashpartitioning(lochierarchy#29, _w0#59, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: hashpartitioning(lochierarchy#35, _w0#71, 5), ENSURE_REQUIREMENTS, [id=#72] (57) Sort [codegen id : 35] -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: [lochierarchy#29 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: [lochierarchy#35 ASC NULLS FIRST, _w0#71 ASC NULLS FIRST, total_sum#32 DESC NULLS LAST], false, 0 (58) Window -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#29, _w0#59, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#29, _w0#59], [total_sum#26 DESC NULLS LAST] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: [rank(total_sum#32) windowspecdefinition(lochierarchy#35, _w0#71, total_sum#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#73], [lochierarchy#35, _w0#71], [total_sum#32 DESC NULLS LAST] (59) Project [codegen id : 36] -Output [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] -Input [6]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59, rank_within_parent#61] +Output [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] +Input [6]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71, rank_within_parent#73] (60) TakeOrderedAndProject -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] -Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#10 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] +Arguments: 100, [lochierarchy#35 DESC NULLS LAST, CASE WHEN (lochierarchy#35 = 0) THEN s_state#10 END ASC NULLS FIRST, rank_within_parent#73 ASC NULLS FIRST], [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] ===== Subqueries ===== @@ -358,6 +358,6 @@ ReusedExchange (61) (61) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#5] -Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#13 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 406acb0e0a27f..159494182ba83 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 @@ -122,109 +122,109 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] (16) Filter [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_store_sk#11) (17) Scan parquet default.store -Output [2]: [s_store_sk#8, s_state#10] +Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#8, s_state#10] +Input [2]: [s_store_sk#14, s_state#15] (19) Filter [codegen id : 2] -Input [2]: [s_store_sk#8, s_state#10] -Condition : isnotnull(s_store_sk#8) +Input [2]: [s_store_sk#14, s_state#15] +Condition : isnotnull(s_store_sk#14) (20) BroadcastExchange -Input [2]: [s_store_sk#8, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +Input [2]: [s_store_sk#14, s_state#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#16] (21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#8] +Left keys [1]: [ss_store_sk#11] +Right keys [1]: [s_store_sk#14] Join condition: None (22) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, ss_sold_date_sk#3, s_state#10] -Input [5]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, s_store_sk#8, s_state#10] +Output [3]: [ss_net_profit#12, ss_sold_date_sk#13, s_state#15] +Input [5]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13, s_store_sk#14, s_state#15] (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#17] (24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#13] +Right keys [1]: [d_date_sk#17] Join condition: None (25) Project [codegen id : 4] -Output [2]: [ss_net_profit#2, s_state#10] -Input [4]: [ss_net_profit#2, ss_sold_date_sk#3, s_state#10, d_date_sk#5] +Output [2]: [ss_net_profit#12, s_state#15] +Input [4]: [ss_net_profit#12, ss_sold_date_sk#13, s_state#15, d_date_sk#17] (26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#2, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#12] -Results [2]: [s_state#10, sum#13] +Input [2]: [ss_net_profit#12, s_state#15] +Keys [1]: [s_state#15] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum#18] +Results [2]: [s_state#15, sum#19] (27) Exchange -Input [2]: [s_state#10, sum#13] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [2]: [s_state#15, sum#19] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [id=#20] (28) HashAggregate [codegen id : 5] -Input [2]: [s_state#10, sum#13] -Keys [1]: [s_state#10] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#15] -Results [3]: [s_state#10 AS s_state#16, s_state#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#15,17,2) AS _w2#17] +Input [2]: [s_state#15, sum#19] +Keys [1]: [s_state#15] +Functions [1]: [sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#12))#21] +Results [3]: [s_state#15 AS s_state#22, s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#12))#21,17,2) AS _w2#23] (29) Exchange -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [id=#24] (30) Sort [codegen id : 6] -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [s_state#10 ASC NULLS FIRST, _w2#17 DESC NULLS LAST], false, 0 +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: [s_state#15 ASC NULLS FIRST, _w2#23 DESC NULLS LAST], false, 0 (31) Window -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#10], [_w2#17 DESC NULLS LAST] +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: [rank(_w2#23) windowspecdefinition(s_state#15, _w2#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#25], [s_state#15], [_w2#23 DESC NULLS LAST] (32) Filter [codegen id : 7] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) +Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] +Condition : (isnotnull(ranking#25) AND (ranking#25 <= 5)) (33) Project [codegen id : 7] -Output [1]: [s_state#16] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] +Output [1]: [s_state#22] +Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] (34) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#20] +Input [1]: [s_state#22] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#26] (35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#10] -Right keys [1]: [s_state#16] +Right keys [1]: [s_state#22] Join condition: None (36) BroadcastExchange Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] @@ -239,115 +239,115 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_county#9, s_state#10 Input [3]: [ss_net_profit#2, s_county#9, s_state#10] Keys [2]: [s_state#10, s_county#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#10, s_county#9, sum#23] +Aggregate Attributes [1]: [sum#28] +Results [3]: [s_state#10, s_county#9, sum#29] (40) Exchange -Input [3]: [s_state#10, s_county#9, sum#23] -Arguments: hashpartitioning(s_state#10, s_county#9, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [s_state#10, s_county#9, sum#29] +Arguments: hashpartitioning(s_state#10, s_county#9, 5), ENSURE_REQUIREMENTS, [id=#30] (41) HashAggregate [codegen id : 10] -Input [3]: [s_state#10, s_county#9, sum#23] +Input [3]: [s_state#10, s_county#9, sum#29] Keys [2]: [s_state#10, s_county#9] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) as decimal(27,2)) AS total_sum#26, s_state#10, s_county#9, 0 AS g_state#27, 0 AS g_county#28, 0 AS lochierarchy#29] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) as decimal(27,2)) AS total_sum#32, s_state#10, s_county#9, 0 AS g_state#33, 0 AS g_county#34, 0 AS lochierarchy#35] (42) ReusedExchange [Reuses operator id: unknown] -Output [3]: [s_state#10, s_county#9, sum#30] +Output [3]: [s_state#36, s_county#37, sum#38] (43) HashAggregate [codegen id : 20] -Input [3]: [s_state#10, s_county#9, sum#30] -Keys [2]: [s_state#10, s_county#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS total_sum#32, s_state#10] +Input [3]: [s_state#36, s_county#37, sum#38] +Keys [2]: [s_state#36, s_county#37] +Functions [1]: [sum(UnscaledValue(ss_net_profit#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#39))#40] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#39))#40,17,2) AS total_sum#41, s_state#36] (44) HashAggregate [codegen id : 20] -Input [2]: [total_sum#32, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(total_sum#32)] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [3]: [s_state#10, sum#35, isEmpty#36] +Input [2]: [total_sum#41, s_state#36] +Keys [1]: [s_state#36] +Functions [1]: [partial_sum(total_sum#41)] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [s_state#36, sum#44, isEmpty#45] (45) Exchange -Input [3]: [s_state#10, sum#35, isEmpty#36] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [s_state#36, sum#44, isEmpty#45] +Arguments: hashpartitioning(s_state#36, 5), ENSURE_REQUIREMENTS, [id=#46] (46) HashAggregate [codegen id : 21] -Input [3]: [s_state#10, sum#35, isEmpty#36] -Keys [1]: [s_state#10] -Functions [1]: [sum(total_sum#32)] -Aggregate Attributes [1]: [sum(total_sum#32)#38] -Results [6]: [sum(total_sum#32)#38 AS total_sum#39, s_state#10, null AS s_county#40, 0 AS g_state#41, 1 AS g_county#42, 1 AS lochierarchy#43] +Input [3]: [s_state#36, sum#44, isEmpty#45] +Keys [1]: [s_state#36] +Functions [1]: [sum(total_sum#41)] +Aggregate Attributes [1]: [sum(total_sum#41)#47] +Results [6]: [sum(total_sum#41)#47 AS total_sum#48, s_state#36, null AS s_county#49, 0 AS g_state#50, 1 AS g_county#51, 1 AS lochierarchy#52] (47) ReusedExchange [Reuses operator id: unknown] -Output [3]: [s_state#10, s_county#9, sum#44] +Output [3]: [s_state#53, s_county#54, sum#55] (48) HashAggregate [codegen id : 31] -Input [3]: [s_state#10, s_county#9, sum#44] -Keys [2]: [s_state#10, s_county#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#45] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#45,17,2) AS total_sum#32] +Input [3]: [s_state#53, s_county#54, sum#55] +Keys [2]: [s_state#53, s_county#54] +Functions [1]: [sum(UnscaledValue(ss_net_profit#56))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#56))#57] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#56))#57,17,2) AS total_sum#41] (49) HashAggregate [codegen id : 31] -Input [1]: [total_sum#32] +Input [1]: [total_sum#41] Keys: [] -Functions [1]: [partial_sum(total_sum#32)] -Aggregate Attributes [2]: [sum#46, isEmpty#47] -Results [2]: [sum#48, isEmpty#49] +Functions [1]: [partial_sum(total_sum#41)] +Aggregate Attributes [2]: [sum#58, isEmpty#59] +Results [2]: [sum#60, isEmpty#61] (50) Exchange -Input [2]: [sum#48, isEmpty#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [sum#60, isEmpty#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] (51) HashAggregate [codegen id : 32] -Input [2]: [sum#48, isEmpty#49] +Input [2]: [sum#60, isEmpty#61] Keys: [] -Functions [1]: [sum(total_sum#32)] -Aggregate Attributes [1]: [sum(total_sum#32)#51] -Results [6]: [sum(total_sum#32)#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] +Functions [1]: [sum(total_sum#41)] +Aggregate Attributes [1]: [sum(total_sum#41)#63] +Results [6]: [sum(total_sum#41)#63 AS total_sum#64, null AS s_state#65, null AS s_county#66, 1 AS g_state#67, 1 AS g_county#68, 2 AS lochierarchy#69] (52) Union (53) HashAggregate [codegen id : 33] -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Keys [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Keys [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Results [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] (54) Exchange -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Arguments: hashpartitioning(total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Arguments: hashpartitioning(total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35, 5), ENSURE_REQUIREMENTS, [id=#70] (55) HashAggregate [codegen id : 34] -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Keys [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Keys [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, CASE WHEN (g_county#28 = 0) THEN s_state#10 END AS _w0#59] +Results [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, CASE WHEN (g_county#34 = 0) THEN s_state#10 END AS _w0#71] (56) Exchange -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: hashpartitioning(lochierarchy#29, _w0#59, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: hashpartitioning(lochierarchy#35, _w0#71, 5), ENSURE_REQUIREMENTS, [id=#72] (57) Sort [codegen id : 35] -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: [lochierarchy#29 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: [lochierarchy#35 ASC NULLS FIRST, _w0#71 ASC NULLS FIRST, total_sum#32 DESC NULLS LAST], false, 0 (58) Window -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#29, _w0#59, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#29, _w0#59], [total_sum#26 DESC NULLS LAST] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: [rank(total_sum#32) windowspecdefinition(lochierarchy#35, _w0#71, total_sum#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#73], [lochierarchy#35, _w0#71], [total_sum#32 DESC NULLS LAST] (59) Project [codegen id : 36] -Output [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] -Input [6]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59, rank_within_parent#61] +Output [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] +Input [6]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71, rank_within_parent#73] (60) TakeOrderedAndProject -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] -Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#10 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] +Arguments: 100, [lochierarchy#35 DESC NULLS LAST, CASE WHEN (lochierarchy#35 = 0) THEN s_state#10 END ASC NULLS FIRST, rank_within_parent#73 ASC NULLS FIRST], [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] ===== Subqueries ===== @@ -358,6 +358,6 @@ ReusedExchange (61) (61) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#5] -Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt index 0d90342fc9bbf..fe8ceb415f571 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt @@ -100,7 +100,7 @@ Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotn Output [2]: [hd_demo_sk#9, hd_buy_potential#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -108,7 +108,7 @@ Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (6) Filter [codegen id : 1] Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = 1001-5000)) AND isnotnull(hd_demo_sk#9)) +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = 1001-5000 )) AND isnotnull(hd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [hd_demo_sk#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt index c897c78880bc2..0e67565e856ad 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt @@ -207,7 +207,7 @@ Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_ Output [2]: [hd_demo_sk#24, hd_buy_potential#25] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] @@ -215,7 +215,7 @@ Input [2]: [hd_demo_sk#24, hd_buy_potential#25] (31) Filter [codegen id : 5] Input [2]: [hd_demo_sk#24, hd_buy_potential#25] -Condition : ((isnotnull(hd_buy_potential#25) AND (hd_buy_potential#25 = 1001-5000)) AND isnotnull(hd_demo_sk#24)) +Condition : ((isnotnull(hd_buy_potential#25) AND (hd_buy_potential#25 = 1001-5000 )) AND isnotnull(hd_demo_sk#24)) (32) Project [codegen id : 5] Output [1]: [hd_demo_sk#24] 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 035f89d3a04f7..7eea8040043d6 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 @@ -199,285 +199,285 @@ Input [2]: [customer_id#18, year_total#19] Arguments: [customer_id#18 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] +Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] +Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] (27) Filter [codegen id : 10] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_customer_sk#1) +Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Condition : isnotnull(ss_customer_sk#21) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#25, d_year#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#5, d_year#6] +Input [2]: [d_date_sk#25, d_year#26] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#5, d_year#6] -Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2002)) AND d_year#6 IN (2001,2002)) AND isnotnull(d_date_sk#5)) +Input [2]: [d_date_sk#25, d_year#26] +Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) (31) BroadcastExchange -Input [2]: [d_date_sk#5, d_year#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +Input [2]: [d_date_sk#25, d_year#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#23] +Right keys [1]: [d_date_sk#25] Join condition: None (33) Project [codegen id : 10] -Output [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Input [5]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6] +Output [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Input [5]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] (34) Exchange -Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Arguments: hashpartitioning(ss_customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#28] (35) Sort [codegen id : 11] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Arguments: [ss_customer_sk#21 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] (37) Sort [codegen id : 13] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ss_customer_sk#21] +Right keys [1]: [c_customer_sk#29] Join condition: None (39) Project [codegen id : 14] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] -Input [7]: [ss_customer_sk#1, ss_net_paid#2, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] +Input [7]: [ss_customer_sk#21, ss_net_paid#22, d_year#26, c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] (40) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#2))] -Aggregate Attributes [1]: [sum#24] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] +Aggregate Attributes [1]: [sum#33] +Results [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] (41) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] +Arguments: hashpartitioning(c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, 5), ENSURE_REQUIREMENTS, [id=#35] (42) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ss_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#2))#27] -Results [4]: [c_customer_id#10 AS customer_id#28, c_first_name#11 AS customer_first_name#29, c_last_name#12 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#2))#27,17,2) AS year_total#31] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] +Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#22))#36] +Results [4]: [c_customer_id#30 AS customer_id#37, c_first_name#31 AS customer_first_name#38, c_last_name#32 AS customer_last_name#39, MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#36,17,2) AS year_total#40] (43) Exchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: hashpartitioning(customer_id#28, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] +Arguments: hashpartitioning(customer_id#37, 5), ENSURE_REQUIREMENTS, [id=#41] (44) Sort [codegen id : 16] -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#28 ASC NULLS FIRST], false, 0 +Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] +Arguments: [customer_id#37 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#28] +Right keys [1]: [customer_id#37] Join condition: None (46) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] (48) Filter [codegen id : 19] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_bill_customer_sk#33) +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_bill_customer_sk#42) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#45, d_year#46] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (51) Project [codegen id : 19] -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] +Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Input [5]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] (52) Exchange -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#47] (53) Sort [codegen id : 20] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Arguments: [ws_bill_customer_sk#42 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] (55) Sort [codegen id : 22] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Arguments: [c_customer_sk#48 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#33] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ws_bill_customer_sk#42] +Right keys [1]: [c_customer_sk#48] Join condition: None (57) Project [codegen id : 23] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] +Input [7]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46, c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] (58) HashAggregate [codegen id : 23] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum#37] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] +Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] +Aggregate Attributes [1]: [sum#52] +Results [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] (59) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] +Arguments: hashpartitioning(c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, 5), ENSURE_REQUIREMENTS, [id=#54] (60) HashAggregate [codegen id : 24] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#40] -Results [2]: [c_customer_id#10 AS customer_id#41, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#40,17,2) AS year_total#42] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] +Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] +Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#55] +Results [2]: [c_customer_id#49 AS customer_id#56, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#55,17,2) AS year_total#57] (61) Filter [codegen id : 24] -Input [2]: [customer_id#41, year_total#42] -Condition : (isnotnull(year_total#42) AND (year_total#42 > 0.00)) +Input [2]: [customer_id#56, year_total#57] +Condition : (isnotnull(year_total#57) AND (year_total#57 > 0.00)) (62) Project [codegen id : 24] -Output [2]: [customer_id#41 AS customer_id#43, year_total#42 AS year_total#44] -Input [2]: [customer_id#41, year_total#42] +Output [2]: [customer_id#56 AS customer_id#58, year_total#57 AS year_total#59] +Input [2]: [customer_id#56, year_total#57] (63) Exchange -Input [2]: [customer_id#43, year_total#44] -Arguments: hashpartitioning(customer_id#43, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [2]: [customer_id#58, year_total#59] +Arguments: hashpartitioning(customer_id#58, 5), ENSURE_REQUIREMENTS, [id=#60] (64) Sort [codegen id : 25] -Input [2]: [customer_id#43, year_total#44] -Arguments: [customer_id#43 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#58, year_total#59] +Arguments: [customer_id#58 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#43] +Right keys [1]: [customer_id#58] Join condition: None (66) Project [codegen id : 26] -Output [7]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44] -Input [8]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#43, year_total#44] +Output [7]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59] +Input [8]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, customer_id#58, year_total#59] (67) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] (69) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_bill_customer_sk#33) +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_bill_customer_sk#61) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#64, d_year#65] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#63] +Right keys [1]: [d_date_sk#64] Join condition: None (72) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] +Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Input [5]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65] (73) Exchange -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Arguments: hashpartitioning(ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#66] (74) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Arguments: [ws_bill_customer_sk#61 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] (76) Sort [codegen id : 31] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] +Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#33] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ws_bill_customer_sk#61] +Right keys [1]: [c_customer_sk#67] Join condition: None (78) Project [codegen id : 32] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] +Input [7]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65, c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] (79) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum#47] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] +Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#62))] +Aggregate Attributes [1]: [sum#71] +Results [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] (80) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] +Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#73] (81) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#50] -Results [2]: [c_customer_id#10 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#50,17,2) AS year_total#52] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] +Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] +Functions [1]: [sum(UnscaledValue(ws_net_paid#62))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#62))#74] +Results [2]: [c_customer_id#68 AS customer_id#75, MakeDecimal(sum(UnscaledValue(ws_net_paid#62))#74,17,2) AS year_total#76] (82) Exchange -Input [2]: [customer_id#51, year_total#52] -Arguments: hashpartitioning(customer_id#51, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [2]: [customer_id#75, year_total#76] +Arguments: hashpartitioning(customer_id#75, 5), ENSURE_REQUIREMENTS, [id=#77] (83) Sort [codegen id : 34] -Input [2]: [customer_id#51, year_total#52] -Arguments: [customer_id#51 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#75, year_total#76] +Arguments: [customer_id#75 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#51] -Join condition: (CASE WHEN (year_total#44 > 0.00) THEN CheckOverflow((promote_precision(year_total#52) / promote_precision(year_total#44)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#31) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#75] +Join condition: (CASE WHEN (year_total#59 > 0.00) THEN CheckOverflow((promote_precision(year_total#76) / promote_precision(year_total#59)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#40) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) (85) Project [codegen id : 35] -Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Input [9]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44, customer_id#51, year_total#52] +Output [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] +Input [9]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59, customer_id#75, year_total#76] (86) TakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: 100, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] +Input [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] +Arguments: 100, [customer_first_name#38 ASC NULLS FIRST, customer_id#37 ASC NULLS FIRST, customer_last_name#39 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39] ===== Subqueries ===== @@ -488,15 +488,15 @@ ReusedExchange (87) (87) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#5, d_year#6] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#21 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#25, d_year#26] -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#21 +Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#24 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 44a902cf6da07..3ee6b56189888 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 @@ -171,282 +171,282 @@ Input [2]: [customer_id#17, year_total#18] Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) (20) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] (22) Filter [codegen id : 6] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Condition : (isnotnull(c_customer_sk#19) AND isnotnull(c_customer_id#20)) (23) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] (25) Filter [codegen id : 4] -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_customer_sk#5) +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_customer_sk#23) (26) BroadcastExchange -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#5] +Left keys [1]: [c_customer_sk#19] +Right keys [1]: [ss_customer_sk#23] Join condition: None (28) Project [codegen id : 6] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25] +Input [7]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#28, d_year#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#10, d_year#11] +Input [2]: [d_date_sk#28, d_year#29] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#10, d_year#11] -Condition : (((isnotnull(d_year#11) AND (d_year#11 = 2002)) AND d_year#11 IN (2001,2002)) AND isnotnull(d_date_sk#10)) +Input [2]: [d_date_sk#28, d_year#29] +Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) (32) BroadcastExchange -Input [2]: [d_date_sk#10, d_year#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Input [2]: [d_date_sk#28, d_year#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#25] +Right keys [1]: [d_date_sk#28] Join condition: None (34) Project [codegen id : 6] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] +Input [7]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#28, d_year#29] (35) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum#22] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] +Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] +Aggregate Attributes [1]: [sum#31] +Results [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] (36) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] +Arguments: hashpartitioning(c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, 5), ENSURE_REQUIREMENTS, [id=#33] (37) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#25] -Results [4]: [c_customer_id#2 AS customer_id#26, c_first_name#3 AS customer_first_name#27, c_last_name#4 AS customer_last_name#28, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#25,17,2) AS year_total#29] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] +Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] +Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#34] +Results [4]: [c_customer_id#20 AS customer_id#35, c_first_name#21 AS customer_first_name#36, c_last_name#22 AS customer_last_name#37, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#34,17,2) AS year_total#38] (38) BroadcastExchange -Input [4]: [customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] +Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#39] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#26] +Right keys [1]: [customer_id#35] Join condition: None (40) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] (42) Filter [codegen id : 10] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] +Condition : (isnotnull(c_customer_sk#40) AND isnotnull(c_customer_id#41)) (43) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] (45) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_bill_customer_sk#31) +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Condition : isnotnull(ws_bill_customer_sk#44) (46) BroadcastExchange -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#31] +Left keys [1]: [c_customer_sk#40] +Right keys [1]: [ws_bill_customer_sk#44] Join condition: None (48) Project [codegen id : 10] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46] +Input [7]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#48, d_year#49] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#46] +Right keys [1]: [d_date_sk#48] Join condition: None (51) Project [codegen id : 10] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] +Input [7]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46, d_date_sk#48, d_year#49] (52) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum#35] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] +Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#45))] +Aggregate Attributes [1]: [sum#50] +Results [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] (53) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] +Arguments: hashpartitioning(c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, 5), ENSURE_REQUIREMENTS, [id=#52] (54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#38] -Results [2]: [c_customer_id#2 AS customer_id#39, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#38,17,2) AS year_total#40] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] +Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] +Functions [1]: [sum(UnscaledValue(ws_net_paid#45))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#45))#53] +Results [2]: [c_customer_id#41 AS customer_id#54, MakeDecimal(sum(UnscaledValue(ws_net_paid#45))#53,17,2) AS year_total#55] (55) Filter [codegen id : 11] -Input [2]: [customer_id#39, year_total#40] -Condition : (isnotnull(year_total#40) AND (year_total#40 > 0.00)) +Input [2]: [customer_id#54, year_total#55] +Condition : (isnotnull(year_total#55) AND (year_total#55 > 0.00)) (56) Project [codegen id : 11] -Output [2]: [customer_id#39 AS customer_id#41, year_total#40 AS year_total#42] -Input [2]: [customer_id#39, year_total#40] +Output [2]: [customer_id#54 AS customer_id#56, year_total#55 AS year_total#57] +Input [2]: [customer_id#54, year_total#55] (57) BroadcastExchange -Input [2]: [customer_id#41, year_total#42] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#43] +Input [2]: [customer_id#56, year_total#57] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#58] (58) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#41] +Right keys [1]: [customer_id#56] Join condition: None (59) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42] -Input [8]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, customer_id#41, year_total#42] +Output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57] +Input [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, customer_id#56, year_total#57] (60) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] (62) Filter [codegen id : 14] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] +Condition : (isnotnull(c_customer_sk#59) AND isnotnull(c_customer_id#60)) (63) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] (65) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_bill_customer_sk#31) +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Condition : isnotnull(ws_bill_customer_sk#63) (66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#31] +Left keys [1]: [c_customer_sk#59] +Right keys [1]: [ws_bill_customer_sk#63] Join condition: None (68) Project [codegen id : 14] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65] +Input [7]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#67, d_year#68] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#65] +Right keys [1]: [d_date_sk#67] Join condition: None (71) Project [codegen id : 14] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] +Input [7]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65, d_date_sk#67, d_year#68] (72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum#45] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] +Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#64))] +Aggregate Attributes [1]: [sum#69] +Results [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] (73) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] +Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, 5), ENSURE_REQUIREMENTS, [id=#71] (74) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#48] -Results [2]: [c_customer_id#2 AS customer_id#49, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#48,17,2) AS year_total#50] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] +Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] +Functions [1]: [sum(UnscaledValue(ws_net_paid#64))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#64))#72] +Results [2]: [c_customer_id#60 AS customer_id#73, MakeDecimal(sum(UnscaledValue(ws_net_paid#64))#72,17,2) AS year_total#74] (75) BroadcastExchange -Input [2]: [customer_id#49, year_total#50] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#51] +Input [2]: [customer_id#73, year_total#74] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#75] (76) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#49] -Join condition: (CASE WHEN (year_total#42 > 0.00) THEN CheckOverflow((promote_precision(year_total#50) / promote_precision(year_total#42)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#29) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#73] +Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#57)), 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) (77) Project [codegen id : 16] -Output [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] -Input [9]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42, customer_id#49, year_total#50] +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#57, customer_id#73, year_total#74] (78) TakeOrderedAndProject -Input [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] -Arguments: 100, [customer_first_name#27 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST, customer_last_name#28 ASC NULLS FIRST], [customer_id#26, customer_first_name#27, customer_last_name#28] +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] ===== Subqueries ===== @@ -457,15 +457,15 @@ ReusedExchange (79) (79) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#10, d_year#11] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#19 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#28, d_year#29] -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#19 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#26 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 955dde96eb177..56998d9197835 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 @@ -153,7 +153,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -161,7 +161,7 @@ Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_categor (6) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books)) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#31] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#36, d_year#37] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#36] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] -Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (42) Exchange -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_item_sk#47) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ws_item_sk#47] +Right keys [1]: [i_item_sk#52] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#57, d_year#58] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#57] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] (55) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] -Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (61) Exchange -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] +Right keys [2]: [wr_order_number#61, wr_item_sk#60] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] +Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#55, sum#56] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Aggregate Attributes [2]: [sum#69, sum#70] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] +Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Condition : isnotnull(cs_item_sk#79) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [cs_item_sk#79] +Right keys [1]: [i_item_sk#85] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] +Input [2]: [d_date_sk#90, d_year#91] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] -Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) +Input [2]: [d_date_sk#90, d_year#91] +Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) (83) BroadcastExchange -Input [2]: [d_date_sk#71, d_year#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [d_date_sk#90, d_year#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [cs_sold_date_sk#83] +Right keys [1]: [d_date_sk#90] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] (86) Exchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] -Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#2, cs_item_sk#1] -Right keys [2]: [cr_order_number#19, cr_item_sk#18] +Left keys [2]: [cs_order_number#80, cs_item_sk#79] +Right keys [2]: [cr_order_number#95, cr_item_sk#94] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Condition : isnotnull(ss_item_sk#26) +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Condition : isnotnull(ss_item_sk#98) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#98] +Right keys [1]: [i_item_sk#103] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#108, d_year#109] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ss_sold_date_sk#102] +Right keys [1]: [d_date_sk#108] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] (101) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] +Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] +Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Condition : isnotnull(ws_item_sk#116) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ws_item_sk#116] +Right keys [1]: [i_item_sk#121] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#126, d_year#127] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ws_sold_date_sk#120] +Right keys [1]: [d_date_sk#126] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] (116) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] +Right keys [2]: [wr_order_number#130, wr_item_sk#129] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] +Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Aggregate Attributes [2]: [sum#135, sum#136] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] (127) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] +Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] (129) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] (130) Sort [codegen id : 50] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 (131) 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#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] (133) TakeOrderedAndProject -Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] -Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST, sales_amt_diff#97 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST, sales_amt_diff#150 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 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 955dde96eb177..56998d9197835 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 @@ -153,7 +153,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -161,7 +161,7 @@ Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_categor (6) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books)) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#31] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#36, d_year#37] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#36] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] -Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (42) Exchange -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_item_sk#47) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ws_item_sk#47] +Right keys [1]: [i_item_sk#52] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#57, d_year#58] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#57] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] (55) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] -Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (61) Exchange -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] +Right keys [2]: [wr_order_number#61, wr_item_sk#60] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] +Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#55, sum#56] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Aggregate Attributes [2]: [sum#69, sum#70] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] +Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Condition : isnotnull(cs_item_sk#79) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [cs_item_sk#79] +Right keys [1]: [i_item_sk#85] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] +Input [2]: [d_date_sk#90, d_year#91] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] -Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) +Input [2]: [d_date_sk#90, d_year#91] +Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) (83) BroadcastExchange -Input [2]: [d_date_sk#71, d_year#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [d_date_sk#90, d_year#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [cs_sold_date_sk#83] +Right keys [1]: [d_date_sk#90] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] (86) Exchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] -Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#2, cs_item_sk#1] -Right keys [2]: [cr_order_number#19, cr_item_sk#18] +Left keys [2]: [cs_order_number#80, cs_item_sk#79] +Right keys [2]: [cr_order_number#95, cr_item_sk#94] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Condition : isnotnull(ss_item_sk#26) +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Condition : isnotnull(ss_item_sk#98) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#98] +Right keys [1]: [i_item_sk#103] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#108, d_year#109] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ss_sold_date_sk#102] +Right keys [1]: [d_date_sk#108] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] (101) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] +Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] +Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Condition : isnotnull(ws_item_sk#116) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ws_item_sk#116] +Right keys [1]: [i_item_sk#121] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#126, d_year#127] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ws_sold_date_sk#120] +Right keys [1]: [d_date_sk#126] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] (116) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] +Right keys [2]: [wr_order_number#130, wr_item_sk#129] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] +Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Aggregate Attributes [2]: [sum#135, sum#136] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] (127) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] +Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] (129) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] (130) Sort [codegen id : 50] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 (131) 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#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] (133) TakeOrderedAndProject -Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] -Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST, sales_amt_diff#97 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST, sales_amt_diff#150 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 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 e6ade3880e2d4..422443509b417 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 @@ -222,38 +222,38 @@ Output [4]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_s Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25] (26) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#26] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Right keys [1]: [cast(d_date_sk#26 as bigint)] Join condition: None (28) Project [codegen id : 6] Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#6] +Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#26] (29) HashAggregate [codegen id : 6] Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] Keys [1]: [s_store_sk#25] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#26, sum#27] -Results [3]: [s_store_sk#25, sum#28, sum#29] +Aggregate Attributes [2]: [sum#27, sum#28] +Results [3]: [s_store_sk#25, sum#29, sum#30] (30) Exchange -Input [3]: [s_store_sk#25, sum#28, sum#29] -Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [s_store_sk#25, sum#29, sum#30] +Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#25, sum#28, sum#29] +Input [3]: [s_store_sk#25, sum#29, sum#30] Keys [1]: [s_store_sk#25] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] -Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] +Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] (32) BroadcastExchange -Input [3]: [s_store_sk#25, returns#33, profit_loss#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [3]: [s_store_sk#25, returns#34, profit_loss#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] @@ -261,341 +261,341 @@ Right keys [1]: [s_store_sk#25] Join condition: None (34) Project [codegen id : 8] -Output [5]: [store channel AS channel#36, s_store_sk#9 AS id#37, sales#18, coalesce(returns#33, 0.00) AS returns#38, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#39] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] +Output [5]: [store channel AS channel#37, s_store_sk#9 AS id#38, sales#18, coalesce(returns#34, 0.00) AS returns#39, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#40] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#34, profit_loss#35] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#45] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#43] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cs_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] +Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum#44, sum#45] -Results [3]: [cs_call_center_sk#40, sum#46, sum#47] +Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum#46, sum#47] +Results [3]: [cs_call_center_sk#41, sum#48, sum#49] (41) Exchange -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] (42) HashAggregate [codegen id : 14] -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] -Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] +Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] (43) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +Output [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#55), dynamicpruningexpression(cr_returned_date_sk#55 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#5)] ReadSchema: struct (44) ColumnarToRow [codegen id : 12] -Input [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +Input [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] (45) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#58] (46) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cr_returned_date_sk#55] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#57] +Right keys [1]: [d_date_sk#58] Join condition: None (47) Project [codegen id : 12] -Output [2]: [cr_return_amount#53, cr_net_loss#54] -Input [4]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55, d_date_sk#6] +Output [2]: [cr_return_amount#55, cr_net_loss#56] +Input [4]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57, d_date_sk#58] (48) HashAggregate [codegen id : 12] -Input [2]: [cr_return_amount#53, cr_net_loss#54] +Input [2]: [cr_return_amount#55, cr_net_loss#56] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#53)), partial_sum(UnscaledValue(cr_net_loss#54))] -Aggregate Attributes [2]: [sum#56, sum#57] -Results [2]: [sum#58, sum#59] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#55)), partial_sum(UnscaledValue(cr_net_loss#56))] +Aggregate Attributes [2]: [sum#59, sum#60] +Results [2]: [sum#61, sum#62] (49) Exchange -Input [2]: [sum#58, sum#59] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#60] +Input [2]: [sum#61, sum#62] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] (50) HashAggregate [codegen id : 13] -Input [2]: [sum#58, sum#59] +Input [2]: [sum#61, sum#62] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#53)), sum(UnscaledValue(cr_net_loss#54))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#53))#61, sum(UnscaledValue(cr_net_loss#54))#62] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#53))#61,17,2) AS returns#63, MakeDecimal(sum(UnscaledValue(cr_net_loss#54))#62,17,2) AS profit_loss#64] +Functions [2]: [sum(UnscaledValue(cr_return_amount#55)), sum(UnscaledValue(cr_net_loss#56))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#55))#64, sum(UnscaledValue(cr_net_loss#56))#65] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#55))#64,17,2) AS returns#66, MakeDecimal(sum(UnscaledValue(cr_net_loss#56))#65,17,2) AS profit_loss#67] (51) BroadcastExchange -Input [2]: [returns#63, profit_loss#64] -Arguments: IdentityBroadcastMode, [id=#65] +Input [2]: [returns#66, profit_loss#67] +Arguments: IdentityBroadcastMode, [id=#68] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#66, cs_call_center_sk#40 AS id#67, sales#51, returns#63, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#64 as decimal(18,2)))), DecimalType(18,2), true) AS profit#68] -Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#63, profit_loss#64] +Output [5]: [catalog channel AS channel#69, cs_call_center_sk#41 AS id#70, sales#53, returns#66, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#67 as decimal(18,2)))), DecimalType(18,2), true) AS profit#71] +Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#66, profit_loss#67] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] -Condition : isnotnull(ws_web_page_sk#69) +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Condition : isnotnull(ws_web_page_sk#72) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#76] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#72] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#75] +Right keys [1]: [d_date_sk#76] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] -Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] +Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] +Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#73] +Output [1]: [wp_web_page_sk#77] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#73] +Input [1]: [wp_web_page_sk#77] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#73] -Condition : isnotnull(wp_web_page_sk#73) +Input [1]: [wp_web_page_sk#77] +Condition : isnotnull(wp_web_page_sk#77) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [1]: [wp_web_page_sk#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#69] -Right keys [1]: [wp_web_page_sk#73] +Left keys [1]: [ws_web_page_sk#72] +Right keys [1]: [wp_web_page_sk#77] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum#75, sum#76] -Results [3]: [wp_web_page_sk#73, sum#77, sum#78] +Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum#79, sum#80] +Results [3]: [wp_web_page_sk#77, sum#81, sum#82] (67) Exchange -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] -Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] +Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : isnotnull(wr_web_page_sk#84) +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Condition : isnotnull(wr_web_page_sk#88) (72) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#88] +Output [1]: [wp_web_page_sk#92] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#84] -Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] +Left keys [1]: [wr_web_page_sk#88] +Right keys [1]: [cast(wp_web_page_sk#92 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [4]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] -Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] +Output [4]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] +Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] (75) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#93] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#87] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#91] +Right keys [1]: [cast(d_date_sk#93 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Input [5]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88, d_date_sk#6] +Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] +Input [5]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92, d_date_sk#93] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum#89, sum#90] -Results [3]: [wp_web_page_sk#88, sum#91, sum#92] +Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] +Keys [1]: [wp_web_page_sk#92] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum#94, sum#95] +Results [3]: [wp_web_page_sk#92, sum#96, sum#97] (79) Exchange -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [3]: [wp_web_page_sk#92, sum#96, sum#97] +Arguments: hashpartitioning(wp_web_page_sk#92, 5), ENSURE_REQUIREMENTS, [id=#98] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] -Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] +Input [3]: [wp_web_page_sk#92, sum#96, sum#97] +Keys [1]: [wp_web_page_sk#92] +Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] +Results [3]: [wp_web_page_sk#92, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] +Input [3]: [wp_web_page_sk#92, returns#101, profit_loss#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#73] -Right keys [1]: [wp_web_page_sk#88] +Left keys [1]: [wp_web_page_sk#77] +Right keys [1]: [wp_web_page_sk#92] Join condition: None (83) Project [codegen id : 22] -Output [5]: [web channel AS channel#99, wp_web_page_sk#73 AS id#100, sales#82, coalesce(returns#96, 0.00) AS returns#101, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#102] -Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] +Output [5]: [web channel AS channel#104, wp_web_page_sk#77 AS id#105, sales#86, coalesce(returns#101, 0.00) AS returns#106, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#107] +Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#92, returns#101, profit_loss#102] (84) Union (85) HashAggregate [codegen id : 23] -Input [5]: [channel#36, id#37, sales#18, returns#38, profit#39] -Keys [2]: [channel#36, id#37] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#38), partial_sum(profit#39)] -Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Results [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Input [5]: [channel#37, id#38, sales#18, returns#39, profit#40] +Keys [2]: [channel#37, id#38] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#39), partial_sum(profit#40)] +Aggregate Attributes [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +Results [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] (86) Exchange -Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#36, id#37, 5), ENSURE_REQUIREMENTS, [id=#115] +Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [id=#120] (87) HashAggregate [codegen id : 24] -Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#18)#116, sum(returns#38)#117, sum(profit#39)#118] -Results [5]: [channel#36, id#37, cast(sum(sales#18)#116 as decimal(37,2)) AS sales#119, cast(sum(returns#38)#117 as decimal(37,2)) AS returns#120, cast(sum(profit#39)#118 as decimal(38,2)) AS profit#121] +Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#40)] +Aggregate Attributes [3]: [sum(sales#18)#121, sum(returns#39)#122, sum(profit#40)#123] +Results [5]: [channel#37, id#38, cast(sum(sales#18)#121 as decimal(37,2)) AS sales#124, cast(sum(returns#39)#122 as decimal(37,2)) AS returns#125, cast(sum(profit#40)#123 as decimal(38,2)) AS profit#126] (88) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] +Output [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] (89) HashAggregate [codegen id : 48] -Input [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#128)] -Aggregate Attributes [3]: [sum(sales#18)#129, sum(returns#38)#130, sum(profit#128)#131] -Results [4]: [channel#36, sum(sales#18)#129 AS sales#132, sum(returns#38)#130 AS returns#133, sum(profit#128)#131 AS profit#134] +Input [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#133)] +Aggregate Attributes [3]: [sum(sales#18)#134, sum(returns#39)#135, sum(profit#133)#136] +Results [4]: [channel#37, sum(sales#18)#134 AS sales#137, sum(returns#39)#135 AS returns#138, sum(profit#133)#136 AS profit#139] (90) HashAggregate [codegen id : 48] -Input [4]: [channel#36, sales#132, returns#133, profit#134] -Keys [1]: [channel#36] -Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] -Aggregate Attributes [6]: [sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Results [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Input [4]: [channel#37, sales#137, returns#138, profit#139] +Keys [1]: [channel#37] +Functions [3]: [partial_sum(sales#137), partial_sum(returns#138), partial_sum(profit#139)] +Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Results [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] (91) Exchange -Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Arguments: hashpartitioning(channel#36, 5), ENSURE_REQUIREMENTS, [id=#147] +Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [id=#152] (92) HashAggregate [codegen id : 49] -Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Keys [1]: [channel#36] -Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] -Aggregate Attributes [3]: [sum(sales#132)#148, sum(returns#133)#149, sum(profit#134)#150] -Results [5]: [channel#36, null AS id#151, sum(sales#132)#148 AS sales#152, sum(returns#133)#149 AS returns#153, sum(profit#134)#150 AS profit#154] +Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Keys [1]: [channel#37] +Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] +Aggregate Attributes [3]: [sum(sales#137)#153, sum(returns#138)#154, sum(profit#139)#155] +Results [5]: [channel#37, null AS id#156, sum(sales#137)#153 AS sales#157, sum(returns#138)#154 AS returns#158, sum(profit#139)#155 AS profit#159] (93) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] +Output [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] (94) HashAggregate [codegen id : 73] -Input [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#161)] -Aggregate Attributes [3]: [sum(sales#18)#162, sum(returns#38)#163, sum(profit#161)#164] -Results [3]: [sum(sales#18)#162 AS sales#132, sum(returns#38)#163 AS returns#133, sum(profit#161)#164 AS profit#134] +Input [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#166)] +Aggregate Attributes [3]: [sum(sales#18)#167, sum(returns#39)#168, sum(profit#166)#169] +Results [3]: [sum(sales#18)#167 AS sales#137, sum(returns#39)#168 AS returns#138, sum(profit#166)#169 AS profit#139] (95) HashAggregate [codegen id : 73] -Input [3]: [sales#132, returns#133, profit#134] +Input [3]: [sales#137, returns#138, profit#139] Keys: [] -Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] -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#137), partial_sum(returns#138), partial_sum(profit#139)] +Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Results [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] (96) Exchange -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#177] +Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#182] (97) HashAggregate [codegen id : 74] -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Keys: [] -Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] -Aggregate Attributes [3]: [sum(sales#132)#178, sum(returns#133)#179, sum(profit#134)#180] -Results [5]: [null AS channel#181, null AS id#182, sum(sales#132)#178 AS sales#183, sum(returns#133)#179 AS returns#184, sum(profit#134)#180 AS profit#185] +Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] +Aggregate Attributes [3]: [sum(sales#137)#183, sum(returns#138)#184, sum(profit#139)#185] +Results [5]: [null AS channel#186, null AS id#187, sum(sales#137)#183 AS sales#188, sum(returns#138)#184 AS returns#189, sum(profit#139)#185 AS profit#190] (98) Union (99) HashAggregate [codegen id : 75] -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] (100) Exchange -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Arguments: hashpartitioning(channel#36, id#37, sales#119, returns#120, profit#121, 5), ENSURE_REQUIREMENTS, [id=#186] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Arguments: hashpartitioning(channel#37, id#38, sales#124, returns#125, profit#126, 5), ENSURE_REQUIREMENTS, [id=#191] (101) HashAggregate [codegen id : 76] -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] (102) TakeOrderedAndProject -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Arguments: 100, [channel#36 ASC NULLS FIRST, id#37 ASC NULLS FIRST], [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#124, returns#125, profit#126] ===== Subqueries ===== @@ -611,14 +611,14 @@ ReusedExchange (104) (104) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#26] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#55 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 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 0ec2ed7cca356..705277b65ff0a 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 @@ -210,392 +210,392 @@ Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_s Condition : isnotnull(sr_store_sk#20) (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#25] (24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None (25) Project [codegen id : 6] Output [3]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22] -Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#6] +Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#25] (26) ReusedExchange [Reuses operator id: 14] -Output [1]: [s_store_sk#25] +Output [1]: [s_store_sk#26] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_store_sk#20] -Right keys [1]: [cast(s_store_sk#25 as bigint)] +Right keys [1]: [cast(s_store_sk#26 as bigint)] Join condition: None (28) Project [codegen id : 6] -Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#25] +Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] +Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#26] (29) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Keys [1]: [s_store_sk#25] +Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] +Keys [1]: [s_store_sk#26] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#26, sum#27] -Results [3]: [s_store_sk#25, sum#28, sum#29] +Aggregate Attributes [2]: [sum#27, sum#28] +Results [3]: [s_store_sk#26, sum#29, sum#30] (30) Exchange -Input [3]: [s_store_sk#25, sum#28, sum#29] -Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [s_store_sk#26, sum#29, sum#30] +Arguments: hashpartitioning(s_store_sk#26, 5), ENSURE_REQUIREMENTS, [id=#31] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#25, sum#28, sum#29] -Keys [1]: [s_store_sk#25] +Input [3]: [s_store_sk#26, sum#29, sum#30] +Keys [1]: [s_store_sk#26] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] -Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] +Results [3]: [s_store_sk#26, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] (32) BroadcastExchange -Input [3]: [s_store_sk#25, returns#33, profit_loss#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [3]: [s_store_sk#26, returns#34, profit_loss#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] -Right keys [1]: [s_store_sk#25] +Right keys [1]: [s_store_sk#26] Join condition: None (34) Project [codegen id : 8] -Output [5]: [store channel AS channel#36, s_store_sk#9 AS id#37, sales#18, coalesce(returns#33, 0.00) AS returns#38, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#39] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] +Output [5]: [store channel AS channel#37, s_store_sk#9 AS id#38, sales#18, coalesce(returns#34, 0.00) AS returns#39, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#40] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#26, returns#34, profit_loss#35] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#45] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#43] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cs_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] +Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum#44, sum#45] -Results [3]: [cs_call_center_sk#40, sum#46, sum#47] +Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum#46, sum#47] +Results [3]: [cs_call_center_sk#41, sum#48, sum#49] (41) Exchange -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] -Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] +Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] (43) BroadcastExchange -Input [3]: [cs_call_center_sk#40, sales#51, profit#52] -Arguments: IdentityBroadcastMode, [id=#53] +Input [3]: [cs_call_center_sk#41, sales#53, profit#54] +Arguments: IdentityBroadcastMode, [id=#55] (44) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Output [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#58), dynamicpruningexpression(cr_returned_date_sk#58 IN dynamicpruning#5)] ReadSchema: struct (45) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Input [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] (46) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#59] (47) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#56] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#58] +Right keys [1]: [d_date_sk#59] Join condition: None (48) Project [codegen id : 13] -Output [2]: [cr_return_amount#54, cr_net_loss#55] -Input [4]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56, d_date_sk#6] +Output [2]: [cr_return_amount#56, cr_net_loss#57] +Input [4]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58, d_date_sk#59] (49) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#54, cr_net_loss#55] +Input [2]: [cr_return_amount#56, cr_net_loss#57] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#54)), partial_sum(UnscaledValue(cr_net_loss#55))] -Aggregate Attributes [2]: [sum#57, sum#58] -Results [2]: [sum#59, sum#60] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#56)), partial_sum(UnscaledValue(cr_net_loss#57))] +Aggregate Attributes [2]: [sum#60, sum#61] +Results [2]: [sum#62, sum#63] (50) Exchange -Input [2]: [sum#59, sum#60] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] +Input [2]: [sum#62, sum#63] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#64] (51) HashAggregate -Input [2]: [sum#59, sum#60] +Input [2]: [sum#62, sum#63] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#54)), sum(UnscaledValue(cr_net_loss#55))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#54))#62, sum(UnscaledValue(cr_net_loss#55))#63] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#54))#62,17,2) AS returns#64, MakeDecimal(sum(UnscaledValue(cr_net_loss#55))#63,17,2) AS profit_loss#65] +Functions [2]: [sum(UnscaledValue(cr_return_amount#56)), sum(UnscaledValue(cr_net_loss#57))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#56))#65, sum(UnscaledValue(cr_net_loss#57))#66] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#56))#65,17,2) AS returns#67, MakeDecimal(sum(UnscaledValue(cr_net_loss#57))#66,17,2) AS profit_loss#68] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#66, cs_call_center_sk#40 AS id#67, sales#51, returns#64, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#65 as decimal(18,2)))), DecimalType(18,2), true) AS profit#68] -Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#64, profit_loss#65] +Output [5]: [catalog channel AS channel#69, cs_call_center_sk#41 AS id#70, sales#53, returns#67, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#68 as decimal(18,2)))), DecimalType(18,2), true) AS profit#71] +Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#67, profit_loss#68] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] -Condition : isnotnull(ws_web_page_sk#69) +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Condition : isnotnull(ws_web_page_sk#72) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#76] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#72] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#75] +Right keys [1]: [d_date_sk#76] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] -Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] +Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] +Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#73] +Output [1]: [wp_web_page_sk#77] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#73] +Input [1]: [wp_web_page_sk#77] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#73] -Condition : isnotnull(wp_web_page_sk#73) +Input [1]: [wp_web_page_sk#77] +Condition : isnotnull(wp_web_page_sk#77) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [1]: [wp_web_page_sk#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#69] -Right keys [1]: [wp_web_page_sk#73] +Left keys [1]: [ws_web_page_sk#72] +Right keys [1]: [wp_web_page_sk#77] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum#75, sum#76] -Results [3]: [wp_web_page_sk#73, sum#77, sum#78] +Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum#79, sum#80] +Results [3]: [wp_web_page_sk#77, sum#81, sum#82] (67) Exchange -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] -Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] +Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : isnotnull(wr_web_page_sk#84) +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Condition : isnotnull(wr_web_page_sk#88) (72) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#92] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#87] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#91] +Right keys [1]: [cast(d_date_sk#92 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86] -Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, d_date_sk#6] +Output [3]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90] +Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, d_date_sk#92] (75) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#88] +Output [1]: [wp_web_page_sk#93] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#84] -Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] +Left keys [1]: [wr_web_page_sk#88] +Right keys [1]: [cast(wp_web_page_sk#93 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum#89, sum#90] -Results [3]: [wp_web_page_sk#88, sum#91, sum#92] +Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] +Keys [1]: [wp_web_page_sk#93] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum#94, sum#95] +Results [3]: [wp_web_page_sk#93, sum#96, sum#97] (79) Exchange -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [3]: [wp_web_page_sk#93, sum#96, sum#97] +Arguments: hashpartitioning(wp_web_page_sk#93, 5), ENSURE_REQUIREMENTS, [id=#98] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] -Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] +Input [3]: [wp_web_page_sk#93, sum#96, sum#97] +Keys [1]: [wp_web_page_sk#93] +Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] +Results [3]: [wp_web_page_sk#93, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] +Input [3]: [wp_web_page_sk#93, returns#101, profit_loss#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#73] -Right keys [1]: [wp_web_page_sk#88] +Left keys [1]: [wp_web_page_sk#77] +Right keys [1]: [wp_web_page_sk#93] Join condition: None (83) Project [codegen id : 22] -Output [5]: [web channel AS channel#99, wp_web_page_sk#73 AS id#100, sales#82, coalesce(returns#96, 0.00) AS returns#101, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#102] -Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] +Output [5]: [web channel AS channel#104, wp_web_page_sk#77 AS id#105, sales#86, coalesce(returns#101, 0.00) AS returns#106, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#107] +Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#93, returns#101, profit_loss#102] (84) Union (85) HashAggregate [codegen id : 23] -Input [5]: [channel#36, id#37, sales#18, returns#38, profit#39] -Keys [2]: [channel#36, id#37] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#38), partial_sum(profit#39)] -Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Results [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Input [5]: [channel#37, id#38, sales#18, returns#39, profit#40] +Keys [2]: [channel#37, id#38] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#39), partial_sum(profit#40)] +Aggregate Attributes [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +Results [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] (86) Exchange -Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#36, id#37, 5), ENSURE_REQUIREMENTS, [id=#115] +Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [id=#120] (87) HashAggregate [codegen id : 24] -Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#18)#116, sum(returns#38)#117, sum(profit#39)#118] -Results [5]: [channel#36, id#37, cast(sum(sales#18)#116 as decimal(37,2)) AS sales#119, cast(sum(returns#38)#117 as decimal(37,2)) AS returns#120, cast(sum(profit#39)#118 as decimal(38,2)) AS profit#121] +Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#40)] +Aggregate Attributes [3]: [sum(sales#18)#121, sum(returns#39)#122, sum(profit#40)#123] +Results [5]: [channel#37, id#38, cast(sum(sales#18)#121 as decimal(37,2)) AS sales#124, cast(sum(returns#39)#122 as decimal(37,2)) AS returns#125, cast(sum(profit#40)#123 as decimal(38,2)) AS profit#126] (88) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] +Output [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] (89) HashAggregate [codegen id : 48] -Input [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#128)] -Aggregate Attributes [3]: [sum(sales#18)#129, sum(returns#38)#130, sum(profit#128)#131] -Results [4]: [channel#36, sum(sales#18)#129 AS sales#132, sum(returns#38)#130 AS returns#133, sum(profit#128)#131 AS profit#134] +Input [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#133)] +Aggregate Attributes [3]: [sum(sales#18)#134, sum(returns#39)#135, sum(profit#133)#136] +Results [4]: [channel#37, sum(sales#18)#134 AS sales#137, sum(returns#39)#135 AS returns#138, sum(profit#133)#136 AS profit#139] (90) HashAggregate [codegen id : 48] -Input [4]: [channel#36, sales#132, returns#133, profit#134] -Keys [1]: [channel#36] -Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] -Aggregate Attributes [6]: [sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Results [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Input [4]: [channel#37, sales#137, returns#138, profit#139] +Keys [1]: [channel#37] +Functions [3]: [partial_sum(sales#137), partial_sum(returns#138), partial_sum(profit#139)] +Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Results [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] (91) Exchange -Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Arguments: hashpartitioning(channel#36, 5), ENSURE_REQUIREMENTS, [id=#147] +Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [id=#152] (92) HashAggregate [codegen id : 49] -Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Keys [1]: [channel#36] -Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] -Aggregate Attributes [3]: [sum(sales#132)#148, sum(returns#133)#149, sum(profit#134)#150] -Results [5]: [channel#36, null AS id#151, sum(sales#132)#148 AS sales#152, sum(returns#133)#149 AS returns#153, sum(profit#134)#150 AS profit#154] +Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Keys [1]: [channel#37] +Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] +Aggregate Attributes [3]: [sum(sales#137)#153, sum(returns#138)#154, sum(profit#139)#155] +Results [5]: [channel#37, null AS id#156, sum(sales#137)#153 AS sales#157, sum(returns#138)#154 AS returns#158, sum(profit#139)#155 AS profit#159] (93) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] +Output [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] (94) HashAggregate [codegen id : 73] -Input [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#161)] -Aggregate Attributes [3]: [sum(sales#18)#162, sum(returns#38)#163, sum(profit#161)#164] -Results [3]: [sum(sales#18)#162 AS sales#132, sum(returns#38)#163 AS returns#133, sum(profit#161)#164 AS profit#134] +Input [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#166)] +Aggregate Attributes [3]: [sum(sales#18)#167, sum(returns#39)#168, sum(profit#166)#169] +Results [3]: [sum(sales#18)#167 AS sales#137, sum(returns#39)#168 AS returns#138, sum(profit#166)#169 AS profit#139] (95) HashAggregate [codegen id : 73] -Input [3]: [sales#132, returns#133, profit#134] +Input [3]: [sales#137, returns#138, profit#139] Keys: [] -Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] -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#137), partial_sum(returns#138), partial_sum(profit#139)] +Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Results [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] (96) Exchange -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#177] +Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#182] (97) HashAggregate [codegen id : 74] -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Keys: [] -Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] -Aggregate Attributes [3]: [sum(sales#132)#178, sum(returns#133)#179, sum(profit#134)#180] -Results [5]: [null AS channel#181, null AS id#182, sum(sales#132)#178 AS sales#183, sum(returns#133)#179 AS returns#184, sum(profit#134)#180 AS profit#185] +Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] +Aggregate Attributes [3]: [sum(sales#137)#183, sum(returns#138)#184, sum(profit#139)#185] +Results [5]: [null AS channel#186, null AS id#187, sum(sales#137)#183 AS sales#188, sum(returns#138)#184 AS returns#189, sum(profit#139)#185 AS profit#190] (98) Union (99) HashAggregate [codegen id : 75] -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] (100) Exchange -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Arguments: hashpartitioning(channel#36, id#37, sales#119, returns#120, profit#121, 5), ENSURE_REQUIREMENTS, [id=#186] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Arguments: hashpartitioning(channel#37, id#38, sales#124, returns#125, profit#126, 5), ENSURE_REQUIREMENTS, [id=#191] (101) HashAggregate [codegen id : 76] -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] (102) TakeOrderedAndProject -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Arguments: 100, [channel#36 ASC NULLS FIRST, id#37 ASC NULLS FIRST], [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#124, returns#125, profit#126] ===== Subqueries ===== @@ -611,14 +611,14 @@ ReusedExchange (104) (104) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#25] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#58 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt index 0bd6422fffa5a..a12ab50a89f43 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt @@ -248,164 +248,164 @@ Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale Input [9]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#43, d_year#44] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#43] Join condition: None (41) Project [codegen id : 13] -Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] -Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#14, d_year#15] +Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] +Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#43, d_year#44] (42) HashAggregate [codegen id : 13] -Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] -Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] +Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [partial_sum(cs_quantity#34), partial_sum(UnscaledValue(cs_wholesale_cost#35)), partial_sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum#43, sum#44, sum#45] -Results [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] +Aggregate Attributes [3]: [sum#45, sum#46, sum#47] +Results [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] (43) Exchange -Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] -Arguments: hashpartitioning(d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] +Arguments: hashpartitioning(d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#51] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] -Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] +Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [sum(cs_quantity#34), sum(UnscaledValue(cs_wholesale_cost#35)), sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum(cs_quantity#34)#50, sum(UnscaledValue(cs_wholesale_cost#35))#51, sum(UnscaledValue(cs_sales_price#36))#52] -Results [6]: [d_year#15 AS cs_sold_year#53, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#54, sum(cs_quantity#34)#50 AS cs_qty#55, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#51,17,2) AS cs_wc#56, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#52,17,2) AS cs_sp#57] +Aggregate Attributes [3]: [sum(cs_quantity#34)#52, sum(UnscaledValue(cs_wholesale_cost#35))#53, sum(UnscaledValue(cs_sales_price#36))#54] +Results [6]: [d_year#44 AS cs_sold_year#55, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#56, sum(cs_quantity#34)#52 AS cs_qty#57, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#53,17,2) AS cs_wc#58, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#54,17,2) AS cs_sp#59] (45) Filter [codegen id : 14] -Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] -Condition : (coalesce(cs_qty#55, 0) > 0) +Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] +Condition : (coalesce(cs_qty#57, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] -Arguments: [cs_sold_year#53 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] +Arguments: [cs_sold_year#55 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#56 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54] +Right keys [3]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] (49) Scan parquet default.web_sales -Output [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Output [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] (51) Filter [codegen id : 16] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Condition : (isnotnull(ws_item_sk#58) AND isnotnull(ws_bill_customer_sk#59)) +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Condition : (isnotnull(ws_item_sk#60) AND isnotnull(ws_bill_customer_sk#61)) (52) Exchange -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Arguments: hashpartitioning(cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint), 5), ENSURE_REQUIREMENTS, [id=#65] +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Arguments: hashpartitioning(cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint), 5), ENSURE_REQUIREMENTS, [id=#67] (53) Sort [codegen id : 17] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Arguments: [cast(ws_order_number#60 as bigint) ASC NULLS FIRST, cast(ws_item_sk#58 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Arguments: [cast(ws_order_number#62 as bigint) ASC NULLS FIRST, cast(ws_item_sk#60 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.web_returns -Output [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Output [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] (56) Filter [codegen id : 18] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] -Condition : (isnotnull(wr_order_number#67) AND isnotnull(wr_item_sk#66)) +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Condition : (isnotnull(wr_order_number#69) AND isnotnull(wr_item_sk#68)) (57) Project [codegen id : 18] -Output [2]: [wr_item_sk#66, wr_order_number#67] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Output [2]: [wr_item_sk#68, wr_order_number#69] +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] (58) Exchange -Input [2]: [wr_item_sk#66, wr_order_number#67] -Arguments: hashpartitioning(wr_order_number#67, wr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [wr_item_sk#68, wr_order_number#69] +Arguments: hashpartitioning(wr_order_number#69, wr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] (59) Sort [codegen id : 19] -Input [2]: [wr_item_sk#66, wr_order_number#67] -Arguments: [wr_order_number#67 ASC NULLS FIRST, wr_item_sk#66 ASC NULLS FIRST], false, 0 +Input [2]: [wr_item_sk#68, wr_order_number#69] +Arguments: [wr_order_number#69 ASC NULLS FIRST, wr_item_sk#68 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint)] -Right keys [2]: [wr_order_number#67, wr_item_sk#66] +Left keys [2]: [cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint)] +Right keys [2]: [wr_order_number#69, wr_item_sk#68] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] -Condition : isnull(wr_order_number#67) +Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] +Condition : isnull(wr_order_number#69) (62) Project [codegen id : 21] -Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] +Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#72, d_year#73] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [ws_sold_date_sk#64] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#66] +Right keys [1]: [d_date_sk#72] Join condition: None (65) Project [codegen id : 21] -Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] -Input [8]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, d_date_sk#14, d_year#15] +Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] +Input [8]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, d_date_sk#72, d_year#73] (66) HashAggregate [codegen id : 21] -Input [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] -Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] -Functions [3]: [partial_sum(ws_quantity#61), partial_sum(UnscaledValue(ws_wholesale_cost#62)), partial_sum(UnscaledValue(ws_sales_price#63))] -Aggregate Attributes [3]: [sum#70, sum#71, sum#72] -Results [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] +Input [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] +Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] +Functions [3]: [partial_sum(ws_quantity#63), partial_sum(UnscaledValue(ws_wholesale_cost#64)), partial_sum(UnscaledValue(ws_sales_price#65))] +Aggregate Attributes [3]: [sum#74, sum#75, sum#76] +Results [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] (67) Exchange -Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] +Arguments: hashpartitioning(d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#80] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] -Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] -Functions [3]: [sum(ws_quantity#61), sum(UnscaledValue(ws_wholesale_cost#62)), sum(UnscaledValue(ws_sales_price#63))] -Aggregate Attributes [3]: [sum(ws_quantity#61)#77, sum(UnscaledValue(ws_wholesale_cost#62))#78, sum(UnscaledValue(ws_sales_price#63))#79] -Results [6]: [d_year#15 AS ws_sold_year#80, ws_item_sk#58, ws_bill_customer_sk#59 AS ws_customer_sk#81, sum(ws_quantity#61)#77 AS ws_qty#82, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#62))#78,17,2) AS ws_wc#83, MakeDecimal(sum(UnscaledValue(ws_sales_price#63))#79,17,2) AS ws_sp#84] +Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] +Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] +Functions [3]: [sum(ws_quantity#63), sum(UnscaledValue(ws_wholesale_cost#64)), sum(UnscaledValue(ws_sales_price#65))] +Aggregate Attributes [3]: [sum(ws_quantity#63)#81, sum(UnscaledValue(ws_wholesale_cost#64))#82, sum(UnscaledValue(ws_sales_price#65))#83] +Results [6]: [d_year#73 AS ws_sold_year#84, ws_item_sk#60, ws_bill_customer_sk#61 AS ws_customer_sk#85, sum(ws_quantity#63)#81 AS ws_qty#86, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#64))#82,17,2) AS ws_wc#87, MakeDecimal(sum(UnscaledValue(ws_sales_price#65))#83,17,2) AS ws_sp#88] (69) Filter [codegen id : 22] -Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] -Condition : (coalesce(ws_qty#82, 0) > 0) +Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] +Condition : (coalesce(ws_qty#86, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] -Arguments: [ws_sold_year#80 ASC NULLS FIRST, ws_item_sk#58 ASC NULLS FIRST, ws_customer_sk#81 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] +Arguments: [ws_sold_year#84 ASC NULLS FIRST, ws_item_sk#60 ASC NULLS FIRST, ws_customer_sk#85 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81] +Right keys [3]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85] Join condition: None (72) Project [codegen id : 23] -Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#82 + cs_qty#55), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#82, 0) + coalesce(cs_qty#55, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#83, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#56, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#84, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#57, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57, ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] +Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#86 + cs_qty#57), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#86, 0) + coalesce(cs_qty#57, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#87, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#58, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#88, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#59, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59, ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] (73) TakeOrderedAndProject -Input [13]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, ratio#85 ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] +Input [13]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, ratio#89 ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt index 2d84cacd5d09c..b14ab47d28afe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt @@ -248,164 +248,164 @@ Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale Input [9]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_order_number#33, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, wr_item_sk#39, wr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#43, d_year#44] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#37] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#43] Join condition: None (41) Project [codegen id : 13] -Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] -Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#14, d_year#15] +Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] +Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#43, d_year#44] (42) HashAggregate [codegen id : 13] -Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] -Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] +Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [partial_sum(ws_quantity#34), partial_sum(UnscaledValue(ws_wholesale_cost#35)), partial_sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum#43, sum#44, sum#45] -Results [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] +Aggregate Attributes [3]: [sum#45, sum#46, sum#47] +Results [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] (43) Exchange -Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] -Arguments: hashpartitioning(d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] +Arguments: hashpartitioning(d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#51] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] -Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] +Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [sum(ws_quantity#34), sum(UnscaledValue(ws_wholesale_cost#35)), sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum(ws_quantity#34)#50, sum(UnscaledValue(ws_wholesale_cost#35))#51, sum(UnscaledValue(ws_sales_price#36))#52] -Results [6]: [d_year#15 AS ws_sold_year#53, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#54, sum(ws_quantity#34)#50 AS ws_qty#55, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#51,17,2) AS ws_wc#56, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#52,17,2) AS ws_sp#57] +Aggregate Attributes [3]: [sum(ws_quantity#34)#52, sum(UnscaledValue(ws_wholesale_cost#35))#53, sum(UnscaledValue(ws_sales_price#36))#54] +Results [6]: [d_year#44 AS ws_sold_year#55, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#56, sum(ws_quantity#34)#52 AS ws_qty#57, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#53,17,2) AS ws_wc#58, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#54,17,2) AS ws_sp#59] (45) Filter [codegen id : 14] -Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] -Condition : (coalesce(ws_qty#55, 0) > 0) +Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] +Condition : (coalesce(ws_qty#57, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] -Arguments: [ws_sold_year#53 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] +Arguments: [ws_sold_year#55 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#56 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54] +Right keys [3]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] (49) Scan parquet default.catalog_sales -Output [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Output [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#66), dynamicpruningexpression(cs_sold_date_sk#66 IN dynamicpruning#8)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] (51) Filter [codegen id : 16] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Condition : (isnotnull(cs_item_sk#59) AND isnotnull(cs_bill_customer_sk#58)) +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Condition : (isnotnull(cs_item_sk#61) AND isnotnull(cs_bill_customer_sk#60)) (52) Exchange -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Arguments: hashpartitioning(cs_order_number#60, cs_item_sk#59, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Arguments: hashpartitioning(cs_order_number#62, cs_item_sk#61, 5), ENSURE_REQUIREMENTS, [id=#67] (53) Sort [codegen id : 17] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Arguments: [cs_order_number#60 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST], false, 0 +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Arguments: [cs_order_number#62 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST], false, 0 (54) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Output [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] (56) Filter [codegen id : 18] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] -Condition : (isnotnull(cr_order_number#67) AND isnotnull(cr_item_sk#66)) +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Condition : (isnotnull(cr_order_number#69) AND isnotnull(cr_item_sk#68)) (57) Project [codegen id : 18] -Output [2]: [cr_item_sk#66, cr_order_number#67] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Output [2]: [cr_item_sk#68, cr_order_number#69] +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] (58) Exchange -Input [2]: [cr_item_sk#66, cr_order_number#67] -Arguments: hashpartitioning(cr_order_number#67, cr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [cr_item_sk#68, cr_order_number#69] +Arguments: hashpartitioning(cr_order_number#69, cr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] (59) Sort [codegen id : 19] -Input [2]: [cr_item_sk#66, cr_order_number#67] -Arguments: [cr_order_number#67 ASC NULLS FIRST, cr_item_sk#66 ASC NULLS FIRST], false, 0 +Input [2]: [cr_item_sk#68, cr_order_number#69] +Arguments: [cr_order_number#69 ASC NULLS FIRST, cr_item_sk#68 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cs_order_number#60, cs_item_sk#59] -Right keys [2]: [cr_order_number#67, cr_item_sk#66] +Left keys [2]: [cs_order_number#62, cs_item_sk#61] +Right keys [2]: [cr_order_number#69, cr_item_sk#68] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] -Condition : isnull(cr_order_number#67) +Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] +Condition : isnull(cr_order_number#69) (62) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] +Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#72, d_year#73] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [cs_sold_date_sk#64] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [cs_sold_date_sk#66] +Right keys [1]: [d_date_sk#72] Join condition: None (65) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] -Input [8]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, d_date_sk#14, d_year#15] +Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] +Input [8]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, d_date_sk#72, d_year#73] (66) HashAggregate [codegen id : 21] -Input [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] -Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] -Functions [3]: [partial_sum(cs_quantity#61), partial_sum(UnscaledValue(cs_wholesale_cost#62)), partial_sum(UnscaledValue(cs_sales_price#63))] -Aggregate Attributes [3]: [sum#70, sum#71, sum#72] -Results [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] +Input [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] +Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] +Functions [3]: [partial_sum(cs_quantity#63), partial_sum(UnscaledValue(cs_wholesale_cost#64)), partial_sum(UnscaledValue(cs_sales_price#65))] +Aggregate Attributes [3]: [sum#74, sum#75, sum#76] +Results [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] (67) Exchange -Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] +Arguments: hashpartitioning(d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#80] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] -Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] -Functions [3]: [sum(cs_quantity#61), sum(UnscaledValue(cs_wholesale_cost#62)), sum(UnscaledValue(cs_sales_price#63))] -Aggregate Attributes [3]: [sum(cs_quantity#61)#77, sum(UnscaledValue(cs_wholesale_cost#62))#78, sum(UnscaledValue(cs_sales_price#63))#79] -Results [6]: [d_year#15 AS cs_sold_year#80, cs_item_sk#59, cs_bill_customer_sk#58 AS cs_customer_sk#81, sum(cs_quantity#61)#77 AS cs_qty#82, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#62))#78,17,2) AS cs_wc#83, MakeDecimal(sum(UnscaledValue(cs_sales_price#63))#79,17,2) AS cs_sp#84] +Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] +Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] +Functions [3]: [sum(cs_quantity#63), sum(UnscaledValue(cs_wholesale_cost#64)), sum(UnscaledValue(cs_sales_price#65))] +Aggregate Attributes [3]: [sum(cs_quantity#63)#81, sum(UnscaledValue(cs_wholesale_cost#64))#82, sum(UnscaledValue(cs_sales_price#65))#83] +Results [6]: [d_year#73 AS cs_sold_year#84, cs_item_sk#61, cs_bill_customer_sk#60 AS cs_customer_sk#85, sum(cs_quantity#63)#81 AS cs_qty#86, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#64))#82,17,2) AS cs_wc#87, MakeDecimal(sum(UnscaledValue(cs_sales_price#65))#83,17,2) AS cs_sp#88] (69) Filter [codegen id : 22] -Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] -Condition : (coalesce(cs_qty#82, 0) > 0) +Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] +Condition : (coalesce(cs_qty#86, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] -Arguments: [cs_sold_year#80 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST, cs_customer_sk#81 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] +Arguments: [cs_sold_year#84 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST, cs_customer_sk#85 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81] +Right keys [3]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85] Join condition: None (72) Project [codegen id : 23] -Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#55 + cs_qty#82), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#55, 0) + coalesce(cs_qty#82, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#56, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#83, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#57, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#84, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57, cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] +Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#57 + cs_qty#86), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#57, 0) + coalesce(cs_qty#86, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#58, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#87, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#59, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#88, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59, cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] (73) TakeOrderedAndProject -Input [13]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, ratio#85 ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] +Input [13]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, ratio#89 ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#8 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 e192ab8d637de..a5d47ede475ca 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 @@ -257,7 +257,7 @@ Input [2]: [d_date_sk#22, d_date#23] (30) Filter [codegen id : 7] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 10442)) AND (d_date#23 <= 10472)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-09-03)) AND isnotnull(d_date_sk#22)) (31) Project [codegen id : 7] Output [1]: [d_date_sk#22] @@ -380,224 +380,224 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#16] +Output [1]: [i_item_sk#61] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#16] +Right keys [1]: [i_item_sk#61] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#16] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#61] (60) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#19] +Output [1]: [p_promo_sk#62] (61) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#19] +Right keys [1]: [p_promo_sk#62] Join condition: None (62) Project [codegen id : 19] Output [6]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#19] +Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#62] (63) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#63] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#63] Join condition: None (65) Project [codegen id : 19] Output [5]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#22] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#63] (66) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 18] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] (68) Filter [codegen id : 18] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Condition : isnotnull(cp_catalog_page_sk#61) +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Condition : isnotnull(cp_catalog_page_sk#64) (69) BroadcastExchange -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#61] +Right keys [1]: [cp_catalog_page_sk#64] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#64, cp_catalog_page_id#65] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Keys [1]: [cp_catalog_page_id#62] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] +Keys [1]: [cp_catalog_page_id#65] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Results [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] (73) Exchange -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [id=#77] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [cp_catalog_page_id#62] +Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Keys [1]: [cp_catalog_page_id#65] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] -Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_page_id#62) AS id#79, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#80, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#81, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#82] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] +Results [5]: [catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#65) AS id#82, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#83, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#84, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#85] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) (78) Exchange -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] -Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (84) Exchange -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] -Right keys [2]: [wr_item_sk#91, wr_order_number#92] +Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] +Right keys [2]: [wr_item_sk#94, wr_order_number#95] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#16] +Output [1]: [i_item_sk#100] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#83] -Right keys [1]: [i_item_sk#16] +Left keys [1]: [ws_item_sk#86] +Right keys [1]: [i_item_sk#100] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, i_item_sk#16] +Output [7]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, i_item_sk#100] (91) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#19] +Output [1]: [p_promo_sk#101] (92) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#85] -Right keys [1]: [p_promo_sk#19] +Left keys [1]: [ws_promo_sk#88] +Right keys [1]: [p_promo_sk#101] Join condition: None (93) Project [codegen id : 29] -Output [6]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [8]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, p_promo_sk#19] +Output [6]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [8]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, p_promo_sk#101] (94) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#102] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#89] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [ws_sold_date_sk#92] +Right keys [1]: [d_date_sk#102] Join condition: None (96) Project [codegen id : 29] -Output [5]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] -Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#22] +Output [5]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] +Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#102] (97) Scan parquet default.web_site -Output [2]: [web_site_sk#97, web_site_id#98] +Output [2]: [web_site_sk#103, web_site_id#104] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (98) ColumnarToRow [codegen id : 28] -Input [2]: [web_site_sk#97, web_site_id#98] +Input [2]: [web_site_sk#103, web_site_id#104] (99) Filter [codegen id : 28] -Input [2]: [web_site_sk#97, web_site_id#98] -Condition : isnotnull(web_site_sk#97) +Input [2]: [web_site_sk#103, web_site_id#104] +Condition : isnotnull(web_site_sk#103) (100) BroadcastExchange -Input [2]: [web_site_sk#97, web_site_id#98] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Input [2]: [web_site_sk#103, web_site_id#104] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#105] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#84] -Right keys [1]: [web_site_sk#97] +Left keys [1]: [ws_web_site_sk#87] +Right keys [1]: [web_site_sk#103] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] +Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] +Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#103, web_site_id#104] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Keys [1]: [web_site_id#98] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] +Keys [1]: [web_site_id#104] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Results [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] (104) Exchange -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] +Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, [id=#116] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Keys [1]: [web_site_id#98] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] -Results [5]: [web channel AS channel#114, concat(web_site, web_site_id#98) AS id#115, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#116, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#117, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#118] +Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Keys [1]: [web_site_id#104] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] +Results [5]: [web channel AS channel#120, concat(web_site, web_site_id#104) AS id#121, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#122, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#123, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#124] (106) Union @@ -605,99 +605,99 @@ Results [5]: [web channel AS channel#114, concat(web_site, web_site_id#98) AS id Input [5]: [channel#42, id#43, sales#44, returns#45, profit#46] Keys [2]: [channel#42, id#43] Functions [3]: [partial_sum(sales#44), partial_sum(returns#45), partial_sum(profit#46)] -Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Results [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Aggregate Attributes [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Results [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] (108) Exchange -Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#131] +Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#137] (109) HashAggregate [codegen id : 32] -Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#132, sum(returns#45)#133, sum(profit#46)#134] -Results [5]: [channel#42, id#43, cast(sum(sales#44)#132 as decimal(37,2)) AS sales#135, cast(sum(returns#45)#133 as decimal(38,2)) AS returns#136, cast(sum(profit#46)#134 as decimal(38,2)) AS profit#137] +Aggregate Attributes [3]: [sum(sales#44)#138, sum(returns#45)#139, sum(profit#46)#140] +Results [5]: [channel#42, id#43, cast(sum(sales#44)#138 as decimal(37,2)) AS sales#141, cast(sum(returns#45)#139 as decimal(38,2)) AS returns#142, cast(sum(profit#46)#140 as decimal(38,2)) AS profit#143] (110) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Output [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] (111) HashAggregate [codegen id : 64] -Input [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Input [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#144, sum(returns#45)#145, sum(profit#46)#146] -Results [4]: [channel#42, sum(sales#44)#144 AS sales#147, sum(returns#45)#145 AS returns#148, sum(profit#46)#146 AS profit#149] +Aggregate Attributes [3]: [sum(sales#44)#150, sum(returns#45)#151, sum(profit#46)#152] +Results [4]: [channel#42, sum(sales#44)#150 AS sales#153, sum(returns#45)#151 AS returns#154, sum(profit#46)#152 AS profit#155] (112) HashAggregate [codegen id : 64] -Input [4]: [channel#42, sales#147, returns#148, profit#149] +Input [4]: [channel#42, sales#153, returns#154, profit#155] Keys [1]: [channel#42] -Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] -Aggregate Attributes [6]: [sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] -Results [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +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#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] (113) Exchange -Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] -Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#162] +Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#168] (114) HashAggregate [codegen id : 65] -Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] Keys [1]: [channel#42] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#163, sum(returns#148)#164, sum(profit#149)#165] -Results [5]: [channel#42, null AS id#166, sum(sales#147)#163 AS sales#167, sum(returns#148)#164 AS returns#168, sum(profit#149)#165 AS profit#169] +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#42, null AS id#172, sum(sales#153)#169 AS sales#173, sum(returns#154)#170 AS returns#174, sum(profit#155)#171 AS profit#175] (115) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Output [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] (116) HashAggregate [codegen id : 97] -Input [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Input [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#176, sum(returns#45)#177, sum(profit#46)#178] -Results [3]: [sum(sales#44)#176 AS sales#147, sum(returns#45)#177 AS returns#148, sum(profit#46)#178 AS profit#149] +Aggregate Attributes [3]: [sum(sales#44)#182, sum(returns#45)#183, sum(profit#46)#184] +Results [3]: [sum(sales#44)#182 AS sales#153, sum(returns#45)#183 AS returns#154, sum(profit#46)#184 AS profit#155] (117) HashAggregate [codegen id : 97] -Input [3]: [sales#147, returns#148, profit#149] +Input [3]: [sales#153, returns#154, profit#155] Keys: [] -Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] -Aggregate Attributes [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] -Results [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Functions [3]: [partial_sum(sales#153), partial_sum(returns#154), partial_sum(profit#155)] +Aggregate Attributes [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Results [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] (118) Exchange -Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#191] +Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#197] (119) HashAggregate [codegen id : 98] -Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] Keys: [] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#192, sum(returns#148)#193, sum(profit#149)#194] -Results [5]: [null AS channel#195, null AS id#196, sum(sales#147)#192 AS sales#197, sum(returns#148)#193 AS returns#198, sum(profit#149)#194 AS profit#199] +Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] +Aggregate Attributes [3]: [sum(sales#153)#198, sum(returns#154)#199, sum(profit#155)#200] +Results [5]: [null AS channel#201, null AS id#202, sum(sales#153)#198 AS sales#203, sum(returns#154)#199 AS returns#204, sum(profit#155)#200 AS profit#205] (120) Union (121) HashAggregate [codegen id : 99] -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] (122) Exchange -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Arguments: hashpartitioning(channel#42, id#43, sales#135, returns#136, profit#137, 5), ENSURE_REQUIREMENTS, [id=#200] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Arguments: hashpartitioning(channel#42, id#43, sales#141, returns#142, profit#143, 5), ENSURE_REQUIREMENTS, [id=#206] (123) HashAggregate [codegen id : 100] -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] (124) TakeOrderedAndProject -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#141, returns#142, profit#143] ===== Subqueries ===== @@ -710,6 +710,6 @@ Output [1]: [d_date_sk#22] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 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 e027ef7e53e8a..9e5bd3aeaa1d1 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 @@ -195,7 +195,7 @@ Input [2]: [d_date_sk#16, d_date#17] (16) Filter [codegen id : 5] Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 10442)) AND (d_date#17 <= 10472)) AND isnotnull(d_date_sk#16)) +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 1998-08-04)) AND (d_date#17 <= 1998-09-03)) AND isnotnull(d_date_sk#16)) (17) Project [codegen id : 5] Output [1]: [d_date_sk#16] @@ -380,224 +380,224 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#61] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#61] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#16] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#61] (60) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] (62) Filter [codegen id : 16] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Condition : isnotnull(cp_catalog_page_sk#61) +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Condition : isnotnull(cp_catalog_page_sk#62) (63) BroadcastExchange -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#61] +Right keys [1]: [cp_catalog_page_sk#62] Join condition: None (65) Project [codegen id : 19] -Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#62, cp_catalog_page_id#63] (66) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#22] +Output [1]: [i_item_sk#65] (67) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#22] +Right keys [1]: [i_item_sk#65] Join condition: None (68) Project [codegen id : 19] -Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, i_item_sk#22] +Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, i_item_sk#65] (69) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#25] +Output [1]: [p_promo_sk#66] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#25] +Right keys [1]: [p_promo_sk#66] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, p_promo_sk#25] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, p_promo_sk#66] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Keys [1]: [cp_catalog_page_id#62] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Keys [1]: [cp_catalog_page_id#63] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Results [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] (73) Exchange -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Arguments: hashpartitioning(cp_catalog_page_id#63, 5), ENSURE_REQUIREMENTS, [id=#77] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [cp_catalog_page_id#62] +Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Keys [1]: [cp_catalog_page_id#63] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] -Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_page_id#62) AS id#79, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#80, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#81, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#82] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] +Results [5]: [catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#63) AS id#82, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#83, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#84, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#85] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) (78) Exchange -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] -Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (84) Exchange -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] -Right keys [2]: [wr_item_sk#91, wr_order_number#92] +Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] +Right keys [2]: [wr_item_sk#94, wr_order_number#95] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#100] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#89] -Right keys [1]: [d_date_sk#16] +Left keys [1]: [ws_sold_date_sk#92] +Right keys [1]: [d_date_sk#100] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#16] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#100] (91) Scan parquet default.web_site -Output [2]: [web_site_sk#97, web_site_id#98] +Output [2]: [web_site_sk#101, web_site_id#102] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 26] -Input [2]: [web_site_sk#97, web_site_id#98] +Input [2]: [web_site_sk#101, web_site_id#102] (93) Filter [codegen id : 26] -Input [2]: [web_site_sk#97, web_site_id#98] -Condition : isnotnull(web_site_sk#97) +Input [2]: [web_site_sk#101, web_site_id#102] +Condition : isnotnull(web_site_sk#101) (94) BroadcastExchange -Input [2]: [web_site_sk#97, web_site_id#98] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Input [2]: [web_site_sk#101, web_site_id#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#84] -Right keys [1]: [web_site_sk#97] +Left keys [1]: [ws_web_site_sk#87] +Right keys [1]: [web_site_sk#101] Join condition: None (96) Project [codegen id : 29] -Output [7]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] +Output [7]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#101, web_site_id#102] (97) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#22] +Output [1]: [i_item_sk#104] (98) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#83] -Right keys [1]: [i_item_sk#22] +Left keys [1]: [ws_item_sk#86] +Right keys [1]: [i_item_sk#104] Join condition: None (99) Project [codegen id : 29] -Output [6]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [8]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, i_item_sk#22] +Output [6]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [8]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, i_item_sk#104] (100) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#25] +Output [1]: [p_promo_sk#105] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#85] -Right keys [1]: [p_promo_sk#25] +Left keys [1]: [ws_promo_sk#88] +Right keys [1]: [p_promo_sk#105] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [7]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, p_promo_sk#25] +Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [7]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, p_promo_sk#105] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Keys [1]: [web_site_id#98] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Keys [1]: [web_site_id#102] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Results [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] (104) Exchange -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] +Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Arguments: hashpartitioning(web_site_id#102, 5), ENSURE_REQUIREMENTS, [id=#116] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Keys [1]: [web_site_id#98] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] -Results [5]: [web channel AS channel#114, concat(web_site, web_site_id#98) AS id#115, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#116, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#117, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#118] +Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Keys [1]: [web_site_id#102] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] +Results [5]: [web channel AS channel#120, concat(web_site, web_site_id#102) AS id#121, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#122, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#123, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#124] (106) Union @@ -605,99 +605,99 @@ Results [5]: [web channel AS channel#114, concat(web_site, web_site_id#98) AS id Input [5]: [channel#42, id#43, sales#44, returns#45, profit#46] Keys [2]: [channel#42, id#43] Functions [3]: [partial_sum(sales#44), partial_sum(returns#45), partial_sum(profit#46)] -Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Results [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Aggregate Attributes [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Results [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] (108) Exchange -Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#131] +Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#137] (109) HashAggregate [codegen id : 32] -Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#132, sum(returns#45)#133, sum(profit#46)#134] -Results [5]: [channel#42, id#43, cast(sum(sales#44)#132 as decimal(37,2)) AS sales#135, cast(sum(returns#45)#133 as decimal(38,2)) AS returns#136, cast(sum(profit#46)#134 as decimal(38,2)) AS profit#137] +Aggregate Attributes [3]: [sum(sales#44)#138, sum(returns#45)#139, sum(profit#46)#140] +Results [5]: [channel#42, id#43, cast(sum(sales#44)#138 as decimal(37,2)) AS sales#141, cast(sum(returns#45)#139 as decimal(38,2)) AS returns#142, cast(sum(profit#46)#140 as decimal(38,2)) AS profit#143] (110) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Output [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] (111) HashAggregate [codegen id : 64] -Input [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Input [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#144, sum(returns#45)#145, sum(profit#46)#146] -Results [4]: [channel#42, sum(sales#44)#144 AS sales#147, sum(returns#45)#145 AS returns#148, sum(profit#46)#146 AS profit#149] +Aggregate Attributes [3]: [sum(sales#44)#150, sum(returns#45)#151, sum(profit#46)#152] +Results [4]: [channel#42, sum(sales#44)#150 AS sales#153, sum(returns#45)#151 AS returns#154, sum(profit#46)#152 AS profit#155] (112) HashAggregate [codegen id : 64] -Input [4]: [channel#42, sales#147, returns#148, profit#149] +Input [4]: [channel#42, sales#153, returns#154, profit#155] Keys [1]: [channel#42] -Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] -Aggregate Attributes [6]: [sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] -Results [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +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#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] (113) Exchange -Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] -Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#162] +Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#168] (114) HashAggregate [codegen id : 65] -Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] Keys [1]: [channel#42] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#163, sum(returns#148)#164, sum(profit#149)#165] -Results [5]: [channel#42, null AS id#166, sum(sales#147)#163 AS sales#167, sum(returns#148)#164 AS returns#168, sum(profit#149)#165 AS profit#169] +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#42, null AS id#172, sum(sales#153)#169 AS sales#173, sum(returns#154)#170 AS returns#174, sum(profit#155)#171 AS profit#175] (115) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Output [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] (116) HashAggregate [codegen id : 97] -Input [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Input [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#176, sum(returns#45)#177, sum(profit#46)#178] -Results [3]: [sum(sales#44)#176 AS sales#147, sum(returns#45)#177 AS returns#148, sum(profit#46)#178 AS profit#149] +Aggregate Attributes [3]: [sum(sales#44)#182, sum(returns#45)#183, sum(profit#46)#184] +Results [3]: [sum(sales#44)#182 AS sales#153, sum(returns#45)#183 AS returns#154, sum(profit#46)#184 AS profit#155] (117) HashAggregate [codegen id : 97] -Input [3]: [sales#147, returns#148, profit#149] +Input [3]: [sales#153, returns#154, profit#155] Keys: [] -Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] -Aggregate Attributes [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] -Results [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Functions [3]: [partial_sum(sales#153), partial_sum(returns#154), partial_sum(profit#155)] +Aggregate Attributes [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Results [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] (118) Exchange -Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#191] +Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#197] (119) HashAggregate [codegen id : 98] -Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] Keys: [] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#192, sum(returns#148)#193, sum(profit#149)#194] -Results [5]: [null AS channel#195, null AS id#196, sum(sales#147)#192 AS sales#197, sum(returns#148)#193 AS returns#198, sum(profit#149)#194 AS profit#199] +Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] +Aggregate Attributes [3]: [sum(sales#153)#198, sum(returns#154)#199, sum(profit#155)#200] +Results [5]: [null AS channel#201, null AS id#202, sum(sales#153)#198 AS sales#203, sum(returns#154)#199 AS returns#204, sum(profit#155)#200 AS profit#205] (120) Union (121) HashAggregate [codegen id : 99] -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] (122) Exchange -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Arguments: hashpartitioning(channel#42, id#43, sales#135, returns#136, profit#137, 5), ENSURE_REQUIREMENTS, [id=#200] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Arguments: hashpartitioning(channel#42, id#43, sales#141, returns#142, profit#143, 5), ENSURE_REQUIREMENTS, [id=#206] (123) HashAggregate [codegen id : 100] -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] (124) TakeOrderedAndProject -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#141, returns#142, profit#143] ===== Subqueries ===== @@ -710,6 +710,6 @@ Output [1]: [d_date_sk#16] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 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 3333d5d1aed52..64f6270e81183 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 @@ -131,60 +131,60 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#15] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#15,17,2) as decimal(27,2)) AS total_sum#16, i_category#10, i_class#9, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] (20) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#10, i_class#9, sum#20] +Output [3]: [i_category#20, i_class#21, sum#22] (21) HashAggregate [codegen id : 8] -Input [3]: [i_category#10, i_class#9, sum#20] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#21] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#21,17,2) AS total_sum#22, i_category#10] +Input [3]: [i_category#20, i_class#21, sum#22] +Keys [2]: [i_category#20, i_class#21] +Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#23))#24] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#23))#24,17,2) AS total_sum#25, i_category#20] (22) HashAggregate [codegen id : 8] -Input [2]: [total_sum#22, i_category#10] -Keys [1]: [i_category#10] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [3]: [i_category#10, sum#25, isEmpty#26] +Input [2]: [total_sum#25, i_category#20] +Keys [1]: [i_category#20] +Functions [1]: [partial_sum(total_sum#25)] +Aggregate Attributes [2]: [sum#26, isEmpty#27] +Results [3]: [i_category#20, sum#28, isEmpty#29] (23) Exchange -Input [3]: [i_category#10, sum#25, isEmpty#26] -Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [i_category#20, sum#28, isEmpty#29] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#30] (24) HashAggregate [codegen id : 9] -Input [3]: [i_category#10, sum#25, isEmpty#26] -Keys [1]: [i_category#10] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#28] -Results [6]: [sum(total_sum#22)#28 AS total_sum#29, i_category#10, null AS i_class#30, 0 AS g_category#31, 1 AS g_class#32, 1 AS lochierarchy#33] +Input [3]: [i_category#20, sum#28, isEmpty#29] +Keys [1]: [i_category#20] +Functions [1]: [sum(total_sum#25)] +Aggregate Attributes [1]: [sum(total_sum#25)#31] +Results [6]: [sum(total_sum#25)#31 AS total_sum#32, i_category#20, null AS i_class#33, 0 AS g_category#34, 1 AS g_class#35, 1 AS lochierarchy#36] (25) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#10, i_class#9, sum#34] +Output [3]: [i_category#37, i_class#38, sum#39] (26) HashAggregate [codegen id : 13] -Input [3]: [i_category#10, i_class#9, sum#34] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#35] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#35,17,2) AS total_sum#22] +Input [3]: [i_category#37, i_class#38, sum#39] +Keys [2]: [i_category#37, i_class#38] +Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#40))#41] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#40))#41,17,2) AS total_sum#25] (27) HashAggregate [codegen id : 13] -Input [1]: [total_sum#22] +Input [1]: [total_sum#25] Keys: [] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [2]: [sum#38, isEmpty#39] +Functions [1]: [partial_sum(total_sum#25)] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [2]: [sum#44, isEmpty#45] (28) Exchange -Input [2]: [sum#38, isEmpty#39] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] +Input [2]: [sum#44, isEmpty#45] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#46] (29) HashAggregate [codegen id : 14] -Input [2]: [sum#38, isEmpty#39] +Input [2]: [sum#44, isEmpty#45] Keys: [] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#41] -Results [6]: [sum(total_sum#22)#41 AS total_sum#42, null AS i_category#43, null AS i_class#44, 1 AS g_category#45, 1 AS g_class#46, 2 AS lochierarchy#47] +Functions [1]: [sum(total_sum#25)] +Aggregate Attributes [1]: [sum(total_sum#25)#47] +Results [6]: [sum(total_sum#25)#47 AS total_sum#48, null AS i_category#49, null AS i_class#50, 1 AS g_category#51, 1 AS g_class#52, 2 AS lochierarchy#53] (30) Union @@ -197,34 +197,34 @@ Results [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, (32) Exchange Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] -Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#48] +Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#54] (33) HashAggregate [codegen id : 16] Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Keys [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#49] +Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#55] (34) Exchange -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: hashpartitioning(lochierarchy#19, _w0#49, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: hashpartitioning(lochierarchy#19, _w0#55, 5), ENSURE_REQUIREMENTS, [id=#56] (35) Sort [codegen id : 17] -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#49 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 (36) Window -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#49, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#51], [lochierarchy#19, _w0#49], [total_sum#16 DESC NULLS LAST] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#55, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#57], [lochierarchy#19, _w0#55], [total_sum#16 DESC NULLS LAST] (37) Project [codegen id : 18] -Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] -Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49, rank_within_parent#51] +Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] +Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55, rank_within_parent#57] (38) TakeOrderedAndProject -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#51 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] +Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#57 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] ===== Subqueries ===== 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 3333d5d1aed52..64f6270e81183 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 @@ -131,60 +131,60 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#15] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#15,17,2) as decimal(27,2)) AS total_sum#16, i_category#10, i_class#9, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] (20) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#10, i_class#9, sum#20] +Output [3]: [i_category#20, i_class#21, sum#22] (21) HashAggregate [codegen id : 8] -Input [3]: [i_category#10, i_class#9, sum#20] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#21] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#21,17,2) AS total_sum#22, i_category#10] +Input [3]: [i_category#20, i_class#21, sum#22] +Keys [2]: [i_category#20, i_class#21] +Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#23))#24] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#23))#24,17,2) AS total_sum#25, i_category#20] (22) HashAggregate [codegen id : 8] -Input [2]: [total_sum#22, i_category#10] -Keys [1]: [i_category#10] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [3]: [i_category#10, sum#25, isEmpty#26] +Input [2]: [total_sum#25, i_category#20] +Keys [1]: [i_category#20] +Functions [1]: [partial_sum(total_sum#25)] +Aggregate Attributes [2]: [sum#26, isEmpty#27] +Results [3]: [i_category#20, sum#28, isEmpty#29] (23) Exchange -Input [3]: [i_category#10, sum#25, isEmpty#26] -Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [i_category#20, sum#28, isEmpty#29] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#30] (24) HashAggregate [codegen id : 9] -Input [3]: [i_category#10, sum#25, isEmpty#26] -Keys [1]: [i_category#10] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#28] -Results [6]: [sum(total_sum#22)#28 AS total_sum#29, i_category#10, null AS i_class#30, 0 AS g_category#31, 1 AS g_class#32, 1 AS lochierarchy#33] +Input [3]: [i_category#20, sum#28, isEmpty#29] +Keys [1]: [i_category#20] +Functions [1]: [sum(total_sum#25)] +Aggregate Attributes [1]: [sum(total_sum#25)#31] +Results [6]: [sum(total_sum#25)#31 AS total_sum#32, i_category#20, null AS i_class#33, 0 AS g_category#34, 1 AS g_class#35, 1 AS lochierarchy#36] (25) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#10, i_class#9, sum#34] +Output [3]: [i_category#37, i_class#38, sum#39] (26) HashAggregate [codegen id : 13] -Input [3]: [i_category#10, i_class#9, sum#34] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#35] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#35,17,2) AS total_sum#22] +Input [3]: [i_category#37, i_class#38, sum#39] +Keys [2]: [i_category#37, i_class#38] +Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#40))#41] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#40))#41,17,2) AS total_sum#25] (27) HashAggregate [codegen id : 13] -Input [1]: [total_sum#22] +Input [1]: [total_sum#25] Keys: [] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [2]: [sum#38, isEmpty#39] +Functions [1]: [partial_sum(total_sum#25)] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [2]: [sum#44, isEmpty#45] (28) Exchange -Input [2]: [sum#38, isEmpty#39] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] +Input [2]: [sum#44, isEmpty#45] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#46] (29) HashAggregate [codegen id : 14] -Input [2]: [sum#38, isEmpty#39] +Input [2]: [sum#44, isEmpty#45] Keys: [] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#41] -Results [6]: [sum(total_sum#22)#41 AS total_sum#42, null AS i_category#43, null AS i_class#44, 1 AS g_category#45, 1 AS g_class#46, 2 AS lochierarchy#47] +Functions [1]: [sum(total_sum#25)] +Aggregate Attributes [1]: [sum(total_sum#25)#47] +Results [6]: [sum(total_sum#25)#47 AS total_sum#48, null AS i_category#49, null AS i_class#50, 1 AS g_category#51, 1 AS g_class#52, 2 AS lochierarchy#53] (30) Union @@ -197,34 +197,34 @@ Results [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, (32) Exchange Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] -Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#48] +Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#54] (33) HashAggregate [codegen id : 16] Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Keys [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#49] +Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#55] (34) Exchange -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: hashpartitioning(lochierarchy#19, _w0#49, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: hashpartitioning(lochierarchy#19, _w0#55, 5), ENSURE_REQUIREMENTS, [id=#56] (35) Sort [codegen id : 17] -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#49 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 (36) Window -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#49, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#51], [lochierarchy#19, _w0#49], [total_sum#16 DESC NULLS LAST] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#55, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#57], [lochierarchy#19, _w0#55], [total_sum#16 DESC NULLS LAST] (37) Project [codegen id : 18] -Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] -Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49, rank_within_parent#51] +Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] +Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55, rank_within_parent#57] (38) TakeOrderedAndProject -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#51 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] +Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#57 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt index b036ff059dbb2..89bb3e5d551a8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt @@ -56,7 +56,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -87,7 +87,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -95,7 +95,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt index 920a4c2a55a26..b7c5f4081a1f3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt @@ -45,7 +45,7 @@ Condition : isnotnull(ss_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -53,7 +53,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -80,7 +80,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] From 1dbba32ab5be0743a2e11fef493e8911e3092947 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 23 Mar 2021 16:57:03 +0800 Subject: [PATCH 18/38] fix SPARK-22748 --- .../sql/catalyst/analysis/AnalysisSuite.scala | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) 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 1a694f81f8119..1410c634aafe1 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 @@ -1051,7 +1051,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("SPARK-22748: Analyze __grouping__id as a literal function") { - assertAnalysisSuccess(EliminateSubqueryAliases(parsePlan( + assertAnalysisSuccess(parsePlan( """ |SELECT grouping__id FROM ( | SELECT grouping__id FROM ( @@ -1059,61 +1059,61 @@ class AnalysisSuite extends AnalysisTest with Matchers { | GROUP BY a, b WITH ROLLUP | ) |) - """.stripMargin)), false) + """.stripMargin), false) - assertAnalysisSuccess(EliminateSubqueryAliases(parsePlan( + assertAnalysisSuccess(parsePlan( """ |SELECT grouping__id FROM ( | SELECT a, b, count(1), grouping__id FROM TaBlE2 | GROUP BY a, b WITH CUBE |) - """.stripMargin)), false) + """.stripMargin), false) - assertAnalysisSuccess(EliminateSubqueryAliases(parsePlan( + assertAnalysisSuccess(parsePlan( """ |SELECT grouping__id FROM ( | SELECT a, b, count(1), grouping__id FROM TaBlE2 | GROUP BY a, b GROUPING SETS ((a, b), ()) |) - """.stripMargin)), false) + """.stripMargin), false) - assertAnalysisSuccess(EliminateSubqueryAliases(parsePlan( + assertAnalysisSuccess(parsePlan( """ |SELECT a, b, count(1) FROM TaBlE2 | GROUP BY CUBE(a, b) HAVING grouping__id > 0 - """.stripMargin)), false) + """.stripMargin), false) - assertAnalysisSuccess(EliminateSubqueryAliases(parsePlan( + assertAnalysisSuccess(parsePlan( """ |SELECT * FROM ( - | SELECT a, b, count(1) FROM TaBlE2 + | SELECT a, b, count(1), grouping__id FROM TaBlE2 | GROUP BY a, b GROUPING SETS ((a, b), ()) |) WHERE grouping__id > 0 - """.stripMargin)), false) + """.stripMargin), false) - assertAnalysisSuccess(EliminateSubqueryAliases(parsePlan( + assertAnalysisSuccess(parsePlan( """ |SELECT * FROM ( - | SELECT a, b, count(1) FROM TaBlE2 + | SELECT a, b, count(1), grouping__id FROM TaBlE2 | GROUP BY a, b GROUPING SETS ((a, b), ()) |) ORDER BY grouping__id > 0 - """.stripMargin)), false) + """.stripMargin), false) - assertAnalysisSuccess(EliminateSubqueryAliases(parsePlan( + assertAnalysisSuccess(parsePlan( """ |SELECT a, b, count(1) FROM TaBlE2 | GROUP BY a, b GROUPING SETS ((a, b), ()) | ORDER BY grouping__id > 0 - """.stripMargin)), false) + """.stripMargin), false) - assertAnalysisError(EliminateSubqueryAliases(parsePlan( + assertAnalysisError(parsePlan( """ |SELECT grouping__id FROM ( | SELECT a, b, count(1), grouping__id FROM TaBlE2 | GROUP BY a, b |) - """.stripMargin)), + """.stripMargin), Seq("grouping_id() can only be used with GroupingSets/Cube/Rollup"), false) } From 0a41d26f59906042b974643b498708693492cff9 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 23 Mar 2021 17:09:27 +0800 Subject: [PATCH 19/38] gen golden files --- .../approved-plans-v1_4/q44.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q44/explain.txt | 4 +- .../approved-plans-v1_4/q47.sf100/explain.txt | 83 +---- .../approved-plans-v1_4/q47/explain.txt | 75 +--- .../approved-plans-v1_4/q57.sf100/explain.txt | 83 +---- .../approved-plans-v1_4/q57/explain.txt | 75 +--- .../approved-plans-v1_4/q67.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q67/explain.txt | 2 +- .../approved-plans-v1_4/q70.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q70/explain.txt | 2 +- .../approved-plans-v2_7/q47.sf100/explain.txt | 83 +---- .../approved-plans-v2_7/q47/explain.txt | 75 +--- .../q51a.sf100/explain.txt | 345 +++--------------- .../approved-plans-v2_7/q51a/explain.txt | 340 +++-------------- .../approved-plans-v2_7/q57.sf100/explain.txt | 83 +---- .../approved-plans-v2_7/q57/explain.txt | 75 +--- .../q67a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q67a/explain.txt | 2 +- .../q70a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q70a/explain.txt | 2 +- 20 files changed, 178 insertions(+), 1165 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt index a6270c8ab533a..84a8547e3f6ef 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt @@ -95,7 +95,7 @@ Arguments: [rank(rank_col#12) windowspecdefinition(rank_col#12 ASC NULLS FIRST, (13) Filter [codegen id : 10] Input [3]: [item_sk#11, rank_col#12, rnk#17] -Condition : ((isnotnull(rnk#17) AND (rnk#17 < 11)) AND isnotnull(item_sk#11)) +Condition : ((rnk#17 < 11) AND isnotnull(item_sk#11)) (14) Project [codegen id : 10] Output [2]: [item_sk#11, rnk#17] @@ -133,7 +133,7 @@ Arguments: [rank(rank_col#24) windowspecdefinition(rank_col#24 DESC NULLS LAST, (22) Filter [codegen id : 7] Input [3]: [item_sk#23, rank_col#24, rnk#27] -Condition : ((isnotnull(rnk#27) AND (rnk#27 < 11)) AND isnotnull(item_sk#23)) +Condition : ((rnk#27 < 11) AND isnotnull(item_sk#23)) (23) Project [codegen id : 7] Output [2]: [item_sk#23, rnk#27] @@ -141,7 +141,7 @@ Input [3]: [item_sk#23, rank_col#24, rnk#27] (24) BroadcastExchange Input [2]: [item_sk#23, rnk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#28] (25) BroadcastHashJoin [codegen id : 10] Left keys [1]: [rnk#17] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index a220cf9ff546f..b42ddd8db7836 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -96,7 +96,7 @@ Arguments: [rank(rank_col#12) windowspecdefinition(rank_col#12 ASC NULLS FIRST, (13) Filter [codegen id : 4] Input [3]: [item_sk#11, rank_col#12, rnk#17] -Condition : ((isnotnull(rnk#17) AND (rnk#17 < 11)) AND isnotnull(item_sk#11)) +Condition : ((rnk#17 < 11) AND isnotnull(item_sk#11)) (14) Project [codegen id : 4] Output [2]: [item_sk#11, rnk#17] @@ -138,7 +138,7 @@ Arguments: [rank(rank_col#24) windowspecdefinition(rank_col#24 DESC NULLS LAST, (23) Filter [codegen id : 8] Input [3]: [item_sk#23, rank_col#24, rnk#27] -Condition : ((isnotnull(rnk#27) AND (rnk#27 < 11)) AND isnotnull(item_sk#23)) +Condition : ((rnk#27 < 11) AND isnotnull(item_sk#23)) (24) Project [codegen id : 8] Output [2]: [item_sk#23, rnk#27] 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 3a356e35707f7..b7ffba01f92c6 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 @@ -213,11 +213,7 @@ Arguments: [rank(d_year#7, d_moy#8) windowspecdefinition(i_category#17, i_brand# (35) Filter [codegen id : 12] Input [9]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28] -<<<<<<< HEAD Condition : ((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#26)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#26 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) -======= -Condition : (((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#26)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#26 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#28)) ->>>>>>> regen (36) Exchange Input [9]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28] @@ -249,123 +245,62 @@ Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_n Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] -<<<<<<< HEAD (43) Project [codegen id : 23] -Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38, rn#40] - -(44) Exchange -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1), 5), ENSURE_REQUIREMENTS, [id=#41] - -(45) Sort [codegen id : 24] -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#40 + 1) ASC NULLS FIRST], false, 0 -======= -(43) Filter [codegen id : 23] -Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] -Condition : isnotnull(rn#41) - -(44) Project [codegen id : 23] Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] -(45) Exchange +(44) Exchange Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1), 5), ENSURE_REQUIREMENTS, [id=#42] -(46) Sort [codegen id : 24] +(45) Sort [codegen id : 24] Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#41 + 1) ASC NULLS FIRST], false, 0 ->>>>>>> regen (46) SortMergeJoin [codegen id : 25] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1)] Join condition: None -<<<<<<< HEAD (47) Project [codegen id : 25] -Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38] -Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] - -(48) ReusedExchange [Reuses operator id: 40] -Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] - -(49) Sort [codegen id : 34] -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 - -(50) Window -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] - -(51) Project [codegen id : 35] -Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] - -(52) Exchange -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1), 5), ENSURE_REQUIREMENTS, [id=#50] - -(53) Sort [codegen id : 36] -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (rn#49 - 1) ASC NULLS FIRST], false, 0 -======= -(48) Project [codegen id : 25] Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39] Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] -(49) ReusedExchange [Reuses operator id: 40] +(48) ReusedExchange [Reuses operator id: 40] Output [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] -(50) Sort [codegen id : 34] +(49) Sort [codegen id : 34] Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST], false, 0 -(51) Window +(50) Window Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] Arguments: [rank(d_year#47, d_moy#48) windowspecdefinition(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#50], [i_category#43, i_brand#44, s_store_name#45, s_company_name#46], [d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST] -(52) Filter [codegen id : 35] -Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] -Condition : isnotnull(rn#50) - -(53) Project [codegen id : 35] +(51) Project [codegen id : 35] Output [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] -(54) Exchange +(52) Exchange Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] Arguments: hashpartitioning(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1), 5), ENSURE_REQUIREMENTS, [id=#51] -(55) Sort [codegen id : 36] +(53) Sort [codegen id : 36] Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, (rn#50 - 1) ASC NULLS FIRST], false, 0 ->>>>>>> regen (54) SortMergeJoin [codegen id : 37] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] Right keys [5]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1)] Join condition: None -<<<<<<< HEAD (55) Project [codegen id : 37] -Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#38 AS psum#51, sum_sales#48 AS nsum#52] -Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] - -(56) TakeOrderedAndProject -Input [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#11 ASC NULLS FIRST], [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] -======= -(57) Project [codegen id : 37] Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#39 AS psum#52, sum_sales#49 AS nsum#53] Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39, i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] -(58) TakeOrderedAndProject +(56) TakeOrderedAndProject Input [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#11 ASC NULLS FIRST], [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] ->>>>>>> regen ===== Subqueries ===== 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 4a96c307dff2f..917b21c247449 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 @@ -194,11 +194,7 @@ Arguments: [rank(d_year#11, d_moy#12) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [9]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27] -<<<<<<< HEAD Condition : ((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#25)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) -======= -Condition : (((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#25)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#27)) ->>>>>>> regen (33) ReusedExchange [Reuses operator id: unknown] Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34] @@ -222,107 +218,54 @@ Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_n 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] -<<<<<<< HEAD (38) Project [codegen id : 15] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] -Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] - -(39) BroadcastExchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] -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=#39] -======= -(38) Filter [codegen id : 15] -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] -Condition : isnotnull(rn#39) - -(39) Project [codegen id : 15] 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] -(40) BroadcastExchange +(39) BroadcastExchange Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] + 1)),false), [id=#40] ->>>>>>> regen +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=#40] (40) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] Join condition: None -<<<<<<< HEAD (41) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] -Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] - -(42) ReusedExchange [Reuses operator id: 35] -Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] - -(43) Sort [codegen id : 21] -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#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, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 - -(44) Window -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] -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#47], [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] - -(45) Project [codegen id : 22] -Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] -Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] - -(46) BroadcastExchange -Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] -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=#48] -======= -(42) Project [codegen id : 23] Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -(43) ReusedExchange [Reuses operator id: 35] +(42) ReusedExchange [Reuses operator id: 35] 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] -(44) Sort [codegen id : 21] +(43) Sort [codegen id : 21] 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 -(45) Window +(44) 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] -(46) Filter [codegen id : 22] -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] -Condition : isnotnull(rn#48) - -(47) Project [codegen id : 22] +(45) Project [codegen id : 22] 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] -(48) BroadcastExchange +(46) BroadcastExchange Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] - 1)),false), [id=#49] ->>>>>>> regen +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=#49] (47) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] Join condition: None -<<<<<<< HEAD (48) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#49, sum_sales#46 AS nsum#50] -Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] - -(49) TakeOrderedAndProject -Input [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] -======= -(50) Project [codegen id : 23] Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -(51) TakeOrderedAndProject +(49) TakeOrderedAndProject Input [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, psum#50, nsum#51] ->>>>>>> regen ===== Subqueries ===== 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 173125b197079..6e39745703215 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 @@ -213,11 +213,7 @@ Arguments: [rank(d_year#7, d_moy#8) windowspecdefinition(i_category#16, i_brand# (35) Filter [codegen id : 12] Input [8]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27] -<<<<<<< HEAD Condition : ((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#25)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) -======= -Condition : (((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#25)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#27)) ->>>>>>> regen (36) Exchange Input [8]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27] @@ -249,123 +245,62 @@ Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#3 Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] -<<<<<<< HEAD (43) Project [codegen id : 23] -Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] - -(44) Exchange -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#38 + 1), 5), ENSURE_REQUIREMENTS, [id=#39] - -(45) Sort [codegen id : 24] -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#38 + 1) ASC NULLS FIRST], false, 0 -======= -(43) Filter [codegen id : 23] -Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] -Condition : isnotnull(rn#39) - -(44) Project [codegen id : 23] Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] -(45) Exchange +(44) Exchange Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] -(46) Sort [codegen id : 24] +(45) Sort [codegen id : 24] Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 ->>>>>>> regen (46) SortMergeJoin [codegen id : 25] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#39 + 1)] Join condition: None -<<<<<<< HEAD (47) Project [codegen id : 25] -Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] -Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] - -(48) ReusedExchange [Reuses operator id: 40] -Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] - -(49) Sort [codegen id : 34] -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 - -(50) Window -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] - -(51) Project [codegen id : 35] -Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] - -(52) Exchange -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (rn#46 - 1), 5), ENSURE_REQUIREMENTS, [id=#47] - -(53) Sort [codegen id : 36] -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 -======= -(48) Project [codegen id : 25] Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] -(49) ReusedExchange [Reuses operator id: 40] +(48) ReusedExchange [Reuses operator id: 40] Output [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] -(50) Sort [codegen id : 34] +(49) Sort [codegen id : 34] Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 -(51) Window +(50) Window Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#41, i_brand#42, cc_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#41, i_brand#42, cc_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] -(52) Filter [codegen id : 35] -Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] -Condition : isnotnull(rn#47) - -(53) Project [codegen id : 35] +(51) Project [codegen id : 35] Output [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] -(54) Exchange +(52) Exchange Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] Arguments: hashpartitioning(i_category#41, i_brand#42, cc_name#43, (rn#47 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] -(55) Sort [codegen id : 36] +(53) Sort [codegen id : 36] Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, (rn#47 - 1) ASC NULLS FIRST], false, 0 ->>>>>>> regen (54) SortMergeJoin [codegen id : 37] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] Right keys [4]: [i_category#41, i_brand#42, cc_name#43, (rn#47 - 1)] Join condition: None -<<<<<<< HEAD (55) Project [codegen id : 37] -Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#48, sum_sales#45 AS nsum#49] -Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] - -(56) TakeOrderedAndProject -Input [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] -======= -(57) Project [codegen id : 37] Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#49, sum_sales#46 AS nsum#50] Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] -(58) TakeOrderedAndProject +(56) TakeOrderedAndProject Input [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] ->>>>>>> regen ===== Subqueries ===== 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 d5b269b02d199..fa24f5ede5453 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 @@ -194,11 +194,7 @@ Arguments: [rank(d_year#11, d_moy#12) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [8]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26] -<<<<<<< HEAD Condition : ((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#24)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) -======= -Condition : (((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#24)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#26)) ->>>>>>> regen (33) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] @@ -222,107 +218,54 @@ Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#2 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] -<<<<<<< HEAD (38) Project [codegen id : 15] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] -Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34, rn#36] - -(39) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#37] -======= -(38) Filter [codegen id : 15] -Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] -Condition : isnotnull(rn#37) - -(39) Project [codegen id : 15] 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] -(40) BroadcastExchange +(39) BroadcastExchange Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] + 1)),false), [id=#38] ->>>>>>> regen +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#38] (40) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] Join condition: None -<<<<<<< HEAD (41) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34] -Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] - -(42) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] - -(43) Sort [codegen id : 21] -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] -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 - -(44) Window -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] -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#44], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] - -(45) Project [codegen id : 22] -Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] -Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43, rn#44] - -(46) BroadcastExchange -Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#45] -======= -(42) Project [codegen id : 23] Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35] Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -(43) ReusedExchange [Reuses operator id: 35] +(42) ReusedExchange [Reuses operator id: 35] Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] -(44) Sort [codegen id : 21] +(43) Sort [codegen id : 21] 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 -(45) Window +(44) 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] -(46) Filter [codegen id : 22] -Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] -Condition : isnotnull(rn#45) - -(47) Project [codegen id : 22] +(45) Project [codegen id : 22] 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] -(48) BroadcastExchange +(46) BroadcastExchange Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] - 1)),false), [id=#46] ->>>>>>> regen +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#46] (47) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] Join condition: None -<<<<<<< HEAD (48) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#34 AS psum#46, sum_sales#43 AS nsum#47] -Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34, i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] - -(49) TakeOrderedAndProject -Input [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, 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#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, psum#46, nsum#47] -======= -(50) Project [codegen id : 23] Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -(51) TakeOrderedAndProject +(49) TakeOrderedAndProject Input [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, 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#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, psum#47, nsum#48] ->>>>>>> regen ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt index 075fd1864222c..3a19c3923737d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt @@ -183,7 +183,7 @@ Arguments: [rank(sumsales#38) windowspecdefinition(i_category#23, sumsales#38 DE (33) Filter [codegen id : 10] Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#38, rk#40] -Condition : (isnotnull(rk#40) AND (rk#40 <= 100)) +Condition : (rk#40 <= 100) (34) TakeOrderedAndProject Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#38, rk#40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt index ba71667cfebe5..3d42f49415640 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt @@ -168,7 +168,7 @@ Arguments: [rank(sumsales#37) windowspecdefinition(i_category#22, sumsales#37 DE (30) Filter [codegen id : 7] Input [10]: [i_category#22, i_class#23, i_brand#24, i_product_name#25, d_year#26, d_qoy#27, d_moy#28, s_store_id#29, sumsales#37, rk#39] -Condition : (isnotnull(rk#39) AND (rk#39 <= 100)) +Condition : (rk#39 <= 100) (31) TakeOrderedAndProject Input [10]: [i_category#22, i_class#23, i_brand#24, i_product_name#25, d_year#26, d_qoy#27, d_moy#28, s_store_id#29, sumsales#37, rk#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index ddce6bf2e14f4..7b8445a46ac54 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -194,7 +194,7 @@ Arguments: [rank(_w2#23) windowspecdefinition(s_state#16, _w2#23 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] -Condition : (isnotnull(ranking#25) AND (ranking#25 <= 5)) +Condition : (ranking#25 <= 5) (33) Project [codegen id : 7] Output [1]: [s_state#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index dbdfc72f92624..91cfba59bb5bc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -194,7 +194,7 @@ Arguments: [rank(_w2#23) windowspecdefinition(s_state#15, _w2#23 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] -Condition : (isnotnull(ranking#25) AND (ranking#25 <= 5)) +Condition : (ranking#25 <= 5) (33) Project [codegen id : 7] Output [1]: [s_state#22] 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 20d80b4503755..636fa32f94212 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 @@ -213,11 +213,7 @@ Arguments: [rank(d_year#7, d_moy#8) windowspecdefinition(i_category#17, i_brand# (35) Filter [codegen id : 12] Input [9]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28] -<<<<<<< HEAD Condition : ((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#26)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#26 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) -======= -Condition : (((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#26)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#26 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#28)) ->>>>>>> regen (36) Exchange Input [9]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28] @@ -249,123 +245,62 @@ Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_n Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] -<<<<<<< HEAD (43) Project [codegen id : 23] -Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38, rn#40] - -(44) Exchange -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1), 5), ENSURE_REQUIREMENTS, [id=#41] - -(45) Sort [codegen id : 24] -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#40 + 1) ASC NULLS FIRST], false, 0 -======= -(43) Filter [codegen id : 23] -Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] -Condition : isnotnull(rn#41) - -(44) Project [codegen id : 23] Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] -(45) Exchange +(44) Exchange Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1), 5), ENSURE_REQUIREMENTS, [id=#42] -(46) Sort [codegen id : 24] +(45) Sort [codegen id : 24] Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#41 + 1) ASC NULLS FIRST], false, 0 ->>>>>>> regen (46) SortMergeJoin [codegen id : 25] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1)] Join condition: None -<<<<<<< HEAD (47) Project [codegen id : 25] -Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38] -Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] - -(48) ReusedExchange [Reuses operator id: 40] -Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] - -(49) Sort [codegen id : 34] -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 - -(50) Window -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] - -(51) Project [codegen id : 35] -Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] - -(52) Exchange -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1), 5), ENSURE_REQUIREMENTS, [id=#50] - -(53) Sort [codegen id : 36] -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (rn#49 - 1) ASC NULLS FIRST], false, 0 -======= -(48) Project [codegen id : 25] Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39] Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] -(49) ReusedExchange [Reuses operator id: 40] +(48) ReusedExchange [Reuses operator id: 40] Output [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] -(50) Sort [codegen id : 34] +(49) Sort [codegen id : 34] Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST], false, 0 -(51) Window +(50) Window Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] Arguments: [rank(d_year#47, d_moy#48) windowspecdefinition(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#50], [i_category#43, i_brand#44, s_store_name#45, s_company_name#46], [d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST] -(52) Filter [codegen id : 35] -Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] -Condition : isnotnull(rn#50) - -(53) Project [codegen id : 35] +(51) Project [codegen id : 35] Output [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] -(54) Exchange +(52) Exchange Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] Arguments: hashpartitioning(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1), 5), ENSURE_REQUIREMENTS, [id=#51] -(55) Sort [codegen id : 36] +(53) Sort [codegen id : 36] Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, (rn#50 - 1) ASC NULLS FIRST], false, 0 ->>>>>>> regen (54) SortMergeJoin [codegen id : 37] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] Right keys [5]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1)] Join condition: None -<<<<<<< HEAD (55) Project [codegen id : 37] -Output [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#38 AS psum#51, sum_sales#48 AS nsum#52] -Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] - -(56) TakeOrderedAndProject -Input [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] -======= -(57) Project [codegen id : 37] Output [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#39 AS psum#52, sum_sales#49 AS nsum#53] Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39, i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] -(58) TakeOrderedAndProject +(56) TakeOrderedAndProject Input [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] ->>>>>>> regen ===== Subqueries ===== 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 9017cd21a3873..734eec1be3565 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 @@ -194,11 +194,7 @@ Arguments: [rank(d_year#11, d_moy#12) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [9]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27] -<<<<<<< HEAD Condition : ((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#25)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) -======= -Condition : (((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#25)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#27)) ->>>>>>> regen (33) ReusedExchange [Reuses operator id: unknown] Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34] @@ -222,107 +218,54 @@ Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_n 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] -<<<<<<< HEAD (38) Project [codegen id : 15] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] -Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] - -(39) BroadcastExchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] -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=#39] -======= -(38) Filter [codegen id : 15] -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] -Condition : isnotnull(rn#39) - -(39) Project [codegen id : 15] 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] -(40) BroadcastExchange +(39) BroadcastExchange Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] + 1)),false), [id=#40] ->>>>>>> regen +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=#40] (40) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] Join condition: None -<<<<<<< HEAD (41) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] -Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] - -(42) ReusedExchange [Reuses operator id: 35] -Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] - -(43) Sort [codegen id : 21] -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#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, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 - -(44) Window -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] -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#47], [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] - -(45) Project [codegen id : 22] -Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] -Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] - -(46) BroadcastExchange -Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] -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=#48] -======= -(42) Project [codegen id : 23] Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -(43) ReusedExchange [Reuses operator id: 35] +(42) ReusedExchange [Reuses operator id: 35] 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] -(44) Sort [codegen id : 21] +(43) Sort [codegen id : 21] 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 -(45) Window +(44) 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] -(46) Filter [codegen id : 22] -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] -Condition : isnotnull(rn#48) - -(47) Project [codegen id : 22] +(45) Project [codegen id : 22] 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] -(48) BroadcastExchange +(46) BroadcastExchange Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] - 1)),false), [id=#49] ->>>>>>> regen +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=#49] (47) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] Join condition: None -<<<<<<< HEAD (48) Project [codegen id : 23] -Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#49, sum_sales#46 AS nsum#50] -Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] - -(49) TakeOrderedAndProject -Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, 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#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#22, psum#49, nsum#50] -======= -(50) Project [codegen id : 23] Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -(51) TakeOrderedAndProject +(49) TakeOrderedAndProject Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, 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#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#22, psum#50, nsum#51] ->>>>>>> regen ===== Subqueries ===== 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 957e2d9a015b4..0a8fdb66edb77 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 @@ -163,55 +163,27 @@ Arguments: hashpartitioning(item_sk#13, 5), ENSURE_REQUIREMENTS, [id=#17] Input [4]: [item_sk#13, d_date#6, sumws#14, rk#16] Arguments: [item_sk#13 ASC NULLS FIRST], false, 0 -<<<<<<< HEAD (20) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] - -(21) Sort [codegen id : 10] -Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] -Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST], false, 0 - -(22) Window -Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#21], [ws_item_sk#1], [d_date#19 ASC NULLS FIRST] - -(23) Project [codegen id : 11] -Output [3]: [item_sk#18, sumws#20, rk#21] -Input [5]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1, rk#21] - -(24) Exchange -Input [3]: [item_sk#18, sumws#20, rk#21] -Arguments: hashpartitioning(item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#22] - -(25) Sort [codegen id : 12] -Input [3]: [item_sk#18, sumws#20, rk#21] -======= -(21) ReusedExchange [Reuses operator id: unknown] Output [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] -(22) Sort [codegen id : 10] +(21) Sort [codegen id : 10] Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] Arguments: [ws_item_sk#21 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST], false, 0 -(23) Window +(22) Window Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] Arguments: [row_number() windowspecdefinition(ws_item_sk#21, d_date#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#22], [ws_item_sk#21], [d_date#19 ASC NULLS FIRST] -(24) Filter [codegen id : 11] -Input [5]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21, rk#22] -Condition : isnotnull(rk#22) - -(25) Project [codegen id : 11] +(23) Project [codegen id : 11] Output [3]: [item_sk#18, sumws#20, rk#22] Input [5]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21, rk#22] -(26) Exchange +(24) Exchange Input [3]: [item_sk#18, sumws#20, rk#22] Arguments: hashpartitioning(item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#23] -(27) Sort [codegen id : 12] +(25) Sort [codegen id : 12] Input [3]: [item_sk#18, sumws#20, rk#22] ->>>>>>> regen Arguments: [item_sk#18 ASC NULLS FIRST], false, 0 (26) SortMergeJoin [codegen id : 13] @@ -230,433 +202,214 @@ Functions [1]: [partial_sum(sumws#20)] Aggregate Attributes [2]: [sum#24, isEmpty#25] Results [5]: [item_sk#13, d_date#6, sumws#14, sum#26, isEmpty#27] -<<<<<<< HEAD (29) HashAggregate [codegen id : 13] -Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] -======= -(31) HashAggregate [codegen id : 13] Input [5]: [item_sk#13, d_date#6, sumws#14, sum#26, isEmpty#27] ->>>>>>> regen Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [sum(sumws#20)] Aggregate Attributes [1]: [sum(sumws#20)#28] Results [3]: [item_sk#13, d_date#6, sum(sumws#20)#28 AS cume_sales#29] -<<<<<<< HEAD (30) Exchange -Input [3]: [item_sk#13, d_date#6, cume_sales#28] -Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#29] - -(31) Sort [codegen id : 14] -Input [3]: [item_sk#13, d_date#6, cume_sales#28] -Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 - -(32) Scan parquet default.store_sales -Output [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] -======= -(32) Exchange Input [3]: [item_sk#13, d_date#6, cume_sales#29] Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#30] -(33) Sort [codegen id : 14] +(31) Sort [codegen id : 14] Input [3]: [item_sk#13, d_date#6, cume_sales#29] Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 -(34) Scan parquet default.store_sales +(32) Scan parquet default.store_sales Output [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] ->>>>>>> regen Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#33), dynamicpruningexpression(ss_sold_date_sk#33 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -<<<<<<< HEAD (33) ColumnarToRow [codegen id : 16] -Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] - -(34) Filter [codegen id : 16] -Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] -Condition : isnotnull(ss_item_sk#30) - -(35) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#33, d_date#34] - -(36) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ss_sold_date_sk#32] -Right keys [1]: [d_date_sk#33] -Join condition: None - -(37) Project [codegen id : 16] -Output [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] -Input [5]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32, d_date_sk#33, d_date#34] - -(38) HashAggregate [codegen id : 16] -Input [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] -Keys [2]: [ss_item_sk#30, d_date#34] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum#35] -Results [3]: [ss_item_sk#30, d_date#34, sum#36] - -(39) Exchange -Input [3]: [ss_item_sk#30, d_date#34, sum#36] -Arguments: hashpartitioning(ss_item_sk#30, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#37] - -(40) HashAggregate [codegen id : 17] -Input [3]: [ss_item_sk#30, d_date#34, sum#36] -Keys [2]: [ss_item_sk#30, d_date#34] -Functions [1]: [sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#31))#38] -Results [4]: [ss_item_sk#30 AS item_sk#39, d_date#34, MakeDecimal(sum(UnscaledValue(ss_sales_price#31))#38,17,2) AS sumss#40, ss_item_sk#30] - -(41) Exchange -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: hashpartitioning(ss_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#41] - -(42) Sort [codegen id : 18] -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 - -(43) Window -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#42], [ss_item_sk#30], [d_date#34 ASC NULLS FIRST] - -(44) Project [codegen id : 19] -Output [4]: [item_sk#39, d_date#34, sumss#40, rk#42] -Input [5]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30, rk#42] - -(45) Exchange -Input [4]: [item_sk#39, d_date#34, sumss#40, rk#42] -Arguments: hashpartitioning(item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#43] - -(46) Sort [codegen id : 20] -Input [4]: [item_sk#39, d_date#34, sumss#40, rk#42] -Arguments: [item_sk#39 ASC NULLS FIRST], false, 0 - -(47) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30] - -(48) Sort [codegen id : 24] -Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30] -Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], false, 0 - -(49) Window -Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30] -Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#47], [ss_item_sk#30], [d_date#45 ASC NULLS FIRST] - -(50) Project [codegen id : 25] -Output [3]: [item_sk#44, sumss#46, rk#47] -Input [5]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30, rk#47] - -(51) Exchange -Input [3]: [item_sk#44, sumss#46, rk#47] -Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, [id=#48] - -(52) Sort [codegen id : 26] -Input [3]: [item_sk#44, sumss#46, rk#47] -Arguments: [item_sk#44 ASC NULLS FIRST], false, 0 - -(53) SortMergeJoin [codegen id : 27] -Left keys [1]: [item_sk#39] -Right keys [1]: [item_sk#44] -Join condition: (rk#42 >= rk#47) - -(54) Project [codegen id : 27] -Output [4]: [item_sk#39, d_date#34, sumss#40, sumss#46] -Input [7]: [item_sk#39, d_date#34, sumss#40, rk#42, item_sk#44, sumss#46, rk#47] - -(55) HashAggregate [codegen id : 27] -Input [4]: [item_sk#39, d_date#34, sumss#40, sumss#46] -Keys [3]: [item_sk#39, d_date#34, sumss#40] -Functions [1]: [partial_sum(sumss#46)] -Aggregate Attributes [2]: [sum#49, isEmpty#50] -Results [5]: [item_sk#39, d_date#34, sumss#40, sum#51, isEmpty#52] - -(56) HashAggregate [codegen id : 27] -Input [5]: [item_sk#39, d_date#34, sumss#40, sum#51, isEmpty#52] -Keys [3]: [item_sk#39, d_date#34, sumss#40] -Functions [1]: [sum(sumss#46)] -Aggregate Attributes [1]: [sum(sumss#46)#53] -Results [3]: [item_sk#39, d_date#34, sum(sumss#46)#53 AS cume_sales#54] - -(57) Exchange -Input [3]: [item_sk#39, d_date#34, cume_sales#54] -Arguments: hashpartitioning(item_sk#39, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#55] - -(58) Sort [codegen id : 28] -Input [3]: [item_sk#39, d_date#34, cume_sales#54] -Arguments: [item_sk#39 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 -======= -(35) ColumnarToRow [codegen id : 16] Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] -(36) Filter [codegen id : 16] +(34) Filter [codegen id : 16] Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] Condition : isnotnull(ss_item_sk#31) -(37) ReusedExchange [Reuses operator id: 8] +(35) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#34, d_date#35] -(38) BroadcastHashJoin [codegen id : 16] +(36) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_sold_date_sk#33] Right keys [1]: [d_date_sk#34] Join condition: None -(39) Project [codegen id : 16] +(37) Project [codegen id : 16] Output [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] Input [5]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33, d_date_sk#34, d_date#35] -(40) HashAggregate [codegen id : 16] +(38) HashAggregate [codegen id : 16] Input [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] Keys [2]: [ss_item_sk#31, d_date#35] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#32))] Aggregate Attributes [1]: [sum#36] Results [3]: [ss_item_sk#31, d_date#35, sum#37] -(41) Exchange +(39) Exchange Input [3]: [ss_item_sk#31, d_date#35, sum#37] Arguments: hashpartitioning(ss_item_sk#31, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#38] -(42) HashAggregate [codegen id : 17] +(40) HashAggregate [codegen id : 17] Input [3]: [ss_item_sk#31, d_date#35, sum#37] Keys [2]: [ss_item_sk#31, d_date#35] Functions [1]: [sum(UnscaledValue(ss_sales_price#32))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#32))#39] Results [4]: [ss_item_sk#31 AS item_sk#40, d_date#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#32))#39,17,2) AS sumss#41, ss_item_sk#31] -(43) Exchange +(41) Exchange Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#42] -(44) Sort [codegen id : 18] +(42) Sort [codegen id : 18] Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] Arguments: [ss_item_sk#31 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 -(45) Window +(43) Window Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] Arguments: [row_number() windowspecdefinition(ss_item_sk#31, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [ss_item_sk#31], [d_date#35 ASC NULLS FIRST] -(46) Filter [codegen id : 19] -Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] -Condition : isnotnull(rk#43) - -(47) Project [codegen id : 19] +(44) Project [codegen id : 19] Output [4]: [item_sk#40, d_date#35, sumss#41, rk#43] Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] -(48) Exchange +(45) Exchange Input [4]: [item_sk#40, d_date#35, sumss#41, rk#43] Arguments: hashpartitioning(item_sk#40, 5), ENSURE_REQUIREMENTS, [id=#44] -(49) Sort [codegen id : 20] +(46) Sort [codegen id : 20] Input [4]: [item_sk#40, d_date#35, sumss#41, rk#43] Arguments: [item_sk#40 ASC NULLS FIRST], false, 0 -(50) ReusedExchange [Reuses operator id: unknown] +(47) ReusedExchange [Reuses operator id: unknown] Output [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] -(51) Sort [codegen id : 24] +(48) Sort [codegen id : 24] Input [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] Arguments: [ss_item_sk#48 ASC NULLS FIRST, d_date#46 ASC NULLS FIRST], false, 0 -(52) Window +(49) Window Input [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] Arguments: [row_number() windowspecdefinition(ss_item_sk#48, d_date#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [ss_item_sk#48], [d_date#46 ASC NULLS FIRST] -(53) Filter [codegen id : 25] -Input [5]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48, rk#49] -Condition : isnotnull(rk#49) - -(54) Project [codegen id : 25] +(50) Project [codegen id : 25] Output [3]: [item_sk#45, sumss#47, rk#49] Input [5]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48, rk#49] -(55) Exchange +(51) Exchange Input [3]: [item_sk#45, sumss#47, rk#49] Arguments: hashpartitioning(item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#50] -(56) Sort [codegen id : 26] +(52) Sort [codegen id : 26] Input [3]: [item_sk#45, sumss#47, rk#49] Arguments: [item_sk#45 ASC NULLS FIRST], false, 0 -(57) SortMergeJoin [codegen id : 27] +(53) SortMergeJoin [codegen id : 27] Left keys [1]: [item_sk#40] Right keys [1]: [item_sk#45] Join condition: (rk#43 >= rk#49) -(58) Project [codegen id : 27] +(54) Project [codegen id : 27] Output [4]: [item_sk#40, d_date#35, sumss#41, sumss#47] Input [7]: [item_sk#40, d_date#35, sumss#41, rk#43, item_sk#45, sumss#47, rk#49] -(59) HashAggregate [codegen id : 27] +(55) HashAggregate [codegen id : 27] Input [4]: [item_sk#40, d_date#35, sumss#41, sumss#47] Keys [3]: [item_sk#40, d_date#35, sumss#41] Functions [1]: [partial_sum(sumss#47)] Aggregate Attributes [2]: [sum#51, isEmpty#52] Results [5]: [item_sk#40, d_date#35, sumss#41, sum#53, isEmpty#54] -(60) HashAggregate [codegen id : 27] +(56) HashAggregate [codegen id : 27] Input [5]: [item_sk#40, d_date#35, sumss#41, sum#53, isEmpty#54] Keys [3]: [item_sk#40, d_date#35, sumss#41] Functions [1]: [sum(sumss#47)] Aggregate Attributes [1]: [sum(sumss#47)#55] Results [3]: [item_sk#40, d_date#35, sum(sumss#47)#55 AS cume_sales#56] -(61) Exchange +(57) Exchange Input [3]: [item_sk#40, d_date#35, cume_sales#56] Arguments: hashpartitioning(item_sk#40, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#57] -(62) Sort [codegen id : 28] +(58) Sort [codegen id : 28] Input [3]: [item_sk#40, d_date#35, cume_sales#56] Arguments: [item_sk#40 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 ->>>>>>> regen (59) SortMergeJoin Left keys [2]: [item_sk#13, d_date#6] Right keys [2]: [item_sk#40, d_date#35] Join condition: None -<<<<<<< HEAD (60) Filter [codegen id : 29] -Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END) - -(61) Project [codegen id : 29] -Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END AS item_sk#56, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#34 END AS d_date#57, cume_sales#28 AS web_sales#58, cume_sales#54 AS store_sales#59] -Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] - -(62) Exchange -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: hashpartitioning(item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#60] - -(63) Sort [codegen id : 30] -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], false, 0 - -(64) Window -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: [row_number() windowspecdefinition(item_sk#56, d_date#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#61], [item_sk#56], [d_date#57 ASC NULLS FIRST] - -(65) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#62, d_date#63, web_sales#64, store_sales#65] - -(66) 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 - -(67) 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] - -(68) 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] - -(69) SortMergeJoin [codegen id : 62] -Left keys [1]: [item_sk#56] -Right keys [1]: [item_sk#62] -Join condition: (rk#61 >= rk#66) - -(70) 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] - -(71) 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] - -(72) HashAggregate [codegen id : 62] -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)#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] - -(73) 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)) - -(74) 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] -======= -(64) Filter [codegen id : 29] Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END) -(65) Project [codegen id : 29] +(61) Project [codegen id : 29] Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END AS item_sk#58, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#35 END AS d_date#59, cume_sales#29 AS web_sales#60, cume_sales#56 AS store_sales#61] Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] -(66) Exchange +(62) Exchange Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Arguments: hashpartitioning(item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#62] -(67) Sort [codegen id : 30] +(63) Sort [codegen id : 30] Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Arguments: [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], false, 0 -(68) Window +(64) Window Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Arguments: [row_number() windowspecdefinition(item_sk#58, d_date#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#63], [item_sk#58], [d_date#59 ASC NULLS FIRST] -(69) Filter [codegen id : 31] -Input [5]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63] -Condition : isnotnull(rk#63) - -(70) ReusedExchange [Reuses operator id: unknown] +(65) ReusedExchange [Reuses operator id: unknown] Output [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] -(71) Sort [codegen id : 61] +(66) Sort [codegen id : 60] Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] Arguments: [item_sk#64 ASC NULLS FIRST, d_date#65 ASC NULLS FIRST], false, 0 -(72) Window +(67) Window Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] Arguments: [row_number() windowspecdefinition(item_sk#64, d_date#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#68], [item_sk#64], [d_date#65 ASC NULLS FIRST] -(73) Filter [codegen id : 62] -Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] -Condition : isnotnull(rk#68) - -(74) Project [codegen id : 62] +(68) Project [codegen id : 61] Output [4]: [item_sk#64, web_sales#66, store_sales#67, rk#68] Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] -(75) SortMergeJoin [codegen id : 63] +(69) SortMergeJoin [codegen id : 62] Left keys [1]: [item_sk#58] Right keys [1]: [item_sk#64] Join condition: (rk#63 >= rk#68) -(76) Project [codegen id : 63] +(70) Project [codegen id : 62] Output [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] Input [9]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63, item_sk#64, web_sales#66, store_sales#67, rk#68] -(77) HashAggregate [codegen id : 63] +(71) HashAggregate [codegen id : 62] Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Functions [2]: [partial_max(web_sales#66), partial_max(store_sales#67)] Aggregate Attributes [2]: [max#69, max#70] Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#71, max#72] -(78) HashAggregate [codegen id : 63] +(72) HashAggregate [codegen id : 62] Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#71, max#72] Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Functions [2]: [max(web_sales#66), max(store_sales#67)] Aggregate Attributes [2]: [max(web_sales#66)#73, max(store_sales#67)#74] Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max(web_sales#66)#73 AS web_cumulative#75, max(store_sales#67)#74 AS store_cumulative#76] -(79) Filter [codegen id : 63] +(73) Filter [codegen id : 62] Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] Condition : ((isnotnull(web_cumulative#75) AND isnotnull(store_cumulative#76)) AND (web_cumulative#75 > store_cumulative#76)) -(80) TakeOrderedAndProject +(74) TakeOrderedAndProject Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] Arguments: 100, [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] ->>>>>>> regen ===== Subqueries ===== @@ -667,10 +420,6 @@ ReusedExchange (75) (75) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#5, d_date#6] -<<<<<<< HEAD -Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#4 -======= -Subquery:2 Hosting operator id = 34 Hosting Expression = ss_sold_date_sk#33 IN dynamicpruning#4 ->>>>>>> regen +Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#33 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 471fe83a95901..b7d3461ce34ea 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 @@ -152,49 +152,24 @@ Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS F Output [4]: [item_sk#13, d_date#6, sumws#14, rk#16] Input [5]: [item_sk#13, d_date#6, sumws#14, ws_item_sk#1, rk#16] -<<<<<<< HEAD (18) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] - -(19) Sort [codegen id : 8] -Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] -Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 - -(20) Window -Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#20], [ws_item_sk#1], [d_date#18 ASC NULLS FIRST] - -(21) Project [codegen id : 9] -Output [3]: [item_sk#17, sumws#19, rk#20] -Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1, rk#20] - -(22) BroadcastExchange -Input [3]: [item_sk#17, sumws#19, rk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -======= -(19) ReusedExchange [Reuses operator id: unknown] Output [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] -(20) Sort [codegen id : 8] +(19) Sort [codegen id : 8] Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] Arguments: [ws_item_sk#20 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 -(21) Window +(20) Window Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] Arguments: [row_number() windowspecdefinition(ws_item_sk#20, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#21], [ws_item_sk#20], [d_date#18 ASC NULLS FIRST] -(22) Filter [codegen id : 9] -Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20, rk#21] -Condition : isnotnull(rk#21) - -(23) Project [codegen id : 9] +(21) Project [codegen id : 9] Output [3]: [item_sk#17, sumws#19, rk#21] Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20, rk#21] -(24) BroadcastExchange +(22) BroadcastExchange Input [3]: [item_sk#17, sumws#19, rk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] ->>>>>>> regen (23) BroadcastHashJoin [codegen id : 10] Left keys [1]: [item_sk#13] @@ -212,433 +187,214 @@ Functions [1]: [partial_sum(sumws#19)] Aggregate Attributes [2]: [sum#23, isEmpty#24] Results [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] -<<<<<<< HEAD (26) Exchange -Input [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] -Arguments: hashpartitioning(item_sk#13, d_date#6, sumws#14, 5), ENSURE_REQUIREMENTS, [id=#26] - -(27) HashAggregate [codegen id : 11] -Input [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] -======= -(28) Exchange Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] Arguments: hashpartitioning(item_sk#13, d_date#6, sumws#14, 5), ENSURE_REQUIREMENTS, [id=#27] -(29) HashAggregate [codegen id : 11] +(27) HashAggregate [codegen id : 11] Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] ->>>>>>> regen Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [sum(sumws#19)] Aggregate Attributes [1]: [sum(sumws#19)#28] Results [3]: [item_sk#13, d_date#6, sum(sumws#19)#28 AS cume_sales#29] -<<<<<<< HEAD (28) Exchange -Input [3]: [item_sk#13, d_date#6, cume_sales#28] -Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#29] - -(29) Sort [codegen id : 12] -Input [3]: [item_sk#13, d_date#6, cume_sales#28] -Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 - -(30) Scan parquet default.store_sales -Output [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] -======= -(30) Exchange Input [3]: [item_sk#13, d_date#6, cume_sales#29] Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#30] -(31) Sort [codegen id : 12] +(29) Sort [codegen id : 12] Input [3]: [item_sk#13, d_date#6, cume_sales#29] Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 -(32) Scan parquet default.store_sales +(30) Scan parquet default.store_sales Output [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] ->>>>>>> regen Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#33), dynamicpruningexpression(ss_sold_date_sk#33 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -<<<<<<< HEAD (31) ColumnarToRow [codegen id : 14] -Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] - -(32) Filter [codegen id : 14] -Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] -Condition : isnotnull(ss_item_sk#30) - -(33) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#33, d_date#34] - -(34) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_sold_date_sk#32] -Right keys [1]: [d_date_sk#33] -Join condition: None - -(35) Project [codegen id : 14] -Output [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] -Input [5]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32, d_date_sk#33, d_date#34] - -(36) HashAggregate [codegen id : 14] -Input [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] -Keys [2]: [ss_item_sk#30, d_date#34] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum#35] -Results [3]: [ss_item_sk#30, d_date#34, sum#36] - -(37) Exchange -Input [3]: [ss_item_sk#30, d_date#34, sum#36] -Arguments: hashpartitioning(ss_item_sk#30, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#37] - -(38) HashAggregate [codegen id : 15] -Input [3]: [ss_item_sk#30, d_date#34, sum#36] -Keys [2]: [ss_item_sk#30, d_date#34] -Functions [1]: [sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#31))#38] -Results [4]: [ss_item_sk#30 AS item_sk#39, d_date#34, MakeDecimal(sum(UnscaledValue(ss_sales_price#31))#38,17,2) AS sumss#40, ss_item_sk#30] - -(39) Exchange -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: hashpartitioning(ss_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#41] - -(40) Sort [codegen id : 16] -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 - -(41) Window -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#42], [ss_item_sk#30], [d_date#34 ASC NULLS FIRST] - -(42) Project [codegen id : 22] -Output [4]: [item_sk#39, d_date#34, sumss#40, rk#42] -Input [5]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30, rk#42] - -(43) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] - -(44) Sort [codegen id : 20] -Input [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] -Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#44 ASC NULLS FIRST], false, 0 - -(45) Window -Input [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] -Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#46], [ss_item_sk#30], [d_date#44 ASC NULLS FIRST] - -(46) Project [codegen id : 21] -Output [3]: [item_sk#43, sumss#45, rk#46] -Input [5]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30, rk#46] - -(47) BroadcastExchange -Input [3]: [item_sk#43, sumss#45, rk#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] - -(48) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [item_sk#39] -Right keys [1]: [item_sk#43] -Join condition: (rk#42 >= rk#46) - -(49) Project [codegen id : 22] -Output [4]: [item_sk#39, d_date#34, sumss#40, sumss#45] -Input [7]: [item_sk#39, d_date#34, sumss#40, rk#42, item_sk#43, sumss#45, rk#46] - -(50) HashAggregate [codegen id : 22] -Input [4]: [item_sk#39, d_date#34, sumss#40, sumss#45] -Keys [3]: [item_sk#39, d_date#34, sumss#40] -Functions [1]: [partial_sum(sumss#45)] -Aggregate Attributes [2]: [sum#48, isEmpty#49] -Results [5]: [item_sk#39, d_date#34, sumss#40, sum#50, isEmpty#51] - -(51) Exchange -Input [5]: [item_sk#39, d_date#34, sumss#40, sum#50, isEmpty#51] -Arguments: hashpartitioning(item_sk#39, d_date#34, sumss#40, 5), ENSURE_REQUIREMENTS, [id=#52] - -(52) HashAggregate [codegen id : 23] -Input [5]: [item_sk#39, d_date#34, sumss#40, sum#50, isEmpty#51] -Keys [3]: [item_sk#39, d_date#34, sumss#40] -Functions [1]: [sum(sumss#45)] -Aggregate Attributes [1]: [sum(sumss#45)#53] -Results [3]: [item_sk#39, d_date#34, sum(sumss#45)#53 AS cume_sales#54] - -(53) Exchange -Input [3]: [item_sk#39, d_date#34, cume_sales#54] -Arguments: hashpartitioning(item_sk#39, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#55] - -(54) Sort [codegen id : 24] -Input [3]: [item_sk#39, d_date#34, cume_sales#54] -Arguments: [item_sk#39 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 -======= -(33) ColumnarToRow [codegen id : 14] Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] -(34) Filter [codegen id : 14] +(32) Filter [codegen id : 14] Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] Condition : isnotnull(ss_item_sk#31) -(35) ReusedExchange [Reuses operator id: 8] +(33) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#34, d_date#35] -(36) BroadcastHashJoin [codegen id : 14] +(34) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_sold_date_sk#33] Right keys [1]: [d_date_sk#34] Join condition: None -(37) Project [codegen id : 14] +(35) Project [codegen id : 14] Output [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] Input [5]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33, d_date_sk#34, d_date#35] -(38) HashAggregate [codegen id : 14] +(36) HashAggregate [codegen id : 14] Input [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] Keys [2]: [ss_item_sk#31, d_date#35] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#32))] Aggregate Attributes [1]: [sum#36] Results [3]: [ss_item_sk#31, d_date#35, sum#37] -(39) Exchange +(37) Exchange Input [3]: [ss_item_sk#31, d_date#35, sum#37] Arguments: hashpartitioning(ss_item_sk#31, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#38] -(40) HashAggregate [codegen id : 15] +(38) HashAggregate [codegen id : 15] Input [3]: [ss_item_sk#31, d_date#35, sum#37] Keys [2]: [ss_item_sk#31, d_date#35] Functions [1]: [sum(UnscaledValue(ss_sales_price#32))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#32))#39] Results [4]: [ss_item_sk#31 AS item_sk#40, d_date#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#32))#39,17,2) AS sumss#41, ss_item_sk#31] -(41) Exchange +(39) Exchange Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#42] -(42) Sort [codegen id : 16] +(40) Sort [codegen id : 16] Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] Arguments: [ss_item_sk#31 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 -(43) Window +(41) Window Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] Arguments: [row_number() windowspecdefinition(ss_item_sk#31, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [ss_item_sk#31], [d_date#35 ASC NULLS FIRST] -(44) Filter [codegen id : 22] -Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] -Condition : isnotnull(rk#43) - -(45) Project [codegen id : 22] +(42) Project [codegen id : 22] Output [4]: [item_sk#40, d_date#35, sumss#41, rk#43] Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] -(46) ReusedExchange [Reuses operator id: unknown] +(43) ReusedExchange [Reuses operator id: unknown] Output [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] -(47) Sort [codegen id : 20] +(44) Sort [codegen id : 20] Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], false, 0 -(48) Window +(45) Window Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] Arguments: [row_number() windowspecdefinition(ss_item_sk#47, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [ss_item_sk#47], [d_date#45 ASC NULLS FIRST] -(49) Filter [codegen id : 21] -Input [5]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47, rk#48] -Condition : isnotnull(rk#48) - -(50) Project [codegen id : 21] +(46) Project [codegen id : 21] Output [3]: [item_sk#44, sumss#46, rk#48] Input [5]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47, rk#48] -(51) BroadcastExchange +(47) BroadcastExchange Input [3]: [item_sk#44, sumss#46, rk#48] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] -(52) BroadcastHashJoin [codegen id : 22] +(48) BroadcastHashJoin [codegen id : 22] Left keys [1]: [item_sk#40] Right keys [1]: [item_sk#44] Join condition: (rk#43 >= rk#48) -(53) Project [codegen id : 22] +(49) Project [codegen id : 22] Output [4]: [item_sk#40, d_date#35, sumss#41, sumss#46] Input [7]: [item_sk#40, d_date#35, sumss#41, rk#43, item_sk#44, sumss#46, rk#48] -(54) HashAggregate [codegen id : 22] +(50) HashAggregate [codegen id : 22] Input [4]: [item_sk#40, d_date#35, sumss#41, sumss#46] Keys [3]: [item_sk#40, d_date#35, sumss#41] Functions [1]: [partial_sum(sumss#46)] Aggregate Attributes [2]: [sum#50, isEmpty#51] Results [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] -(55) Exchange +(51) Exchange Input [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] Arguments: hashpartitioning(item_sk#40, d_date#35, sumss#41, 5), ENSURE_REQUIREMENTS, [id=#54] -(56) HashAggregate [codegen id : 23] +(52) HashAggregate [codegen id : 23] Input [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] Keys [3]: [item_sk#40, d_date#35, sumss#41] Functions [1]: [sum(sumss#46)] Aggregate Attributes [1]: [sum(sumss#46)#55] Results [3]: [item_sk#40, d_date#35, sum(sumss#46)#55 AS cume_sales#56] -(57) Exchange +(53) Exchange Input [3]: [item_sk#40, d_date#35, cume_sales#56] Arguments: hashpartitioning(item_sk#40, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#57] -(58) Sort [codegen id : 24] +(54) Sort [codegen id : 24] Input [3]: [item_sk#40, d_date#35, cume_sales#56] Arguments: [item_sk#40 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 ->>>>>>> regen (55) SortMergeJoin Left keys [2]: [item_sk#13, d_date#6] Right keys [2]: [item_sk#40, d_date#35] Join condition: None -<<<<<<< HEAD (56) Filter [codegen id : 25] -Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END) - -(57) Project [codegen id : 25] -Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END AS item_sk#56, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#34 END AS d_date#57, cume_sales#28 AS web_sales#58, cume_sales#54 AS store_sales#59] -Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] - -(58) Exchange -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: hashpartitioning(item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#60] - -(59) Sort [codegen id : 26] -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], false, 0 - -(60) Window -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: [row_number() windowspecdefinition(item_sk#56, d_date#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#61], [item_sk#56], [d_date#57 ASC NULLS FIRST] - -(61) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#62, d_date#63, web_sales#64, store_sales#65] - -(62) 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 - -(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] - -(64) 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] - -(65) 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] - -(66) BroadcastHashJoin [codegen id : 54] -Left keys [1]: [item_sk#56] -Right keys [1]: [item_sk#62] -Join condition: (rk#61 >= rk#66) - -(67) 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] - -(68) 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] - -(69) HashAggregate [codegen id : 54] -Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#70, max#71] -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] - -(70) 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)) - -(71) 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] -======= -(60) Filter [codegen id : 25] Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END) -(61) Project [codegen id : 25] +(57) Project [codegen id : 25] Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END AS item_sk#58, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#35 END AS d_date#59, cume_sales#29 AS web_sales#60, cume_sales#56 AS store_sales#61] Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] -(62) Exchange +(58) Exchange Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Arguments: hashpartitioning(item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#62] -(63) Sort [codegen id : 26] +(59) Sort [codegen id : 26] Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Arguments: [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], false, 0 -(64) Window +(60) Window Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Arguments: [row_number() windowspecdefinition(item_sk#58, d_date#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#63], [item_sk#58], [d_date#59 ASC NULLS FIRST] -(65) Filter [codegen id : 54] -Input [5]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63] -Condition : isnotnull(rk#63) - -(66) ReusedExchange [Reuses operator id: unknown] +(61) ReusedExchange [Reuses operator id: unknown] Output [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] -(67) Sort [codegen id : 52] +(62) Sort [codegen id : 52] Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] Arguments: [item_sk#64 ASC NULLS FIRST, d_date#65 ASC NULLS FIRST], false, 0 -(68) Window +(63) Window Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] Arguments: [row_number() windowspecdefinition(item_sk#64, d_date#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#68], [item_sk#64], [d_date#65 ASC NULLS FIRST] -(69) Filter [codegen id : 53] -Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] -Condition : isnotnull(rk#68) - -(70) Project [codegen id : 53] +(64) Project [codegen id : 53] Output [4]: [item_sk#64, web_sales#66, store_sales#67, rk#68] Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] -(71) BroadcastExchange +(65) BroadcastExchange Input [4]: [item_sk#64, web_sales#66, store_sales#67, rk#68] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] -(72) BroadcastHashJoin [codegen id : 54] +(66) BroadcastHashJoin [codegen id : 54] Left keys [1]: [item_sk#58] Right keys [1]: [item_sk#64] Join condition: (rk#63 >= rk#68) -(73) Project [codegen id : 54] +(67) Project [codegen id : 54] Output [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] Input [9]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63, item_sk#64, web_sales#66, store_sales#67, rk#68] -(74) HashAggregate [codegen id : 54] +(68) HashAggregate [codegen id : 54] Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Functions [2]: [partial_max(web_sales#66), partial_max(store_sales#67)] Aggregate Attributes [2]: [max#70, max#71] Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#72, max#73] -(75) HashAggregate [codegen id : 54] +(69) HashAggregate [codegen id : 54] Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#72, max#73] Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Functions [2]: [max(web_sales#66), max(store_sales#67)] Aggregate Attributes [2]: [max(web_sales#66)#74, max(store_sales#67)#75] Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max(web_sales#66)#74 AS web_cumulative#76, max(store_sales#67)#75 AS store_cumulative#77] -(76) Filter [codegen id : 54] +(70) Filter [codegen id : 54] Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] Condition : ((isnotnull(web_cumulative#76) AND isnotnull(store_cumulative#77)) AND (web_cumulative#76 > store_cumulative#77)) -(77) TakeOrderedAndProject +(71) TakeOrderedAndProject Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] Arguments: 100, [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] ->>>>>>> regen ===== Subqueries ===== @@ -649,10 +405,6 @@ ReusedExchange (72) (72) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#5, d_date#6] -<<<<<<< HEAD -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#4 -======= -Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#33 IN dynamicpruning#4 ->>>>>>> regen +Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#33 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 8d2776513f90b..e3de4a7a053b1 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 @@ -213,11 +213,7 @@ Arguments: [rank(d_year#7, d_moy#8) windowspecdefinition(i_category#16, i_brand# (35) Filter [codegen id : 12] Input [8]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27] -<<<<<<< HEAD Condition : ((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#25)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) -======= -Condition : (((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#25)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#27)) ->>>>>>> regen (36) Exchange Input [8]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27] @@ -249,123 +245,62 @@ Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#3 Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] -<<<<<<< HEAD (43) Project [codegen id : 23] -Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] - -(44) Exchange -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#38 + 1), 5), ENSURE_REQUIREMENTS, [id=#39] - -(45) Sort [codegen id : 24] -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#38 + 1) ASC NULLS FIRST], false, 0 -======= -(43) Filter [codegen id : 23] -Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] -Condition : isnotnull(rn#39) - -(44) Project [codegen id : 23] Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] -(45) Exchange +(44) Exchange Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] -(46) Sort [codegen id : 24] +(45) Sort [codegen id : 24] Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 ->>>>>>> regen (46) SortMergeJoin [codegen id : 25] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#39 + 1)] Join condition: None -<<<<<<< HEAD (47) Project [codegen id : 25] -Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] -Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] - -(48) ReusedExchange [Reuses operator id: 40] -Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] - -(49) Sort [codegen id : 34] -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 - -(50) Window -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] - -(51) Project [codegen id : 35] -Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] - -(52) Exchange -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (rn#46 - 1), 5), ENSURE_REQUIREMENTS, [id=#47] - -(53) Sort [codegen id : 36] -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 -======= -(48) Project [codegen id : 25] Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] -(49) ReusedExchange [Reuses operator id: 40] +(48) ReusedExchange [Reuses operator id: 40] Output [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] -(50) Sort [codegen id : 34] +(49) Sort [codegen id : 34] Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 -(51) Window +(50) Window Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#41, i_brand#42, cc_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#41, i_brand#42, cc_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] -(52) Filter [codegen id : 35] -Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] -Condition : isnotnull(rn#47) - -(53) Project [codegen id : 35] +(51) Project [codegen id : 35] Output [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] -(54) Exchange +(52) Exchange Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] Arguments: hashpartitioning(i_category#41, i_brand#42, cc_name#43, (rn#47 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] -(55) Sort [codegen id : 36] +(53) Sort [codegen id : 36] Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, (rn#47 - 1) ASC NULLS FIRST], false, 0 ->>>>>>> regen (54) SortMergeJoin [codegen id : 37] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] Right keys [4]: [i_category#41, i_brand#42, cc_name#43, (rn#47 - 1)] Join condition: None -<<<<<<< HEAD (55) Project [codegen id : 37] -Output [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#48, sum_sales#45 AS nsum#49] -Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] - -(56) TakeOrderedAndProject -Input [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 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#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] -======= -(57) Project [codegen id : 37] Output [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#49, sum_sales#46 AS nsum#50] Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] -(58) TakeOrderedAndProject +(56) TakeOrderedAndProject Input [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] ->>>>>>> regen ===== Subqueries ===== 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 f1956e47f453e..a6742cf4ab1cf 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 @@ -194,11 +194,7 @@ Arguments: [rank(d_year#11, d_moy#12) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [8]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26] -<<<<<<< HEAD Condition : ((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#24)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) -======= -Condition : (((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#24)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#26)) ->>>>>>> regen (33) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] @@ -222,107 +218,54 @@ Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#2 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] -<<<<<<< HEAD (38) Project [codegen id : 15] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] -Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34, rn#36] - -(39) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#37] -======= -(38) Filter [codegen id : 15] -Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] -Condition : isnotnull(rn#37) - -(39) Project [codegen id : 15] 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] -(40) BroadcastExchange +(39) BroadcastExchange Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] + 1)),false), [id=#38] ->>>>>>> regen +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#38] (40) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] Join condition: None -<<<<<<< HEAD (41) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34] -Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] - -(42) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] - -(43) Sort [codegen id : 21] -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] -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 - -(44) Window -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] -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#44], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] - -(45) Project [codegen id : 22] -Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] -Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43, rn#44] - -(46) BroadcastExchange -Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#45] -======= -(42) Project [codegen id : 23] Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35] Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -(43) ReusedExchange [Reuses operator id: 35] +(42) ReusedExchange [Reuses operator id: 35] Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] -(44) Sort [codegen id : 21] +(43) Sort [codegen id : 21] 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 -(45) Window +(44) 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] -(46) Filter [codegen id : 22] -Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] -Condition : isnotnull(rn#45) - -(47) Project [codegen id : 22] +(45) Project [codegen id : 22] 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] -(48) BroadcastExchange +(46) BroadcastExchange Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] - 1)),false), [id=#46] ->>>>>>> regen +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#46] (47) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] Join condition: None -<<<<<<< HEAD (48) Project [codegen id : 23] -Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#34 AS psum#46, sum_sales#43 AS nsum#47] -Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34, i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] - -(49) TakeOrderedAndProject -Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, 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#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#21, psum#46, nsum#47] -======= -(50) Project [codegen id : 23] Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -(51) TakeOrderedAndProject +(49) TakeOrderedAndProject Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, 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#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#21, psum#47, nsum#48] ->>>>>>> regen ===== Subqueries ===== 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 13bcda00cb11d..af8035e67b061 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 @@ -445,7 +445,7 @@ Arguments: [rank(sumsales#29) windowspecdefinition(i_category#20, sumsales#29 DE (73) Filter [codegen id : 82] Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#228] -Condition : (isnotnull(rk#228) AND (rk#228 <= 100)) +Condition : (rk#228 <= 100) (74) TakeOrderedAndProject Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#228] 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 ffa5e96fad67e..e10a5ea0aff0b 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 @@ -430,7 +430,7 @@ Arguments: [rank(sumsales#28) windowspecdefinition(i_category#19, sumsales#28 DE (70) Filter [codegen id : 55] Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#227] -Condition : (isnotnull(rk#227) AND (rk#227 <= 100)) +Condition : (rk#227 <= 100) (71) TakeOrderedAndProject Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#227] 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 d99363440b2bf..35ed13e84c742 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 @@ -207,7 +207,7 @@ Arguments: [rank(_w2#23) windowspecdefinition(s_state#16, _w2#23 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] -Condition : (isnotnull(ranking#25) AND (ranking#25 <= 5)) +Condition : (ranking#25 <= 5) (33) Project [codegen id : 7] Output [1]: [s_state#22] 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 159494182ba83..df0ee68806062 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 @@ -207,7 +207,7 @@ Arguments: [rank(_w2#23) windowspecdefinition(s_state#15, _w2#23 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] -Condition : (isnotnull(ranking#25) AND (ranking#25 <= 5)) +Condition : (ranking#25 <= 5) (33) Project [codegen id : 7] Output [1]: [s_state#22] From 1a9f23b4958f4a3c7d671eb0f0d60d1652a63e19 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 23 Mar 2021 17:23:19 +0800 Subject: [PATCH 20/38] Revert "gen golden files" This reverts commit a7398ae949a04ec7fab575438a5583056ed7fbb7. --- .../approved-plans-v1_4/q44.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q44/explain.txt | 4 +- .../approved-plans-v1_4/q47.sf100/explain.txt | 83 ++++- .../approved-plans-v1_4/q47/explain.txt | 75 +++- .../approved-plans-v1_4/q57.sf100/explain.txt | 83 ++++- .../approved-plans-v1_4/q57/explain.txt | 75 +++- .../approved-plans-v1_4/q67.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q67/explain.txt | 2 +- .../approved-plans-v1_4/q70.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q70/explain.txt | 2 +- .../approved-plans-v2_7/q47.sf100/explain.txt | 83 ++++- .../approved-plans-v2_7/q47/explain.txt | 75 +++- .../q51a.sf100/explain.txt | 345 +++++++++++++++--- .../approved-plans-v2_7/q51a/explain.txt | 340 ++++++++++++++--- .../approved-plans-v2_7/q57.sf100/explain.txt | 83 ++++- .../approved-plans-v2_7/q57/explain.txt | 75 +++- .../q67a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q67a/explain.txt | 2 +- .../q70a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q70a/explain.txt | 2 +- 20 files changed, 1165 insertions(+), 178 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt index 84a8547e3f6ef..a6270c8ab533a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt @@ -95,7 +95,7 @@ Arguments: [rank(rank_col#12) windowspecdefinition(rank_col#12 ASC NULLS FIRST, (13) Filter [codegen id : 10] Input [3]: [item_sk#11, rank_col#12, rnk#17] -Condition : ((rnk#17 < 11) AND isnotnull(item_sk#11)) +Condition : ((isnotnull(rnk#17) AND (rnk#17 < 11)) AND isnotnull(item_sk#11)) (14) Project [codegen id : 10] Output [2]: [item_sk#11, rnk#17] @@ -133,7 +133,7 @@ Arguments: [rank(rank_col#24) windowspecdefinition(rank_col#24 DESC NULLS LAST, (22) Filter [codegen id : 7] Input [3]: [item_sk#23, rank_col#24, rnk#27] -Condition : ((rnk#27 < 11) AND isnotnull(item_sk#23)) +Condition : ((isnotnull(rnk#27) AND (rnk#27 < 11)) AND isnotnull(item_sk#23)) (23) Project [codegen id : 7] Output [2]: [item_sk#23, rnk#27] @@ -141,7 +141,7 @@ Input [3]: [item_sk#23, rank_col#24, rnk#27] (24) BroadcastExchange Input [2]: [item_sk#23, rnk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#28] (25) BroadcastHashJoin [codegen id : 10] Left keys [1]: [rnk#17] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index b42ddd8db7836..a220cf9ff546f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -96,7 +96,7 @@ Arguments: [rank(rank_col#12) windowspecdefinition(rank_col#12 ASC NULLS FIRST, (13) Filter [codegen id : 4] Input [3]: [item_sk#11, rank_col#12, rnk#17] -Condition : ((rnk#17 < 11) AND isnotnull(item_sk#11)) +Condition : ((isnotnull(rnk#17) AND (rnk#17 < 11)) AND isnotnull(item_sk#11)) (14) Project [codegen id : 4] Output [2]: [item_sk#11, rnk#17] @@ -138,7 +138,7 @@ Arguments: [rank(rank_col#24) windowspecdefinition(rank_col#24 DESC NULLS LAST, (23) Filter [codegen id : 8] Input [3]: [item_sk#23, rank_col#24, rnk#27] -Condition : ((rnk#27 < 11) AND isnotnull(item_sk#23)) +Condition : ((isnotnull(rnk#27) AND (rnk#27 < 11)) AND isnotnull(item_sk#23)) (24) Project [codegen id : 8] Output [2]: [item_sk#23, rnk#27] 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 b7ffba01f92c6..3a356e35707f7 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 @@ -213,7 +213,11 @@ Arguments: [rank(d_year#7, d_moy#8) windowspecdefinition(i_category#17, i_brand# (35) Filter [codegen id : 12] Input [9]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28] +<<<<<<< HEAD Condition : ((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#26)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#26 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +======= +Condition : (((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#26)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#26 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#28)) +>>>>>>> regen (36) Exchange Input [9]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28] @@ -245,62 +249,123 @@ Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_n Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +<<<<<<< HEAD (43) Project [codegen id : 23] +Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] +Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38, rn#40] + +(44) Exchange +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1), 5), ENSURE_REQUIREMENTS, [id=#41] + +(45) Sort [codegen id : 24] +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] +Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#40 + 1) ASC NULLS FIRST], false, 0 +======= +(43) Filter [codegen id : 23] +Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] +Condition : isnotnull(rn#41) + +(44) Project [codegen id : 23] Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] -(44) Exchange +(45) Exchange Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1), 5), ENSURE_REQUIREMENTS, [id=#42] -(45) Sort [codegen id : 24] +(46) Sort [codegen id : 24] Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#41 + 1) ASC NULLS FIRST], false, 0 +>>>>>>> regen (46) SortMergeJoin [codegen id : 25] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1)] Join condition: None +<<<<<<< HEAD (47) Project [codegen id : 25] +Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38] +Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] + +(48) ReusedExchange [Reuses operator id: 40] +Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] + +(49) Sort [codegen id : 34] +Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 + +(50) Window +Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] + +(51) Project [codegen id : 35] +Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] +Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] + +(52) Exchange +Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] +Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1), 5), ENSURE_REQUIREMENTS, [id=#50] + +(53) Sort [codegen id : 36] +Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] +Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (rn#49 - 1) ASC NULLS FIRST], false, 0 +======= +(48) Project [codegen id : 25] Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39] Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] -(48) ReusedExchange [Reuses operator id: 40] +(49) ReusedExchange [Reuses operator id: 40] Output [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] -(49) Sort [codegen id : 34] +(50) Sort [codegen id : 34] Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST], false, 0 -(50) Window +(51) Window Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] Arguments: [rank(d_year#47, d_moy#48) windowspecdefinition(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#50], [i_category#43, i_brand#44, s_store_name#45, s_company_name#46], [d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST] -(51) Project [codegen id : 35] +(52) Filter [codegen id : 35] +Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] +Condition : isnotnull(rn#50) + +(53) Project [codegen id : 35] Output [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] -(52) Exchange +(54) Exchange Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] Arguments: hashpartitioning(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1), 5), ENSURE_REQUIREMENTS, [id=#51] -(53) Sort [codegen id : 36] +(55) Sort [codegen id : 36] Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, (rn#50 - 1) ASC NULLS FIRST], false, 0 +>>>>>>> regen (54) SortMergeJoin [codegen id : 37] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] Right keys [5]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1)] Join condition: None +<<<<<<< HEAD (55) Project [codegen id : 37] +Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#38 AS psum#51, sum_sales#48 AS nsum#52] +Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] + +(56) TakeOrderedAndProject +Input [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#11 ASC NULLS FIRST], [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] +======= +(57) Project [codegen id : 37] Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#39 AS psum#52, sum_sales#49 AS nsum#53] Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39, i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] -(56) TakeOrderedAndProject +(58) TakeOrderedAndProject Input [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#11 ASC NULLS FIRST], [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] +>>>>>>> regen ===== Subqueries ===== 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 917b21c247449..4a96c307dff2f 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 @@ -194,7 +194,11 @@ Arguments: [rank(d_year#11, d_moy#12) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [9]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27] +<<<<<<< HEAD Condition : ((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#25)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +======= +Condition : (((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#25)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#27)) +>>>>>>> regen (33) ReusedExchange [Reuses operator id: unknown] Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34] @@ -218,54 +222,107 @@ Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_n 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] +<<<<<<< HEAD (38) Project [codegen id : 15] +Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] +Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] + +(39) BroadcastExchange +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] +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=#39] +======= +(38) Filter [codegen id : 15] +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] +Condition : isnotnull(rn#39) + +(39) Project [codegen id : 15] 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] -(39) BroadcastExchange +(40) BroadcastExchange Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -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=#40] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] + 1)),false), [id=#40] +>>>>>>> regen (40) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] Join condition: None +<<<<<<< HEAD (41) Project [codegen id : 23] +Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] +Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] + +(42) ReusedExchange [Reuses operator id: 35] +Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] + +(43) Sort [codegen id : 21] +Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#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, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 + +(44) Window +Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] +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#47], [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] + +(45) Project [codegen id : 22] +Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] +Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] + +(46) BroadcastExchange +Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] +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=#48] +======= +(42) Project [codegen id : 23] Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -(42) ReusedExchange [Reuses operator id: 35] +(43) ReusedExchange [Reuses operator id: 35] 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] -(43) Sort [codegen id : 21] +(44) Sort [codegen id : 21] 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 -(44) Window +(45) 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] -(45) Project [codegen id : 22] +(46) Filter [codegen id : 22] +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] +Condition : isnotnull(rn#48) + +(47) Project [codegen id : 22] 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] -(46) BroadcastExchange +(48) BroadcastExchange Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -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=#49] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] - 1)),false), [id=#49] +>>>>>>> regen (47) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] Join condition: None +<<<<<<< HEAD (48) Project [codegen id : 23] +Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#49, sum_sales#46 AS nsum#50] +Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] + +(49) TakeOrderedAndProject +Input [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] +======= +(50) Project [codegen id : 23] Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -(49) TakeOrderedAndProject +(51) TakeOrderedAndProject Input [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, psum#50, nsum#51] +>>>>>>> regen ===== Subqueries ===== 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 6e39745703215..173125b197079 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 @@ -213,7 +213,11 @@ Arguments: [rank(d_year#7, d_moy#8) windowspecdefinition(i_category#16, i_brand# (35) Filter [codegen id : 12] Input [8]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27] +<<<<<<< HEAD Condition : ((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#25)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +======= +Condition : (((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#25)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#27)) +>>>>>>> regen (36) Exchange Input [8]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27] @@ -245,62 +249,123 @@ Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#3 Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +<<<<<<< HEAD (43) Project [codegen id : 23] +Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] + +(44) Exchange +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#38 + 1), 5), ENSURE_REQUIREMENTS, [id=#39] + +(45) Sort [codegen id : 24] +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] +Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#38 + 1) ASC NULLS FIRST], false, 0 +======= +(43) Filter [codegen id : 23] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] +Condition : isnotnull(rn#39) + +(44) Project [codegen id : 23] Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] -(44) Exchange +(45) Exchange Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] -(45) Sort [codegen id : 24] +(46) Sort [codegen id : 24] Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 +>>>>>>> regen (46) SortMergeJoin [codegen id : 25] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#39 + 1)] Join condition: None +<<<<<<< HEAD (47) Project [codegen id : 25] +Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] +Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] + +(48) ReusedExchange [Reuses operator id: 40] +Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] + +(49) Sort [codegen id : 34] +Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 + +(50) Window +Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] + +(51) Project [codegen id : 35] +Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] +Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] + +(52) Exchange +Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] +Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (rn#46 - 1), 5), ENSURE_REQUIREMENTS, [id=#47] + +(53) Sort [codegen id : 36] +Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 +======= +(48) Project [codegen id : 25] Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] -(48) ReusedExchange [Reuses operator id: 40] +(49) ReusedExchange [Reuses operator id: 40] Output [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] -(49) Sort [codegen id : 34] +(50) Sort [codegen id : 34] Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 -(50) Window +(51) Window Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#41, i_brand#42, cc_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#41, i_brand#42, cc_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] -(51) Project [codegen id : 35] +(52) Filter [codegen id : 35] +Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] +Condition : isnotnull(rn#47) + +(53) Project [codegen id : 35] Output [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] -(52) Exchange +(54) Exchange Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] Arguments: hashpartitioning(i_category#41, i_brand#42, cc_name#43, (rn#47 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] -(53) Sort [codegen id : 36] +(55) Sort [codegen id : 36] Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, (rn#47 - 1) ASC NULLS FIRST], false, 0 +>>>>>>> regen (54) SortMergeJoin [codegen id : 37] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] Right keys [4]: [i_category#41, i_brand#42, cc_name#43, (rn#47 - 1)] Join condition: None +<<<<<<< HEAD (55) Project [codegen id : 37] +Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#48, sum_sales#45 AS nsum#49] +Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] + +(56) TakeOrderedAndProject +Input [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] +======= +(57) Project [codegen id : 37] Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#49, sum_sales#46 AS nsum#50] Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] -(56) TakeOrderedAndProject +(58) TakeOrderedAndProject Input [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] +>>>>>>> regen ===== Subqueries ===== 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 fa24f5ede5453..d5b269b02d199 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 @@ -194,7 +194,11 @@ Arguments: [rank(d_year#11, d_moy#12) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [8]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26] +<<<<<<< HEAD Condition : ((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#24)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +======= +Condition : (((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#24)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#26)) +>>>>>>> regen (33) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] @@ -218,54 +222,107 @@ Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#2 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] +<<<<<<< HEAD (38) Project [codegen id : 15] +Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] +Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34, rn#36] + +(39) BroadcastExchange +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#37] +======= +(38) Filter [codegen id : 15] +Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] +Condition : isnotnull(rn#37) + +(39) Project [codegen id : 15] 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] -(39) BroadcastExchange +(40) BroadcastExchange Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] + 1)),false), [id=#38] +>>>>>>> regen (40) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] Join condition: None +<<<<<<< HEAD (41) Project [codegen id : 23] +Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34] +Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] + +(42) ReusedExchange [Reuses operator id: 35] +Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] + +(43) Sort [codegen id : 21] +Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] +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 + +(44) Window +Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] +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#44], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] + +(45) Project [codegen id : 22] +Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] +Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43, rn#44] + +(46) BroadcastExchange +Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#45] +======= +(42) Project [codegen id : 23] Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35] Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -(42) ReusedExchange [Reuses operator id: 35] +(43) ReusedExchange [Reuses operator id: 35] Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] -(43) Sort [codegen id : 21] +(44) Sort [codegen id : 21] 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 -(44) Window +(45) 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] -(45) Project [codegen id : 22] +(46) Filter [codegen id : 22] +Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] +Condition : isnotnull(rn#45) + +(47) Project [codegen id : 22] 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] -(46) BroadcastExchange +(48) BroadcastExchange Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#46] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] - 1)),false), [id=#46] +>>>>>>> regen (47) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] Join condition: None +<<<<<<< HEAD (48) Project [codegen id : 23] +Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#34 AS psum#46, sum_sales#43 AS nsum#47] +Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34, i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] + +(49) TakeOrderedAndProject +Input [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, 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#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, psum#46, nsum#47] +======= +(50) Project [codegen id : 23] Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -(49) TakeOrderedAndProject +(51) TakeOrderedAndProject Input [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, 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#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, psum#47, nsum#48] +>>>>>>> regen ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt index 3a19c3923737d..075fd1864222c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt @@ -183,7 +183,7 @@ Arguments: [rank(sumsales#38) windowspecdefinition(i_category#23, sumsales#38 DE (33) Filter [codegen id : 10] Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#38, rk#40] -Condition : (rk#40 <= 100) +Condition : (isnotnull(rk#40) AND (rk#40 <= 100)) (34) TakeOrderedAndProject Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#38, rk#40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt index 3d42f49415640..ba71667cfebe5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt @@ -168,7 +168,7 @@ Arguments: [rank(sumsales#37) windowspecdefinition(i_category#22, sumsales#37 DE (30) Filter [codegen id : 7] Input [10]: [i_category#22, i_class#23, i_brand#24, i_product_name#25, d_year#26, d_qoy#27, d_moy#28, s_store_id#29, sumsales#37, rk#39] -Condition : (rk#39 <= 100) +Condition : (isnotnull(rk#39) AND (rk#39 <= 100)) (31) TakeOrderedAndProject Input [10]: [i_category#22, i_class#23, i_brand#24, i_product_name#25, d_year#26, d_qoy#27, d_moy#28, s_store_id#29, sumsales#37, rk#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index 7b8445a46ac54..ddce6bf2e14f4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -194,7 +194,7 @@ Arguments: [rank(_w2#23) windowspecdefinition(s_state#16, _w2#23 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] -Condition : (ranking#25 <= 5) +Condition : (isnotnull(ranking#25) AND (ranking#25 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index 91cfba59bb5bc..dbdfc72f92624 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -194,7 +194,7 @@ Arguments: [rank(_w2#23) windowspecdefinition(s_state#15, _w2#23 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] -Condition : (ranking#25 <= 5) +Condition : (isnotnull(ranking#25) AND (ranking#25 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#22] 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 636fa32f94212..20d80b4503755 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 @@ -213,7 +213,11 @@ Arguments: [rank(d_year#7, d_moy#8) windowspecdefinition(i_category#17, i_brand# (35) Filter [codegen id : 12] Input [9]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28] +<<<<<<< HEAD Condition : ((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#26)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#26 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +======= +Condition : (((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#26)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#26 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#28)) +>>>>>>> regen (36) Exchange Input [9]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28] @@ -245,62 +249,123 @@ Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_n Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +<<<<<<< HEAD (43) Project [codegen id : 23] +Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] +Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38, rn#40] + +(44) Exchange +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1), 5), ENSURE_REQUIREMENTS, [id=#41] + +(45) Sort [codegen id : 24] +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] +Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#40 + 1) ASC NULLS FIRST], false, 0 +======= +(43) Filter [codegen id : 23] +Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] +Condition : isnotnull(rn#41) + +(44) Project [codegen id : 23] Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] -(44) Exchange +(45) Exchange Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1), 5), ENSURE_REQUIREMENTS, [id=#42] -(45) Sort [codegen id : 24] +(46) Sort [codegen id : 24] Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#41 + 1) ASC NULLS FIRST], false, 0 +>>>>>>> regen (46) SortMergeJoin [codegen id : 25] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1)] Join condition: None +<<<<<<< HEAD (47) Project [codegen id : 25] +Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38] +Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] + +(48) ReusedExchange [Reuses operator id: 40] +Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] + +(49) Sort [codegen id : 34] +Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 + +(50) Window +Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] + +(51) Project [codegen id : 35] +Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] +Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] + +(52) Exchange +Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] +Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1), 5), ENSURE_REQUIREMENTS, [id=#50] + +(53) Sort [codegen id : 36] +Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] +Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (rn#49 - 1) ASC NULLS FIRST], false, 0 +======= +(48) Project [codegen id : 25] Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39] Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] -(48) ReusedExchange [Reuses operator id: 40] +(49) ReusedExchange [Reuses operator id: 40] Output [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] -(49) Sort [codegen id : 34] +(50) Sort [codegen id : 34] Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST], false, 0 -(50) Window +(51) Window Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] Arguments: [rank(d_year#47, d_moy#48) windowspecdefinition(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#50], [i_category#43, i_brand#44, s_store_name#45, s_company_name#46], [d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST] -(51) Project [codegen id : 35] +(52) Filter [codegen id : 35] +Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] +Condition : isnotnull(rn#50) + +(53) Project [codegen id : 35] Output [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] -(52) Exchange +(54) Exchange Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] Arguments: hashpartitioning(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1), 5), ENSURE_REQUIREMENTS, [id=#51] -(53) Sort [codegen id : 36] +(55) Sort [codegen id : 36] Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, (rn#50 - 1) ASC NULLS FIRST], false, 0 +>>>>>>> regen (54) SortMergeJoin [codegen id : 37] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] Right keys [5]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1)] Join condition: None +<<<<<<< HEAD (55) Project [codegen id : 37] +Output [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#38 AS psum#51, sum_sales#48 AS nsum#52] +Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] + +(56) TakeOrderedAndProject +Input [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] +======= +(57) Project [codegen id : 37] Output [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#39 AS psum#52, sum_sales#49 AS nsum#53] Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39, i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] -(56) TakeOrderedAndProject +(58) TakeOrderedAndProject Input [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] +>>>>>>> regen ===== Subqueries ===== 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 734eec1be3565..9017cd21a3873 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 @@ -194,7 +194,11 @@ Arguments: [rank(d_year#11, d_moy#12) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [9]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27] +<<<<<<< HEAD Condition : ((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#25)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +======= +Condition : (((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#25)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#27)) +>>>>>>> regen (33) ReusedExchange [Reuses operator id: unknown] Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34] @@ -218,54 +222,107 @@ Arguments: [i_category#28 ASC NULLS FIRST, i_brand#29 ASC NULLS FIRST, s_store_n 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] +<<<<<<< HEAD (38) Project [codegen id : 15] +Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] +Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] + +(39) BroadcastExchange +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] +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=#39] +======= +(38) Filter [codegen id : 15] +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] +Condition : isnotnull(rn#39) + +(39) Project [codegen id : 15] 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] -(39) BroadcastExchange +(40) BroadcastExchange Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -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=#40] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] + 1)),false), [id=#40] +>>>>>>> regen (40) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] Join condition: None +<<<<<<< HEAD (41) Project [codegen id : 23] +Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] +Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] + +(42) ReusedExchange [Reuses operator id: 35] +Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] + +(43) Sort [codegen id : 21] +Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#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, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 + +(44) Window +Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] +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#47], [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] + +(45) Project [codegen id : 22] +Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] +Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] + +(46) BroadcastExchange +Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] +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=#48] +======= +(42) Project [codegen id : 23] Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -(42) ReusedExchange [Reuses operator id: 35] +(43) ReusedExchange [Reuses operator id: 35] 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] -(43) Sort [codegen id : 21] +(44) Sort [codegen id : 21] 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 -(44) Window +(45) 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] -(45) Project [codegen id : 22] +(46) Filter [codegen id : 22] +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] +Condition : isnotnull(rn#48) + +(47) Project [codegen id : 22] 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] -(46) BroadcastExchange +(48) BroadcastExchange Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -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=#49] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] - 1)),false), [id=#49] +>>>>>>> regen (47) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] Join condition: None +<<<<<<< HEAD (48) Project [codegen id : 23] +Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#49, sum_sales#46 AS nsum#50] +Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] + +(49) TakeOrderedAndProject +Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, 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#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#22, psum#49, nsum#50] +======= +(50) Project [codegen id : 23] Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -(49) TakeOrderedAndProject +(51) TakeOrderedAndProject Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, 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#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#22, psum#50, nsum#51] +>>>>>>> regen ===== Subqueries ===== 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 0a8fdb66edb77..957e2d9a015b4 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 @@ -163,27 +163,55 @@ Arguments: hashpartitioning(item_sk#13, 5), ENSURE_REQUIREMENTS, [id=#17] Input [4]: [item_sk#13, d_date#6, sumws#14, rk#16] Arguments: [item_sk#13 ASC NULLS FIRST], false, 0 +<<<<<<< HEAD (20) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] +Output [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] (21) Sort [codegen id : 10] +Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] +Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST], false, 0 + +(22) Window +Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#21], [ws_item_sk#1], [d_date#19 ASC NULLS FIRST] + +(23) Project [codegen id : 11] +Output [3]: [item_sk#18, sumws#20, rk#21] +Input [5]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1, rk#21] + +(24) Exchange +Input [3]: [item_sk#18, sumws#20, rk#21] +Arguments: hashpartitioning(item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#22] + +(25) Sort [codegen id : 12] +Input [3]: [item_sk#18, sumws#20, rk#21] +======= +(21) ReusedExchange [Reuses operator id: unknown] +Output [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] + +(22) Sort [codegen id : 10] Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] Arguments: [ws_item_sk#21 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST], false, 0 -(22) Window +(23) Window Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] Arguments: [row_number() windowspecdefinition(ws_item_sk#21, d_date#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#22], [ws_item_sk#21], [d_date#19 ASC NULLS FIRST] -(23) Project [codegen id : 11] +(24) Filter [codegen id : 11] +Input [5]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21, rk#22] +Condition : isnotnull(rk#22) + +(25) Project [codegen id : 11] Output [3]: [item_sk#18, sumws#20, rk#22] Input [5]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21, rk#22] -(24) Exchange +(26) Exchange Input [3]: [item_sk#18, sumws#20, rk#22] Arguments: hashpartitioning(item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#23] -(25) Sort [codegen id : 12] +(27) Sort [codegen id : 12] Input [3]: [item_sk#18, sumws#20, rk#22] +>>>>>>> regen Arguments: [item_sk#18 ASC NULLS FIRST], false, 0 (26) SortMergeJoin [codegen id : 13] @@ -202,214 +230,433 @@ Functions [1]: [partial_sum(sumws#20)] Aggregate Attributes [2]: [sum#24, isEmpty#25] Results [5]: [item_sk#13, d_date#6, sumws#14, sum#26, isEmpty#27] +<<<<<<< HEAD (29) HashAggregate [codegen id : 13] +Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +======= +(31) HashAggregate [codegen id : 13] Input [5]: [item_sk#13, d_date#6, sumws#14, sum#26, isEmpty#27] +>>>>>>> regen Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [sum(sumws#20)] Aggregate Attributes [1]: [sum(sumws#20)#28] Results [3]: [item_sk#13, d_date#6, sum(sumws#20)#28 AS cume_sales#29] +<<<<<<< HEAD (30) Exchange +Input [3]: [item_sk#13, d_date#6, cume_sales#28] +Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#29] + +(31) Sort [codegen id : 14] +Input [3]: [item_sk#13, d_date#6, cume_sales#28] +Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 + +(32) Scan parquet default.store_sales +Output [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +======= +(32) Exchange Input [3]: [item_sk#13, d_date#6, cume_sales#29] Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#30] -(31) Sort [codegen id : 14] +(33) Sort [codegen id : 14] Input [3]: [item_sk#13, d_date#6, cume_sales#29] Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 -(32) Scan parquet default.store_sales +(34) Scan parquet default.store_sales Output [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] +>>>>>>> regen Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#33), dynamicpruningexpression(ss_sold_date_sk#33 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct +<<<<<<< HEAD (33) ColumnarToRow [codegen id : 16] -Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] +Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] (34) Filter [codegen id : 16] +Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +Condition : isnotnull(ss_item_sk#30) + +(35) ReusedExchange [Reuses operator id: 8] +Output [2]: [d_date_sk#33, d_date#34] + +(36) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ss_sold_date_sk#32] +Right keys [1]: [d_date_sk#33] +Join condition: None + +(37) Project [codegen id : 16] +Output [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] +Input [5]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32, d_date_sk#33, d_date#34] + +(38) HashAggregate [codegen id : 16] +Input [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] +Keys [2]: [ss_item_sk#30, d_date#34] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#31))] +Aggregate Attributes [1]: [sum#35] +Results [3]: [ss_item_sk#30, d_date#34, sum#36] + +(39) Exchange +Input [3]: [ss_item_sk#30, d_date#34, sum#36] +Arguments: hashpartitioning(ss_item_sk#30, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#37] + +(40) HashAggregate [codegen id : 17] +Input [3]: [ss_item_sk#30, d_date#34, sum#36] +Keys [2]: [ss_item_sk#30, d_date#34] +Functions [1]: [sum(UnscaledValue(ss_sales_price#31))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#31))#38] +Results [4]: [ss_item_sk#30 AS item_sk#39, d_date#34, MakeDecimal(sum(UnscaledValue(ss_sales_price#31))#38,17,2) AS sumss#40, ss_item_sk#30] + +(41) Exchange +Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] +Arguments: hashpartitioning(ss_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#41] + +(42) Sort [codegen id : 18] +Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] +Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 + +(43) Window +Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] +Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#42], [ss_item_sk#30], [d_date#34 ASC NULLS FIRST] + +(44) Project [codegen id : 19] +Output [4]: [item_sk#39, d_date#34, sumss#40, rk#42] +Input [5]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30, rk#42] + +(45) Exchange +Input [4]: [item_sk#39, d_date#34, sumss#40, rk#42] +Arguments: hashpartitioning(item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#43] + +(46) Sort [codegen id : 20] +Input [4]: [item_sk#39, d_date#34, sumss#40, rk#42] +Arguments: [item_sk#39 ASC NULLS FIRST], false, 0 + +(47) ReusedExchange [Reuses operator id: unknown] +Output [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30] + +(48) Sort [codegen id : 24] +Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30] +Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], false, 0 + +(49) Window +Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30] +Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#47], [ss_item_sk#30], [d_date#45 ASC NULLS FIRST] + +(50) Project [codegen id : 25] +Output [3]: [item_sk#44, sumss#46, rk#47] +Input [5]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30, rk#47] + +(51) Exchange +Input [3]: [item_sk#44, sumss#46, rk#47] +Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, [id=#48] + +(52) Sort [codegen id : 26] +Input [3]: [item_sk#44, sumss#46, rk#47] +Arguments: [item_sk#44 ASC NULLS FIRST], false, 0 + +(53) SortMergeJoin [codegen id : 27] +Left keys [1]: [item_sk#39] +Right keys [1]: [item_sk#44] +Join condition: (rk#42 >= rk#47) + +(54) Project [codegen id : 27] +Output [4]: [item_sk#39, d_date#34, sumss#40, sumss#46] +Input [7]: [item_sk#39, d_date#34, sumss#40, rk#42, item_sk#44, sumss#46, rk#47] + +(55) HashAggregate [codegen id : 27] +Input [4]: [item_sk#39, d_date#34, sumss#40, sumss#46] +Keys [3]: [item_sk#39, d_date#34, sumss#40] +Functions [1]: [partial_sum(sumss#46)] +Aggregate Attributes [2]: [sum#49, isEmpty#50] +Results [5]: [item_sk#39, d_date#34, sumss#40, sum#51, isEmpty#52] + +(56) HashAggregate [codegen id : 27] +Input [5]: [item_sk#39, d_date#34, sumss#40, sum#51, isEmpty#52] +Keys [3]: [item_sk#39, d_date#34, sumss#40] +Functions [1]: [sum(sumss#46)] +Aggregate Attributes [1]: [sum(sumss#46)#53] +Results [3]: [item_sk#39, d_date#34, sum(sumss#46)#53 AS cume_sales#54] + +(57) Exchange +Input [3]: [item_sk#39, d_date#34, cume_sales#54] +Arguments: hashpartitioning(item_sk#39, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#55] + +(58) Sort [codegen id : 28] +Input [3]: [item_sk#39, d_date#34, cume_sales#54] +Arguments: [item_sk#39 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 +======= +(35) ColumnarToRow [codegen id : 16] +Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] + +(36) Filter [codegen id : 16] Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] Condition : isnotnull(ss_item_sk#31) -(35) ReusedExchange [Reuses operator id: 8] +(37) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#34, d_date#35] -(36) BroadcastHashJoin [codegen id : 16] +(38) BroadcastHashJoin [codegen id : 16] Left keys [1]: [ss_sold_date_sk#33] Right keys [1]: [d_date_sk#34] Join condition: None -(37) Project [codegen id : 16] +(39) Project [codegen id : 16] Output [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] Input [5]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33, d_date_sk#34, d_date#35] -(38) HashAggregate [codegen id : 16] +(40) HashAggregate [codegen id : 16] Input [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] Keys [2]: [ss_item_sk#31, d_date#35] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#32))] Aggregate Attributes [1]: [sum#36] Results [3]: [ss_item_sk#31, d_date#35, sum#37] -(39) Exchange +(41) Exchange Input [3]: [ss_item_sk#31, d_date#35, sum#37] Arguments: hashpartitioning(ss_item_sk#31, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#38] -(40) HashAggregate [codegen id : 17] +(42) HashAggregate [codegen id : 17] Input [3]: [ss_item_sk#31, d_date#35, sum#37] Keys [2]: [ss_item_sk#31, d_date#35] Functions [1]: [sum(UnscaledValue(ss_sales_price#32))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#32))#39] Results [4]: [ss_item_sk#31 AS item_sk#40, d_date#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#32))#39,17,2) AS sumss#41, ss_item_sk#31] -(41) Exchange +(43) Exchange Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#42] -(42) Sort [codegen id : 18] +(44) Sort [codegen id : 18] Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] Arguments: [ss_item_sk#31 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 -(43) Window +(45) Window Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] Arguments: [row_number() windowspecdefinition(ss_item_sk#31, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [ss_item_sk#31], [d_date#35 ASC NULLS FIRST] -(44) Project [codegen id : 19] +(46) Filter [codegen id : 19] +Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] +Condition : isnotnull(rk#43) + +(47) Project [codegen id : 19] Output [4]: [item_sk#40, d_date#35, sumss#41, rk#43] Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] -(45) Exchange +(48) Exchange Input [4]: [item_sk#40, d_date#35, sumss#41, rk#43] Arguments: hashpartitioning(item_sk#40, 5), ENSURE_REQUIREMENTS, [id=#44] -(46) Sort [codegen id : 20] +(49) Sort [codegen id : 20] Input [4]: [item_sk#40, d_date#35, sumss#41, rk#43] Arguments: [item_sk#40 ASC NULLS FIRST], false, 0 -(47) ReusedExchange [Reuses operator id: unknown] +(50) ReusedExchange [Reuses operator id: unknown] Output [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] -(48) Sort [codegen id : 24] +(51) Sort [codegen id : 24] Input [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] Arguments: [ss_item_sk#48 ASC NULLS FIRST, d_date#46 ASC NULLS FIRST], false, 0 -(49) Window +(52) Window Input [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] Arguments: [row_number() windowspecdefinition(ss_item_sk#48, d_date#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [ss_item_sk#48], [d_date#46 ASC NULLS FIRST] -(50) Project [codegen id : 25] +(53) Filter [codegen id : 25] +Input [5]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48, rk#49] +Condition : isnotnull(rk#49) + +(54) Project [codegen id : 25] Output [3]: [item_sk#45, sumss#47, rk#49] Input [5]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48, rk#49] -(51) Exchange +(55) Exchange Input [3]: [item_sk#45, sumss#47, rk#49] Arguments: hashpartitioning(item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#50] -(52) Sort [codegen id : 26] +(56) Sort [codegen id : 26] Input [3]: [item_sk#45, sumss#47, rk#49] Arguments: [item_sk#45 ASC NULLS FIRST], false, 0 -(53) SortMergeJoin [codegen id : 27] +(57) SortMergeJoin [codegen id : 27] Left keys [1]: [item_sk#40] Right keys [1]: [item_sk#45] Join condition: (rk#43 >= rk#49) -(54) Project [codegen id : 27] +(58) Project [codegen id : 27] Output [4]: [item_sk#40, d_date#35, sumss#41, sumss#47] Input [7]: [item_sk#40, d_date#35, sumss#41, rk#43, item_sk#45, sumss#47, rk#49] -(55) HashAggregate [codegen id : 27] +(59) HashAggregate [codegen id : 27] Input [4]: [item_sk#40, d_date#35, sumss#41, sumss#47] Keys [3]: [item_sk#40, d_date#35, sumss#41] Functions [1]: [partial_sum(sumss#47)] Aggregate Attributes [2]: [sum#51, isEmpty#52] Results [5]: [item_sk#40, d_date#35, sumss#41, sum#53, isEmpty#54] -(56) HashAggregate [codegen id : 27] +(60) HashAggregate [codegen id : 27] Input [5]: [item_sk#40, d_date#35, sumss#41, sum#53, isEmpty#54] Keys [3]: [item_sk#40, d_date#35, sumss#41] Functions [1]: [sum(sumss#47)] Aggregate Attributes [1]: [sum(sumss#47)#55] Results [3]: [item_sk#40, d_date#35, sum(sumss#47)#55 AS cume_sales#56] -(57) Exchange +(61) Exchange Input [3]: [item_sk#40, d_date#35, cume_sales#56] Arguments: hashpartitioning(item_sk#40, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#57] -(58) Sort [codegen id : 28] +(62) Sort [codegen id : 28] Input [3]: [item_sk#40, d_date#35, cume_sales#56] Arguments: [item_sk#40 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 +>>>>>>> regen (59) SortMergeJoin Left keys [2]: [item_sk#13, d_date#6] Right keys [2]: [item_sk#40, d_date#35] Join condition: None +<<<<<<< HEAD (60) Filter [codegen id : 29] +Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END) + +(61) Project [codegen id : 29] +Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END AS item_sk#56, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#34 END AS d_date#57, cume_sales#28 AS web_sales#58, cume_sales#54 AS store_sales#59] +Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] + +(62) Exchange +Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] +Arguments: hashpartitioning(item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#60] + +(63) Sort [codegen id : 30] +Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] +Arguments: [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], false, 0 + +(64) Window +Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] +Arguments: [row_number() windowspecdefinition(item_sk#56, d_date#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#61], [item_sk#56], [d_date#57 ASC NULLS FIRST] + +(65) ReusedExchange [Reuses operator id: unknown] +Output [4]: [item_sk#62, d_date#63, web_sales#64, store_sales#65] + +(66) 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 + +(67) 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] + +(68) 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] + +(69) SortMergeJoin [codegen id : 62] +Left keys [1]: [item_sk#56] +Right keys [1]: [item_sk#62] +Join condition: (rk#61 >= rk#66) + +(70) 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] + +(71) 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] + +(72) HashAggregate [codegen id : 62] +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)#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] + +(73) 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)) + +(74) 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] +======= +(64) Filter [codegen id : 29] Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END) -(61) Project [codegen id : 29] +(65) Project [codegen id : 29] Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END AS item_sk#58, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#35 END AS d_date#59, cume_sales#29 AS web_sales#60, cume_sales#56 AS store_sales#61] Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] -(62) Exchange +(66) Exchange Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Arguments: hashpartitioning(item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#62] -(63) Sort [codegen id : 30] +(67) Sort [codegen id : 30] Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Arguments: [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], false, 0 -(64) Window +(68) Window Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Arguments: [row_number() windowspecdefinition(item_sk#58, d_date#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#63], [item_sk#58], [d_date#59 ASC NULLS FIRST] -(65) ReusedExchange [Reuses operator id: unknown] +(69) Filter [codegen id : 31] +Input [5]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63] +Condition : isnotnull(rk#63) + +(70) ReusedExchange [Reuses operator id: unknown] Output [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] -(66) Sort [codegen id : 60] +(71) Sort [codegen id : 61] Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] Arguments: [item_sk#64 ASC NULLS FIRST, d_date#65 ASC NULLS FIRST], false, 0 -(67) Window +(72) Window Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] Arguments: [row_number() windowspecdefinition(item_sk#64, d_date#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#68], [item_sk#64], [d_date#65 ASC NULLS FIRST] -(68) Project [codegen id : 61] +(73) Filter [codegen id : 62] +Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] +Condition : isnotnull(rk#68) + +(74) Project [codegen id : 62] Output [4]: [item_sk#64, web_sales#66, store_sales#67, rk#68] Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] -(69) SortMergeJoin [codegen id : 62] +(75) SortMergeJoin [codegen id : 63] Left keys [1]: [item_sk#58] Right keys [1]: [item_sk#64] Join condition: (rk#63 >= rk#68) -(70) Project [codegen id : 62] +(76) Project [codegen id : 63] Output [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] Input [9]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63, item_sk#64, web_sales#66, store_sales#67, rk#68] -(71) HashAggregate [codegen id : 62] +(77) HashAggregate [codegen id : 63] Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Functions [2]: [partial_max(web_sales#66), partial_max(store_sales#67)] Aggregate Attributes [2]: [max#69, max#70] Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#71, max#72] -(72) HashAggregate [codegen id : 62] +(78) HashAggregate [codegen id : 63] Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#71, max#72] Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Functions [2]: [max(web_sales#66), max(store_sales#67)] Aggregate Attributes [2]: [max(web_sales#66)#73, max(store_sales#67)#74] Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max(web_sales#66)#73 AS web_cumulative#75, max(store_sales#67)#74 AS store_cumulative#76] -(73) Filter [codegen id : 62] +(79) Filter [codegen id : 63] Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] Condition : ((isnotnull(web_cumulative#75) AND isnotnull(store_cumulative#76)) AND (web_cumulative#75 > store_cumulative#76)) -(74) TakeOrderedAndProject +(80) TakeOrderedAndProject Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] Arguments: 100, [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] +>>>>>>> regen ===== Subqueries ===== @@ -420,6 +667,10 @@ ReusedExchange (75) (75) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#5, d_date#6] -Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#33 IN dynamicpruning#4 +<<<<<<< HEAD +Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#4 +======= +Subquery:2 Hosting operator id = 34 Hosting Expression = ss_sold_date_sk#33 IN dynamicpruning#4 +>>>>>>> regen 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 b7d3461ce34ea..471fe83a95901 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 @@ -152,24 +152,49 @@ Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS F Output [4]: [item_sk#13, d_date#6, sumws#14, rk#16] Input [5]: [item_sk#13, d_date#6, sumws#14, ws_item_sk#1, rk#16] +<<<<<<< HEAD (18) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] +Output [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] (19) Sort [codegen id : 8] +Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] +Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 + +(20) Window +Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#20], [ws_item_sk#1], [d_date#18 ASC NULLS FIRST] + +(21) Project [codegen id : 9] +Output [3]: [item_sk#17, sumws#19, rk#20] +Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1, rk#20] + +(22) BroadcastExchange +Input [3]: [item_sk#17, sumws#19, rk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +======= +(19) ReusedExchange [Reuses operator id: unknown] +Output [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] + +(20) Sort [codegen id : 8] Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] Arguments: [ws_item_sk#20 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 -(20) Window +(21) Window Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] Arguments: [row_number() windowspecdefinition(ws_item_sk#20, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#21], [ws_item_sk#20], [d_date#18 ASC NULLS FIRST] -(21) Project [codegen id : 9] +(22) Filter [codegen id : 9] +Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20, rk#21] +Condition : isnotnull(rk#21) + +(23) Project [codegen id : 9] Output [3]: [item_sk#17, sumws#19, rk#21] Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20, rk#21] -(22) BroadcastExchange +(24) BroadcastExchange Input [3]: [item_sk#17, sumws#19, rk#21] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +>>>>>>> regen (23) BroadcastHashJoin [codegen id : 10] Left keys [1]: [item_sk#13] @@ -187,214 +212,433 @@ Functions [1]: [partial_sum(sumws#19)] Aggregate Attributes [2]: [sum#23, isEmpty#24] Results [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +<<<<<<< HEAD (26) Exchange +Input [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] +Arguments: hashpartitioning(item_sk#13, d_date#6, sumws#14, 5), ENSURE_REQUIREMENTS, [id=#26] + +(27) HashAggregate [codegen id : 11] +Input [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] +======= +(28) Exchange Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] Arguments: hashpartitioning(item_sk#13, d_date#6, sumws#14, 5), ENSURE_REQUIREMENTS, [id=#27] -(27) HashAggregate [codegen id : 11] +(29) HashAggregate [codegen id : 11] Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +>>>>>>> regen Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [sum(sumws#19)] Aggregate Attributes [1]: [sum(sumws#19)#28] Results [3]: [item_sk#13, d_date#6, sum(sumws#19)#28 AS cume_sales#29] +<<<<<<< HEAD (28) Exchange +Input [3]: [item_sk#13, d_date#6, cume_sales#28] +Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#29] + +(29) Sort [codegen id : 12] +Input [3]: [item_sk#13, d_date#6, cume_sales#28] +Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 + +(30) Scan parquet default.store_sales +Output [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +======= +(30) Exchange Input [3]: [item_sk#13, d_date#6, cume_sales#29] Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#30] -(29) Sort [codegen id : 12] +(31) Sort [codegen id : 12] Input [3]: [item_sk#13, d_date#6, cume_sales#29] Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 -(30) Scan parquet default.store_sales +(32) Scan parquet default.store_sales Output [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] +>>>>>>> regen Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(ss_sold_date_sk#33), dynamicpruningexpression(ss_sold_date_sk#33 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct +<<<<<<< HEAD (31) ColumnarToRow [codegen id : 14] -Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] +Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] (32) Filter [codegen id : 14] +Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +Condition : isnotnull(ss_item_sk#30) + +(33) ReusedExchange [Reuses operator id: 8] +Output [2]: [d_date_sk#33, d_date#34] + +(34) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ss_sold_date_sk#32] +Right keys [1]: [d_date_sk#33] +Join condition: None + +(35) Project [codegen id : 14] +Output [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] +Input [5]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32, d_date_sk#33, d_date#34] + +(36) HashAggregate [codegen id : 14] +Input [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] +Keys [2]: [ss_item_sk#30, d_date#34] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#31))] +Aggregate Attributes [1]: [sum#35] +Results [3]: [ss_item_sk#30, d_date#34, sum#36] + +(37) Exchange +Input [3]: [ss_item_sk#30, d_date#34, sum#36] +Arguments: hashpartitioning(ss_item_sk#30, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#37] + +(38) HashAggregate [codegen id : 15] +Input [3]: [ss_item_sk#30, d_date#34, sum#36] +Keys [2]: [ss_item_sk#30, d_date#34] +Functions [1]: [sum(UnscaledValue(ss_sales_price#31))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#31))#38] +Results [4]: [ss_item_sk#30 AS item_sk#39, d_date#34, MakeDecimal(sum(UnscaledValue(ss_sales_price#31))#38,17,2) AS sumss#40, ss_item_sk#30] + +(39) Exchange +Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] +Arguments: hashpartitioning(ss_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#41] + +(40) Sort [codegen id : 16] +Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] +Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 + +(41) Window +Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] +Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#42], [ss_item_sk#30], [d_date#34 ASC NULLS FIRST] + +(42) Project [codegen id : 22] +Output [4]: [item_sk#39, d_date#34, sumss#40, rk#42] +Input [5]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30, rk#42] + +(43) ReusedExchange [Reuses operator id: unknown] +Output [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] + +(44) Sort [codegen id : 20] +Input [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] +Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#44 ASC NULLS FIRST], false, 0 + +(45) Window +Input [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] +Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#46], [ss_item_sk#30], [d_date#44 ASC NULLS FIRST] + +(46) Project [codegen id : 21] +Output [3]: [item_sk#43, sumss#45, rk#46] +Input [5]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30, rk#46] + +(47) BroadcastExchange +Input [3]: [item_sk#43, sumss#45, rk#46] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] + +(48) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [item_sk#39] +Right keys [1]: [item_sk#43] +Join condition: (rk#42 >= rk#46) + +(49) Project [codegen id : 22] +Output [4]: [item_sk#39, d_date#34, sumss#40, sumss#45] +Input [7]: [item_sk#39, d_date#34, sumss#40, rk#42, item_sk#43, sumss#45, rk#46] + +(50) HashAggregate [codegen id : 22] +Input [4]: [item_sk#39, d_date#34, sumss#40, sumss#45] +Keys [3]: [item_sk#39, d_date#34, sumss#40] +Functions [1]: [partial_sum(sumss#45)] +Aggregate Attributes [2]: [sum#48, isEmpty#49] +Results [5]: [item_sk#39, d_date#34, sumss#40, sum#50, isEmpty#51] + +(51) Exchange +Input [5]: [item_sk#39, d_date#34, sumss#40, sum#50, isEmpty#51] +Arguments: hashpartitioning(item_sk#39, d_date#34, sumss#40, 5), ENSURE_REQUIREMENTS, [id=#52] + +(52) HashAggregate [codegen id : 23] +Input [5]: [item_sk#39, d_date#34, sumss#40, sum#50, isEmpty#51] +Keys [3]: [item_sk#39, d_date#34, sumss#40] +Functions [1]: [sum(sumss#45)] +Aggregate Attributes [1]: [sum(sumss#45)#53] +Results [3]: [item_sk#39, d_date#34, sum(sumss#45)#53 AS cume_sales#54] + +(53) Exchange +Input [3]: [item_sk#39, d_date#34, cume_sales#54] +Arguments: hashpartitioning(item_sk#39, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#55] + +(54) Sort [codegen id : 24] +Input [3]: [item_sk#39, d_date#34, cume_sales#54] +Arguments: [item_sk#39 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 +======= +(33) ColumnarToRow [codegen id : 14] +Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] + +(34) Filter [codegen id : 14] Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] Condition : isnotnull(ss_item_sk#31) -(33) ReusedExchange [Reuses operator id: 8] +(35) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#34, d_date#35] -(34) BroadcastHashJoin [codegen id : 14] +(36) BroadcastHashJoin [codegen id : 14] Left keys [1]: [ss_sold_date_sk#33] Right keys [1]: [d_date_sk#34] Join condition: None -(35) Project [codegen id : 14] +(37) Project [codegen id : 14] Output [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] Input [5]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33, d_date_sk#34, d_date#35] -(36) HashAggregate [codegen id : 14] +(38) HashAggregate [codegen id : 14] Input [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] Keys [2]: [ss_item_sk#31, d_date#35] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#32))] Aggregate Attributes [1]: [sum#36] Results [3]: [ss_item_sk#31, d_date#35, sum#37] -(37) Exchange +(39) Exchange Input [3]: [ss_item_sk#31, d_date#35, sum#37] Arguments: hashpartitioning(ss_item_sk#31, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#38] -(38) HashAggregate [codegen id : 15] +(40) HashAggregate [codegen id : 15] Input [3]: [ss_item_sk#31, d_date#35, sum#37] Keys [2]: [ss_item_sk#31, d_date#35] Functions [1]: [sum(UnscaledValue(ss_sales_price#32))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#32))#39] Results [4]: [ss_item_sk#31 AS item_sk#40, d_date#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#32))#39,17,2) AS sumss#41, ss_item_sk#31] -(39) Exchange +(41) Exchange Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#42] -(40) Sort [codegen id : 16] +(42) Sort [codegen id : 16] Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] Arguments: [ss_item_sk#31 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 -(41) Window +(43) Window Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] Arguments: [row_number() windowspecdefinition(ss_item_sk#31, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [ss_item_sk#31], [d_date#35 ASC NULLS FIRST] -(42) Project [codegen id : 22] +(44) Filter [codegen id : 22] +Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] +Condition : isnotnull(rk#43) + +(45) Project [codegen id : 22] Output [4]: [item_sk#40, d_date#35, sumss#41, rk#43] Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] -(43) ReusedExchange [Reuses operator id: unknown] +(46) ReusedExchange [Reuses operator id: unknown] Output [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] -(44) Sort [codegen id : 20] +(47) Sort [codegen id : 20] Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] Arguments: [ss_item_sk#47 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], false, 0 -(45) Window +(48) Window Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] Arguments: [row_number() windowspecdefinition(ss_item_sk#47, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [ss_item_sk#47], [d_date#45 ASC NULLS FIRST] -(46) Project [codegen id : 21] +(49) Filter [codegen id : 21] +Input [5]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47, rk#48] +Condition : isnotnull(rk#48) + +(50) Project [codegen id : 21] Output [3]: [item_sk#44, sumss#46, rk#48] Input [5]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47, rk#48] -(47) BroadcastExchange +(51) BroadcastExchange Input [3]: [item_sk#44, sumss#46, rk#48] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] -(48) BroadcastHashJoin [codegen id : 22] +(52) BroadcastHashJoin [codegen id : 22] Left keys [1]: [item_sk#40] Right keys [1]: [item_sk#44] Join condition: (rk#43 >= rk#48) -(49) Project [codegen id : 22] +(53) Project [codegen id : 22] Output [4]: [item_sk#40, d_date#35, sumss#41, sumss#46] Input [7]: [item_sk#40, d_date#35, sumss#41, rk#43, item_sk#44, sumss#46, rk#48] -(50) HashAggregate [codegen id : 22] +(54) HashAggregate [codegen id : 22] Input [4]: [item_sk#40, d_date#35, sumss#41, sumss#46] Keys [3]: [item_sk#40, d_date#35, sumss#41] Functions [1]: [partial_sum(sumss#46)] Aggregate Attributes [2]: [sum#50, isEmpty#51] Results [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] -(51) Exchange +(55) Exchange Input [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] Arguments: hashpartitioning(item_sk#40, d_date#35, sumss#41, 5), ENSURE_REQUIREMENTS, [id=#54] -(52) HashAggregate [codegen id : 23] +(56) HashAggregate [codegen id : 23] Input [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] Keys [3]: [item_sk#40, d_date#35, sumss#41] Functions [1]: [sum(sumss#46)] Aggregate Attributes [1]: [sum(sumss#46)#55] Results [3]: [item_sk#40, d_date#35, sum(sumss#46)#55 AS cume_sales#56] -(53) Exchange +(57) Exchange Input [3]: [item_sk#40, d_date#35, cume_sales#56] Arguments: hashpartitioning(item_sk#40, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#57] -(54) Sort [codegen id : 24] +(58) Sort [codegen id : 24] Input [3]: [item_sk#40, d_date#35, cume_sales#56] Arguments: [item_sk#40 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 +>>>>>>> regen (55) SortMergeJoin Left keys [2]: [item_sk#13, d_date#6] Right keys [2]: [item_sk#40, d_date#35] Join condition: None +<<<<<<< HEAD (56) Filter [codegen id : 25] +Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END) + +(57) Project [codegen id : 25] +Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END AS item_sk#56, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#34 END AS d_date#57, cume_sales#28 AS web_sales#58, cume_sales#54 AS store_sales#59] +Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] + +(58) Exchange +Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] +Arguments: hashpartitioning(item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#60] + +(59) Sort [codegen id : 26] +Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] +Arguments: [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], false, 0 + +(60) Window +Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] +Arguments: [row_number() windowspecdefinition(item_sk#56, d_date#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#61], [item_sk#56], [d_date#57 ASC NULLS FIRST] + +(61) ReusedExchange [Reuses operator id: unknown] +Output [4]: [item_sk#62, d_date#63, web_sales#64, store_sales#65] + +(62) 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 + +(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] + +(64) 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] + +(65) 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] + +(66) BroadcastHashJoin [codegen id : 54] +Left keys [1]: [item_sk#56] +Right keys [1]: [item_sk#62] +Join condition: (rk#61 >= rk#66) + +(67) 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] + +(68) 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] + +(69) HashAggregate [codegen id : 54] +Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#70, max#71] +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] + +(70) 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)) + +(71) 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] +======= +(60) Filter [codegen id : 25] Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END) -(57) Project [codegen id : 25] +(61) Project [codegen id : 25] Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END AS item_sk#58, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#35 END AS d_date#59, cume_sales#29 AS web_sales#60, cume_sales#56 AS store_sales#61] Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] -(58) Exchange +(62) Exchange Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Arguments: hashpartitioning(item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#62] -(59) Sort [codegen id : 26] +(63) Sort [codegen id : 26] Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Arguments: [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], false, 0 -(60) Window +(64) Window Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Arguments: [row_number() windowspecdefinition(item_sk#58, d_date#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#63], [item_sk#58], [d_date#59 ASC NULLS FIRST] -(61) ReusedExchange [Reuses operator id: unknown] +(65) Filter [codegen id : 54] +Input [5]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63] +Condition : isnotnull(rk#63) + +(66) ReusedExchange [Reuses operator id: unknown] Output [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] -(62) Sort [codegen id : 52] +(67) Sort [codegen id : 52] Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] Arguments: [item_sk#64 ASC NULLS FIRST, d_date#65 ASC NULLS FIRST], false, 0 -(63) Window +(68) Window Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] Arguments: [row_number() windowspecdefinition(item_sk#64, d_date#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#68], [item_sk#64], [d_date#65 ASC NULLS FIRST] -(64) Project [codegen id : 53] +(69) Filter [codegen id : 53] +Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] +Condition : isnotnull(rk#68) + +(70) Project [codegen id : 53] Output [4]: [item_sk#64, web_sales#66, store_sales#67, rk#68] Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] -(65) BroadcastExchange +(71) BroadcastExchange Input [4]: [item_sk#64, web_sales#66, store_sales#67, rk#68] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] -(66) BroadcastHashJoin [codegen id : 54] +(72) BroadcastHashJoin [codegen id : 54] Left keys [1]: [item_sk#58] Right keys [1]: [item_sk#64] Join condition: (rk#63 >= rk#68) -(67) Project [codegen id : 54] +(73) Project [codegen id : 54] Output [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] Input [9]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63, item_sk#64, web_sales#66, store_sales#67, rk#68] -(68) HashAggregate [codegen id : 54] +(74) HashAggregate [codegen id : 54] Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Functions [2]: [partial_max(web_sales#66), partial_max(store_sales#67)] Aggregate Attributes [2]: [max#70, max#71] Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#72, max#73] -(69) HashAggregate [codegen id : 54] +(75) HashAggregate [codegen id : 54] Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#72, max#73] Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] Functions [2]: [max(web_sales#66), max(store_sales#67)] Aggregate Attributes [2]: [max(web_sales#66)#74, max(store_sales#67)#75] Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max(web_sales#66)#74 AS web_cumulative#76, max(store_sales#67)#75 AS store_cumulative#77] -(70) Filter [codegen id : 54] +(76) Filter [codegen id : 54] Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] Condition : ((isnotnull(web_cumulative#76) AND isnotnull(store_cumulative#77)) AND (web_cumulative#76 > store_cumulative#77)) -(71) TakeOrderedAndProject +(77) TakeOrderedAndProject Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] Arguments: 100, [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] +>>>>>>> regen ===== Subqueries ===== @@ -405,6 +649,10 @@ ReusedExchange (72) (72) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#5, d_date#6] -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#33 IN dynamicpruning#4 +<<<<<<< HEAD +Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#4 +======= +Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#33 IN dynamicpruning#4 +>>>>>>> regen 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 e3de4a7a053b1..8d2776513f90b 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 @@ -213,7 +213,11 @@ Arguments: [rank(d_year#7, d_moy#8) windowspecdefinition(i_category#16, i_brand# (35) Filter [codegen id : 12] Input [8]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27] +<<<<<<< HEAD Condition : ((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#25)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +======= +Condition : (((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#25)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#27)) +>>>>>>> regen (36) Exchange Input [8]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27] @@ -245,62 +249,123 @@ Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#3 Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +<<<<<<< HEAD (43) Project [codegen id : 23] +Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] + +(44) Exchange +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#38 + 1), 5), ENSURE_REQUIREMENTS, [id=#39] + +(45) Sort [codegen id : 24] +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] +Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#38 + 1) ASC NULLS FIRST], false, 0 +======= +(43) Filter [codegen id : 23] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] +Condition : isnotnull(rn#39) + +(44) Project [codegen id : 23] Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] -(44) Exchange +(45) Exchange Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] -(45) Sort [codegen id : 24] +(46) Sort [codegen id : 24] Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 +>>>>>>> regen (46) SortMergeJoin [codegen id : 25] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#39 + 1)] Join condition: None +<<<<<<< HEAD (47) Project [codegen id : 25] +Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] +Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] + +(48) ReusedExchange [Reuses operator id: 40] +Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] + +(49) Sort [codegen id : 34] +Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 + +(50) Window +Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] + +(51) Project [codegen id : 35] +Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] +Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] + +(52) Exchange +Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] +Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (rn#46 - 1), 5), ENSURE_REQUIREMENTS, [id=#47] + +(53) Sort [codegen id : 36] +Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 +======= +(48) Project [codegen id : 25] Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] -(48) ReusedExchange [Reuses operator id: 40] +(49) ReusedExchange [Reuses operator id: 40] Output [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] -(49) Sort [codegen id : 34] +(50) Sort [codegen id : 34] Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 -(50) Window +(51) Window Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#41, i_brand#42, cc_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#41, i_brand#42, cc_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] -(51) Project [codegen id : 35] +(52) Filter [codegen id : 35] +Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] +Condition : isnotnull(rn#47) + +(53) Project [codegen id : 35] Output [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] -(52) Exchange +(54) Exchange Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] Arguments: hashpartitioning(i_category#41, i_brand#42, cc_name#43, (rn#47 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] -(53) Sort [codegen id : 36] +(55) Sort [codegen id : 36] Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, (rn#47 - 1) ASC NULLS FIRST], false, 0 +>>>>>>> regen (54) SortMergeJoin [codegen id : 37] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] Right keys [4]: [i_category#41, i_brand#42, cc_name#43, (rn#47 - 1)] Join condition: None +<<<<<<< HEAD (55) Project [codegen id : 37] +Output [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#48, sum_sales#45 AS nsum#49] +Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] + +(56) TakeOrderedAndProject +Input [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 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#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] +======= +(57) Project [codegen id : 37] Output [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#49, sum_sales#46 AS nsum#50] Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] -(56) TakeOrderedAndProject +(58) TakeOrderedAndProject Input [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] +>>>>>>> regen ===== Subqueries ===== 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 a6742cf4ab1cf..f1956e47f453e 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 @@ -194,7 +194,11 @@ Arguments: [rank(d_year#11, d_moy#12) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [8]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26] +<<<<<<< HEAD Condition : ((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#24)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) +======= +Condition : (((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#24)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#26)) +>>>>>>> regen (33) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] @@ -218,54 +222,107 @@ Arguments: [i_category#27 ASC NULLS FIRST, i_brand#28 ASC NULLS FIRST, cc_name#2 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] +<<<<<<< HEAD (38) Project [codegen id : 15] +Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] +Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34, rn#36] + +(39) BroadcastExchange +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#37] +======= +(38) Filter [codegen id : 15] +Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] +Condition : isnotnull(rn#37) + +(39) Project [codegen id : 15] 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] -(39) BroadcastExchange +(40) BroadcastExchange Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] + 1)),false), [id=#38] +>>>>>>> regen (40) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] Join condition: None +<<<<<<< HEAD (41) Project [codegen id : 23] +Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34] +Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] + +(42) ReusedExchange [Reuses operator id: 35] +Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] + +(43) Sort [codegen id : 21] +Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] +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 + +(44) Window +Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] +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#44], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] + +(45) Project [codegen id : 22] +Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] +Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43, rn#44] + +(46) BroadcastExchange +Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#45] +======= +(42) Project [codegen id : 23] Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35] Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -(42) ReusedExchange [Reuses operator id: 35] +(43) ReusedExchange [Reuses operator id: 35] Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] -(43) Sort [codegen id : 21] +(44) Sort [codegen id : 21] 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 -(44) Window +(45) 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] -(45) Project [codegen id : 22] +(46) Filter [codegen id : 22] +Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] +Condition : isnotnull(rn#45) + +(47) Project [codegen id : 22] 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] -(46) BroadcastExchange +(48) BroadcastExchange Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#46] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] - 1)),false), [id=#46] +>>>>>>> regen (47) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] Join condition: None +<<<<<<< HEAD (48) Project [codegen id : 23] +Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#34 AS psum#46, sum_sales#43 AS nsum#47] +Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34, i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] + +(49) TakeOrderedAndProject +Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, 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#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#21, psum#46, nsum#47] +======= +(50) Project [codegen id : 23] Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -(49) TakeOrderedAndProject +(51) TakeOrderedAndProject Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, 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#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#21, psum#47, nsum#48] +>>>>>>> regen ===== Subqueries ===== 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 af8035e67b061..13bcda00cb11d 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 @@ -445,7 +445,7 @@ Arguments: [rank(sumsales#29) windowspecdefinition(i_category#20, sumsales#29 DE (73) Filter [codegen id : 82] Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#228] -Condition : (rk#228 <= 100) +Condition : (isnotnull(rk#228) AND (rk#228 <= 100)) (74) TakeOrderedAndProject Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#228] 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 e10a5ea0aff0b..ffa5e96fad67e 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 @@ -430,7 +430,7 @@ Arguments: [rank(sumsales#28) windowspecdefinition(i_category#19, sumsales#28 DE (70) Filter [codegen id : 55] Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#227] -Condition : (rk#227 <= 100) +Condition : (isnotnull(rk#227) AND (rk#227 <= 100)) (71) TakeOrderedAndProject Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#227] 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 35ed13e84c742..d99363440b2bf 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 @@ -207,7 +207,7 @@ Arguments: [rank(_w2#23) windowspecdefinition(s_state#16, _w2#23 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] -Condition : (ranking#25 <= 5) +Condition : (isnotnull(ranking#25) AND (ranking#25 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#22] 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 df0ee68806062..159494182ba83 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 @@ -207,7 +207,7 @@ Arguments: [rank(_w2#23) windowspecdefinition(s_state#15, _w2#23 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] -Condition : (ranking#25 <= 5) +Condition : (isnotnull(ranking#25) AND (ranking#25 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#22] From eb23ddf20b9d8671f7d8dd0d5837d5285f1cf384 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 23 Mar 2021 17:23:51 +0800 Subject: [PATCH 21/38] Revert "regen" This reverts commit 682a2081926dbae97cf6c17b12d3866e92081525. --- .../q10.sf100/explain.txt | 102 +- .../approved-plans-modified/q10/explain.txt | 100 +- .../q27.sf100/explain.txt | 162 +-- .../approved-plans-modified/q27/explain.txt | 162 +-- .../q34.sf100/explain.txt | 4 +- .../approved-plans-modified/q34/explain.txt | 4 +- .../q46.sf100/explain.txt | 18 +- .../approved-plans-modified/q46/explain.txt | 14 +- .../q53.sf100/explain.txt | 6 +- .../approved-plans-modified/q53/explain.txt | 6 +- .../q59.sf100/explain.txt | 102 +- .../approved-plans-modified/q59/explain.txt | 102 +- .../q63.sf100/explain.txt | 6 +- .../approved-plans-modified/q63/explain.txt | 6 +- .../q65.sf100/explain.txt | 86 +- .../approved-plans-modified/q65/explain.txt | 48 +- .../q68.sf100/explain.txt | 20 +- .../approved-plans-modified/q68/explain.txt | 14 +- .../q7.sf100/explain.txt | 4 +- .../approved-plans-modified/q7/explain.txt | 4 +- .../q73.sf100/explain.txt | 4 +- .../approved-plans-modified/q73/explain.txt | 4 +- .../q89.sf100/explain.txt | 6 +- .../approved-plans-modified/q89/explain.txt | 6 +- .../q98.sf100/explain.txt | 6 +- .../approved-plans-modified/q98/explain.txt | 6 +- .../approved-plans-v1_4/q1.sf100/explain.txt | 122 +- .../approved-plans-v1_4/q1/explain.txt | 116 +- .../approved-plans-v1_4/q10.sf100/explain.txt | 104 +- .../approved-plans-v1_4/q10/explain.txt | 94 +- .../approved-plans-v1_4/q11.sf100/explain.txt | 272 ++--- .../approved-plans-v1_4/q11/explain.txt | 254 ++-- .../approved-plans-v1_4/q12.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q12/explain.txt | 6 +- .../approved-plans-v1_4/q13.sf100/explain.txt | 8 +- .../approved-plans-v1_4/q13/explain.txt | 8 +- .../q14a.sf100/explain.txt | 602 ++++----- .../approved-plans-v1_4/q14a/explain.txt | 536 ++++---- .../q14b.sf100/explain.txt | 542 ++++----- .../approved-plans-v1_4/q14b/explain.txt | 488 ++++---- .../approved-plans-v1_4/q16.sf100/explain.txt | 128 +- .../approved-plans-v1_4/q16/explain.txt | 128 +- .../approved-plans-v1_4/q17.sf100/explain.txt | 8 +- .../approved-plans-v1_4/q17/explain.txt | 8 +- .../approved-plans-v1_4/q18.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q18/explain.txt | 4 +- .../approved-plans-v1_4/q2.sf100/explain.txt | 52 +- .../approved-plans-v1_4/q2/explain.txt | 52 +- .../approved-plans-v1_4/q20.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q20/explain.txt | 6 +- .../approved-plans-v1_4/q21.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q21/explain.txt | 2 +- .../q23a.sf100/explain.txt | 362 +++--- .../approved-plans-v1_4/q23a/explain.txt | 274 ++--- .../q23b.sf100/explain.txt | 528 ++++---- .../approved-plans-v1_4/q23b/explain.txt | 404 +++--- .../q24a.sf100/explain.txt | 210 ++-- .../approved-plans-v1_4/q24a/explain.txt | 174 +-- .../q24b.sf100/explain.txt | 210 ++-- .../approved-plans-v1_4/q24b/explain.txt | 174 +-- .../approved-plans-v1_4/q26.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q26/explain.txt | 4 +- .../approved-plans-v1_4/q27.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q27/explain.txt | 4 +- .../approved-plans-v1_4/q28.sf100/explain.txt | 300 ++--- .../approved-plans-v1_4/q28/explain.txt | 300 ++--- .../approved-plans-v1_4/q30.sf100/explain.txt | 142 +-- .../approved-plans-v1_4/q30/explain.txt | 130 +- .../approved-plans-v1_4/q31.sf100/explain.txt | 430 +++---- .../approved-plans-v1_4/q31/explain.txt | 362 +++--- .../approved-plans-v1_4/q32.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q32/explain.txt | 2 +- .../approved-plans-v1_4/q34.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q34/explain.txt | 4 +- .../approved-plans-v1_4/q35.sf100/explain.txt | 116 +- .../approved-plans-v1_4/q35/explain.txt | 96 +- .../approved-plans-v1_4/q37.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q37/explain.txt | 2 +- .../approved-plans-v1_4/q38.sf100/explain.txt | 24 +- .../approved-plans-v1_4/q38/explain.txt | 12 +- .../q39a.sf100/explain.txt | 110 +- .../approved-plans-v1_4/q39a/explain.txt | 106 +- .../q39b.sf100/explain.txt | 110 +- .../approved-plans-v1_4/q39b/explain.txt | 106 +- .../approved-plans-v1_4/q4.sf100/explain.txt | 442 +++---- .../approved-plans-v1_4/q4/explain.txt | 412 +++---- .../approved-plans-v1_4/q44.sf100/explain.txt | 118 +- .../approved-plans-v1_4/q44/explain.txt | 118 +- .../approved-plans-v1_4/q45.sf100/explain.txt | 36 +- .../approved-plans-v1_4/q45/explain.txt | 36 +- .../approved-plans-v1_4/q46.sf100/explain.txt | 44 +- .../approved-plans-v1_4/q46/explain.txt | 14 +- .../approved-plans-v1_4/q47.sf100/explain.txt | 85 +- .../approved-plans-v1_4/q47/explain.txt | 77 +- .../approved-plans-v1_4/q48.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q48/explain.txt | 6 +- .../approved-plans-v1_4/q49.sf100/explain.txt | 198 +-- .../approved-plans-v1_4/q49/explain.txt | 152 +-- .../approved-plans-v1_4/q5.sf100/explain.txt | 164 +-- .../approved-plans-v1_4/q5/explain.txt | 176 +-- .../approved-plans-v1_4/q53.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q53/explain.txt | 6 +- .../approved-plans-v1_4/q54.sf100/explain.txt | 122 +- .../approved-plans-v1_4/q54/explain.txt | 124 +- .../approved-plans-v1_4/q57.sf100/explain.txt | 85 +- .../approved-plans-v1_4/q57/explain.txt | 77 +- .../approved-plans-v1_4/q58.sf100/explain.txt | 226 ++-- .../approved-plans-v1_4/q58/explain.txt | 228 ++-- .../approved-plans-v1_4/q59.sf100/explain.txt | 70 +- .../approved-plans-v1_4/q59/explain.txt | 70 +- .../approved-plans-v1_4/q61.sf100/explain.txt | 84 +- .../approved-plans-v1_4/q61/explain.txt | 94 +- .../approved-plans-v1_4/q63.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q63/explain.txt | 6 +- .../approved-plans-v1_4/q64.sf100/explain.txt | 320 ++--- .../approved-plans-v1_4/q64/explain.txt | 288 ++--- .../approved-plans-v1_4/q65.sf100/explain.txt | 98 +- .../approved-plans-v1_4/q65/explain.txt | 48 +- .../approved-plans-v1_4/q66.sf100/explain.txt | 70 +- .../approved-plans-v1_4/q66/explain.txt | 74 +- .../approved-plans-v1_4/q68.sf100/explain.txt | 44 +- .../approved-plans-v1_4/q68/explain.txt | 14 +- .../approved-plans-v1_4/q69.sf100/explain.txt | 96 +- .../approved-plans-v1_4/q69/explain.txt | 94 +- .../approved-plans-v1_4/q7.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q7/explain.txt | 4 +- .../approved-plans-v1_4/q70.sf100/explain.txt | 136 +-- .../approved-plans-v1_4/q70/explain.txt | 136 +-- .../approved-plans-v1_4/q71.sf100/explain.txt | 82 +- .../approved-plans-v1_4/q71/explain.txt | 82 +- .../approved-plans-v1_4/q72.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q72/explain.txt | 4 +- .../approved-plans-v1_4/q73.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q73/explain.txt | 4 +- .../approved-plans-v1_4/q74.sf100/explain.txt | 268 ++-- .../approved-plans-v1_4/q74/explain.txt | 250 ++-- .../approved-plans-v1_4/q75.sf100/explain.txt | 372 +++--- .../approved-plans-v1_4/q75/explain.txt | 372 +++--- .../approved-plans-v1_4/q76.sf100/explain.txt | 80 +- .../approved-plans-v1_4/q76/explain.txt | 64 +- .../approved-plans-v1_4/q77.sf100/explain.txt | 284 ++--- .../approved-plans-v1_4/q77/explain.txt | 300 ++--- .../approved-plans-v1_4/q78.sf100/explain.txt | 146 +-- .../approved-plans-v1_4/q78/explain.txt | 146 +-- .../approved-plans-v1_4/q8.sf100/explain.txt | 120 +- .../approved-plans-v1_4/q8/explain.txt | 106 +- .../approved-plans-v1_4/q80.sf100/explain.txt | 206 ++-- .../approved-plans-v1_4/q80/explain.txt | 210 ++-- .../approved-plans-v1_4/q81.sf100/explain.txt | 144 +-- .../approved-plans-v1_4/q81/explain.txt | 126 +- .../approved-plans-v1_4/q82.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q82/explain.txt | 2 +- .../approved-plans-v1_4/q85.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q85/explain.txt | 6 +- .../approved-plans-v1_4/q87.sf100/explain.txt | 24 +- .../approved-plans-v1_4/q87/explain.txt | 12 +- .../approved-plans-v1_4/q88.sf100/explain.txt | 504 ++++---- .../approved-plans-v1_4/q88/explain.txt | 504 ++++---- .../approved-plans-v1_4/q89.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q89/explain.txt | 6 +- .../approved-plans-v1_4/q9.sf100/explain.txt | 416 +++---- .../approved-plans-v1_4/q9/explain.txt | 416 +++---- .../approved-plans-v1_4/q90.sf100/explain.txt | 80 +- .../approved-plans-v1_4/q90/explain.txt | 80 +- .../approved-plans-v1_4/q91.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q91/explain.txt | 4 +- .../approved-plans-v1_4/q92.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q92/explain.txt | 2 +- .../approved-plans-v1_4/q93.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q93/explain.txt | 4 +- .../approved-plans-v1_4/q94.sf100/explain.txt | 136 +-- .../approved-plans-v1_4/q94/explain.txt | 136 +-- .../approved-plans-v1_4/q95.sf100/explain.txt | 196 +-- .../approved-plans-v1_4/q95/explain.txt | 196 +-- .../approved-plans-v1_4/q97.sf100/explain.txt | 40 +- .../approved-plans-v1_4/q97/explain.txt | 40 +- .../approved-plans-v1_4/q98.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q98/explain.txt | 6 +- .../q10a.sf100/explain.txt | 96 +- .../approved-plans-v2_7/q10a/explain.txt | 94 +- .../approved-plans-v2_7/q11.sf100/explain.txt | 268 ++-- .../approved-plans-v2_7/q11/explain.txt | 250 ++-- .../approved-plans-v2_7/q12.sf100/explain.txt | 6 +- .../approved-plans-v2_7/q12/explain.txt | 6 +- .../approved-plans-v2_7/q14.sf100/explain.txt | 542 ++++----- .../approved-plans-v2_7/q14/explain.txt | 488 ++++---- .../q14a.sf100/explain.txt | 1082 ++++++++--------- .../approved-plans-v2_7/q14a/explain.txt | 1016 ++++++++-------- .../q18a.sf100/explain.txt | 402 +++--- .../approved-plans-v2_7/q18a/explain.txt | 410 +++---- .../approved-plans-v2_7/q20.sf100/explain.txt | 6 +- .../approved-plans-v2_7/q20/explain.txt | 6 +- .../q22a.sf100/explain.txt | 122 +- .../approved-plans-v2_7/q22a/explain.txt | 122 +- .../approved-plans-v2_7/q24.sf100/explain.txt | 186 +-- .../approved-plans-v2_7/q24/explain.txt | 174 +-- .../q27a.sf100/explain.txt | 162 +-- .../approved-plans-v2_7/q27a/explain.txt | 162 +-- .../approved-plans-v2_7/q34.sf100/explain.txt | 4 +- .../approved-plans-v2_7/q34/explain.txt | 4 +- .../approved-plans-v2_7/q35.sf100/explain.txt | 116 +- .../approved-plans-v2_7/q35/explain.txt | 96 +- .../q35a.sf100/explain.txt | 116 +- .../approved-plans-v2_7/q35a/explain.txt | 96 +- .../q36a.sf100/explain.txt | 92 +- .../approved-plans-v2_7/q36a/explain.txt | 92 +- .../approved-plans-v2_7/q47.sf100/explain.txt | 85 +- .../approved-plans-v2_7/q47/explain.txt | 77 +- .../approved-plans-v2_7/q49.sf100/explain.txt | 198 +-- .../approved-plans-v2_7/q49/explain.txt | 152 +-- .../q51a.sf100/explain.txt | 267 +--- .../approved-plans-v2_7/q51a/explain.txt | 264 +--- .../approved-plans-v2_7/q57.sf100/explain.txt | 85 +- .../approved-plans-v2_7/q57/explain.txt | 77 +- .../approved-plans-v2_7/q5a.sf100/explain.txt | 232 ++-- .../approved-plans-v2_7/q5a/explain.txt | 244 ++-- .../approved-plans-v2_7/q64.sf100/explain.txt | 320 ++--- .../approved-plans-v2_7/q64/explain.txt | 288 ++--- .../q67a.sf100/explain.txt | 296 ++--- .../approved-plans-v2_7/q67a/explain.txt | 296 ++--- .../q70a.sf100/explain.txt | 212 ++-- .../approved-plans-v2_7/q70a/explain.txt | 212 ++-- .../approved-plans-v2_7/q72.sf100/explain.txt | 4 +- .../approved-plans-v2_7/q72/explain.txt | 4 +- .../approved-plans-v2_7/q74.sf100/explain.txt | 268 ++-- .../approved-plans-v2_7/q74/explain.txt | 250 ++-- .../approved-plans-v2_7/q75.sf100/explain.txt | 372 +++--- .../approved-plans-v2_7/q75/explain.txt | 372 +++--- .../q77a.sf100/explain.txt | 364 +++--- .../approved-plans-v2_7/q77a/explain.txt | 380 +++--- .../approved-plans-v2_7/q78.sf100/explain.txt | 146 +-- .../approved-plans-v2_7/q78/explain.txt | 146 +-- .../q80a.sf100/explain.txt | 270 ++-- .../approved-plans-v2_7/q80a/explain.txt | 274 ++--- .../q86a.sf100/explain.txt | 92 +- .../approved-plans-v2_7/q86a/explain.txt | 92 +- .../approved-plans-v2_7/q98.sf100/explain.txt | 6 +- .../approved-plans-v2_7/q98/explain.txt | 6 +- 238 files changed, 15890 insertions(+), 16877 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt index 1fdcba1ed4de4..8a4b341ac925b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt @@ -137,22 +137,22 @@ Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] Condition : isnotnull(cs_ship_customer_sk#13) (19) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#8] (20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] +Right keys [1]: [d_date_sk#8] Join condition: None (21) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13 AS customer_sk#16] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] +Output [1]: [cs_ship_customer_sk#13 AS customer_sk#15] +Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#8] (22) Union (23) Exchange Input [1]: [customer_sk#12] -Arguments: hashpartitioning(customer_sk#12, 5), ENSURE_REQUIREMENTS, [id=#17] +Arguments: hashpartitioning(customer_sk#12, 5), ENSURE_REQUIREMENTS, [id=#16] (24) Sort [codegen id : 7] Input [1]: [customer_sk#12] @@ -164,43 +164,43 @@ Right keys [1]: [customer_sk#12] Join condition: None (26) Scan parquet default.store_sales -Output [2]: [ss_customer_sk#18, ss_sold_date_sk#19] +Output [2]: [ss_customer_sk#17, ss_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#19), dynamicpruningexpression(ss_sold_date_sk#19 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 9] -Input [2]: [ss_customer_sk#18, ss_sold_date_sk#19] +Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] (28) Filter [codegen id : 9] -Input [2]: [ss_customer_sk#18, ss_sold_date_sk#19] -Condition : isnotnull(ss_customer_sk#18) +Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] +Condition : isnotnull(ss_customer_sk#17) (29) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#20] +Output [1]: [d_date_sk#8] (30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#19] -Right keys [1]: [d_date_sk#20] +Left keys [1]: [ss_sold_date_sk#18] +Right keys [1]: [d_date_sk#8] Join condition: None (31) Project [codegen id : 9] -Output [1]: [ss_customer_sk#18 AS customer_sk#21] -Input [3]: [ss_customer_sk#18, ss_sold_date_sk#19, d_date_sk#20] +Output [1]: [ss_customer_sk#17 AS customer_sk#19] +Input [3]: [ss_customer_sk#17, ss_sold_date_sk#18, d_date_sk#8] (32) Exchange -Input [1]: [customer_sk#21] -Arguments: hashpartitioning(customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [customer_sk#19] +Arguments: hashpartitioning(customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#20] (33) Sort [codegen id : 10] -Input [1]: [customer_sk#21] -Arguments: [customer_sk#21 ASC NULLS FIRST], false, 0 +Input [1]: [customer_sk#19] +Arguments: [customer_sk#19 ASC NULLS FIRST], false, 0 (34) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#21] +Right keys [1]: [customer_sk#19] Join condition: None (35) Project [codegen id : 12] @@ -208,84 +208,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (36) Scan parquet default.customer_address -Output [2]: [ca_address_sk#23, ca_county#24] +Output [2]: [ca_address_sk#21, ca_county#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 11] -Input [2]: [ca_address_sk#23, ca_county#24] +Input [2]: [ca_address_sk#21, ca_county#22] (38) Filter [codegen id : 11] -Input [2]: [ca_address_sk#23, ca_county#24] -Condition : (ca_county#24 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#23)) +Input [2]: [ca_address_sk#21, ca_county#22] +Condition : (ca_county#22 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#21)) (39) Project [codegen id : 11] -Output [1]: [ca_address_sk#23] -Input [2]: [ca_address_sk#23, ca_county#24] +Output [1]: [ca_address_sk#21] +Input [2]: [ca_address_sk#21, ca_county#22] (40) BroadcastExchange -Input [1]: [ca_address_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] +Input [1]: [ca_address_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#23] +Right keys [1]: [ca_address_sk#21] Join condition: None (42) Project [codegen id : 12] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#23] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21] (43) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (44) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (45) ColumnarToRow -Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (46) Filter -Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -Condition : isnotnull(cd_demo_sk#27) +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Condition : isnotnull(cd_demo_sk#25) (47) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#27] +Right keys [1]: [cd_demo_sk#25] Join condition: None (48) Project [codegen id : 13] -Output [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (49) HashAggregate [codegen id : 13] -Input [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#36] -Results [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] +Aggregate Attributes [1]: [count#34] +Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#35] (50) Exchange -Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] -Arguments: hashpartitioning(cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#35] +Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#36] (51) HashAggregate [codegen id : 14] -Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] -Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#35] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#39] -Results [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, count(1)#39 AS cnt1#40, cd_purchase_estimate#31, count(1)#39 AS cnt2#41, cd_credit_rating#32, count(1)#39 AS cnt3#42, cd_dep_count#33, count(1)#39 AS cnt4#43, cd_dep_employed_count#34, count(1)#39 AS cnt5#44, cd_dep_college_count#35, count(1)#39 AS cnt6#45] +Aggregate Attributes [1]: [count(1)#37] +Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#37 AS cnt1#38, cd_purchase_estimate#29, count(1)#37 AS cnt2#39, cd_credit_rating#30, count(1)#37 AS cnt3#40, cd_dep_count#31, count(1)#37 AS cnt4#41, cd_dep_employed_count#32, count(1)#37 AS cnt5#42, cd_dep_college_count#33, count(1)#37 AS cnt6#43] (52) TakeOrderedAndProject -Input [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#40, cd_purchase_estimate#31, cnt2#41, cd_credit_rating#32, cnt3#42, cd_dep_count#33, cnt4#43, cd_dep_employed_count#34, cnt5#44, cd_dep_college_count#35, cnt6#45] -Arguments: 100, [cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_education_status#30 ASC NULLS FIRST, cd_purchase_estimate#31 ASC NULLS FIRST, cd_credit_rating#32 ASC NULLS FIRST, cd_dep_count#33 ASC NULLS FIRST, cd_dep_employed_count#34 ASC NULLS FIRST, cd_dep_college_count#35 ASC NULLS FIRST], [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#40, cd_purchase_estimate#31, cnt2#41, cd_credit_rating#32, cnt3#42, cd_dep_count#33, cnt4#43, cd_dep_employed_count#34, cnt5#44, cd_dep_college_count#35, cnt6#45] +Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#38, cd_purchase_estimate#29, cnt2#39, cd_credit_rating#30, cnt3#40, cd_dep_count#31, cnt4#41, cd_dep_employed_count#32, cnt5#42, cd_dep_college_count#33, cnt6#43] +Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#38, cd_purchase_estimate#29, cnt2#39, cd_credit_rating#30, cnt3#40, cd_dep_count#31, cnt4#41, cd_dep_employed_count#32, cnt5#42, cd_dep_college_count#33, cnt6#43] ===== Subqueries ===== @@ -298,6 +298,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#19 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt index ff55b1c511456..6f64579d09c78 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt @@ -125,22 +125,22 @@ Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] Condition : isnotnull(cs_ship_customer_sk#12) (17) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#7] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#7] Join condition: None (19) Project [codegen id : 4] -Output [1]: [cs_ship_customer_sk#12 AS customer_sk#15] -Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] +Output [1]: [cs_ship_customer_sk#12 AS customer_sk#14] +Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#7] (20) Union (21) BroadcastExchange Input [1]: [customer_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (22) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] @@ -148,39 +148,39 @@ Right keys [1]: [customer_sk#11] Join condition: None (23) Scan parquet default.store_sales -Output [2]: [ss_customer_sk#17, ss_sold_date_sk#18] +Output [2]: [ss_customer_sk#16, ss_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ss_sold_date_sk#17), dynamicpruningexpression(ss_sold_date_sk#17 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] +Input [2]: [ss_customer_sk#16, ss_sold_date_sk#17] (25) Filter [codegen id : 6] -Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#17) +Input [2]: [ss_customer_sk#16, ss_sold_date_sk#17] +Condition : isnotnull(ss_customer_sk#16) (26) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#19] +Output [1]: [d_date_sk#7] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#18] -Right keys [1]: [d_date_sk#19] +Left keys [1]: [ss_sold_date_sk#17] +Right keys [1]: [d_date_sk#7] Join condition: None (28) Project [codegen id : 6] -Output [1]: [ss_customer_sk#17 AS customer_sk#20] -Input [3]: [ss_customer_sk#17, ss_sold_date_sk#18, d_date_sk#19] +Output [1]: [ss_customer_sk#16 AS customer_sk#18] +Input [3]: [ss_customer_sk#16, ss_sold_date_sk#17, d_date_sk#7] (29) BroadcastExchange -Input [1]: [customer_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [customer_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (30) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#20] +Right keys [1]: [customer_sk#18] Join condition: None (31) Project [codegen id : 9] @@ -188,84 +188,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (32) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_county#23] +Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (33) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#22, ca_county#23] +Input [2]: [ca_address_sk#20, ca_county#21] (34) Filter [codegen id : 7] -Input [2]: [ca_address_sk#22, ca_county#23] -Condition : (ca_county#23 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#22)) +Input [2]: [ca_address_sk#20, ca_county#21] +Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) (35) Project [codegen id : 7] -Output [1]: [ca_address_sk#22] -Input [2]: [ca_address_sk#22, ca_county#23] +Output [1]: [ca_address_sk#20] +Input [2]: [ca_address_sk#20, ca_county#21] (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#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#20] Join condition: None (38) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#22] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] (39) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] (41) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Condition : isnotnull(cd_demo_sk#25) +Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Condition : isnotnull(cd_demo_sk#23) (42) BroadcastExchange -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] (43) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#23] Join condition: None (44) Project [codegen id : 9] -Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] (45) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#35] -Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Aggregate Attributes [1]: [count#33] +Results [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] (46) Exchange -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] +Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#35] (47) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] +Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] +Aggregate Attributes [1]: [count(1)#36] +Results [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#36 AS cnt1#37, cd_purchase_estimate#27, count(1)#36 AS cnt2#38, cd_credit_rating#28, count(1)#36 AS cnt3#39, cd_dep_count#29, count(1)#36 AS cnt4#40, cd_dep_employed_count#30, count(1)#36 AS cnt5#41, cd_dep_college_count#31, count(1)#36 AS cnt6#42] (48) TakeOrderedAndProject -Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] +Input [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] +Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] ===== Subqueries ===== @@ -278,6 +278,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 14 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#17 IN dynamicpruning#6 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 8f7cbc7d0ffbc..baed833e42a47 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +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#61), (ss_sold_date_sk#61 >= 2451545), (ss_sold_date_sk#61 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#61 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 (34) ColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +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] (35) Filter [codegen id : 11] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] -Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) +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)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#62] +Output [1]: [d_date_sk#10] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#61] -Right keys [1]: [d_date_sk#62] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] Join condition: None (38) Project [codegen id : 11] -Output [7]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] -Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#62] +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] (39) Scan parquet default.store -Output [2]: [s_store_sk#63, s_state#64] +Output [2]: [s_store_sk#18, s_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_state, [TN,AL,SD]), IsNotNull(s_store_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 8] -Input [2]: [s_store_sk#63, s_state#64] +Input [2]: [s_store_sk#18, s_state#19] (41) Filter [codegen id : 8] -Input [2]: [s_store_sk#63, s_state#64] -Condition : (s_state#64 IN (TN,AL,SD) AND isnotnull(s_store_sk#63)) +Input [2]: [s_store_sk#18, s_state#19] +Condition : (s_state#19 IN (TN,AL,SD) AND isnotnull(s_store_sk#18)) (42) Project [codegen id : 8] -Output [1]: [s_store_sk#63] -Input [2]: [s_store_sk#63, s_state#64] +Output [1]: [s_store_sk#18] +Input [2]: [s_store_sk#18, s_state#19] (43) BroadcastExchange -Input [1]: [s_store_sk#63] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] +Input [1]: [s_store_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] (44) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#56] -Right keys [1]: [s_store_sk#63] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None (45) Project [codegen id : 11] -Output [6]: [ss_item_sk#54, ss_cdemo_sk#55, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, s_store_sk#63] +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#18] (46) ReusedExchange [Reuses operator id: 15] -Output [1]: [cd_demo_sk#66] +Output [1]: [cd_demo_sk#13] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#55] -Right keys [1]: [cd_demo_sk#66] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#13] Join condition: None (48) Project [codegen id : 11] -Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] -Input [7]: [ss_item_sk#54, ss_cdemo_sk#55, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, cd_demo_sk#66] +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#13] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#67, i_item_id#68] +Output [2]: [i_item_sk#21, i_item_id#22] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#67] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#21] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] -Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] +Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] +Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#68] +Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#22] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] -Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] +Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] (53) Exchange -Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Keys [1]: [i_item_id#68] +Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Keys [1]: [i_item_id#22] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] -Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] +Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] +Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] (55) Scan parquet default.store_sales -Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +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#103), (ss_sold_date_sk#103 >= 2451545), (ss_sold_date_sk#103 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#103 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 (56) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +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] (57) Filter [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) +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)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#104] +Output [1]: [d_date_sk#10] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#103] -Right keys [1]: [d_date_sk#104] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] Join condition: None (60) Project [codegen id : 17] -Output [7]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#104] +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] (61) ReusedExchange [Reuses operator id: 43] -Output [1]: [s_store_sk#105] +Output [1]: [s_store_sk#18] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#98] -Right keys [1]: [s_store_sk#105] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#96, ss_cdemo_sk#97, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#105] +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#18] (64) ReusedExchange [Reuses operator id: 15] -Output [1]: [cd_demo_sk#106] +Output [1]: [cd_demo_sk#13] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#97] -Right keys [1]: [cd_demo_sk#106] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#13] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [7]: [ss_item_sk#96, ss_cdemo_sk#97, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, cd_demo_sk#106] +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#13] (67) Scan parquet default.item -Output [1]: [i_item_sk#107] +Output [1]: [i_item_sk#21] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#107] +Input [1]: [i_item_sk#21] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#107] -Condition : isnotnull(i_item_sk#107) +Input [1]: [i_item_sk#21] +Condition : isnotnull(i_item_sk#21) (70) BroadcastExchange -Input [1]: [i_item_sk#107] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] +Input [1]: [i_item_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#96] -Right keys [1]: [i_item_sk#107] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#21] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] -Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] +Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] +Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] -Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] +Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] (74) Exchange -Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] +Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] -Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] +Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] +Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 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 bfb9e8384817d..735f763ae5d20 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +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#61), (ss_sold_date_sk#61 >= 2451545), (ss_sold_date_sk#61 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#61 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 (34) ColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +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] (35) Filter [codegen id : 11] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] -Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) +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)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#62] +Output [1]: [cd_demo_sk#10] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#55] -Right keys [1]: [cd_demo_sk#62] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (38) Project [codegen id : 11] -Output [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] -Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, cd_demo_sk#62] +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] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#63] +Output [1]: [d_date_sk#15] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#61] -Right keys [1]: [d_date_sk#63] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (41) Project [codegen id : 11] -Output [6]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] -Input [8]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#63] +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] (42) Scan parquet default.store -Output [2]: [s_store_sk#64, s_state#65] +Output [2]: [s_store_sk#18, s_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_state, [TN,AL,SD]), IsNotNull(s_store_sk)] ReadSchema: struct (43) ColumnarToRow [codegen id : 9] -Input [2]: [s_store_sk#64, s_state#65] +Input [2]: [s_store_sk#18, s_state#19] (44) Filter [codegen id : 9] -Input [2]: [s_store_sk#64, s_state#65] -Condition : (s_state#65 IN (TN,AL,SD) AND isnotnull(s_store_sk#64)) +Input [2]: [s_store_sk#18, s_state#19] +Condition : (s_state#19 IN (TN,AL,SD) AND isnotnull(s_store_sk#18)) (45) Project [codegen id : 9] -Output [1]: [s_store_sk#64] -Input [2]: [s_store_sk#64, s_state#65] +Output [1]: [s_store_sk#18] +Input [2]: [s_store_sk#18, s_state#19] (46) BroadcastExchange -Input [1]: [s_store_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#66] +Input [1]: [s_store_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#56] -Right keys [1]: [s_store_sk#64] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None (48) Project [codegen id : 11] -Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] -Input [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, s_store_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_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#18] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#67, i_item_id#68] +Output [2]: [i_item_sk#21, i_item_id#22] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#67] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#21] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] -Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] +Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] +Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#68] +Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#22] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] -Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] +Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] (53) Exchange -Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Keys [1]: [i_item_id#68] +Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Keys [1]: [i_item_id#22] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] -Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] +Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] +Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] (55) Scan parquet default.store_sales -Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +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#103), (ss_sold_date_sk#103 >= 2451545), (ss_sold_date_sk#103 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#103 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 (56) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +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] (57) Filter [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) +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)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#104] +Output [1]: [cd_demo_sk#10] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#97] -Right keys [1]: [cd_demo_sk#104] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (60) Project [codegen id : 17] -Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] +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] (61) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#105] +Output [1]: [d_date_sk#15] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#103] -Right keys [1]: [d_date_sk#105] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#105] +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] (64) ReusedExchange [Reuses operator id: 46] -Output [1]: [s_store_sk#106] +Output [1]: [s_store_sk#18] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#98] -Right keys [1]: [s_store_sk#106] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#106] +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#18] (67) Scan parquet default.item -Output [1]: [i_item_sk#107] +Output [1]: [i_item_sk#21] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#107] +Input [1]: [i_item_sk#21] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#107] -Condition : isnotnull(i_item_sk#107) +Input [1]: [i_item_sk#21] +Condition : isnotnull(i_item_sk#21) (70) BroadcastExchange -Input [1]: [i_item_sk#107] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] +Input [1]: [i_item_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#96] -Right keys [1]: [i_item_sk#107] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#21] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] -Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] +Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] +Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] -Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] +Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] (74) Exchange -Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] +Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] -Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] +Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] +Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#15] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt index ebc6009080bfa..ff33c0b00120b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34.sf100/explain.txt @@ -121,7 +121,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -129,7 +129,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = Unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = Unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt index cd556c0bccb46..5ebb8e180e1e0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q34/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = Unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = Unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt index e62003f82faec..4eb212c336669 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt @@ -261,24 +261,24 @@ Input [7]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_add Arguments: [c_current_addr_sk#36 ASC NULLS FIRST], false, 0 (47) ReusedExchange [Reuses operator id: 30] -Output [2]: [ca_address_sk#41, ca_city#42] +Output [2]: [ca_address_sk#22, ca_city#23] (48) Sort [codegen id : 15] -Input [2]: [ca_address_sk#41, ca_city#42] -Arguments: [ca_address_sk#41 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#22, ca_city#23] +Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#36] -Right keys [1]: [ca_address_sk#41] -Join condition: NOT (ca_city#42 = bought_city#31) +Right keys [1]: [ca_address_sk#22] +Join condition: NOT (ca_city#23 = bought_city#31) (50) Project [codegen id : 16] -Output [7]: [c_last_name#38, c_first_name#37, ca_city#42, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#36, c_first_name#37, c_last_name#38, ca_address_sk#41, ca_city#42] +Output [7]: [c_last_name#38, c_first_name#37, ca_city#23, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#36, c_first_name#37, c_last_name#38, ca_address_sk#22, ca_city#23] (51) TakeOrderedAndProject -Input [7]: [c_last_name#38, c_first_name#37, ca_city#42, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Arguments: 100, [c_last_name#38 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, ca_city#42 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#38, c_first_name#37, ca_city#42, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [7]: [c_last_name#38, c_first_name#37, ca_city#23, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Arguments: 100, [c_last_name#38 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, ca_city#23 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#38, c_first_name#37, ca_city#23, bought_city#31, ss_ticket_number#5, amt#32, profit#33] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt index a00e2eeac49b7..951558ca1b130 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt @@ -225,20 +225,20 @@ Output [7]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_ad Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#31, amt#32, profit#33, c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#39, ca_city#40] +Output [2]: [ca_address_sk#21, ca_city#22] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#39] -Join condition: NOT (ca_city#40 = bought_city#31) +Right keys [1]: [ca_address_sk#21] +Join condition: NOT (ca_city#22 = bought_city#31) (42) Project [codegen id : 8] -Output [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#39, ca_city#40] +Output [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#21, ca_city#22] (43) TakeOrderedAndProject -Input [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#40 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#22 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt index a3223d60c96be..92db706d594f2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] (3) Filter [codegen id : 1] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 1] Output [2]: [i_item_sk#1, i_manufact_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (30) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#26, _w0#27, avg_quarterly_sales#29] -Condition : (isnotnull(avg_quarterly_sales#29) AND ((avg_quarterly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_quarterly_sales#29) AND (avg_quarterly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (31) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#26, avg_quarterly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt index 172e66022046d..8b29f7c168765 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] (3) Filter [codegen id : 4] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 4] Output [2]: [i_item_sk#1, i_manufact_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (30) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#26, _w0#27, avg_quarterly_sales#29] -Condition : (isnotnull(avg_quarterly_sales#29) AND ((avg_quarterly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_quarterly_sales#29) AND (avg_quarterly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (31) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#26, avg_quarterly_sales#29] 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..4cdcd9685acbe 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))#67, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72] +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))#67,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72,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#37, s_store_id#38] 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#37, s_store_id#38] (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#37, s_store_id#38] +Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) (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#37, s_store_id#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] (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#37] 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#38] +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#37, s_store_id#38] (41) Scan parquet default.date_dim -Output [2]: [d_month_seq#82, d_week_seq#83] +Output [2]: [d_month_seq#74, d_week_seq#75] 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#74, d_week_seq#75] (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#74, d_week_seq#75] +Condition : (((isnotnull(d_month_seq#74) AND (d_month_seq#74 >= 1197)) AND (d_month_seq#74 <= 1208)) AND isnotnull(d_week_seq#75)) (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#75] +Input [2]: [d_month_seq#74, d_week_seq#75] (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#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] (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#75] 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#77, s_store_id#38 AS s_store_id2#78, sun_sales#30 AS sun_sales2#79, mon_sales#31 AS mon_sales2#80, wed_sales#33 AS wed_sales2#81, thu_sales#34 AS thu_sales2#82, fri_sales#35 AS fri_sales2#83, sat_sales#36 AS sat_sales2#84] +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#38, d_week_seq#75] (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#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#85] (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#78, (d_week_seq2#77 - 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#79)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#86, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#80)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#87, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#88, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#81)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#89, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#82)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#90, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#83)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#91, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#84)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#92] +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#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] (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)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] +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)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] 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..4cdcd9685acbe 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))#67, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72] +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))#67,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72,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#37, s_store_id#38] 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#37, s_store_id#38] (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#37, s_store_id#38] +Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) (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#37, s_store_id#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] (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#37] 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#38] +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#37, s_store_id#38] (41) Scan parquet default.date_dim -Output [2]: [d_month_seq#82, d_week_seq#83] +Output [2]: [d_month_seq#74, d_week_seq#75] 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#74, d_week_seq#75] (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#74, d_week_seq#75] +Condition : (((isnotnull(d_month_seq#74) AND (d_month_seq#74 >= 1197)) AND (d_month_seq#74 <= 1208)) AND isnotnull(d_week_seq#75)) (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#75] +Input [2]: [d_month_seq#74, d_week_seq#75] (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#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] (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#75] 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#77, s_store_id#38 AS s_store_id2#78, sun_sales#30 AS sun_sales2#79, mon_sales#31 AS mon_sales2#80, wed_sales#33 AS wed_sales2#81, thu_sales#34 AS thu_sales2#82, fri_sales#35 AS fri_sales2#83, sat_sales#36 AS sat_sales2#84] +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#38, d_week_seq#75] (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#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#85] (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#78, (d_week_seq2#77 - 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#79)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#86, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#80)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#87, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#88, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#81)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#89, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#82)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#90, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#83)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#91, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#84)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#92] +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#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] (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)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] +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)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt index 75cdaecea7595..8e0aaf2c479b8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] (3) Filter [codegen id : 1] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 1] Output [2]: [i_item_sk#1, i_manager_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manager_id#5, specifiedwindowfram (30) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#26, _w0#27, avg_monthly_sales#29] -Condition : (isnotnull(avg_monthly_sales#29) AND ((avg_monthly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_monthly_sales#29) AND (avg_monthly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (31) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#26, avg_monthly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt index c8e0821dfacda..712ba76f2878a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] (3) Filter [codegen id : 4] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 4] Output [2]: [i_item_sk#1, i_manager_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manager_id#5, specifiedwindowfram (30) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#26, _w0#27, avg_monthly_sales#29] -Condition : (isnotnull(avg_monthly_sales#29) AND ((avg_monthly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_monthly_sales#29) AND (avg_monthly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (31) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#26, avg_monthly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt index 240f579025038..741a644c4fdbc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt @@ -127,123 +127,123 @@ Input [4]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17 Condition : isnotnull(ss_store_sk#15) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#6] (19) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#17] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#6] Join condition: None (20) Project [codegen id : 4] Output [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] -Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#18] +Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#6] (21) HashAggregate [codegen id : 4] Input [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] +Aggregate Attributes [1]: [sum#18] +Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] (22) Exchange -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] -Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] +Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#20] (23) HashAggregate [codegen id : 5] -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#22] -Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#22,17,2) AS revenue#23] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#21] +Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#21,17,2) AS revenue#22] (24) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#15, revenue#23] +Input [2]: [ss_store_sk#15, revenue#22] Keys [1]: [ss_store_sk#15] -Functions [1]: [partial_avg(revenue#23)] -Aggregate Attributes [2]: [sum#24, count#25] -Results [3]: [ss_store_sk#15, sum#26, count#27] +Functions [1]: [partial_avg(revenue#22)] +Aggregate Attributes [2]: [sum#23, count#24] +Results [3]: [ss_store_sk#15, sum#25, count#26] (25) Exchange -Input [3]: [ss_store_sk#15, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [ss_store_sk#15, sum#25, count#26] +Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#27] (26) HashAggregate [codegen id : 6] -Input [3]: [ss_store_sk#15, sum#26, count#27] +Input [3]: [ss_store_sk#15, sum#25, count#26] Keys [1]: [ss_store_sk#15] -Functions [1]: [avg(revenue#23)] -Aggregate Attributes [1]: [avg(revenue#23)#29] -Results [2]: [ss_store_sk#15, avg(revenue#23)#29 AS ave#30] +Functions [1]: [avg(revenue#22)] +Aggregate Attributes [1]: [avg(revenue#22)#28] +Results [2]: [ss_store_sk#15, avg(revenue#22)#28 AS ave#29] (27) BroadcastExchange -Input [2]: [ss_store_sk#15, ave#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [2]: [ss_store_sk#15, ave#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] (28) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_store_sk#2] Right keys [1]: [ss_store_sk#15] -Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#30)), DecimalType(23,7), true)) +Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#29)), DecimalType(23,7), true)) (29) Project [codegen id : 7] Output [3]: [ss_store_sk#2, ss_item_sk#1, revenue#13] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#30] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#29] (30) BroadcastExchange Input [3]: [ss_store_sk#2, ss_item_sk#1, revenue#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (31) Scan parquet default.store -Output [2]: [s_store_sk#33, s_store_name#34] +Output [2]: [s_store_sk#32, s_store_name#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (32) ColumnarToRow -Input [2]: [s_store_sk#33, s_store_name#34] +Input [2]: [s_store_sk#32, s_store_name#33] (33) Filter -Input [2]: [s_store_sk#33, s_store_name#34] -Condition : isnotnull(s_store_sk#33) +Input [2]: [s_store_sk#32, s_store_name#33] +Condition : isnotnull(s_store_sk#32) (34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#33] +Right keys [1]: [s_store_sk#32] Join condition: None (35) Project [codegen id : 8] -Output [3]: [ss_item_sk#1, revenue#13, s_store_name#34] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#33, s_store_name#34] +Output [3]: [ss_item_sk#1, revenue#13, s_store_name#33] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#32, s_store_name#33] (36) BroadcastExchange -Input [3]: [ss_item_sk#1, revenue#13, s_store_name#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [3]: [ss_item_sk#1, revenue#13, s_store_name#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] (37) Scan parquet default.item -Output [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Output [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (38) ColumnarToRow -Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] (39) Filter -Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] -Condition : isnotnull(i_item_sk#36) +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Condition : isnotnull(i_item_sk#35) (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#36] +Right keys [1]: [i_item_sk#35] Join condition: None (41) Project [codegen id : 9] -Output [6]: [s_store_name#34, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] -Input [8]: [ss_item_sk#1, revenue#13, s_store_name#34, i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Output [6]: [s_store_name#33, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [8]: [ss_item_sk#1, revenue#13, s_store_name#33, i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] (42) TakeOrderedAndProject -Input [6]: [s_store_name#34, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] -Arguments: 100, [s_store_name#34 ASC NULLS FIRST, i_item_desc#37 ASC NULLS FIRST], [s_store_name#34, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Input [6]: [s_store_name#33, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Arguments: 100, [s_store_name#33 ASC NULLS FIRST, i_item_desc#36 ASC NULLS FIRST], [s_store_name#33, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt index 43ef320ed235f..0851190e54fe0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt @@ -181,65 +181,65 @@ Input [4]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26 Condition : isnotnull(ss_store_sk#24) (30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#8] (31) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#8] Join condition: None (32) Project [codegen id : 6] Output [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] -Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#27] +Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#8] (33) HashAggregate [codegen id : 6] Input [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#28] -Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] +Aggregate Attributes [1]: [sum#27] +Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] (34) Exchange -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] -Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] +Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#29] (35) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] -Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS revenue#32] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#30] +Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#30,17,2) AS revenue#31] (36) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#24, revenue#32] +Input [2]: [ss_store_sk#24, revenue#31] Keys [1]: [ss_store_sk#24] -Functions [1]: [partial_avg(revenue#32)] -Aggregate Attributes [2]: [sum#33, count#34] -Results [3]: [ss_store_sk#24, sum#35, count#36] +Functions [1]: [partial_avg(revenue#31)] +Aggregate Attributes [2]: [sum#32, count#33] +Results [3]: [ss_store_sk#24, sum#34, count#35] (37) Exchange -Input [3]: [ss_store_sk#24, sum#35, count#36] -Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [ss_store_sk#24, sum#34, count#35] +Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#36] (38) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#24, sum#35, count#36] +Input [3]: [ss_store_sk#24, sum#34, count#35] Keys [1]: [ss_store_sk#24] -Functions [1]: [avg(revenue#32)] -Aggregate Attributes [1]: [avg(revenue#32)#38] -Results [2]: [ss_store_sk#24, avg(revenue#32)#38 AS ave#39] +Functions [1]: [avg(revenue#31)] +Aggregate Attributes [1]: [avg(revenue#31)#37] +Results [2]: [ss_store_sk#24, avg(revenue#31)#37 AS ave#38] (39) BroadcastExchange -Input [2]: [ss_store_sk#24, ave#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] +Input [2]: [ss_store_sk#24, ave#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#24] -Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#39)), DecimalType(23,7), true)) +Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#38)), DecimalType(23,7), true)) (41) Project [codegen id : 9] Output [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#39] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#38] (42) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt index 0b96b76a0ffc4..10ac72a6b44de 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt @@ -232,31 +232,31 @@ Input [8]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37 Arguments: HashedRelationBroadcastMode(List(cast(input[5, int, true] as bigint)),false), [id=#44] (41) Scan parquet default.customer_address -Output [2]: [ca_address_sk#45, ca_city#46] +Output [2]: [ca_address_sk#23, ca_city#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] ReadSchema: struct (42) ColumnarToRow -Input [2]: [ca_address_sk#45, ca_city#46] +Input [2]: [ca_address_sk#23, ca_city#24] (43) Filter -Input [2]: [ca_address_sk#45, ca_city#46] -Condition : (isnotnull(ca_address_sk#45) AND isnotnull(ca_city#46)) +Input [2]: [ca_address_sk#23, ca_city#24] +Condition : (isnotnull(ca_address_sk#23) AND isnotnull(ca_city#24)) (44) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#41] -Right keys [1]: [ca_address_sk#45] -Join condition: NOT (ca_city#46 = bought_city#35) +Right keys [1]: [ca_address_sk#23] +Join condition: NOT (ca_city#24 = bought_city#35) (45) Project [codegen id : 8] -Output [8]: [c_last_name#43, c_first_name#42, ca_city#46, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#41, c_first_name#42, c_last_name#43, ca_address_sk#45, ca_city#46] +Output [8]: [c_last_name#43, c_first_name#42, ca_city#24, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#41, c_first_name#42, c_last_name#43, ca_address_sk#23, ca_city#24] (46) TakeOrderedAndProject -Input [8]: [c_last_name#43, c_first_name#42, ca_city#46, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Arguments: 100, [c_last_name#43 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#43, c_first_name#42, ca_city#46, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [8]: [c_last_name#43, c_first_name#42, ca_city#24, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Arguments: 100, [c_last_name#43 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#43, c_first_name#42, ca_city#24, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt index f50329db7e05e..81faa98299f82 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt @@ -225,20 +225,20 @@ Output [8]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#3 Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_customer_sk#39, c_current_addr_sk#40, c_first_name#41, c_last_name#42] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#44, ca_city#45] +Output [2]: [ca_address_sk#22, ca_city#23] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#40] -Right keys [1]: [ca_address_sk#44] -Join condition: NOT (ca_city#45 = bought_city#35) +Right keys [1]: [ca_address_sk#22] +Join condition: NOT (ca_city#23 = bought_city#35) (42) Project [codegen id : 8] -Output [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#44, ca_city#45] +Output [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#22, ca_city#23] (43) TakeOrderedAndProject -Input [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt index daa0ca7b16562..fe4db3541cbd9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt @@ -116,7 +116,7 @@ Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_p Output [4]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -124,7 +124,7 @@ Input [4]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_stat (20) Filter [codegen id : 3] Input [4]: [cd_demo_sk#17, cd_gender#18, cd_marital_status#19, cd_education_status#20] -Condition : ((((((isnotnull(cd_gender#18) AND isnotnull(cd_marital_status#19)) AND isnotnull(cd_education_status#20)) AND (cd_gender#18 = F)) AND (cd_marital_status#19 = W)) AND (cd_education_status#20 = Primary )) AND isnotnull(cd_demo_sk#17)) +Condition : ((((((isnotnull(cd_gender#18) AND isnotnull(cd_marital_status#19)) AND isnotnull(cd_education_status#20)) AND (cd_gender#18 = F)) AND (cd_marital_status#19 = W)) AND (cd_education_status#20 = Primary)) AND isnotnull(cd_demo_sk#17)) (21) Project [codegen id : 3] Output [1]: [cd_demo_sk#17] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt index a9da9118323b6..c1f0c1bd01451 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnul Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = F)) AND (cd_marital_status#12 = W)) AND (cd_education_status#13 = Primary )) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = F)) AND (cd_marital_status#12 = W)) AND (cd_education_status#13 = Primary)) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt index c2909d9f5531f..9dc7b54bf5bb6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt @@ -60,7 +60,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint) Output [4]: [hd_demo_sk#7, hd_buy_potential#8, hd_dep_count#9, hd_vehicle_count#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (6) ColumnarToRow @@ -68,7 +68,7 @@ Input [4]: [hd_demo_sk#7, hd_buy_potential#8, hd_dep_count#9, hd_vehicle_count#1 (7) Filter Input [4]: [hd_demo_sk#7, hd_buy_potential#8, hd_dep_count#9, hd_vehicle_count#10] -Condition : (((((isnotnull(hd_vehicle_count#10) AND isnotnull(hd_dep_count#9)) AND ((hd_buy_potential#8 = >10000 ) OR (hd_buy_potential#8 = Unknown ))) AND (hd_vehicle_count#10 > 0)) AND ((cast(hd_dep_count#9 as double) / cast(hd_vehicle_count#10 as double)) > 1.0)) AND isnotnull(hd_demo_sk#7)) +Condition : (((((isnotnull(hd_vehicle_count#10) AND isnotnull(hd_dep_count#9)) AND ((hd_buy_potential#8 = >10000) OR (hd_buy_potential#8 = Unknown))) AND (hd_vehicle_count#10 > 0)) AND ((cast(hd_dep_count#9 as double) / cast(hd_vehicle_count#10 as double)) > 1.0)) AND isnotnull(hd_demo_sk#7)) (8) Project Output [1]: [hd_demo_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt index 5ec772ca3d638..559ef93fb70ac 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = Unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = Unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt index a325bf820a4f9..393b6a4da4255 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt @@ -109,7 +109,7 @@ Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, d_moy#8, s_store_sk#1 Output [4]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(In(i_category, [Home ,Books ,Electronics ]),In(i_class, [wallpaper ,parenting ,musical ])),And(In(i_category, [Shoes ,Jewelry ,Men ]),In(i_class, [womens ,birdal ,pants ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(In(i_category, [Home,Books,Electronics]),In(i_class, [wallpaper,parenting,musical])),And(In(i_category, [Shoes,Jewelry,Men]),In(i_class, [womens,birdal,pants]))), IsNotNull(i_item_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] @@ -117,7 +117,7 @@ Input [4]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17] (19) Filter [codegen id : 3] Input [4]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17] -Condition : (((i_category#17 IN (Home ,Books ,Electronics ) AND i_class#16 IN (wallpaper ,parenting ,musical )) OR (i_category#17 IN (Shoes ,Jewelry ,Men ) AND i_class#16 IN (womens ,birdal ,pants ))) AND isnotnull(i_item_sk#14)) +Condition : (((i_category#17 IN (Home,Books,Electronics) AND i_class#16 IN (wallpaper,parenting,musical)) OR (i_category#17 IN (Shoes,Jewelry,Men) AND i_class#16 IN (womens,birdal,pants))) AND isnotnull(i_item_sk#14)) (20) BroadcastExchange Input [4]: [i_item_sk#14, i_brand#15, i_class#16, i_category#17] @@ -164,7 +164,7 @@ Arguments: [avg(_w0#24) windowspecdefinition(i_category#17, i_brand#15, s_store_ (29) Filter [codegen id : 7] Input [9]: [i_category#17, i_class#16, i_brand#15, s_store_name#11, s_company_name#12, d_moy#8, sum_sales#23, _w0#24, avg_monthly_sales#26] -Condition : (isnotnull(avg_monthly_sales#26) AND (NOT (avg_monthly_sales#26 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_monthly_sales#26) AND NOT (avg_monthly_sales#26 = 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (30) Project [codegen id : 7] Output [8]: [i_category#17, i_class#16, i_brand#15, s_store_name#11, s_company_name#12, d_moy#8, sum_sales#23, avg_monthly_sales#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt index 2ec1ea034361a..0210877799a98 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject (31) Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(In(i_category, [Home ,Books ,Electronics ]),In(i_class, [wallpaper ,parenting ,musical ])),And(In(i_category, [Shoes ,Jewelry ,Men ]),In(i_class, [womens ,birdal ,pants ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(In(i_category, [Home,Books,Electronics]),In(i_class, [wallpaper,parenting,musical])),And(In(i_category, [Shoes,Jewelry,Men]),In(i_class, [womens,birdal,pants]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -44,7 +44,7 @@ Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] (3) Filter [codegen id : 4] Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((i_category#4 IN (Home ,Books ,Electronics ) AND i_class#3 IN (wallpaper ,parenting ,musical )) OR (i_category#4 IN (Shoes ,Jewelry ,Men ) AND i_class#3 IN (womens ,birdal ,pants ))) AND isnotnull(i_item_sk#1)) +Condition : (((i_category#4 IN (Home,Books,Electronics) AND i_class#3 IN (wallpaper,parenting,musical)) OR (i_category#4 IN (Shoes,Jewelry,Men) AND i_class#3 IN (womens,birdal,pants))) AND isnotnull(i_item_sk#1)) (4) Scan parquet default.store_sales Output [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] @@ -164,7 +164,7 @@ Arguments: [avg(_w0#24) windowspecdefinition(i_category#4, i_brand#2, s_store_na (29) Filter [codegen id : 7] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, _w0#24, avg_monthly_sales#26] -Condition : (isnotnull(avg_monthly_sales#26) AND (NOT (avg_monthly_sales#26 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_monthly_sales#26) AND NOT (avg_monthly_sales#26 = 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (30) Project [codegen id : 7] Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, avg_monthly_sales#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt index 9d369b5d211fc..09e8dbc57e336 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt @@ -57,7 +57,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((((isnotnull(d_date#6) AND (d_date#6 >= 2001-01-01)) AND (d_date#6 <= 2001-01-31)) AND (d_date_sk#5 >= 2451911)) AND (d_date_sk#5 <= 2451941)) AND isnotnull(d_date_sk#5)) +Condition : (((((isnotnull(d_date#6) AND (d_date#6 >= 11323)) AND (d_date#6 <= 11353)) AND (d_date_sk#5 >= 2451911)) AND (d_date_sk#5 <= 2451941)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -88,7 +88,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Jewelry ,Sports ,Books ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Jewelry,Sports,Books]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -96,7 +96,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Jewelry ,Sports ,Books ) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Jewelry,Sports,Books) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt index e31c029b4932e..d46901132fcc3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt @@ -46,7 +46,7 @@ Condition : isnotnull(ss_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Jewelry ,Sports ,Books ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Jewelry,Sports,Books]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -54,7 +54,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Jewelry ,Sports ,Books ) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Jewelry,Sports,Books) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -81,7 +81,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((((isnotnull(d_date#13) AND (d_date#13 >= 2001-01-01)) AND (d_date#13 <= 2001-01-31)) AND (d_date_sk#12 >= 2451911)) AND (d_date_sk#12 <= 2451941)) AND isnotnull(d_date_sk#12)) +Condition : (((((isnotnull(d_date#13) AND (d_date#13 >= 11323)) AND (d_date#13 <= 11353)) AND (d_date_sk#12 >= 2451911)) AND (d_date_sk#12 <= 2451941)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] 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 a950e7b9e8639..9c68e55238aee 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 @@ -117,158 +117,158 @@ Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) (15) Scan parquet default.store_returns -Output [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] +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#19), dynamicpruningexpression(sr_returned_date_sk#19 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 (16) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] +Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] (17) Filter [codegen id : 4] -Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#17) +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) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#20] +Output [1]: [d_date_sk#6] (19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [cast(d_date_sk#20 as bigint)] +Left keys [1]: [sr_returned_date_sk#4] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (20) Project [codegen id : 4] -Output [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] -Input [5]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19, d_date_sk#20] +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] (21) HashAggregate [codegen id : 4] -Input [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] -Keys [2]: [sr_customer_sk#16, sr_store_sk#17] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#18))] -Aggregate Attributes [1]: [sum#21] -Results [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] +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#16] +Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] (22) Exchange -Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] -Arguments: hashpartitioning(sr_customer_sk#16, sr_store_sk#17, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] +Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#18] (23) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] -Keys [2]: [sr_customer_sk#16, sr_store_sk#17] -Functions [1]: [sum(UnscaledValue(sr_return_amt#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#18))#24] -Results [2]: [sr_store_sk#17 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#18))#24,17,2) AS ctr_total_return#15] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] +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))#19] +Results [2]: [sr_store_sk#2 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#19,17,2) AS ctr_total_return#15] (24) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#14, ctr_total_return#15] Keys [1]: [ctr_store_sk#14] Functions [1]: [partial_avg(ctr_total_return#15)] -Aggregate Attributes [2]: [sum#25, count#26] -Results [3]: [ctr_store_sk#14, sum#27, count#28] +Aggregate Attributes [2]: [sum#20, count#21] +Results [3]: [ctr_store_sk#14, sum#22, count#23] (25) Exchange -Input [3]: [ctr_store_sk#14, sum#27, count#28] -Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ctr_store_sk#14, sum#22, count#23] +Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#24] (26) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#14, sum#27, count#28] +Input [3]: [ctr_store_sk#14, sum#22, count#23] Keys [1]: [ctr_store_sk#14] Functions [1]: [avg(ctr_total_return#15)] -Aggregate Attributes [1]: [avg(ctr_total_return#15)#30] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14 AS ctr_store_sk#14#32] +Aggregate Attributes [1]: [avg(ctr_total_return#15)#25] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#25) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14 AS ctr_store_sk#14#27] (27) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) +Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#26) (28) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#33] +Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] +Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#28] (29) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [ctr_store_sk#14#32] -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) +Right keys [1]: [ctr_store_sk#14#27] +Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#26) (30) Project [codegen id : 8] Output [2]: [ctr_customer_sk#13, ctr_store_sk#14] -Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] +Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] (31) Scan parquet default.store -Output [2]: [s_store_sk#34, s_state#35] +Output [2]: [s_store_sk#29, s_state#30] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#34, s_state#35] +Input [2]: [s_store_sk#29, s_state#30] (33) Filter [codegen id : 7] -Input [2]: [s_store_sk#34, s_state#35] -Condition : ((isnotnull(s_state#35) AND (s_state#35 = TN)) AND isnotnull(s_store_sk#34)) +Input [2]: [s_store_sk#29, s_state#30] +Condition : ((isnotnull(s_state#30) AND (s_state#30 = TN)) AND isnotnull(s_store_sk#29)) (34) Project [codegen id : 7] -Output [1]: [s_store_sk#34] -Input [2]: [s_store_sk#34, s_state#35] +Output [1]: [s_store_sk#29] +Input [2]: [s_store_sk#29, s_state#30] (35) BroadcastExchange -Input [1]: [s_store_sk#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [1]: [s_store_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (36) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [cast(s_store_sk#34 as bigint)] +Right keys [1]: [cast(s_store_sk#29 as bigint)] Join condition: None (37) Project [codegen id : 8] Output [1]: [ctr_customer_sk#13] -Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#34] +Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#29] (38) Exchange Input [1]: [ctr_customer_sk#13] -Arguments: hashpartitioning(ctr_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#37] +Arguments: hashpartitioning(ctr_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#32] (39) Sort [codegen id : 9] Input [1]: [ctr_customer_sk#13] Arguments: [ctr_customer_sk#13 ASC NULLS FIRST], false, 0 (40) Scan parquet default.customer -Output [2]: [c_customer_sk#38, c_customer_id#39] +Output [2]: [c_customer_sk#33, c_customer_id#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [2]: [c_customer_sk#38, c_customer_id#39] +Input [2]: [c_customer_sk#33, c_customer_id#34] (42) Filter [codegen id : 10] -Input [2]: [c_customer_sk#38, c_customer_id#39] -Condition : isnotnull(c_customer_sk#38) +Input [2]: [c_customer_sk#33, c_customer_id#34] +Condition : isnotnull(c_customer_sk#33) (43) Exchange -Input [2]: [c_customer_sk#38, c_customer_id#39] -Arguments: hashpartitioning(cast(c_customer_sk#38 as bigint), 5), ENSURE_REQUIREMENTS, [id=#40] +Input [2]: [c_customer_sk#33, c_customer_id#34] +Arguments: hashpartitioning(cast(c_customer_sk#33 as bigint), 5), ENSURE_REQUIREMENTS, [id=#35] (44) Sort [codegen id : 11] -Input [2]: [c_customer_sk#38, c_customer_id#39] -Arguments: [cast(c_customer_sk#38 as bigint) ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#33, c_customer_id#34] +Arguments: [cast(c_customer_sk#33 as bigint) ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 12] Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [cast(c_customer_sk#38 as bigint)] +Right keys [1]: [cast(c_customer_sk#33 as bigint)] Join condition: None (46) Project [codegen id : 12] -Output [1]: [c_customer_id#39] -Input [3]: [ctr_customer_sk#13, c_customer_sk#38, c_customer_id#39] +Output [1]: [c_customer_id#34] +Input [3]: [ctr_customer_sk#13, c_customer_sk#33, c_customer_id#34] (47) TakeOrderedAndProject -Input [1]: [c_customer_id#39] -Arguments: 100, [c_customer_id#39 ASC NULLS FIRST], [c_customer_id#39] +Input [1]: [c_customer_id#34] +Arguments: 100, [c_customer_id#34 ASC NULLS FIRST], [c_customer_id#34] ===== Subqueries ===== @@ -279,6 +279,6 @@ ReusedExchange (48) (48) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 15 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 15 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 015228686d39b..4487f04c8293d 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 @@ -114,146 +114,146 @@ Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) (15) Scan parquet default.store_returns -Output [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] +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#19), dynamicpruningexpression(sr_returned_date_sk#19 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 (16) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] +Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] (17) Filter [codegen id : 4] -Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#17) +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) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#20] +Output [1]: [d_date_sk#6] (19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [cast(d_date_sk#20 as bigint)] +Left keys [1]: [sr_returned_date_sk#4] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (20) Project [codegen id : 4] -Output [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] -Input [5]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19, d_date_sk#20] +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] (21) HashAggregate [codegen id : 4] -Input [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] -Keys [2]: [sr_customer_sk#16, sr_store_sk#17] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#18))] -Aggregate Attributes [1]: [sum#21] -Results [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] +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#16] +Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] (22) Exchange -Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] -Arguments: hashpartitioning(sr_customer_sk#16, sr_store_sk#17, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] +Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#18] (23) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] -Keys [2]: [sr_customer_sk#16, sr_store_sk#17] -Functions [1]: [sum(UnscaledValue(sr_return_amt#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#18))#24] -Results [2]: [sr_store_sk#17 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#18))#24,17,2) AS ctr_total_return#15] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] +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))#19] +Results [2]: [sr_store_sk#2 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#19,17,2) AS ctr_total_return#15] (24) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#14, ctr_total_return#15] Keys [1]: [ctr_store_sk#14] Functions [1]: [partial_avg(ctr_total_return#15)] -Aggregate Attributes [2]: [sum#25, count#26] -Results [3]: [ctr_store_sk#14, sum#27, count#28] +Aggregate Attributes [2]: [sum#20, count#21] +Results [3]: [ctr_store_sk#14, sum#22, count#23] (25) Exchange -Input [3]: [ctr_store_sk#14, sum#27, count#28] -Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ctr_store_sk#14, sum#22, count#23] +Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#24] (26) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#14, sum#27, count#28] +Input [3]: [ctr_store_sk#14, sum#22, count#23] Keys [1]: [ctr_store_sk#14] Functions [1]: [avg(ctr_total_return#15)] -Aggregate Attributes [1]: [avg(ctr_total_return#15)#30] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14 AS ctr_store_sk#14#32] +Aggregate Attributes [1]: [avg(ctr_total_return#15)#25] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#25) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14 AS ctr_store_sk#14#27] (27) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) +Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#26) (28) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#33] +Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] +Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#28] (29) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [ctr_store_sk#14#32] -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) +Right keys [1]: [ctr_store_sk#14#27] +Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#26) (30) Project [codegen id : 9] Output [2]: [ctr_customer_sk#13, ctr_store_sk#14] -Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] +Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] (31) Scan parquet default.store -Output [2]: [s_store_sk#34, s_state#35] +Output [2]: [s_store_sk#29, s_state#30] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#34, s_state#35] +Input [2]: [s_store_sk#29, s_state#30] (33) Filter [codegen id : 7] -Input [2]: [s_store_sk#34, s_state#35] -Condition : ((isnotnull(s_state#35) AND (s_state#35 = TN)) AND isnotnull(s_store_sk#34)) +Input [2]: [s_store_sk#29, s_state#30] +Condition : ((isnotnull(s_state#30) AND (s_state#30 = TN)) AND isnotnull(s_store_sk#29)) (34) Project [codegen id : 7] -Output [1]: [s_store_sk#34] -Input [2]: [s_store_sk#34, s_state#35] +Output [1]: [s_store_sk#29] +Input [2]: [s_store_sk#29, s_state#30] (35) BroadcastExchange -Input [1]: [s_store_sk#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [1]: [s_store_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [cast(s_store_sk#34 as bigint)] +Right keys [1]: [cast(s_store_sk#29 as bigint)] Join condition: None (37) Project [codegen id : 9] Output [1]: [ctr_customer_sk#13] -Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#34] +Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#29] (38) Scan parquet default.customer -Output [2]: [c_customer_sk#37, c_customer_id#38] +Output [2]: [c_customer_sk#32, c_customer_id#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 8] -Input [2]: [c_customer_sk#37, c_customer_id#38] +Input [2]: [c_customer_sk#32, c_customer_id#33] (40) Filter [codegen id : 8] -Input [2]: [c_customer_sk#37, c_customer_id#38] -Condition : isnotnull(c_customer_sk#37) +Input [2]: [c_customer_sk#32, c_customer_id#33] +Condition : isnotnull(c_customer_sk#32) (41) BroadcastExchange -Input [2]: [c_customer_sk#37, c_customer_id#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#39] +Input [2]: [c_customer_sk#32, c_customer_id#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [cast(c_customer_sk#37 as bigint)] +Right keys [1]: [cast(c_customer_sk#32 as bigint)] Join condition: None (43) Project [codegen id : 9] -Output [1]: [c_customer_id#38] -Input [3]: [ctr_customer_sk#13, c_customer_sk#37, c_customer_id#38] +Output [1]: [c_customer_id#33] +Input [3]: [ctr_customer_sk#13, c_customer_sk#32, c_customer_id#33] (44) TakeOrderedAndProject -Input [1]: [c_customer_id#38] -Arguments: 100, [c_customer_id#38 ASC NULLS FIRST], [c_customer_id#38] +Input [1]: [c_customer_id#33] +Arguments: 100, [c_customer_id#33 ASC NULLS FIRST], [c_customer_id#33] ===== Subqueries ===== @@ -264,6 +264,6 @@ ReusedExchange (45) (45) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 15 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 15 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/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt index 2dd0a13e74a74..c25973f527384 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt @@ -143,20 +143,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#17] +Output [1]: [d_date_sk#10] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#17] +Right keys [1]: [d_date_sk#10] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] -Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] +Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] (23) Exchange Input [1]: [ws_bill_customer_sk#15] -Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] +Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] @@ -168,38 +168,38 @@ Right keys [1]: [ws_bill_customer_sk#15] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] +Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] +Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#21] +Output [1]: [d_date_sk#10] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#21] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#10] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#19] -Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] +Output [1]: [cs_ship_customer_sk#18] +Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] (31) Exchange -Input [1]: [cs_ship_customer_sk#19] -Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [cs_ship_customer_sk#18] +Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#19] -Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#18] +Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#19] +Right keys [1]: [cs_ship_customer_sk#18] Join condition: None (34) Filter [codegen id : 13] @@ -211,96 +211,96 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (36) Scan parquet default.customer_address -Output [2]: [ca_address_sk#23, ca_county#24] +Output [2]: [ca_address_sk#21, ca_county#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 12] -Input [2]: [ca_address_sk#23, ca_county#24] +Input [2]: [ca_address_sk#21, ca_county#22] (38) Filter [codegen id : 12] -Input [2]: [ca_address_sk#23, ca_county#24] -Condition : (ca_county#24 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#23)) +Input [2]: [ca_address_sk#21, ca_county#22] +Condition : (ca_county#22 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#21)) (39) Project [codegen id : 12] -Output [1]: [ca_address_sk#23] -Input [2]: [ca_address_sk#23, ca_county#24] +Output [1]: [ca_address_sk#21] +Input [2]: [ca_address_sk#21, ca_county#22] (40) BroadcastExchange -Input [1]: [ca_address_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] +Input [1]: [ca_address_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (41) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#23] +Right keys [1]: [ca_address_sk#21] Join condition: None (42) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#23] +Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#21] (43) Exchange Input [1]: [c_current_cdemo_sk#4] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#26] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#24] (44) Sort [codegen id : 14] Input [1]: [c_current_cdemo_sk#4] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 (45) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (46) ColumnarToRow [codegen id : 15] -Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (47) Filter [codegen id : 15] -Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -Condition : isnotnull(cd_demo_sk#27) +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Condition : isnotnull(cd_demo_sk#25) (48) Exchange -Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -Arguments: hashpartitioning(cd_demo_sk#27, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: hashpartitioning(cd_demo_sk#25, 5), ENSURE_REQUIREMENTS, [id=#34] (49) Sort [codegen id : 16] -Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -Arguments: [cd_demo_sk#27 ASC NULLS FIRST], false, 0 +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: [cd_demo_sk#25 ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 17] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#27] +Right keys [1]: [cd_demo_sk#25] Join condition: None (51) Project [codegen id : 17] -Output [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (52) HashAggregate [codegen id : 17] -Input [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#37] -Results [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] +Aggregate Attributes [1]: [count#35] +Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] (53) Exchange -Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] -Arguments: hashpartitioning(cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] (54) HashAggregate [codegen id : 18] -Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] -Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#40] -Results [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, count(1)#40 AS cnt1#41, cd_purchase_estimate#31, count(1)#40 AS cnt2#42, cd_credit_rating#32, count(1)#40 AS cnt3#43, cd_dep_count#33, count(1)#40 AS cnt4#44, cd_dep_employed_count#34, count(1)#40 AS cnt5#45, cd_dep_college_count#35, count(1)#40 AS cnt6#46] +Aggregate Attributes [1]: [count(1)#38] +Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] (55) TakeOrderedAndProject -Input [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#41, cd_purchase_estimate#31, cnt2#42, cd_credit_rating#32, cnt3#43, cd_dep_count#33, cnt4#44, cd_dep_employed_count#34, cnt5#45, cd_dep_college_count#35, cnt6#46] -Arguments: 100, [cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_education_status#30 ASC NULLS FIRST, cd_purchase_estimate#31 ASC NULLS FIRST, cd_credit_rating#32 ASC NULLS FIRST, cd_dep_count#33 ASC NULLS FIRST, cd_dep_employed_count#34 ASC NULLS FIRST, cd_dep_college_count#35 ASC NULLS FIRST], [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#41, cd_purchase_estimate#31, cnt2#42, cd_credit_rating#32, cnt3#43, cd_dep_count#33, cnt4#44, cd_dep_employed_count#34, cnt5#45, cd_dep_college_count#35, cnt6#46] +Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] +Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] ===== Subqueries ===== @@ -313,6 +313,6 @@ Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index 370551a2de652..07bcd309e9ce6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -123,20 +123,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#9] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#9] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#16] +Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] @@ -144,34 +144,34 @@ Right keys [1]: [ws_bill_customer_sk#14] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#20] +Output [1]: [d_date_sk#9] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#20] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#9] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#18] -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#20] +Output [1]: [cs_ship_customer_sk#17] +Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [cs_ship_customer_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#17] Join condition: None (29) Filter [codegen id : 9] @@ -183,84 +183,84 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_county#23] +Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#22, ca_county#23] +Input [2]: [ca_address_sk#20, ca_county#21] (33) Filter [codegen id : 7] -Input [2]: [ca_address_sk#22, ca_county#23] -Condition : (ca_county#23 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#22)) +Input [2]: [ca_address_sk#20, ca_county#21] +Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#20)) (34) Project [codegen id : 7] -Output [1]: [ca_address_sk#22] -Input [2]: [ca_address_sk#22, ca_county#23] +Output [1]: [ca_address_sk#20] +Input [2]: [ca_address_sk#20, ca_county#21] (35) 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#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#20] Join condition: None (37) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22] +Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] (38) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] (40) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Condition : isnotnull(cd_demo_sk#25) +Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Condition : isnotnull(cd_demo_sk#23) (41) BroadcastExchange -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] (42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#23] Join condition: None (43) Project [codegen id : 9] -Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] (44) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#35] -Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Aggregate Attributes [1]: [count#33] +Results [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] (45) Exchange -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] +Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#35] (46) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] +Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] +Aggregate Attributes [1]: [count(1)#36] +Results [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#36 AS cnt1#37, cd_purchase_estimate#27, count(1)#36 AS cnt2#38, cd_credit_rating#28, count(1)#36 AS cnt3#39, cd_dep_count#29, count(1)#36 AS cnt4#40, cd_dep_employed_count#30, count(1)#36 AS cnt5#41, cd_dep_college_count#31, count(1)#36 AS cnt6#42] (47) TakeOrderedAndProject -Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] +Input [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] +Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] ===== Subqueries ===== @@ -273,6 +273,6 @@ Output [1]: [d_date_sk#9] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 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 92e8e38634ea6..da2143dc20a90 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 @@ -200,289 +200,289 @@ Input [2]: [customer_id#23, year_total#24] Arguments: [customer_id#23 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] +Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] (27) Filter [codegen id : 10] -Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] -Condition : isnotnull(ss_customer_sk#26) +Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] +Condition : isnotnull(ss_customer_sk#1) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#31, d_year#32] +Output [2]: [d_date_sk#6, d_year#7] 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 (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#31, d_year#32] +Input [2]: [d_date_sk#6, d_year#7] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#31, d_year#32] -Condition : ((isnotnull(d_year#32) AND (d_year#32 = 2002)) AND isnotnull(d_date_sk#31)) +Input [2]: [d_date_sk#6, d_year#7] +Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) (31) BroadcastExchange -Input [2]: [d_date_sk#31, d_year#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +Input [2]: [d_date_sk#6, d_year#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#29] -Right keys [1]: [d_date_sk#31] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#6] Join condition: None (33) Project [codegen id : 10] -Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Input [6]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29, d_date_sk#31, d_year#32] +Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4, d_date_sk#6, d_year#7] (34) Exchange -Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#28] (35) Sort [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (37) Sort [codegen id : 13] -Input [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] -Arguments: [c_customer_sk#35 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#35] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#10] Join condition: None (39) Project [codegen id : 14] -Output [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Input [12]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32, c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Input [12]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (40) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#43] -Results [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] +Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#29] +Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] (41) Exchange -Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] +Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, 5), ENSURE_REQUIREMENTS, [id=#31] (42) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] -Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46] -Results [3]: [c_customer_id#36 AS customer_id#47, c_preferred_cust_flag#39 AS customer_preferred_cust_flag#48, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46,18,2) AS year_total#49] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32] +Results [3]: [c_customer_id#11 AS customer_id#33, c_preferred_cust_flag#14 AS customer_preferred_cust_flag#34, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32,18,2) AS year_total#35] (43) Exchange -Input [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] -Arguments: hashpartitioning(customer_id#47, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [3]: [customer_id#33, customer_preferred_cust_flag#34, year_total#35] +Arguments: hashpartitioning(customer_id#33, 5), ENSURE_REQUIREMENTS, [id=#36] (44) Sort [codegen id : 16] -Input [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] -Arguments: [customer_id#47 ASC NULLS FIRST], false, 0 +Input [3]: [customer_id#33, customer_preferred_cust_flag#34, year_total#35] +Arguments: [customer_id#33 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#47] +Right keys [1]: [customer_id#33] Join condition: None (46) Project [codegen id : 17] -Output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49] -Input [5]: [customer_id#23, year_total#24, customer_id#47, customer_preferred_cust_flag#48, year_total#49] +Output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35] +Input [5]: [customer_id#23, year_total#24, customer_id#33, customer_preferred_cust_flag#34, year_total#35] (47) 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#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] 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#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (48) 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#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] (49) 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#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Condition : isnotnull(ws_bill_customer_sk#37) (50) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#55, d_year#56] +Output [2]: [d_date_sk#6, d_year#7] (51) 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#40] +Right keys [1]: [d_date_sk#6] Join condition: None (52) 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#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Input [6]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#6, d_year#7] (53) 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#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Arguments: hashpartitioning(ws_bill_customer_sk#37, 5), ENSURE_REQUIREMENTS, [id=#41] (54) 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#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Arguments: [ws_bill_customer_sk#37 ASC NULLS FIRST], false, 0 (55) ReusedExchange [Reuses operator id: 15] -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#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (56) 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#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 (57) 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#37] +Right keys [1]: [c_customer_sk#10] Join condition: None (58) 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#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Input [12]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (59) 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#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#42] +Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#43] (60) 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#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#43] +Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#44] (61) 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#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#43] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45] +Results [2]: [c_customer_id#11 AS customer_id#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45,18,2) AS year_total#47] (62) 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#46, year_total#47] +Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) (63) Project [codegen id : 24] -Output [2]: [customer_id#70 AS customer_id#72, year_total#71 AS year_total#73] -Input [2]: [customer_id#70, year_total#71] +Output [2]: [customer_id#46 AS customer_id#48, year_total#47 AS year_total#49] +Input [2]: [customer_id#46, year_total#47] (64) Exchange -Input [2]: [customer_id#72, year_total#73] -Arguments: hashpartitioning(customer_id#72, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [2]: [customer_id#48, year_total#49] +Arguments: hashpartitioning(customer_id#48, 5), ENSURE_REQUIREMENTS, [id=#50] (65) Sort [codegen id : 25] -Input [2]: [customer_id#72, year_total#73] -Arguments: [customer_id#72 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#48, year_total#49] +Arguments: [customer_id#48 ASC NULLS FIRST], false, 0 (66) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#72] +Right keys [1]: [customer_id#48] Join condition: None (67) Project [codegen id : 26] -Output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73] -Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, customer_id#72, year_total#73] +Output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35, year_total#49] +Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35, customer_id#48, year_total#49] (68) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] +Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#78), dynamicpruningexpression(ws_sold_date_sk#78 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] (70) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] -Condition : isnotnull(ws_bill_customer_sk#75) +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Condition : isnotnull(ws_bill_customer_sk#37) (71) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#79, d_year#80] +Output [2]: [d_date_sk#6, d_year#7] (72) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#78] -Right keys [1]: [d_date_sk#79] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#6] Join condition: None (73) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Input [6]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78, d_date_sk#79, d_year#80] +Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Input [6]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#6, d_year#7] (74) Exchange -Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Arguments: hashpartitioning(ws_bill_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Arguments: hashpartitioning(ws_bill_customer_sk#37, 5), ENSURE_REQUIREMENTS, [id=#51] (75) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Arguments: [ws_bill_customer_sk#75 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Arguments: [ws_bill_customer_sk#37 ASC NULLS FIRST], false, 0 (76) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] +Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (77) Sort [codegen id : 31] -Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] -Arguments: [c_customer_sk#82 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#75] -Right keys [1]: [c_customer_sk#82] +Left keys [1]: [ws_bill_customer_sk#37] +Right keys [1]: [c_customer_sk#10] Join condition: None (79) Project [codegen id : 32] -Output [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Input [12]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80, c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] +Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Input [12]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (80) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#90] -Results [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] +Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#52] +Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#53] (81) Exchange -Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] -Arguments: hashpartitioning(c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#53] +Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#54] (82) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] -Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93] -Results [2]: [c_customer_id#83 AS customer_id#94, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93,18,2) AS year_total#95] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#53] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55] +Results [2]: [c_customer_id#11 AS customer_id#56, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55,18,2) AS year_total#57] (83) Exchange -Input [2]: [customer_id#94, year_total#95] -Arguments: hashpartitioning(customer_id#94, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [2]: [customer_id#56, year_total#57] +Arguments: hashpartitioning(customer_id#56, 5), ENSURE_REQUIREMENTS, [id=#58] (84) Sort [codegen id : 34] -Input [2]: [customer_id#94, year_total#95] -Arguments: [customer_id#94 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#56, year_total#57] +Arguments: [customer_id#56 ASC NULLS FIRST], false, 0 (85) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#94] -Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#95) / promote_precision(year_total#73)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#49) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE null END) +Right keys [1]: [customer_id#56] +Join condition: (CASE WHEN (year_total#49 > 0.00) THEN CheckOverflow((promote_precision(year_total#57) / promote_precision(year_total#49)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#35) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE null END) (86) Project [codegen id : 35] -Output [1]: [customer_preferred_cust_flag#48] -Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73, customer_id#94, year_total#95] +Output [1]: [customer_preferred_cust_flag#34] +Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35, year_total#49, customer_id#56, year_total#57] (87) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#48] -Arguments: 100, [customer_preferred_cust_flag#48 ASC NULLS FIRST], [customer_preferred_cust_flag#48] +Input [1]: [customer_preferred_cust_flag#34] +Arguments: 100, [customer_preferred_cust_flag#34 ASC NULLS FIRST], [customer_preferred_cust_flag#34] ===== Subqueries ===== @@ -493,15 +493,15 @@ ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#6, d_year#7] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#30 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#26 ReusedExchange (89) (89) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#31, d_year#32] +Output [2]: [d_date_sk#6, d_year#7] -Subquery:3 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#78 IN dynamicpruning#30 +Subquery:4 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#26 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 4081f69732168..5a5249a3aa81b 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 @@ -172,286 +172,286 @@ Input [2]: [customer_id#22, year_total#23] Condition : (isnotnull(year_total#23) AND (year_total#23 > 0.00)) (20) Scan parquet default.customer -Output [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] -Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_customer_id#25)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (23) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] 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#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] (25) Filter [codegen id : 4] -Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] -Condition : isnotnull(ss_customer_sk#32) +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_customer_sk#9) (26) BroadcastExchange -Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#24] -Right keys [1]: [ss_customer_sk#32] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#9] Join condition: None (28) Project [codegen id : 6] -Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] -Input [12]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#38, d_year#39] +Output [2]: [d_date_sk#15, d_year#16] 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 (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#38, d_year#39] +Input [2]: [d_date_sk#15, d_year#16] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) +Input [2]: [d_date_sk#15, d_year#16] +Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2002)) AND isnotnull(d_date_sk#15)) (32) BroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] +Input [2]: [d_date_sk#15, d_year#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#35] -Right keys [1]: [d_date_sk#38] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#15] Join condition: None (34) Project [codegen id : 6] -Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] -Input [12]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#38, d_year#39] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#15, d_year#16] (35) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] -Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#41] -Results [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#27] +Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] (36) Exchange -Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] -Arguments: hashpartitioning(c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [id=#29] (37) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] -Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44] -Results [3]: [c_customer_id#25 AS customer_id#45, c_preferred_cust_flag#28 AS customer_preferred_cust_flag#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44,18,2) AS year_total#47] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30] +Results [3]: [c_customer_id#2 AS customer_id#31, c_preferred_cust_flag#5 AS customer_preferred_cust_flag#32, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30,18,2) AS year_total#33] (38) BroadcastExchange -Input [3]: [customer_id#45, customer_preferred_cust_flag#46, year_total#47] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] +Input [3]: [customer_id#31, customer_preferred_cust_flag#32, year_total#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#45] +Right keys [1]: [customer_id#31] Join condition: None (40) Project [codegen id : 16] -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#32, year_total#33] +Input [5]: [customer_id#22, year_total#23, customer_id#31, customer_preferred_cust_flag#32, year_total#33] (41) 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#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (42) 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#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (43) 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#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (44) 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#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] 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#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (45) 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#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] (46) 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#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_bill_customer_sk#35) (47) 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#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#39] (48) 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#1] +Right keys [1]: [ws_bill_customer_sk#35] Join condition: None (49) 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#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] (50) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#62, d_year#63] +Output [2]: [d_date_sk#15, d_year#16] (51) 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#38] +Right keys [1]: [d_date_sk#15] Join condition: None (52) 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#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38, d_date_sk#15, d_year#16] (53) 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#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#40] +Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#41] (54) 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#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#41] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#42] (55) 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#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#41] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#43] +Results [2]: [c_customer_id#2 AS customer_id#44, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#43,18,2) AS year_total#45] (56) 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#44, year_total#45] +Condition : (isnotnull(year_total#45) AND (year_total#45 > 0.00)) (57) Project [codegen id : 11] -Output [2]: [customer_id#68 AS customer_id#70, year_total#69 AS year_total#71] -Input [2]: [customer_id#68, year_total#69] +Output [2]: [customer_id#44 AS customer_id#46, year_total#45 AS year_total#47] +Input [2]: [customer_id#44, year_total#45] (58) BroadcastExchange -Input [2]: [customer_id#70, year_total#71] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#72] +Input [2]: [customer_id#46, year_total#47] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] (59) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#70] +Right keys [1]: [customer_id#46] Join condition: None (60) Project [codegen id : 16] -Output [5]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, year_total#71] -Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, customer_id#70, year_total#71] +Output [5]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33, year_total#47] +Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33, customer_id#46, year_total#47] (61) Scan parquet default.customer -Output [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (62) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (63) Filter [codegen id : 14] -Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] -Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_customer_id#74)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (64) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Output [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#84), dynamicpruningexpression(ws_sold_date_sk#84 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] (66) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] -Condition : isnotnull(ws_bill_customer_sk#81) +Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_bill_customer_sk#35) (67) BroadcastExchange -Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#85] +Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] (68) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#73] -Right keys [1]: [ws_bill_customer_sk#81] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#35] Join condition: None (69) Project [codegen id : 14] -Output [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] -Input [12]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] (70) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#86, d_year#87] +Output [2]: [d_date_sk#15, d_year#16] (71) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#84] -Right keys [1]: [d_date_sk#86] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#15] Join condition: None (72) Project [codegen id : 14] -Output [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, d_year#87] -Input [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84, d_date_sk#86, d_year#87] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38, d_date_sk#15, d_year#16] (73) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, d_year#87] -Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#88] -Results [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#50] +Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#51] (74) Exchange -Input [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] -Arguments: hashpartitioning(c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#51] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#52] (75) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] -Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))#91] -Results [2]: [c_customer_id#74 AS customer_id#92, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))#91,18,2) AS year_total#93] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#51] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#53] +Results [2]: [c_customer_id#2 AS customer_id#54, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#53,18,2) AS year_total#55] (76) BroadcastExchange -Input [2]: [customer_id#92, year_total#93] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#94] +Input [2]: [customer_id#54, year_total#55] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] (77) BroadcastHashJoin [codegen id : 16] 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 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#54] +Join condition: (CASE WHEN (year_total#47 > 0.00) THEN CheckOverflow((promote_precision(year_total#55) / promote_precision(year_total#47)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#33) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE null END) (78) Project [codegen id : 16] -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#71, customer_id#92, year_total#93] +Output [1]: [customer_preferred_cust_flag#32] +Input [7]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33, year_total#47, customer_id#54, year_total#55] (79) 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#32] +Arguments: 100, [customer_preferred_cust_flag#32 ASC NULLS FIRST], [customer_preferred_cust_flag#32] ===== Subqueries ===== @@ -462,15 +462,15 @@ ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#15, d_year#16] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#24 ReusedExchange (81) (81) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#38, d_year#39] +Output [2]: [d_date_sk#15, d_year#16] -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#84 IN dynamicpruning#36 +Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt index 039a38acb9f22..4e258bb08b1d8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt @@ -55,7 +55,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -86,7 +86,7 @@ Arguments: [ws_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -94,7 +94,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt index 0c22f6333e5a9..0ae1f849bca42 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt @@ -44,7 +44,7 @@ Condition : isnotnull(ws_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -52,7 +52,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -79,7 +79,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt index 55dca59270306..e400eea2cea66 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt @@ -58,7 +58,7 @@ Condition : (((((isnotnull(ss_store_sk#4) AND isnotnull(ss_addr_sk#3)) AND isnot Output [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -66,7 +66,7 @@ Input [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] (6) Filter [codegen id : 1] Input [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] -Condition : (isnotnull(cd_demo_sk#12) AND ((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree )) OR ((cd_marital_status#13 = S) AND (cd_education_status#14 = College ))) OR ((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree )))) +Condition : (isnotnull(cd_demo_sk#12) AND ((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree)) OR ((cd_marital_status#13 = S) AND (cd_education_status#14 = College))) OR ((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree)))) (7) BroadcastExchange Input [3]: [cd_demo_sk#12, cd_marital_status#13, cd_education_status#14] @@ -75,7 +75,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (8) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#12] -Join condition: ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) +Join condition: ((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#13 = S) AND (cd_education_status#14 = College)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) (9) Project [codegen id : 6] Output [11]: [ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, cd_marital_status#13, cd_education_status#14] @@ -102,7 +102,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (14) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#16] -Join condition: (((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#17 = 3)) OR (((((cd_marital_status#13 = S) AND (cd_education_status#14 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#17 = 1))) OR (((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#17 = 1))) +Join condition: (((((((cd_marital_status#13 = M) AND (cd_education_status#14 = Advanced Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#17 = 3)) OR (((((cd_marital_status#13 = S) AND (cd_education_status#14 = College)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#17 = 1))) OR (((((cd_marital_status#13 = W) AND (cd_education_status#14 = 2 yr Degree)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#17 = 1))) (15) Project [codegen id : 6] Output [7]: [ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt index 027fde38ff7fa..560738591c930 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/explain.txt @@ -147,7 +147,7 @@ Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ex Output [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] ReadSchema: struct (25) ColumnarToRow [codegen id : 4] @@ -155,7 +155,7 @@ Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] (26) Filter [codegen id : 4] Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] -Condition : (isnotnull(cd_demo_sk#21) AND ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) OR ((cd_marital_status#22 = S) AND (cd_education_status#23 = College ))) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )))) +Condition : (isnotnull(cd_demo_sk#21) AND ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) OR ((cd_marital_status#22 = S) AND (cd_education_status#23 = College))) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)))) (27) BroadcastExchange Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] @@ -164,7 +164,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (28) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#21] -Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) +Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) (29) Project [codegen id : 6] Output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#22, cd_education_status#23] @@ -191,7 +191,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (34) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_hdemo_sk#2] Right keys [1]: [hd_demo_sk#25] -Join condition: (((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#26 = 3)) OR (((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#26 = 1))) OR (((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#26 = 1))) +Join condition: (((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#26 = 3)) OR (((((cd_marital_status#22 = S) AND (cd_education_status#23 = College)) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#26 = 1))) OR (((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#26 = 1))) (35) Project [codegen id : 6] Output [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt index e4ec487623d2c..6033ad52be4e9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt @@ -173,580 +173,580 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (10) ColumnarToRow [codegen id : 10] -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] (11) Filter [codegen id : 10] -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_item_sk#11) +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#12, d_year#13] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#14, d_year#15] +Input [2]: [d_date_sk#12, d_year#13] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#14, d_year#15] -Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1999)) AND (d_year#15 <= 2001)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_year#15] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (16) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (17) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (18) Project [codegen id : 10] -Output [1]: [ss_item_sk#11] -Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] +Output [1]: [ss_item_sk#1] +Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] (19) Scan parquet default.item -Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Condition : (((isnotnull(i_item_sk#15) AND isnotnull(i_brand_id#16)) AND isnotnull(i_class_id#17)) AND isnotnull(i_category_id#18)) (22) Exchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: hashpartitioning(coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] (23) Sort [codegen id : 5] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: [coalesce(i_brand_id#16, 0) ASC NULLS FIRST, isnull(i_brand_id#16) ASC NULLS FIRST, coalesce(i_class_id#17, 0) ASC NULLS FIRST, isnull(i_class_id#17) ASC NULLS FIRST, coalesce(i_category_id#18, 0) ASC NULLS FIRST, isnull(i_category_id#18) ASC NULLS FIRST], false, 0 (24) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 8] -Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] (26) Filter [codegen id : 8] -Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] -Condition : isnotnull(cs_item_sk#22) +Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +Condition : isnotnull(cs_item_sk#20) (27) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#24] +Output [1]: [d_date_sk#22] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#22] Join condition: None (29) Project [codegen id : 8] -Output [1]: [cs_item_sk#22] -Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] +Output [1]: [cs_item_sk#20] +Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#22] (30) Scan parquet default.item -Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Output [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] (32) Filter [codegen id : 7] -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -Condition : isnotnull(i_item_sk#25) +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Condition : isnotnull(i_item_sk#23) (33) BroadcastExchange -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#22] -Right keys [1]: [i_item_sk#25] +Left keys [1]: [cs_item_sk#20] +Right keys [1]: [i_item_sk#23] Join condition: None (35) Project [codegen id : 8] -Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Output [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Input [5]: [cs_item_sk#20, i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] (36) Exchange -Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: hashpartitioning(coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26), 5), ENSURE_REQUIREMENTS, [id=#28] (37) Sort [codegen id : 9] -Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 +Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: [coalesce(i_brand_id#24, 0) ASC NULLS FIRST, isnull(i_brand_id#24) ASC NULLS FIRST, coalesce(i_class_id#25, 0) ASC NULLS FIRST, isnull(i_class_id#25) ASC NULLS FIRST, coalesce(i_category_id#26, 0) ASC NULLS FIRST, isnull(i_category_id#26) ASC NULLS FIRST], false, 0 (38) SortMergeJoin -Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] -Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] +Left keys [6]: [coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18)] +Right keys [6]: [coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26)] Join condition: None (39) BroadcastExchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#11] -Right keys [1]: [i_item_sk#17] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#15] Join condition: None (41) Project [codegen id : 10] -Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] -Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [3]: [i_brand_id#16 AS brand_id#30, i_class_id#17 AS class_id#31, i_category_id#18 AS category_id#32] +Input [5]: [ss_item_sk#1, i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] (42) HashAggregate [codegen id : 10] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#30, class_id#31, category_id#32] (43) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] (44) HashAggregate [codegen id : 11] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#30, class_id#31, category_id#32] (45) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32), 5), ENSURE_REQUIREMENTS, [id=#34] (46) Sort [codegen id : 12] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: [coalesce(brand_id#30, 0) ASC NULLS FIRST, isnull(brand_id#30) ASC NULLS FIRST, coalesce(class_id#31, 0) ASC NULLS FIRST, isnull(class_id#31) ASC NULLS FIRST, coalesce(category_id#32, 0) ASC NULLS FIRST, isnull(category_id#32) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Output [2]: [ws_item_sk#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 15] -Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] (49) Filter [codegen id : 15] -Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_item_sk#37) +Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#35) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#39] +Output [1]: [d_date_sk#37] (51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#39] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#37] Join condition: None (52) Project [codegen id : 15] -Output [1]: [ws_item_sk#37] -Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] +Output [1]: [ws_item_sk#35] +Input [3]: [ws_item_sk#35, ws_sold_date_sk#36, d_date_sk#37] (53) ReusedExchange [Reuses operator id: 33] -Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] +Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] (54) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#40] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [i_item_sk#38] Join condition: None (55) Project [codegen id : 15] -Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] +Output [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Input [5]: [ws_item_sk#35, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] (56) Exchange -Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] +Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: hashpartitioning(coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41), 5), ENSURE_REQUIREMENTS, [id=#42] (57) Sort [codegen id : 16] -Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 +Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: [coalesce(i_brand_id#39, 0) ASC NULLS FIRST, isnull(i_brand_id#39) ASC NULLS FIRST, coalesce(i_class_id#40, 0) ASC NULLS FIRST, isnull(i_class_id#40) ASC NULLS FIRST, coalesce(i_category_id#41, 0) ASC NULLS FIRST, isnull(i_category_id#41) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] -Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41)] Join condition: None (59) HashAggregate [codegen id : 17] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#30, class_id#31, category_id#32] (60) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#43] (61) HashAggregate [codegen id : 18] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#30, class_id#31, category_id#32] (62) BroadcastExchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#44] (63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#32, class_id#33, category_id#34] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None (64) Project [codegen id : 19] -Output [1]: [i_item_sk#7 AS ss_item_sk#47] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] +Output [1]: [i_item_sk#7 AS ss_item_sk#45] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#30, class_id#31, category_id#32] (65) Exchange -Input [1]: [ss_item_sk#47] -Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [1]: [ss_item_sk#45] +Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] (66) Sort [codegen id : 20] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#47] +Right keys [1]: [ss_item_sk#45] Join condition: None (68) Scan parquet default.date_dim -Output [3]: [d_date_sk#49, d_year#50, d_moy#51] +Output [3]: [d_date_sk#12, d_year#13, d_moy#47] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 21] -Input [3]: [d_date_sk#49, d_year#50, d_moy#51] +Input [3]: [d_date_sk#12, d_year#13, d_moy#47] (70) Filter [codegen id : 21] -Input [3]: [d_date_sk#49, d_year#50, d_moy#51] -Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2001)) AND (d_moy#51 = 11)) AND isnotnull(d_date_sk#49)) +Input [3]: [d_date_sk#12, d_year#13, d_moy#47] +Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#47)) AND (d_year#13 = 2001)) AND (d_moy#47 = 11)) AND isnotnull(d_date_sk#12)) (71) Project [codegen id : 21] -Output [1]: [d_date_sk#49] -Input [3]: [d_date_sk#49, d_year#50, d_moy#51] +Output [1]: [d_date_sk#12] +Input [3]: [d_date_sk#12, d_year#13, d_moy#47] (72) BroadcastExchange -Input [1]: [d_date_sk#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] (73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#49] +Right keys [1]: [d_date_sk#12] Join condition: None (74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (75) Scan parquet default.item -Output [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 22] -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (77) Filter [codegen id : 22] -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Condition : isnotnull(i_item_sk#53) +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : isnotnull(i_item_sk#7) (78) Exchange -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: hashpartitioning(i_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#49] (79) Sort [codegen id : 23] -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [i_item_sk#53 ASC NULLS FIRST], false, 0 +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#45] (81) Sort [codegen id : 41] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#53] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [i_item_sk#7] +Right keys [1]: [ss_item_sk#45] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] (84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#53] +Right keys [1]: [i_item_sk#7] Join condition: None (85) Project [codegen id : 42] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (86) HashAggregate [codegen id : 42] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] -Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#59, isEmpty#60, count#61] -Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] +Aggregate Attributes [3]: [sum#51, isEmpty#52, count#53] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] (87) Exchange -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] -Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#57] (88) HashAggregate [codegen id : 43] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] -Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66, count(1)#67] -Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] (89) Filter [codegen id : 43] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(32,6)) > cast(Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (90) Project [codegen id : 43] -Output [6]: [sales#68, number_sales#69, store AS channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +Output [6]: [sales#60, number_sales#61, store AS channel#65, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] (91) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Output [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#77), dynamicpruningexpression(cs_sold_date_sk#77 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 44] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] (93) Filter [codegen id : 44] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Condition : isnotnull(cs_item_sk#74) +Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Condition : isnotnull(cs_item_sk#20) (94) Exchange -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Arguments: hashpartitioning(cs_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#68] (95) Sort [codegen id : 45] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Arguments: [cs_item_sk#74 ASC NULLS FIRST], false, 0 +Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Arguments: [cs_item_sk#20 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#45] (97) Sort [codegen id : 63] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [cs_item_sk#74] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [cs_item_sk#20] +Right keys [1]: [ss_item_sk#45] Join condition: None (99) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#79] +Output [1]: [d_date_sk#69] (100) BroadcastHashJoin [codegen id : 85] -Left keys [1]: [cs_sold_date_sk#77] -Right keys [1]: [d_date_sk#79] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#69] Join condition: None (101) Project [codegen id : 85] -Output [3]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76] -Input [5]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77, d_date_sk#79] +Output [3]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67] +Input [5]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21, d_date_sk#69] (102) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] +Output [4]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73] (103) BroadcastHashJoin [codegen id : 85] -Left keys [1]: [cs_item_sk#74] -Right keys [1]: [i_item_sk#80] +Left keys [1]: [cs_item_sk#20] +Right keys [1]: [i_item_sk#70] Join condition: None (104) Project [codegen id : 85] -Output [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] -Input [7]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] +Output [5]: [cs_quantity#66, cs_list_price#67, i_brand_id#71, i_class_id#72, i_category_id#73] +Input [7]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73] (105) HashAggregate [codegen id : 85] -Input [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] -Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +Input [5]: [cs_quantity#66, cs_list_price#67, i_brand_id#71, i_class_id#72, i_category_id#73] +Keys [3]: [i_brand_id#71, i_class_id#72, i_category_id#73] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#74, isEmpty#75, count#76] +Results [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] (106) Exchange -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] -Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] +Arguments: hashpartitioning(i_brand_id#71, i_class_id#72, i_category_id#73, 5), ENSURE_REQUIREMENTS, [id=#80] (107) HashAggregate [codegen id : 86] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] -Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] +Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] +Keys [3]: [i_brand_id#71, i_class_id#72, i_category_id#73] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81, count(1)#82] +Results [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sales#83, count(1)#82 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] (108) Filter [codegen id : 86] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (109) Project [codegen id : 86] -Output [6]: [sales#93, number_sales#94, catalog AS channel#96, i_brand_id#81, i_class_id#82, i_category_id#83] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] +Output [6]: [sales#83, number_sales#84, catalog AS channel#86, i_brand_id#71, i_class_id#72, i_category_id#73] +Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] (110) Scan parquet default.web_sales -Output [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Output [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#100), dynamicpruningexpression(ws_sold_date_sk#100 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (111) ColumnarToRow [codegen id : 87] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] (112) Filter [codegen id : 87] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Condition : isnotnull(ws_item_sk#97) +Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#35) (113) Exchange -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Arguments: hashpartitioning(ws_item_sk#97, 5), ENSURE_REQUIREMENTS, [id=#101] +Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Arguments: hashpartitioning(ws_item_sk#35, 5), ENSURE_REQUIREMENTS, [id=#89] (114) Sort [codegen id : 88] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Arguments: [ws_item_sk#97 ASC NULLS FIRST], false, 0 +Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Arguments: [ws_item_sk#35 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#45] (116) Sort [codegen id : 106] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (117) SortMergeJoin -Left keys [1]: [ws_item_sk#97] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [ss_item_sk#45] Join condition: None (118) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#102] +Output [1]: [d_date_sk#90] (119) BroadcastHashJoin [codegen id : 128] -Left keys [1]: [ws_sold_date_sk#100] -Right keys [1]: [d_date_sk#102] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#90] Join condition: None (120) Project [codegen id : 128] -Output [3]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99] -Input [5]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100, d_date_sk#102] +Output [3]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88] +Input [5]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36, d_date_sk#90] (121) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] +Output [4]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94] (122) BroadcastHashJoin [codegen id : 128] -Left keys [1]: [ws_item_sk#97] -Right keys [1]: [i_item_sk#103] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [i_item_sk#91] Join condition: None (123) Project [codegen id : 128] -Output [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] -Input [7]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] +Output [5]: [ws_quantity#87, ws_list_price#88, i_brand_id#92, i_class_id#93, i_category_id#94] +Input [7]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94] (124) HashAggregate [codegen id : 128] -Input [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] -Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#107, isEmpty#108, count#109] -Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +Input [5]: [ws_quantity#87, ws_list_price#88, i_brand_id#92, i_class_id#93, i_category_id#94] +Keys [3]: [i_brand_id#92, i_class_id#93, i_category_id#94] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#95, isEmpty#96, count#97] +Results [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] (125) Exchange -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] -Arguments: hashpartitioning(i_brand_id#104, i_class_id#105, i_category_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] +Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] +Arguments: hashpartitioning(i_brand_id#92, i_class_id#93, i_category_id#94, 5), ENSURE_REQUIREMENTS, [id=#101] (126) HashAggregate [codegen id : 129] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] -Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114, count(1)#115] -Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sales#116, count(1)#115 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] +Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] +Keys [3]: [i_brand_id#92, i_class_id#93, i_category_id#94] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102, count(1)#103] +Results [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sales#104, count(1)#103 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] (127) Filter [codegen id : 129] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (128) Project [codegen id : 129] -Output [6]: [sales#116, number_sales#117, web AS channel#119, i_brand_id#104, i_class_id#105, i_category_id#106] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] +Output [6]: [sales#104, number_sales#105, web AS channel#107, i_brand_id#92, i_class_id#93, i_category_id#94] +Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] (129) Union (130) Expand [codegen id : 130] -Input [6]: [sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [List(sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, 0), List(sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, null, 1), List(sales#68, number_sales#69, channel#73, i_brand_id#54, null, null, 3), List(sales#68, number_sales#69, channel#73, null, null, null, 7), List(sales#68, number_sales#69, null, null, null, null, 15)], [sales#68, number_sales#69, channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] +Input [6]: [sales#60, number_sales#61, channel#65, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [List(sales#60, number_sales#61, channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, 0), List(sales#60, number_sales#61, channel#65, i_brand_id#8, i_class_id#9, null, 1), List(sales#60, number_sales#61, channel#65, i_brand_id#8, null, null, 3), List(sales#60, number_sales#61, channel#65, null, null, null, 7), List(sales#60, number_sales#61, null, null, null, null, 15)], [sales#60, number_sales#61, channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112] (131) HashAggregate [codegen id : 130] -Input [7]: [sales#68, number_sales#69, channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] -Keys [5]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#125, isEmpty#126, sum#127] -Results [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] +Input [7]: [sales#60, number_sales#61, channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112] +Keys [5]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112] +Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] +Aggregate Attributes [3]: [sum#113, isEmpty#114, sum#115] +Results [8]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112, sum#116, isEmpty#117, sum#118] (132) Exchange -Input [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] -Arguments: hashpartitioning(channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, 5), ENSURE_REQUIREMENTS, [id=#131] +Input [8]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112, sum#116, isEmpty#117, sum#118] +Arguments: hashpartitioning(channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112, 5), ENSURE_REQUIREMENTS, [id=#119] (133) HashAggregate [codegen id : 131] -Input [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] -Keys [5]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#132, sum(number_sales#69)#133] -Results [6]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales#68)#132 AS sum(sales)#134, sum(number_sales#69)#133 AS sum(number_sales)#135] +Input [8]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112, sum#116, isEmpty#117, sum#118] +Keys [5]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112] +Functions [2]: [sum(sales#60), sum(number_sales#61)] +Aggregate Attributes [2]: [sum(sales#60)#120, sum(number_sales#61)#121] +Results [6]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, sum(sales#60)#120 AS sum(sales)#122, sum(number_sales#61)#121 AS sum(number_sales)#123] (134) TakeOrderedAndProject -Input [6]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales)#134, sum(number_sales)#135] -Arguments: 100, [channel#120 ASC NULLS FIRST, i_brand_id#121 ASC NULLS FIRST, i_class_id#122 ASC NULLS FIRST, i_category_id#123 ASC NULLS FIRST], [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales)#134, sum(number_sales)#135] +Input [6]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, sum(sales)#122, sum(number_sales)#123] +Arguments: 100, [channel#108 ASC NULLS FIRST, i_brand_id#109 ASC NULLS FIRST, i_class_id#110 ASC NULLS FIRST, i_category_id#111 ASC NULLS FIRST], [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, sum(sales)#122, sum(number_sales)#123] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#71, [id=#72] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#63, [id=#64] * HashAggregate (157) +- Exchange (156) +- * HashAggregate (155) @@ -773,145 +773,145 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (135) Scan parquet default.store_sales -Output [3]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138] +Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#138), dynamicpruningexpression(ss_sold_date_sk#138 IN dynamicpruning#139)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#124)] ReadSchema: struct (136) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138] +Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (137) Scan parquet default.date_dim -Output [2]: [d_date_sk#140, d_year#141] +Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (138) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#140, d_year#141] +Input [2]: [d_date_sk#12, d_year#13] (139) Filter [codegen id : 1] -Input [2]: [d_date_sk#140, d_year#141] -Condition : (((isnotnull(d_year#141) AND (d_year#141 >= 1999)) AND (d_year#141 <= 2001)) AND isnotnull(d_date_sk#140)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (140) Project [codegen id : 1] -Output [1]: [d_date_sk#140] -Input [2]: [d_date_sk#140, d_year#141] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (141) BroadcastExchange -Input [1]: [d_date_sk#140] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#142] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#125] (142) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#138] -Right keys [1]: [d_date_sk#140] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (143) Project [codegen id : 2] -Output [2]: [ss_quantity#136 AS quantity#143, ss_list_price#137 AS list_price#144] -Input [4]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138, d_date_sk#140] +Output [2]: [ss_quantity#2 AS quantity#126, ss_list_price#3 AS list_price#127] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (144) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147] +Output [3]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#147), dynamicpruningexpression(cs_sold_date_sk#147 IN dynamicpruning#139)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#124)] ReadSchema: struct (145) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147] +Input [3]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] (146) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#148] +Output [1]: [d_date_sk#128] (147) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#147] -Right keys [1]: [d_date_sk#148] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#128] Join condition: None (148) Project [codegen id : 4] -Output [2]: [cs_quantity#145 AS quantity#149, cs_list_price#146 AS list_price#150] -Input [4]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147, d_date_sk#148] +Output [2]: [cs_quantity#66 AS quantity#129, cs_list_price#67 AS list_price#130] +Input [4]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21, d_date_sk#128] (149) Scan parquet default.web_sales -Output [3]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153] +Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#153), dynamicpruningexpression(ws_sold_date_sk#153 IN dynamicpruning#139)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#124)] ReadSchema: struct (150) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153] +Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] (151) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#154] +Output [1]: [d_date_sk#131] (152) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#153] -Right keys [1]: [d_date_sk#154] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#131] Join condition: None (153) Project [codegen id : 6] -Output [2]: [ws_quantity#151 AS quantity#155, ws_list_price#152 AS list_price#156] -Input [4]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153, d_date_sk#154] +Output [2]: [ws_quantity#87 AS quantity#132, ws_list_price#88 AS list_price#133] +Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36, d_date_sk#131] (154) Union (155) HashAggregate [codegen id : 7] -Input [2]: [quantity#143, list_price#144] +Input [2]: [quantity#126, list_price#127] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#157, count#158] -Results [2]: [sum#159, count#160] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#126 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#127 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#134, count#135] +Results [2]: [sum#136, count#137] (156) Exchange -Input [2]: [sum#159, count#160] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#161] +Input [2]: [sum#136, count#137] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#138] (157) HashAggregate [codegen id : 8] -Input [2]: [sum#159, count#160] +Input [2]: [sum#136, count#137] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))#162] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))#162 AS average_sales#163] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#126 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#127 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#126 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#127 as decimal(12,2)))), DecimalType(18,2), true))#139] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#126 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#127 as decimal(12,2)))), DecimalType(18,2), true))#139 AS average_sales#140] -Subquery:2 Hosting operator id = 135 Hosting Expression = ss_sold_date_sk#138 IN dynamicpruning#139 +Subquery:2 Hosting operator id = 135 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#124 ReusedExchange (158) (158) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#140] +Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 144 Hosting Expression = cs_sold_date_sk#147 IN dynamicpruning#139 +Subquery:3 Hosting operator id = 144 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#124 -Subquery:4 Hosting operator id = 149 Hosting Expression = ws_sold_date_sk#153 IN dynamicpruning#139 +Subquery:4 Hosting operator id = 149 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#124 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (159) (159) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#49] +Output [1]: [d_date_sk#12] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 ReusedExchange (160) (160) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#11 -Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#77 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 6f61fc8e96ae1..34aa615cf910c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -146,512 +146,512 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (10) Scan parquet default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (11) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] (12) Filter [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14)) (13) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_item_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] (15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) +Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Condition : isnotnull(cs_item_sk#15) (16) Scan parquet default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (18) Filter [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : isnotnull(i_item_sk#17) (19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [i_item_sk#17] Join condition: None (21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (22) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_year#25] +Output [2]: [d_date_sk#22, d_year#23] 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 (23) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#24, d_year#25] +Input [2]: [d_date_sk#22, d_year#23] (24) Filter [codegen id : 2] -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) +Input [2]: [d_date_sk#22, d_year#23] +Condition : (((isnotnull(d_year#23) AND (d_year#23 >= 1999)) AND (d_year#23 <= 2001)) AND isnotnull(d_date_sk#22)) (25) Project [codegen id : 2] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_year#25] +Output [1]: [d_date_sk#22] +Input [2]: [d_date_sk#22, d_year#23] (26) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (27) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#22] Join condition: None (28) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] +Output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] +Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#22] (29) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] +Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] (30) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] +Left keys [6]: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)] +Right keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] Join condition: None (31) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] (32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#11] Join condition: None (33) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Output [4]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14] +Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] (34) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#27] (35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#27] Join condition: None (36) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] +Output [3]: [i_brand_id#12 AS brand_id#28, i_class_id#13 AS class_id#29, i_category_id#14 AS category_id#30] +Input [5]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#27] (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#28, class_id#29, category_id#30] (38) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [id=#31] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#28, class_id#29, category_id#30] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Output [2]: [ws_item_sk#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_item_sk#34) +Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Condition : isnotnull(ws_item_sk#32) (43) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] +Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#36] +Left keys [1]: [ws_item_sk#32] +Right keys [1]: [i_item_sk#34] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] +Output [4]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37] +Input [6]: [ws_item_sk#32, ws_sold_date_sk#33, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] (46) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#40] +Output [1]: [d_date_sk#38] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#40] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#38] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] +Output [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +Input [5]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] (49) BroadcastExchange -Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] +Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#39] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] -Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] +Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] +Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#28, class_id#29, category_id#30] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#28, class_id#29, category_id#30] (53) BroadcastExchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#40] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#30, class_id#31, category_id#32] +Right keys [3]: [brand_id#28, class_id#29, category_id#30] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#43] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] +Output [1]: [i_item_sk#6 AS ss_item_sk#41] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] (56) BroadcastExchange -Input [1]: [ss_item_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] +Input [1]: [ss_item_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#43] +Right keys [1]: [ss_item_sk#41] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] -Condition : isnotnull(i_item_sk#45) +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : isnotnull(i_item_sk#6) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#41] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#45] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [i_item_sk#6] +Right keys [1]: [ss_item_sk#41] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#45] +Right keys [1]: [i_item_sk#6] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (66) Scan parquet default.date_dim -Output [3]: [d_date_sk#50, d_year#51, d_moy#52] +Output [3]: [d_date_sk#27, d_year#44, d_moy#45] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [3]: [d_date_sk#50, d_year#51, d_moy#52] +Input [3]: [d_date_sk#27, d_year#44, d_moy#45] (68) Filter [codegen id : 24] -Input [3]: [d_date_sk#50, d_year#51, d_moy#52] -Condition : ((((isnotnull(d_year#51) AND isnotnull(d_moy#52)) AND (d_year#51 = 2001)) AND (d_moy#52 = 11)) AND isnotnull(d_date_sk#50)) +Input [3]: [d_date_sk#27, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#27)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#50] -Input [3]: [d_date_sk#50, d_year#51, d_moy#52] +Output [1]: [d_date_sk#27] +Input [3]: [d_date_sk#27, d_year#44, d_moy#45] (70) BroadcastExchange -Input [1]: [d_date_sk#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#50] +Right keys [1]: [d_date_sk#27] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#27] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] -Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] +Aggregate Attributes [3]: [sum#47, isEmpty#48, count#49] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] (74) Exchange -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] -Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#53] (75) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] -Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] (76) Filter [codegen id : 26] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [sales#63, number_sales#64, store AS channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Output [6]: [sales#56, number_sales#57, store AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] (78) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +Output [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +Input [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] (80) Filter [codegen id : 51] -Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] -Condition : isnotnull(cs_item_sk#69) +Input [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Condition : isnotnull(cs_item_sk#15) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#41] (82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#69] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [ss_item_sk#41] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] +Output [4]: [i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] (84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#69] -Right keys [1]: [i_item_sk#73] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [i_item_sk#64] Join condition: None (85) Project [codegen id : 51] -Output [6]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76] -Input [8]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] +Output [6]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_brand_id#65, i_class_id#66, i_category_id#67] +Input [8]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] (86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#77] +Output [1]: [d_date_sk#68] (87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#72] -Right keys [1]: [d_date_sk#77] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#68] Join condition: None (88) Project [codegen id : 51] -Output [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] -Input [7]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76, d_date_sk#77] +Output [5]: [cs_quantity#62, cs_list_price#63, i_brand_id#65, i_class_id#66, i_category_id#67] +Input [7]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_brand_id#65, i_class_id#66, i_category_id#67, d_date_sk#68] (89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] -Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#78, isEmpty#79, count#80] -Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Input [5]: [cs_quantity#62, cs_list_price#63, i_brand_id#65, i_class_id#66, i_category_id#67] +Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] +Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] (90) Exchange -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] -Arguments: hashpartitioning(i_brand_id#74, i_class_id#75, i_category_id#76, 5), ENSURE_REQUIREMENTS, [id=#84] +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] +Arguments: hashpartitioning(i_brand_id#65, i_class_id#66, i_category_id#67, 5), ENSURE_REQUIREMENTS, [id=#75] (91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] -Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85, count(1)#86] -Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sales#87, count(1)#86 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] +Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76, count(1)#77] +Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sales#78, count(1)#77 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] (92) Filter [codegen id : 52] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [sales#87, number_sales#88, catalog AS channel#90, i_brand_id#74, i_class_id#75, i_category_id#76] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] +Output [6]: [sales#78, number_sales#79, catalog AS channel#81, i_brand_id#65, i_class_id#66, i_category_id#67] +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] (94) Scan parquet default.web_sales -Output [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Output [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Input [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] (96) Filter [codegen id : 77] -Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] -Condition : isnotnull(ws_item_sk#91) +Input [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Condition : isnotnull(ws_item_sk#32) (97) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#41] (98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#91] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [ws_item_sk#32] +Right keys [1]: [ss_item_sk#41] Join condition: None (99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] +Output [4]: [i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] (100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#91] -Right keys [1]: [i_item_sk#95] +Left keys [1]: [ws_item_sk#32] +Right keys [1]: [i_item_sk#84] Join condition: None (101) Project [codegen id : 77] -Output [6]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98] -Input [8]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] +Output [6]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_brand_id#85, i_class_id#86, i_category_id#87] +Input [8]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] (102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#99] +Output [1]: [d_date_sk#88] (103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#94] -Right keys [1]: [d_date_sk#99] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#88] Join condition: None (104) Project [codegen id : 77] -Output [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] -Input [7]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98, d_date_sk#99] +Output [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#85, i_class_id#86, i_category_id#87] +Input [7]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_brand_id#85, i_class_id#86, i_category_id#87, d_date_sk#88] (105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] -Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#100, isEmpty#101, count#102] -Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Input [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#85, i_class_id#86, i_category_id#87] +Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#89, isEmpty#90, count#91] +Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] (106) Exchange -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] -Arguments: hashpartitioning(i_brand_id#96, i_class_id#97, i_category_id#98, 5), ENSURE_REQUIREMENTS, [id=#106] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, 5), ENSURE_REQUIREMENTS, [id=#95] (107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] -Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] -Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#109, count(1)#108 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96, count(1)#97] +Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sales#98, count(1)#97 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] (108) Filter [codegen id : 78] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [sales#109, number_sales#110, web AS channel#112, i_brand_id#96, i_class_id#97, i_category_id#98] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] +Output [6]: [sales#98, number_sales#99, web AS channel#101, i_brand_id#85, i_class_id#86, i_category_id#87] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] (110) Union (111) Expand [codegen id : 79] -Input [6]: [sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] -Arguments: [List(sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, 0), List(sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, null, 1), List(sales#63, number_sales#64, channel#68, i_brand_id#46, null, null, 3), List(sales#63, number_sales#64, channel#68, null, null, null, 7), List(sales#63, number_sales#64, null, null, null, null, 15)], [sales#63, number_sales#64, channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] +Input [6]: [sales#56, number_sales#57, channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: [List(sales#56, number_sales#57, channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#56, number_sales#57, channel#61, i_brand_id#7, i_class_id#8, null, 1), List(sales#56, number_sales#57, channel#61, i_brand_id#7, null, null, 3), List(sales#56, number_sales#57, channel#61, null, null, null, 7), List(sales#56, number_sales#57, null, null, null, null, 15)], [sales#56, number_sales#57, channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] (112) HashAggregate [codegen id : 79] -Input [7]: [sales#63, number_sales#64, channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] -Keys [5]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#118, isEmpty#119, sum#120] -Results [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] +Input [7]: [sales#56, number_sales#57, channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] +Keys [5]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] +Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] +Aggregate Attributes [3]: [sum#107, isEmpty#108, sum#109] +Results [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] (113) Exchange -Input [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] -Arguments: hashpartitioning(channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] +Arguments: hashpartitioning(channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] (114) HashAggregate [codegen id : 80] -Input [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] -Keys [5]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#125, sum(number_sales#64)#126] -Results [6]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales#63)#125 AS sum(sales)#127, sum(number_sales#64)#126 AS sum(number_sales)#128] +Input [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] +Keys [5]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] +Functions [2]: [sum(sales#56), sum(number_sales#57)] +Aggregate Attributes [2]: [sum(sales#56)#114, sum(number_sales#57)#115] +Results [6]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales#56)#114 AS sum(sales)#116, sum(number_sales#57)#115 AS sum(number_sales)#117] (115) TakeOrderedAndProject -Input [6]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales)#127, sum(number_sales)#128] -Arguments: 100, [channel#113 ASC NULLS FIRST, i_brand_id#114 ASC NULLS FIRST, i_class_id#115 ASC NULLS FIRST, i_category_id#116 ASC NULLS FIRST], [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales)#127, sum(number_sales)#128] +Input [6]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales)#116, sum(number_sales)#117] +Arguments: 100, [channel#102 ASC NULLS FIRST, i_brand_id#103 ASC NULLS FIRST, i_class_id#104 ASC NULLS FIRST, i_category_id#105 ASC NULLS FIRST], [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales)#116, sum(number_sales)#117] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#66, [id=#67] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#59, [id=#60] * HashAggregate (138) +- Exchange (137) +- * HashAggregate (136) @@ -678,145 +678,145 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (116) Scan parquet default.store_sales -Output [3]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131] +Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#131), dynamicpruningexpression(ss_sold_date_sk#131 IN dynamicpruning#132)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#118)] ReadSchema: struct (117) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131] +Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (118) Scan parquet default.date_dim -Output [2]: [d_date_sk#133, d_year#134] +Output [2]: [d_date_sk#27, d_year#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (119) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#133, d_year#134] +Input [2]: [d_date_sk#27, d_year#44] (120) Filter [codegen id : 1] -Input [2]: [d_date_sk#133, d_year#134] -Condition : (((isnotnull(d_year#134) AND (d_year#134 >= 1999)) AND (d_year#134 <= 2001)) AND isnotnull(d_date_sk#133)) +Input [2]: [d_date_sk#27, d_year#44] +Condition : (((isnotnull(d_year#44) AND (d_year#44 >= 1999)) AND (d_year#44 <= 2001)) AND isnotnull(d_date_sk#27)) (121) Project [codegen id : 1] -Output [1]: [d_date_sk#133] -Input [2]: [d_date_sk#133, d_year#134] +Output [1]: [d_date_sk#27] +Input [2]: [d_date_sk#27, d_year#44] (122) BroadcastExchange -Input [1]: [d_date_sk#133] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#135] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#119] (123) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#131] -Right keys [1]: [d_date_sk#133] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#27] Join condition: None (124) Project [codegen id : 2] -Output [2]: [ss_quantity#129 AS quantity#136, ss_list_price#130 AS list_price#137] -Input [4]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131, d_date_sk#133] +Output [2]: [ss_quantity#2 AS quantity#120, ss_list_price#3 AS list_price#121] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#27] (125) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140] +Output [3]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#140), dynamicpruningexpression(cs_sold_date_sk#140 IN dynamicpruning#132)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#118)] ReadSchema: struct (126) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140] +Input [3]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] (127) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#141] +Output [1]: [d_date_sk#122] (128) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#140] -Right keys [1]: [d_date_sk#141] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#122] Join condition: None (129) Project [codegen id : 4] -Output [2]: [cs_quantity#138 AS quantity#142, cs_list_price#139 AS list_price#143] -Input [4]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140, d_date_sk#141] +Output [2]: [cs_quantity#62 AS quantity#123, cs_list_price#63 AS list_price#124] +Input [4]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, d_date_sk#122] (130) Scan parquet default.web_sales -Output [3]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146] +Output [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#146), dynamicpruningexpression(ws_sold_date_sk#146 IN dynamicpruning#132)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#118)] ReadSchema: struct (131) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146] +Input [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] (132) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#147] +Output [1]: [d_date_sk#125] (133) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#146] -Right keys [1]: [d_date_sk#147] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#125] Join condition: None (134) Project [codegen id : 6] -Output [2]: [ws_quantity#144 AS quantity#148, ws_list_price#145 AS list_price#149] -Input [4]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146, d_date_sk#147] +Output [2]: [ws_quantity#82 AS quantity#126, ws_list_price#83 AS list_price#127] +Input [4]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, d_date_sk#125] (135) Union (136) HashAggregate [codegen id : 7] -Input [2]: [quantity#136, list_price#137] +Input [2]: [quantity#120, list_price#121] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#150, count#151] -Results [2]: [sum#152, count#153] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#120 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#121 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#128, count#129] +Results [2]: [sum#130, count#131] (137) Exchange -Input [2]: [sum#152, count#153] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#154] +Input [2]: [sum#130, count#131] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#132] (138) HashAggregate [codegen id : 8] -Input [2]: [sum#152, count#153] +Input [2]: [sum#130, count#131] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#155] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#155 AS average_sales#156] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#120 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#121 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#120 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#121 as decimal(12,2)))), DecimalType(18,2), true))#133] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#120 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#121 as decimal(12,2)))), DecimalType(18,2), true))#133 AS average_sales#134] -Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#131 IN dynamicpruning#132 +Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#118 ReusedExchange (139) (139) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#133] +Output [1]: [d_date_sk#27] -Subquery:3 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#140 IN dynamicpruning#132 +Subquery:3 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#118 -Subquery:4 Hosting operator id = 130 Hosting Expression = ws_sold_date_sk#146 IN dynamicpruning#132 +Subquery:4 Hosting operator id = 130 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#118 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#50] +Output [1]: [d_date_sk#27] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 ReusedExchange (141) (141) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#27] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#10 -Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt index fab231dc80a6c..013c3339f6ec3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt @@ -155,499 +155,499 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (10) ColumnarToRow [codegen id : 10] -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] (11) Filter [codegen id : 10] -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_item_sk#11) +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#12, d_year#13] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#14, d_year#15] +Input [2]: [d_date_sk#12, d_year#13] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#14, d_year#15] -Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1999)) AND (d_year#15 <= 2001)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_year#15] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (16) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (17) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (18) Project [codegen id : 10] -Output [1]: [ss_item_sk#11] -Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] +Output [1]: [ss_item_sk#1] +Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] (19) Scan parquet default.item -Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Condition : (((isnotnull(i_item_sk#15) AND isnotnull(i_brand_id#16)) AND isnotnull(i_class_id#17)) AND isnotnull(i_category_id#18)) (22) Exchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: hashpartitioning(coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] (23) Sort [codegen id : 5] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: [coalesce(i_brand_id#16, 0) ASC NULLS FIRST, isnull(i_brand_id#16) ASC NULLS FIRST, coalesce(i_class_id#17, 0) ASC NULLS FIRST, isnull(i_class_id#17) ASC NULLS FIRST, coalesce(i_category_id#18, 0) ASC NULLS FIRST, isnull(i_category_id#18) ASC NULLS FIRST], false, 0 (24) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 8] -Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] (26) Filter [codegen id : 8] -Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] -Condition : isnotnull(cs_item_sk#22) +Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +Condition : isnotnull(cs_item_sk#20) (27) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#24] +Output [1]: [d_date_sk#22] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#22] Join condition: None (29) Project [codegen id : 8] -Output [1]: [cs_item_sk#22] -Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] +Output [1]: [cs_item_sk#20] +Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#22] (30) Scan parquet default.item -Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Output [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] (32) Filter [codegen id : 7] -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -Condition : isnotnull(i_item_sk#25) +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Condition : isnotnull(i_item_sk#23) (33) BroadcastExchange -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#22] -Right keys [1]: [i_item_sk#25] +Left keys [1]: [cs_item_sk#20] +Right keys [1]: [i_item_sk#23] Join condition: None (35) Project [codegen id : 8] -Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Output [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Input [5]: [cs_item_sk#20, i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] (36) Exchange -Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: hashpartitioning(coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26), 5), ENSURE_REQUIREMENTS, [id=#28] (37) Sort [codegen id : 9] -Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 +Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: [coalesce(i_brand_id#24, 0) ASC NULLS FIRST, isnull(i_brand_id#24) ASC NULLS FIRST, coalesce(i_class_id#25, 0) ASC NULLS FIRST, isnull(i_class_id#25) ASC NULLS FIRST, coalesce(i_category_id#26, 0) ASC NULLS FIRST, isnull(i_category_id#26) ASC NULLS FIRST], false, 0 (38) SortMergeJoin -Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] -Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] +Left keys [6]: [coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18)] +Right keys [6]: [coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26)] Join condition: None (39) BroadcastExchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#11] -Right keys [1]: [i_item_sk#17] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#15] Join condition: None (41) Project [codegen id : 10] -Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] -Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [3]: [i_brand_id#16 AS brand_id#30, i_class_id#17 AS class_id#31, i_category_id#18 AS category_id#32] +Input [5]: [ss_item_sk#1, i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] (42) HashAggregate [codegen id : 10] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#30, class_id#31, category_id#32] (43) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] (44) HashAggregate [codegen id : 11] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#30, class_id#31, category_id#32] (45) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32), 5), ENSURE_REQUIREMENTS, [id=#34] (46) Sort [codegen id : 12] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: [coalesce(brand_id#30, 0) ASC NULLS FIRST, isnull(brand_id#30) ASC NULLS FIRST, coalesce(class_id#31, 0) ASC NULLS FIRST, isnull(class_id#31) ASC NULLS FIRST, coalesce(category_id#32, 0) ASC NULLS FIRST, isnull(category_id#32) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Output [2]: [ws_item_sk#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 15] -Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] (49) Filter [codegen id : 15] -Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_item_sk#37) +Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#35) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#39] +Output [1]: [d_date_sk#37] (51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#39] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#37] Join condition: None (52) Project [codegen id : 15] -Output [1]: [ws_item_sk#37] -Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] +Output [1]: [ws_item_sk#35] +Input [3]: [ws_item_sk#35, ws_sold_date_sk#36, d_date_sk#37] (53) ReusedExchange [Reuses operator id: 33] -Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] +Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] (54) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#40] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [i_item_sk#38] Join condition: None (55) Project [codegen id : 15] -Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] +Output [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Input [5]: [ws_item_sk#35, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] (56) Exchange -Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] +Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: hashpartitioning(coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41), 5), ENSURE_REQUIREMENTS, [id=#42] (57) Sort [codegen id : 16] -Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 +Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: [coalesce(i_brand_id#39, 0) ASC NULLS FIRST, isnull(i_brand_id#39) ASC NULLS FIRST, coalesce(i_class_id#40, 0) ASC NULLS FIRST, isnull(i_class_id#40) ASC NULLS FIRST, coalesce(i_category_id#41, 0) ASC NULLS FIRST, isnull(i_category_id#41) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] -Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41)] Join condition: None (59) HashAggregate [codegen id : 17] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#30, class_id#31, category_id#32] (60) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#43] (61) HashAggregate [codegen id : 18] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#30, class_id#31, category_id#32] (62) BroadcastExchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#44] (63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#32, class_id#33, category_id#34] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None (64) Project [codegen id : 19] -Output [1]: [i_item_sk#7 AS ss_item_sk#47] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] +Output [1]: [i_item_sk#7 AS ss_item_sk#45] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#30, class_id#31, category_id#32] (65) Exchange -Input [1]: [ss_item_sk#47] -Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [1]: [ss_item_sk#45] +Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] (66) Sort [codegen id : 20] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#47] +Right keys [1]: [ss_item_sk#45] Join condition: None (68) Scan parquet default.date_dim -Output [2]: [d_date_sk#49, d_week_seq#50] +Output [2]: [d_date_sk#12, d_week_seq#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 21] -Input [2]: [d_date_sk#49, d_week_seq#50] +Input [2]: [d_date_sk#12, d_week_seq#47] (70) Filter [codegen id : 21] -Input [2]: [d_date_sk#49, d_week_seq#50] -Condition : ((isnotnull(d_week_seq#50) AND (d_week_seq#50 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#49)) +Input [2]: [d_date_sk#12, d_week_seq#47] +Condition : ((isnotnull(d_week_seq#47) AND (d_week_seq#47 = Subquery scalar-subquery#48, [id=#49])) AND isnotnull(d_date_sk#12)) (71) Project [codegen id : 21] -Output [1]: [d_date_sk#49] -Input [2]: [d_date_sk#49, d_week_seq#50] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_week_seq#47] (72) BroadcastExchange -Input [1]: [d_date_sk#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] (73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#49] +Right keys [1]: [d_date_sk#12] Join condition: None (74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (75) Scan parquet default.item -Output [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (76) ColumnarToRow [codegen id : 22] -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (77) Filter [codegen id : 22] -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -Condition : (((isnotnull(i_item_sk#54) AND isnotnull(i_brand_id#55)) AND isnotnull(i_class_id#56)) AND isnotnull(i_category_id#57)) +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (78) Exchange -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -Arguments: hashpartitioning(i_item_sk#54, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#51] (79) Sort [codegen id : 23] -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -Arguments: [i_item_sk#54 ASC NULLS FIRST], false, 0 +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#45] (81) Sort [codegen id : 41] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#54] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [i_item_sk#7] +Right keys [1]: [ss_item_sk#45] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#59] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] (84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#54] +Right keys [1]: [i_item_sk#7] Join condition: None (85) Project [codegen id : 42] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (86) HashAggregate [codegen id : 42] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] -Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#60, isEmpty#61, count#62] -Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] +Aggregate Attributes [3]: [sum#53, isEmpty#54, count#55] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] (87) Exchange -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] -Arguments: hashpartitioning(i_brand_id#55, i_class_id#56, i_category_id#57, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#59] (88) HashAggregate [codegen id : 86] -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] -Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67, count(1)#68] -Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#69, count(1)#68 AS number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60, count(1)#61] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sales#62, count(1)#61 AS number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] (89) Filter [codegen id : 86] -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(32,6)) > cast(Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 as decimal(32,6)) > cast(Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) (90) Project [codegen id : 86] -Output [6]: [store AS channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70] -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] +Output [6]: [store AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] (91) Scan parquet default.store_sales -Output [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Output [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] +PartitionFilters: [isnotnull(ss_sold_date_sk#71), dynamicpruningexpression(ss_sold_date_sk#71 IN dynamicpruning#72)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 43] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] (93) Filter [codegen id : 43] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Condition : isnotnull(ss_item_sk#75) +Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Condition : isnotnull(ss_item_sk#68) (94) Exchange -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Arguments: hashpartitioning(ss_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Arguments: hashpartitioning(ss_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#73] (95) Sort [codegen id : 44] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Arguments: [ss_item_sk#75 ASC NULLS FIRST], false, 0 +Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Arguments: [ss_item_sk#68 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#45] (97) Sort [codegen id : 62] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ss_item_sk#75] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [ss_item_sk#68] +Right keys [1]: [ss_item_sk#45] Join condition: None (99) Scan parquet default.date_dim -Output [2]: [d_date_sk#81, d_week_seq#82] +Output [2]: [d_date_sk#74, d_week_seq#75] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (100) ColumnarToRow [codegen id : 63] -Input [2]: [d_date_sk#81, d_week_seq#82] +Input [2]: [d_date_sk#74, d_week_seq#75] (101) Filter [codegen id : 63] -Input [2]: [d_date_sk#81, d_week_seq#82] -Condition : ((isnotnull(d_week_seq#82) AND (d_week_seq#82 = Subquery scalar-subquery#83, [id=#84])) AND isnotnull(d_date_sk#81)) +Input [2]: [d_date_sk#74, d_week_seq#75] +Condition : ((isnotnull(d_week_seq#75) AND (d_week_seq#75 = Subquery scalar-subquery#76, [id=#77])) AND isnotnull(d_date_sk#74)) (102) Project [codegen id : 63] -Output [1]: [d_date_sk#81] -Input [2]: [d_date_sk#81, d_week_seq#82] +Output [1]: [d_date_sk#74] +Input [2]: [d_date_sk#74, d_week_seq#75] (103) BroadcastExchange -Input [1]: [d_date_sk#81] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#85] +Input [1]: [d_date_sk#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] (104) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_sold_date_sk#78] -Right keys [1]: [d_date_sk#81] +Left keys [1]: [ss_sold_date_sk#71] +Right keys [1]: [d_date_sk#74] Join condition: None (105) Project [codegen id : 84] -Output [3]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77] -Input [5]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#81] +Output [3]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70] +Input [5]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71, d_date_sk#74] (106) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] +Output [4]: [i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] (107) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#75] -Right keys [1]: [i_item_sk#86] +Left keys [1]: [ss_item_sk#68] +Right keys [1]: [i_item_sk#79] Join condition: None (108) Project [codegen id : 84] -Output [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] -Input [7]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] +Output [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#80, i_class_id#81, i_category_id#82] +Input [7]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] (109) HashAggregate [codegen id : 84] -Input [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] -Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] -Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] +Input [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#80, i_class_id#81, i_category_id#82] +Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] +Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] (110) Exchange -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] -Arguments: hashpartitioning(i_brand_id#87, i_class_id#88, i_category_id#89, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] +Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [id=#89] (111) HashAggregate [codegen id : 85] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] -Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] -Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#99, count(1)#98 AS number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] +Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90, count(1)#91] +Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sales#92, count(1)#91 AS number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] (112) Filter [codegen id : 85] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) (113) Project [codegen id : 85] -Output [6]: [store AS channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] +Output [6]: [store AS channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] (114) BroadcastExchange -Input [6]: [channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#103] +Input [6]: [channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#96] (115) BroadcastHashJoin [codegen id : 86] -Left keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] -Right keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Right keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] Join condition: None (116) TakeOrderedAndProject -Input [12]: [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Arguments: 100, [i_brand_id#55 ASC NULLS FIRST, i_class_id#56 ASC NULLS FIRST, i_category_id#57 ASC NULLS FIRST], [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Input [12]: [channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#72, [id=#73] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#65, [id=#66] * HashAggregate (139) +- Exchange (138) +- * HashAggregate (137) @@ -674,140 +674,140 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (117) Scan parquet default.store_sales -Output [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] +Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#106), dynamicpruningexpression(ss_sold_date_sk#106 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#97)] ReadSchema: struct (118) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] +Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (119) Scan parquet default.date_dim -Output [2]: [d_date_sk#108, d_year#109] +Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (120) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#108, d_year#109] +Input [2]: [d_date_sk#12, d_year#13] (121) Filter [codegen id : 1] -Input [2]: [d_date_sk#108, d_year#109] -Condition : (((isnotnull(d_year#109) AND (d_year#109 >= 1999)) AND (d_year#109 <= 2001)) AND isnotnull(d_date_sk#108)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (122) Project [codegen id : 1] -Output [1]: [d_date_sk#108] -Input [2]: [d_date_sk#108, d_year#109] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (123) BroadcastExchange -Input [1]: [d_date_sk#108] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#110] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] (124) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#106] -Right keys [1]: [d_date_sk#108] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (125) Project [codegen id : 2] -Output [2]: [ss_quantity#104 AS quantity#111, ss_list_price#105 AS list_price#112] -Input [4]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106, d_date_sk#108] +Output [2]: [ss_quantity#2 AS quantity#99, ss_list_price#3 AS list_price#100] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (126) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] +Output [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#115), dynamicpruningexpression(cs_sold_date_sk#115 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#97)] ReadSchema: struct (127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] +Input [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21] (128) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#116] +Output [1]: [d_date_sk#103] (129) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#115] -Right keys [1]: [d_date_sk#116] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#103] Join condition: None (130) Project [codegen id : 4] -Output [2]: [cs_quantity#113 AS quantity#117, cs_list_price#114 AS list_price#118] -Input [4]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115, d_date_sk#116] +Output [2]: [cs_quantity#101 AS quantity#104, cs_list_price#102 AS list_price#105] +Input [4]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21, d_date_sk#103] (131) Scan parquet default.web_sales -Output [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] +Output [3]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#97)] ReadSchema: struct (132) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] +Input [3]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36] (133) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#122] +Output [1]: [d_date_sk#108] (134) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#121] -Right keys [1]: [d_date_sk#122] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#108] Join condition: None (135) Project [codegen id : 6] -Output [2]: [ws_quantity#119 AS quantity#123, ws_list_price#120 AS list_price#124] -Input [4]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121, d_date_sk#122] +Output [2]: [ws_quantity#106 AS quantity#109, ws_list_price#107 AS list_price#110] +Input [4]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36, d_date_sk#108] (136) Union (137) HashAggregate [codegen id : 7] -Input [2]: [quantity#111, list_price#112] +Input [2]: [quantity#99, list_price#100] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#125, count#126] -Results [2]: [sum#127, count#128] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#111, count#112] +Results [2]: [sum#113, count#114] (138) Exchange -Input [2]: [sum#127, count#128] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] +Input [2]: [sum#113, count#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] (139) HashAggregate [codegen id : 8] -Input [2]: [sum#127, count#128] +Input [2]: [sum#113, count#114] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130 AS average_sales#131] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#116] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#116 AS average_sales#117] -Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#106 IN dynamicpruning#107 +Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#97 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#108] +Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#115 IN dynamicpruning#107 +Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#97 -Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#107 +Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#97 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (141) (141) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#49] +Output [1]: [d_date_sk#12] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 ReusedExchange (142) (142) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#11 -Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#48, [id=#49] * Project (146) +- * Filter (145) +- * ColumnarToRow (144) @@ -815,33 +815,33 @@ Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquer (143) Scan parquet default.date_dim -Output [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Output [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] (145) Filter [codegen id : 1] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] -Condition : (((((isnotnull(d_year#133) AND isnotnull(d_moy#134)) AND isnotnull(d_dom#135)) AND (d_year#133 = 2000)) AND (d_moy#134 = 12)) AND (d_dom#135 = 11)) +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#13 = 2000)) AND (d_moy#118 = 12)) AND (d_dom#119 = 11)) (146) Project [codegen id : 1] -Output [1]: [d_week_seq#132] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Output [1]: [d_week_seq#47] +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] -Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] +Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#65, [id=#66] -Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#79 +Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#71 IN dynamicpruning#72 ReusedExchange (147) (147) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#81] +Output [1]: [d_date_sk#74] -Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#83, [id=#84] +Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#76, [id=#77] * Project (151) +- * Filter (150) +- * ColumnarToRow (149) @@ -849,21 +849,21 @@ Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subqu (148) Scan parquet default.date_dim -Output [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Output [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (149) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] (150) Filter [codegen id : 1] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] -Condition : (((((isnotnull(d_year#137) AND isnotnull(d_moy#138)) AND isnotnull(d_dom#139)) AND (d_year#137 = 1999)) AND (d_moy#138 = 12)) AND (d_dom#139 = 11)) +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#13 = 1999)) AND (d_moy#118 = 12)) AND (d_dom#119 = 11)) (151) Project [codegen id : 1] -Output [1]: [d_week_seq#136] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Output [1]: [d_week_seq#47] +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index ae653b43971db..83604fd5cdff8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -131,443 +131,443 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (10) Scan parquet default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (11) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] (12) Filter [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14)) (13) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_item_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] (15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) +Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Condition : isnotnull(cs_item_sk#15) (16) Scan parquet default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (18) Filter [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : isnotnull(i_item_sk#17) (19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [i_item_sk#17] Join condition: None (21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (22) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_year#25] +Output [2]: [d_date_sk#22, d_year#23] 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 (23) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#24, d_year#25] +Input [2]: [d_date_sk#22, d_year#23] (24) Filter [codegen id : 2] -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) +Input [2]: [d_date_sk#22, d_year#23] +Condition : (((isnotnull(d_year#23) AND (d_year#23 >= 1999)) AND (d_year#23 <= 2001)) AND isnotnull(d_date_sk#22)) (25) Project [codegen id : 2] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_year#25] +Output [1]: [d_date_sk#22] +Input [2]: [d_date_sk#22, d_year#23] (26) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (27) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#22] Join condition: None (28) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] +Output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] +Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#22] (29) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] +Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] (30) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] +Left keys [6]: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)] +Right keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] Join condition: None (31) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] (32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#11] Join condition: None (33) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Output [4]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14] +Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] (34) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#27] (35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#27] Join condition: None (36) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] +Output [3]: [i_brand_id#12 AS brand_id#28, i_class_id#13 AS class_id#29, i_category_id#14 AS category_id#30] +Input [5]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#27] (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#28, class_id#29, category_id#30] (38) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [id=#31] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#28, class_id#29, category_id#30] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Output [2]: [ws_item_sk#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_item_sk#34) +Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Condition : isnotnull(ws_item_sk#32) (43) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] +Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#36] +Left keys [1]: [ws_item_sk#32] +Right keys [1]: [i_item_sk#34] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] +Output [4]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37] +Input [6]: [ws_item_sk#32, ws_sold_date_sk#33, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] (46) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#40] +Output [1]: [d_date_sk#38] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#40] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#38] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] +Output [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +Input [5]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] (49) BroadcastExchange -Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] +Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#39] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] -Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] +Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] +Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#28, class_id#29, category_id#30] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#28, class_id#29, category_id#30] (53) BroadcastExchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#40] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#30, class_id#31, category_id#32] +Right keys [3]: [brand_id#28, class_id#29, category_id#30] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#43] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] +Output [1]: [i_item_sk#6 AS ss_item_sk#41] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] (56) BroadcastExchange -Input [1]: [ss_item_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] +Input [1]: [ss_item_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#43] +Right keys [1]: [ss_item_sk#41] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] -Condition : (((isnotnull(i_item_sk#45) AND isnotnull(i_brand_id#46)) AND isnotnull(i_class_id#47)) AND isnotnull(i_category_id#48)) +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#41] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#45] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [i_item_sk#6] +Right keys [1]: [ss_item_sk#41] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#45] +Right keys [1]: [i_item_sk#6] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (66) Scan parquet default.date_dim -Output [2]: [d_date_sk#50, d_week_seq#51] +Output [2]: [d_date_sk#27, d_week_seq#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [2]: [d_date_sk#50, d_week_seq#51] +Input [2]: [d_date_sk#27, d_week_seq#44] (68) Filter [codegen id : 24] -Input [2]: [d_date_sk#50, d_week_seq#51] -Condition : ((isnotnull(d_week_seq#51) AND (d_week_seq#51 = Subquery scalar-subquery#52, [id=#53])) AND isnotnull(d_date_sk#50)) +Input [2]: [d_date_sk#27, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#27)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#50] -Input [2]: [d_date_sk#50, d_week_seq#51] +Output [1]: [d_date_sk#27] +Input [2]: [d_date_sk#27, d_week_seq#44] (70) BroadcastExchange -Input [1]: [d_date_sk#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#50] +Right keys [1]: [d_date_sk#27] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#27] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] -Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] +Aggregate Attributes [3]: [sum#48, isEmpty#49, count#50] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] (74) Exchange -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] -Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#54] (75) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] -Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55, count(1)#56] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sales#57, count(1)#56 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] (76) Filter [codegen id : 52] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59 as decimal(32,6)) > cast(Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) (77) Project [codegen id : 52] -Output [6]: [store AS channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] +Output [6]: [store AS channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] (78) Scan parquet default.store_sales -Output [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +Output [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] +PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] (80) Filter [codegen id : 50] -Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] -Condition : isnotnull(ss_item_sk#70) +Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] +Condition : isnotnull(ss_item_sk#63) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#41] (82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#70] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [ss_item_sk#63] +Right keys [1]: [ss_item_sk#41] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] +Output [4]: [i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] (84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#70] -Right keys [1]: [i_item_sk#75] +Left keys [1]: [ss_item_sk#63] +Right keys [1]: [i_item_sk#68] Join condition: None (85) Project [codegen id : 50] -Output [6]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] -Input [8]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] +Output [6]: [ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_brand_id#69, i_class_id#70, i_category_id#71] +Input [8]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#79, d_week_seq#80] +Output [2]: [d_date_sk#72, d_week_seq#73] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#79, d_week_seq#80] +Input [2]: [d_date_sk#72, d_week_seq#73] (88) Filter [codegen id : 49] -Input [2]: [d_date_sk#79, d_week_seq#80] -Condition : ((isnotnull(d_week_seq#80) AND (d_week_seq#80 = Subquery scalar-subquery#81, [id=#82])) AND isnotnull(d_date_sk#79)) +Input [2]: [d_date_sk#72, d_week_seq#73] +Condition : ((isnotnull(d_week_seq#73) AND (d_week_seq#73 = Subquery scalar-subquery#74, [id=#75])) AND isnotnull(d_date_sk#72)) (89) Project [codegen id : 49] -Output [1]: [d_date_sk#79] -Input [2]: [d_date_sk#79, d_week_seq#80] +Output [1]: [d_date_sk#72] +Input [2]: [d_date_sk#72, d_week_seq#73] (90) BroadcastExchange -Input [1]: [d_date_sk#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#83] +Input [1]: [d_date_sk#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] (91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#73] -Right keys [1]: [d_date_sk#79] +Left keys [1]: [ss_sold_date_sk#66] +Right keys [1]: [d_date_sk#72] Join condition: None (92) Project [codegen id : 50] -Output [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] -Input [7]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78, d_date_sk#79] +Output [5]: [ss_quantity#64, ss_list_price#65, i_brand_id#69, i_class_id#70, i_category_id#71] +Input [7]: [ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_brand_id#69, i_class_id#70, i_category_id#71, d_date_sk#72] (93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] -Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] +Input [5]: [ss_quantity#64, ss_list_price#65, i_brand_id#69, i_class_id#70, i_category_id#71] +Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#77, isEmpty#78, count#79] +Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] (94) Exchange -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] -Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] +Arguments: hashpartitioning(i_brand_id#69, i_class_id#70, i_category_id#71, 5), ENSURE_REQUIREMENTS, [id=#83] (95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] -Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] +Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84, count(1)#85] +Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sales#86, count(1)#85 AS number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] (96) Filter [codegen id : 51] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) (97) Project [codegen id : 51] -Output [6]: [store AS channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] +Output [6]: [store AS channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] (98) BroadcastExchange -Input [6]: [channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#97] +Input [6]: [channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#90] (99) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] -Right keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Right keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] Join condition: None (100) TakeOrderedAndProject -Input [12]: [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Arguments: 100, [i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Input [12]: [channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] +Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#67, [id=#68] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#60, [id=#61] * HashAggregate (123) +- Exchange (122) +- * HashAggregate (121) @@ -594,140 +594,140 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] +Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#100), dynamicpruningexpression(ss_sold_date_sk#100 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#91)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] +Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (103) Scan parquet default.date_dim -Output [2]: [d_date_sk#102, d_year#103] +Output [2]: [d_date_sk#27, d_year#92] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#102, d_year#103] +Input [2]: [d_date_sk#27, d_year#92] (105) Filter [codegen id : 1] -Input [2]: [d_date_sk#102, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 <= 2001)) AND isnotnull(d_date_sk#102)) +Input [2]: [d_date_sk#27, d_year#92] +Condition : (((isnotnull(d_year#92) AND (d_year#92 >= 1999)) AND (d_year#92 <= 2001)) AND isnotnull(d_date_sk#27)) (106) Project [codegen id : 1] -Output [1]: [d_date_sk#102] -Input [2]: [d_date_sk#102, d_year#103] +Output [1]: [d_date_sk#27] +Input [2]: [d_date_sk#27, d_year#92] (107) BroadcastExchange -Input [1]: [d_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#104] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] (108) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#100] -Right keys [1]: [d_date_sk#102] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#27] Join condition: None (109) Project [codegen id : 2] -Output [2]: [ss_quantity#98 AS quantity#105, ss_list_price#99 AS list_price#106] -Input [4]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100, d_date_sk#102] +Output [2]: [ss_quantity#2 AS quantity#94, ss_list_price#3 AS list_price#95] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#27] (110) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] +Output [3]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#109), dynamicpruningexpression(cs_sold_date_sk#109 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#91)] ReadSchema: struct (111) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] +Input [3]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16] (112) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#110] +Output [1]: [d_date_sk#98] (113) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#109] -Right keys [1]: [d_date_sk#110] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#98] Join condition: None (114) Project [codegen id : 4] -Output [2]: [cs_quantity#107 AS quantity#111, cs_list_price#108 AS list_price#112] -Input [4]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109, d_date_sk#110] +Output [2]: [cs_quantity#96 AS quantity#99, cs_list_price#97 AS list_price#100] +Input [4]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16, d_date_sk#98] (115) Scan parquet default.web_sales -Output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] +Output [3]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#115), dynamicpruningexpression(ws_sold_date_sk#115 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#91)] ReadSchema: struct (116) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] +Input [3]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33] (117) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#116] +Output [1]: [d_date_sk#103] (118) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#115] -Right keys [1]: [d_date_sk#116] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#103] Join condition: None (119) Project [codegen id : 6] -Output [2]: [ws_quantity#113 AS quantity#117, ws_list_price#114 AS list_price#118] -Input [4]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115, d_date_sk#116] +Output [2]: [ws_quantity#101 AS quantity#104, ws_list_price#102 AS list_price#105] +Input [4]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33, d_date_sk#103] (120) Union (121) HashAggregate [codegen id : 7] -Input [2]: [quantity#105, list_price#106] +Input [2]: [quantity#94, list_price#95] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#119, count#120] -Results [2]: [sum#121, count#122] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#106, count#107] +Results [2]: [sum#108, count#109] (122) Exchange -Input [2]: [sum#121, count#122] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#123] +Input [2]: [sum#108, count#109] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] (123) HashAggregate [codegen id : 8] -Input [2]: [sum#121, count#122] +Input [2]: [sum#108, count#109] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124 AS average_sales#125] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))#111] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))#111 AS average_sales#112] -Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#100 IN dynamicpruning#101 +Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#91 ReusedExchange (124) (124) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#102] +Output [1]: [d_date_sk#27] -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#109 IN dynamicpruning#101 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#91 -Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#115 IN dynamicpruning#101 +Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#91 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#50] +Output [1]: [d_date_sk#27] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 ReusedExchange (126) (126) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#27] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#10 -Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#45, [id=#46] * Project (130) +- * Filter (129) +- * ColumnarToRow (128) @@ -735,33 +735,33 @@ Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer (127) Scan parquet default.date_dim -Output [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Output [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (128) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] (129) Filter [codegen id : 1] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] -Condition : (((((isnotnull(d_year#127) AND isnotnull(d_moy#128)) AND isnotnull(d_dom#129)) AND (d_year#127 = 2000)) AND (d_moy#128 = 12)) AND (d_dom#129 = 11)) +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#92 = 2000)) AND (d_moy#113 = 12)) AND (d_dom#114 = 11)) (130) Project [codegen id : 1] -Output [1]: [d_week_seq#126] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Output [1]: [d_week_seq#44] +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] -Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] +Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#60, [id=#61] -Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 +Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67 ReusedExchange (131) (131) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#79] +Output [1]: [d_date_sk#72] -Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#81, [id=#82] +Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#74, [id=#75] * Project (135) +- * Filter (134) +- * ColumnarToRow (133) @@ -769,21 +769,21 @@ Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subque (132) Scan parquet default.date_dim -Output [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Output [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (133) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] (134) Filter [codegen id : 1] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] -Condition : (((((isnotnull(d_year#131) AND isnotnull(d_moy#132)) AND isnotnull(d_dom#133)) AND (d_year#131 = 1999)) AND (d_moy#132 = 12)) AND (d_dom#133 = 11)) +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#92 = 1999)) AND (d_moy#113 = 12)) AND (d_dom#114 = 11)) (135) Project [codegen id : 1] -Output [1]: [d_week_seq#130] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Output [1]: [d_week_seq#44] +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index 7604ea1e786d8..adfa3e44f996d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -74,187 +74,187 @@ Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet default.catalog_sales -Output [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] +Output [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] +Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] (9) Project [codegen id : 3] -Output [2]: [cs_warehouse_sk#10, cs_order_number#11] -Input [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] +Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#10, cs_order_number#5 AS cs_order_number#5#11] +Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] (10) Exchange -Input [2]: [cs_warehouse_sk#10, cs_order_number#11] -Arguments: hashpartitioning(cs_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13] +Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] +Arguments: hashpartitioning(cs_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] (11) Sort [codegen id : 4] -Input [2]: [cs_warehouse_sk#10, cs_order_number#11] -Arguments: [cs_order_number#11 ASC NULLS FIRST], false, 0 +Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] +Arguments: [cs_order_number#5#11 ASC NULLS FIRST], false, 0 (12) SortMergeJoin Left keys [1]: [cs_order_number#5] -Right keys [1]: [cs_order_number#11] -Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#10) +Right keys [1]: [cs_order_number#5#11] +Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#10) (13) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] (14) Scan parquet default.catalog_returns -Output [2]: [cr_order_number#14, cr_returned_date_sk#15] +Output [2]: [cr_order_number#13, cr_returned_date_sk#14] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] ReadSchema: struct (15) ColumnarToRow [codegen id : 6] -Input [2]: [cr_order_number#14, cr_returned_date_sk#15] +Input [2]: [cr_order_number#13, cr_returned_date_sk#14] (16) Project [codegen id : 6] -Output [1]: [cr_order_number#14] -Input [2]: [cr_order_number#14, cr_returned_date_sk#15] +Output [1]: [cr_order_number#13] +Input [2]: [cr_order_number#13, cr_returned_date_sk#14] (17) Exchange -Input [1]: [cr_order_number#14] -Arguments: hashpartitioning(cr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [1]: [cr_order_number#13] +Arguments: hashpartitioning(cr_order_number#13, 5), ENSURE_REQUIREMENTS, [id=#15] (18) Sort [codegen id : 7] -Input [1]: [cr_order_number#14] -Arguments: [cr_order_number#14 ASC NULLS FIRST], false, 0 +Input [1]: [cr_order_number#13] +Arguments: [cr_order_number#13 ASC NULLS FIRST], false, 0 (19) SortMergeJoin Left keys [1]: [cs_order_number#5] -Right keys [1]: [cr_order_number#14] +Right keys [1]: [cr_order_number#13] Join condition: None (20) Scan parquet default.customer_address -Output [2]: [ca_address_sk#17, ca_state#18] +Output [2]: [ca_address_sk#16, ca_state#17] 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 (21) ColumnarToRow [codegen id : 8] -Input [2]: [ca_address_sk#17, ca_state#18] +Input [2]: [ca_address_sk#16, ca_state#17] (22) Filter [codegen id : 8] -Input [2]: [ca_address_sk#17, ca_state#18] -Condition : ((isnotnull(ca_state#18) AND (ca_state#18 = GA)) AND isnotnull(ca_address_sk#17)) +Input [2]: [ca_address_sk#16, ca_state#17] +Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_address_sk#16)) (23) Project [codegen id : 8] -Output [1]: [ca_address_sk#17] -Input [2]: [ca_address_sk#17, ca_state#18] +Output [1]: [ca_address_sk#16] +Input [2]: [ca_address_sk#16, ca_state#17] (24) BroadcastExchange -Input [1]: [ca_address_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [ca_address_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#17] +Right keys [1]: [ca_address_sk#16] Join condition: None (26) Project [codegen id : 11] Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#17] +Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] (27) Scan parquet default.call_center -Output [2]: [cc_call_center_sk#20, cc_county#21] +Output [2]: [cc_call_center_sk#19, cc_county#20] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 9] -Input [2]: [cc_call_center_sk#20, cc_county#21] +Input [2]: [cc_call_center_sk#19, cc_county#20] (29) Filter [codegen id : 9] -Input [2]: [cc_call_center_sk#20, cc_county#21] -Condition : ((isnotnull(cc_county#21) AND (cc_county#21 = Williamson County)) AND isnotnull(cc_call_center_sk#20)) +Input [2]: [cc_call_center_sk#19, cc_county#20] +Condition : ((isnotnull(cc_county#20) AND (cc_county#20 = Williamson County)) AND isnotnull(cc_call_center_sk#19)) (30) Project [codegen id : 9] -Output [1]: [cc_call_center_sk#20] -Input [2]: [cc_call_center_sk#20, cc_county#21] +Output [1]: [cc_call_center_sk#19] +Input [2]: [cc_call_center_sk#19, cc_county#20] (31) BroadcastExchange -Input [1]: [cc_call_center_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [cc_call_center_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] -Right keys [1]: [cc_call_center_sk#20] +Right keys [1]: [cc_call_center_sk#19] Join condition: None (33) Project [codegen id : 11] Output [4]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [6]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#20] +Input [6]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#19] (34) Scan parquet default.date_dim -Output [2]: [d_date_sk#23, d_date#24] +Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 10] -Input [2]: [d_date_sk#23, d_date#24] +Input [2]: [d_date_sk#22, d_date#23] (36) Filter [codegen id : 10] -Input [2]: [d_date_sk#23, d_date#24] -Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 2002-02-01)) AND (d_date#24 <= 2002-04-02)) AND isnotnull(d_date_sk#23)) +Input [2]: [d_date_sk#22, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11719)) AND (d_date#23 <= 11779)) AND isnotnull(d_date_sk#22)) (37) Project [codegen id : 10] -Output [1]: [d_date_sk#23] -Input [2]: [d_date_sk#23, d_date#24] +Output [1]: [d_date_sk#22] +Input [2]: [d_date_sk#22, d_date#23] (38) BroadcastExchange -Input [1]: [d_date_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#23] +Right keys [1]: [d_date_sk#22] Join condition: None (40) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [5]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#23] +Input [5]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#22] (41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] -Results [3]: [cs_order_number#5, sum#28, sum#29] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] +Results [3]: [cs_order_number#5, sum#27, sum#28] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_order_number#5, sum#28, sum#29] +Input [3]: [cs_order_number#5, sum#27, sum#28] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] -Results [3]: [cs_order_number#5, sum#28, sum#29] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] +Results [3]: [cs_order_number#5, sum#27, sum#28] (43) HashAggregate [codegen id : 11] -Input [3]: [cs_order_number#5, sum#28, sum#29] +Input [3]: [cs_order_number#5, sum#27, sum#28] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] -Results [3]: [sum#28, sum#29, count#31] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] +Results [3]: [sum#27, sum#28, count#30] (44) Exchange -Input [3]: [sum#28, sum#29, count#31] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] +Input [3]: [sum#27, sum#28, count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] (45) HashAggregate [codegen id : 12] -Input [3]: [sum#28, sum#29, count#31] +Input [3]: [sum#27, sum#28, count#30] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] -Results [3]: [count(cs_order_number#5)#30 AS order count #33, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#26,17,2) AS total shipping cost #34, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#27,17,2) AS total net profit #35] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] +Results [3]: [count(cs_order_number#5)#29 AS order count #32, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#25,17,2) AS total shipping cost #33, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#26,17,2) AS total net profit #34] (46) Sort [codegen id : 12] -Input [3]: [order count #33, total shipping cost #34, total net profit #35] -Arguments: [order count #33 ASC NULLS FIRST], true, 0 +Input [3]: [order count #32, total shipping cost #33, total net profit #34] +Arguments: [order count #32 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index 683d83235cce5..5640564564396 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -74,187 +74,187 @@ Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet default.catalog_sales -Output [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] +Output [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] +Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] (9) Project [codegen id : 3] -Output [2]: [cs_warehouse_sk#10, cs_order_number#11] -Input [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] +Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#10, cs_order_number#5 AS cs_order_number#5#11] +Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] (10) Exchange -Input [2]: [cs_warehouse_sk#10, cs_order_number#11] -Arguments: hashpartitioning(cs_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13] +Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] +Arguments: hashpartitioning(cs_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] (11) Sort [codegen id : 4] -Input [2]: [cs_warehouse_sk#10, cs_order_number#11] -Arguments: [cs_order_number#11 ASC NULLS FIRST], false, 0 +Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] +Arguments: [cs_order_number#5#11 ASC NULLS FIRST], false, 0 (12) SortMergeJoin Left keys [1]: [cs_order_number#5] -Right keys [1]: [cs_order_number#11] -Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#10) +Right keys [1]: [cs_order_number#5#11] +Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#10) (13) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] (14) Scan parquet default.catalog_returns -Output [2]: [cr_order_number#14, cr_returned_date_sk#15] +Output [2]: [cr_order_number#13, cr_returned_date_sk#14] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] ReadSchema: struct (15) ColumnarToRow [codegen id : 6] -Input [2]: [cr_order_number#14, cr_returned_date_sk#15] +Input [2]: [cr_order_number#13, cr_returned_date_sk#14] (16) Project [codegen id : 6] -Output [1]: [cr_order_number#14] -Input [2]: [cr_order_number#14, cr_returned_date_sk#15] +Output [1]: [cr_order_number#13] +Input [2]: [cr_order_number#13, cr_returned_date_sk#14] (17) Exchange -Input [1]: [cr_order_number#14] -Arguments: hashpartitioning(cr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [1]: [cr_order_number#13] +Arguments: hashpartitioning(cr_order_number#13, 5), ENSURE_REQUIREMENTS, [id=#15] (18) Sort [codegen id : 7] -Input [1]: [cr_order_number#14] -Arguments: [cr_order_number#14 ASC NULLS FIRST], false, 0 +Input [1]: [cr_order_number#13] +Arguments: [cr_order_number#13 ASC NULLS FIRST], false, 0 (19) SortMergeJoin Left keys [1]: [cs_order_number#5] -Right keys [1]: [cr_order_number#14] +Right keys [1]: [cr_order_number#13] Join condition: None (20) Scan parquet default.date_dim -Output [2]: [d_date_sk#17, d_date#18] +Output [2]: [d_date_sk#16, d_date#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#17, d_date#18] +Input [2]: [d_date_sk#16, d_date#17] (22) Filter [codegen id : 8] -Input [2]: [d_date_sk#17, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 2002-02-01)) AND (d_date#18 <= 2002-04-02)) AND isnotnull(d_date_sk#17)) +Input [2]: [d_date_sk#16, d_date#17] +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 11719)) AND (d_date#17 <= 11779)) AND isnotnull(d_date_sk#16)) (23) Project [codegen id : 8] -Output [1]: [d_date_sk#17] -Input [2]: [d_date_sk#17, d_date#18] +Output [1]: [d_date_sk#16] +Input [2]: [d_date_sk#16, d_date#17] (24) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [d_date_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#17] +Right keys [1]: [d_date_sk#16] Join condition: None (26) Project [codegen id : 11] Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#17] +Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#16] (27) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +Output [2]: [ca_address_sk#19, ca_state#20] 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 (28) ColumnarToRow [codegen id : 9] -Input [2]: [ca_address_sk#20, ca_state#21] +Input [2]: [ca_address_sk#19, ca_state#20] (29) Filter [codegen id : 9] -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : ((isnotnull(ca_state#21) AND (ca_state#21 = GA)) AND isnotnull(ca_address_sk#20)) +Input [2]: [ca_address_sk#19, ca_state#20] +Condition : ((isnotnull(ca_state#20) AND (ca_state#20 = GA)) AND isnotnull(ca_address_sk#19)) (30) Project [codegen id : 9] -Output [1]: [ca_address_sk#20] -Input [2]: [ca_address_sk#20, ca_state#21] +Output [1]: [ca_address_sk#19] +Input [2]: [ca_address_sk#19, ca_state#20] (31) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [ca_address_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#19] Join condition: None (33) Project [codegen id : 11] Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#20] +Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#19] (34) Scan parquet default.call_center -Output [2]: [cc_call_center_sk#23, cc_county#24] +Output [2]: [cc_call_center_sk#22, cc_county#23] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 10] -Input [2]: [cc_call_center_sk#23, cc_county#24] +Input [2]: [cc_call_center_sk#22, cc_county#23] (36) Filter [codegen id : 10] -Input [2]: [cc_call_center_sk#23, cc_county#24] -Condition : ((isnotnull(cc_county#24) AND (cc_county#24 = Williamson County)) AND isnotnull(cc_call_center_sk#23)) +Input [2]: [cc_call_center_sk#22, cc_county#23] +Condition : ((isnotnull(cc_county#23) AND (cc_county#23 = Williamson County)) AND isnotnull(cc_call_center_sk#22)) (37) Project [codegen id : 10] -Output [1]: [cc_call_center_sk#23] -Input [2]: [cc_call_center_sk#23, cc_county#24] +Output [1]: [cc_call_center_sk#22] +Input [2]: [cc_call_center_sk#22, cc_county#23] (38) BroadcastExchange -Input [1]: [cc_call_center_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] +Input [1]: [cc_call_center_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] -Right keys [1]: [cc_call_center_sk#23] +Right keys [1]: [cc_call_center_sk#22] Join condition: None (40) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#23] +Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#22] (41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] -Results [3]: [cs_order_number#5, sum#28, sum#29] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] +Results [3]: [cs_order_number#5, sum#27, sum#28] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_order_number#5, sum#28, sum#29] +Input [3]: [cs_order_number#5, sum#27, sum#28] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] -Results [3]: [cs_order_number#5, sum#28, sum#29] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] +Results [3]: [cs_order_number#5, sum#27, sum#28] (43) HashAggregate [codegen id : 11] -Input [3]: [cs_order_number#5, sum#28, sum#29] +Input [3]: [cs_order_number#5, sum#27, sum#28] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] -Results [3]: [sum#28, sum#29, count#31] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] +Results [3]: [sum#27, sum#28, count#30] (44) Exchange -Input [3]: [sum#28, sum#29, count#31] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] +Input [3]: [sum#27, sum#28, count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] (45) HashAggregate [codegen id : 12] -Input [3]: [sum#28, sum#29, count#31] +Input [3]: [sum#27, sum#28, count#30] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] -Results [3]: [count(cs_order_number#5)#30 AS order count #33, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#26,17,2) AS total shipping cost #34, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#27,17,2) AS total net profit #35] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] +Results [3]: [count(cs_order_number#5)#29 AS order count #32, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#25,17,2) AS total shipping cost #33, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#26,17,2) AS total net profit #34] (46) Sort [codegen id : 12] -Input [3]: [order count #33, total shipping cost #34, total net profit #35] -Arguments: [order count #33 ASC NULLS FIRST], true, 0 +Input [3]: [order count #32, total shipping cost #33, total net profit #34] +Arguments: [order count #32 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index 8918c2a36e2ec..97467fe1dc1db 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -77,7 +77,7 @@ Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isno Output [2]: [d_date_sk#8, d_quarter_name#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -85,7 +85,7 @@ Input [2]: [d_date_sk#8, d_quarter_name#9] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#8, d_quarter_name#9] -Condition : ((isnotnull(d_quarter_name#9) AND (rpad(d_quarter_name#9, 6, ) = 2001Q1)) AND isnotnull(d_date_sk#8)) +Condition : ((isnotnull(d_quarter_name#9) AND (d_quarter_name#9 = 2001Q1)) AND isnotnull(d_date_sk#8)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#8] @@ -197,7 +197,7 @@ Condition : ((isnotnull(sr_customer_sk#21) AND isnotnull(sr_item_sk#20)) AND isn Output [2]: [d_date_sk#26, d_quarter_name#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 9] @@ -205,7 +205,7 @@ Input [2]: [d_date_sk#26, d_quarter_name#27] (33) Filter [codegen id : 9] Input [2]: [d_date_sk#26, d_quarter_name#27] -Condition : (rpad(d_quarter_name#27, 6, ) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#26)) +Condition : (d_quarter_name#27 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#26)) (34) Project [codegen id : 9] Output [1]: [d_date_sk#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index dafa24943be82..32c8de9a1d16e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -124,7 +124,7 @@ Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_i Output [2]: [d_date_sk#21, d_quarter_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 3] @@ -132,7 +132,7 @@ Input [2]: [d_date_sk#21, d_quarter_name#22] (18) Filter [codegen id : 3] Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : ((isnotnull(d_quarter_name#22) AND (rpad(d_quarter_name#22, 6, ) = 2001Q1)) AND isnotnull(d_date_sk#21)) +Condition : ((isnotnull(d_quarter_name#22) AND (d_quarter_name#22 = 2001Q1)) AND isnotnull(d_date_sk#21)) (19) Project [codegen id : 3] Output [1]: [d_date_sk#21] @@ -155,7 +155,7 @@ Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_re Output [2]: [d_date_sk#24, d_quarter_name#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] @@ -163,7 +163,7 @@ Input [2]: [d_date_sk#24, d_quarter_name#25] (25) Filter [codegen id : 4] Input [2]: [d_date_sk#24, d_quarter_name#25] -Condition : (rpad(d_quarter_name#25, 6, ) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) +Condition : (d_quarter_name#25 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) (26) Project [codegen id : 4] Output [1]: [d_date_sk#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt index 85aed12cc9d4f..020e92edb9d38 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt @@ -73,7 +73,7 @@ Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1) Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -81,7 +81,7 @@ Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14 (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown )) AND isnotnull(cd_demo_sk#11)) +Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown)) AND isnotnull(cd_demo_sk#11)) (7) Project [codegen id : 1] Output [2]: [cd_demo_sk#11, cd_dep_count#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt index ed68f03734b13..b3b8cabfb6880 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt @@ -67,7 +67,7 @@ Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1) Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -75,7 +75,7 @@ Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14 (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] -Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown )) AND isnotnull(cd_demo_sk#11)) +Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown)) AND isnotnull(cd_demo_sk#11)) (7) Project [codegen id : 1] Output [2]: [cd_demo_sk#11, cd_dep_count#14] 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..f540e067bbc92 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))#60, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66] +Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#60,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66,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#67, d_year#68] 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#67, d_year#68] (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#67, d_year#68] +Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) (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#67] +Input [2]: [d_week_seq#67, d_year#68] (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#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] (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#67] 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#70, sun_sales#35 AS sun_sales2#71, mon_sales#36 AS mon_sales2#72, tue_sales#37 AS tue_sales2#73, wed_sales#38 AS wed_sales2#74, thu_sales#39 AS thu_sales2#75, fri_sales#40 AS fri_sales2#76, sat_sales#41 AS sat_sales2#77] +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#67] (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#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] +Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#78] (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#70 - 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#71)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#79, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#72)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#80, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#73)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#81, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#74)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#82, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#75)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#83, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#76)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#84, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#77)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#85] +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#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] (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)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] +Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#86] (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)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] 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..f540e067bbc92 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))#60, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66] +Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#60,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66,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#67, d_year#68] 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#67, d_year#68] (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#67, d_year#68] +Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) (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#67] +Input [2]: [d_week_seq#67, d_year#68] (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#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] (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#67] 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#70, sun_sales#35 AS sun_sales2#71, mon_sales#36 AS mon_sales2#72, tue_sales#37 AS tue_sales2#73, wed_sales#38 AS wed_sales2#74, thu_sales#39 AS thu_sales2#75, fri_sales#40 AS fri_sales2#76, sat_sales#41 AS sat_sales2#77] +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#67] (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#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] +Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#78] (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#70 - 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#71)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#79, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#72)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#80, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#73)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#81, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#74)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#82, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#75)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#83, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#76)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#84, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#77)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#85] +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#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] (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)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] +Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#86] (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)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] 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/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt index a40286f9f8f83..adb5346969d06 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt @@ -55,7 +55,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -86,7 +86,7 @@ Arguments: [cs_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -94,7 +94,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt index 4f5eebc0c300c..1816433bdd129 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt @@ -44,7 +44,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -52,7 +52,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -79,7 +79,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt index efb45b5ccdb7f..a04d4e6f5a5c2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt @@ -148,7 +148,7 @@ Results [4]: [w_warehouse_name#14, i_item_id#7, sum(CASE WHEN (d_date#11 < 2000- (26) Filter [codegen id : 5] Input [4]: [w_warehouse_name#14, i_item_id#7, inv_before#23, inv_after#24] -Condition : ((isnotnull(inv_before#23) AND isnotnull(inv_after#24)) AND (((inv_before#23 > 0) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) >= 0.666667)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) <= 1.5))) +Condition : (((((isnotnull(inv_before#23) AND isnotnull(inv_after#24)) AND (inv_before#23 > 0)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) >= 0.666667)) AND (inv_before#23 > 0)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) <= 1.5)) (27) TakeOrderedAndProject Input [4]: [w_warehouse_name#14, i_item_id#7, inv_before#23, inv_after#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt index 7fcbe1befa6b4..38d49852134cd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt @@ -148,7 +148,7 @@ Results [4]: [w_warehouse_name#7, i_item_id#10, sum(CASE WHEN (d_date#14 < 2000- (26) Filter [codegen id : 5] Input [4]: [w_warehouse_name#7, i_item_id#10, inv_before#23, inv_after#24] -Condition : ((isnotnull(inv_before#23) AND isnotnull(inv_after#24)) AND (((inv_before#23 > 0) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) >= 0.666667)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) <= 1.5))) +Condition : (((((isnotnull(inv_before#23) AND isnotnull(inv_after#24)) AND (inv_before#23 > 0)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) >= 0.666667)) AND (inv_before#23 > 0)) AND ((cast(inv_after#24 as double) / cast(inv_before#23 as double)) <= 1.5)) (27) TakeOrderedAndProject Input [4]: [w_warehouse_name#7, i_item_id#10, inv_before#23, inv_after#24] 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 f42a7615324c2..aa96edd4c7fba 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 @@ -246,91 +246,91 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (33) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] (35) Filter [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Condition : isnotnull(ss_customer_sk#26) (36) Project [codegen id : 11] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] (37) Exchange Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#30] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#29] (38) Sort [codegen id : 12] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (39) Scan parquet default.customer -Output [1]: [c_customer_sk#31] +Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 13] -Input [1]: [c_customer_sk#31] +Input [1]: [c_customer_sk#30] (41) Filter [codegen id : 13] -Input [1]: [c_customer_sk#31] -Condition : isnotnull(c_customer_sk#31) +Input [1]: [c_customer_sk#30] +Condition : isnotnull(c_customer_sk#30) (42) Exchange -Input [1]: [c_customer_sk#31] -Arguments: hashpartitioning(c_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [1]: [c_customer_sk#30] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#31] (43) Sort [codegen id : 14] -Input [1]: [c_customer_sk#31] -Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (44) SortMergeJoin [codegen id : 15] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#31] +Right keys [1]: [c_customer_sk#30] Join condition: None (45) Project [codegen id : 15] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (46) HashAggregate [codegen id : 15] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] -Keys [1]: [c_customer_sk#31] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [3]: [c_customer_sk#31, sum#35, isEmpty#36] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] (47) HashAggregate [codegen id : 15] -Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] -Keys [1]: [c_customer_sk#31] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (48) Filter [codegen id : 15] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (49) Project [codegen id : 15] -Output [1]: [c_customer_sk#31] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (50) Sort [codegen id : 15] -Input [1]: [c_customer_sk#31] -Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (51) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#31] +Right keys [1]: [c_customer_sk#30] Join condition: None (52) Project [codegen id : 17] @@ -338,209 +338,209 @@ Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (53) Scan parquet default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [3]: [d_date_sk#11, d_year#13, d_moy#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 16] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Input [3]: [d_date_sk#11, d_year#13, d_moy#40] (55) Filter [codegen id : 16] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) +Input [3]: [d_date_sk#11, d_year#13, d_moy#40] +Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#40)) AND (d_year#13 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#11)) (56) Project [codegen id : 16] -Output [1]: [d_date_sk#43] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [1]: [d_date_sk#11] +Input [3]: [d_date_sk#11, d_year#13, d_moy#40] (57) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] (58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#11] Join condition: None (59) Project [codegen id : 17] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#47] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] +Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#42] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#11] (60) Scan parquet default.web_sales -Output [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] ReadSchema: struct (61) ColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] (62) Exchange -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: hashpartitioning(ws_item_sk#48, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: hashpartitioning(ws_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#48] (63) Sort [codegen id : 19] -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: [ws_item_sk#48 ASC NULLS FIRST], false, 0 +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: [ws_item_sk#43 ASC NULLS FIRST], false, 0 (64) ReusedExchange [Reuses operator id: unknown] -Output [2]: [ss_item_sk#54, d_date#55] +Output [2]: [ss_item_sk#8, d_date#12] (65) Sort [codegen id : 22] -Input [2]: [ss_item_sk#54, d_date#55] -Arguments: [ss_item_sk#54 ASC NULLS FIRST], false, 0 +Input [2]: [ss_item_sk#8, d_date#12] +Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 (66) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#56, i_item_desc#57] +Output [2]: [i_item_sk#16, i_item_desc#17] (67) Sort [codegen id : 24] -Input [2]: [i_item_sk#56, i_item_desc#57] -Arguments: [i_item_sk#56 ASC NULLS FIRST], false, 0 +Input [2]: [i_item_sk#16, i_item_desc#17] +Arguments: [i_item_sk#16 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#56] +Left keys [1]: [ss_item_sk#8] +Right keys [1]: [i_item_sk#16] Join condition: None (69) Project [codegen id : 25] -Output [3]: [d_date#55, i_item_sk#56, i_item_desc#57] -Input [4]: [ss_item_sk#54, d_date#55, i_item_sk#56, i_item_desc#57] +Output [3]: [d_date#12, i_item_sk#16, i_item_desc#17] +Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#16, i_item_desc#17] (70) HashAggregate [codegen id : 25] -Input [3]: [d_date#55, i_item_sk#56, i_item_desc#57] -Keys [3]: [substr(i_item_desc#57, 1, 30) AS substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55] +Input [3]: [d_date#12, i_item_sk#16, i_item_desc#17] +Keys [3]: [substr(i_item_desc#17, 1, 30) AS substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#59] -Results [4]: [substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55, count#60] +Aggregate Attributes [1]: [count#50] +Results [4]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12, count#51] (71) HashAggregate [codegen id : 25] -Input [4]: [substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55, count#60] -Keys [3]: [substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55] +Input [4]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12, count#51] +Keys [3]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#61] -Results [2]: [i_item_sk#56 AS item_sk#23, count(1)#61 AS count(1)#62] +Aggregate Attributes [1]: [count(1)#52] +Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#52 AS count(1)#53] (72) Filter [codegen id : 25] -Input [2]: [item_sk#23, count(1)#62] -Condition : (count(1)#62 > 4) +Input [2]: [item_sk#23, count(1)#53] +Condition : (count(1)#53 > 4) (73) Project [codegen id : 25] Output [1]: [item_sk#23] -Input [2]: [item_sk#23, count(1)#62] +Input [2]: [item_sk#23, count(1)#53] (74) Sort [codegen id : 25] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 (75) SortMergeJoin -Left keys [1]: [ws_item_sk#48] +Left keys [1]: [ws_item_sk#43] Right keys [1]: [item_sk#23] Join condition: None (76) Project [codegen id : 26] -Output [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] (77) Exchange -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: hashpartitioning(ws_bill_customer_sk#49, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [id=#54] (78) Sort [codegen id : 27] -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: [ws_bill_customer_sk#49 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 (79) ReusedExchange [Reuses operator id: 37] -Output [3]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66] +Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] (80) Sort [codegen id : 29] -Input [3]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66] -Arguments: [ss_customer_sk#64 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (81) ReusedExchange [Reuses operator id: 42] -Output [1]: [c_customer_sk#67] +Output [1]: [c_customer_sk#30] (82) Sort [codegen id : 31] -Input [1]: [c_customer_sk#67] -Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (83) SortMergeJoin [codegen id : 32] -Left keys [1]: [ss_customer_sk#64] -Right keys [1]: [c_customer_sk#67] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#30] Join condition: None (84) Project [codegen id : 32] -Output [3]: [ss_quantity#65, ss_sales_price#66, c_customer_sk#67] -Input [4]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66, c_customer_sk#67] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (85) HashAggregate [codegen id : 32] -Input [3]: [ss_quantity#65, ss_sales_price#66, c_customer_sk#67] -Keys [1]: [c_customer_sk#67] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#68, isEmpty#69] -Results [3]: [c_customer_sk#67, sum#70, isEmpty#71] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#55, isEmpty#56] +Results [3]: [c_customer_sk#30, sum#57, isEmpty#58] (86) HashAggregate [codegen id : 32] -Input [3]: [c_customer_sk#67, sum#70, isEmpty#71] -Keys [1]: [c_customer_sk#67] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))#72] -Results [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))#72 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] +Input [3]: [c_customer_sk#30, sum#57, isEmpty#58] +Keys [1]: [c_customer_sk#30] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#59] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#59 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] (87) Filter [codegen id : 32] -Input [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (88) Project [codegen id : 32] -Output [1]: [c_customer_sk#67] -Input [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] (89) Sort [codegen id : 32] -Input [1]: [c_customer_sk#67] -Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#49] -Right keys [1]: [c_customer_sk#67] +Left keys [1]: [ws_bill_customer_sk#44] +Right keys [1]: [c_customer_sk#30] Join condition: None (91) Project [codegen id : 34] -Output [3]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [3]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] (92) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#74] +Output [1]: [d_date_sk#11] (93) BroadcastHashJoin [codegen id : 34] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#74] +Left keys [1]: [ws_sold_date_sk#47] +Right keys [1]: [d_date_sk#11] Join condition: None (94) Project [codegen id : 34] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#51 as decimal(12,2)))), DecimalType(18,2), true) AS sales#75] -Input [4]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52, d_date_sk#74] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2), true) AS sales#61] +Input [4]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#11] (95) Union (96) HashAggregate [codegen id : 35] -Input [1]: [sales#47] +Input [1]: [sales#42] Keys: [] -Functions [1]: [partial_sum(sales#47)] -Aggregate Attributes [2]: [sum#76, isEmpty#77] -Results [2]: [sum#78, isEmpty#79] +Functions [1]: [partial_sum(sales#42)] +Aggregate Attributes [2]: [sum#62, isEmpty#63] +Results [2]: [sum#64, isEmpty#65] (97) Exchange -Input [2]: [sum#78, isEmpty#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#80] +Input [2]: [sum#64, isEmpty#65] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] (98) HashAggregate [codegen id : 36] -Input [2]: [sum#78, isEmpty#79] +Input [2]: [sum#64, isEmpty#65] Keys: [] -Functions [1]: [sum(sales#47)] -Aggregate Attributes [1]: [sum(sales#47)#81] -Results [1]: [sum(sales#47)#81 AS sum(sales)#82] +Functions [1]: [sum(sales#42)] +Aggregate Attributes [1]: [sum(sales#42)#67] +Results [1]: [sum(sales#42)#67 AS sum(sales)#68] ===== Subqueries ===== @@ -549,7 +549,7 @@ ReusedExchange (99) (99) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#43] +Output [1]: [d_date_sk#11] Subquery:2 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 ReusedExchange (100) @@ -558,7 +558,7 @@ ReusedExchange (100) (100) ReusedExchange [Reuses operator id: 12] Output [2]: [d_date_sk#11, d_date#12] -Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#38, [id=#39] * HashAggregate (124) +- Exchange (123) +- * HashAggregate (122) @@ -586,131 +586,131 @@ Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#69)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] (103) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] -Condition : isnotnull(ss_customer_sk#83) +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Condition : isnotnull(ss_customer_sk#26) (104) Scan parquet default.date_dim -Output [2]: [d_date_sk#88, d_year#89] +Output [2]: [d_date_sk#11, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (105) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#88, d_year#89] +Input [2]: [d_date_sk#11, d_year#13] (106) Filter [codegen id : 1] -Input [2]: [d_date_sk#88, d_year#89] -Condition : (d_year#89 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#88)) +Input [2]: [d_date_sk#11, d_year#13] +Condition : (d_year#13 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) (107) Project [codegen id : 1] -Output [1]: [d_date_sk#88] -Input [2]: [d_date_sk#88, d_year#89] +Output [1]: [d_date_sk#11] +Input [2]: [d_date_sk#11, d_year#13] (108) BroadcastExchange -Input [1]: [d_date_sk#88] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#90] +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#70] (109) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#86] -Right keys [1]: [d_date_sk#88] +Left keys [1]: [ss_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] Join condition: None (110) Project [codegen id : 2] -Output [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] -Input [5]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86, d_date_sk#88] +Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9, d_date_sk#11] (111) Exchange -Input [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] -Arguments: hashpartitioning(ss_customer_sk#83, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#71] (112) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] -Arguments: [ss_customer_sk#83 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (113) Scan parquet default.customer -Output [1]: [c_customer_sk#92] +Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (114) ColumnarToRow [codegen id : 4] -Input [1]: [c_customer_sk#92] +Input [1]: [c_customer_sk#30] (115) Filter [codegen id : 4] -Input [1]: [c_customer_sk#92] -Condition : isnotnull(c_customer_sk#92) +Input [1]: [c_customer_sk#30] +Condition : isnotnull(c_customer_sk#30) (116) Exchange -Input [1]: [c_customer_sk#92] -Arguments: hashpartitioning(c_customer_sk#92, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [1]: [c_customer_sk#30] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#72] (117) Sort [codegen id : 5] -Input [1]: [c_customer_sk#92] -Arguments: [c_customer_sk#92 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (118) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#83] -Right keys [1]: [c_customer_sk#92] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#30] Join condition: None (119) Project [codegen id : 6] -Output [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#92] -Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, c_customer_sk#92] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (120) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#92] -Keys [1]: [c_customer_sk#92] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#94, isEmpty#95] -Results [3]: [c_customer_sk#92, sum#96, isEmpty#97] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#73, isEmpty#74] +Results [3]: [c_customer_sk#30, sum#75, isEmpty#76] (121) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#92, sum#96, isEmpty#97] -Keys [1]: [c_customer_sk#92] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98 AS csales#99] +Input [3]: [c_customer_sk#30, sum#75, isEmpty#76] +Keys [1]: [c_customer_sk#30] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#77] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#77 AS csales#78] (122) HashAggregate [codegen id : 6] -Input [1]: [csales#99] +Input [1]: [csales#78] Keys: [] -Functions [1]: [partial_max(csales#99)] -Aggregate Attributes [1]: [max#100] -Results [1]: [max#101] +Functions [1]: [partial_max(csales#78)] +Aggregate Attributes [1]: [max#79] +Results [1]: [max#80] (123) Exchange -Input [1]: [max#101] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#102] +Input [1]: [max#80] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] (124) HashAggregate [codegen id : 7] -Input [1]: [max#101] +Input [1]: [max#80] Keys: [] -Functions [1]: [max(csales#99)] -Aggregate Attributes [1]: [max(csales#99)#103] -Results [1]: [max(csales#99)#103 AS tpcds_cmax#104] +Functions [1]: [max(csales#78)] +Aggregate Attributes [1]: [max(csales#78)#82] +Results [1]: [max(csales#78)#82 AS tpcds_cmax#83] -Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#87 +Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#69 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 108] -Output [1]: [d_date_sk#88] +Output [1]: [d_date_sk#11] -Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:6 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] 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 8c2aed03ce0cc..692671c9f287d 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 @@ -206,83 +206,83 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (29) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] (31) Filter [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Condition : isnotnull(ss_customer_sk#26) (32) Project [codegen id : 8] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] (33) Scan parquet default.customer -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 7] -Input [1]: [c_customer_sk#30] +Input [1]: [c_customer_sk#29] (35) Filter [codegen id : 7] -Input [1]: [c_customer_sk#30] -Condition : isnotnull(c_customer_sk#30) +Input [1]: [c_customer_sk#29] +Condition : isnotnull(c_customer_sk#29) (36) BroadcastExchange -Input [1]: [c_customer_sk#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [1]: [c_customer_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (37) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#29] Join condition: None (38) Project [codegen id : 8] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#29] (39) HashAggregate [codegen id : 8] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Keys [1]: [c_customer_sk#29] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Aggregate Attributes [2]: [sum#31, isEmpty#32] +Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] (40) Exchange -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#35] (41) HashAggregate [codegen id : 9] -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Keys [1]: [c_customer_sk#30] +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#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] +Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (42) Filter [codegen id : 9] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (43) Project [codegen id : 9] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Output [1]: [c_customer_sk#29] +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (44) Sort [codegen id : 9] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (45) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#29] Join condition: None (46) Project [codegen id : 11] @@ -290,128 +290,128 @@ Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (47) Scan parquet default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [3]: [d_date_sk#10, d_year#12, d_moy#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 10] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Input [3]: [d_date_sk#10, d_year#12, d_moy#40] (49) Filter [codegen id : 10] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) +Input [3]: [d_date_sk#10, d_year#12, d_moy#40] +Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#40)) AND (d_year#12 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#10)) (50) Project [codegen id : 10] -Output [1]: [d_date_sk#43] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [1]: [d_date_sk#10] +Input [3]: [d_date_sk#10, d_year#12, d_moy#40] (51) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Input [1]: [d_date_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] (52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#10] Join condition: None (53) Project [codegen id : 11] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#47] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] +Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#42] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#10] (54) Scan parquet default.web_sales -Output [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] ReadSchema: struct (55) ColumnarToRow [codegen id : 16] -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] (56) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] (57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#48] +Left keys [1]: [ws_item_sk#43] Right keys [1]: [item_sk#22] Join condition: None (58) Project [codegen id : 16] -Output [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] (59) Exchange -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: hashpartitioning(ws_bill_customer_sk#49, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [id=#48] (60) Sort [codegen id : 17] -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: [ws_bill_customer_sk#49 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 (61) ReusedExchange [Reuses operator id: 40] -Output [3]: [c_customer_sk#54, sum#55, isEmpty#56] +Output [3]: [c_customer_sk#29, sum#49, isEmpty#50] (62) HashAggregate [codegen id : 20] -Input [3]: [c_customer_sk#54, sum#55, isEmpty#56] -Keys [1]: [c_customer_sk#54] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))#59] -Results [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))#59 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] +Input [3]: [c_customer_sk#29, sum#49, isEmpty#50] +Keys [1]: [c_customer_sk#29] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#51] +Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#51 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] (63) Filter [codegen id : 20] -Input [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (64) Project [codegen id : 20] -Output [1]: [c_customer_sk#54] -Input [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] +Output [1]: [c_customer_sk#29] +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] (65) Sort [codegen id : 20] -Input [1]: [c_customer_sk#54] -Arguments: [c_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (66) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#49] -Right keys [1]: [c_customer_sk#54] +Left keys [1]: [ws_bill_customer_sk#44] +Right keys [1]: [c_customer_sk#29] Join condition: None (67) Project [codegen id : 22] -Output [3]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [3]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] (68) ReusedExchange [Reuses operator id: 51] -Output [1]: [d_date_sk#61] +Output [1]: [d_date_sk#10] (69) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#61] +Left keys [1]: [ws_sold_date_sk#47] +Right keys [1]: [d_date_sk#10] Join condition: None (70) Project [codegen id : 22] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#51 as decimal(12,2)))), DecimalType(18,2), true) AS sales#62] -Input [4]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52, d_date_sk#61] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2), true) AS sales#53] +Input [4]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#10] (71) Union (72) HashAggregate [codegen id : 23] -Input [1]: [sales#47] +Input [1]: [sales#42] Keys: [] -Functions [1]: [partial_sum(sales#47)] -Aggregate Attributes [2]: [sum#63, isEmpty#64] -Results [2]: [sum#65, isEmpty#66] +Functions [1]: [partial_sum(sales#42)] +Aggregate Attributes [2]: [sum#54, isEmpty#55] +Results [2]: [sum#56, isEmpty#57] (73) Exchange -Input [2]: [sum#65, isEmpty#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] +Input [2]: [sum#56, isEmpty#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#58] (74) HashAggregate [codegen id : 24] -Input [2]: [sum#65, isEmpty#66] +Input [2]: [sum#56, isEmpty#57] Keys: [] -Functions [1]: [sum(sales#47)] -Aggregate Attributes [1]: [sum(sales#47)#68] -Results [1]: [sum(sales#47)#68 AS sum(sales)#69] +Functions [1]: [sum(sales#42)] +Aggregate Attributes [1]: [sum(sales#42)#59] +Results [1]: [sum(sales#42)#59 AS sum(sales)#60] ===== Subqueries ===== @@ -420,7 +420,7 @@ ReusedExchange (75) (75) ReusedExchange [Reuses operator id: 51] -Output [1]: [d_date_sk#43] +Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 ReusedExchange (76) @@ -429,7 +429,7 @@ ReusedExchange (76) (76) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#10, d_date#11] -Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#38, [id=#39] * HashAggregate (98) +- Exchange (97) +- * HashAggregate (96) @@ -455,123 +455,123 @@ Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquer (77) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#61)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (78) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] (79) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] -Condition : isnotnull(ss_customer_sk#70) +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Condition : isnotnull(ss_customer_sk#26) (80) Scan parquet default.customer -Output [1]: [c_customer_sk#75] +Output [1]: [c_customer_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (81) ColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#75] +Input [1]: [c_customer_sk#29] (82) Filter [codegen id : 1] -Input [1]: [c_customer_sk#75] -Condition : isnotnull(c_customer_sk#75) +Input [1]: [c_customer_sk#29] +Condition : isnotnull(c_customer_sk#29) (83) BroadcastExchange -Input [1]: [c_customer_sk#75] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] +Input [1]: [c_customer_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#62] (84) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#70] -Right keys [1]: [c_customer_sk#75] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#29] Join condition: None (85) Project [codegen id : 3] -Output [4]: [ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75] -Input [5]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75] +Output [4]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] +Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#77, d_year#78] +Output [2]: [d_date_sk#10, d_year#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#77, d_year#78] +Input [2]: [d_date_sk#10, d_year#12] (88) Filter [codegen id : 2] -Input [2]: [d_date_sk#77, d_year#78] -Condition : (d_year#78 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#77)) +Input [2]: [d_date_sk#10, d_year#12] +Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) (89) Project [codegen id : 2] -Output [1]: [d_date_sk#77] -Input [2]: [d_date_sk#77, d_year#78] +Output [1]: [d_date_sk#10] +Input [2]: [d_date_sk#10, d_year#12] (90) BroadcastExchange -Input [1]: [d_date_sk#77] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#79] +Input [1]: [d_date_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] (91) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#73] -Right keys [1]: [d_date_sk#77] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] Join condition: None (92) Project [codegen id : 3] -Output [3]: [ss_quantity#71, ss_sales_price#72, c_customer_sk#75] -Input [5]: [ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75, d_date_sk#77] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Input [5]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29, d_date_sk#10] (93) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#71, ss_sales_price#72, c_customer_sk#75] -Keys [1]: [c_customer_sk#75] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#80, isEmpty#81] -Results [3]: [c_customer_sk#75, sum#82, isEmpty#83] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Keys [1]: [c_customer_sk#29] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#64, isEmpty#65] +Results [3]: [c_customer_sk#29, sum#66, isEmpty#67] (94) Exchange -Input [3]: [c_customer_sk#75, sum#82, isEmpty#83] -Arguments: hashpartitioning(c_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#84] +Input [3]: [c_customer_sk#29, sum#66, isEmpty#67] +Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#68] (95) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#75, sum#82, isEmpty#83] -Keys [1]: [c_customer_sk#75] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))#85] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))#85 AS csales#86] +Input [3]: [c_customer_sk#29, sum#66, isEmpty#67] +Keys [1]: [c_customer_sk#29] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS csales#70] (96) HashAggregate [codegen id : 4] -Input [1]: [csales#86] +Input [1]: [csales#70] Keys: [] -Functions [1]: [partial_max(csales#86)] -Aggregate Attributes [1]: [max#87] -Results [1]: [max#88] +Functions [1]: [partial_max(csales#70)] +Aggregate Attributes [1]: [max#71] +Results [1]: [max#72] (97) Exchange -Input [1]: [max#88] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#89] +Input [1]: [max#72] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#73] (98) HashAggregate [codegen id : 5] -Input [1]: [max#88] +Input [1]: [max#72] Keys: [] -Functions [1]: [max(csales#86)] -Aggregate Attributes [1]: [max(csales#86)#90] -Results [1]: [max(csales#86)#90 AS tpcds_cmax#91] +Functions [1]: [max(csales#70)] +Aggregate Attributes [1]: [max(csales#70)#74] +Results [1]: [max(csales#70)#74 AS tpcds_cmax#75] -Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 +Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#61 ReusedExchange (99) (99) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#77] +Output [1]: [d_date_sk#10] -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:6 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] 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 e75b223ea599a..35a42fa256325 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 @@ -290,145 +290,145 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (34) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] (36) Filter [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Condition : isnotnull(ss_customer_sk#26) (37) Project [codegen id : 11] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] (38) Exchange Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#30] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#29] (39) Sort [codegen id : 12] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (40) Scan parquet default.customer -Output [1]: [c_customer_sk#31] +Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 13] -Input [1]: [c_customer_sk#31] +Input [1]: [c_customer_sk#30] (42) Filter [codegen id : 13] -Input [1]: [c_customer_sk#31] -Condition : isnotnull(c_customer_sk#31) +Input [1]: [c_customer_sk#30] +Condition : isnotnull(c_customer_sk#30) (43) Exchange -Input [1]: [c_customer_sk#31] -Arguments: hashpartitioning(c_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [1]: [c_customer_sk#30] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#31] (44) Sort [codegen id : 14] -Input [1]: [c_customer_sk#31] -Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 15] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#31] +Right keys [1]: [c_customer_sk#30] Join condition: None (46) Project [codegen id : 15] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (47) HashAggregate [codegen id : 15] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] -Keys [1]: [c_customer_sk#31] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [3]: [c_customer_sk#31, sum#35, isEmpty#36] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] (48) HashAggregate [codegen id : 15] -Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] -Keys [1]: [c_customer_sk#31] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (49) Filter [codegen id : 15] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (50) Project [codegen id : 15] -Output [1]: [c_customer_sk#31] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (51) Sort [codegen id : 15] -Input [1]: [c_customer_sk#31] -Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (52) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#31] +Right keys [1]: [c_customer_sk#30] Join condition: None (53) Scan parquet default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [3]: [d_date_sk#11, d_year#13, d_moy#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 16] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Input [3]: [d_date_sk#11, d_year#13, d_moy#40] (55) Filter [codegen id : 16] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) +Input [3]: [d_date_sk#11, d_year#13, d_moy#40] +Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#40)) AND (d_year#13 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#11)) (56) Project [codegen id : 16] -Output [1]: [d_date_sk#43] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [1]: [d_date_sk#11] +Input [3]: [d_date_sk#11, d_year#13, d_moy#40] (57) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] (58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#11] Join condition: None (59) Project [codegen id : 17] Output [3]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4] -Input [5]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] +Input [5]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#11] (60) Scan parquet default.customer -Output [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] +Output [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 18] -Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] +Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] (62) Filter [codegen id : 18] -Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] -Condition : isnotnull(c_customer_sk#47) +Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Condition : isnotnull(c_customer_sk#30) (63) Exchange -Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] -Arguments: hashpartitioning(c_customer_sk#47, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#44] (64) Sort [codegen id : 19] -Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] -Arguments: [c_customer_sk#47 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (65) ReusedExchange [Reuses operator id: 38] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] @@ -438,327 +438,327 @@ Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (67) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#31] +Output [1]: [c_customer_sk#30] (68) Sort [codegen id : 23] -Input [1]: [c_customer_sk#31] -Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (69) SortMergeJoin [codegen id : 24] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#31] +Right keys [1]: [c_customer_sk#30] Join condition: None (70) Project [codegen id : 24] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (71) HashAggregate [codegen id : 24] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] -Keys [1]: [c_customer_sk#31] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [3]: [c_customer_sk#31, sum#35, isEmpty#36] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] (72) HashAggregate [codegen id : 24] -Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] -Keys [1]: [c_customer_sk#31] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (73) Filter [codegen id : 24] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (74) Project [codegen id : 24] -Output [1]: [c_customer_sk#31] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Output [1]: [c_customer_sk#30 AS c_customer_sk#30#45] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (75) Sort [codegen id : 24] -Input [1]: [c_customer_sk#31] -Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30#45] +Arguments: [c_customer_sk#30#45 ASC NULLS FIRST], false, 0 (76) SortMergeJoin -Left keys [1]: [c_customer_sk#47] -Right keys [1]: [c_customer_sk#31] +Left keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#30#45] Join condition: None (77) SortMergeJoin [codegen id : 25] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#47] +Right keys [1]: [c_customer_sk#30] Join condition: None (78) Project [codegen id : 25] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#48, c_last_name#49] -Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#47, c_first_name#48, c_last_name#49] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#42, c_last_name#43] +Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#30, c_first_name#42, c_last_name#43] (79) HashAggregate [codegen id : 25] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#48, c_last_name#49] -Keys [2]: [c_last_name#49, c_first_name#48] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#42, c_last_name#43] +Keys [2]: [c_last_name#43, c_first_name#42] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#51, isEmpty#52] -Results [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] +Aggregate Attributes [2]: [sum#46, isEmpty#47] +Results [4]: [c_last_name#43, c_first_name#42, sum#48, isEmpty#49] (80) Exchange -Input [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] -Arguments: hashpartitioning(c_last_name#49, c_first_name#48, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [c_last_name#43, c_first_name#42, sum#48, isEmpty#49] +Arguments: hashpartitioning(c_last_name#43, c_first_name#42, 5), ENSURE_REQUIREMENTS, [id=#50] (81) HashAggregate [codegen id : 26] -Input [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] -Keys [2]: [c_last_name#49, c_first_name#48] +Input [4]: [c_last_name#43, c_first_name#42, sum#48, isEmpty#49] +Keys [2]: [c_last_name#43, c_first_name#42] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#56] -Results [3]: [c_last_name#49, c_first_name#48, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sales#57] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#51] +Results [3]: [c_last_name#43, c_first_name#42, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#51 AS sales#52] (82) Scan parquet default.web_sales -Output [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Output [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#57), dynamicpruningexpression(ws_sold_date_sk#57 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (83) ColumnarToRow [codegen id : 27] -Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] (84) Filter [codegen id : 27] -Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Condition : isnotnull(ws_bill_customer_sk#59) +Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Condition : isnotnull(ws_bill_customer_sk#54) (85) Exchange -Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Arguments: hashpartitioning(ws_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Arguments: hashpartitioning(ws_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#58] (86) Sort [codegen id : 28] -Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Arguments: [ws_item_sk#58 ASC NULLS FIRST], false, 0 +Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Arguments: [ws_item_sk#53 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: unknown] -Output [2]: [ss_item_sk#64, d_date#65] +Output [2]: [ss_item_sk#8, d_date#12] (88) Sort [codegen id : 31] -Input [2]: [ss_item_sk#64, d_date#65] -Arguments: [ss_item_sk#64 ASC NULLS FIRST], false, 0 +Input [2]: [ss_item_sk#8, d_date#12] +Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 (89) ReusedExchange [Reuses operator id: 21] -Output [2]: [i_item_sk#66, i_item_desc#67] +Output [2]: [i_item_sk#16, i_item_desc#17] (90) Sort [codegen id : 33] -Input [2]: [i_item_sk#66, i_item_desc#67] -Arguments: [i_item_sk#66 ASC NULLS FIRST], false, 0 +Input [2]: [i_item_sk#16, i_item_desc#17] +Arguments: [i_item_sk#16 ASC NULLS FIRST], false, 0 (91) SortMergeJoin [codegen id : 34] -Left keys [1]: [ss_item_sk#64] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#8] +Right keys [1]: [i_item_sk#16] Join condition: None (92) Project [codegen id : 34] -Output [3]: [d_date#65, i_item_sk#66, i_item_desc#67] -Input [4]: [ss_item_sk#64, d_date#65, i_item_sk#66, i_item_desc#67] +Output [3]: [d_date#12, i_item_sk#16, i_item_desc#17] +Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#16, i_item_desc#17] (93) HashAggregate [codegen id : 34] -Input [3]: [d_date#65, i_item_sk#66, i_item_desc#67] -Keys [3]: [substr(i_item_desc#67, 1, 30) AS substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65] +Input [3]: [d_date#12, i_item_sk#16, i_item_desc#17] +Keys [3]: [substr(i_item_desc#17, 1, 30) AS substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#69] -Results [4]: [substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65, count#70] +Aggregate Attributes [1]: [count#60] +Results [4]: [substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12, count#61] (94) HashAggregate [codegen id : 34] -Input [4]: [substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65, count#70] -Keys [3]: [substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65] +Input [4]: [substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12, count#61] +Keys [3]: [substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#71] -Results [2]: [i_item_sk#66 AS item_sk#23, count(1)#71 AS count(1)#72] +Aggregate Attributes [1]: [count(1)#62] +Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#62 AS count(1)#63] (95) Filter [codegen id : 34] -Input [2]: [item_sk#23, count(1)#72] -Condition : (count(1)#72 > 4) +Input [2]: [item_sk#23, count(1)#63] +Condition : (count(1)#63 > 4) (96) Project [codegen id : 34] Output [1]: [item_sk#23] -Input [2]: [item_sk#23, count(1)#72] +Input [2]: [item_sk#23, count(1)#63] (97) Sort [codegen id : 34] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ws_item_sk#58] +Left keys [1]: [ws_item_sk#53] Right keys [1]: [item_sk#23] Join condition: None (99) Project [codegen id : 35] -Output [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Output [4]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] (100) Exchange -Input [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Arguments: hashpartitioning(ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [4]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Arguments: hashpartitioning(ws_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [id=#64] (101) Sort [codegen id : 36] -Input [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Arguments: [ws_bill_customer_sk#59 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Arguments: [ws_bill_customer_sk#54 ASC NULLS FIRST], false, 0 (102) ReusedExchange [Reuses operator id: 38] -Output [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] +Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] (103) Sort [codegen id : 38] -Input [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] -Arguments: [ss_customer_sk#74 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (104) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#77] +Output [1]: [c_customer_sk#30] (105) Sort [codegen id : 40] -Input [1]: [c_customer_sk#77] -Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (106) SortMergeJoin [codegen id : 41] -Left keys [1]: [ss_customer_sk#74] -Right keys [1]: [c_customer_sk#77] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#30] Join condition: None (107) Project [codegen id : 41] -Output [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] -Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (108) HashAggregate [codegen id : 41] -Input [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] -Keys [1]: [c_customer_sk#77] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#78, isEmpty#79] -Results [3]: [c_customer_sk#77, sum#80, isEmpty#81] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [3]: [c_customer_sk#30, sum#67, isEmpty#68] (109) HashAggregate [codegen id : 41] -Input [3]: [c_customer_sk#77, sum#80, isEmpty#81] -Keys [1]: [c_customer_sk#77] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82] -Results [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Input [3]: [c_customer_sk#30, sum#67, isEmpty#68] +Keys [1]: [c_customer_sk#30] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] (110) Filter [codegen id : 41] -Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (111) Project [codegen id : 41] -Output [1]: [c_customer_sk#77] -Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] (112) Sort [codegen id : 41] -Input [1]: [c_customer_sk#77] -Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (113) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#59] -Right keys [1]: [c_customer_sk#77] +Left keys [1]: [ws_bill_customer_sk#54] +Right keys [1]: [c_customer_sk#30] Join condition: None (114) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#84] +Output [1]: [d_date_sk#11] (115) BroadcastHashJoin [codegen id : 43] -Left keys [1]: [ws_sold_date_sk#62] -Right keys [1]: [d_date_sk#84] +Left keys [1]: [ws_sold_date_sk#57] +Right keys [1]: [d_date_sk#11] Join condition: None (116) Project [codegen id : 43] -Output [3]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61] -Input [5]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62, d_date_sk#84] +Output [3]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56] +Input [5]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57, d_date_sk#11] (117) ReusedExchange [Reuses operator id: 63] -Output [3]: [c_customer_sk#85, c_first_name#86, c_last_name#87] +Output [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] (118) Sort [codegen id : 45] -Input [3]: [c_customer_sk#85, c_first_name#86, c_last_name#87] -Arguments: [c_customer_sk#85 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (119) ReusedExchange [Reuses operator id: 38] -Output [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] +Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] (120) Sort [codegen id : 47] -Input [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] -Arguments: [ss_customer_sk#74 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (121) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#77] +Output [1]: [c_customer_sk#30] (122) Sort [codegen id : 49] -Input [1]: [c_customer_sk#77] -Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (123) SortMergeJoin [codegen id : 50] -Left keys [1]: [ss_customer_sk#74] -Right keys [1]: [c_customer_sk#77] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#30] Join condition: None (124) Project [codegen id : 50] -Output [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] -Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (125) HashAggregate [codegen id : 50] -Input [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] -Keys [1]: [c_customer_sk#77] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#78, isEmpty#79] -Results [3]: [c_customer_sk#77, sum#80, isEmpty#81] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [3]: [c_customer_sk#30, sum#67, isEmpty#68] (126) HashAggregate [codegen id : 50] -Input [3]: [c_customer_sk#77, sum#80, isEmpty#81] -Keys [1]: [c_customer_sk#77] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82] -Results [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Input [3]: [c_customer_sk#30, sum#67, isEmpty#68] +Keys [1]: [c_customer_sk#30] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] (127) Filter [codegen id : 50] -Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (128) Project [codegen id : 50] -Output [1]: [c_customer_sk#77] -Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Output [1]: [c_customer_sk#30 AS c_customer_sk#30#71] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] (129) Sort [codegen id : 50] -Input [1]: [c_customer_sk#77] -Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30#71] +Arguments: [c_customer_sk#30#71 ASC NULLS FIRST], false, 0 (130) SortMergeJoin -Left keys [1]: [c_customer_sk#85] -Right keys [1]: [c_customer_sk#77] +Left keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#30#71] Join condition: None (131) SortMergeJoin [codegen id : 51] -Left keys [1]: [ws_bill_customer_sk#59] -Right keys [1]: [c_customer_sk#85] +Left keys [1]: [ws_bill_customer_sk#54] +Right keys [1]: [c_customer_sk#30] Join condition: None (132) Project [codegen id : 51] -Output [4]: [ws_quantity#60, ws_list_price#61, c_first_name#86, c_last_name#87] -Input [6]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, c_customer_sk#85, c_first_name#86, c_last_name#87] +Output [4]: [ws_quantity#55, ws_list_price#56, c_first_name#42, c_last_name#43] +Input [6]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, c_customer_sk#30, c_first_name#42, c_last_name#43] (133) HashAggregate [codegen id : 51] -Input [4]: [ws_quantity#60, ws_list_price#61, c_first_name#86, c_last_name#87] -Keys [2]: [c_last_name#87, c_first_name#86] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#88, isEmpty#89] -Results [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] +Input [4]: [ws_quantity#55, ws_list_price#56, c_first_name#42, c_last_name#43] +Keys [2]: [c_last_name#43, c_first_name#42] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#72, isEmpty#73] +Results [4]: [c_last_name#43, c_first_name#42, sum#74, isEmpty#75] (134) Exchange -Input [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] -Arguments: hashpartitioning(c_last_name#87, c_first_name#86, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [4]: [c_last_name#43, c_first_name#42, sum#74, isEmpty#75] +Arguments: hashpartitioning(c_last_name#43, c_first_name#42, 5), ENSURE_REQUIREMENTS, [id=#76] (135) HashAggregate [codegen id : 52] -Input [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] -Keys [2]: [c_last_name#87, c_first_name#86] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#93] -Results [3]: [c_last_name#87, c_first_name#86, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#93 AS sales#94] +Input [4]: [c_last_name#43, c_first_name#42, sum#74, isEmpty#75] +Keys [2]: [c_last_name#43, c_first_name#42] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))#77] +Results [3]: [c_last_name#43, c_first_name#42, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))#77 AS sales#78] (136) Union (137) TakeOrderedAndProject -Input [3]: [c_last_name#49, c_first_name#48, sales#57] -Arguments: 100, [c_last_name#49 ASC NULLS FIRST, c_first_name#48 ASC NULLS FIRST, sales#57 ASC NULLS FIRST], [c_last_name#49, c_first_name#48, sales#57] +Input [3]: [c_last_name#43, c_first_name#42, sales#52] +Arguments: 100, [c_last_name#43 ASC NULLS FIRST, c_first_name#42 ASC NULLS FIRST, sales#52 ASC NULLS FIRST], [c_last_name#43, c_first_name#42, sales#52] ===== Subqueries ===== @@ -767,7 +767,7 @@ ReusedExchange (138) (138) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#43] +Output [1]: [d_date_sk#11] Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 ReusedExchange (139) @@ -776,7 +776,7 @@ ReusedExchange (139) (139) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#11, d_date#12] -Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#38, [id=#39] * HashAggregate (163) +- Exchange (162) +- * HashAggregate (161) @@ -804,135 +804,135 @@ Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (140) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#98), dynamicpruningexpression(ss_sold_date_sk#98 IN dynamicpruning#99)] +PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#79)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (141) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] (142) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] -Condition : isnotnull(ss_customer_sk#95) +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Condition : isnotnull(ss_customer_sk#26) (143) Scan parquet default.date_dim -Output [2]: [d_date_sk#100, d_year#101] +Output [2]: [d_date_sk#11, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#100, d_year#101] +Input [2]: [d_date_sk#11, d_year#13] (145) Filter [codegen id : 1] -Input [2]: [d_date_sk#100, d_year#101] -Condition : (d_year#101 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#100)) +Input [2]: [d_date_sk#11, d_year#13] +Condition : (d_year#13 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) (146) Project [codegen id : 1] -Output [1]: [d_date_sk#100] -Input [2]: [d_date_sk#100, d_year#101] +Output [1]: [d_date_sk#11] +Input [2]: [d_date_sk#11, d_year#13] (147) BroadcastExchange -Input [1]: [d_date_sk#100] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#102] +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#80] (148) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#98] -Right keys [1]: [d_date_sk#100] +Left keys [1]: [ss_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] Join condition: None (149) Project [codegen id : 2] -Output [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] -Input [5]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98, d_date_sk#100] +Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9, d_date_sk#11] (150) Exchange -Input [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] -Arguments: hashpartitioning(ss_customer_sk#95, 5), ENSURE_REQUIREMENTS, [id=#103] +Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#81] (151) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] -Arguments: [ss_customer_sk#95 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (152) Scan parquet default.customer -Output [1]: [c_customer_sk#104] +Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (153) ColumnarToRow [codegen id : 4] -Input [1]: [c_customer_sk#104] +Input [1]: [c_customer_sk#30] (154) Filter [codegen id : 4] -Input [1]: [c_customer_sk#104] -Condition : isnotnull(c_customer_sk#104) +Input [1]: [c_customer_sk#30] +Condition : isnotnull(c_customer_sk#30) (155) Exchange -Input [1]: [c_customer_sk#104] -Arguments: hashpartitioning(c_customer_sk#104, 5), ENSURE_REQUIREMENTS, [id=#105] +Input [1]: [c_customer_sk#30] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#82] (156) Sort [codegen id : 5] -Input [1]: [c_customer_sk#104] -Arguments: [c_customer_sk#104 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (157) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#95] -Right keys [1]: [c_customer_sk#104] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#30] Join condition: None (158) Project [codegen id : 6] -Output [3]: [ss_quantity#96, ss_sales_price#97, c_customer_sk#104] -Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, c_customer_sk#104] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (159) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#96, ss_sales_price#97, c_customer_sk#104] -Keys [1]: [c_customer_sk#104] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#106, isEmpty#107] -Results [3]: [c_customer_sk#104, sum#108, isEmpty#109] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#83, isEmpty#84] +Results [3]: [c_customer_sk#30, sum#85, isEmpty#86] (160) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#104, sum#108, isEmpty#109] -Keys [1]: [c_customer_sk#104] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))#110] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))#110 AS csales#111] +Input [3]: [c_customer_sk#30, sum#85, isEmpty#86] +Keys [1]: [c_customer_sk#30] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#87] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#87 AS csales#88] (161) HashAggregate [codegen id : 6] -Input [1]: [csales#111] +Input [1]: [csales#88] Keys: [] -Functions [1]: [partial_max(csales#111)] -Aggregate Attributes [1]: [max#112] -Results [1]: [max#113] +Functions [1]: [partial_max(csales#88)] +Aggregate Attributes [1]: [max#89] +Results [1]: [max#90] (162) Exchange -Input [1]: [max#113] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] +Input [1]: [max#90] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#91] (163) HashAggregate [codegen id : 7] -Input [1]: [max#113] +Input [1]: [max#90] Keys: [] -Functions [1]: [max(csales#111)] -Aggregate Attributes [1]: [max(csales#111)#115] -Results [1]: [max(csales#111)#115 AS tpcds_cmax#116] +Functions [1]: [max(csales#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#98 IN dynamicpruning#99 +Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#79 ReusedExchange (164) (164) ReusedExchange [Reuses operator id: 147] -Output [1]: [d_date_sk#100] +Output [1]: [d_date_sk#11] -Subquery:5 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:5 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] -Subquery:6 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#57 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 110 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:7 Hosting operator id = 110 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] -Subquery:8 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:8 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] 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 f5c64908c8fb9..385cf017e58c3 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 @@ -240,340 +240,340 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (30) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] (32) Filter [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Condition : isnotnull(ss_customer_sk#26) (33) Project [codegen id : 8] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] (34) Scan parquet default.customer -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 7] -Input [1]: [c_customer_sk#30] +Input [1]: [c_customer_sk#29] (36) Filter [codegen id : 7] -Input [1]: [c_customer_sk#30] -Condition : isnotnull(c_customer_sk#30) +Input [1]: [c_customer_sk#29] +Condition : isnotnull(c_customer_sk#29) (37) BroadcastExchange -Input [1]: [c_customer_sk#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [1]: [c_customer_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (38) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#29] Join condition: None (39) Project [codegen id : 8] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#29] (40) HashAggregate [codegen id : 8] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Keys [1]: [c_customer_sk#29] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Aggregate Attributes [2]: [sum#31, isEmpty#32] +Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] (41) Exchange -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#35] (42) HashAggregate [codegen id : 9] -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Keys [1]: [c_customer_sk#30] +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#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] +Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (43) Filter [codegen id : 9] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (44) Project [codegen id : 9] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Output [1]: [c_customer_sk#29] +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (45) Sort [codegen id : 9] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (46) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#29] Join condition: None (47) Scan parquet default.customer -Output [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Output [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 10] -Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] (49) Filter [codegen id : 10] -Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] -Condition : isnotnull(c_customer_sk#43) +Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Condition : isnotnull(c_customer_sk#29) (50) Exchange -Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] -Arguments: hashpartitioning(c_customer_sk#43, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#42] (51) Sort [codegen id : 11] -Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] -Arguments: [c_customer_sk#43 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (52) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Output [3]: [c_customer_sk#29, sum#33, isEmpty#34] (53) HashAggregate [codegen id : 14] -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Keys [1]: [c_customer_sk#30] +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#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] +Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (54) Filter [codegen id : 14] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (55) Project [codegen id : 14] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Output [1]: [c_customer_sk#29 AS c_customer_sk#29#43] +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (56) Sort [codegen id : 14] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29#43] +Arguments: [c_customer_sk#29#43 ASC NULLS FIRST], false, 0 (57) SortMergeJoin -Left keys [1]: [c_customer_sk#43] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#29#43] Join condition: None (58) BroadcastExchange -Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] +Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] (59) BroadcastHashJoin [codegen id : 16] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#43] +Right keys [1]: [c_customer_sk#29] Join condition: None (60) Project [codegen id : 16] -Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#44, c_last_name#45] -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#43, c_first_name#44, c_last_name#45] +Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#40, c_last_name#41] +Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#29, c_first_name#40, c_last_name#41] (61) Scan parquet default.date_dim -Output [3]: [d_date_sk#48, d_year#49, d_moy#50] +Output [3]: [d_date_sk#10, d_year#12, d_moy#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 15] -Input [3]: [d_date_sk#48, d_year#49, d_moy#50] +Input [3]: [d_date_sk#10, d_year#12, d_moy#45] (63) Filter [codegen id : 15] -Input [3]: [d_date_sk#48, d_year#49, d_moy#50] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2000)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#48)) +Input [3]: [d_date_sk#10, d_year#12, d_moy#45] +Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#45)) AND (d_year#12 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#10)) (64) Project [codegen id : 15] -Output [1]: [d_date_sk#48] -Input [3]: [d_date_sk#48, d_year#49, d_moy#50] +Output [1]: [d_date_sk#10] +Input [3]: [d_date_sk#10, d_year#12, d_moy#45] (65) BroadcastExchange -Input [1]: [d_date_sk#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] +Input [1]: [d_date_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] (66) BroadcastHashJoin [codegen id : 16] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#48] +Right keys [1]: [d_date_sk#10] Join condition: None (67) Project [codegen id : 16] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#44, c_last_name#45] -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#44, c_last_name#45, d_date_sk#48] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#40, c_last_name#41] +Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#40, c_last_name#41, d_date_sk#10] (68) HashAggregate [codegen id : 16] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#44, c_last_name#45] -Keys [2]: [c_last_name#45, c_first_name#44] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#40, c_last_name#41] +Keys [2]: [c_last_name#41, c_first_name#40] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#47, isEmpty#48] +Results [4]: [c_last_name#41, c_first_name#40, sum#49, isEmpty#50] (69) Exchange -Input [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] -Arguments: hashpartitioning(c_last_name#45, c_first_name#44, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [4]: [c_last_name#41, c_first_name#40, sum#49, isEmpty#50] +Arguments: hashpartitioning(c_last_name#41, c_first_name#40, 5), ENSURE_REQUIREMENTS, [id=#51] (70) HashAggregate [codegen id : 17] -Input [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] -Keys [2]: [c_last_name#45, c_first_name#44] +Input [4]: [c_last_name#41, c_first_name#40, sum#49, isEmpty#50] +Keys [2]: [c_last_name#41, c_first_name#40] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#57] -Results [3]: [c_last_name#45, c_first_name#44, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#57 AS sales#58] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#52] +Results [3]: [c_last_name#41, c_first_name#40, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#52 AS sales#53] (71) Scan parquet default.web_sales -Output [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Output [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 22] -Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] (73) Filter [codegen id : 22] -Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_customer_sk#60) +Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Condition : isnotnull(ws_bill_customer_sk#55) (74) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] (75) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_item_sk#59] +Left keys [1]: [ws_item_sk#54] Right keys [1]: [item_sk#22] Join condition: None (76) Project [codegen id : 22] -Output [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] -Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Output [4]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] (77) Exchange -Input [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] -Arguments: hashpartitioning(ws_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [4]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Arguments: hashpartitioning(ws_bill_customer_sk#55, 5), ENSURE_REQUIREMENTS, [id=#59] (78) Sort [codegen id : 23] -Input [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] -Arguments: [ws_bill_customer_sk#60 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Arguments: [ws_bill_customer_sk#55 ASC NULLS FIRST], false, 0 (79) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#65, sum#66, isEmpty#67] +Output [3]: [c_customer_sk#29, sum#60, isEmpty#61] (80) HashAggregate [codegen id : 26] -Input [3]: [c_customer_sk#65, sum#66, isEmpty#67] -Keys [1]: [c_customer_sk#65] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70] -Results [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] +Input [3]: [c_customer_sk#29, sum#60, isEmpty#61] +Keys [1]: [c_customer_sk#29] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62] +Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] (81) Filter [codegen id : 26] -Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (82) Project [codegen id : 26] -Output [1]: [c_customer_sk#65] -Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] +Output [1]: [c_customer_sk#29] +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] (83) Sort [codegen id : 26] -Input [1]: [c_customer_sk#65] -Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (84) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#60] -Right keys [1]: [c_customer_sk#65] +Left keys [1]: [ws_bill_customer_sk#55] +Right keys [1]: [c_customer_sk#29] Join condition: None (85) ReusedExchange [Reuses operator id: 50] -Output [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] +Output [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] (86) Sort [codegen id : 28] -Input [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] -Arguments: [c_customer_sk#72 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#65, sum#66, isEmpty#67] +Output [3]: [c_customer_sk#29, sum#60, isEmpty#61] (88) HashAggregate [codegen id : 31] -Input [3]: [c_customer_sk#65, sum#66, isEmpty#67] -Keys [1]: [c_customer_sk#65] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70] -Results [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] +Input [3]: [c_customer_sk#29, sum#60, isEmpty#61] +Keys [1]: [c_customer_sk#29] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62] +Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] (89) Filter [codegen id : 31] -Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (90) Project [codegen id : 31] -Output [1]: [c_customer_sk#65] -Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] +Output [1]: [c_customer_sk#29 AS c_customer_sk#29#64] +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] (91) Sort [codegen id : 31] -Input [1]: [c_customer_sk#65] -Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29#64] +Arguments: [c_customer_sk#29#64 ASC NULLS FIRST], false, 0 (92) SortMergeJoin -Left keys [1]: [c_customer_sk#72] -Right keys [1]: [c_customer_sk#65] +Left keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#29#64] Join condition: None (93) BroadcastExchange -Input [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] +Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] (94) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ws_bill_customer_sk#60] -Right keys [1]: [c_customer_sk#72] +Left keys [1]: [ws_bill_customer_sk#55] +Right keys [1]: [c_customer_sk#29] Join condition: None (95) Project [codegen id : 33] -Output [5]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74] -Input [7]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_customer_sk#72, c_first_name#73, c_last_name#74] +Output [5]: [ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58, c_first_name#40, c_last_name#41] +Input [7]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58, c_customer_sk#29, c_first_name#40, c_last_name#41] (96) ReusedExchange [Reuses operator id: 65] -Output [1]: [d_date_sk#76] +Output [1]: [d_date_sk#10] (97) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ws_sold_date_sk#63] -Right keys [1]: [d_date_sk#76] +Left keys [1]: [ws_sold_date_sk#58] +Right keys [1]: [d_date_sk#10] Join condition: None (98) Project [codegen id : 33] -Output [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74] -Input [6]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74, d_date_sk#76] +Output [4]: [ws_quantity#56, ws_list_price#57, c_first_name#40, c_last_name#41] +Input [6]: [ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58, c_first_name#40, c_last_name#41, d_date_sk#10] (99) HashAggregate [codegen id : 33] -Input [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74] -Keys [2]: [c_last_name#74, c_first_name#73] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#77, isEmpty#78] -Results [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] +Input [4]: [ws_quantity#56, ws_list_price#57, c_first_name#40, c_last_name#41] +Keys [2]: [c_last_name#41, c_first_name#40] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#66, isEmpty#67] +Results [4]: [c_last_name#41, c_first_name#40, sum#68, isEmpty#69] (100) Exchange -Input [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] -Arguments: hashpartitioning(c_last_name#74, c_first_name#73, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [4]: [c_last_name#41, c_first_name#40, sum#68, isEmpty#69] +Arguments: hashpartitioning(c_last_name#41, c_first_name#40, 5), ENSURE_REQUIREMENTS, [id=#70] (101) HashAggregate [codegen id : 34] -Input [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] -Keys [2]: [c_last_name#74, c_first_name#73] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#82] -Results [3]: [c_last_name#74, c_first_name#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sales#83] +Input [4]: [c_last_name#41, c_first_name#40, sum#68, isEmpty#69] +Keys [2]: [c_last_name#41, c_first_name#40] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))#71] +Results [3]: [c_last_name#41, c_first_name#40, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sales#72] (102) Union (103) TakeOrderedAndProject -Input [3]: [c_last_name#45, c_first_name#44, sales#58] -Arguments: 100, [c_last_name#45 ASC NULLS FIRST, c_first_name#44 ASC NULLS FIRST, sales#58 ASC NULLS FIRST], [c_last_name#45, c_first_name#44, sales#58] +Input [3]: [c_last_name#41, c_first_name#40, sales#53] +Arguments: 100, [c_last_name#41 ASC NULLS FIRST, c_first_name#40 ASC NULLS FIRST, sales#53 ASC NULLS FIRST], [c_last_name#41, c_first_name#40, sales#53] ===== Subqueries ===== @@ -582,7 +582,7 @@ ReusedExchange (104) (104) ReusedExchange [Reuses operator id: 65] -Output [1]: [d_date_sk#48] +Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 ReusedExchange (105) @@ -591,7 +591,7 @@ ReusedExchange (105) (105) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#10, d_date#11] -Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#38, [id=#39] * HashAggregate (127) +- Exchange (126) +- * HashAggregate (125) @@ -617,127 +617,127 @@ Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquer (106) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (107) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] (108) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] -Condition : isnotnull(ss_customer_sk#84) +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Condition : isnotnull(ss_customer_sk#26) (109) Scan parquet default.customer -Output [1]: [c_customer_sk#89] +Output [1]: [c_customer_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (110) ColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#89] +Input [1]: [c_customer_sk#29] (111) Filter [codegen id : 1] -Input [1]: [c_customer_sk#89] -Condition : isnotnull(c_customer_sk#89) +Input [1]: [c_customer_sk#29] +Condition : isnotnull(c_customer_sk#29) (112) BroadcastExchange -Input [1]: [c_customer_sk#89] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#90] +Input [1]: [c_customer_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] (113) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#84] -Right keys [1]: [c_customer_sk#89] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#29] Join condition: None (114) Project [codegen id : 3] -Output [4]: [ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89] -Input [5]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89] +Output [4]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] +Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] (115) Scan parquet default.date_dim -Output [2]: [d_date_sk#91, d_year#92] +Output [2]: [d_date_sk#10, d_year#12] 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 (116) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#91, d_year#92] +Input [2]: [d_date_sk#10, d_year#12] (117) Filter [codegen id : 2] -Input [2]: [d_date_sk#91, d_year#92] -Condition : (d_year#92 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#91)) +Input [2]: [d_date_sk#10, d_year#12] +Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) (118) Project [codegen id : 2] -Output [1]: [d_date_sk#91] -Input [2]: [d_date_sk#91, d_year#92] +Output [1]: [d_date_sk#10] +Input [2]: [d_date_sk#10, d_year#12] (119) BroadcastExchange -Input [1]: [d_date_sk#91] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] +Input [1]: [d_date_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#75] (120) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#87] -Right keys [1]: [d_date_sk#91] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] Join condition: None (121) Project [codegen id : 3] -Output [3]: [ss_quantity#85, ss_sales_price#86, c_customer_sk#89] -Input [5]: [ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89, d_date_sk#91] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Input [5]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29, d_date_sk#10] (122) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#85, ss_sales_price#86, c_customer_sk#89] -Keys [1]: [c_customer_sk#89] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#94, isEmpty#95] -Results [3]: [c_customer_sk#89, sum#96, isEmpty#97] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Keys [1]: [c_customer_sk#29] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#76, isEmpty#77] +Results [3]: [c_customer_sk#29, sum#78, isEmpty#79] (123) Exchange -Input [3]: [c_customer_sk#89, sum#96, isEmpty#97] -Arguments: hashpartitioning(c_customer_sk#89, 5), ENSURE_REQUIREMENTS, [id=#98] +Input [3]: [c_customer_sk#29, sum#78, isEmpty#79] +Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#80] (124) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#89, sum#96, isEmpty#97] -Keys [1]: [c_customer_sk#89] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))#99] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))#99 AS csales#100] +Input [3]: [c_customer_sk#29, sum#78, isEmpty#79] +Keys [1]: [c_customer_sk#29] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#81] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#81 AS csales#82] (125) HashAggregate [codegen id : 4] -Input [1]: [csales#100] +Input [1]: [csales#82] Keys: [] -Functions [1]: [partial_max(csales#100)] -Aggregate Attributes [1]: [max#101] -Results [1]: [max#102] +Functions [1]: [partial_max(csales#82)] +Aggregate Attributes [1]: [max#83] +Results [1]: [max#84] (126) Exchange -Input [1]: [max#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103] +Input [1]: [max#84] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#85] (127) HashAggregate [codegen id : 5] -Input [1]: [max#102] +Input [1]: [max#84] Keys: [] -Functions [1]: [max(csales#100)] -Aggregate Attributes [1]: [max(csales#100)#104] -Results [1]: [max(csales#100)#104 AS tpcds_cmax#105] +Functions [1]: [max(csales#82)] +Aggregate Attributes [1]: [max(csales#82)#86] +Results [1]: [max(csales#82)#86 AS tpcds_cmax#87] -Subquery:4 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#88 +Subquery:4 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#73 ReusedExchange (128) (128) ReusedExchange [Reuses operator id: 119] -Output [1]: [d_date_sk#91] +Output [1]: [d_date_sk#10] -Subquery:5 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:5 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] -Subquery:6 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:7 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] -Subquery:8 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:8 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] 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 7895ee1838ad7..e9a0ada3801b4 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 @@ -73,7 +73,7 @@ Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s 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_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -81,7 +81,7 @@ Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_ (7) Filter [codegen id : 1] 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_color#10) AND (i_color#10 = pale )) AND isnotnull(i_item_sk#7)) +Condition : ((isnotnull(i_color#10) AND (i_color#10 = pale)) AND isnotnull(i_item_sk#7)) (8) BroadcastExchange Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] @@ -340,248 +340,248 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (51) Scan parquet default.store_sales -Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +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 (52) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +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] (53) Filter [codegen id : 2] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] -Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) +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)) (54) Project [codegen id : 2] -Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +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] (55) Scan parquet default.store -Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Output [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (56) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] (57) Filter [codegen id : 1] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Condition : (((isnotnull(s_market_id#22) AND (s_market_id#22 = 8)) AND isnotnull(s_store_sk#20)) AND isnotnull(s_zip#24)) (58) Project [codegen id : 1] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Output [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] (59) BroadcastExchange -Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] +Input [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] (60) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#52] -Right keys [1]: [s_store_sk#56] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#20] Join condition: None (61) Project [codegen id : 2] -Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] +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#20, s_store_name#21, s_state#23, s_zip#24] (62) Exchange -Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#51] (63) Sort [codegen id : 3] -Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: [ss_customer_sk#51 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#21, s_state#23, s_zip#24] +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (64) Scan parquet default.customer -Output [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (65) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] (66) Filter [codegen id : 4] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_birth_country#66)) +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_birth_country#18)) (67) Exchange -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, [id=#67] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#52] (68) Sort [codegen id : 5] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: [c_customer_sk#63 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 (69) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#51] -Right keys [1]: [c_customer_sk#63] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#15] Join condition: None (70) Project [codegen id : 6] -Output [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Input [11]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] (71) Exchange -Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#53] (72) Sort [codegen id : 7] -Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (73) Scan parquet default.item -Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +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 (74) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] (75) Filter [codegen id : 8] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Condition : isnotnull(i_item_sk#69) +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) (76) Exchange -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: hashpartitioning(i_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] +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=#54] (77) Sort [codegen id : 9] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: [i_item_sk#69 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 (78) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#50] -Right keys [1]: [i_item_sk#69] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#7] Join condition: None (79) Project [codegen id : 10] -Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] (80) Exchange -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: hashpartitioning(c_birth_country#66, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: hashpartitioning(c_birth_country#18, s_zip#24, 5), ENSURE_REQUIREMENTS, [id=#55] (81) Sort [codegen id : 11] -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: [c_birth_country#66 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: [c_birth_country#18 ASC NULLS FIRST, s_zip#24 ASC NULLS FIRST], false, 0 (82) Scan parquet default.customer_address -Output [3]: [ca_state#77, ca_zip#78, ca_country#79] +Output [3]: [ca_state#26, ca_zip#27, ca_country#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (83) ColumnarToRow [codegen id : 12] -Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] (84) Filter [codegen id : 12] -Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Condition : (isnotnull(ca_country#28) AND isnotnull(ca_zip#27)) (85) Exchange -Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -Arguments: hashpartitioning(upper(ca_country#79), ca_zip#78, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Arguments: hashpartitioning(upper(ca_country#28), ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#56] (86) Sort [codegen id : 13] -Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -Arguments: [upper(ca_country#79) ASC NULLS FIRST, ca_zip#78 ASC NULLS FIRST], false, 0 +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Arguments: [upper(ca_country#28) ASC NULLS FIRST, ca_zip#27 ASC NULLS FIRST], false, 0 (87) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#66, s_zip#60] -Right keys [2]: [upper(ca_country#79), ca_zip#78] +Left keys [2]: [c_birth_country#18, s_zip#24] +Right keys [2]: [upper(ca_country#28), ca_zip#27] Join condition: None (88) Project [codegen id : 14] -Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] -Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, ca_zip#78, ca_country#79] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] +Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, ca_zip#27, ca_country#28] (89) Exchange -Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] -Arguments: hashpartitioning(cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint), 5), ENSURE_REQUIREMENTS, [id=#81] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#57] (90) Sort [codegen id : 15] -Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] -Arguments: [cast(ss_ticket_number#53 as bigint) ASC NULLS FIRST, cast(ss_item_sk#50 as bigint) ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] +Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (91) Scan parquet default.store_returns -Output [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Output [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 16] -Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] (93) Filter [codegen id : 16] -Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] -Condition : (isnotnull(sr_ticket_number#83) AND isnotnull(sr_item_sk#82)) +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) (94) Project [codegen id : 16] -Output [2]: [sr_item_sk#82, sr_ticket_number#83] -Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Output [2]: [sr_item_sk#31, sr_ticket_number#32] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] (95) Exchange -Input [2]: [sr_item_sk#82, sr_ticket_number#83] -Arguments: hashpartitioning(sr_ticket_number#83, sr_item_sk#82, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#58] (96) Sort [codegen id : 17] -Input [2]: [sr_item_sk#82, sr_ticket_number#83] -Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 (97) SortMergeJoin [codegen id : 18] -Left keys [2]: [cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint)] -Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] +Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] +Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] Join condition: None (98) Project [codegen id : 18] -Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] -Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, sr_item_sk#82, sr_ticket_number#83] +Output [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, sr_item_sk#31, sr_ticket_number#32] (99) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] -Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] -Aggregate Attributes [1]: [sum#86] -Results [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Input [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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#59] +Results [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] (100) Exchange -Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] -Arguments: hashpartitioning(c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#61] (101) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] -Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] -Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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))#62] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#62,17,2) AS netpaid#39] (102) HashAggregate [codegen id : 19] Input [1]: [netpaid#39] Keys: [] Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#90, count#91] -Results [2]: [sum#92, count#93] +Aggregate Attributes [2]: [sum#63, count#64] +Results [2]: [sum#65, count#66] (103) Exchange -Input [2]: [sum#92, count#93] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] +Input [2]: [sum#65, count#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] (104) HashAggregate [codegen id : 20] -Input [2]: [sum#92, count#93] +Input [2]: [sum#65, count#66] Keys: [] Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#95] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] +Aggregate Attributes [1]: [avg(netpaid#39)#68] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#68)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#69] 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 0565ae76a440d..3b304ebe3f649 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 @@ -144,7 +144,7 @@ Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_stor 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_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] @@ -152,7 +152,7 @@ Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, (24) Filter [codegen id : 6] 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_color#21) AND (i_color#21 = pale )) AND isnotnull(i_item_sk#18)) +Condition : ((isnotnull(i_color#21) AND (i_color#21 = pale)) AND isnotnull(i_item_sk#18)) (25) BroadcastExchange Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] @@ -316,212 +316,212 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (48) Scan parquet default.store_sales -Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +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 (49) ColumnarToRow [codegen id : 1] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +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] (50) Filter [codegen id : 1] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] -Condition : (((isnotnull(ss_ticket_number#52) AND isnotnull(ss_item_sk#49)) AND isnotnull(ss_store_sk#51)) AND isnotnull(ss_customer_sk#50)) +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)) (51) Project [codegen id : 1] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +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] (52) Exchange -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: hashpartitioning(cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#49] (53) Sort [codegen id : 2] -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [cast(ss_ticket_number#52 as bigint) ASC NULLS FIRST, cast(ss_item_sk#49 as bigint) 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: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.store_returns -Output [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Output [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] (56) Filter [codegen id : 3] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] -Condition : (isnotnull(sr_ticket_number#57) AND isnotnull(sr_item_sk#56)) +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Condition : (isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#8)) (57) Project [codegen id : 3] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Output [2]: [sr_item_sk#8, sr_ticket_number#9] +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] (58) Exchange -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: hashpartitioning(sr_ticket_number#57, sr_item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [2]: [sr_item_sk#8, sr_ticket_number#9] +Arguments: hashpartitioning(sr_ticket_number#9, sr_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#50] (59) Sort [codegen id : 4] -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST], false, 0 +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 (60) SortMergeJoin [codegen id : 9] -Left keys [2]: [cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint)] -Right keys [2]: [sr_ticket_number#57, sr_item_sk#56] +Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] +Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] Join condition: None (61) Project [codegen id : 9] -Output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#56, sr_ticket_number#57] +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] (62) Scan parquet default.store -Output [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Output [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (63) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] (64) Filter [codegen id : 5] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] -Condition : (((isnotnull(s_market_id#62) AND (s_market_id#62 = 8)) AND isnotnull(s_store_sk#60)) AND isnotnull(s_zip#64)) +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Condition : (((isnotnull(s_market_id#14) AND (s_market_id#14 = 8)) AND isnotnull(s_store_sk#12)) AND isnotnull(s_zip#16)) (65) Project [codegen id : 5] -Output [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] (66) BroadcastExchange -Input [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] +Input [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] (67) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#51] -Right keys [1]: [s_store_sk#60] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#12] Join condition: None (68) Project [codegen id : 9] -Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64] -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +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] (69) Scan parquet default.item -Output [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +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 (70) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] (71) Filter [codegen id : 6] -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Condition : isnotnull(i_item_sk#66) +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) (72) BroadcastExchange -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#72] +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=#52] (73) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#49] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#18] Join condition: None (74) Project [codegen id : 9] -Output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +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] (75) Scan parquet default.customer -Output [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Output [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (76) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] (77) Filter [codegen id : 7] -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] -Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_birth_country#76)) +Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_birth_country#28)) (78) BroadcastExchange -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] +Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] (79) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#50] -Right keys [1]: [c_customer_sk#73] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#25] Join condition: None (80) Project [codegen id : 9] -Output [12]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76] -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +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] (81) Scan parquet default.customer_address -Output [3]: [ca_state#78, ca_zip#79, ca_country#80] +Output [3]: [ca_state#30, ca_zip#31, ca_country#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (82) ColumnarToRow [codegen id : 8] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Input [3]: [ca_state#30, ca_zip#31, ca_country#32] (83) Filter [codegen id : 8] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) +Input [3]: [ca_state#30, ca_zip#31, ca_country#32] +Condition : (isnotnull(ca_country#32) AND isnotnull(ca_zip#31)) (84) BroadcastExchange -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#81] +Input [3]: [ca_state#30, ca_zip#31, ca_country#32] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#54] (85) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#76, s_zip#64] -Right keys [2]: [upper(ca_country#80), ca_zip#79] +Left keys [2]: [c_birth_country#28, s_zip#16] +Right keys [2]: [upper(ca_country#32), ca_zip#31] Join condition: None (86) Project [codegen id : 9] -Output [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] -Input [15]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76, ca_state#78, ca_zip#79, ca_country#80] +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] (87) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] -Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#82] -Results [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +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#55] +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#56] (88) Exchange -Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] -Arguments: hashpartitioning(c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, [id=#84] +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#56] +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=#57] (89) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] -Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#85] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#38] +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#56] +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))#58] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#58,17,2) AS netpaid#38] (90) HashAggregate [codegen id : 10] Input [1]: [netpaid#38] Keys: [] Functions [1]: [partial_avg(netpaid#38)] -Aggregate Attributes [2]: [sum#86, count#87] -Results [2]: [sum#88, count#89] +Aggregate Attributes [2]: [sum#59, count#60] +Results [2]: [sum#61, count#62] (91) 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] (92) HashAggregate [codegen id : 11] -Input [2]: [sum#88, count#89] +Input [2]: [sum#61, count#62] Keys: [] Functions [1]: [avg(netpaid#38)] -Aggregate Attributes [1]: [avg(netpaid#38)#91] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] +Aggregate Attributes [1]: [avg(netpaid#38)#64] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#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.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt index e45459d529104..695f6efe560e4 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 @@ -73,7 +73,7 @@ Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s 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_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon), IsNotNull(i_item_sk)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -81,7 +81,7 @@ Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_ (7) Filter [codegen id : 1] 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_color#10) AND (i_color#10 = chiffon )) AND isnotnull(i_item_sk#7)) +Condition : ((isnotnull(i_color#10) AND (i_color#10 = chiffon)) AND isnotnull(i_item_sk#7)) (8) BroadcastExchange Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] @@ -340,248 +340,248 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (51) Scan parquet default.store_sales -Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +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 (52) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +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] (53) Filter [codegen id : 2] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] -Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) +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)) (54) Project [codegen id : 2] -Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +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] (55) Scan parquet default.store -Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Output [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (56) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] (57) Filter [codegen id : 1] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Condition : (((isnotnull(s_market_id#22) AND (s_market_id#22 = 8)) AND isnotnull(s_store_sk#20)) AND isnotnull(s_zip#24)) (58) Project [codegen id : 1] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Output [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] (59) BroadcastExchange -Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] +Input [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] (60) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#52] -Right keys [1]: [s_store_sk#56] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#20] Join condition: None (61) Project [codegen id : 2] -Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] +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#20, s_store_name#21, s_state#23, s_zip#24] (62) Exchange -Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#51] (63) Sort [codegen id : 3] -Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: [ss_customer_sk#51 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#21, s_state#23, s_zip#24] +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (64) Scan parquet default.customer -Output [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (65) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] (66) Filter [codegen id : 4] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_birth_country#66)) +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_birth_country#18)) (67) Exchange -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, [id=#67] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#52] (68) Sort [codegen id : 5] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: [c_customer_sk#63 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 (69) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#51] -Right keys [1]: [c_customer_sk#63] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#15] Join condition: None (70) Project [codegen id : 6] -Output [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Input [11]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] (71) Exchange -Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#53] (72) Sort [codegen id : 7] -Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (73) Scan parquet default.item -Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +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 (74) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] (75) Filter [codegen id : 8] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Condition : isnotnull(i_item_sk#69) +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) (76) Exchange -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: hashpartitioning(i_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] +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=#54] (77) Sort [codegen id : 9] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: [i_item_sk#69 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 (78) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#50] -Right keys [1]: [i_item_sk#69] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#7] Join condition: None (79) Project [codegen id : 10] -Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] (80) Exchange -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: hashpartitioning(c_birth_country#66, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: hashpartitioning(c_birth_country#18, s_zip#24, 5), ENSURE_REQUIREMENTS, [id=#55] (81) Sort [codegen id : 11] -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: [c_birth_country#66 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: [c_birth_country#18 ASC NULLS FIRST, s_zip#24 ASC NULLS FIRST], false, 0 (82) Scan parquet default.customer_address -Output [3]: [ca_state#77, ca_zip#78, ca_country#79] +Output [3]: [ca_state#26, ca_zip#27, ca_country#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (83) ColumnarToRow [codegen id : 12] -Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] (84) Filter [codegen id : 12] -Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Condition : (isnotnull(ca_country#28) AND isnotnull(ca_zip#27)) (85) Exchange -Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -Arguments: hashpartitioning(upper(ca_country#79), ca_zip#78, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Arguments: hashpartitioning(upper(ca_country#28), ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#56] (86) Sort [codegen id : 13] -Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -Arguments: [upper(ca_country#79) ASC NULLS FIRST, ca_zip#78 ASC NULLS FIRST], false, 0 +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Arguments: [upper(ca_country#28) ASC NULLS FIRST, ca_zip#27 ASC NULLS FIRST], false, 0 (87) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#66, s_zip#60] -Right keys [2]: [upper(ca_country#79), ca_zip#78] +Left keys [2]: [c_birth_country#18, s_zip#24] +Right keys [2]: [upper(ca_country#28), ca_zip#27] Join condition: None (88) Project [codegen id : 14] -Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] -Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, ca_zip#78, ca_country#79] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] +Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, ca_zip#27, ca_country#28] (89) Exchange -Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] -Arguments: hashpartitioning(cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint), 5), ENSURE_REQUIREMENTS, [id=#81] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#57] (90) Sort [codegen id : 15] -Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] -Arguments: [cast(ss_ticket_number#53 as bigint) ASC NULLS FIRST, cast(ss_item_sk#50 as bigint) ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] +Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (91) Scan parquet default.store_returns -Output [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Output [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 16] -Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] (93) Filter [codegen id : 16] -Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] -Condition : (isnotnull(sr_ticket_number#83) AND isnotnull(sr_item_sk#82)) +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) (94) Project [codegen id : 16] -Output [2]: [sr_item_sk#82, sr_ticket_number#83] -Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Output [2]: [sr_item_sk#31, sr_ticket_number#32] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] (95) Exchange -Input [2]: [sr_item_sk#82, sr_ticket_number#83] -Arguments: hashpartitioning(sr_ticket_number#83, sr_item_sk#82, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#58] (96) Sort [codegen id : 17] -Input [2]: [sr_item_sk#82, sr_ticket_number#83] -Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 (97) SortMergeJoin [codegen id : 18] -Left keys [2]: [cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint)] -Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] +Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] +Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] Join condition: None (98) Project [codegen id : 18] -Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] -Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, sr_item_sk#82, sr_ticket_number#83] +Output [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, sr_item_sk#31, sr_ticket_number#32] (99) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] -Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] -Aggregate Attributes [1]: [sum#86] -Results [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Input [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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#59] +Results [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] (100) Exchange -Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] -Arguments: hashpartitioning(c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#61] (101) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] -Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] -Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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))#62] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#62,17,2) AS netpaid#39] (102) HashAggregate [codegen id : 19] Input [1]: [netpaid#39] Keys: [] Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#90, count#91] -Results [2]: [sum#92, count#93] +Aggregate Attributes [2]: [sum#63, count#64] +Results [2]: [sum#65, count#66] (103) Exchange -Input [2]: [sum#92, count#93] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] +Input [2]: [sum#65, count#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] (104) HashAggregate [codegen id : 20] -Input [2]: [sum#92, count#93] +Input [2]: [sum#65, count#66] Keys: [] Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#95] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] +Aggregate Attributes [1]: [avg(netpaid#39)#68] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#68)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#69] 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 aeaf3f56b9e13..49f76525f88d8 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 @@ -144,7 +144,7 @@ Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_stor 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_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon), IsNotNull(i_item_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] @@ -152,7 +152,7 @@ Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, (24) Filter [codegen id : 6] 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_color#21) AND (i_color#21 = chiffon )) AND isnotnull(i_item_sk#18)) +Condition : ((isnotnull(i_color#21) AND (i_color#21 = chiffon)) AND isnotnull(i_item_sk#18)) (25) BroadcastExchange Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] @@ -316,212 +316,212 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (48) Scan parquet default.store_sales -Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +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 (49) ColumnarToRow [codegen id : 1] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +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] (50) Filter [codegen id : 1] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] -Condition : (((isnotnull(ss_ticket_number#52) AND isnotnull(ss_item_sk#49)) AND isnotnull(ss_store_sk#51)) AND isnotnull(ss_customer_sk#50)) +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)) (51) Project [codegen id : 1] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +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] (52) Exchange -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: hashpartitioning(cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#49] (53) Sort [codegen id : 2] -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [cast(ss_ticket_number#52 as bigint) ASC NULLS FIRST, cast(ss_item_sk#49 as bigint) 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: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.store_returns -Output [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Output [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] (56) Filter [codegen id : 3] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] -Condition : (isnotnull(sr_ticket_number#57) AND isnotnull(sr_item_sk#56)) +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Condition : (isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#8)) (57) Project [codegen id : 3] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Output [2]: [sr_item_sk#8, sr_ticket_number#9] +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] (58) Exchange -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: hashpartitioning(sr_ticket_number#57, sr_item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [2]: [sr_item_sk#8, sr_ticket_number#9] +Arguments: hashpartitioning(sr_ticket_number#9, sr_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#50] (59) Sort [codegen id : 4] -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST], false, 0 +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 (60) SortMergeJoin [codegen id : 9] -Left keys [2]: [cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint)] -Right keys [2]: [sr_ticket_number#57, sr_item_sk#56] +Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] +Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] Join condition: None (61) Project [codegen id : 9] -Output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#56, sr_ticket_number#57] +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] (62) Scan parquet default.store -Output [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Output [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (63) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] (64) Filter [codegen id : 5] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] -Condition : (((isnotnull(s_market_id#62) AND (s_market_id#62 = 8)) AND isnotnull(s_store_sk#60)) AND isnotnull(s_zip#64)) +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Condition : (((isnotnull(s_market_id#14) AND (s_market_id#14 = 8)) AND isnotnull(s_store_sk#12)) AND isnotnull(s_zip#16)) (65) Project [codegen id : 5] -Output [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] (66) BroadcastExchange -Input [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] +Input [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] (67) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#51] -Right keys [1]: [s_store_sk#60] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#12] Join condition: None (68) Project [codegen id : 9] -Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64] -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +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] (69) Scan parquet default.item -Output [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +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 (70) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] (71) Filter [codegen id : 6] -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Condition : isnotnull(i_item_sk#66) +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) (72) BroadcastExchange -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#72] +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=#52] (73) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#49] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#18] Join condition: None (74) Project [codegen id : 9] -Output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +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] (75) Scan parquet default.customer -Output [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Output [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (76) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] (77) Filter [codegen id : 7] -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] -Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_birth_country#76)) +Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_birth_country#28)) (78) BroadcastExchange -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] +Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] (79) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#50] -Right keys [1]: [c_customer_sk#73] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#25] Join condition: None (80) Project [codegen id : 9] -Output [12]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76] -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +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] (81) Scan parquet default.customer_address -Output [3]: [ca_state#78, ca_zip#79, ca_country#80] +Output [3]: [ca_state#30, ca_zip#31, ca_country#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (82) ColumnarToRow [codegen id : 8] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Input [3]: [ca_state#30, ca_zip#31, ca_country#32] (83) Filter [codegen id : 8] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) +Input [3]: [ca_state#30, ca_zip#31, ca_country#32] +Condition : (isnotnull(ca_country#32) AND isnotnull(ca_zip#31)) (84) BroadcastExchange -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#81] +Input [3]: [ca_state#30, ca_zip#31, ca_country#32] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#54] (85) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#76, s_zip#64] -Right keys [2]: [upper(ca_country#80), ca_zip#79] +Left keys [2]: [c_birth_country#28, s_zip#16] +Right keys [2]: [upper(ca_country#32), ca_zip#31] Join condition: None (86) Project [codegen id : 9] -Output [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] -Input [15]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76, ca_state#78, ca_zip#79, ca_country#80] +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] (87) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] -Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#82] -Results [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +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#55] +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#56] (88) Exchange -Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] -Arguments: hashpartitioning(c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, [id=#84] +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#56] +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=#57] (89) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] -Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#85] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#38] +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#56] +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))#58] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#58,17,2) AS netpaid#38] (90) HashAggregate [codegen id : 10] Input [1]: [netpaid#38] Keys: [] Functions [1]: [partial_avg(netpaid#38)] -Aggregate Attributes [2]: [sum#86, count#87] -Results [2]: [sum#88, count#89] +Aggregate Attributes [2]: [sum#59, count#60] +Results [2]: [sum#61, count#62] (91) 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] (92) HashAggregate [codegen id : 11] -Input [2]: [sum#88, count#89] +Input [2]: [sum#61, count#62] Keys: [] Functions [1]: [avg(netpaid#38)] -Aggregate Attributes [1]: [avg(netpaid#38)#91] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] +Aggregate Attributes [1]: [avg(netpaid#38)#64] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#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/q26.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt index 40565dc76a7ba..e38776ac2784b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isn Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt index c86f121ad35a9..0de1f44569c19 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(cs_bill_cdemo_sk#1) AND isnotnull(cs_item_sk#2)) AND isn Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt index 8c530f5a37885..f4a07e6ac8f21 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnu Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt index 8c530f5a37885..f4a07e6ac8f21 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnu Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt index eec45ea549531..9f15199d8c8b7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt @@ -126,306 +126,306 @@ Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#6 / 100.0) as decimal(11,6)) AS B1_LP#15, count(ss_list_price#3)#7 AS B1_CNT#16, count(ss_list_price#3)#12 AS B1_CNTD#17] (11) Scan parquet default.store_sales -Output [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (13) Filter [codegen id : 3] -Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] -Condition : (((isnotnull(ss_quantity#18) AND (ss_quantity#18 >= 6)) AND (ss_quantity#18 <= 10)) AND ((((ss_list_price#20 >= 90.00) AND (ss_list_price#20 <= 100.00)) OR ((ss_coupon_amt#21 >= 2323.00) AND (ss_coupon_amt#21 <= 3323.00))) OR ((ss_wholesale_cost#19 >= 31.00) AND (ss_wholesale_cost#19 <= 51.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 6)) AND (ss_quantity#1 <= 10)) AND ((((ss_list_price#3 >= 90.00) AND (ss_list_price#3 <= 100.00)) OR ((ss_coupon_amt#4 >= 2323.00) AND (ss_coupon_amt#4 <= 3323.00))) OR ((ss_wholesale_cost#2 >= 31.00) AND (ss_wholesale_cost#2 <= 51.00)))) (14) Project [codegen id : 3] -Output [1]: [ss_list_price#20] -Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (15) HashAggregate [codegen id : 3] -Input [1]: [ss_list_price#20] -Keys [1]: [ss_list_price#20] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#20)), partial_count(ss_list_price#20)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] -Results [4]: [ss_list_price#20, sum#25, count#26, count#27] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] +Results [4]: [ss_list_price#3, sum#20, count#21, count#22] (16) Exchange -Input [4]: [ss_list_price#20, sum#25, count#26, count#27] -Arguments: hashpartitioning(ss_list_price#20, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [4]: [ss_list_price#3, sum#20, count#21, count#22] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#23] (17) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#20, sum#25, count#26, count#27] -Keys [1]: [ss_list_price#20] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] -Results [4]: [ss_list_price#20, sum#25, count#26, count#27] +Input [4]: [ss_list_price#3, sum#20, count#21, count#22] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] +Results [4]: [ss_list_price#3, sum#20, count#21, count#22] (18) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#20, sum#25, count#26, count#27] +Input [4]: [ss_list_price#3, sum#20, count#21, count#22] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20), partial_count(distinct ss_list_price#20)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] -Results [4]: [sum#25, count#26, count#27, count#30] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] +Results [4]: [sum#20, count#21, count#22, count#25] (19) Exchange -Input [4]: [sum#25, count#26, count#27, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] +Input [4]: [sum#20, count#21, count#22, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] (20) HashAggregate [codegen id : 5] -Input [4]: [sum#25, count#26, count#27, count#30] +Input [4]: [sum#20, count#21, count#22, count#25] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#20)), count(ss_list_price#20), count(distinct ss_list_price#20)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#20))#23 / 100.0) as decimal(11,6)) AS B2_LP#32, count(ss_list_price#20)#24 AS B2_CNT#33, count(ss_list_price#20)#29 AS B2_CNTD#34] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#18 / 100.0) as decimal(11,6)) AS B2_LP#27, count(ss_list_price#3)#19 AS B2_CNT#28, count(ss_list_price#3)#24 AS B2_CNTD#29] (21) BroadcastExchange -Input [3]: [B2_LP#32, B2_CNT#33, B2_CNTD#34] -Arguments: IdentityBroadcastMode, [id=#35] +Input [3]: [B2_LP#27, B2_CNT#28, B2_CNTD#29] +Arguments: IdentityBroadcastMode, [id=#30] (22) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (23) Scan parquet default.store_sales -Output [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (25) Filter [codegen id : 6] -Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] -Condition : (((isnotnull(ss_quantity#36) AND (ss_quantity#36 >= 11)) AND (ss_quantity#36 <= 15)) AND ((((ss_list_price#38 >= 142.00) AND (ss_list_price#38 <= 152.00)) OR ((ss_coupon_amt#39 >= 12214.00) AND (ss_coupon_amt#39 <= 13214.00))) OR ((ss_wholesale_cost#37 >= 79.00) AND (ss_wholesale_cost#37 <= 99.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 11)) AND (ss_quantity#1 <= 15)) AND ((((ss_list_price#3 >= 142.00) AND (ss_list_price#3 <= 152.00)) OR ((ss_coupon_amt#4 >= 12214.00) AND (ss_coupon_amt#4 <= 13214.00))) OR ((ss_wholesale_cost#2 >= 79.00) AND (ss_wholesale_cost#2 <= 99.00)))) (26) Project [codegen id : 6] -Output [1]: [ss_list_price#38] -Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (27) HashAggregate [codegen id : 6] -Input [1]: [ss_list_price#38] -Keys [1]: [ss_list_price#38] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#38)), partial_count(ss_list_price#38)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] -Results [4]: [ss_list_price#38, sum#43, count#44, count#45] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] +Results [4]: [ss_list_price#3, sum#33, count#34, count#35] (28) Exchange -Input [4]: [ss_list_price#38, sum#43, count#44, count#45] -Arguments: hashpartitioning(ss_list_price#38, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [4]: [ss_list_price#3, sum#33, count#34, count#35] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#36] (29) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#38, sum#43, count#44, count#45] -Keys [1]: [ss_list_price#38] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] -Results [4]: [ss_list_price#38, sum#43, count#44, count#45] +Input [4]: [ss_list_price#3, sum#33, count#34, count#35] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] +Results [4]: [ss_list_price#3, sum#33, count#34, count#35] (30) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#38, sum#43, count#44, count#45] +Input [4]: [ss_list_price#3, sum#33, count#34, count#35] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38), partial_count(distinct ss_list_price#38)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] -Results [4]: [sum#43, count#44, count#45, count#48] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] +Results [4]: [sum#33, count#34, count#35, count#38] (31) Exchange -Input [4]: [sum#43, count#44, count#45, count#48] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [sum#33, count#34, count#35, count#38] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#39] (32) HashAggregate [codegen id : 8] -Input [4]: [sum#43, count#44, count#45, count#48] +Input [4]: [sum#33, count#34, count#35, count#38] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#38)), count(ss_list_price#38), count(distinct ss_list_price#38)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#38))#41 / 100.0) as decimal(11,6)) AS B3_LP#50, count(ss_list_price#38)#42 AS B3_CNT#51, count(ss_list_price#38)#47 AS B3_CNTD#52] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#31 / 100.0) as decimal(11,6)) AS B3_LP#40, count(ss_list_price#3)#32 AS B3_CNT#41, count(ss_list_price#3)#37 AS B3_CNTD#42] (33) BroadcastExchange -Input [3]: [B3_LP#50, B3_CNT#51, B3_CNTD#52] -Arguments: IdentityBroadcastMode, [id=#53] +Input [3]: [B3_LP#40, B3_CNT#41, B3_CNTD#42] +Arguments: IdentityBroadcastMode, [id=#43] (34) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (35) Scan parquet default.store_sales -Output [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] ReadSchema: struct (36) ColumnarToRow [codegen id : 9] -Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (37) Filter [codegen id : 9] -Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] -Condition : (((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 16)) AND (ss_quantity#54 <= 20)) AND ((((ss_list_price#56 >= 135.00) AND (ss_list_price#56 <= 145.00)) OR ((ss_coupon_amt#57 >= 6071.00) AND (ss_coupon_amt#57 <= 7071.00))) OR ((ss_wholesale_cost#55 >= 38.00) AND (ss_wholesale_cost#55 <= 58.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 16)) AND (ss_quantity#1 <= 20)) AND ((((ss_list_price#3 >= 135.00) AND (ss_list_price#3 <= 145.00)) OR ((ss_coupon_amt#4 >= 6071.00) AND (ss_coupon_amt#4 <= 7071.00))) OR ((ss_wholesale_cost#2 >= 38.00) AND (ss_wholesale_cost#2 <= 58.00)))) (38) Project [codegen id : 9] -Output [1]: [ss_list_price#56] -Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (39) HashAggregate [codegen id : 9] -Input [1]: [ss_list_price#56] -Keys [1]: [ss_list_price#56] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#56)), partial_count(ss_list_price#56)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] -Results [4]: [ss_list_price#56, sum#61, count#62, count#63] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] +Results [4]: [ss_list_price#3, sum#46, count#47, count#48] (40) Exchange -Input [4]: [ss_list_price#56, sum#61, count#62, count#63] -Arguments: hashpartitioning(ss_list_price#56, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [4]: [ss_list_price#3, sum#46, count#47, count#48] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#49] (41) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#56, sum#61, count#62, count#63] -Keys [1]: [ss_list_price#56] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] -Results [4]: [ss_list_price#56, sum#61, count#62, count#63] +Input [4]: [ss_list_price#3, sum#46, count#47, count#48] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] +Results [4]: [ss_list_price#3, sum#46, count#47, count#48] (42) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#56, sum#61, count#62, count#63] +Input [4]: [ss_list_price#3, sum#46, count#47, count#48] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56), partial_count(distinct ss_list_price#56)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] -Results [4]: [sum#61, count#62, count#63, count#66] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] +Results [4]: [sum#46, count#47, count#48, count#51] (43) Exchange -Input [4]: [sum#61, count#62, count#63, count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] +Input [4]: [sum#46, count#47, count#48, count#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] (44) HashAggregate [codegen id : 11] -Input [4]: [sum#61, count#62, count#63, count#66] +Input [4]: [sum#46, count#47, count#48, count#51] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#56)), count(ss_list_price#56), count(distinct ss_list_price#56)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#56))#59 / 100.0) as decimal(11,6)) AS B4_LP#68, count(ss_list_price#56)#60 AS B4_CNT#69, count(ss_list_price#56)#65 AS B4_CNTD#70] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#44 / 100.0) as decimal(11,6)) AS B4_LP#53, count(ss_list_price#3)#45 AS B4_CNT#54, count(ss_list_price#3)#50 AS B4_CNTD#55] (45) BroadcastExchange -Input [3]: [B4_LP#68, B4_CNT#69, B4_CNTD#70] -Arguments: IdentityBroadcastMode, [id=#71] +Input [3]: [B4_LP#53, B4_CNT#54, B4_CNTD#55] +Arguments: IdentityBroadcastMode, [id=#56] (46) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (47) Scan parquet default.store_sales -Output [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] ReadSchema: struct (48) ColumnarToRow [codegen id : 12] -Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (49) Filter [codegen id : 12] -Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] -Condition : (((isnotnull(ss_quantity#72) AND (ss_quantity#72 >= 21)) AND (ss_quantity#72 <= 25)) AND ((((ss_list_price#74 >= 122.00) AND (ss_list_price#74 <= 132.00)) OR ((ss_coupon_amt#75 >= 836.00) AND (ss_coupon_amt#75 <= 1836.00))) OR ((ss_wholesale_cost#73 >= 17.00) AND (ss_wholesale_cost#73 <= 37.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 21)) AND (ss_quantity#1 <= 25)) AND ((((ss_list_price#3 >= 122.00) AND (ss_list_price#3 <= 132.00)) OR ((ss_coupon_amt#4 >= 836.00) AND (ss_coupon_amt#4 <= 1836.00))) OR ((ss_wholesale_cost#2 >= 17.00) AND (ss_wholesale_cost#2 <= 37.00)))) (50) Project [codegen id : 12] -Output [1]: [ss_list_price#74] -Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (51) HashAggregate [codegen id : 12] -Input [1]: [ss_list_price#74] -Keys [1]: [ss_list_price#74] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#74)), partial_count(ss_list_price#74)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] -Results [4]: [ss_list_price#74, sum#79, count#80, count#81] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] +Results [4]: [ss_list_price#3, sum#59, count#60, count#61] (52) Exchange -Input [4]: [ss_list_price#74, sum#79, count#80, count#81] -Arguments: hashpartitioning(ss_list_price#74, 5), ENSURE_REQUIREMENTS, [id=#82] +Input [4]: [ss_list_price#3, sum#59, count#60, count#61] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#62] (53) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#74, sum#79, count#80, count#81] -Keys [1]: [ss_list_price#74] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] -Results [4]: [ss_list_price#74, sum#79, count#80, count#81] +Input [4]: [ss_list_price#3, sum#59, count#60, count#61] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] +Results [4]: [ss_list_price#3, sum#59, count#60, count#61] (54) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#74, sum#79, count#80, count#81] +Input [4]: [ss_list_price#3, sum#59, count#60, count#61] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74), partial_count(distinct ss_list_price#74)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] -Results [4]: [sum#79, count#80, count#81, count#84] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] +Results [4]: [sum#59, count#60, count#61, count#64] (55) Exchange -Input [4]: [sum#79, count#80, count#81, count#84] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#85] +Input [4]: [sum#59, count#60, count#61, count#64] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] (56) HashAggregate [codegen id : 14] -Input [4]: [sum#79, count#80, count#81, count#84] +Input [4]: [sum#59, count#60, count#61, count#64] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#74)), count(ss_list_price#74), count(distinct ss_list_price#74)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#74))#77 / 100.0) as decimal(11,6)) AS B5_LP#86, count(ss_list_price#74)#78 AS B5_CNT#87, count(ss_list_price#74)#83 AS B5_CNTD#88] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#57 / 100.0) as decimal(11,6)) AS B5_LP#66, count(ss_list_price#3)#58 AS B5_CNT#67, count(ss_list_price#3)#63 AS B5_CNTD#68] (57) BroadcastExchange -Input [3]: [B5_LP#86, B5_CNT#87, B5_CNTD#88] -Arguments: IdentityBroadcastMode, [id=#89] +Input [3]: [B5_LP#66, B5_CNT#67, B5_CNTD#68] +Arguments: IdentityBroadcastMode, [id=#69] (58) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (59) Scan parquet default.store_sales -Output [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] ReadSchema: struct (60) ColumnarToRow [codegen id : 15] -Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (61) Filter [codegen id : 15] -Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] -Condition : (((isnotnull(ss_quantity#90) AND (ss_quantity#90 >= 26)) AND (ss_quantity#90 <= 30)) AND ((((ss_list_price#92 >= 154.00) AND (ss_list_price#92 <= 164.00)) OR ((ss_coupon_amt#93 >= 7326.00) AND (ss_coupon_amt#93 <= 8326.00))) OR ((ss_wholesale_cost#91 >= 7.00) AND (ss_wholesale_cost#91 <= 27.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 26)) AND (ss_quantity#1 <= 30)) AND ((((ss_list_price#3 >= 154.00) AND (ss_list_price#3 <= 164.00)) OR ((ss_coupon_amt#4 >= 7326.00) AND (ss_coupon_amt#4 <= 8326.00))) OR ((ss_wholesale_cost#2 >= 7.00) AND (ss_wholesale_cost#2 <= 27.00)))) (62) Project [codegen id : 15] -Output [1]: [ss_list_price#92] -Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (63) HashAggregate [codegen id : 15] -Input [1]: [ss_list_price#92] -Keys [1]: [ss_list_price#92] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#92)), partial_count(ss_list_price#92)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] -Results [4]: [ss_list_price#92, sum#97, count#98, count#99] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] +Results [4]: [ss_list_price#3, sum#72, count#73, count#74] (64) Exchange -Input [4]: [ss_list_price#92, sum#97, count#98, count#99] -Arguments: hashpartitioning(ss_list_price#92, 5), ENSURE_REQUIREMENTS, [id=#100] +Input [4]: [ss_list_price#3, sum#72, count#73, count#74] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#75] (65) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#92, sum#97, count#98, count#99] -Keys [1]: [ss_list_price#92] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] -Results [4]: [ss_list_price#92, sum#97, count#98, count#99] +Input [4]: [ss_list_price#3, sum#72, count#73, count#74] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] +Results [4]: [ss_list_price#3, sum#72, count#73, count#74] (66) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#92, sum#97, count#98, count#99] +Input [4]: [ss_list_price#3, sum#72, count#73, count#74] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92), partial_count(distinct ss_list_price#92)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] -Results [4]: [sum#97, count#98, count#99, count#102] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] +Results [4]: [sum#72, count#73, count#74, count#77] (67) Exchange -Input [4]: [sum#97, count#98, count#99, count#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103] +Input [4]: [sum#72, count#73, count#74, count#77] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] (68) HashAggregate [codegen id : 17] -Input [4]: [sum#97, count#98, count#99, count#102] +Input [4]: [sum#72, count#73, count#74, count#77] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#92)), count(ss_list_price#92), count(distinct ss_list_price#92)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#92))#95 / 100.0) as decimal(11,6)) AS B6_LP#104, count(ss_list_price#92)#96 AS B6_CNT#105, count(ss_list_price#92)#101 AS B6_CNTD#106] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#70 / 100.0) as decimal(11,6)) AS B6_LP#79, count(ss_list_price#3)#71 AS B6_CNT#80, count(ss_list_price#3)#76 AS B6_CNTD#81] (69) BroadcastExchange -Input [3]: [B6_LP#104, B6_CNT#105, B6_CNTD#106] -Arguments: IdentityBroadcastMode, [id=#107] +Input [3]: [B6_LP#79, B6_CNT#80, B6_CNTD#81] +Arguments: IdentityBroadcastMode, [id=#82] (70) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt index eec45ea549531..9f15199d8c8b7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt @@ -126,306 +126,306 @@ Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#6 / 100.0) as decimal(11,6)) AS B1_LP#15, count(ss_list_price#3)#7 AS B1_CNT#16, count(ss_list_price#3)#12 AS B1_CNTD#17] (11) Scan parquet default.store_sales -Output [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (13) Filter [codegen id : 3] -Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] -Condition : (((isnotnull(ss_quantity#18) AND (ss_quantity#18 >= 6)) AND (ss_quantity#18 <= 10)) AND ((((ss_list_price#20 >= 90.00) AND (ss_list_price#20 <= 100.00)) OR ((ss_coupon_amt#21 >= 2323.00) AND (ss_coupon_amt#21 <= 3323.00))) OR ((ss_wholesale_cost#19 >= 31.00) AND (ss_wholesale_cost#19 <= 51.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 6)) AND (ss_quantity#1 <= 10)) AND ((((ss_list_price#3 >= 90.00) AND (ss_list_price#3 <= 100.00)) OR ((ss_coupon_amt#4 >= 2323.00) AND (ss_coupon_amt#4 <= 3323.00))) OR ((ss_wholesale_cost#2 >= 31.00) AND (ss_wholesale_cost#2 <= 51.00)))) (14) Project [codegen id : 3] -Output [1]: [ss_list_price#20] -Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (15) HashAggregate [codegen id : 3] -Input [1]: [ss_list_price#20] -Keys [1]: [ss_list_price#20] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#20)), partial_count(ss_list_price#20)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] -Results [4]: [ss_list_price#20, sum#25, count#26, count#27] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] +Results [4]: [ss_list_price#3, sum#20, count#21, count#22] (16) Exchange -Input [4]: [ss_list_price#20, sum#25, count#26, count#27] -Arguments: hashpartitioning(ss_list_price#20, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [4]: [ss_list_price#3, sum#20, count#21, count#22] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#23] (17) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#20, sum#25, count#26, count#27] -Keys [1]: [ss_list_price#20] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] -Results [4]: [ss_list_price#20, sum#25, count#26, count#27] +Input [4]: [ss_list_price#3, sum#20, count#21, count#22] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] +Results [4]: [ss_list_price#3, sum#20, count#21, count#22] (18) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#20, sum#25, count#26, count#27] +Input [4]: [ss_list_price#3, sum#20, count#21, count#22] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20), partial_count(distinct ss_list_price#20)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] -Results [4]: [sum#25, count#26, count#27, count#30] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] +Results [4]: [sum#20, count#21, count#22, count#25] (19) Exchange -Input [4]: [sum#25, count#26, count#27, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] +Input [4]: [sum#20, count#21, count#22, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] (20) HashAggregate [codegen id : 5] -Input [4]: [sum#25, count#26, count#27, count#30] +Input [4]: [sum#20, count#21, count#22, count#25] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#20)), count(ss_list_price#20), count(distinct ss_list_price#20)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#20))#23 / 100.0) as decimal(11,6)) AS B2_LP#32, count(ss_list_price#20)#24 AS B2_CNT#33, count(ss_list_price#20)#29 AS B2_CNTD#34] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#18 / 100.0) as decimal(11,6)) AS B2_LP#27, count(ss_list_price#3)#19 AS B2_CNT#28, count(ss_list_price#3)#24 AS B2_CNTD#29] (21) BroadcastExchange -Input [3]: [B2_LP#32, B2_CNT#33, B2_CNTD#34] -Arguments: IdentityBroadcastMode, [id=#35] +Input [3]: [B2_LP#27, B2_CNT#28, B2_CNTD#29] +Arguments: IdentityBroadcastMode, [id=#30] (22) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (23) Scan parquet default.store_sales -Output [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (25) Filter [codegen id : 6] -Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] -Condition : (((isnotnull(ss_quantity#36) AND (ss_quantity#36 >= 11)) AND (ss_quantity#36 <= 15)) AND ((((ss_list_price#38 >= 142.00) AND (ss_list_price#38 <= 152.00)) OR ((ss_coupon_amt#39 >= 12214.00) AND (ss_coupon_amt#39 <= 13214.00))) OR ((ss_wholesale_cost#37 >= 79.00) AND (ss_wholesale_cost#37 <= 99.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 11)) AND (ss_quantity#1 <= 15)) AND ((((ss_list_price#3 >= 142.00) AND (ss_list_price#3 <= 152.00)) OR ((ss_coupon_amt#4 >= 12214.00) AND (ss_coupon_amt#4 <= 13214.00))) OR ((ss_wholesale_cost#2 >= 79.00) AND (ss_wholesale_cost#2 <= 99.00)))) (26) Project [codegen id : 6] -Output [1]: [ss_list_price#38] -Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (27) HashAggregate [codegen id : 6] -Input [1]: [ss_list_price#38] -Keys [1]: [ss_list_price#38] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#38)), partial_count(ss_list_price#38)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] -Results [4]: [ss_list_price#38, sum#43, count#44, count#45] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] +Results [4]: [ss_list_price#3, sum#33, count#34, count#35] (28) Exchange -Input [4]: [ss_list_price#38, sum#43, count#44, count#45] -Arguments: hashpartitioning(ss_list_price#38, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [4]: [ss_list_price#3, sum#33, count#34, count#35] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#36] (29) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#38, sum#43, count#44, count#45] -Keys [1]: [ss_list_price#38] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] -Results [4]: [ss_list_price#38, sum#43, count#44, count#45] +Input [4]: [ss_list_price#3, sum#33, count#34, count#35] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] +Results [4]: [ss_list_price#3, sum#33, count#34, count#35] (30) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#38, sum#43, count#44, count#45] +Input [4]: [ss_list_price#3, sum#33, count#34, count#35] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38), partial_count(distinct ss_list_price#38)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] -Results [4]: [sum#43, count#44, count#45, count#48] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] +Results [4]: [sum#33, count#34, count#35, count#38] (31) Exchange -Input [4]: [sum#43, count#44, count#45, count#48] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [sum#33, count#34, count#35, count#38] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#39] (32) HashAggregate [codegen id : 8] -Input [4]: [sum#43, count#44, count#45, count#48] +Input [4]: [sum#33, count#34, count#35, count#38] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#38)), count(ss_list_price#38), count(distinct ss_list_price#38)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#38))#41 / 100.0) as decimal(11,6)) AS B3_LP#50, count(ss_list_price#38)#42 AS B3_CNT#51, count(ss_list_price#38)#47 AS B3_CNTD#52] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#31 / 100.0) as decimal(11,6)) AS B3_LP#40, count(ss_list_price#3)#32 AS B3_CNT#41, count(ss_list_price#3)#37 AS B3_CNTD#42] (33) BroadcastExchange -Input [3]: [B3_LP#50, B3_CNT#51, B3_CNTD#52] -Arguments: IdentityBroadcastMode, [id=#53] +Input [3]: [B3_LP#40, B3_CNT#41, B3_CNTD#42] +Arguments: IdentityBroadcastMode, [id=#43] (34) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (35) Scan parquet default.store_sales -Output [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] ReadSchema: struct (36) ColumnarToRow [codegen id : 9] -Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (37) Filter [codegen id : 9] -Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] -Condition : (((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 16)) AND (ss_quantity#54 <= 20)) AND ((((ss_list_price#56 >= 135.00) AND (ss_list_price#56 <= 145.00)) OR ((ss_coupon_amt#57 >= 6071.00) AND (ss_coupon_amt#57 <= 7071.00))) OR ((ss_wholesale_cost#55 >= 38.00) AND (ss_wholesale_cost#55 <= 58.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 16)) AND (ss_quantity#1 <= 20)) AND ((((ss_list_price#3 >= 135.00) AND (ss_list_price#3 <= 145.00)) OR ((ss_coupon_amt#4 >= 6071.00) AND (ss_coupon_amt#4 <= 7071.00))) OR ((ss_wholesale_cost#2 >= 38.00) AND (ss_wholesale_cost#2 <= 58.00)))) (38) Project [codegen id : 9] -Output [1]: [ss_list_price#56] -Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (39) HashAggregate [codegen id : 9] -Input [1]: [ss_list_price#56] -Keys [1]: [ss_list_price#56] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#56)), partial_count(ss_list_price#56)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] -Results [4]: [ss_list_price#56, sum#61, count#62, count#63] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] +Results [4]: [ss_list_price#3, sum#46, count#47, count#48] (40) Exchange -Input [4]: [ss_list_price#56, sum#61, count#62, count#63] -Arguments: hashpartitioning(ss_list_price#56, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [4]: [ss_list_price#3, sum#46, count#47, count#48] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#49] (41) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#56, sum#61, count#62, count#63] -Keys [1]: [ss_list_price#56] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] -Results [4]: [ss_list_price#56, sum#61, count#62, count#63] +Input [4]: [ss_list_price#3, sum#46, count#47, count#48] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] +Results [4]: [ss_list_price#3, sum#46, count#47, count#48] (42) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#56, sum#61, count#62, count#63] +Input [4]: [ss_list_price#3, sum#46, count#47, count#48] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56), partial_count(distinct ss_list_price#56)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] -Results [4]: [sum#61, count#62, count#63, count#66] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] +Results [4]: [sum#46, count#47, count#48, count#51] (43) Exchange -Input [4]: [sum#61, count#62, count#63, count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] +Input [4]: [sum#46, count#47, count#48, count#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] (44) HashAggregate [codegen id : 11] -Input [4]: [sum#61, count#62, count#63, count#66] +Input [4]: [sum#46, count#47, count#48, count#51] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#56)), count(ss_list_price#56), count(distinct ss_list_price#56)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#56))#59 / 100.0) as decimal(11,6)) AS B4_LP#68, count(ss_list_price#56)#60 AS B4_CNT#69, count(ss_list_price#56)#65 AS B4_CNTD#70] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#44 / 100.0) as decimal(11,6)) AS B4_LP#53, count(ss_list_price#3)#45 AS B4_CNT#54, count(ss_list_price#3)#50 AS B4_CNTD#55] (45) BroadcastExchange -Input [3]: [B4_LP#68, B4_CNT#69, B4_CNTD#70] -Arguments: IdentityBroadcastMode, [id=#71] +Input [3]: [B4_LP#53, B4_CNT#54, B4_CNTD#55] +Arguments: IdentityBroadcastMode, [id=#56] (46) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (47) Scan parquet default.store_sales -Output [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] ReadSchema: struct (48) ColumnarToRow [codegen id : 12] -Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (49) Filter [codegen id : 12] -Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] -Condition : (((isnotnull(ss_quantity#72) AND (ss_quantity#72 >= 21)) AND (ss_quantity#72 <= 25)) AND ((((ss_list_price#74 >= 122.00) AND (ss_list_price#74 <= 132.00)) OR ((ss_coupon_amt#75 >= 836.00) AND (ss_coupon_amt#75 <= 1836.00))) OR ((ss_wholesale_cost#73 >= 17.00) AND (ss_wholesale_cost#73 <= 37.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 21)) AND (ss_quantity#1 <= 25)) AND ((((ss_list_price#3 >= 122.00) AND (ss_list_price#3 <= 132.00)) OR ((ss_coupon_amt#4 >= 836.00) AND (ss_coupon_amt#4 <= 1836.00))) OR ((ss_wholesale_cost#2 >= 17.00) AND (ss_wholesale_cost#2 <= 37.00)))) (50) Project [codegen id : 12] -Output [1]: [ss_list_price#74] -Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (51) HashAggregate [codegen id : 12] -Input [1]: [ss_list_price#74] -Keys [1]: [ss_list_price#74] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#74)), partial_count(ss_list_price#74)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] -Results [4]: [ss_list_price#74, sum#79, count#80, count#81] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] +Results [4]: [ss_list_price#3, sum#59, count#60, count#61] (52) Exchange -Input [4]: [ss_list_price#74, sum#79, count#80, count#81] -Arguments: hashpartitioning(ss_list_price#74, 5), ENSURE_REQUIREMENTS, [id=#82] +Input [4]: [ss_list_price#3, sum#59, count#60, count#61] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#62] (53) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#74, sum#79, count#80, count#81] -Keys [1]: [ss_list_price#74] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] -Results [4]: [ss_list_price#74, sum#79, count#80, count#81] +Input [4]: [ss_list_price#3, sum#59, count#60, count#61] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] +Results [4]: [ss_list_price#3, sum#59, count#60, count#61] (54) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#74, sum#79, count#80, count#81] +Input [4]: [ss_list_price#3, sum#59, count#60, count#61] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74), partial_count(distinct ss_list_price#74)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] -Results [4]: [sum#79, count#80, count#81, count#84] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] +Results [4]: [sum#59, count#60, count#61, count#64] (55) Exchange -Input [4]: [sum#79, count#80, count#81, count#84] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#85] +Input [4]: [sum#59, count#60, count#61, count#64] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] (56) HashAggregate [codegen id : 14] -Input [4]: [sum#79, count#80, count#81, count#84] +Input [4]: [sum#59, count#60, count#61, count#64] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#74)), count(ss_list_price#74), count(distinct ss_list_price#74)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#74))#77 / 100.0) as decimal(11,6)) AS B5_LP#86, count(ss_list_price#74)#78 AS B5_CNT#87, count(ss_list_price#74)#83 AS B5_CNTD#88] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#57 / 100.0) as decimal(11,6)) AS B5_LP#66, count(ss_list_price#3)#58 AS B5_CNT#67, count(ss_list_price#3)#63 AS B5_CNTD#68] (57) BroadcastExchange -Input [3]: [B5_LP#86, B5_CNT#87, B5_CNTD#88] -Arguments: IdentityBroadcastMode, [id=#89] +Input [3]: [B5_LP#66, B5_CNT#67, B5_CNTD#68] +Arguments: IdentityBroadcastMode, [id=#69] (58) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (59) Scan parquet default.store_sales -Output [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] ReadSchema: struct (60) ColumnarToRow [codegen id : 15] -Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (61) Filter [codegen id : 15] -Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] -Condition : (((isnotnull(ss_quantity#90) AND (ss_quantity#90 >= 26)) AND (ss_quantity#90 <= 30)) AND ((((ss_list_price#92 >= 154.00) AND (ss_list_price#92 <= 164.00)) OR ((ss_coupon_amt#93 >= 7326.00) AND (ss_coupon_amt#93 <= 8326.00))) OR ((ss_wholesale_cost#91 >= 7.00) AND (ss_wholesale_cost#91 <= 27.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 26)) AND (ss_quantity#1 <= 30)) AND ((((ss_list_price#3 >= 154.00) AND (ss_list_price#3 <= 164.00)) OR ((ss_coupon_amt#4 >= 7326.00) AND (ss_coupon_amt#4 <= 8326.00))) OR ((ss_wholesale_cost#2 >= 7.00) AND (ss_wholesale_cost#2 <= 27.00)))) (62) Project [codegen id : 15] -Output [1]: [ss_list_price#92] -Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (63) HashAggregate [codegen id : 15] -Input [1]: [ss_list_price#92] -Keys [1]: [ss_list_price#92] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#92)), partial_count(ss_list_price#92)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] -Results [4]: [ss_list_price#92, sum#97, count#98, count#99] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] +Results [4]: [ss_list_price#3, sum#72, count#73, count#74] (64) Exchange -Input [4]: [ss_list_price#92, sum#97, count#98, count#99] -Arguments: hashpartitioning(ss_list_price#92, 5), ENSURE_REQUIREMENTS, [id=#100] +Input [4]: [ss_list_price#3, sum#72, count#73, count#74] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#75] (65) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#92, sum#97, count#98, count#99] -Keys [1]: [ss_list_price#92] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] -Results [4]: [ss_list_price#92, sum#97, count#98, count#99] +Input [4]: [ss_list_price#3, sum#72, count#73, count#74] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] +Results [4]: [ss_list_price#3, sum#72, count#73, count#74] (66) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#92, sum#97, count#98, count#99] +Input [4]: [ss_list_price#3, sum#72, count#73, count#74] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92), partial_count(distinct ss_list_price#92)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] -Results [4]: [sum#97, count#98, count#99, count#102] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] +Results [4]: [sum#72, count#73, count#74, count#77] (67) Exchange -Input [4]: [sum#97, count#98, count#99, count#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103] +Input [4]: [sum#72, count#73, count#74, count#77] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] (68) HashAggregate [codegen id : 17] -Input [4]: [sum#97, count#98, count#99, count#102] +Input [4]: [sum#72, count#73, count#74, count#77] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#92)), count(ss_list_price#92), count(distinct ss_list_price#92)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#92))#95 / 100.0) as decimal(11,6)) AS B6_LP#104, count(ss_list_price#92)#96 AS B6_CNT#105, count(ss_list_price#92)#101 AS B6_CNTD#106] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#70 / 100.0) as decimal(11,6)) AS B6_LP#79, count(ss_list_price#3)#71 AS B6_CNT#80, count(ss_list_price#3)#76 AS B6_CNTD#81] (69) BroadcastExchange -Input [3]: [B6_LP#104, B6_CNT#105, B6_CNTD#106] -Arguments: IdentityBroadcastMode, [id=#107] +Input [3]: [B6_LP#79, B6_CNT#80, B6_CNTD#81] +Arguments: IdentityBroadcastMode, [id=#82] (70) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None 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 9e84e385be302..ca09ec6ea1a42 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 @@ -168,170 +168,170 @@ Input [3]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19] Condition : isnotnull(ctr_total_return#19) (24) Scan parquet default.web_returns -Output [4]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23] +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#23), dynamicpruningexpression(wr_returned_date_sk#23 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 (25) ColumnarToRow [codegen id : 8] -Input [4]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23] +Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] (26) Filter [codegen id : 8] -Input [4]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23] -Condition : isnotnull(wr_returning_addr_sk#21) +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) (27) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#24] +Output [1]: [d_date_sk#6] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#24 as bigint)] +Left keys [1]: [wr_returned_date_sk#4] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (29) Project [codegen id : 8] -Output [3]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22] -Input [5]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23, d_date_sk#24] +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] (30) Exchange -Input [3]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22] -Arguments: hashpartitioning(wr_returning_addr_sk#21, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] +Arguments: hashpartitioning(wr_returning_addr_sk#2, 5), ENSURE_REQUIREMENTS, [id=#20] (31) Sort [codegen id : 9] -Input [3]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22] -Arguments: [wr_returning_addr_sk#21 ASC NULLS FIRST], false, 0 +Input [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] +Arguments: [wr_returning_addr_sk#2 ASC NULLS FIRST], false, 0 (32) ReusedExchange [Reuses operator id: 16] -Output [2]: [ca_address_sk#26, ca_state#27] +Output [2]: [ca_address_sk#10, ca_state#11] (33) Sort [codegen id : 11] -Input [2]: [ca_address_sk#26, ca_state#27] -Arguments: [cast(ca_address_sk#26 as bigint) ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#10, ca_state#11] +Arguments: [cast(ca_address_sk#10 as bigint) ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 12] -Left keys [1]: [wr_returning_addr_sk#21] -Right keys [1]: [cast(ca_address_sk#26 as bigint)] +Left keys [1]: [wr_returning_addr_sk#2] +Right keys [1]: [cast(ca_address_sk#10 as bigint)] Join condition: None (35) Project [codegen id : 12] -Output [3]: [wr_returning_customer_sk#20, wr_return_amt#22, ca_state#27] -Input [5]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, ca_address_sk#26, ca_state#27] +Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#11] +Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#10, ca_state#11] (36) HashAggregate [codegen id : 12] -Input [3]: [wr_returning_customer_sk#20, wr_return_amt#22, ca_state#27] -Keys [2]: [wr_returning_customer_sk#20, ca_state#27] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#22))] -Aggregate Attributes [1]: [sum#28] -Results [3]: [wr_returning_customer_sk#20, ca_state#27, sum#29] +Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#11] +Keys [2]: [wr_returning_customer_sk#1, ca_state#11] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] +Aggregate Attributes [1]: [sum#21] +Results [3]: [wr_returning_customer_sk#1, ca_state#11, sum#22] (37) Exchange -Input [3]: [wr_returning_customer_sk#20, ca_state#27, sum#29] -Arguments: hashpartitioning(wr_returning_customer_sk#20, ca_state#27, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [wr_returning_customer_sk#1, ca_state#11, sum#22] +Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#11, 5), ENSURE_REQUIREMENTS, [id=#23] (38) HashAggregate [codegen id : 13] -Input [3]: [wr_returning_customer_sk#20, ca_state#27, sum#29] -Keys [2]: [wr_returning_customer_sk#20, ca_state#27] -Functions [1]: [sum(UnscaledValue(wr_return_amt#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#22))#31] -Results [2]: [ca_state#27 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(wr_return_amt#22))#31,17,2) AS ctr_total_return#19] +Input [3]: [wr_returning_customer_sk#1, ca_state#11, sum#22] +Keys [2]: [wr_returning_customer_sk#1, ca_state#11] +Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#24] +Results [2]: [ca_state#11 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#24,17,2) AS ctr_total_return#19] (39) HashAggregate [codegen id : 13] Input [2]: [ctr_state#18, ctr_total_return#19] Keys [1]: [ctr_state#18] Functions [1]: [partial_avg(ctr_total_return#19)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [3]: [ctr_state#18, sum#34, count#35] +Aggregate Attributes [2]: [sum#25, count#26] +Results [3]: [ctr_state#18, sum#27, count#28] (40) Exchange -Input [3]: [ctr_state#18, sum#34, count#35] -Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ctr_state#18, sum#27, count#28] +Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#29] (41) HashAggregate [codegen id : 14] -Input [3]: [ctr_state#18, sum#34, count#35] +Input [3]: [ctr_state#18, sum#27, count#28] Keys [1]: [ctr_state#18] Functions [1]: [avg(ctr_total_return#19)] -Aggregate Attributes [1]: [avg(ctr_total_return#19)#37] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#37) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#38, ctr_state#18 AS ctr_state#18#39] +Aggregate Attributes [1]: [avg(ctr_total_return#19)#30] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#18 AS ctr_state#18#32] (42) Filter [codegen id : 14] -Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#38) +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) (43) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#40] +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#33] (44) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ctr_state#18] -Right keys [1]: [ctr_state#18#39] -Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#38) +Right keys [1]: [ctr_state#18#32] +Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) (45) Project [codegen id : 17] Output [2]: [ctr_customer_sk#17, ctr_total_return#19] -Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] +Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] (46) Scan parquet default.customer -Output [14]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] +Output [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 16] -Input [14]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] +Input [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] (48) Filter [codegen id : 16] -Input [14]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] -Condition : (isnotnull(c_customer_sk#41) AND isnotnull(c_current_addr_sk#43)) +Input [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Condition : (isnotnull(c_customer_sk#34) AND isnotnull(c_current_addr_sk#36)) (49) Scan parquet default.customer_address -Output [2]: [ca_address_sk#55, ca_state#56] +Output [2]: [ca_address_sk#10, ca_state#11] 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 (50) ColumnarToRow [codegen id : 15] -Input [2]: [ca_address_sk#55, ca_state#56] +Input [2]: [ca_address_sk#10, ca_state#11] (51) Filter [codegen id : 15] -Input [2]: [ca_address_sk#55, ca_state#56] -Condition : ((isnotnull(ca_state#56) AND (ca_state#56 = GA)) AND isnotnull(ca_address_sk#55)) +Input [2]: [ca_address_sk#10, ca_state#11] +Condition : ((isnotnull(ca_state#11) AND (ca_state#11 = GA)) AND isnotnull(ca_address_sk#10)) (52) Project [codegen id : 15] -Output [1]: [ca_address_sk#55] -Input [2]: [ca_address_sk#55, ca_state#56] +Output [1]: [ca_address_sk#10] +Input [2]: [ca_address_sk#10, ca_state#11] (53) BroadcastExchange -Input [1]: [ca_address_sk#55] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#57] +Input [1]: [ca_address_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] (54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_addr_sk#43] -Right keys [1]: [ca_address_sk#55] +Left keys [1]: [c_current_addr_sk#36] +Right keys [1]: [ca_address_sk#10] Join condition: None (55) Project [codegen id : 16] -Output [13]: [c_customer_sk#41, c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] -Input [15]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ca_address_sk#55] +Output [13]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Input [15]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ca_address_sk#10] (56) BroadcastExchange -Input [13]: [c_customer_sk#41, c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] +Input [13]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] (57) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ctr_customer_sk#17] -Right keys [1]: [cast(c_customer_sk#41 as bigint)] +Right keys [1]: [cast(c_customer_sk#34 as bigint)] Join condition: None (58) Project [codegen id : 17] -Output [13]: [c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ctr_total_return#19] -Input [15]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#41, c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] +Output [13]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#19] +Input [15]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] (59) TakeOrderedAndProject -Input [13]: [c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ctr_total_return#19] -Arguments: 100, [c_customer_id#42 ASC NULLS FIRST, c_salutation#44 ASC NULLS FIRST, c_first_name#45 ASC NULLS FIRST, c_last_name#46 ASC NULLS FIRST, c_preferred_cust_flag#47 ASC NULLS FIRST, c_birth_day#48 ASC NULLS FIRST, c_birth_month#49 ASC NULLS FIRST, c_birth_year#50 ASC NULLS FIRST, c_birth_country#51 ASC NULLS FIRST, c_login#52 ASC NULLS FIRST, c_email_address#53 ASC NULLS FIRST, c_last_review_date#54 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ctr_total_return#19] +Input [13]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#19] +Arguments: 100, [c_customer_id#35 ASC NULLS FIRST, c_salutation#37 ASC NULLS FIRST, c_first_name#38 ASC NULLS FIRST, c_last_name#39 ASC NULLS FIRST, c_preferred_cust_flag#40 ASC NULLS FIRST, c_birth_day#41 ASC NULLS FIRST, c_birth_month#42 ASC NULLS FIRST, c_birth_year#43 ASC NULLS FIRST, c_birth_country#44 ASC NULLS FIRST, c_login#45 ASC NULLS FIRST, c_email_address#46 ASC NULLS FIRST, c_last_review_date#47 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#19] ===== Subqueries ===== @@ -342,6 +342,6 @@ ReusedExchange (60) (60) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 24 Hosting Expression = wr_returned_date_sk#23 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 24 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/q30/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt index 1aaebad9f1920..57b85bc9db340 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 @@ -150,158 +150,158 @@ Input [3]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18] Condition : isnotnull(ctr_total_return#18) (21) Scan parquet default.web_returns -Output [4]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, wr_returned_date_sk#22] +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#22), dynamicpruningexpression(wr_returned_date_sk#22 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 (22) ColumnarToRow [codegen id : 6] -Input [4]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, wr_returned_date_sk#22] +Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] (23) Filter [codegen id : 6] -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) +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) (24) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#23] +Output [1]: [d_date_sk#6] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returned_date_sk#22] -Right keys [1]: [cast(d_date_sk#23 as bigint)] +Left keys [1]: [wr_returned_date_sk#4] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (26) Project [codegen id : 6] -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#23] +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] (27) ReusedExchange [Reuses operator id: 14] -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#9, ca_state#10] (28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returning_addr_sk#20] -Right keys [1]: [cast(ca_address_sk#24 as bigint)] +Left keys [1]: [wr_returning_addr_sk#2] +Right keys [1]: [cast(ca_address_sk#9 as bigint)] Join condition: None (29) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#19, wr_return_amt#21, ca_state#25] -Input [5]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, ca_address_sk#24, ca_state#25] +Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] +Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#9, ca_state#10] (30) HashAggregate [codegen id : 6] -Input [3]: [wr_returning_customer_sk#19, wr_return_amt#21, ca_state#25] -Keys [2]: [wr_returning_customer_sk#19, ca_state#25] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#21))] -Aggregate Attributes [1]: [sum#26] -Results [3]: [wr_returning_customer_sk#19, ca_state#25, sum#27] +Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] +Keys [2]: [wr_returning_customer_sk#1, ca_state#10] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] +Aggregate Attributes [1]: [sum#19] +Results [3]: [wr_returning_customer_sk#1, ca_state#10, sum#20] (31) Exchange -Input [3]: [wr_returning_customer_sk#19, ca_state#25, sum#27] -Arguments: hashpartitioning(wr_returning_customer_sk#19, ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#20] +Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#21] (32) HashAggregate [codegen id : 7] -Input [3]: [wr_returning_customer_sk#19, ca_state#25, sum#27] -Keys [2]: [wr_returning_customer_sk#19, ca_state#25] -Functions [1]: [sum(UnscaledValue(wr_return_amt#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#21))#29] -Results [2]: [ca_state#25 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(wr_return_amt#21))#29,17,2) AS ctr_total_return#18] +Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#20] +Keys [2]: [wr_returning_customer_sk#1, ca_state#10] +Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#22] +Results [2]: [ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#22,17,2) AS ctr_total_return#18] (33) HashAggregate [codegen id : 7] Input [2]: [ctr_state#17, ctr_total_return#18] Keys [1]: [ctr_state#17] Functions [1]: [partial_avg(ctr_total_return#18)] -Aggregate Attributes [2]: [sum#30, count#31] -Results [3]: [ctr_state#17, sum#32, count#33] +Aggregate Attributes [2]: [sum#23, count#24] +Results [3]: [ctr_state#17, sum#25, count#26] (34) Exchange -Input [3]: [ctr_state#17, sum#32, count#33] -Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [ctr_state#17, sum#25, count#26] +Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#27] (35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#17, sum#32, count#33] +Input [3]: [ctr_state#17, sum#25, count#26] Keys [1]: [ctr_state#17] Functions [1]: [avg(ctr_total_return#18)] -Aggregate Attributes [1]: [avg(ctr_total_return#18)#35] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#35) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#36, ctr_state#17 AS ctr_state#17#37] +Aggregate Attributes [1]: [avg(ctr_total_return#18)#28] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#28) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#29, ctr_state#17 AS ctr_state#17#30] (36) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#36) +Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) (37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#38] +Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#31] (38) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#17] -Right keys [1]: [ctr_state#17#37] -Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#36) +Right keys [1]: [ctr_state#17#30] +Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) (39) Project [codegen id : 11] Output [2]: [ctr_customer_sk#16, ctr_total_return#18] -Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] +Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] (40) Scan parquet default.customer -Output [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] +Output [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] +Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] (42) Filter [codegen id : 9] -Input [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] -Condition : (isnotnull(c_customer_sk#39) AND isnotnull(c_current_addr_sk#41)) +Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34)) (43) BroadcastExchange -Input [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] +Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#46] (44) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#16] -Right keys [1]: [cast(c_customer_sk#39 as bigint)] +Right keys [1]: [cast(c_customer_sk#32 as bigint)] Join condition: None (45) Project [codegen id : 11] -Output [14]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] -Input [16]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] +Output [14]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +Input [16]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] (46) Scan parquet default.customer_address -Output [2]: [ca_address_sk#54, ca_state#55] +Output [2]: [ca_address_sk#9, ca_state#10] 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 (47) ColumnarToRow [codegen id : 10] -Input [2]: [ca_address_sk#54, ca_state#55] +Input [2]: [ca_address_sk#9, ca_state#10] (48) Filter [codegen id : 10] -Input [2]: [ca_address_sk#54, ca_state#55] -Condition : ((isnotnull(ca_state#55) AND (ca_state#55 = GA)) AND isnotnull(ca_address_sk#54)) +Input [2]: [ca_address_sk#9, ca_state#10] +Condition : ((isnotnull(ca_state#10) AND (ca_state#10 = GA)) AND isnotnull(ca_address_sk#9)) (49) Project [codegen id : 10] -Output [1]: [ca_address_sk#54] -Input [2]: [ca_address_sk#54, ca_state#55] +Output [1]: [ca_address_sk#9] +Input [2]: [ca_address_sk#9, ca_state#10] (50) BroadcastExchange -Input [1]: [ca_address_sk#54] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +Input [1]: [ca_address_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] (51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#41] -Right keys [1]: [ca_address_sk#54] +Left keys [1]: [c_current_addr_sk#34] +Right keys [1]: [ca_address_sk#9] Join condition: None (52) Project [codegen id : 11] -Output [13]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ctr_total_return#18] -Input [15]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ca_address_sk#54] +Output [13]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#18] +Input [15]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ca_address_sk#9] (53) TakeOrderedAndProject -Input [13]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ctr_total_return#18] -Arguments: 100, [c_customer_id#40 ASC NULLS FIRST, c_salutation#42 ASC NULLS FIRST, c_first_name#43 ASC NULLS FIRST, c_last_name#44 ASC NULLS FIRST, c_preferred_cust_flag#45 ASC NULLS FIRST, c_birth_day#46 ASC NULLS FIRST, c_birth_month#47 ASC NULLS FIRST, c_birth_year#48 ASC NULLS FIRST, c_birth_country#49 ASC NULLS FIRST, c_login#50 ASC NULLS FIRST, c_email_address#51 ASC NULLS FIRST, c_last_review_date#52 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ctr_total_return#18] +Input [13]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#18] +Arguments: 100, [c_customer_id#33 ASC NULLS FIRST, c_salutation#35 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, c_last_name#37 ASC NULLS FIRST, c_preferred_cust_flag#38 ASC NULLS FIRST, c_birth_day#39 ASC NULLS FIRST, c_birth_month#40 ASC NULLS FIRST, c_birth_year#41 ASC NULLS FIRST, c_birth_country#42 ASC NULLS FIRST, c_login#43 ASC NULLS FIRST, c_email_address#44 ASC NULLS FIRST, c_last_review_date#45 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#18] ===== Subqueries ===== @@ -312,6 +312,6 @@ ReusedExchange (54) (54) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 21 Hosting Expression = wr_returned_date_sk#22 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 21 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 1703da84e7914..61aa5ea5e7e5c 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 @@ -220,452 +220,452 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#16] Results [2]: [ca_county#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#16,17,2) AS store_sales#17] (22) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20] +Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#18)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 8] -Input [3]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20] +Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] (24) Filter [codegen id : 8] -Input [3]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20] -Condition : isnotnull(ss_addr_sk#18) +Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_addr_sk#1) (25) Scan parquet default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_qoy#24] +Output [3]: [d_date_sk#19, d_year#20, d_qoy#21] 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 (26) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#22, d_year#23, d_qoy#24] +Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] (27) Filter [codegen id : 7] -Input [3]: [d_date_sk#22, d_year#23, d_qoy#24] -Condition : ((((isnotnull(d_qoy#24) AND isnotnull(d_year#23)) AND (d_qoy#24 = 3)) AND (d_year#23 = 2000)) AND isnotnull(d_date_sk#22)) +Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] +Condition : ((((isnotnull(d_qoy#21) AND isnotnull(d_year#20)) AND (d_qoy#21 = 3)) AND (d_year#20 = 2000)) AND isnotnull(d_date_sk#19)) (28) BroadcastExchange -Input [3]: [d_date_sk#22, d_year#23, d_qoy#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] +Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] (29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#19] Join condition: None (30) Project [codegen id : 8] -Output [4]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24] -Input [6]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23, d_qoy#24] +Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21] +Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#19, d_year#20, d_qoy#21] (31) Exchange -Input [4]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24] -Arguments: hashpartitioning(ss_addr_sk#18, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21] +Arguments: hashpartitioning(ss_addr_sk#1, 5), ENSURE_REQUIREMENTS, [id=#23] (32) Sort [codegen id : 9] -Input [4]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24] -Arguments: [ss_addr_sk#18 ASC NULLS FIRST], false, 0 +Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21] +Arguments: [ss_addr_sk#1 ASC NULLS FIRST], false, 0 (33) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#27, ca_county#28] +Output [2]: [ca_address_sk#24, ca_county#25] (34) Sort [codegen id : 11] -Input [2]: [ca_address_sk#27, ca_county#28] -Arguments: [ca_address_sk#27 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#24, ca_county#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 (35) SortMergeJoin [codegen id : 12] -Left keys [1]: [ss_addr_sk#18] -Right keys [1]: [ca_address_sk#27] +Left keys [1]: [ss_addr_sk#1] +Right keys [1]: [ca_address_sk#24] Join condition: None (36) Project [codegen id : 12] -Output [4]: [ss_ext_sales_price#19, d_year#23, d_qoy#24, ca_county#28] -Input [6]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24, ca_address_sk#27, ca_county#28] +Output [4]: [ss_ext_sales_price#2, d_year#20, d_qoy#21, ca_county#25] +Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21, ca_address_sk#24, ca_county#25] (37) HashAggregate [codegen id : 12] -Input [4]: [ss_ext_sales_price#19, d_year#23, d_qoy#24, ca_county#28] -Keys [3]: [ca_county#28, d_qoy#24, d_year#23] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#19))] -Aggregate Attributes [1]: [sum#29] -Results [4]: [ca_county#28, d_qoy#24, d_year#23, sum#30] +Input [4]: [ss_ext_sales_price#2, d_year#20, d_qoy#21, ca_county#25] +Keys [3]: [ca_county#25, d_qoy#21, d_year#20] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#26] +Results [4]: [ca_county#25, d_qoy#21, d_year#20, sum#27] (38) Exchange -Input [4]: [ca_county#28, d_qoy#24, d_year#23, sum#30] -Arguments: hashpartitioning(ca_county#28, d_qoy#24, d_year#23, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [4]: [ca_county#25, d_qoy#21, d_year#20, sum#27] +Arguments: hashpartitioning(ca_county#25, d_qoy#21, d_year#20, 5), ENSURE_REQUIREMENTS, [id=#28] (39) HashAggregate [codegen id : 13] -Input [4]: [ca_county#28, d_qoy#24, d_year#23, sum#30] -Keys [3]: [ca_county#28, d_qoy#24, d_year#23] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#19))#32] -Results [2]: [ca_county#28, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#19))#32,17,2) AS store_sales#33] +Input [4]: [ca_county#25, d_qoy#21, d_year#20, sum#27] +Keys [3]: [ca_county#25, d_qoy#21, d_year#20] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#29] +Results [2]: [ca_county#25, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#29,17,2) AS store_sales#30] (40) BroadcastExchange -Input [2]: [ca_county#28, store_sales#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] +Input [2]: [ca_county#25, store_sales#30] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#31] (41) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ca_county#11] -Right keys [1]: [ca_county#28] +Right keys [1]: [ca_county#25] Join condition: None (42) Project [codegen id : 42] -Output [3]: [ca_county#11, store_sales#17, store_sales#33] -Input [4]: [ca_county#11, store_sales#17, ca_county#28, store_sales#33] +Output [3]: [ca_county#11, store_sales#17, store_sales#30] +Input [4]: [ca_county#11, store_sales#17, ca_county#25, store_sales#30] (43) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37] +Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#37), dynamicpruningexpression(ss_sold_date_sk#37 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 15] -Input [3]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37] +Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] (45) Filter [codegen id : 15] -Input [3]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37] -Condition : isnotnull(ss_addr_sk#35) +Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_addr_sk#1) (46) Scan parquet default.date_dim -Output [3]: [d_date_sk#39, d_year#40, d_qoy#41] +Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] 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 (47) ColumnarToRow [codegen id : 14] -Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] +Input [3]: [d_date_sk#33, d_year#34, d_qoy#35] (48) Filter [codegen id : 14] -Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] -Condition : ((((isnotnull(d_qoy#41) AND isnotnull(d_year#40)) AND (d_qoy#41 = 1)) AND (d_year#40 = 2000)) AND isnotnull(d_date_sk#39)) +Input [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Condition : ((((isnotnull(d_qoy#35) AND isnotnull(d_year#34)) AND (d_qoy#35 = 1)) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) (49) BroadcastExchange -Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#42] +Input [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] (50) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_sold_date_sk#37] -Right keys [1]: [d_date_sk#39] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#33] Join condition: None (51) Project [codegen id : 15] -Output [4]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41] -Input [6]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37, d_date_sk#39, d_year#40, d_qoy#41] +Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35] +Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#33, d_year#34, d_qoy#35] (52) Exchange -Input [4]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41] -Arguments: hashpartitioning(ss_addr_sk#35, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35] +Arguments: hashpartitioning(ss_addr_sk#1, 5), ENSURE_REQUIREMENTS, [id=#37] (53) Sort [codegen id : 16] -Input [4]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41] -Arguments: [ss_addr_sk#35 ASC NULLS FIRST], false, 0 +Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35] +Arguments: [ss_addr_sk#1 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#44, ca_county#45] +Output [2]: [ca_address_sk#38, ca_county#39] (55) Sort [codegen id : 18] -Input [2]: [ca_address_sk#44, ca_county#45] -Arguments: [ca_address_sk#44 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#38, ca_county#39] +Arguments: [ca_address_sk#38 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 19] -Left keys [1]: [ss_addr_sk#35] -Right keys [1]: [ca_address_sk#44] +Left keys [1]: [ss_addr_sk#1] +Right keys [1]: [ca_address_sk#38] Join condition: None (57) Project [codegen id : 19] -Output [4]: [ss_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#45] -Input [6]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41, ca_address_sk#44, ca_county#45] +Output [4]: [ss_ext_sales_price#2, d_year#34, d_qoy#35, ca_county#39] +Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35, ca_address_sk#38, ca_county#39] (58) HashAggregate [codegen id : 19] -Input [4]: [ss_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#45] -Keys [3]: [ca_county#45, d_qoy#41, d_year#40] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#36))] -Aggregate Attributes [1]: [sum#46] -Results [4]: [ca_county#45, d_qoy#41, d_year#40, sum#47] +Input [4]: [ss_ext_sales_price#2, d_year#34, d_qoy#35, ca_county#39] +Keys [3]: [ca_county#39, d_qoy#35, d_year#34] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#40] +Results [4]: [ca_county#39, d_qoy#35, d_year#34, sum#41] (59) Exchange -Input [4]: [ca_county#45, d_qoy#41, d_year#40, sum#47] -Arguments: hashpartitioning(ca_county#45, d_qoy#41, d_year#40, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [4]: [ca_county#39, d_qoy#35, d_year#34, sum#41] +Arguments: hashpartitioning(ca_county#39, d_qoy#35, d_year#34, 5), ENSURE_REQUIREMENTS, [id=#42] (60) HashAggregate [codegen id : 20] -Input [4]: [ca_county#45, d_qoy#41, d_year#40, sum#47] -Keys [3]: [ca_county#45, d_qoy#41, d_year#40] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#36))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#36))#49] -Results [3]: [ca_county#45, d_year#40, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#36))#49,17,2) AS store_sales#50] +Input [4]: [ca_county#39, d_qoy#35, d_year#34, sum#41] +Keys [3]: [ca_county#39, d_qoy#35, d_year#34] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#43] +Results [3]: [ca_county#39, d_year#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#43,17,2) AS store_sales#44] (61) BroadcastExchange -Input [3]: [ca_county#45, d_year#40, store_sales#50] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#51] +Input [3]: [ca_county#39, d_year#34, store_sales#44] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#45] (62) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ca_county#11] -Right keys [1]: [ca_county#45] +Right keys [1]: [ca_county#39] Join condition: None (63) Project [codegen id : 42] -Output [5]: [store_sales#17, store_sales#33, ca_county#45, d_year#40, store_sales#50] -Input [6]: [ca_county#11, store_sales#17, store_sales#33, ca_county#45, d_year#40, store_sales#50] +Output [5]: [store_sales#17, store_sales#30, ca_county#39, d_year#34, store_sales#44] +Input [6]: [ca_county#11, store_sales#17, store_sales#30, ca_county#39, d_year#34, store_sales#44] (64) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, ws_sold_date_sk#54] +Output [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, ws_sold_date_sk#54] +Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] (66) Filter [codegen id : 22] -Input [3]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, ws_sold_date_sk#54] -Condition : isnotnull(ws_bill_addr_sk#52) +Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Condition : isnotnull(ws_bill_addr_sk#46) (67) ReusedExchange [Reuses operator id: 49] -Output [3]: [d_date_sk#55, d_year#56, d_qoy#57] +Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] (68) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#54] -Right keys [1]: [d_date_sk#55] +Left keys [1]: [ws_sold_date_sk#48] +Right keys [1]: [d_date_sk#49] Join condition: None (69) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57] -Input [6]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, ws_sold_date_sk#54, d_date_sk#55, d_year#56, d_qoy#57] +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] (70) Exchange -Input [4]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57] -Arguments: hashpartitioning(ws_bill_addr_sk#52, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51] +Arguments: hashpartitioning(ws_bill_addr_sk#46, 5), ENSURE_REQUIREMENTS, [id=#52] (71) Sort [codegen id : 23] -Input [4]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57] -Arguments: [ws_bill_addr_sk#52 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51] +Arguments: [ws_bill_addr_sk#46 ASC NULLS FIRST], false, 0 (72) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#59, ca_county#60] +Output [2]: [ca_address_sk#53, ca_county#54] (73) Sort [codegen id : 25] -Input [2]: [ca_address_sk#59, ca_county#60] -Arguments: [ca_address_sk#59 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#53, ca_county#54] +Arguments: [ca_address_sk#53 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 26] -Left keys [1]: [ws_bill_addr_sk#52] -Right keys [1]: [ca_address_sk#59] +Left keys [1]: [ws_bill_addr_sk#46] +Right keys [1]: [ca_address_sk#53] Join condition: None (75) Project [codegen id : 26] -Output [4]: [ws_ext_sales_price#53, d_year#56, d_qoy#57, ca_county#60] -Input [6]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57, ca_address_sk#59, ca_county#60] +Output [4]: [ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_county#54] +Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_address_sk#53, ca_county#54] (76) HashAggregate [codegen id : 26] -Input [4]: [ws_ext_sales_price#53, d_year#56, d_qoy#57, ca_county#60] -Keys [3]: [ca_county#60, d_qoy#57, d_year#56] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#53))] -Aggregate Attributes [1]: [sum#61] -Results [4]: [ca_county#60, d_qoy#57, d_year#56, sum#62] +Input [4]: [ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_county#54] +Keys [3]: [ca_county#54, d_qoy#51, d_year#50] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#47))] +Aggregate Attributes [1]: [sum#55] +Results [4]: [ca_county#54, d_qoy#51, d_year#50, sum#56] (77) Exchange -Input [4]: [ca_county#60, d_qoy#57, d_year#56, sum#62] -Arguments: hashpartitioning(ca_county#60, d_qoy#57, d_year#56, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [4]: [ca_county#54, d_qoy#51, d_year#50, sum#56] +Arguments: hashpartitioning(ca_county#54, d_qoy#51, d_year#50, 5), ENSURE_REQUIREMENTS, [id=#57] (78) HashAggregate [codegen id : 41] -Input [4]: [ca_county#60, d_qoy#57, d_year#56, sum#62] -Keys [3]: [ca_county#60, d_qoy#57, d_year#56] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#53))#64] -Results [2]: [ca_county#60, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#53))#64,17,2) AS web_sales#65] +Input [4]: [ca_county#54, d_qoy#51, d_year#50, sum#56] +Keys [3]: [ca_county#54, 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))#58] +Results [2]: [ca_county#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#47))#58,17,2) AS web_sales#59] (79) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, ws_sold_date_sk#68] +Output [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (80) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, ws_sold_date_sk#68] +Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] (81) Filter [codegen id : 28] -Input [3]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, ws_sold_date_sk#68] -Condition : isnotnull(ws_bill_addr_sk#66) +Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Condition : isnotnull(ws_bill_addr_sk#46) (82) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#69, d_year#70, d_qoy#71] +Output [3]: [d_date_sk#60, d_year#61, d_qoy#62] (83) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#68] -Right keys [1]: [d_date_sk#69] +Left keys [1]: [ws_sold_date_sk#48] +Right keys [1]: [d_date_sk#60] Join condition: None (84) Project [codegen id : 28] -Output [4]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71] -Input [6]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, ws_sold_date_sk#68, d_date_sk#69, d_year#70, d_qoy#71] +Output [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62] +Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48, d_date_sk#60, d_year#61, d_qoy#62] (85) Exchange -Input [4]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71] -Arguments: hashpartitioning(ws_bill_addr_sk#66, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62] +Arguments: hashpartitioning(ws_bill_addr_sk#46, 5), ENSURE_REQUIREMENTS, [id=#63] (86) Sort [codegen id : 29] -Input [4]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71] -Arguments: [ws_bill_addr_sk#66 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62] +Arguments: [ws_bill_addr_sk#46 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#73, ca_county#74] +Output [2]: [ca_address_sk#64, ca_county#65] (88) Sort [codegen id : 31] -Input [2]: [ca_address_sk#73, ca_county#74] -Arguments: [ca_address_sk#73 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#64, ca_county#65] +Arguments: [ca_address_sk#64 ASC NULLS FIRST], false, 0 (89) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_addr_sk#66] -Right keys [1]: [ca_address_sk#73] +Left keys [1]: [ws_bill_addr_sk#46] +Right keys [1]: [ca_address_sk#64] Join condition: None (90) Project [codegen id : 32] -Output [4]: [ws_ext_sales_price#67, d_year#70, d_qoy#71, ca_county#74] -Input [6]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71, ca_address_sk#73, ca_county#74] +Output [4]: [ws_ext_sales_price#47, d_year#61, d_qoy#62, ca_county#65] +Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62, ca_address_sk#64, ca_county#65] (91) HashAggregate [codegen id : 32] -Input [4]: [ws_ext_sales_price#67, d_year#70, d_qoy#71, ca_county#74] -Keys [3]: [ca_county#74, d_qoy#71, d_year#70] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#67))] -Aggregate Attributes [1]: [sum#75] -Results [4]: [ca_county#74, d_qoy#71, d_year#70, sum#76] +Input [4]: [ws_ext_sales_price#47, d_year#61, d_qoy#62, ca_county#65] +Keys [3]: [ca_county#65, d_qoy#62, d_year#61] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#47))] +Aggregate Attributes [1]: [sum#66] +Results [4]: [ca_county#65, d_qoy#62, d_year#61, sum#67] (92) Exchange -Input [4]: [ca_county#74, d_qoy#71, d_year#70, sum#76] -Arguments: hashpartitioning(ca_county#74, d_qoy#71, d_year#70, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [4]: [ca_county#65, d_qoy#62, d_year#61, sum#67] +Arguments: hashpartitioning(ca_county#65, d_qoy#62, d_year#61, 5), ENSURE_REQUIREMENTS, [id=#68] (93) HashAggregate [codegen id : 33] -Input [4]: [ca_county#74, d_qoy#71, d_year#70, sum#76] -Keys [3]: [ca_county#74, d_qoy#71, d_year#70] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#67))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#67))#78] -Results [2]: [ca_county#74, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#67))#78,17,2) AS web_sales#79] +Input [4]: [ca_county#65, d_qoy#62, d_year#61, sum#67] +Keys [3]: [ca_county#65, d_qoy#62, d_year#61] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#47))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#47))#69] +Results [2]: [ca_county#65, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#47))#69,17,2) AS web_sales#70] (94) BroadcastExchange -Input [2]: [ca_county#74, web_sales#79] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#80] +Input [2]: [ca_county#65, web_sales#70] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#71] (95) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#60] -Right keys [1]: [ca_county#74] +Left keys [1]: [ca_county#54] +Right keys [1]: [ca_county#65] Join condition: None (96) Project [codegen id : 41] -Output [3]: [ca_county#60, web_sales#65, web_sales#79] -Input [4]: [ca_county#60, web_sales#65, ca_county#74, web_sales#79] +Output [3]: [ca_county#54, web_sales#59, web_sales#70] +Input [4]: [ca_county#54, web_sales#59, ca_county#65, web_sales#70] (97) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, ws_sold_date_sk#83] +Output [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#18)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (98) ColumnarToRow [codegen id : 35] -Input [3]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, ws_sold_date_sk#83] +Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] (99) Filter [codegen id : 35] -Input [3]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, ws_sold_date_sk#83] -Condition : isnotnull(ws_bill_addr_sk#81) +Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Condition : isnotnull(ws_bill_addr_sk#46) (100) ReusedExchange [Reuses operator id: 28] -Output [3]: [d_date_sk#84, d_year#85, d_qoy#86] +Output [3]: [d_date_sk#72, d_year#73, d_qoy#74] (101) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_sold_date_sk#83] -Right keys [1]: [d_date_sk#84] +Left keys [1]: [ws_sold_date_sk#48] +Right keys [1]: [d_date_sk#72] Join condition: None (102) Project [codegen id : 35] -Output [4]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86] -Input [6]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, ws_sold_date_sk#83, d_date_sk#84, d_year#85, d_qoy#86] +Output [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74] +Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48, d_date_sk#72, d_year#73, d_qoy#74] (103) Exchange -Input [4]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86] -Arguments: hashpartitioning(ws_bill_addr_sk#81, 5), ENSURE_REQUIREMENTS, [id=#87] +Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74] +Arguments: hashpartitioning(ws_bill_addr_sk#46, 5), ENSURE_REQUIREMENTS, [id=#75] (104) Sort [codegen id : 36] -Input [4]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86] -Arguments: [ws_bill_addr_sk#81 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74] +Arguments: [ws_bill_addr_sk#46 ASC NULLS FIRST], false, 0 (105) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#88, ca_county#89] +Output [2]: [ca_address_sk#76, ca_county#77] (106) Sort [codegen id : 38] -Input [2]: [ca_address_sk#88, ca_county#89] -Arguments: [ca_address_sk#88 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#76, ca_county#77] +Arguments: [ca_address_sk#76 ASC NULLS FIRST], false, 0 (107) SortMergeJoin [codegen id : 39] -Left keys [1]: [ws_bill_addr_sk#81] -Right keys [1]: [ca_address_sk#88] +Left keys [1]: [ws_bill_addr_sk#46] +Right keys [1]: [ca_address_sk#76] Join condition: None (108) Project [codegen id : 39] -Output [4]: [ws_ext_sales_price#82, d_year#85, d_qoy#86, ca_county#89] -Input [6]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86, ca_address_sk#88, ca_county#89] +Output [4]: [ws_ext_sales_price#47, d_year#73, d_qoy#74, ca_county#77] +Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74, ca_address_sk#76, ca_county#77] (109) HashAggregate [codegen id : 39] -Input [4]: [ws_ext_sales_price#82, d_year#85, d_qoy#86, ca_county#89] -Keys [3]: [ca_county#89, d_qoy#86, d_year#85] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#82))] -Aggregate Attributes [1]: [sum#90] -Results [4]: [ca_county#89, d_qoy#86, d_year#85, sum#91] +Input [4]: [ws_ext_sales_price#47, d_year#73, d_qoy#74, ca_county#77] +Keys [3]: [ca_county#77, d_qoy#74, d_year#73] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#47))] +Aggregate Attributes [1]: [sum#78] +Results [4]: [ca_county#77, d_qoy#74, d_year#73, sum#79] (110) Exchange -Input [4]: [ca_county#89, d_qoy#86, d_year#85, sum#91] -Arguments: hashpartitioning(ca_county#89, d_qoy#86, d_year#85, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [4]: [ca_county#77, d_qoy#74, d_year#73, sum#79] +Arguments: hashpartitioning(ca_county#77, d_qoy#74, d_year#73, 5), ENSURE_REQUIREMENTS, [id=#80] (111) HashAggregate [codegen id : 40] -Input [4]: [ca_county#89, d_qoy#86, d_year#85, sum#91] -Keys [3]: [ca_county#89, d_qoy#86, d_year#85] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#82))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#82))#93] -Results [2]: [ca_county#89, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#82))#93,17,2) AS web_sales#94] +Input [4]: [ca_county#77, d_qoy#74, d_year#73, sum#79] +Keys [3]: [ca_county#77, d_qoy#74, d_year#73] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#47))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#47))#81] +Results [2]: [ca_county#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#47))#81,17,2) AS web_sales#82] (112) BroadcastExchange -Input [2]: [ca_county#89, web_sales#94] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#95] +Input [2]: [ca_county#77, web_sales#82] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#83] (113) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#60] -Right keys [1]: [ca_county#89] +Left keys [1]: [ca_county#54] +Right keys [1]: [ca_county#77] Join condition: None (114) Project [codegen id : 41] -Output [4]: [ca_county#60, web_sales#65, web_sales#79, web_sales#94] -Input [5]: [ca_county#60, web_sales#65, web_sales#79, ca_county#89, web_sales#94] +Output [4]: [ca_county#54, web_sales#59, web_sales#70, web_sales#82] +Input [5]: [ca_county#54, web_sales#59, web_sales#70, ca_county#77, web_sales#82] (115) BroadcastExchange -Input [4]: [ca_county#60, web_sales#65, web_sales#79, web_sales#94] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#96] +Input [4]: [ca_county#54, web_sales#59, web_sales#70, web_sales#82] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#84] (116) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ca_county#45] -Right keys [1]: [ca_county#60] -Join condition: ((CASE WHEN (web_sales#65 > 0.00) THEN CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#65)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#50 > 0.00) THEN CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#50)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#79 > 0.00) THEN CheckOverflow((promote_precision(web_sales#94) / promote_precision(web_sales#79)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#17 > 0.00) THEN CheckOverflow((promote_precision(store_sales#33) / promote_precision(store_sales#17)), DecimalType(37,20), true) ELSE null END)) +Left keys [1]: [ca_county#39] +Right keys [1]: [ca_county#54] +Join condition: ((CASE WHEN (web_sales#59 > 0.00) THEN CheckOverflow((promote_precision(web_sales#70) / promote_precision(web_sales#59)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#44 > 0.00) THEN CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#44)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#70 > 0.00) THEN CheckOverflow((promote_precision(web_sales#82) / promote_precision(web_sales#70)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#17 > 0.00) THEN CheckOverflow((promote_precision(store_sales#30) / promote_precision(store_sales#17)), DecimalType(37,20), true) ELSE null END)) (117) Project [codegen id : 42] -Output [6]: [ca_county#45, d_year#40, CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#65)), DecimalType(37,20), true) AS web_q1_q2_increase#97, CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#50)), DecimalType(37,20), true) AS store_q1_q2_increase#98, CheckOverflow((promote_precision(web_sales#94) / promote_precision(web_sales#79)), DecimalType(37,20), true) AS web_q2_q3_increase#99, CheckOverflow((promote_precision(store_sales#33) / promote_precision(store_sales#17)), DecimalType(37,20), true) AS store_q2_q3_increase#100] -Input [9]: [store_sales#17, store_sales#33, ca_county#45, d_year#40, store_sales#50, ca_county#60, web_sales#65, web_sales#79, web_sales#94] +Output [6]: [ca_county#39, d_year#34, CheckOverflow((promote_precision(web_sales#70) / promote_precision(web_sales#59)), DecimalType(37,20), true) AS web_q1_q2_increase#85, CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#44)), DecimalType(37,20), true) AS store_q1_q2_increase#86, CheckOverflow((promote_precision(web_sales#82) / promote_precision(web_sales#70)), DecimalType(37,20), true) AS web_q2_q3_increase#87, CheckOverflow((promote_precision(store_sales#30) / promote_precision(store_sales#17)), DecimalType(37,20), true) AS store_q2_q3_increase#88] +Input [9]: [store_sales#17, store_sales#30, ca_county#39, d_year#34, store_sales#44, ca_county#54, web_sales#59, web_sales#70, web_sales#82] (118) Exchange -Input [6]: [ca_county#45, d_year#40, web_q1_q2_increase#97, store_q1_q2_increase#98, web_q2_q3_increase#99, store_q2_q3_increase#100] -Arguments: rangepartitioning(ca_county#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#101] +Input [6]: [ca_county#39, d_year#34, web_q1_q2_increase#85, store_q1_q2_increase#86, web_q2_q3_increase#87, store_q2_q3_increase#88] +Arguments: rangepartitioning(ca_county#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#89] (119) Sort [codegen id : 43] -Input [6]: [ca_county#45, d_year#40, web_q1_q2_increase#97, store_q1_q2_increase#98, web_q2_q3_increase#99, store_q2_q3_increase#100] -Arguments: [ca_county#45 ASC NULLS FIRST], true, 0 +Input [6]: [ca_county#39, d_year#34, web_q1_q2_increase#85, store_q1_q2_increase#86, web_q2_q3_increase#87, store_q2_q3_increase#88] +Arguments: [ca_county#39 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -676,24 +676,24 @@ ReusedExchange (120) (120) ReusedExchange [Reuses operator id: 7] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#18 ReusedExchange (121) (121) ReusedExchange [Reuses operator id: 28] -Output [3]: [d_date_sk#22, d_year#23, d_qoy#24] +Output [3]: [d_date_sk#19, d_year#20, d_qoy#21] -Subquery:3 Hosting operator id = 43 Hosting Expression = ss_sold_date_sk#37 IN dynamicpruning#38 +Subquery:3 Hosting operator id = 43 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#32 ReusedExchange (122) (122) ReusedExchange [Reuses operator id: 49] -Output [3]: [d_date_sk#39, d_year#40, d_qoy#41] +Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] -Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#38 +Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#32 -Subquery:5 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#4 +Subquery:5 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#4 -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#21 +Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#18 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 f47a7af5a15e3..81a4a7fed99b2 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 @@ -188,383 +188,383 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#15] Results [3]: [ca_county#10, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#15,17,2) AS store_sales#16] (19) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19] +Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#19), dynamicpruningexpression(ss_sold_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#17)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (20) ColumnarToRow [codegen id : 6] -Input [3]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19] +Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] (21) Filter [codegen id : 6] -Input [3]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19] -Condition : isnotnull(ss_addr_sk#17) +Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_addr_sk#1) (22) Scan parquet default.date_dim -Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] +Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 4] -Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] +Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] (24) Filter [codegen id : 4] -Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] -Condition : ((((isnotnull(d_qoy#23) AND isnotnull(d_year#22)) AND (d_qoy#23 = 2)) AND (d_year#22 = 2000)) AND isnotnull(d_date_sk#21)) +Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) (25) 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=#24] +Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#19] -Right keys [1]: [d_date_sk#21] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#18] Join condition: None (27) Project [codegen id : 6] -Output [4]: [ss_addr_sk#17, ss_ext_sales_price#18, d_year#22, d_qoy#23] -Input [6]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19, d_date_sk#21, d_year#22, d_qoy#23] +Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#19, d_qoy#20] +Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#18, d_year#19, d_qoy#20] (28) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#25, ca_county#26] +Output [2]: [ca_address_sk#22, ca_county#23] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#17] -Right keys [1]: [ca_address_sk#25] +Left keys [1]: [ss_addr_sk#1] +Right keys [1]: [ca_address_sk#22] Join condition: None (30) Project [codegen id : 6] -Output [4]: [ss_ext_sales_price#18, d_year#22, d_qoy#23, ca_county#26] -Input [6]: [ss_addr_sk#17, ss_ext_sales_price#18, d_year#22, d_qoy#23, ca_address_sk#25, ca_county#26] +Output [4]: [ss_ext_sales_price#2, d_year#19, d_qoy#20, ca_county#23] +Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#19, d_qoy#20, ca_address_sk#22, ca_county#23] (31) HashAggregate [codegen id : 6] -Input [4]: [ss_ext_sales_price#18, d_year#22, d_qoy#23, ca_county#26] -Keys [3]: [ca_county#26, d_qoy#23, d_year#22] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#18))] -Aggregate Attributes [1]: [sum#27] -Results [4]: [ca_county#26, d_qoy#23, d_year#22, sum#28] +Input [4]: [ss_ext_sales_price#2, d_year#19, d_qoy#20, ca_county#23] +Keys [3]: [ca_county#23, d_qoy#20, d_year#19] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#24] +Results [4]: [ca_county#23, d_qoy#20, d_year#19, sum#25] (32) Exchange -Input [4]: [ca_county#26, d_qoy#23, d_year#22, sum#28] -Arguments: hashpartitioning(ca_county#26, d_qoy#23, d_year#22, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [4]: [ca_county#23, d_qoy#20, d_year#19, sum#25] +Arguments: hashpartitioning(ca_county#23, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, [id=#26] (33) HashAggregate [codegen id : 7] -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] +Input [4]: [ca_county#23, d_qoy#20, d_year#19, sum#25] +Keys [3]: [ca_county#23, d_qoy#20, d_year#19] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#27] +Results [2]: [ca_county#23, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#27,17,2) AS store_sales#28] (34) BroadcastExchange -Input [2]: [ca_county#26, store_sales#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] +Input [2]: [ca_county#23, store_sales#28] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#29] (35) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#10] -Right keys [1]: [ca_county#26] +Right keys [1]: [ca_county#23] Join condition: None (36) 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#1, ss_ext_sales_price#2, ss_sold_date_sk#3] 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#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 10] -Input [3]: [ss_addr_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] +Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] (38) Filter [codegen id : 10] -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#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_addr_sk#1) (39) Scan parquet default.date_dim -Output [3]: [d_date_sk#37, d_year#38, d_qoy#39] +Output [3]: [d_date_sk#31, d_year#32, d_qoy#33] 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 (40) ColumnarToRow [codegen id : 8] -Input [3]: [d_date_sk#37, d_year#38, d_qoy#39] +Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] (41) Filter [codegen id : 8] -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 = 3)) AND (d_year#38 = 2000)) AND isnotnull(d_date_sk#37)) +Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] +Condition : ((((isnotnull(d_qoy#33) AND isnotnull(d_year#32)) AND (d_qoy#33 = 3)) AND (d_year#32 = 2000)) AND isnotnull(d_date_sk#31)) (42) 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=#40] +Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (43) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#35] -Right keys [1]: [d_date_sk#37] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#31] Join condition: None (44) Project [codegen id : 10] -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#1, ss_ext_sales_price#2, d_year#32, d_qoy#33] +Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#31, d_year#32, d_qoy#33] (45) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#41, ca_county#42] +Output [2]: [ca_address_sk#35, ca_county#36] (46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_addr_sk#33] -Right keys [1]: [ca_address_sk#41] +Left keys [1]: [ss_addr_sk#1] +Right keys [1]: [ca_address_sk#35] Join condition: None (47) Project [codegen id : 10] -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#2, d_year#32, d_qoy#33, ca_county#36] +Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#32, d_qoy#33, ca_address_sk#35, ca_county#36] (48) HashAggregate [codegen id : 10] -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#2, d_year#32, d_qoy#33, ca_county#36] +Keys [3]: [ca_county#36, d_qoy#33, d_year#32] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#37] +Results [4]: [ca_county#36, d_qoy#33, d_year#32, sum#38] (49) 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#36, d_qoy#33, d_year#32, sum#38] +Arguments: hashpartitioning(ca_county#36, d_qoy#33, d_year#32, 5), ENSURE_REQUIREMENTS, [id=#39] (50) HashAggregate [codegen id : 11] -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 [2]: [ca_county#42, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#46,17,2) AS store_sales#47] +Input [4]: [ca_county#36, d_qoy#33, d_year#32, sum#38] +Keys [3]: [ca_county#36, d_qoy#33, d_year#32] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#40] +Results [2]: [ca_county#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#40,17,2) AS store_sales#41] (51) BroadcastExchange -Input [2]: [ca_county#42, store_sales#47] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] +Input [2]: [ca_county#36, store_sales#41] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#42] (52) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#26] -Right keys [1]: [ca_county#42] +Left keys [1]: [ca_county#23] +Right keys [1]: [ca_county#36] Join condition: None (53) Project [codegen id : 24] -Output [5]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47] -Input [7]: [ca_county#10, d_year#6, store_sales#16, ca_county#26, store_sales#31, ca_county#42, store_sales#47] +Output [5]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41] +Input [7]: [ca_county#10, d_year#6, store_sales#16, ca_county#23, store_sales#28, ca_county#36, store_sales#41] (54) 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#43, ws_ext_sales_price#44, ws_sold_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 14] -Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] (56) Filter [codegen id : 14] -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#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Condition : isnotnull(ws_bill_addr_sk#43) (57) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#52, d_year#53, d_qoy#54] +Output [3]: [d_date_sk#46, d_year#47, d_qoy#48] (58) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#52] +Left keys [1]: [ws_sold_date_sk#45] +Right keys [1]: [d_date_sk#46] Join condition: None (59) Project [codegen id : 14] -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#43, ws_ext_sales_price#44, d_year#47, d_qoy#48] +Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45, d_date_sk#46, d_year#47, d_qoy#48] (60) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#55, ca_county#56] +Output [2]: [ca_address_sk#49, ca_county#50] (61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#49] -Right keys [1]: [ca_address_sk#55] +Left keys [1]: [ws_bill_addr_sk#43] +Right keys [1]: [ca_address_sk#49] Join condition: None (62) Project [codegen id : 14] -Output [4]: [ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_county#56] -Input [6]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_address_sk#55, ca_county#56] +Output [4]: [ws_ext_sales_price#44, d_year#47, d_qoy#48, ca_county#50] +Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#47, d_qoy#48, ca_address_sk#49, ca_county#50] (63) HashAggregate [codegen id : 14] -Input [4]: [ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_county#56] -Keys [3]: [ca_county#56, d_qoy#54, d_year#53] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#50))] -Aggregate Attributes [1]: [sum#57] -Results [4]: [ca_county#56, d_qoy#54, d_year#53, sum#58] +Input [4]: [ws_ext_sales_price#44, d_year#47, d_qoy#48, ca_county#50] +Keys [3]: [ca_county#50, d_qoy#48, d_year#47] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#44))] +Aggregate Attributes [1]: [sum#51] +Results [4]: [ca_county#50, d_qoy#48, d_year#47, sum#52] (64) Exchange -Input [4]: [ca_county#56, d_qoy#54, d_year#53, sum#58] -Arguments: hashpartitioning(ca_county#56, d_qoy#54, d_year#53, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [4]: [ca_county#50, d_qoy#48, d_year#47, sum#52] +Arguments: hashpartitioning(ca_county#50, d_qoy#48, d_year#47, 5), ENSURE_REQUIREMENTS, [id=#53] (65) HashAggregate [codegen id : 15] -Input [4]: [ca_county#56, d_qoy#54, d_year#53, sum#58] -Keys [3]: [ca_county#56, 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))#60] -Results [2]: [ca_county#56, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#50))#60,17,2) AS web_sales#61] +Input [4]: [ca_county#50, d_qoy#48, d_year#47, sum#52] +Keys [3]: [ca_county#50, d_qoy#48, d_year#47] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#44))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#44))#54] +Results [2]: [ca_county#50, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#44))#54,17,2) AS web_sales#55] (66) BroadcastExchange -Input [2]: [ca_county#56, web_sales#61] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#62] +Input [2]: [ca_county#50, web_sales#55] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] (67) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#10] -Right keys [1]: [ca_county#56] +Right keys [1]: [ca_county#50] Join condition: None (68) 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#43, ws_ext_sales_price#44, ws_sold_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#17)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 18] -Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] (70) Filter [codegen id : 18] -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#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Condition : isnotnull(ws_bill_addr_sk#43) (71) ReusedExchange [Reuses operator id: 25] -Output [3]: [d_date_sk#66, d_year#67, d_qoy#68] +Output [3]: [d_date_sk#57, d_year#58, d_qoy#59] (72) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#66] +Left keys [1]: [ws_sold_date_sk#45] +Right keys [1]: [d_date_sk#57] Join condition: None (73) Project [codegen id : 18] -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#43, ws_ext_sales_price#44, d_year#58, d_qoy#59] +Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45, d_date_sk#57, d_year#58, d_qoy#59] (74) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#69, ca_county#70] +Output [2]: [ca_address_sk#60, ca_county#61] (75) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_addr_sk#63] -Right keys [1]: [ca_address_sk#69] +Left keys [1]: [ws_bill_addr_sk#43] +Right keys [1]: [ca_address_sk#60] Join condition: None (76) Project [codegen id : 18] -Output [4]: [ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_county#70] -Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_address_sk#69, ca_county#70] +Output [4]: [ws_ext_sales_price#44, d_year#58, d_qoy#59, ca_county#61] +Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#58, d_qoy#59, ca_address_sk#60, ca_county#61] (77) HashAggregate [codegen id : 18] -Input [4]: [ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_county#70] -Keys [3]: [ca_county#70, d_qoy#68, d_year#67] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#64))] -Aggregate Attributes [1]: [sum#71] -Results [4]: [ca_county#70, d_qoy#68, d_year#67, sum#72] +Input [4]: [ws_ext_sales_price#44, d_year#58, d_qoy#59, ca_county#61] +Keys [3]: [ca_county#61, d_qoy#59, d_year#58] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#44))] +Aggregate Attributes [1]: [sum#62] +Results [4]: [ca_county#61, d_qoy#59, d_year#58, sum#63] (78) Exchange -Input [4]: [ca_county#70, d_qoy#68, d_year#67, sum#72] -Arguments: hashpartitioning(ca_county#70, d_qoy#68, d_year#67, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [4]: [ca_county#61, d_qoy#59, d_year#58, sum#63] +Arguments: hashpartitioning(ca_county#61, d_qoy#59, d_year#58, 5), ENSURE_REQUIREMENTS, [id=#64] (79) HashAggregate [codegen id : 19] -Input [4]: [ca_county#70, d_qoy#68, d_year#67, sum#72] -Keys [3]: [ca_county#70, 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))#74] -Results [2]: [ca_county#70, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#64))#74,17,2) AS web_sales#75] +Input [4]: [ca_county#61, d_qoy#59, d_year#58, sum#63] +Keys [3]: [ca_county#61, d_qoy#59, d_year#58] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#44))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#44))#65] +Results [2]: [ca_county#61, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#44))#65,17,2) AS web_sales#66] (80) BroadcastExchange -Input [2]: [ca_county#70, web_sales#75] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#76] +Input [2]: [ca_county#61, web_sales#66] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#67] (81) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#56] -Right keys [1]: [ca_county#70] -Join condition: (CASE WHEN (web_sales#61 > 0.00) THEN CheckOverflow((promote_precision(web_sales#75) / promote_precision(web_sales#61)), 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#50] +Right keys [1]: [ca_county#61] +Join condition: (CASE WHEN (web_sales#55 > 0.00) THEN CheckOverflow((promote_precision(web_sales#66) / promote_precision(web_sales#55)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#16 > 0.00) THEN CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#16)), DecimalType(37,20), true) ELSE null END) (82) Project [codegen id : 24] -Output [8]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47, ca_county#56, web_sales#61, web_sales#75] -Input [9]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47, ca_county#56, web_sales#61, ca_county#70, web_sales#75] +Output [8]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41, ca_county#50, web_sales#55, web_sales#66] +Input [9]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41, ca_county#50, web_sales#55, ca_county#61, web_sales#66] (83) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79] +Output [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (84) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79] +Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] (85) Filter [codegen id : 22] -Input [3]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79] -Condition : isnotnull(ws_bill_addr_sk#77) +Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Condition : isnotnull(ws_bill_addr_sk#43) (86) ReusedExchange [Reuses operator id: 42] -Output [3]: [d_date_sk#80, d_year#81, d_qoy#82] +Output [3]: [d_date_sk#68, d_year#69, d_qoy#70] (87) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#79] -Right keys [1]: [d_date_sk#80] +Left keys [1]: [ws_sold_date_sk#45] +Right keys [1]: [d_date_sk#68] Join condition: None (88) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, d_year#81, d_qoy#82] -Input [6]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79, d_date_sk#80, d_year#81, d_qoy#82] +Output [4]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#69, d_qoy#70] +Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45, d_date_sk#68, d_year#69, d_qoy#70] (89) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#83, ca_county#84] +Output [2]: [ca_address_sk#71, ca_county#72] (90) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_bill_addr_sk#77] -Right keys [1]: [ca_address_sk#83] +Left keys [1]: [ws_bill_addr_sk#43] +Right keys [1]: [ca_address_sk#71] Join condition: None (91) Project [codegen id : 22] -Output [4]: [ws_ext_sales_price#78, d_year#81, d_qoy#82, ca_county#84] -Input [6]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, d_year#81, d_qoy#82, ca_address_sk#83, ca_county#84] +Output [4]: [ws_ext_sales_price#44, d_year#69, d_qoy#70, ca_county#72] +Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#69, d_qoy#70, ca_address_sk#71, ca_county#72] (92) HashAggregate [codegen id : 22] -Input [4]: [ws_ext_sales_price#78, d_year#81, d_qoy#82, ca_county#84] -Keys [3]: [ca_county#84, d_qoy#82, d_year#81] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#78))] -Aggregate Attributes [1]: [sum#85] -Results [4]: [ca_county#84, d_qoy#82, d_year#81, sum#86] +Input [4]: [ws_ext_sales_price#44, d_year#69, d_qoy#70, ca_county#72] +Keys [3]: [ca_county#72, d_qoy#70, d_year#69] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#44))] +Aggregate Attributes [1]: [sum#73] +Results [4]: [ca_county#72, d_qoy#70, d_year#69, sum#74] (93) Exchange -Input [4]: [ca_county#84, d_qoy#82, d_year#81, sum#86] -Arguments: hashpartitioning(ca_county#84, d_qoy#82, d_year#81, 5), ENSURE_REQUIREMENTS, [id=#87] +Input [4]: [ca_county#72, d_qoy#70, d_year#69, sum#74] +Arguments: hashpartitioning(ca_county#72, d_qoy#70, d_year#69, 5), ENSURE_REQUIREMENTS, [id=#75] (94) HashAggregate [codegen id : 23] -Input [4]: [ca_county#84, d_qoy#82, d_year#81, sum#86] -Keys [3]: [ca_county#84, d_qoy#82, d_year#81] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#78))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#78))#88] -Results [2]: [ca_county#84, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#78))#88,17,2) AS web_sales#89] +Input [4]: [ca_county#72, d_qoy#70, d_year#69, sum#74] +Keys [3]: [ca_county#72, d_qoy#70, d_year#69] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#44))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#44))#76] +Results [2]: [ca_county#72, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#44))#76,17,2) AS web_sales#77] (95) BroadcastExchange -Input [2]: [ca_county#84, web_sales#89] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#90] +Input [2]: [ca_county#72, web_sales#77] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#78] (96) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#56] -Right keys [1]: [ca_county#84] -Join condition: (CASE WHEN (web_sales#75 > 0.00) THEN CheckOverflow((promote_precision(web_sales#89) / promote_precision(web_sales#75)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#31 > 0.00) THEN CheckOverflow((promote_precision(store_sales#47) / promote_precision(store_sales#31)), DecimalType(37,20), true) ELSE null END) +Left keys [1]: [ca_county#50] +Right keys [1]: [ca_county#72] +Join condition: (CASE WHEN (web_sales#66 > 0.00) THEN CheckOverflow((promote_precision(web_sales#77) / promote_precision(web_sales#66)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#28 > 0.00) THEN CheckOverflow((promote_precision(store_sales#41) / promote_precision(store_sales#28)), DecimalType(37,20), true) ELSE null END) (97) Project [codegen id : 24] -Output [6]: [ca_county#10, d_year#6, CheckOverflow((promote_precision(web_sales#75) / promote_precision(web_sales#61)), DecimalType(37,20), true) AS web_q1_q2_increase#91, CheckOverflow((promote_precision(store_sales#31) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q1_q2_increase#92, CheckOverflow((promote_precision(web_sales#89) / promote_precision(web_sales#75)), DecimalType(37,20), true) AS web_q2_q3_increase#93, CheckOverflow((promote_precision(store_sales#47) / promote_precision(store_sales#31)), DecimalType(37,20), true) AS store_q2_q3_increase#94] -Input [10]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47, ca_county#56, web_sales#61, web_sales#75, ca_county#84, web_sales#89] +Output [6]: [ca_county#10, d_year#6, CheckOverflow((promote_precision(web_sales#66) / promote_precision(web_sales#55)), DecimalType(37,20), true) AS web_q1_q2_increase#79, CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q1_q2_increase#80, CheckOverflow((promote_precision(web_sales#77) / promote_precision(web_sales#66)), DecimalType(37,20), true) AS web_q2_q3_increase#81, CheckOverflow((promote_precision(store_sales#41) / promote_precision(store_sales#28)), DecimalType(37,20), true) AS store_q2_q3_increase#82] +Input [10]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41, ca_county#50, web_sales#55, web_sales#66, ca_county#72, web_sales#77] (98) Exchange -Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#91, store_q1_q2_increase#92, web_q2_q3_increase#93, store_q2_q3_increase#94] -Arguments: rangepartitioning(ca_county#10 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#95] +Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#79, store_q1_q2_increase#80, web_q2_q3_increase#81, store_q2_q3_increase#82] +Arguments: rangepartitioning(ca_county#10 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#83] (99) Sort [codegen id : 25] -Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#91, store_q1_q2_increase#92, web_q2_q3_increase#93, store_q2_q3_increase#94] +Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#79, store_q1_q2_increase#80, web_q2_q3_increase#81, store_q2_q3_increase#82] Arguments: [ca_county#10 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -576,24 +576,24 @@ ReusedExchange (100) (100) ReusedExchange [Reuses operator id: 7] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#19 IN dynamicpruning#20 +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#17 ReusedExchange (101) (101) ReusedExchange [Reuses operator id: 25] -Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] +Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Subquery:3 Hosting operator id = 36 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 +Subquery:3 Hosting operator id = 36 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#30 ReusedExchange (102) (102) ReusedExchange [Reuses operator id: 42] -Output [3]: [d_date_sk#37, d_year#38, d_qoy#39] +Output [3]: [d_date_sk#31, d_year#32, d_qoy#33] -Subquery:4 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#20 +Subquery:5 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#17 -Subquery:6 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#36 +Subquery:6 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#30 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt index 5d171e5f595b9..5c886e9f2295d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt @@ -81,7 +81,7 @@ Input [2]: [d_date_sk#8, d_date#9] (11) Filter [codegen id : 2] Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) (12) Project [codegen id : 2] Output [1]: [d_date_sk#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index ad918310a918a..91cdd05907a75 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -105,7 +105,7 @@ Input [2]: [d_date_sk#11, d_date#12] (16) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10983)) AND (d_date#12 <= 11073)) AND isnotnull(d_date_sk#11)) (17) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt index ea9994910c3b9..60b7965371e06 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.sf100/explain.txt @@ -121,7 +121,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -129,7 +129,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt index df5b93da51771..6979de4765247 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt index 9da5f36eb0553..7098389df6776 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt @@ -145,20 +145,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#17] +Output [1]: [d_date_sk#10] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#17] +Right keys [1]: [d_date_sk#10] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] -Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] +Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] (23) Exchange Input [1]: [ws_bill_customer_sk#15] -Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] +Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] @@ -170,38 +170,38 @@ Right keys [1]: [ws_bill_customer_sk#15] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] +Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] +Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#21] +Output [1]: [d_date_sk#10] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#21] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#10] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#19] -Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] +Output [1]: [cs_ship_customer_sk#18] +Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] (31) Exchange -Input [1]: [cs_ship_customer_sk#19] -Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [cs_ship_customer_sk#18] +Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#19] -Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#18] +Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#19] +Right keys [1]: [cs_ship_customer_sk#18] Join condition: None (34) Filter [codegen id : 12] @@ -214,103 +214,103 @@ Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2 (36) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#23] +Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#21] (37) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 (38) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#22, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 14] -Input [2]: [ca_address_sk#24, ca_state#25] +Input [2]: [ca_address_sk#22, ca_state#23] (40) Filter [codegen id : 14] -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : isnotnull(ca_address_sk#24) +Input [2]: [ca_address_sk#22, ca_state#23] +Condition : isnotnull(ca_address_sk#22) (41) Exchange -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [ca_address_sk#22, ca_state#23] +Arguments: hashpartitioning(ca_address_sk#22, 5), ENSURE_REQUIREMENTS, [id=#24] (42) Sort [codegen id : 15] -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#22, ca_state#23] +Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#24] +Right keys [1]: [ca_address_sk#22] Join condition: None (44) Project [codegen id : 16] -Output [2]: [c_current_cdemo_sk#4, ca_state#25] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24, ca_state#25] +Output [2]: [c_current_cdemo_sk#4, ca_state#23] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] (45) Exchange -Input [2]: [c_current_cdemo_sk#4, ca_state#25] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [2]: [c_current_cdemo_sk#4, ca_state#23] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#25] (46) Sort [codegen id : 17] -Input [2]: [c_current_cdemo_sk#4, ca_state#25] +Input [2]: [c_current_cdemo_sk#4, ca_state#23] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 (47) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 18] -Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] (49) Filter [codegen id : 18] -Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Condition : isnotnull(cd_demo_sk#28) +Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Condition : isnotnull(cd_demo_sk#26) (50) Exchange -Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Arguments: hashpartitioning(cd_demo_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] (51) Sort [codegen id : 19] -Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 +Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Arguments: [cd_demo_sk#26 ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#28] +Right keys [1]: [cd_demo_sk#26] Join condition: None (53) Project [codegen id : 20] -Output [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Input [8]: [c_current_cdemo_sk#4, ca_state#25, cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] (54) HashAggregate [codegen id : 20] -Input [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#31), partial_max(cd_dep_count#31), partial_avg(cd_dep_count#31), partial_min(cd_dep_employed_count#32), partial_max(cd_dep_employed_count#32), partial_avg(cd_dep_employed_count#32), partial_min(cd_dep_college_count#33), partial_max(cd_dep_college_count#33), partial_avg(cd_dep_college_count#33)] -Aggregate Attributes [13]: [count#35, min#36, max#37, sum#38, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47] -Results [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] +Input [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Functions [10]: [partial_count(1), partial_min(cd_dep_count#29), partial_max(cd_dep_count#29), partial_avg(cd_dep_count#29), partial_min(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_avg(cd_dep_employed_count#30), partial_min(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_avg(cd_dep_college_count#31)] +Aggregate Attributes [13]: [count#33, min#34, max#35, sum#36, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45] +Results [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] (55) Exchange -Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] -Arguments: hashpartitioning(ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] +Arguments: hashpartitioning(ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#59] (56) HashAggregate [codegen id : 21] -Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] -Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Functions [10]: [count(1), min(cd_dep_count#31), max(cd_dep_count#31), avg(cd_dep_count#31), min(cd_dep_employed_count#32), max(cd_dep_employed_count#32), avg(cd_dep_employed_count#32), min(cd_dep_college_count#33), max(cd_dep_college_count#33), avg(cd_dep_college_count#33)] -Aggregate Attributes [10]: [count(1)#62, min(cd_dep_count#31)#63, max(cd_dep_count#31)#64, avg(cd_dep_count#31)#65, min(cd_dep_employed_count#32)#66, max(cd_dep_employed_count#32)#67, avg(cd_dep_employed_count#32)#68, min(cd_dep_college_count#33)#69, max(cd_dep_college_count#33)#70, avg(cd_dep_college_count#33)#71] -Results [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, count(1)#62 AS cnt1#72, min(cd_dep_count#31)#63 AS min(cd_dep_count)#73, max(cd_dep_count#31)#64 AS max(cd_dep_count)#74, avg(cd_dep_count#31)#65 AS avg(cd_dep_count)#75, cd_dep_employed_count#32, count(1)#62 AS cnt2#76, min(cd_dep_employed_count#32)#66 AS min(cd_dep_employed_count)#77, max(cd_dep_employed_count#32)#67 AS max(cd_dep_employed_count)#78, avg(cd_dep_employed_count#32)#68 AS avg(cd_dep_employed_count)#79, cd_dep_college_count#33, count(1)#62 AS cnt3#80, min(cd_dep_college_count#33)#69 AS min(cd_dep_college_count)#81, max(cd_dep_college_count#33)#70 AS max(cd_dep_college_count)#82, avg(cd_dep_college_count#33)#71 AS avg(cd_dep_college_count)#83, cd_dep_count#31 AS aggOrder#84] +Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] +Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Functions [10]: [count(1), min(cd_dep_count#29), max(cd_dep_count#29), avg(cd_dep_count#29), min(cd_dep_employed_count#30), max(cd_dep_employed_count#30), avg(cd_dep_employed_count#30), min(cd_dep_college_count#31), max(cd_dep_college_count#31), avg(cd_dep_college_count#31)] +Aggregate Attributes [10]: [count(1)#60, min(cd_dep_count#29)#61, max(cd_dep_count#29)#62, avg(cd_dep_count#29)#63, min(cd_dep_employed_count#30)#64, max(cd_dep_employed_count#30)#65, avg(cd_dep_employed_count#30)#66, min(cd_dep_college_count#31)#67, max(cd_dep_college_count#31)#68, avg(cd_dep_college_count#31)#69] +Results [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, count(1)#60 AS cnt1#70, min(cd_dep_count#29)#61 AS min(cd_dep_count)#71, max(cd_dep_count#29)#62 AS max(cd_dep_count)#72, avg(cd_dep_count#29)#63 AS avg(cd_dep_count)#73, cd_dep_employed_count#30, count(1)#60 AS cnt2#74, min(cd_dep_employed_count#30)#64 AS min(cd_dep_employed_count)#75, max(cd_dep_employed_count#30)#65 AS max(cd_dep_employed_count)#76, avg(cd_dep_employed_count#30)#66 AS avg(cd_dep_employed_count)#77, cd_dep_college_count#31, count(1)#60 AS cnt3#78, min(cd_dep_college_count#31)#67 AS min(cd_dep_college_count)#79, max(cd_dep_college_count#31)#68 AS max(cd_dep_college_count)#80, avg(cd_dep_college_count#31)#69 AS avg(cd_dep_college_count)#81, cd_dep_count#29 AS aggOrder#82] (57) TakeOrderedAndProject -Input [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cnt1#72, min(cd_dep_count)#73, max(cd_dep_count)#74, avg(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, min(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, avg(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, min(cd_dep_college_count)#81, max(cd_dep_college_count)#82, avg(cd_dep_college_count)#83, aggOrder#84] -Arguments: 100, [ca_state#25 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, aggOrder#84 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [ca_state#25, cd_gender#29, cd_marital_status#30, cnt1#72, min(cd_dep_count)#73, max(cd_dep_count)#74, avg(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, min(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, avg(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, min(cd_dep_college_count)#81, max(cd_dep_college_count)#82, avg(cd_dep_college_count)#83] +Input [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cnt1#70, min(cd_dep_count)#71, max(cd_dep_count)#72, avg(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, min(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, avg(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, min(cd_dep_college_count)#79, max(cd_dep_college_count)#80, avg(cd_dep_college_count)#81, aggOrder#82] +Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, aggOrder#82 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#23, cd_gender#27, cd_marital_status#28, cnt1#70, min(cd_dep_count)#71, max(cd_dep_count)#72, avg(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, min(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, avg(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, min(cd_dep_college_count)#79, max(cd_dep_college_count)#80, avg(cd_dep_college_count)#81] ===== Subqueries ===== @@ -323,6 +323,6 @@ Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index 48ae824834450..4dab13491784a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -122,20 +122,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#9] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#9] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#16] +Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] @@ -143,34 +143,34 @@ Right keys [1]: [ws_bill_customer_sk#14] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#20] +Output [1]: [d_date_sk#9] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#20] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#9] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#18] -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#20] +Output [1]: [cs_ship_customer_sk#17] +Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [cs_ship_customer_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#17] Join condition: None (29) Filter [codegen id : 9] @@ -182,80 +182,80 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_state#23] +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#22, ca_state#23] +Input [2]: [ca_address_sk#20, ca_state#21] (33) Filter [codegen id : 7] -Input [2]: [ca_address_sk#22, ca_state#23] -Condition : isnotnull(ca_address_sk#22) +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) (34) BroadcastExchange -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] +Input [2]: [ca_address_sk#20, ca_state#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] (35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#20] Join condition: None (36) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#23] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] +Output [2]: [c_current_cdemo_sk#4, ca_state#21] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] (37) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (39) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#25) +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Condition : isnotnull(cd_demo_sk#23) (40) BroadcastExchange -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#23] Join condition: None (42) Project [codegen id : 9] -Output [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (43) HashAggregate [codegen id : 9] -Input [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#28), partial_max(cd_dep_count#28), partial_avg(cd_dep_count#28), partial_min(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_avg(cd_dep_employed_count#29), partial_min(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_avg(cd_dep_college_count#30)] -Aggregate Attributes [13]: [count#32, min#33, max#34, sum#35, count#36, min#37, max#38, sum#39, count#40, min#41, max#42, sum#43, count#44] -Results [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] +Input [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [10]: [partial_count(1), partial_min(cd_dep_count#26), partial_max(cd_dep_count#26), partial_avg(cd_dep_count#26), partial_min(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_avg(cd_dep_employed_count#27), partial_min(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_avg(cd_dep_college_count#28)] +Aggregate Attributes [13]: [count#30, min#31, max#32, sum#33, count#34, min#35, max#36, sum#37, count#38, min#39, max#40, sum#41, count#42] +Results [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] (44) Exchange -Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] -Arguments: hashpartitioning(ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] +Arguments: hashpartitioning(ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [id=#56] (45) HashAggregate [codegen id : 10] -Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] -Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [10]: [count(1), min(cd_dep_count#28), max(cd_dep_count#28), avg(cd_dep_count#28), min(cd_dep_employed_count#29), max(cd_dep_employed_count#29), avg(cd_dep_employed_count#29), min(cd_dep_college_count#30), max(cd_dep_college_count#30), avg(cd_dep_college_count#30)] -Aggregate Attributes [10]: [count(1)#59, min(cd_dep_count#28)#60, max(cd_dep_count#28)#61, avg(cd_dep_count#28)#62, min(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, avg(cd_dep_employed_count#29)#65, min(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, avg(cd_dep_college_count#30)#68] -Results [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, count(1)#59 AS cnt1#69, min(cd_dep_count#28)#60 AS min(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, avg(cd_dep_count#28)#62 AS avg(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, min(cd_dep_employed_count#29)#63 AS min(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, avg(cd_dep_employed_count#29)#65 AS avg(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, min(cd_dep_college_count#30)#66 AS min(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, avg(cd_dep_college_count#30)#68 AS avg(cd_dep_college_count)#80, cd_dep_count#28 AS aggOrder#81] +Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] +Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [10]: [count(1), min(cd_dep_count#26), max(cd_dep_count#26), avg(cd_dep_count#26), min(cd_dep_employed_count#27), max(cd_dep_employed_count#27), avg(cd_dep_employed_count#27), min(cd_dep_college_count#28), max(cd_dep_college_count#28), avg(cd_dep_college_count#28)] +Aggregate Attributes [10]: [count(1)#57, min(cd_dep_count#26)#58, max(cd_dep_count#26)#59, avg(cd_dep_count#26)#60, min(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, avg(cd_dep_employed_count#27)#63, min(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, avg(cd_dep_college_count#28)#66] +Results [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, count(1)#57 AS cnt1#67, min(cd_dep_count#26)#58 AS min(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, avg(cd_dep_count#26)#60 AS avg(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, min(cd_dep_employed_count#27)#61 AS min(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, avg(cd_dep_employed_count#27)#63 AS avg(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, min(cd_dep_college_count#28)#64 AS min(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, avg(cd_dep_college_count#28)#66 AS avg(cd_dep_college_count)#78, cd_dep_count#26 AS aggOrder#79] (46) TakeOrderedAndProject -Input [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80, aggOrder#81] -Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, aggOrder#81 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#23, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80] +Input [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78, aggOrder#79] +Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, aggOrder#79 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#21, cd_gender#24, cd_marital_status#25, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78] ===== Subqueries ===== @@ -268,6 +268,6 @@ Output [1]: [d_date_sk#9] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt index 1f2c169ba0921..3f5ed7b48a9db 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt @@ -95,7 +95,7 @@ Input [2]: [d_date_sk#11, d_date#12] (14) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10988)) AND (d_date#12 <= 11048)) AND isnotnull(d_date_sk#11)) (15) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt index 9cc89345f5a1c..04c3f07b903f0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt @@ -92,7 +92,7 @@ Input [2]: [d_date_sk#11, d_date#12] (14) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-01)) AND (d_date#12 <= 2000-04-01)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10988)) AND (d_date#12 <= 11048)) AND isnotnull(d_date_sk#11)) (15) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt index 71accc7295b0f..776585e5bd97f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.sf100/explain.txt @@ -177,11 +177,11 @@ Results [3]: [c_last_name#11, c_first_name#10, d_date#5] (23) Exchange Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#14] (24) Sort [codegen id : 8] Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 0) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 (25) Scan parquet default.catalog_sales Output [2]: [cs_bill_customer_sk#15, cs_sold_date_sk#16] @@ -254,15 +254,15 @@ Results [3]: [c_last_name#22, c_first_name#21, d_date#18] (40) Exchange Input [3]: [c_last_name#22, c_first_name#21, d_date#18] -Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18), 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 0), isnull(d_date#18), 5), ENSURE_REQUIREMENTS, [id=#24] (41) Sort [codegen id : 16] Input [3]: [c_last_name#22, c_first_name#21, d_date#18] -Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 1970-01-01) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 0) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 (42) SortMergeJoin -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18)] +Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 0), isnull(d_date#18)] Join condition: None (43) HashAggregate [codegen id : 17] @@ -285,11 +285,11 @@ Results [3]: [c_last_name#11, c_first_name#10, d_date#5] (46) Exchange Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#26] +Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#26] (47) Sort [codegen id : 19] Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 0) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 (48) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#27, ws_sold_date_sk#28] @@ -362,15 +362,15 @@ Results [3]: [c_last_name#34, c_first_name#33, d_date#30] (63) Exchange Input [3]: [c_last_name#34, c_first_name#33, d_date#30] -Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30), 5), ENSURE_REQUIREMENTS, [id=#36] +Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 0), isnull(d_date#30), 5), ENSURE_REQUIREMENTS, [id=#36] (64) Sort [codegen id : 27] Input [3]: [c_last_name#34, c_first_name#33, d_date#30] -Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 1970-01-01) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 0) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 (65) SortMergeJoin -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30)] +Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 0), isnull(d_date#30)] Join condition: None (66) HashAggregate [codegen id : 28] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt index c4e679eb902e0..cedc6c7f4c576 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/explain.txt @@ -205,11 +205,11 @@ Results [3]: [c_last_name#19, c_first_name#18, d_date#16] (32) BroadcastExchange Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 0), isnull(input[2, date, true])),false), [id=#21] (33) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)] +Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 0), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 0), isnull(d_date#16)] Join condition: None (34) HashAggregate [codegen id : 12] @@ -285,11 +285,11 @@ Results [3]: [c_last_name#28, c_first_name#27, d_date#25] (48) BroadcastExchange Input [3]: [c_last_name#28, c_first_name#27, d_date#25] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [id=#30] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 0), isnull(input[2, date, true])),false), [id=#30] (49) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#28, ), isnull(c_last_name#28), coalesce(c_first_name#27, ), isnull(c_first_name#27), coalesce(d_date#25, 1970-01-01), isnull(d_date#25)] +Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 0), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#28, ), isnull(c_last_name#28), coalesce(c_first_name#27, ), isnull(c_first_name#27), coalesce(d_date#25, 0), isnull(d_date#25)] Join condition: None (50) HashAggregate [codegen id : 12] 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 5eb63f2a046cf..a53bec582637d 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 @@ -176,7 +176,7 @@ Results [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stddev_samp(cast(inv_qua (26) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stdev#28, mean#29] -Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND (NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0))) +Condition : (((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) (27) Project [codegen id : 5] Output [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -191,121 +191,121 @@ Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30] Arguments: [i_item_sk#10 ASC NULLS FIRST, w_warehouse_sk#12 ASC NULLS FIRST], false, 0 (30) Scan parquet default.inventory -Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] +Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#35), dynamicpruningexpression(inv_date_sk#35 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#32)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 10] -Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] +Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] (32) Filter [codegen id : 10] -Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] -Condition : (isnotnull(inv_item_sk#32) AND isnotnull(inv_warehouse_sk#33)) +Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) (33) Scan parquet default.date_dim -Output [3]: [d_date_sk#37, d_year#38, d_moy#39] +Output [3]: [d_date_sk#33, d_year#34, d_moy#35] 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 (34) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#37, d_year#38, d_moy#39] +Input [3]: [d_date_sk#33, d_year#34, d_moy#35] (35) Filter [codegen id : 7] -Input [3]: [d_date_sk#37, d_year#38, d_moy#39] -Condition : ((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2001)) AND (d_moy#39 = 2)) AND isnotnull(d_date_sk#37)) +Input [3]: [d_date_sk#33, d_year#34, d_moy#35] +Condition : ((((isnotnull(d_year#34) AND isnotnull(d_moy#35)) AND (d_year#34 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#33)) (36) Project [codegen id : 7] -Output [2]: [d_date_sk#37, d_moy#39] -Input [3]: [d_date_sk#37, d_year#38, d_moy#39] +Output [2]: [d_date_sk#33, d_moy#35] +Input [3]: [d_date_sk#33, d_year#34, d_moy#35] (37) BroadcastExchange -Input [2]: [d_date_sk#37, d_moy#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] +Input [2]: [d_date_sk#33, d_moy#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_date_sk#35] -Right keys [1]: [d_date_sk#37] +Left keys [1]: [inv_date_sk#4] +Right keys [1]: [d_date_sk#33] Join condition: None (39) Project [codegen id : 10] -Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39] -Input [6]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35, d_date_sk#37, d_moy#39] +Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35] +Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#33, d_moy#35] (40) ReusedExchange [Reuses operator id: 14] -Output [1]: [i_item_sk#41] +Output [1]: [i_item_sk#37] (41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_item_sk#32] -Right keys [1]: [i_item_sk#41] +Left keys [1]: [inv_item_sk#1] +Right keys [1]: [i_item_sk#37] Join condition: None (42) Project [codegen id : 10] -Output [4]: [inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41] -Input [5]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41] +Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37] +Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37] (43) ReusedExchange [Reuses operator id: 20] -Output [2]: [w_warehouse_sk#42, w_warehouse_name#43] +Output [2]: [w_warehouse_sk#38, w_warehouse_name#39] (44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#33] -Right keys [1]: [w_warehouse_sk#42] +Left keys [1]: [inv_warehouse_sk#2] +Right keys [1]: [w_warehouse_sk#38] Join condition: None (45) Project [codegen id : 10] -Output [5]: [inv_quantity_on_hand#34, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43, d_moy#39] -Input [6]: [inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43] +Output [5]: [inv_quantity_on_hand#3, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39, d_moy#35] +Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39] (46) HashAggregate [codegen id : 10] -Input [5]: [inv_quantity_on_hand#34, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43, d_moy#39] -Keys [4]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#34 as double)), partial_avg(inv_quantity_on_hand#34)] -Aggregate Attributes [5]: [n#44, avg#45, m2#46, sum#47, count#48] -Results [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] +Input [5]: [inv_quantity_on_hand#3, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39, d_moy#35] +Keys [4]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] +Aggregate Attributes [5]: [n#40, avg#41, m2#42, sum#43, count#44] +Results [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, n#45, avg#46, m2#47, sum#48, count#49] (47) Exchange -Input [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] -Arguments: hashpartitioning(w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, n#45, avg#46, m2#47, sum#48, count#49] +Arguments: hashpartitioning(w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, 5), ENSURE_REQUIREMENTS, [id=#50] (48) HashAggregate [codegen id : 11] -Input [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] -Keys [4]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#34 as double)), avg(inv_quantity_on_hand#34)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#34 as double))#55, avg(inv_quantity_on_hand#34)#56] -Results [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stddev_samp(cast(inv_quantity_on_hand#34 as double))#55 AS stdev#57, avg(inv_quantity_on_hand#34)#56 AS mean#58] +Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#35] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#51, avg(inv_quantity_on_hand#3)#52] +Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#3 as double))#51 AS stdev#53, avg(inv_quantity_on_hand#3)#52 AS mean#54] (49) Filter [codegen id : 11] -Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stdev#57, mean#58] -Condition : ((isnotnull(mean#58) AND isnotnull(stdev#57)) AND (NOT (mean#58 = 0.0) AND ((stdev#57 / mean#58) > 1.0))) +Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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)) (50) Project [codegen id : 11] -Output [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, CASE WHEN (mean#58 = 0.0) THEN null ELSE (stdev#57 / mean#58) END AS cov#59] -Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stdev#57, mean#58] +Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#35, stdev#53, mean#54] (51) Exchange -Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] -Arguments: hashpartitioning(i_item_sk#41, w_warehouse_sk#42, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] +Arguments: hashpartitioning(i_item_sk#37, w_warehouse_sk#38, 5), ENSURE_REQUIREMENTS, [id=#56] (52) Sort [codegen id : 12] -Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] -Arguments: [i_item_sk#41 ASC NULLS FIRST, w_warehouse_sk#42 ASC NULLS FIRST], false, 0 +Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] +Arguments: [i_item_sk#37 ASC NULLS FIRST, w_warehouse_sk#38 ASC NULLS FIRST], false, 0 (53) SortMergeJoin [codegen id : 13] Left keys [2]: [i_item_sk#10, w_warehouse_sk#12] -Right keys [2]: [i_item_sk#41, w_warehouse_sk#42] +Right keys [2]: [i_item_sk#37, w_warehouse_sk#38] Join condition: None (54) Exchange -Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] -Arguments: rangepartitioning(w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, mean#58 ASC NULLS FIRST, cov#59 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] +Arguments: rangepartitioning(w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#57] (55) Sort [codegen id : 14] -Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] -Arguments: [w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, mean#58 ASC NULLS FIRST, cov#59 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] +Arguments: [w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -316,11 +316,11 @@ ReusedExchange (56) (56) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#6, d_moy#8] -Subquery:2 Hosting operator id = 30 Hosting Expression = inv_date_sk#35 IN dynamicpruning#36 +Subquery:2 Hosting operator id = 30 Hosting Expression = inv_date_sk#4 IN dynamicpruning#32 ReusedExchange (57) (57) ReusedExchange [Reuses operator id: 37] -Output [2]: [d_date_sk#37, d_moy#39] +Output [2]: [d_date_sk#33, d_moy#35] 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 e77de53c5adcb..c61480a05a4cf 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 @@ -173,124 +173,124 @@ Results [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stddev_samp(cast(inv_quan (26) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] -Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND (NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0))) +Condition : (((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) (27) Project [codegen id : 10] Output [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] (28) Scan parquet default.inventory -Output [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] +Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#34), dynamicpruningexpression(inv_date_sk#34 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#31)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] +Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] (30) Filter [codegen id : 8] -Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] -Condition : (isnotnull(inv_item_sk#31) AND isnotnull(inv_warehouse_sk#32)) +Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) (31) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#36] +Output [1]: [i_item_sk#32] (32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#31] -Right keys [1]: [i_item_sk#36] +Left keys [1]: [inv_item_sk#1] +Right keys [1]: [i_item_sk#32] Join condition: None (33) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36] -Input [5]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36] +Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32] +Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32] (34) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#37, w_warehouse_name#38] +Output [2]: [w_warehouse_sk#33, w_warehouse_name#34] (35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#32] -Right keys [1]: [w_warehouse_sk#37] +Left keys [1]: [inv_warehouse_sk#2] +Right keys [1]: [w_warehouse_sk#33] Join condition: None (36) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38] -Input [6]: [inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38] +Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] +Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] (37) Scan parquet default.date_dim -Output [3]: [d_date_sk#39, d_year#40, d_moy#41] +Output [3]: [d_date_sk#35, d_year#36, d_moy#37] 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 (38) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#39, d_year#40, d_moy#41] +Input [3]: [d_date_sk#35, d_year#36, d_moy#37] (39) Filter [codegen id : 7] -Input [3]: [d_date_sk#39, d_year#40, d_moy#41] -Condition : ((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2001)) AND (d_moy#41 = 2)) AND isnotnull(d_date_sk#39)) +Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2001)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) (40) Project [codegen id : 7] -Output [2]: [d_date_sk#39, d_moy#41] -Input [3]: [d_date_sk#39, d_year#40, d_moy#41] +Output [2]: [d_date_sk#35, d_moy#37] +Input [3]: [d_date_sk#35, d_year#36, d_moy#37] (41) BroadcastExchange -Input [2]: [d_date_sk#39, d_moy#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [2]: [d_date_sk#35, d_moy#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] (42) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#34] -Right keys [1]: [d_date_sk#39] +Left keys [1]: [inv_date_sk#4] +Right keys [1]: [d_date_sk#35] Join condition: None (43) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#33, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_moy#41] -Input [7]: [inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_date_sk#39, d_moy#41] +Output [5]: [inv_quantity_on_hand#3, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] +Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_date_sk#35, d_moy#37] (44) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#33, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_moy#41] -Keys [4]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#33 as double)), partial_avg(inv_quantity_on_hand#33)] -Aggregate Attributes [5]: [n#43, avg#44, m2#45, sum#46, count#47] -Results [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] +Input [5]: [inv_quantity_on_hand#3, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] +Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] +Aggregate Attributes [5]: [n#39, avg#40, m2#41, sum#42, count#43] +Results [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] (45) Exchange -Input [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] -Arguments: hashpartitioning(w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] +Arguments: hashpartitioning(w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, 5), ENSURE_REQUIREMENTS, [id=#49] (46) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] -Keys [4]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#33 as double)), avg(inv_quantity_on_hand#33)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#33 as double))#54, avg(inv_quantity_on_hand#33)#55] -Results [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stddev_samp(cast(inv_quantity_on_hand#33 as double))#54 AS stdev#56, avg(inv_quantity_on_hand#33)#55 AS mean#57] +Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] +Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#50, avg(inv_quantity_on_hand#3)#51] +Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_quantity_on_hand#3 as double))#50 AS stdev#52, avg(inv_quantity_on_hand#3)#51 AS mean#53] (47) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stdev#56, mean#57] -Condition : ((isnotnull(mean#57) AND isnotnull(stdev#56)) AND (NOT (mean#57 = 0.0) AND ((stdev#56 / mean#57) > 1.0))) +Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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)) (48) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, CASE WHEN (mean#57 = 0.0) THEN null ELSE (stdev#56 / mean#57) END AS cov#58] -Input [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stdev#56, mean#57] +Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] +Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#52, mean#53] (49) BroadcastExchange -Input [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [id=#59] +Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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] (50) BroadcastHashJoin [codegen id : 10] Left keys [2]: [i_item_sk#6, w_warehouse_sk#8] -Right keys [2]: [i_item_sk#36, w_warehouse_sk#37] +Right keys [2]: [i_item_sk#32, w_warehouse_sk#33] Join condition: None (51) Exchange -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] -Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, mean#57 ASC NULLS FIRST, cov#58 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, cov#54] +Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, mean#53 ASC NULLS FIRST, cov#54 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#56] (52) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] -Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, mean#57 ASC NULLS FIRST, cov#58 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, cov#54] +Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, mean#53 ASC NULLS FIRST, cov#54 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -301,11 +301,11 @@ ReusedExchange (53) (53) ReusedExchange [Reuses operator id: 20] Output [2]: [d_date_sk#11, d_moy#13] -Subquery:2 Hosting operator id = 28 Hosting Expression = inv_date_sk#34 IN dynamicpruning#35 +Subquery:2 Hosting operator id = 28 Hosting Expression = inv_date_sk#4 IN dynamicpruning#31 ReusedExchange (54) (54) ReusedExchange [Reuses operator id: 41] -Output [2]: [d_date_sk#39, d_moy#41] +Output [2]: [d_date_sk#35, d_moy#37] 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 e2e3760fe2d03..6bc5980231252 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 @@ -176,7 +176,7 @@ Results [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stddev_samp(cast(inv_qua (26) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stdev#28, mean#29] -Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND ((NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0)) AND ((stdev#28 / mean#29) > 1.5))) +Condition : (((((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.5)) (27) Project [codegen id : 5] Output [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -191,121 +191,121 @@ Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30] Arguments: [i_item_sk#10 ASC NULLS FIRST, w_warehouse_sk#12 ASC NULLS FIRST], false, 0 (30) Scan parquet default.inventory -Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] +Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#35), dynamicpruningexpression(inv_date_sk#35 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#32)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 10] -Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] +Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] (32) Filter [codegen id : 10] -Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] -Condition : (isnotnull(inv_item_sk#32) AND isnotnull(inv_warehouse_sk#33)) +Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) (33) Scan parquet default.date_dim -Output [3]: [d_date_sk#37, d_year#38, d_moy#39] +Output [3]: [d_date_sk#33, d_year#34, d_moy#35] 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 (34) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#37, d_year#38, d_moy#39] +Input [3]: [d_date_sk#33, d_year#34, d_moy#35] (35) Filter [codegen id : 7] -Input [3]: [d_date_sk#37, d_year#38, d_moy#39] -Condition : ((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2001)) AND (d_moy#39 = 2)) AND isnotnull(d_date_sk#37)) +Input [3]: [d_date_sk#33, d_year#34, d_moy#35] +Condition : ((((isnotnull(d_year#34) AND isnotnull(d_moy#35)) AND (d_year#34 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#33)) (36) Project [codegen id : 7] -Output [2]: [d_date_sk#37, d_moy#39] -Input [3]: [d_date_sk#37, d_year#38, d_moy#39] +Output [2]: [d_date_sk#33, d_moy#35] +Input [3]: [d_date_sk#33, d_year#34, d_moy#35] (37) BroadcastExchange -Input [2]: [d_date_sk#37, d_moy#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] +Input [2]: [d_date_sk#33, d_moy#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_date_sk#35] -Right keys [1]: [d_date_sk#37] +Left keys [1]: [inv_date_sk#4] +Right keys [1]: [d_date_sk#33] Join condition: None (39) Project [codegen id : 10] -Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39] -Input [6]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35, d_date_sk#37, d_moy#39] +Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35] +Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#33, d_moy#35] (40) ReusedExchange [Reuses operator id: 14] -Output [1]: [i_item_sk#41] +Output [1]: [i_item_sk#37] (41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_item_sk#32] -Right keys [1]: [i_item_sk#41] +Left keys [1]: [inv_item_sk#1] +Right keys [1]: [i_item_sk#37] Join condition: None (42) Project [codegen id : 10] -Output [4]: [inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41] -Input [5]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41] +Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37] +Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37] (43) ReusedExchange [Reuses operator id: 20] -Output [2]: [w_warehouse_sk#42, w_warehouse_name#43] +Output [2]: [w_warehouse_sk#38, w_warehouse_name#39] (44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#33] -Right keys [1]: [w_warehouse_sk#42] +Left keys [1]: [inv_warehouse_sk#2] +Right keys [1]: [w_warehouse_sk#38] Join condition: None (45) Project [codegen id : 10] -Output [5]: [inv_quantity_on_hand#34, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43, d_moy#39] -Input [6]: [inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43] +Output [5]: [inv_quantity_on_hand#3, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39, d_moy#35] +Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39] (46) HashAggregate [codegen id : 10] -Input [5]: [inv_quantity_on_hand#34, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43, d_moy#39] -Keys [4]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#34 as double)), partial_avg(inv_quantity_on_hand#34)] -Aggregate Attributes [5]: [n#44, avg#45, m2#46, sum#47, count#48] -Results [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] +Input [5]: [inv_quantity_on_hand#3, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39, d_moy#35] +Keys [4]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] +Aggregate Attributes [5]: [n#40, avg#41, m2#42, sum#43, count#44] +Results [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, n#45, avg#46, m2#47, sum#48, count#49] (47) Exchange -Input [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] -Arguments: hashpartitioning(w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, n#45, avg#46, m2#47, sum#48, count#49] +Arguments: hashpartitioning(w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, 5), ENSURE_REQUIREMENTS, [id=#50] (48) HashAggregate [codegen id : 11] -Input [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] -Keys [4]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#34 as double)), avg(inv_quantity_on_hand#34)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#34 as double))#55, avg(inv_quantity_on_hand#34)#56] -Results [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stddev_samp(cast(inv_quantity_on_hand#34 as double))#55 AS stdev#57, avg(inv_quantity_on_hand#34)#56 AS mean#58] +Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#35] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#51, avg(inv_quantity_on_hand#3)#52] +Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#3 as double))#51 AS stdev#53, avg(inv_quantity_on_hand#3)#52 AS mean#54] (49) Filter [codegen id : 11] -Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stdev#57, mean#58] -Condition : ((isnotnull(mean#58) AND isnotnull(stdev#57)) AND (NOT (mean#58 = 0.0) AND ((stdev#57 / mean#58) > 1.0))) +Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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)) (50) Project [codegen id : 11] -Output [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, CASE WHEN (mean#58 = 0.0) THEN null ELSE (stdev#57 / mean#58) END AS cov#59] -Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stdev#57, mean#58] +Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#35, stdev#53, mean#54] (51) Exchange -Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] -Arguments: hashpartitioning(i_item_sk#41, w_warehouse_sk#42, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] +Arguments: hashpartitioning(i_item_sk#37, w_warehouse_sk#38, 5), ENSURE_REQUIREMENTS, [id=#56] (52) Sort [codegen id : 12] -Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] -Arguments: [i_item_sk#41 ASC NULLS FIRST, w_warehouse_sk#42 ASC NULLS FIRST], false, 0 +Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] +Arguments: [i_item_sk#37 ASC NULLS FIRST, w_warehouse_sk#38 ASC NULLS FIRST], false, 0 (53) SortMergeJoin [codegen id : 13] Left keys [2]: [i_item_sk#10, w_warehouse_sk#12] -Right keys [2]: [i_item_sk#41, w_warehouse_sk#42] +Right keys [2]: [i_item_sk#37, w_warehouse_sk#38] Join condition: None (54) Exchange -Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] -Arguments: rangepartitioning(w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, mean#58 ASC NULLS FIRST, cov#59 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] +Arguments: rangepartitioning(w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#57] (55) Sort [codegen id : 14] -Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] -Arguments: [w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, mean#58 ASC NULLS FIRST, cov#59 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] +Arguments: [w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -316,11 +316,11 @@ ReusedExchange (56) (56) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#6, d_moy#8] -Subquery:2 Hosting operator id = 30 Hosting Expression = inv_date_sk#35 IN dynamicpruning#36 +Subquery:2 Hosting operator id = 30 Hosting Expression = inv_date_sk#4 IN dynamicpruning#32 ReusedExchange (57) (57) ReusedExchange [Reuses operator id: 37] -Output [2]: [d_date_sk#37, d_moy#39] +Output [2]: [d_date_sk#33, d_moy#35] 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 d60cd37ce7bf1..0ffe40240be2b 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 @@ -173,124 +173,124 @@ Results [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stddev_samp(cast(inv_quan (26) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] -Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND ((NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0)) AND ((stdev#28 / mean#29) > 1.5))) +Condition : (((((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.5)) (27) Project [codegen id : 10] Output [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] (28) Scan parquet default.inventory -Output [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] +Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#34), dynamicpruningexpression(inv_date_sk#34 IN dynamicpruning#35)] +PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#31)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] +Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] (30) Filter [codegen id : 8] -Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] -Condition : (isnotnull(inv_item_sk#31) AND isnotnull(inv_warehouse_sk#32)) +Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) (31) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#36] +Output [1]: [i_item_sk#32] (32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#31] -Right keys [1]: [i_item_sk#36] +Left keys [1]: [inv_item_sk#1] +Right keys [1]: [i_item_sk#32] Join condition: None (33) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36] -Input [5]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36] +Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32] +Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32] (34) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#37, w_warehouse_name#38] +Output [2]: [w_warehouse_sk#33, w_warehouse_name#34] (35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#32] -Right keys [1]: [w_warehouse_sk#37] +Left keys [1]: [inv_warehouse_sk#2] +Right keys [1]: [w_warehouse_sk#33] Join condition: None (36) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38] -Input [6]: [inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38] +Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] +Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] (37) Scan parquet default.date_dim -Output [3]: [d_date_sk#39, d_year#40, d_moy#41] +Output [3]: [d_date_sk#35, d_year#36, d_moy#37] 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 (38) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#39, d_year#40, d_moy#41] +Input [3]: [d_date_sk#35, d_year#36, d_moy#37] (39) Filter [codegen id : 7] -Input [3]: [d_date_sk#39, d_year#40, d_moy#41] -Condition : ((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2001)) AND (d_moy#41 = 2)) AND isnotnull(d_date_sk#39)) +Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2001)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) (40) Project [codegen id : 7] -Output [2]: [d_date_sk#39, d_moy#41] -Input [3]: [d_date_sk#39, d_year#40, d_moy#41] +Output [2]: [d_date_sk#35, d_moy#37] +Input [3]: [d_date_sk#35, d_year#36, d_moy#37] (41) BroadcastExchange -Input [2]: [d_date_sk#39, d_moy#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [2]: [d_date_sk#35, d_moy#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] (42) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#34] -Right keys [1]: [d_date_sk#39] +Left keys [1]: [inv_date_sk#4] +Right keys [1]: [d_date_sk#35] Join condition: None (43) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#33, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_moy#41] -Input [7]: [inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_date_sk#39, d_moy#41] +Output [5]: [inv_quantity_on_hand#3, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] +Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_date_sk#35, d_moy#37] (44) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#33, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_moy#41] -Keys [4]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#33 as double)), partial_avg(inv_quantity_on_hand#33)] -Aggregate Attributes [5]: [n#43, avg#44, m2#45, sum#46, count#47] -Results [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] +Input [5]: [inv_quantity_on_hand#3, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] +Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] +Aggregate Attributes [5]: [n#39, avg#40, m2#41, sum#42, count#43] +Results [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] (45) Exchange -Input [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] -Arguments: hashpartitioning(w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] +Arguments: hashpartitioning(w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, 5), ENSURE_REQUIREMENTS, [id=#49] (46) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] -Keys [4]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#33 as double)), avg(inv_quantity_on_hand#33)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#33 as double))#54, avg(inv_quantity_on_hand#33)#55] -Results [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stddev_samp(cast(inv_quantity_on_hand#33 as double))#54 AS stdev#56, avg(inv_quantity_on_hand#33)#55 AS mean#57] +Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] +Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#50, avg(inv_quantity_on_hand#3)#51] +Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_quantity_on_hand#3 as double))#50 AS stdev#52, avg(inv_quantity_on_hand#3)#51 AS mean#53] (47) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stdev#56, mean#57] -Condition : ((isnotnull(mean#57) AND isnotnull(stdev#56)) AND (NOT (mean#57 = 0.0) AND ((stdev#56 / mean#57) > 1.0))) +Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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)) (48) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, CASE WHEN (mean#57 = 0.0) THEN null ELSE (stdev#56 / mean#57) END AS cov#58] -Input [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stdev#56, mean#57] +Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] +Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#52, mean#53] (49) BroadcastExchange -Input [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [id=#59] +Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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] (50) BroadcastHashJoin [codegen id : 10] Left keys [2]: [i_item_sk#6, w_warehouse_sk#8] -Right keys [2]: [i_item_sk#36, w_warehouse_sk#37] +Right keys [2]: [i_item_sk#32, w_warehouse_sk#33] Join condition: None (51) Exchange -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] -Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, mean#57 ASC NULLS FIRST, cov#58 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, cov#54] +Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, mean#53 ASC NULLS FIRST, cov#54 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#56] (52) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] -Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, mean#57 ASC NULLS FIRST, cov#58 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, cov#54] +Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, mean#53 ASC NULLS FIRST, cov#54 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -301,11 +301,11 @@ ReusedExchange (53) (53) ReusedExchange [Reuses operator id: 20] Output [2]: [d_date_sk#11, d_moy#13] -Subquery:2 Hosting operator id = 28 Hosting Expression = inv_date_sk#34 IN dynamicpruning#35 +Subquery:2 Hosting operator id = 28 Hosting Expression = inv_date_sk#4 IN dynamicpruning#31 ReusedExchange (54) (54) ReusedExchange [Reuses operator id: 41] -Output [2]: [d_date_sk#39, d_moy#41] +Output [2]: [d_date_sk#35, d_moy#37] 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 ce8b8bed5a26e..7b6bd35bfe180 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 @@ -239,465 +239,465 @@ Input [2]: [customer_id#27, year_total#28] Arguments: [customer_id#27 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Output [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6] 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#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6] (27) Filter [codegen id : 10] -Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] -Condition : isnotnull(ss_customer_sk#30) +Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6] +Condition : isnotnull(ss_customer_sk#1) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#37, d_year#38] +Output [2]: [d_date_sk#8, d_year#9] 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 (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#37, d_year#38] +Input [2]: [d_date_sk#8, d_year#9] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#37, d_year#38] -Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_sk#37)) +Input [2]: [d_date_sk#8, d_year#9] +Condition : ((isnotnull(d_year#9) AND (d_year#9 = 2002)) AND isnotnull(d_date_sk#8)) (31) BroadcastExchange -Input [2]: [d_date_sk#37, d_year#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#39] +Input [2]: [d_date_sk#8, d_year#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#35] -Right keys [1]: [d_date_sk#37] +Left keys [1]: [ss_sold_date_sk#6] +Right keys [1]: [d_date_sk#8] Join condition: None (33) Project [codegen id : 10] -Output [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] -Input [8]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#37, d_year#38] +Output [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] +Input [8]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6, d_date_sk#8, d_year#9] (34) Exchange -Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] -Arguments: hashpartitioning(ss_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] +Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#32] (35) Sort [codegen id : 11] -Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] -Arguments: [ss_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] +Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (37) Sort [codegen id : 13] -Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] -Arguments: [c_customer_sk#41 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#30] -Right keys [1]: [c_customer_sk#41] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#12] Join condition: None (39) Project [codegen id : 14] -Output [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] -Input [14]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38, c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] +Input [14]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (40) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] -Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#49, isEmpty#50] -Results [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, sum#51, isEmpty#52] +Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#33, isEmpty#34] +Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#35, isEmpty#36] (41) Exchange -Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, sum#51, isEmpty#52] -Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#35, isEmpty#36] +Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#37] (42) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, sum#51, isEmpty#52] -Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 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#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#54] -Results [8]: [c_customer_id#42 AS customer_id#55, c_first_name#43 AS customer_first_name#56, c_last_name#44 AS customer_last_name#57, c_preferred_cust_flag#45 AS customer_preferred_cust_flag#58, c_birth_country#46 AS customer_birth_country#59, c_login#47 AS customer_login#60, c_email_address#48 AS customer_email_address#61, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#54 AS year_total#62] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#35, isEmpty#36] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 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#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#38] +Results [8]: [c_customer_id#13 AS customer_id#39, c_first_name#14 AS customer_first_name#40, c_last_name#15 AS customer_last_name#41, c_preferred_cust_flag#16 AS customer_preferred_cust_flag#42, c_birth_country#17 AS customer_birth_country#43, c_login#18 AS customer_login#44, c_email_address#19 AS customer_email_address#45, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#38 AS year_total#46] (43) Exchange -Input [8]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62] -Arguments: hashpartitioning(customer_id#55, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [8]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46] +Arguments: hashpartitioning(customer_id#39, 5), ENSURE_REQUIREMENTS, [id=#47] (44) Sort [codegen id : 16] -Input [8]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62] -Arguments: [customer_id#55 ASC NULLS FIRST], false, 0 +Input [8]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46] +Arguments: [customer_id#39 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#55] +Right keys [1]: [customer_id#39] Join condition: None (46) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69] +Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#69), dynamicpruningexpression(cs_sold_date_sk#69 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#53), dynamicpruningexpression(cs_sold_date_sk#53 IN dynamicpruning#7)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69] +Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] (48) Filter [codegen id : 19] -Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69] -Condition : isnotnull(cs_bill_customer_sk#64) +Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] +Condition : isnotnull(cs_bill_customer_sk#48) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#70, d_year#71] +Output [2]: [d_date_sk#8, d_year#9] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_sold_date_sk#69] -Right keys [1]: [d_date_sk#70] +Left keys [1]: [cs_sold_date_sk#53] +Right keys [1]: [d_date_sk#8] Join condition: None (51) Project [codegen id : 19] -Output [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] -Input [8]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69, d_date_sk#70, d_year#71] +Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Input [8]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53, d_date_sk#8, d_year#9] (52) Exchange -Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] -Arguments: hashpartitioning(cs_bill_customer_sk#64, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Arguments: hashpartitioning(cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, [id=#54] (53) Sort [codegen id : 20] -Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] -Arguments: [cs_bill_customer_sk#64 ASC NULLS FIRST], false, 0 +Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Arguments: [cs_bill_customer_sk#48 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (55) Sort [codegen id : 22] -Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] -Arguments: [c_customer_sk#73 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#64] -Right keys [1]: [c_customer_sk#73] +Left keys [1]: [cs_bill_customer_sk#48] +Right keys [1]: [c_customer_sk#12] Join condition: None (57) Project [codegen id : 23] -Output [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] -Input [14]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71, c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Input [14]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (58) HashAggregate [codegen id : 23] -Input [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] -Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#81, isEmpty#82] -Results [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, sum#83, isEmpty#84] +Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#55, isEmpty#56] +Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#57, isEmpty#58] (59) Exchange -Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, sum#83, isEmpty#84] -Arguments: hashpartitioning(c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#57, isEmpty#58] +Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#59] (60) HashAggregate [codegen id : 24] -Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, sum#83, isEmpty#84] -Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 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#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86] -Results [2]: [c_customer_id#74 AS customer_id#87, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86 AS year_total#88] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#57, isEmpty#58] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 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#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#60] +Results [2]: [c_customer_id#13 AS customer_id#61, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#60 AS year_total#62] (61) Filter [codegen id : 24] -Input [2]: [customer_id#87, year_total#88] -Condition : (isnotnull(year_total#88) AND (year_total#88 > 0.000000)) +Input [2]: [customer_id#61, year_total#62] +Condition : (isnotnull(year_total#62) AND (year_total#62 > 0.000000)) (62) Project [codegen id : 24] -Output [2]: [customer_id#87 AS customer_id#89, year_total#88 AS year_total#90] -Input [2]: [customer_id#87, year_total#88] +Output [2]: [customer_id#61 AS customer_id#63, year_total#62 AS year_total#64] +Input [2]: [customer_id#61, year_total#62] (63) Exchange -Input [2]: [customer_id#89, year_total#90] -Arguments: hashpartitioning(customer_id#89, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [2]: [customer_id#63, year_total#64] +Arguments: hashpartitioning(customer_id#63, 5), ENSURE_REQUIREMENTS, [id=#65] (64) Sort [codegen id : 25] -Input [2]: [customer_id#89, year_total#90] -Arguments: [customer_id#89 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#63, year_total#64] +Arguments: [customer_id#63 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#89] +Right keys [1]: [customer_id#63] Join condition: None (66) Project [codegen id : 26] -Output [11]: [customer_id#27, year_total#28, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62, year_total#90] -Input [12]: [customer_id#27, year_total#28, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62, customer_id#89, year_total#90] +Output [11]: [customer_id#27, year_total#28, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46, year_total#64] +Input [12]: [customer_id#27, year_total#28, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46, customer_id#63, year_total#64] (67) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97] +Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#97), dynamicpruningexpression(cs_sold_date_sk#97 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(cs_sold_date_sk#53), dynamicpruningexpression(cs_sold_date_sk#53 IN dynamicpruning#30)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97] +Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] (69) Filter [codegen id : 28] -Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97] -Condition : isnotnull(cs_bill_customer_sk#92) +Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] +Condition : isnotnull(cs_bill_customer_sk#48) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#98, d_year#99] +Output [2]: [d_date_sk#8, d_year#9] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#97] -Right keys [1]: [d_date_sk#98] +Left keys [1]: [cs_sold_date_sk#53] +Right keys [1]: [d_date_sk#8] Join condition: None (72) Project [codegen id : 28] -Output [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] -Input [8]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97, d_date_sk#98, d_year#99] +Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Input [8]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53, d_date_sk#8, d_year#9] (73) Exchange -Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] -Arguments: hashpartitioning(cs_bill_customer_sk#92, 5), ENSURE_REQUIREMENTS, [id=#100] +Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Arguments: hashpartitioning(cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, [id=#66] (74) Sort [codegen id : 29] -Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] -Arguments: [cs_bill_customer_sk#92 ASC NULLS FIRST], false, 0 +Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Arguments: [cs_bill_customer_sk#48 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] +Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (76) Sort [codegen id : 31] -Input [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] -Arguments: [c_customer_sk#101 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [cs_bill_customer_sk#92] -Right keys [1]: [c_customer_sk#101] +Left keys [1]: [cs_bill_customer_sk#48] +Right keys [1]: [c_customer_sk#12] Join condition: None (78) Project [codegen id : 32] -Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] -Input [14]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99, c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] +Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Input [14]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (79) HashAggregate [codegen id : 32] -Input [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] -Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#109, isEmpty#110] -Results [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, sum#111, isEmpty#112] +Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#67, isEmpty#68] +Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#69, isEmpty#70] (80) Exchange -Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, sum#111, isEmpty#112] -Arguments: hashpartitioning(c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, 5), ENSURE_REQUIREMENTS, [id=#113] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#69, isEmpty#70] +Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#71] (81) HashAggregate [codegen id : 33] -Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, sum#111, isEmpty#112] -Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 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#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#114] -Results [2]: [c_customer_id#102 AS customer_id#115, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#114 AS year_total#116] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#69, isEmpty#70] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 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#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#72] +Results [2]: [c_customer_id#13 AS customer_id#73, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#72 AS year_total#74] (82) Exchange -Input [2]: [customer_id#115, year_total#116] -Arguments: hashpartitioning(customer_id#115, 5), ENSURE_REQUIREMENTS, [id=#117] +Input [2]: [customer_id#73, year_total#74] +Arguments: hashpartitioning(customer_id#73, 5), ENSURE_REQUIREMENTS, [id=#75] (83) Sort [codegen id : 34] -Input [2]: [customer_id#115, year_total#116] -Arguments: [customer_id#115 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#73, year_total#74] +Arguments: [customer_id#73 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#115] -Join condition: (CASE WHEN (year_total#90 > 0.000000) THEN CheckOverflow((promote_precision(year_total#116) / promote_precision(year_total#90)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#28 > 0.000000) THEN CheckOverflow((promote_precision(year_total#62) / promote_precision(year_total#28)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#73] +Join condition: (CASE WHEN (year_total#64 > 0.000000) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#64)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#28 > 0.000000) THEN CheckOverflow((promote_precision(year_total#46) / promote_precision(year_total#28)), DecimalType(38,14), true) ELSE null END) (85) Project [codegen id : 35] -Output [10]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116] -Input [13]: [customer_id#27, year_total#28, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62, year_total#90, customer_id#115, year_total#116] +Output [10]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74] +Input [13]: [customer_id#27, year_total#28, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46, year_total#64, customer_id#73, year_total#74] (86) Scan parquet default.web_sales -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] +Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#123), dynamicpruningexpression(ws_sold_date_sk#123 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 37] -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] +Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] (88) Filter [codegen id : 37] -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) +Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Condition : isnotnull(ws_bill_customer_sk#76) (89) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#124, d_year#125] +Output [2]: [d_date_sk#8, d_year#9] (90) BroadcastHashJoin [codegen id : 37] -Left keys [1]: [ws_sold_date_sk#123] -Right keys [1]: [d_date_sk#124] +Left keys [1]: [ws_sold_date_sk#81] +Right keys [1]: [d_date_sk#8] Join condition: None (91) Project [codegen id : 37] -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, d_year#125] -Input [8]: [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, d_date_sk#124, d_year#125] +Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Input [8]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#8, d_year#9] (92) Exchange -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, d_year#125] -Arguments: hashpartitioning(ws_bill_customer_sk#118, 5), ENSURE_REQUIREMENTS, [id=#126] +Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Arguments: hashpartitioning(ws_bill_customer_sk#76, 5), ENSURE_REQUIREMENTS, [id=#82] (93) Sort [codegen id : 38] -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, d_year#125] -Arguments: [ws_bill_customer_sk#118 ASC NULLS FIRST], false, 0 +Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Arguments: [ws_bill_customer_sk#76 ASC NULLS FIRST], false, 0 (94) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#127, c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134] +Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (95) Sort [codegen id : 40] -Input [8]: [c_customer_sk#127, c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134] -Arguments: [c_customer_sk#127 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 (96) SortMergeJoin [codegen id : 41] -Left keys [1]: [ws_bill_customer_sk#118] -Right keys [1]: [c_customer_sk#127] +Left keys [1]: [ws_bill_customer_sk#76] +Right keys [1]: [c_customer_sk#12] Join condition: None (97) Project [codegen id : 41] -Output [12]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, d_year#125] -Input [14]: [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, d_year#125, c_customer_sk#127, c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134] +Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Input [14]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (98) HashAggregate [codegen id : 41] -Input [12]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, d_year#125] -Keys [8]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125] -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#135, isEmpty#136] -Results [10]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, sum#137, isEmpty#138] +Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#83, isEmpty#84] +Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#85, isEmpty#86] (99) Exchange -Input [10]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, sum#137, isEmpty#138] -Arguments: hashpartitioning(c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, 5), ENSURE_REQUIREMENTS, [id=#139] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#85, isEmpty#86] +Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#87] (100) HashAggregate [codegen id : 42] -Input [10]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, sum#137, isEmpty#138] -Keys [8]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125] -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))#140] -Results [2]: [c_customer_id#128 AS customer_id#141, 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))#140 AS year_total#142] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#85, isEmpty#86] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 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#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#88] +Results [2]: [c_customer_id#13 AS customer_id#89, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#88 AS year_total#90] (101) Filter [codegen id : 42] -Input [2]: [customer_id#141, year_total#142] -Condition : (isnotnull(year_total#142) AND (year_total#142 > 0.000000)) +Input [2]: [customer_id#89, year_total#90] +Condition : (isnotnull(year_total#90) AND (year_total#90 > 0.000000)) (102) Project [codegen id : 42] -Output [2]: [customer_id#141 AS customer_id#143, year_total#142 AS year_total#144] -Input [2]: [customer_id#141, year_total#142] +Output [2]: [customer_id#89 AS customer_id#91, year_total#90 AS year_total#92] +Input [2]: [customer_id#89, year_total#90] (103) Exchange -Input [2]: [customer_id#143, year_total#144] -Arguments: hashpartitioning(customer_id#143, 5), ENSURE_REQUIREMENTS, [id=#145] +Input [2]: [customer_id#91, year_total#92] +Arguments: hashpartitioning(customer_id#91, 5), ENSURE_REQUIREMENTS, [id=#93] (104) Sort [codegen id : 43] -Input [2]: [customer_id#143, year_total#144] -Arguments: [customer_id#143 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#91, year_total#92] +Arguments: [customer_id#91 ASC NULLS FIRST], false, 0 (105) SortMergeJoin [codegen id : 44] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#143] +Right keys [1]: [customer_id#91] Join condition: None (106) Project [codegen id : 44] -Output [11]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116, year_total#144] -Input [12]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116, customer_id#143, year_total#144] +Output [11]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74, year_total#92] +Input [12]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74, customer_id#91, year_total#92] (107) 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#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#151), dynamicpruningexpression(ws_sold_date_sk#151 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 46] -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#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] (109) Filter [codegen id : 46] -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#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Condition : isnotnull(ws_bill_customer_sk#76) (110) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#152, d_year#153] +Output [2]: [d_date_sk#8, d_year#9] (111) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [ws_sold_date_sk#151] -Right keys [1]: [d_date_sk#152] +Left keys [1]: [ws_sold_date_sk#81] +Right keys [1]: [d_date_sk#8] Join condition: None (112) Project [codegen id : 46] -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, d_year#153] -Input [8]: [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, d_date_sk#152, d_year#153] +Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Input [8]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#8, d_year#9] (113) Exchange -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, d_year#153] -Arguments: hashpartitioning(ws_bill_customer_sk#146, 5), ENSURE_REQUIREMENTS, [id=#154] +Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Arguments: hashpartitioning(ws_bill_customer_sk#76, 5), ENSURE_REQUIREMENTS, [id=#94] (114) Sort [codegen id : 47] -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, d_year#153] -Arguments: [ws_bill_customer_sk#146 ASC NULLS FIRST], false, 0 +Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Arguments: [ws_bill_customer_sk#76 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] +Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (116) Sort [codegen id : 49] -Input [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] -Arguments: [c_customer_sk#155 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 (117) SortMergeJoin [codegen id : 50] -Left keys [1]: [ws_bill_customer_sk#146] -Right keys [1]: [c_customer_sk#155] +Left keys [1]: [ws_bill_customer_sk#76] +Right keys [1]: [c_customer_sk#12] Join condition: None (118) Project [codegen id : 50] -Output [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] -Input [14]: [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, d_year#153, c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] +Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Input [14]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (119) HashAggregate [codegen id : 50] -Input [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] -Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] -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#163, isEmpty#164] -Results [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] +Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#95, isEmpty#96] +Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#97, isEmpty#98] (120) Exchange -Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] -Arguments: hashpartitioning(c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, 5), ENSURE_REQUIREMENTS, [id=#167] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#97, isEmpty#98] +Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#99] (121) HashAggregate [codegen id : 51] -Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] -Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] -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))#168] -Results [2]: [c_customer_id#156 AS customer_id#169, 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))#168 AS year_total#170] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#97, isEmpty#98] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 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#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#100] +Results [2]: [c_customer_id#13 AS customer_id#101, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#100 AS year_total#102] (122) Exchange -Input [2]: [customer_id#169, year_total#170] -Arguments: hashpartitioning(customer_id#169, 5), ENSURE_REQUIREMENTS, [id=#171] +Input [2]: [customer_id#101, year_total#102] +Arguments: hashpartitioning(customer_id#101, 5), ENSURE_REQUIREMENTS, [id=#103] (123) Sort [codegen id : 52] -Input [2]: [customer_id#169, year_total#170] -Arguments: [customer_id#169 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#101, year_total#102] +Arguments: [customer_id#101 ASC NULLS FIRST], false, 0 (124) SortMergeJoin [codegen id : 53] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#169] -Join condition: (CASE WHEN (year_total#90 > 0.000000) THEN CheckOverflow((promote_precision(year_total#116) / promote_precision(year_total#90)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#144 > 0.000000) THEN CheckOverflow((promote_precision(year_total#170) / promote_precision(year_total#144)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#101] +Join condition: (CASE WHEN (year_total#64 > 0.000000) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#64)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#92 > 0.000000) THEN CheckOverflow((promote_precision(year_total#102) / promote_precision(year_total#92)), DecimalType(38,14), true) ELSE null END) (125) Project [codegen id : 53] -Output [7]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61] -Input [13]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116, year_total#144, customer_id#169, year_total#170] +Output [7]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45] +Input [13]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74, year_total#92, customer_id#101, year_total#102] (126) TakeOrderedAndProject -Input [7]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61] -Arguments: 100, [customer_id#55 ASC NULLS FIRST, customer_first_name#56 ASC NULLS FIRST, customer_last_name#57 ASC NULLS FIRST, customer_preferred_cust_flag#58 ASC NULLS FIRST, customer_birth_country#59 ASC NULLS FIRST, customer_login#60 ASC NULLS FIRST, customer_email_address#61 ASC NULLS FIRST], [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61] +Input [7]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45] +Arguments: 100, [customer_id#39 ASC NULLS FIRST, customer_first_name#40 ASC NULLS FIRST, customer_last_name#41 ASC NULLS FIRST, customer_preferred_cust_flag#42 ASC NULLS FIRST, customer_birth_country#43 ASC NULLS FIRST, customer_login#44 ASC NULLS FIRST, customer_email_address#45 ASC NULLS FIRST], [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45] ===== Subqueries ===== @@ -708,19 +708,19 @@ ReusedExchange (127) (127) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#8, d_year#9] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#30 ReusedExchange (128) (128) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#37, d_year#38] +Output [2]: [d_date_sk#8, d_year#9] -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#69 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#7 -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#97 IN dynamicpruning#36 +Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#30 -Subquery:5 Hosting operator id = 86 Hosting Expression = ws_sold_date_sk#123 IN dynamicpruning#7 +Subquery:5 Hosting operator id = 86 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#7 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#151 IN dynamicpruning#36 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#30 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 1cf27630d2994..80dee3d164511 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 @@ -209,460 +209,460 @@ Input [2]: [customer_id#26, year_total#27] Condition : (isnotnull(year_total#27) AND (year_total#27 > 0.000000)) (20) Scan parquet default.customer -Output [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Condition : (isnotnull(c_customer_sk#28) AND isnotnull(c_customer_id#29)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (23) Scan parquet default.store_sales -Output [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] +Output [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#41), dynamicpruningexpression(ss_sold_date_sk#41 IN dynamicpruning#42)] +PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] +Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] (25) Filter [codegen id : 4] -Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Condition : isnotnull(ss_customer_sk#36) +Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Condition : isnotnull(ss_customer_sk#9) (26) BroadcastExchange -Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] +Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#28] -Right keys [1]: [ss_customer_sk#36] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#9] Join condition: None (28) Project [codegen id : 6] -Output [12]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Input [14]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#44, d_year#45] +Output [2]: [d_date_sk#17, d_year#18] 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 (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#44, d_year#45] +Input [2]: [d_date_sk#17, d_year#18] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#44, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) +Input [2]: [d_date_sk#17, d_year#18] +Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2002)) AND isnotnull(d_date_sk#17)) (32) BroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#46] +Input [2]: [d_date_sk#17, d_year#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#41] -Right keys [1]: [d_date_sk#44] +Left keys [1]: [ss_sold_date_sk#14] +Right keys [1]: [d_date_sk#17] Join condition: None (34) Project [codegen id : 6] -Output [12]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#45] -Input [14]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41, d_date_sk#44, d_year#45] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#18] +Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14, d_date_sk#17, d_year#18] (35) HashAggregate [codegen id : 6] -Input [12]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#45] -Keys [8]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#47, isEmpty#48] -Results [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, sum#49, isEmpty#50] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#18] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#31, isEmpty#32] +Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#33, isEmpty#34] (36) Exchange -Input [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, sum#49, isEmpty#50] -Arguments: hashpartitioning(c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#33, isEmpty#34] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#35] (37) HashAggregate [codegen id : 7] -Input [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, sum#49, isEmpty#50] -Keys [8]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 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#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#52] -Results [8]: [c_customer_id#29 AS customer_id#53, c_first_name#30 AS customer_first_name#54, c_last_name#31 AS customer_last_name#55, c_preferred_cust_flag#32 AS customer_preferred_cust_flag#56, c_birth_country#33 AS customer_birth_country#57, c_login#34 AS customer_login#58, c_email_address#35 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#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#52 AS year_total#60] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#33, isEmpty#34] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 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#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#36] +Results [8]: [c_customer_id#2 AS customer_id#37, c_first_name#3 AS customer_first_name#38, c_last_name#4 AS customer_last_name#39, c_preferred_cust_flag#5 AS customer_preferred_cust_flag#40, c_birth_country#6 AS customer_birth_country#41, c_login#7 AS customer_login#42, c_email_address#8 AS customer_email_address#43, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#36 AS year_total#44] (38) BroadcastExchange -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: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#61] +Input [8]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#45] (39) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#53] +Right keys [1]: [customer_id#37] Join condition: None (40) Scan parquet default.customer -Output [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (42) Filter [codegen id : 10] -Input [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] -Condition : (isnotnull(c_customer_sk#62) AND isnotnull(c_customer_id#63)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (43) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] +Output [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#75), dynamicpruningexpression(cs_sold_date_sk#75 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cs_sold_date_sk#51), dynamicpruningexpression(cs_sold_date_sk#51 IN dynamicpruning#15)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] +Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] (45) Filter [codegen id : 8] -Input [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] -Condition : isnotnull(cs_bill_customer_sk#70) +Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Condition : isnotnull(cs_bill_customer_sk#46) (46) BroadcastExchange -Input [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] +Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#62] -Right keys [1]: [cs_bill_customer_sk#70] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [cs_bill_customer_sk#46] Join condition: None (48) Project [codegen id : 10] -Output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] -Input [14]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#77, d_year#78] +Output [2]: [d_date_sk#17, d_year#18] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#75] -Right keys [1]: [d_date_sk#77] +Left keys [1]: [cs_sold_date_sk#51] +Right keys [1]: [d_date_sk#17] Join condition: None (51) Project [codegen id : 10] -Output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, d_year#78] -Input [14]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75, d_date_sk#77, d_year#78] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] +Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51, d_date_sk#17, d_year#18] (52) HashAggregate [codegen id : 10] -Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, d_year#78] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 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#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, sum#81, isEmpty#82] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#53, isEmpty#54] +Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#55, isEmpty#56] (53) Exchange -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, sum#81, isEmpty#82] -Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#55, isEmpty#56] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#57] (54) HashAggregate [codegen id : 11] -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, sum#81, isEmpty#82] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 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#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#84] -Results [2]: [c_customer_id#63 AS customer_id#85, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#84 AS year_total#86] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#55, isEmpty#56] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 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#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#58] +Results [2]: [c_customer_id#2 AS customer_id#59, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#58 AS year_total#60] (55) Filter [codegen id : 11] -Input [2]: [customer_id#85, year_total#86] -Condition : (isnotnull(year_total#86) AND (year_total#86 > 0.000000)) +Input [2]: [customer_id#59, year_total#60] +Condition : (isnotnull(year_total#60) AND (year_total#60 > 0.000000)) (56) Project [codegen id : 11] -Output [2]: [customer_id#85 AS customer_id#87, year_total#86 AS year_total#88] -Input [2]: [customer_id#85, year_total#86] +Output [2]: [customer_id#59 AS customer_id#61, year_total#60 AS year_total#62] +Input [2]: [customer_id#59, year_total#60] (57) BroadcastExchange -Input [2]: [customer_id#87, year_total#88] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#89] +Input [2]: [customer_id#61, year_total#62] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#63] (58) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#87] +Right keys [1]: [customer_id#61] Join condition: None (59) Project [codegen id : 24] -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#88] -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#87, year_total#88] +Output [11]: [customer_id#26, year_total#27, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44, year_total#62] +Input [12]: [customer_id#26, year_total#27, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44, customer_id#61, year_total#62] (60) Scan parquet default.customer -Output [8]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (62) Filter [codegen id : 14] -Input [8]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97] -Condition : (isnotnull(c_customer_sk#90) AND isnotnull(c_customer_id#91)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (63) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] +Output [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#103), dynamicpruningexpression(cs_sold_date_sk#103 IN dynamicpruning#42)] +PartitionFilters: [isnotnull(cs_sold_date_sk#51), dynamicpruningexpression(cs_sold_date_sk#51 IN dynamicpruning#28)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] +Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] (65) Filter [codegen id : 12] -Input [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] -Condition : isnotnull(cs_bill_customer_sk#98) +Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Condition : isnotnull(cs_bill_customer_sk#46) (66) BroadcastExchange -Input [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#104] +Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#90] -Right keys [1]: [cs_bill_customer_sk#98] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [cs_bill_customer_sk#46] Join condition: None (68) Project [codegen id : 14] -Output [12]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] -Input [14]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#105, d_year#106] +Output [2]: [d_date_sk#17, d_year#18] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#103] -Right keys [1]: [d_date_sk#105] +Left keys [1]: [cs_sold_date_sk#51] +Right keys [1]: [d_date_sk#17] Join condition: None (71) Project [codegen id : 14] -Output [12]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, d_year#106] -Input [14]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103, d_date_sk#105, d_year#106] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] +Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51, d_date_sk#17, d_year#18] (72) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, d_year#106] -Keys [8]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#107, isEmpty#108] -Results [10]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, sum#109, isEmpty#110] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#67, isEmpty#68] (73) Exchange -Input [10]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, sum#109, isEmpty#110] -Arguments: hashpartitioning(c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, 5), ENSURE_REQUIREMENTS, [id=#111] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#67, isEmpty#68] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#69] (74) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, sum#109, isEmpty#110] -Keys [8]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 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#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112] -Results [2]: [c_customer_id#91 AS customer_id#113, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112 AS year_total#114] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#67, isEmpty#68] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 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#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#70] +Results [2]: [c_customer_id#2 AS customer_id#71, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#70 AS year_total#72] (75) BroadcastExchange -Input [2]: [customer_id#113, year_total#114] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#115] +Input [2]: [customer_id#71, year_total#72] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#73] (76) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#113] -Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), 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#71] +Join condition: (CASE WHEN (year_total#62 > 0.000000) THEN CheckOverflow((promote_precision(year_total#72) / promote_precision(year_total#62)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#27 > 0.000000) THEN CheckOverflow((promote_precision(year_total#44) / promote_precision(year_total#27)), DecimalType(38,14), true) ELSE null END) (77) Project [codegen id : 24] -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#88, year_total#114] -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#88, customer_id#113, year_total#114] +Output [10]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72] +Input [13]: [customer_id#26, year_total#27, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44, year_total#62, customer_id#71, year_total#72] (78) Scan parquet default.customer -Output [8]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (79) ColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (80) Filter [codegen id : 18] -Input [8]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123] -Condition : (isnotnull(c_customer_sk#116) AND isnotnull(c_customer_id#117)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (81) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] +Output [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#129), dynamicpruningexpression(ws_sold_date_sk#129 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#15)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (82) ColumnarToRow [codegen id : 16] -Input [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] +Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] (83) Filter [codegen id : 16] -Input [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] -Condition : isnotnull(ws_bill_customer_sk#124) +Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Condition : isnotnull(ws_bill_customer_sk#74) (84) BroadcastExchange -Input [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#130] +Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#80] (85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#116] -Right keys [1]: [ws_bill_customer_sk#124] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#74] Join condition: None (86) Project [codegen id : 18] -Output [12]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] -Input [14]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] (87) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#131, d_year#132] +Output [2]: [d_date_sk#17, d_year#18] (88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#129] -Right keys [1]: [d_date_sk#131] +Left keys [1]: [ws_sold_date_sk#79] +Right keys [1]: [d_date_sk#17] Join condition: None (89) Project [codegen id : 18] -Output [12]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, d_year#132] -Input [14]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129, d_date_sk#131, d_year#132] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] +Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#17, d_year#18] (90) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, d_year#132] -Keys [8]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#133, isEmpty#134] -Results [10]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, sum#135, isEmpty#136] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#81, isEmpty#82] +Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#83, isEmpty#84] (91) Exchange -Input [10]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, sum#135, isEmpty#136] -Arguments: hashpartitioning(c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#83, isEmpty#84] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#85] (92) HashAggregate [codegen id : 19] -Input [10]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, sum#135, isEmpty#136] -Keys [8]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 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#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138] -Results [2]: [c_customer_id#117 AS customer_id#139, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138 AS year_total#140] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#83, isEmpty#84] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 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#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86] +Results [2]: [c_customer_id#2 AS customer_id#87, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86 AS year_total#88] (93) Filter [codegen id : 19] -Input [2]: [customer_id#139, year_total#140] -Condition : (isnotnull(year_total#140) AND (year_total#140 > 0.000000)) +Input [2]: [customer_id#87, year_total#88] +Condition : (isnotnull(year_total#88) AND (year_total#88 > 0.000000)) (94) Project [codegen id : 19] -Output [2]: [customer_id#139 AS customer_id#141, year_total#140 AS year_total#142] -Input [2]: [customer_id#139, year_total#140] +Output [2]: [customer_id#87 AS customer_id#89, year_total#88 AS year_total#90] +Input [2]: [customer_id#87, year_total#88] (95) BroadcastExchange -Input [2]: [customer_id#141, year_total#142] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#143] +Input [2]: [customer_id#89, year_total#90] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#91] (96) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#141] +Right keys [1]: [customer_id#89] Join condition: None (97) Project [codegen id : 24] -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#88, year_total#114, year_total#142] -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#88, year_total#114, customer_id#141, year_total#142] +Output [11]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72, year_total#90] +Input [12]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72, customer_id#89, year_total#90] (98) Scan parquet default.customer -Output [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (99) ColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (100) Filter [codegen id : 22] -Input [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] -Condition : (isnotnull(c_customer_sk#144) AND isnotnull(c_customer_id#145)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (101) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Output [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#157), dynamicpruningexpression(ws_sold_date_sk#157 IN dynamicpruning#42)] +PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 20] -Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] (103) Filter [codegen id : 20] -Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] -Condition : isnotnull(ws_bill_customer_sk#152) +Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Condition : isnotnull(ws_bill_customer_sk#74) (104) BroadcastExchange -Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#158] +Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] (105) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#144] -Right keys [1]: [ws_bill_customer_sk#152] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#74] Join condition: None (106) Project [codegen id : 22] -Output [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] -Input [14]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] (107) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#159, d_year#160] +Output [2]: [d_date_sk#17, d_year#18] (108) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#157] -Right keys [1]: [d_date_sk#159] +Left keys [1]: [ws_sold_date_sk#79] +Right keys [1]: [d_date_sk#17] Join condition: None (109) Project [codegen id : 22] -Output [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, d_year#160] -Input [14]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157, d_date_sk#159, d_year#160] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] +Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#17, d_year#18] (110) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, d_year#160] -Keys [8]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#161, isEmpty#162] -Results [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#93, isEmpty#94] +Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#95, isEmpty#96] (111) Exchange -Input [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] -Arguments: hashpartitioning(c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, 5), ENSURE_REQUIREMENTS, [id=#165] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#95, isEmpty#96] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#97] (112) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] -Keys [8]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 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#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166] -Results [2]: [c_customer_id#145 AS customer_id#167, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166 AS year_total#168] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#95, isEmpty#96] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 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#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#98] +Results [2]: [c_customer_id#2 AS customer_id#99, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#98 AS year_total#100] (113) BroadcastExchange -Input [2]: [customer_id#167, year_total#168] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#169] +Input [2]: [customer_id#99, year_total#100] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#101] (114) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#167] -Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#142 > 0.000000) THEN CheckOverflow((promote_precision(year_total#168) / promote_precision(year_total#142)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#99] +Join condition: (CASE WHEN (year_total#62 > 0.000000) THEN CheckOverflow((promote_precision(year_total#72) / promote_precision(year_total#62)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#90 > 0.000000) THEN CheckOverflow((promote_precision(year_total#100) / promote_precision(year_total#90)), DecimalType(38,14), true) ELSE null END) (115) Project [codegen id : 24] -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#88, year_total#114, year_total#142, customer_id#167, year_total#168] +Output [7]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43] +Input [13]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72, year_total#90, customer_id#99, year_total#100] (116) 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#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43] +Arguments: 100, [customer_id#37 ASC NULLS FIRST, customer_first_name#38 ASC NULLS FIRST, customer_last_name#39 ASC NULLS FIRST, customer_preferred_cust_flag#40 ASC NULLS FIRST, customer_birth_country#41 ASC NULLS FIRST, customer_login#42 ASC NULLS FIRST, customer_email_address#43 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43] ===== Subqueries ===== @@ -673,19 +673,19 @@ ReusedExchange (117) (117) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#17, d_year#18] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#41 IN dynamicpruning#42 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#28 ReusedExchange (118) (118) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#44, d_year#45] +Output [2]: [d_date_sk#17, d_year#18] -Subquery:3 Hosting operator id = 43 Hosting Expression = cs_sold_date_sk#75 IN dynamicpruning#15 +Subquery:3 Hosting operator id = 43 Hosting Expression = cs_sold_date_sk#51 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 63 Hosting Expression = cs_sold_date_sk#103 IN dynamicpruning#42 +Subquery:4 Hosting operator id = 63 Hosting Expression = cs_sold_date_sk#51 IN dynamicpruning#28 -Subquery:5 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#129 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#157 IN dynamicpruning#42 +Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#28 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt index a6270c8ab533a..35924d2ffd9a9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt @@ -102,98 +102,98 @@ Output [2]: [item_sk#11, rnk#17] Input [3]: [item_sk#11, rank_col#12, rnk#17] (15) ReusedExchange [Reuses operator id: 6] -Output [3]: [ss_item_sk#18, sum#19, count#20] +Output [3]: [ss_item_sk#1, sum#18, count#19] (16) HashAggregate [codegen id : 5] -Input [3]: [ss_item_sk#18, sum#19, count#20] -Keys [1]: [ss_item_sk#18] -Functions [1]: [avg(UnscaledValue(ss_net_profit#21))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#21))#22] -Results [3]: [ss_item_sk#18 AS item_sk#23, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS rank_col#24, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#21)#25] +Input [3]: [ss_item_sk#1, sum#18, count#19] +Keys [1]: [ss_item_sk#1] +Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#20] +Results [3]: [ss_item_sk#1 AS item_sk#21, cast((avg(UnscaledValue(ss_net_profit#3))#20 / 100.0) as decimal(11,6)) AS rank_col#22, cast((avg(UnscaledValue(ss_net_profit#3))#20 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#3)#23] (17) Filter [codegen id : 5] -Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] -Condition : (isnotnull(avg(ss_net_profit#21)#25) AND (cast(avg(ss_net_profit#21)#25 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) +Input [3]: [item_sk#21, rank_col#22, avg(ss_net_profit#3)#23] +Condition : (isnotnull(avg(ss_net_profit#3)#23) AND (cast(avg(ss_net_profit#3)#23 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) (18) Project [codegen id : 5] -Output [2]: [item_sk#23, rank_col#24] -Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] +Output [2]: [item_sk#21, rank_col#22] +Input [3]: [item_sk#21, rank_col#22, avg(ss_net_profit#3)#23] (19) Exchange -Input [2]: [item_sk#23, rank_col#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [item_sk#21, rank_col#22] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] (20) Sort [codegen id : 6] -Input [2]: [item_sk#23, rank_col#24] -Arguments: [rank_col#24 DESC NULLS LAST], false, 0 +Input [2]: [item_sk#21, rank_col#22] +Arguments: [rank_col#22 DESC NULLS LAST], false, 0 (21) Window -Input [2]: [item_sk#23, rank_col#24] -Arguments: [rank(rank_col#24) windowspecdefinition(rank_col#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#27], [rank_col#24 DESC NULLS LAST] +Input [2]: [item_sk#21, rank_col#22] +Arguments: [rank(rank_col#22) windowspecdefinition(rank_col#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#25], [rank_col#22 DESC NULLS LAST] (22) Filter [codegen id : 7] -Input [3]: [item_sk#23, rank_col#24, rnk#27] -Condition : ((isnotnull(rnk#27) AND (rnk#27 < 11)) AND isnotnull(item_sk#23)) +Input [3]: [item_sk#21, rank_col#22, rnk#25] +Condition : ((isnotnull(rnk#25) AND (rnk#25 < 11)) AND isnotnull(item_sk#21)) (23) Project [codegen id : 7] -Output [2]: [item_sk#23, rnk#27] -Input [3]: [item_sk#23, rank_col#24, rnk#27] +Output [2]: [item_sk#21, rnk#25] +Input [3]: [item_sk#21, rank_col#22, rnk#25] (24) BroadcastExchange -Input [2]: [item_sk#23, rnk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#28] +Input [2]: [item_sk#21, rnk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#26] (25) BroadcastHashJoin [codegen id : 10] Left keys [1]: [rnk#17] -Right keys [1]: [rnk#27] +Right keys [1]: [rnk#25] Join condition: None (26) Project [codegen id : 10] -Output [3]: [item_sk#11, rnk#17, item_sk#23] -Input [4]: [item_sk#11, rnk#17, item_sk#23, rnk#27] +Output [3]: [item_sk#11, rnk#17, item_sk#21] +Input [4]: [item_sk#11, rnk#17, item_sk#21, rnk#25] (27) Scan parquet default.item -Output [2]: [i_item_sk#29, i_product_name#30] +Output [2]: [i_item_sk#27, i_product_name#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 8] -Input [2]: [i_item_sk#29, i_product_name#30] +Input [2]: [i_item_sk#27, i_product_name#28] (29) Filter [codegen id : 8] -Input [2]: [i_item_sk#29, i_product_name#30] -Condition : isnotnull(i_item_sk#29) +Input [2]: [i_item_sk#27, i_product_name#28] +Condition : isnotnull(i_item_sk#27) (30) BroadcastExchange -Input [2]: [i_item_sk#29, i_product_name#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [2]: [i_item_sk#27, i_product_name#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [item_sk#11] -Right keys [1]: [i_item_sk#29] +Right keys [1]: [i_item_sk#27] Join condition: None (32) Project [codegen id : 10] -Output [3]: [rnk#17, item_sk#23, i_product_name#30] -Input [5]: [item_sk#11, rnk#17, item_sk#23, i_item_sk#29, i_product_name#30] +Output [3]: [rnk#17, item_sk#21, i_product_name#28] +Input [5]: [item_sk#11, rnk#17, item_sk#21, i_item_sk#27, i_product_name#28] (33) ReusedExchange [Reuses operator id: 30] -Output [2]: [i_item_sk#32, i_product_name#33] +Output [2]: [i_item_sk#30, i_product_name#31] (34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [item_sk#23] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [item_sk#21] +Right keys [1]: [i_item_sk#30] Join condition: None (35) Project [codegen id : 10] -Output [3]: [rnk#17, i_product_name#30 AS best_performing#34, i_product_name#33 AS worst_performing#35] -Input [5]: [rnk#17, item_sk#23, i_product_name#30, i_item_sk#32, i_product_name#33] +Output [3]: [rnk#17, i_product_name#28 AS best_performing#32, i_product_name#31 AS worst_performing#33] +Input [5]: [rnk#17, item_sk#21, i_product_name#28, i_item_sk#30, i_product_name#31] (36) TakeOrderedAndProject -Input [3]: [rnk#17, best_performing#34, worst_performing#35] -Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#34, worst_performing#35] +Input [3]: [rnk#17, best_performing#32, worst_performing#33] +Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#32, worst_performing#33] ===== Subqueries ===== @@ -208,40 +208,40 @@ Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery (37) Scan parquet default.store_sales -Output [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] +Output [4]: [ss_addr_sk#34, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 1] -Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] +Input [4]: [ss_addr_sk#34, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] (39) Filter [codegen id : 1] -Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] -Condition : ((isnotnull(ss_store_sk#37) AND (ss_store_sk#37 = 4)) AND isnull(ss_addr_sk#36)) +Input [4]: [ss_addr_sk#34, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) AND isnull(ss_addr_sk#34)) (40) Project [codegen id : 1] -Output [2]: [ss_store_sk#37, ss_net_profit#38] -Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] +Output [2]: [ss_store_sk#2, ss_net_profit#3] +Input [4]: [ss_addr_sk#34, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] (41) HashAggregate [codegen id : 1] -Input [2]: [ss_store_sk#37, ss_net_profit#38] -Keys [1]: [ss_store_sk#37] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#38))] -Aggregate Attributes [2]: [sum#40, count#41] -Results [3]: [ss_store_sk#37, sum#42, count#43] +Input [2]: [ss_store_sk#2, ss_net_profit#3] +Keys [1]: [ss_store_sk#2] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] +Aggregate Attributes [2]: [sum#35, count#36] +Results [3]: [ss_store_sk#2, sum#37, count#38] (42) Exchange -Input [3]: [ss_store_sk#37, sum#42, count#43] -Arguments: hashpartitioning(ss_store_sk#37, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [3]: [ss_store_sk#2, sum#37, count#38] +Arguments: hashpartitioning(ss_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#39] (43) HashAggregate [codegen id : 2] -Input [3]: [ss_store_sk#37, sum#42, count#43] -Keys [1]: [ss_store_sk#37] -Functions [1]: [avg(UnscaledValue(ss_net_profit#38))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#38))#45] -Results [1]: [cast((avg(UnscaledValue(ss_net_profit#38))#45 / 100.0) as decimal(11,6)) AS rank_col#46] +Input [3]: [ss_store_sk#2, sum#37, count#38] +Keys [1]: [ss_store_sk#2] +Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#40] +Results [1]: [cast((avg(UnscaledValue(ss_net_profit#3))#40 / 100.0) as decimal(11,6)) AS rank_col#41] Subquery:2 Hosting operator id = 17 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index a220cf9ff546f..4ca133ffd838d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -107,98 +107,98 @@ Input [2]: [item_sk#11, rnk#17] Arguments: [rnk#17 ASC NULLS FIRST], false, 0 (16) ReusedExchange [Reuses operator id: 6] -Output [3]: [ss_item_sk#18, sum#19, count#20] +Output [3]: [ss_item_sk#1, sum#18, count#19] (17) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#18, sum#19, count#20] -Keys [1]: [ss_item_sk#18] -Functions [1]: [avg(UnscaledValue(ss_net_profit#21))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#21))#22] -Results [3]: [ss_item_sk#18 AS item_sk#23, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS rank_col#24, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#21)#25] +Input [3]: [ss_item_sk#1, sum#18, count#19] +Keys [1]: [ss_item_sk#1] +Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#20] +Results [3]: [ss_item_sk#1 AS item_sk#21, cast((avg(UnscaledValue(ss_net_profit#3))#20 / 100.0) as decimal(11,6)) AS rank_col#22, cast((avg(UnscaledValue(ss_net_profit#3))#20 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#3)#23] (18) Filter [codegen id : 6] -Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] -Condition : (isnotnull(avg(ss_net_profit#21)#25) AND (cast(avg(ss_net_profit#21)#25 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) +Input [3]: [item_sk#21, rank_col#22, avg(ss_net_profit#3)#23] +Condition : (isnotnull(avg(ss_net_profit#3)#23) AND (cast(avg(ss_net_profit#3)#23 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) (19) Project [codegen id : 6] -Output [2]: [item_sk#23, rank_col#24] -Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] +Output [2]: [item_sk#21, rank_col#22] +Input [3]: [item_sk#21, rank_col#22, avg(ss_net_profit#3)#23] (20) Exchange -Input [2]: [item_sk#23, rank_col#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [item_sk#21, rank_col#22] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] (21) Sort [codegen id : 7] -Input [2]: [item_sk#23, rank_col#24] -Arguments: [rank_col#24 DESC NULLS LAST], false, 0 +Input [2]: [item_sk#21, rank_col#22] +Arguments: [rank_col#22 DESC NULLS LAST], false, 0 (22) Window -Input [2]: [item_sk#23, rank_col#24] -Arguments: [rank(rank_col#24) windowspecdefinition(rank_col#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#27], [rank_col#24 DESC NULLS LAST] +Input [2]: [item_sk#21, rank_col#22] +Arguments: [rank(rank_col#22) windowspecdefinition(rank_col#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#25], [rank_col#22 DESC NULLS LAST] (23) Filter [codegen id : 8] -Input [3]: [item_sk#23, rank_col#24, rnk#27] -Condition : ((isnotnull(rnk#27) AND (rnk#27 < 11)) AND isnotnull(item_sk#23)) +Input [3]: [item_sk#21, rank_col#22, rnk#25] +Condition : ((isnotnull(rnk#25) AND (rnk#25 < 11)) AND isnotnull(item_sk#21)) (24) Project [codegen id : 8] -Output [2]: [item_sk#23, rnk#27] -Input [3]: [item_sk#23, rank_col#24, rnk#27] +Output [2]: [item_sk#21, rnk#25] +Input [3]: [item_sk#21, rank_col#22, rnk#25] (25) Sort [codegen id : 8] -Input [2]: [item_sk#23, rnk#27] -Arguments: [rnk#27 ASC NULLS FIRST], false, 0 +Input [2]: [item_sk#21, rnk#25] +Arguments: [rnk#25 ASC NULLS FIRST], false, 0 (26) SortMergeJoin [codegen id : 11] Left keys [1]: [rnk#17] -Right keys [1]: [rnk#27] +Right keys [1]: [rnk#25] Join condition: None (27) Project [codegen id : 11] -Output [3]: [item_sk#11, rnk#17, item_sk#23] -Input [4]: [item_sk#11, rnk#17, item_sk#23, rnk#27] +Output [3]: [item_sk#11, rnk#17, item_sk#21] +Input [4]: [item_sk#11, rnk#17, item_sk#21, rnk#25] (28) Scan parquet default.item -Output [2]: [i_item_sk#28, i_product_name#29] +Output [2]: [i_item_sk#26, i_product_name#27] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_sk#28, i_product_name#29] +Input [2]: [i_item_sk#26, i_product_name#27] (30) Filter [codegen id : 9] -Input [2]: [i_item_sk#28, i_product_name#29] -Condition : isnotnull(i_item_sk#28) +Input [2]: [i_item_sk#26, i_product_name#27] +Condition : isnotnull(i_item_sk#26) (31) BroadcastExchange -Input [2]: [i_item_sk#28, i_product_name#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [2]: [i_item_sk#26, i_product_name#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] (32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [item_sk#11] -Right keys [1]: [i_item_sk#28] +Right keys [1]: [i_item_sk#26] Join condition: None (33) Project [codegen id : 11] -Output [3]: [rnk#17, item_sk#23, i_product_name#29] -Input [5]: [item_sk#11, rnk#17, item_sk#23, i_item_sk#28, i_product_name#29] +Output [3]: [rnk#17, item_sk#21, i_product_name#27] +Input [5]: [item_sk#11, rnk#17, item_sk#21, i_item_sk#26, i_product_name#27] (34) ReusedExchange [Reuses operator id: 31] -Output [2]: [i_item_sk#31, i_product_name#32] +Output [2]: [i_item_sk#29, i_product_name#30] (35) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [item_sk#23] -Right keys [1]: [i_item_sk#31] +Left keys [1]: [item_sk#21] +Right keys [1]: [i_item_sk#29] Join condition: None (36) Project [codegen id : 11] -Output [3]: [rnk#17, i_product_name#29 AS best_performing#33, i_product_name#32 AS worst_performing#34] -Input [5]: [rnk#17, item_sk#23, i_product_name#29, i_item_sk#31, i_product_name#32] +Output [3]: [rnk#17, i_product_name#27 AS best_performing#31, i_product_name#30 AS worst_performing#32] +Input [5]: [rnk#17, item_sk#21, i_product_name#27, i_item_sk#29, i_product_name#30] (37) TakeOrderedAndProject -Input [3]: [rnk#17, best_performing#33, worst_performing#34] -Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#33, worst_performing#34] +Input [3]: [rnk#17, best_performing#31, worst_performing#32] +Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#31, worst_performing#32] ===== Subqueries ===== @@ -213,40 +213,40 @@ Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery (38) Scan parquet default.store_sales -Output [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] +Output [4]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 1] -Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] +Input [4]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] (40) Filter [codegen id : 1] -Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_store_sk#36) AND (ss_store_sk#36 = 4)) AND isnull(ss_addr_sk#35)) +Input [4]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) AND isnull(ss_addr_sk#33)) (41) Project [codegen id : 1] -Output [2]: [ss_store_sk#36, ss_net_profit#37] -Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] +Output [2]: [ss_store_sk#2, ss_net_profit#3] +Input [4]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] (42) HashAggregate [codegen id : 1] -Input [2]: [ss_store_sk#36, ss_net_profit#37] -Keys [1]: [ss_store_sk#36] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#37))] -Aggregate Attributes [2]: [sum#39, count#40] -Results [3]: [ss_store_sk#36, sum#41, count#42] +Input [2]: [ss_store_sk#2, ss_net_profit#3] +Keys [1]: [ss_store_sk#2] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] +Aggregate Attributes [2]: [sum#34, count#35] +Results [3]: [ss_store_sk#2, sum#36, count#37] (43) Exchange -Input [3]: [ss_store_sk#36, sum#41, count#42] -Arguments: hashpartitioning(ss_store_sk#36, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [3]: [ss_store_sk#2, sum#36, count#37] +Arguments: hashpartitioning(ss_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#38] (44) HashAggregate [codegen id : 2] -Input [3]: [ss_store_sk#36, sum#41, count#42] -Keys [1]: [ss_store_sk#36] -Functions [1]: [avg(UnscaledValue(ss_net_profit#37))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#37))#44] -Results [1]: [cast((avg(UnscaledValue(ss_net_profit#37))#44 / 100.0) as decimal(11,6)) AS rank_col#45] +Input [3]: [ss_store_sk#2, sum#36, count#37] +Keys [1]: [ss_store_sk#2] +Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#39] +Results [1]: [cast((avg(UnscaledValue(ss_net_profit#3))#39 / 100.0) as decimal(11,6)) AS rank_col#40] Subquery:2 Hosting operator id = 18 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt index 701414b22eb80..22f746eb1d91a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt @@ -199,30 +199,30 @@ Output [4]: [ws_sales_price#4, ca_city#19, ca_zip#20, i_item_id#12] Input [6]: [ws_bill_customer_sk#3, ws_sales_price#4, i_item_id#12, c_customer_sk#15, ca_city#19, ca_zip#20] (35) Scan parquet default.item -Output [2]: [i_item_sk#23, i_item_id#24] +Output [2]: [i_item_sk#11, i_item_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_item_sk, [2,3,5,7,11,13,17,19,23,29])] ReadSchema: struct (36) ColumnarToRow [codegen id : 11] -Input [2]: [i_item_sk#23, i_item_id#24] +Input [2]: [i_item_sk#11, i_item_id#12] (37) Filter [codegen id : 11] -Input [2]: [i_item_sk#23, i_item_id#24] -Condition : i_item_sk#23 IN (2,3,5,7,11,13,17,19,23,29) +Input [2]: [i_item_sk#11, i_item_id#12] +Condition : i_item_sk#11 IN (2,3,5,7,11,13,17,19,23,29) (38) Project [codegen id : 11] -Output [1]: [i_item_id#24] -Input [2]: [i_item_sk#23, i_item_id#24] +Output [1]: [i_item_id#12 AS i_item_id#12#23] +Input [2]: [i_item_sk#11, i_item_id#12] (39) BroadcastExchange -Input [1]: [i_item_id#24] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#25] +Input [1]: [i_item_id#12#23] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#24] (40) BroadcastHashJoin [codegen id : 12] Left keys [1]: [i_item_id#12] -Right keys [1]: [i_item_id#24] +Right keys [1]: [i_item_id#12#23] Join condition: None (41) Filter [codegen id : 12] @@ -237,23 +237,23 @@ Input [5]: [ws_sales_price#4, ca_city#19, ca_zip#20, i_item_id#12, exists#1] Input [3]: [ws_sales_price#4, ca_city#19, ca_zip#20] Keys [2]: [ca_zip#20, ca_city#19] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#26] -Results [3]: [ca_zip#20, ca_city#19, sum#27] +Aggregate Attributes [1]: [sum#25] +Results [3]: [ca_zip#20, ca_city#19, sum#26] (44) Exchange -Input [3]: [ca_zip#20, ca_city#19, sum#27] -Arguments: hashpartitioning(ca_zip#20, ca_city#19, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [ca_zip#20, ca_city#19, sum#26] +Arguments: hashpartitioning(ca_zip#20, ca_city#19, 5), ENSURE_REQUIREMENTS, [id=#27] (45) HashAggregate [codegen id : 13] -Input [3]: [ca_zip#20, ca_city#19, sum#27] +Input [3]: [ca_zip#20, ca_city#19, sum#26] Keys [2]: [ca_zip#20, ca_city#19] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#29] -Results [3]: [ca_zip#20, ca_city#19, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#29,17,2) AS sum(ws_sales_price)#30] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#28] +Results [3]: [ca_zip#20, ca_city#19, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#28,17,2) AS sum(ws_sales_price)#29] (46) TakeOrderedAndProject -Input [3]: [ca_zip#20, ca_city#19, sum(ws_sales_price)#30] -Arguments: 100, [ca_zip#20 ASC NULLS FIRST, ca_city#19 ASC NULLS FIRST], [ca_zip#20, ca_city#19, sum(ws_sales_price)#30] +Input [3]: [ca_zip#20, ca_city#19, sum(ws_sales_price)#29] +Arguments: 100, [ca_zip#20 ASC NULLS FIRST, ca_city#19 ASC NULLS FIRST], [ca_zip#20, ca_city#19, sum(ws_sales_price)#29] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt index f3a37f9e8767e..b81498ddcea34 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt @@ -169,30 +169,30 @@ Output [4]: [ws_sales_price#4, ca_city#11, ca_zip#12, i_item_id#19] Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#11, ca_zip#12, i_item_sk#18, i_item_id#19] (29) Scan parquet default.item -Output [2]: [i_item_sk#21, i_item_id#22] +Output [2]: [i_item_sk#18, i_item_id#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_item_sk, [2,3,5,7,11,13,17,19,23,29])] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] -Input [2]: [i_item_sk#21, i_item_id#22] +Input [2]: [i_item_sk#18, i_item_id#19] (31) Filter [codegen id : 5] -Input [2]: [i_item_sk#21, i_item_id#22] -Condition : i_item_sk#21 IN (2,3,5,7,11,13,17,19,23,29) +Input [2]: [i_item_sk#18, i_item_id#19] +Condition : i_item_sk#18 IN (2,3,5,7,11,13,17,19,23,29) (32) Project [codegen id : 5] -Output [1]: [i_item_id#22] -Input [2]: [i_item_sk#21, i_item_id#22] +Output [1]: [i_item_id#19 AS i_item_id#19#21] +Input [2]: [i_item_sk#18, i_item_id#19] (33) BroadcastExchange -Input [1]: [i_item_id#22] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#23] +Input [1]: [i_item_id#19#21] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#22] (34) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_id#19] -Right keys [1]: [i_item_id#22] +Right keys [1]: [i_item_id#19#21] Join condition: None (35) Filter [codegen id : 6] @@ -207,23 +207,23 @@ Input [5]: [ws_sales_price#4, ca_city#11, ca_zip#12, i_item_id#19, exists#1] Input [3]: [ws_sales_price#4, ca_city#11, ca_zip#12] Keys [2]: [ca_zip#12, ca_city#11] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#24] -Results [3]: [ca_zip#12, ca_city#11, sum#25] +Aggregate Attributes [1]: [sum#23] +Results [3]: [ca_zip#12, ca_city#11, sum#24] (38) Exchange -Input [3]: [ca_zip#12, ca_city#11, sum#25] -Arguments: hashpartitioning(ca_zip#12, ca_city#11, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [3]: [ca_zip#12, ca_city#11, sum#24] +Arguments: hashpartitioning(ca_zip#12, ca_city#11, 5), ENSURE_REQUIREMENTS, [id=#25] (39) HashAggregate [codegen id : 7] -Input [3]: [ca_zip#12, ca_city#11, sum#25] +Input [3]: [ca_zip#12, ca_city#11, sum#24] Keys [2]: [ca_zip#12, ca_city#11] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#27] -Results [3]: [ca_zip#12, ca_city#11, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#27,17,2) AS sum(ws_sales_price)#28] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#26] +Results [3]: [ca_zip#12, ca_city#11, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#26,17,2) AS sum(ws_sales_price)#27] (40) TakeOrderedAndProject -Input [3]: [ca_zip#12, ca_city#11, sum(ws_sales_price)#28] -Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#11 ASC NULLS FIRST], [ca_zip#12, ca_city#11, sum(ws_sales_price)#28] +Input [3]: [ca_zip#12, ca_city#11, sum(ws_sales_price)#27] +Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#11 ASC NULLS FIRST], [ca_zip#12, ca_city#11, sum(ws_sales_price)#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt index ac35640c7b004..78ade994364a7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt @@ -230,55 +230,55 @@ Input [5]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt Arguments: [ss_addr_sk#12 ASC NULLS FIRST], false, 0 (41) ReusedExchange [Reuses operator id: 9] -Output [2]: [ca_address_sk#31, ca_city#32] +Output [2]: [ca_address_sk#6, ca_city#7] (42) Sort [codegen id : 13] -Input [2]: [ca_address_sk#31, ca_city#32] -Arguments: [ca_address_sk#31 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#6, ca_city#7] +Arguments: [ca_address_sk#6 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 14] Left keys [1]: [ss_addr_sk#12] -Right keys [1]: [ca_address_sk#31] +Right keys [1]: [ca_address_sk#6] Join condition: None (44) Project [codegen id : 14] -Output [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#32] -Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_address_sk#31, ca_city#32] +Output [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#7] +Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_address_sk#6, ca_city#7] (45) HashAggregate [codegen id : 14] -Input [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#32] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32] +Input [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#7] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#15)), partial_sum(UnscaledValue(ss_net_profit#16))] -Aggregate Attributes [2]: [sum#33, sum#34] -Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32, sum#35, sum#36] +Aggregate Attributes [2]: [sum#31, sum#32] +Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#33, sum#34] (46) HashAggregate [codegen id : 14] -Input [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32, sum#35, sum#36] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32] +Input [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#33, sum#34] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#15)), sum(UnscaledValue(ss_net_profit#16))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#15))#37, sum(UnscaledValue(ss_net_profit#16))#38] -Results [5]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#32 AS bought_city#39, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#15))#37,17,2) AS amt#40, MakeDecimal(sum(UnscaledValue(ss_net_profit#16))#38,17,2) AS profit#41] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#15))#35, sum(UnscaledValue(ss_net_profit#16))#36] +Results [5]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#7 AS bought_city#37, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#15))#35,17,2) AS amt#38, MakeDecimal(sum(UnscaledValue(ss_net_profit#16))#36,17,2) AS profit#39] (47) Exchange -Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#39, amt#40, profit#41] -Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#37, amt#38, profit#39] +Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#40] (48) Sort [codegen id : 15] -Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#39, amt#40, profit#41] +Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#37, amt#38, profit#39] Arguments: [ss_customer_sk#10 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 16] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#10] -Join condition: NOT (ca_city#7 = bought_city#39) +Join condition: NOT (ca_city#7 = bought_city#37) (50) Project [codegen id : 16] -Output [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#39, ss_ticket_number#14, amt#40, profit#41] -Input [9]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#39, amt#40, profit#41] +Output [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#37, ss_ticket_number#14, amt#38, profit#39] +Input [9]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#37, amt#38, profit#39] (51) TakeOrderedAndProject -Input [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#39, ss_ticket_number#14, amt#40, profit#41] -Arguments: 100, [c_last_name#4 ASC NULLS FIRST, c_first_name#3 ASC NULLS FIRST, ca_city#7 ASC NULLS FIRST, bought_city#39 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#39, ss_ticket_number#14, amt#40, profit#41] +Input [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#37, ss_ticket_number#14, amt#38, profit#39] +Arguments: 100, [c_last_name#4 ASC NULLS FIRST, c_first_name#3 ASC NULLS FIRST, ca_city#7 ASC NULLS FIRST, bought_city#37 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#37, ss_ticket_number#14, amt#38, profit#39] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt index 2532a14d7907f..e9bc22389d8a6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt @@ -225,20 +225,20 @@ Output [7]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_ad Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#31, amt#32, profit#33, c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#39, ca_city#40] +Output [2]: [ca_address_sk#21, ca_city#22] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#39] -Join condition: NOT (ca_city#40 = bought_city#31) +Right keys [1]: [ca_address_sk#21] +Join condition: NOT (ca_city#22 = bought_city#31) (42) Project [codegen id : 8] -Output [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#39, ca_city#40] +Output [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#21, ca_city#22] (43) TakeOrderedAndProject -Input [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#40 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#22 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] ===== Subqueries ===== 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 3a356e35707f7..dc19996f9957b 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 @@ -213,11 +213,7 @@ Arguments: [rank(d_year#7, d_moy#8) windowspecdefinition(i_category#17, i_brand# (35) Filter [codegen id : 12] Input [9]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28] -<<<<<<< HEAD Condition : ((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#26)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#26 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) -======= -Condition : (((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#26)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#26 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#28)) ->>>>>>> regen (36) Exchange Input [9]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28] @@ -233,23 +229,22 @@ Output [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_ye (39) HashAggregate [codegen id : 21] Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum#36] Keys [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35] -Functions [1]: [sum(UnscaledValue(ss_sales_price#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#37))#38] -Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#37))#38,17,2) AS sum_sales#39] +Functions [1]: [sum(UnscaledValue(ss_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#3))#37] +Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#3))#37,17,2) AS sum_sales#38] (40) Exchange -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#39] (41) Sort [codegen id : 22] -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST], false, 0 (42) Window -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] -<<<<<<< HEAD (43) Project [codegen id : 23] Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38, rn#40] @@ -261,30 +256,12 @@ Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_compan (45) Sort [codegen id : 24] Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#40 + 1) ASC NULLS FIRST], false, 0 -======= -(43) Filter [codegen id : 23] -Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] -Condition : isnotnull(rn#41) - -(44) Project [codegen id : 23] -Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] -Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] - -(45) Exchange -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1), 5), ENSURE_REQUIREMENTS, [id=#42] - -(46) Sort [codegen id : 24] -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#41 + 1) ASC NULLS FIRST], false, 0 ->>>>>>> regen (46) SortMergeJoin [codegen id : 25] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1)] +Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1)] Join condition: None -<<<<<<< HEAD (47) Project [codegen id : 25] Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38] Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] @@ -311,45 +288,12 @@ Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_compan (53) Sort [codegen id : 36] Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (rn#49 - 1) ASC NULLS FIRST], false, 0 -======= -(48) Project [codegen id : 25] -Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39] -Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] - -(49) ReusedExchange [Reuses operator id: 40] -Output [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] - -(50) Sort [codegen id : 34] -Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] -Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST], false, 0 - -(51) Window -Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] -Arguments: [rank(d_year#47, d_moy#48) windowspecdefinition(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#50], [i_category#43, i_brand#44, s_store_name#45, s_company_name#46], [d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST] - -(52) Filter [codegen id : 35] -Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] -Condition : isnotnull(rn#50) - -(53) Project [codegen id : 35] -Output [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] -Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] - -(54) Exchange -Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] -Arguments: hashpartitioning(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1), 5), ENSURE_REQUIREMENTS, [id=#51] - -(55) Sort [codegen id : 36] -Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] -Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, (rn#50 - 1) ASC NULLS FIRST], false, 0 ->>>>>>> regen (54) SortMergeJoin [codegen id : 37] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1)] +Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1)] Join condition: None -<<<<<<< HEAD (55) Project [codegen id : 37] Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#38 AS psum#51, sum_sales#48 AS nsum#52] Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] @@ -357,15 +301,6 @@ Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_ye (56) TakeOrderedAndProject Input [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#11 ASC NULLS FIRST], [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] -======= -(57) Project [codegen id : 37] -Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#39 AS psum#52, sum_sales#49 AS nsum#53] -Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39, i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] - -(58) TakeOrderedAndProject -Input [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#11 ASC NULLS FIRST], [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] ->>>>>>> regen ===== Subqueries ===== 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 4a96c307dff2f..3d58d26e6a3a4 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 @@ -194,11 +194,7 @@ Arguments: [rank(d_year#11, d_moy#12) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [9]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27] -<<<<<<< HEAD Condition : ((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#25)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) -======= -Condition : (((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#25)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#27)) ->>>>>>> regen (33) ReusedExchange [Reuses operator id: unknown] Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34] @@ -206,23 +202,22 @@ Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_ye (34) HashAggregate [codegen id : 13] 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] +Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#35] +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#6))#35,17,2) AS sum_sales#36] (35) 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#36] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#37] (36) Sort [codegen id : 14] -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#36] 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 (37) 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#36] +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#38], [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] -<<<<<<< HEAD (38) Project [codegen id : 15] Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] @@ -230,26 +225,12 @@ Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_yea (39) BroadcastExchange Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] 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=#39] -======= -(38) Filter [codegen id : 15] -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] -Condition : isnotnull(rn#39) - -(39) Project [codegen id : 15] -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] - -(40) BroadcastExchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] + 1)),false), [id=#40] ->>>>>>> regen (40) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -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#38 + 1)] Join condition: None -<<<<<<< HEAD (41) Project [codegen id : 23] Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] @@ -272,41 +253,12 @@ Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_yea (46) BroadcastExchange Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] 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=#48] -======= -(42) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] -Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] - -(43) ReusedExchange [Reuses operator id: 35] -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] - -(44) Sort [codegen id : 21] -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 - -(45) 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] - -(46) Filter [codegen id : 22] -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] -Condition : isnotnull(rn#48) - -(47) Project [codegen id : 22] -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] - -(48) BroadcastExchange -Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] - 1)),false), [id=#49] ->>>>>>> regen (47) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -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#47 - 1)] Join condition: None -<<<<<<< HEAD (48) Project [codegen id : 23] Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#49, sum_sales#46 AS nsum#50] Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] @@ -314,15 +266,6 @@ Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year (49) TakeOrderedAndProject Input [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] -======= -(50) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] -Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] - -(51) TakeOrderedAndProject -Input [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, psum#50, nsum#51] ->>>>>>> regen ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/explain.txt index 2f87af685f2e5..bd8d979c32d17 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.sf100/explain.txt @@ -79,7 +79,7 @@ Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_ Output [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree )),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree ))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College )))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree)),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College)))] ReadSchema: struct (11) ColumnarToRow [codegen id : 2] @@ -87,7 +87,7 @@ Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] (12) Filter [codegen id : 2] Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] -Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree )) OR ((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree ))) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College )))) +Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree)) OR ((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree))) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College)))) (13) BroadcastExchange Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] @@ -96,7 +96,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (14) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#11] -Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) +Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree)) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree)) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College)) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) (15) Project [codegen id : 5] Output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt index 2f87af685f2e5..bd8d979c32d17 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/explain.txt @@ -79,7 +79,7 @@ Input [8]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_ Output [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree )),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree ))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College )))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree)),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College)))] ReadSchema: struct (11) ColumnarToRow [codegen id : 2] @@ -87,7 +87,7 @@ Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] (12) Filter [codegen id : 2] Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] -Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree )) OR ((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree ))) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College )))) +Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree)) OR ((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree))) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College)))) (13) BroadcastExchange Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] @@ -96,7 +96,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (14) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_cdemo_sk#1] Right keys [1]: [cd_demo_sk#11] -Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) +Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = 4 yr Degree)) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#12 = D) AND (cd_education_status#13 = 2 yr Degree)) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College)) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) (15) Project [codegen id : 5] Output [4]: [ss_addr_sk#2, ss_quantity#4, ss_net_profit#6, ss_sold_date_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt index 3f30183af3a94..e93d0e9503493 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt @@ -250,225 +250,225 @@ Output [5]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_net_profit#47, cs_sold_date_sk#48] (36) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#49] +Output [1]: [d_date_sk#8] (37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_sold_date_sk#48] -Right keys [1]: [d_date_sk#49] +Right keys [1]: [d_date_sk#8] Join condition: None (38) Project [codegen id : 12] Output [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#49] +Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#8] (39) Exchange Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#50] +Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#49] (40) Sort [codegen id : 13] Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] Arguments: [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST], false, 0 (41) Scan parquet default.catalog_returns -Output [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Output [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (42) ColumnarToRow [codegen id : 14] -Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] (43) Filter [codegen id : 14] -Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] -Condition : (((isnotnull(cr_return_amount#54) AND (cr_return_amount#54 > 10000.00)) AND isnotnull(cr_order_number#52)) AND isnotnull(cr_item_sk#51)) +Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Condition : (((isnotnull(cr_return_amount#53) AND (cr_return_amount#53 > 10000.00)) AND isnotnull(cr_order_number#51)) AND isnotnull(cr_item_sk#50)) (44) Project [codegen id : 14] -Output [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] -Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Output [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] (45) Exchange -Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] -Arguments: hashpartitioning(cr_order_number#52, cr_item_sk#51, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#55] (46) Sort [codegen id : 15] -Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] -Arguments: [cr_order_number#52 ASC NULLS FIRST, cr_item_sk#51 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Arguments: [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 16] Left keys [2]: [cs_order_number#44, cs_item_sk#43] -Right keys [2]: [cr_order_number#52, cr_item_sk#51] +Right keys [2]: [cr_order_number#51, cr_item_sk#50] Join condition: None (48) Project [codegen id : 16] -Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] -Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] +Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] (49) HashAggregate [codegen id : 16] -Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] +Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] Keys [1]: [cs_item_sk#43] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#53, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] -Results [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#52, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#56, sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] +Results [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] (50) Exchange -Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#68] (51) HashAggregate [codegen id : 17] -Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] Keys [1]: [cs_item_sk#43] -Functions [4]: [sum(coalesce(cr_return_quantity#53, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#53, 0))#70, sum(coalesce(cs_quantity#45, 0))#71, sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73] -Results [3]: [cs_item_sk#43 AS item#74, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#53, 0))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#71 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#75, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#76] +Functions [4]: [sum(coalesce(cr_return_quantity#52, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#52, 0))#69, sum(coalesce(cs_quantity#45, 0))#70, sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72] +Results [3]: [cs_item_sk#43 AS item#73, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#52, 0))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#70 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#74, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#75] (52) Exchange -Input [3]: [item#74, return_ratio#75, currency_ratio#76] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#77] +Input [3]: [item#73, return_ratio#74, currency_ratio#75] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] (53) Sort [codegen id : 18] -Input [3]: [item#74, return_ratio#75, currency_ratio#76] -Arguments: [return_ratio#75 ASC NULLS FIRST], false, 0 +Input [3]: [item#73, return_ratio#74, currency_ratio#75] +Arguments: [return_ratio#74 ASC NULLS FIRST], false, 0 (54) Window -Input [3]: [item#74, return_ratio#75, currency_ratio#76] -Arguments: [rank(return_ratio#75) windowspecdefinition(return_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#78], [return_ratio#75 ASC NULLS FIRST] +Input [3]: [item#73, return_ratio#74, currency_ratio#75] +Arguments: [rank(return_ratio#74) windowspecdefinition(return_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#77], [return_ratio#74 ASC NULLS FIRST] (55) Sort [codegen id : 19] -Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] -Arguments: [currency_ratio#76 ASC NULLS FIRST], false, 0 +Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] +Arguments: [currency_ratio#75 ASC NULLS FIRST], false, 0 (56) Window -Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] -Arguments: [rank(currency_ratio#76) windowspecdefinition(currency_ratio#76 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#79], [currency_ratio#76 ASC NULLS FIRST] +Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] +Arguments: [rank(currency_ratio#75) windowspecdefinition(currency_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#78], [currency_ratio#75 ASC NULLS FIRST] (57) Filter [codegen id : 20] -Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] -Condition : ((return_rank#78 <= 10) OR (currency_rank#79 <= 10)) +Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] +Condition : ((return_rank#77 <= 10) OR (currency_rank#78 <= 10)) (58) Project [codegen id : 20] -Output [5]: [catalog AS channel#80, item#74, return_ratio#75, return_rank#78, currency_rank#79] -Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] +Output [5]: [catalog AS channel#79, item#73, return_ratio#74, return_rank#77, currency_rank#78] +Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] (59) Scan parquet default.store_sales -Output [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Output [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_sold_date_sk#85 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 22] -Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] (61) Filter [codegen id : 22] -Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] -Condition : (((((((isnotnull(ss_net_profit#85) AND isnotnull(ss_net_paid#84)) AND isnotnull(ss_quantity#83)) AND (ss_net_profit#85 > 1.00)) AND (ss_net_paid#84 > 0.00)) AND (ss_quantity#83 > 0)) AND isnotnull(ss_ticket_number#82)) AND isnotnull(ss_item_sk#81)) +Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Condition : (((((((isnotnull(ss_net_profit#84) AND isnotnull(ss_net_paid#83)) AND isnotnull(ss_quantity#82)) AND (ss_net_profit#84 > 1.00)) AND (ss_net_paid#83 > 0.00)) AND (ss_quantity#82 > 0)) AND isnotnull(ss_ticket_number#81)) AND isnotnull(ss_item_sk#80)) (62) Project [codegen id : 22] -Output [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86] -Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Output [5]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85] +Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] (63) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#87] +Output [1]: [d_date_sk#8] (64) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ss_sold_date_sk#86] -Right keys [1]: [d_date_sk#87] +Left keys [1]: [ss_sold_date_sk#85] +Right keys [1]: [d_date_sk#8] Join condition: None (65) Project [codegen id : 22] -Output [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] -Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86, d_date_sk#87] +Output [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] +Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85, d_date_sk#8] (66) Exchange -Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] -Arguments: hashpartitioning(cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint), 5), ENSURE_REQUIREMENTS, [id=#88] +Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] +Arguments: hashpartitioning(cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint), 5), ENSURE_REQUIREMENTS, [id=#86] (67) Sort [codegen id : 23] -Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] -Arguments: [cast(ss_ticket_number#82 as bigint) ASC NULLS FIRST, cast(ss_item_sk#81 as bigint) ASC NULLS FIRST], false, 0 +Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] +Arguments: [cast(ss_ticket_number#81 as bigint) ASC NULLS FIRST, cast(ss_item_sk#80 as bigint) ASC NULLS FIRST], false, 0 (68) Scan parquet default.store_returns -Output [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Output [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 24] -Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] (70) Filter [codegen id : 24] -Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] -Condition : (((isnotnull(sr_return_amt#92) AND (sr_return_amt#92 > 10000.00)) AND isnotnull(sr_ticket_number#90)) AND isnotnull(sr_item_sk#89)) +Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Condition : (((isnotnull(sr_return_amt#90) AND (sr_return_amt#90 > 10000.00)) AND isnotnull(sr_ticket_number#88)) AND isnotnull(sr_item_sk#87)) (71) Project [codegen id : 24] -Output [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] -Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Output [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] (72) Exchange -Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] -Arguments: hashpartitioning(sr_ticket_number#90, sr_item_sk#89, 5), ENSURE_REQUIREMENTS, [id=#94] +Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Arguments: hashpartitioning(sr_ticket_number#88, sr_item_sk#87, 5), ENSURE_REQUIREMENTS, [id=#92] (73) Sort [codegen id : 25] -Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] -Arguments: [sr_ticket_number#90 ASC NULLS FIRST, sr_item_sk#89 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Arguments: [sr_ticket_number#88 ASC NULLS FIRST, sr_item_sk#87 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 26] -Left keys [2]: [cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint)] -Right keys [2]: [sr_ticket_number#90, sr_item_sk#89] +Left keys [2]: [cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint)] +Right keys [2]: [sr_ticket_number#88, sr_item_sk#87] Join condition: None (75) Project [codegen id : 26] -Output [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] -Input [8]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Output [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] +Input [8]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] (76) HashAggregate [codegen id : 26] -Input [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] -Keys [1]: [ss_item_sk#81] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#91, 0)), partial_sum(coalesce(ss_quantity#83, 0)), partial_sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#95, sum#96, sum#97, isEmpty#98, sum#99, isEmpty#100] -Results [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] +Input [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] +Keys [1]: [ss_item_sk#80] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#89, 0)), partial_sum(coalesce(ss_quantity#82, 0)), partial_sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#93, sum#94, sum#95, isEmpty#96, sum#97, isEmpty#98] +Results [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] (77) Exchange -Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] -Arguments: hashpartitioning(ss_item_sk#81, 5), ENSURE_REQUIREMENTS, [id=#107] +Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Arguments: hashpartitioning(ss_item_sk#80, 5), ENSURE_REQUIREMENTS, [id=#105] (78) HashAggregate [codegen id : 27] -Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] -Keys [1]: [ss_item_sk#81] -Functions [4]: [sum(coalesce(sr_return_quantity#91, 0)), sum(coalesce(ss_quantity#83, 0)), sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#91, 0))#108, sum(coalesce(ss_quantity#83, 0))#109, sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110, sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111] -Results [3]: [ss_item_sk#81 AS item#112, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#91, 0))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#83, 0))#109 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#113, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#114] +Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Keys [1]: [ss_item_sk#80] +Functions [4]: [sum(coalesce(sr_return_quantity#89, 0)), sum(coalesce(ss_quantity#82, 0)), sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#89, 0))#106, sum(coalesce(ss_quantity#82, 0))#107, sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108, sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109] +Results [3]: [ss_item_sk#80 AS item#110, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#89, 0))#106 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#82, 0))#107 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#111, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#112] (79) Exchange -Input [3]: [item#112, return_ratio#113, currency_ratio#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] +Input [3]: [item#110, return_ratio#111, currency_ratio#112] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] (80) Sort [codegen id : 28] -Input [3]: [item#112, return_ratio#113, currency_ratio#114] -Arguments: [return_ratio#113 ASC NULLS FIRST], false, 0 +Input [3]: [item#110, return_ratio#111, currency_ratio#112] +Arguments: [return_ratio#111 ASC NULLS FIRST], false, 0 (81) Window -Input [3]: [item#112, return_ratio#113, currency_ratio#114] -Arguments: [rank(return_ratio#113) windowspecdefinition(return_ratio#113 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#116], [return_ratio#113 ASC NULLS FIRST] +Input [3]: [item#110, return_ratio#111, currency_ratio#112] +Arguments: [rank(return_ratio#111) windowspecdefinition(return_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#114], [return_ratio#111 ASC NULLS FIRST] (82) Sort [codegen id : 29] -Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] -Arguments: [currency_ratio#114 ASC NULLS FIRST], false, 0 +Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] +Arguments: [currency_ratio#112 ASC NULLS FIRST], false, 0 (83) Window -Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] -Arguments: [rank(currency_ratio#114) windowspecdefinition(currency_ratio#114 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#117], [currency_ratio#114 ASC NULLS FIRST] +Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] +Arguments: [rank(currency_ratio#112) windowspecdefinition(currency_ratio#112 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#115], [currency_ratio#112 ASC NULLS FIRST] (84) Filter [codegen id : 30] -Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] -Condition : ((return_rank#116 <= 10) OR (currency_rank#117 <= 10)) +Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] +Condition : ((return_rank#114 <= 10) OR (currency_rank#115 <= 10)) (85) Project [codegen id : 30] -Output [5]: [store AS channel#118, item#112, return_ratio#113, return_rank#116, currency_rank#117] -Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] +Output [5]: [store AS channel#116, item#110, return_ratio#111, return_rank#114, currency_rank#115] +Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] (86) Union @@ -481,7 +481,7 @@ Results [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_ran (88) Exchange Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] -Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#119] +Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#117] (89) HashAggregate [codegen id : 32] Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] @@ -505,6 +505,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#85 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index a98f59f84be7d..aeaca4deaae0c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -260,170 +260,170 @@ Output [6]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, Input [9]: [cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_item_sk#49, cr_order_number#50, cr_return_quantity#51, cr_return_amount#52] (40) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#54] +Output [1]: [d_date_sk#14] (41) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#47] -Right keys [1]: [d_date_sk#54] +Right keys [1]: [d_date_sk#14] Join condition: None (42) Project [codegen id : 10] Output [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] -Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#54] +Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#14] (43) HashAggregate [codegen id : 10] Input [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] Keys [1]: [cs_item_sk#42] Functions [4]: [partial_sum(coalesce(cr_return_quantity#51, 0)), partial_sum(coalesce(cs_quantity#44, 0)), partial_sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Results [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Aggregate Attributes [6]: [sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] +Results [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] (44) Exchange -Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#67] +Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] +Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#66] (45) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] Keys [1]: [cs_item_sk#42] Functions [4]: [sum(coalesce(cr_return_quantity#51, 0)), sum(coalesce(cs_quantity#44, 0)), sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#68, sum(coalesce(cs_quantity#44, 0))#69, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71] -Results [3]: [cs_item_sk#42 AS item#72, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#68 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#69 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#73, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#74] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#67, sum(coalesce(cs_quantity#44, 0))#68, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70] +Results [3]: [cs_item_sk#42 AS item#71, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#67 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#68 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#72, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#73] (46) Exchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#75] +Input [3]: [item#71, return_ratio#72, currency_ratio#73] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#74] (47) Sort [codegen id : 12] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [return_ratio#73 ASC NULLS FIRST], false, 0 +Input [3]: [item#71, return_ratio#72, currency_ratio#73] +Arguments: [return_ratio#72 ASC NULLS FIRST], false, 0 (48) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#76], [return_ratio#73 ASC NULLS FIRST] +Input [3]: [item#71, return_ratio#72, currency_ratio#73] +Arguments: [rank(return_ratio#72) windowspecdefinition(return_ratio#72 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#72 ASC NULLS FIRST] (49) Sort [codegen id : 13] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 +Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] +Arguments: [currency_ratio#73 ASC NULLS FIRST], false, 0 (50) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#77], [currency_ratio#74 ASC NULLS FIRST] +Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] +Arguments: [rank(currency_ratio#73) windowspecdefinition(currency_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#73 ASC NULLS FIRST] (51) Filter [codegen id : 14] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] -Condition : ((return_rank#76 <= 10) OR (currency_rank#77 <= 10)) +Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] +Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) (52) Project [codegen id : 14] -Output [5]: [catalog AS channel#78, item#72, return_ratio#73, return_rank#76, currency_rank#77] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] +Output [5]: [catalog AS channel#77, item#71, return_ratio#72, return_rank#75, currency_rank#76] +Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] (53) Scan parquet default.store_sales -Output [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Output [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#84), dynamicpruningexpression(ss_sold_date_sk#84 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#83), dynamicpruningexpression(ss_sold_date_sk#83 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 15] -Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] (55) Filter [codegen id : 15] -Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] -Condition : (((((((isnotnull(ss_net_profit#83) AND isnotnull(ss_net_paid#82)) AND isnotnull(ss_quantity#81)) AND (ss_net_profit#83 > 1.00)) AND (ss_net_paid#82 > 0.00)) AND (ss_quantity#81 > 0)) AND isnotnull(ss_ticket_number#80)) AND isnotnull(ss_item_sk#79)) +Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Condition : (((((((isnotnull(ss_net_profit#82) AND isnotnull(ss_net_paid#81)) AND isnotnull(ss_quantity#80)) AND (ss_net_profit#82 > 1.00)) AND (ss_net_paid#81 > 0.00)) AND (ss_quantity#80 > 0)) AND isnotnull(ss_ticket_number#79)) AND isnotnull(ss_item_sk#78)) (56) Project [codegen id : 15] -Output [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] -Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Output [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] +Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] (57) BroadcastExchange -Input [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#85] +Input [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#84] (58) Scan parquet default.store_returns -Output [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Output [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (59) ColumnarToRow -Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] (60) Filter -Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] -Condition : (((isnotnull(sr_return_amt#89) AND (sr_return_amt#89 > 10000.00)) AND isnotnull(sr_ticket_number#87)) AND isnotnull(sr_item_sk#86)) +Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Condition : (((isnotnull(sr_return_amt#88) AND (sr_return_amt#88 > 10000.00)) AND isnotnull(sr_ticket_number#86)) AND isnotnull(sr_item_sk#85)) (61) Project -Output [4]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] -Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Output [4]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] +Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] (62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [cast(ss_ticket_number#80 as bigint), cast(ss_item_sk#79 as bigint)] -Right keys [2]: [sr_ticket_number#87, sr_item_sk#86] +Left keys [2]: [cast(ss_ticket_number#79 as bigint), cast(ss_item_sk#78 as bigint)] +Right keys [2]: [sr_ticket_number#86, sr_item_sk#85] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89] -Input [9]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] +Output [6]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88] +Input [9]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] (64) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#91] +Output [1]: [d_date_sk#14] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#84] -Right keys [1]: [d_date_sk#91] +Left keys [1]: [ss_sold_date_sk#83] +Right keys [1]: [d_date_sk#14] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] -Input [7]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89, d_date_sk#91] +Output [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] +Input [7]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88, d_date_sk#14] (67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] -Keys [1]: [ss_item_sk#79] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#88, 0)), partial_sum(coalesce(ss_quantity#81, 0)), partial_sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Results [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Input [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] +Keys [1]: [ss_item_sk#78] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#87, 0)), partial_sum(coalesce(ss_quantity#80, 0)), partial_sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Results [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] (68) Exchange -Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] -Arguments: hashpartitioning(ss_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#104] +Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Arguments: hashpartitioning(ss_item_sk#78, 5), ENSURE_REQUIREMENTS, [id=#102] (69) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] -Keys [1]: [ss_item_sk#79] -Functions [4]: [sum(coalesce(sr_return_quantity#88, 0)), sum(coalesce(ss_quantity#81, 0)), sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#88, 0))#105, sum(coalesce(ss_quantity#81, 0))#106, sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107, sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108] -Results [3]: [ss_item_sk#79 AS item#109, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#88, 0))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#81, 0))#106 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#110, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#111] +Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Keys [1]: [ss_item_sk#78] +Functions [4]: [sum(coalesce(sr_return_quantity#87, 0)), sum(coalesce(ss_quantity#80, 0)), sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#87, 0))#103, sum(coalesce(ss_quantity#80, 0))#104, sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105, sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106] +Results [3]: [ss_item_sk#78 AS item#107, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#87, 0))#103 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#80, 0))#104 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#108, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#109] (70) Exchange -Input [3]: [item#109, return_ratio#110, currency_ratio#111] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#112] +Input [3]: [item#107, return_ratio#108, currency_ratio#109] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] (71) Sort [codegen id : 19] -Input [3]: [item#109, return_ratio#110, currency_ratio#111] -Arguments: [return_ratio#110 ASC NULLS FIRST], false, 0 +Input [3]: [item#107, return_ratio#108, currency_ratio#109] +Arguments: [return_ratio#108 ASC NULLS FIRST], false, 0 (72) Window -Input [3]: [item#109, return_ratio#110, currency_ratio#111] -Arguments: [rank(return_ratio#110) windowspecdefinition(return_ratio#110 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#113], [return_ratio#110 ASC NULLS FIRST] +Input [3]: [item#107, return_ratio#108, currency_ratio#109] +Arguments: [rank(return_ratio#108) windowspecdefinition(return_ratio#108 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#111], [return_ratio#108 ASC NULLS FIRST] (73) Sort [codegen id : 20] -Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] -Arguments: [currency_ratio#111 ASC NULLS FIRST], false, 0 +Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] +Arguments: [currency_ratio#109 ASC NULLS FIRST], false, 0 (74) Window -Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] -Arguments: [rank(currency_ratio#111) windowspecdefinition(currency_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#114], [currency_ratio#111 ASC NULLS FIRST] +Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] +Arguments: [rank(currency_ratio#109) windowspecdefinition(currency_ratio#109 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#112], [currency_ratio#109 ASC NULLS FIRST] (75) Filter [codegen id : 21] -Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] -Condition : ((return_rank#113 <= 10) OR (currency_rank#114 <= 10)) +Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] +Condition : ((return_rank#111 <= 10) OR (currency_rank#112 <= 10)) (76) Project [codegen id : 21] -Output [5]: [store AS channel#115, item#109, return_ratio#110, return_rank#113, currency_rank#114] -Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] +Output [5]: [store AS channel#113, item#107, return_ratio#108, return_rank#111, currency_rank#112] +Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] (77) Union @@ -436,7 +436,7 @@ Results [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_ran (79) Exchange Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] -Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#116] +Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#114] (80) HashAggregate [codegen id : 23] Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] @@ -460,6 +460,6 @@ Output [1]: [d_date_sk#14] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#84 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#83 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index 28a457258eff7..2d3dea5d84565 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -160,7 +160,7 @@ Input [2]: [d_date_sk#25, d_date#26] (18) Filter [codegen id : 4] Input [2]: [d_date_sk#25, d_date#26] -Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 2000-08-23)) AND (d_date#26 <= 2000-09-06)) AND isnotnull(d_date_sk#25)) +Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 11192)) AND (d_date#26 <= 11206)) AND isnotnull(d_date_sk#25)) (19) Project [codegen id : 4] Output [1]: [d_date_sk#25] @@ -265,193 +265,193 @@ Output [6]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, Input [8]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] (41) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#70] +Output [1]: [d_date_sk#25] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#70] +Right keys [1]: [d_date_sk#25] Join condition: None (43) Project [codegen id : 11] Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#70] +Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#25] (44) HashAggregate [codegen id : 11] Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] -Results [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] +Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] +Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] (45) Exchange -Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] -Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] +Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#84, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS RETURNS#85, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#86, catalog channel AS channel#87, concat(catalog_page, cp_catalog_page_id#68) AS id#88] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#83, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS RETURNS#84, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#85, catalog channel AS channel#86, concat(catalog_page, cp_catalog_page_id#68) AS id#87] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Condition : isnotnull(ws_web_site_sk#89) +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Condition : isnotnull(ws_web_site_sk#88) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] (53) Exchange -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] -Arguments: hashpartitioning(wr_item_sk#99, wr_order_number#100, 5), ENSURE_REQUIREMENTS, [id=#104] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Arguments: hashpartitioning(wr_item_sk#98, wr_order_number#99, 5), ENSURE_REQUIREMENTS, [id=#103] (54) Sort [codegen id : 15] -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] -Arguments: [wr_item_sk#99 ASC NULLS FIRST, wr_order_number#100 ASC NULLS FIRST], false, 0 +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Arguments: [wr_item_sk#98 ASC NULLS FIRST, wr_order_number#99 ASC NULLS FIRST], false, 0 (55) Scan parquet default.web_sales -Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 16] -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] (57) Filter [codegen id : 16] -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] -Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) (58) Project [codegen id : 16] -Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] (59) Exchange -Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] -Arguments: hashpartitioning(cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint), 5), ENSURE_REQUIREMENTS, [id=#109] +Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Arguments: hashpartitioning(cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint), 5), ENSURE_REQUIREMENTS, [id=#106] (60) Sort [codegen id : 17] -Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] -Arguments: [cast(ws_item_sk#105 as bigint) ASC NULLS FIRST, cast(ws_order_number#107 as bigint) ASC NULLS FIRST], false, 0 +Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Arguments: [cast(ws_item_sk#104 as bigint) ASC NULLS FIRST, cast(ws_order_number#105 as bigint) ASC NULLS FIRST], false, 0 (61) SortMergeJoin [codegen id : 18] -Left keys [2]: [wr_item_sk#99, wr_order_number#100] -Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] +Left keys [2]: [wr_item_sk#98, wr_order_number#99] +Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None (62) Project [codegen id : 18] -Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#110, wr_returned_date_sk#103 AS date_sk#111, 0.00 AS sales_price#112, 0.00 AS profit#113, wr_return_amt#101 AS return_amt#114, wr_net_loss#102 AS net_loss#115] -Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#107, wr_returned_date_sk#102 AS date_sk#108, 0.00 AS sales_price#109, 0.00 AS profit#110, wr_return_amt#100 AS return_amt#111, wr_net_loss#101 AS net_loss#112] +Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] (63) Union (64) Scan parquet default.web_site -Output [2]: [web_site_sk#116, web_site_id#117] +Output [2]: [web_site_sk#113, web_site_id#114] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 19] -Input [2]: [web_site_sk#116, web_site_id#117] +Input [2]: [web_site_sk#113, web_site_id#114] (66) Filter [codegen id : 19] -Input [2]: [web_site_sk#116, web_site_id#117] -Condition : isnotnull(web_site_sk#116) +Input [2]: [web_site_sk#113, web_site_id#114] +Condition : isnotnull(web_site_sk#113) (67) BroadcastExchange -Input [2]: [web_site_sk#116, web_site_id#117] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] +Input [2]: [web_site_sk#113, web_site_id#114] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] (68) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [wsr_web_site_sk#93] -Right keys [1]: [web_site_sk#116] +Left keys [1]: [wsr_web_site_sk#92] +Right keys [1]: [web_site_sk#113] Join condition: None (69) Project [codegen id : 21] -Output [6]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Input [8]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] +Output [6]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] +Input [8]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#113, web_site_id#114] (70) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#119] +Output [1]: [d_date_sk#25] (71) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [date_sk#94] -Right keys [1]: [cast(d_date_sk#119 as bigint)] +Left keys [1]: [date_sk#93] +Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None (72) Project [codegen id : 21] -Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Input [7]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117, d_date_sk#119] +Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] +Input [7]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114, d_date_sk#25] (73) HashAggregate [codegen id : 21] -Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Keys [1]: [web_site_id#117] -Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] -Aggregate Attributes [4]: [sum#120, sum#121, sum#122, sum#123] -Results [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] +Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] +Keys [1]: [web_site_id#114] +Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] +Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] +Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] (74) Exchange -Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] -Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#128] +Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] +Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] (75) HashAggregate [codegen id : 22] -Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] -Keys [1]: [web_site_id#117] -Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#129, sum(UnscaledValue(return_amt#97))#130, sum(UnscaledValue(profit#96))#131, sum(UnscaledValue(net_loss#98))#132] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#95))#129,17,2) AS sales#133, MakeDecimal(sum(UnscaledValue(return_amt#97))#130,17,2) AS RETURNS#134, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#131,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#132,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#135, web channel AS channel#136, concat(web_site, web_site_id#117) AS id#137] +Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] +Keys [1]: [web_site_id#114] +Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#125, sum(UnscaledValue(return_amt#96))#126, sum(UnscaledValue(profit#95))#127, sum(UnscaledValue(net_loss#97))#128] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#94))#125,17,2) AS sales#129, MakeDecimal(sum(UnscaledValue(return_amt#96))#126,17,2) AS RETURNS#130, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#131, web channel AS channel#132, concat(web_site, web_site_id#114) AS id#133] (76) Union (77) Expand [codegen id : 23] Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45] -Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#138, id#139, spark_grouping_id#140] +Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#134, id#135, spark_grouping_id#136] (78) HashAggregate [codegen id : 23] -Input [6]: [sales#41, returns#42, profit#43, channel#138, id#139, spark_grouping_id#140] -Keys [3]: [channel#138, id#139, spark_grouping_id#140] +Input [6]: [sales#41, returns#42, profit#43, channel#134, id#135, spark_grouping_id#136] +Keys [3]: [channel#134, id#135, spark_grouping_id#136] Functions [3]: [partial_sum(sales#41), partial_sum(returns#42), partial_sum(profit#43)] -Aggregate Attributes [6]: [sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Results [9]: [channel#138, id#139, spark_grouping_id#140, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] +Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] +Results [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] (79) Exchange -Input [9]: [channel#138, id#139, spark_grouping_id#140, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -Arguments: hashpartitioning(channel#138, id#139, spark_grouping_id#140, 5), ENSURE_REQUIREMENTS, [id=#153] +Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Arguments: hashpartitioning(channel#134, id#135, spark_grouping_id#136, 5), ENSURE_REQUIREMENTS, [id=#149] (80) HashAggregate [codegen id : 24] -Input [9]: [channel#138, id#139, spark_grouping_id#140, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -Keys [3]: [channel#138, id#139, spark_grouping_id#140] +Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Keys [3]: [channel#134, id#135, spark_grouping_id#136] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] -Aggregate Attributes [3]: [sum(sales#41)#154, sum(returns#42)#155, sum(profit#43)#156] -Results [5]: [channel#138, id#139, sum(sales#41)#154 AS sales#157, sum(returns#42)#155 AS returns#158, sum(profit#43)#156 AS profit#159] +Aggregate Attributes [3]: [sum(sales#41)#150, sum(returns#42)#151, sum(profit#43)#152] +Results [5]: [channel#134, id#135, sum(sales#41)#150 AS sales#153, sum(returns#42)#151 AS returns#154, sum(profit#43)#152 AS profit#155] (81) TakeOrderedAndProject -Input [5]: [channel#138, id#139, sales#157, returns#158, profit#159] -Arguments: 100, [channel#138 ASC NULLS FIRST, id#139 ASC NULLS FIRST], [channel#138, id#139, sales#157, returns#158, profit#159] +Input [5]: [channel#134, id#135, sales#153, returns#154, profit#155] +Arguments: 100, [channel#134 ASC NULLS FIRST, id#135 ASC NULLS FIRST], [channel#134, id#135, sales#153, returns#154, profit#155] ===== Subqueries ===== @@ -469,12 +469,12 @@ ReusedExchange (83) (83) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#70] +Output [1]: [d_date_sk#25] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index cb130ce17795a..f6fe8bee89369 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -130,7 +130,7 @@ Input [2]: [d_date_sk#22, d_date#23] (12) Filter [codegen id : 3] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11192)) AND (d_date#23 <= 11206)) AND isnotnull(d_date_sk#22)) (13) Project [codegen id : 3] Output [1]: [d_date_sk#22] @@ -235,208 +235,208 @@ Input [4]: [cr_catalog_page_sk#57, cr_return_amount#58, cr_net_loss#59, cr_retur (34) Union (35) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#67] +Output [1]: [d_date_sk#22] (36) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#67] +Right keys [1]: [d_date_sk#22] Join condition: None (37) Project [codegen id : 11] Output [5]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56] -Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#67] +Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#22] (38) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Output [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] (40) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] -Condition : isnotnull(cp_catalog_page_sk#68) +Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Condition : isnotnull(cp_catalog_page_sk#67) (41) BroadcastExchange -Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#70] +Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#69] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [page_sk#51] -Right keys [1]: [cp_catalog_page_sk#68] +Right keys [1]: [cp_catalog_page_sk#67] Join condition: None (43) Project [codegen id : 11] -Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] -Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#68, cp_catalog_page_id#69] +Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] +Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] (44) HashAggregate [codegen id : 11] -Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] -Keys [1]: [cp_catalog_page_id#69] +Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] +Keys [1]: [cp_catalog_page_id#68] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] -Results [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] +Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] +Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] (45) Exchange -Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] -Arguments: hashpartitioning(cp_catalog_page_id#69, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] -Keys [1]: [cp_catalog_page_id#69] +Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Keys [1]: [cp_catalog_page_id#68] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#84, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS RETURNS#85, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#86, catalog channel AS channel#87, concat(catalog_page, cp_catalog_page_id#69) AS id#88] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#83, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS RETURNS#84, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#85, catalog channel AS channel#86, concat(catalog_page, cp_catalog_page_id#68) AS id#87] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Condition : isnotnull(ws_web_site_sk#89) +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Condition : isnotnull(ws_web_site_sk#88) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] (53) BroadcastExchange -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#104] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#103] (54) Scan parquet default.web_sales -Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (55) ColumnarToRow -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] (56) Filter -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] -Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) (57) Project -Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] (58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#99, wr_order_number#100] -Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] +Left keys [2]: [wr_item_sk#98, wr_order_number#99] +Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None (59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#109, wr_returned_date_sk#103 AS date_sk#110, 0.00 AS sales_price#111, 0.00 AS profit#112, wr_return_amt#101 AS return_amt#113, wr_net_loss#102 AS net_loss#114] -Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#106, wr_returned_date_sk#102 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#100 AS return_amt#110, wr_net_loss#101 AS net_loss#111] +Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] (60) Union (61) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#115] +Output [1]: [d_date_sk#22] (62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#94] -Right keys [1]: [cast(d_date_sk#115 as bigint)] +Left keys [1]: [date_sk#93] +Right keys [1]: [cast(d_date_sk#22 as bigint)] Join condition: None (63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98] -Input [7]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, d_date_sk#115] +Output [5]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97] +Input [7]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, d_date_sk#22] (64) Scan parquet default.web_site -Output [2]: [web_site_sk#116, web_site_id#117] +Output [2]: [web_site_sk#112, web_site_id#113] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#116, web_site_id#117] +Input [2]: [web_site_sk#112, web_site_id#113] (66) Filter [codegen id : 17] -Input [2]: [web_site_sk#116, web_site_id#117] -Condition : isnotnull(web_site_sk#116) +Input [2]: [web_site_sk#112, web_site_id#113] +Condition : isnotnull(web_site_sk#112) (67) BroadcastExchange -Input [2]: [web_site_sk#116, web_site_id#117] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] +Input [2]: [web_site_sk#112, web_site_id#113] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#114] (68) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#93] -Right keys [1]: [web_site_sk#116] +Left keys [1]: [wsr_web_site_sk#92] +Right keys [1]: [web_site_sk#112] Join condition: None (69) Project [codegen id : 18] -Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Input [7]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] +Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] +Input [7]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#112, web_site_id#113] (70) HashAggregate [codegen id : 18] -Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Keys [1]: [web_site_id#117] -Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] -Aggregate Attributes [4]: [sum#119, sum#120, sum#121, sum#122] -Results [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] +Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] +Keys [1]: [web_site_id#113] +Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] +Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] +Results [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] (71) Exchange -Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] -Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#127] +Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] +Arguments: hashpartitioning(web_site_id#113, 5), ENSURE_REQUIREMENTS, [id=#123] (72) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] -Keys [1]: [web_site_id#117] -Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#128, sum(UnscaledValue(return_amt#97))#129, sum(UnscaledValue(profit#96))#130, sum(UnscaledValue(net_loss#98))#131] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#95))#128,17,2) AS sales#132, MakeDecimal(sum(UnscaledValue(return_amt#97))#129,17,2) AS RETURNS#133, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#130,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#131,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#134, web channel AS channel#135, concat(web_site, web_site_id#117) AS id#136] +Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] +Keys [1]: [web_site_id#113] +Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#124, sum(UnscaledValue(return_amt#96))#125, sum(UnscaledValue(profit#95))#126, sum(UnscaledValue(net_loss#97))#127] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#94))#124,17,2) AS sales#128, MakeDecimal(sum(UnscaledValue(return_amt#96))#125,17,2) AS RETURNS#129, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#126,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#127,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#130, web channel AS channel#131, concat(web_site, web_site_id#113) AS id#132] (73) Union (74) Expand [codegen id : 20] Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45] -Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#137, id#138, spark_grouping_id#139] +Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#133, id#134, spark_grouping_id#135] (75) HashAggregate [codegen id : 20] -Input [6]: [sales#41, returns#42, profit#43, channel#137, id#138, spark_grouping_id#139] -Keys [3]: [channel#137, id#138, spark_grouping_id#139] +Input [6]: [sales#41, returns#42, profit#43, channel#133, id#134, spark_grouping_id#135] +Keys [3]: [channel#133, id#134, spark_grouping_id#135] Functions [3]: [partial_sum(sales#41), partial_sum(returns#42), partial_sum(profit#43)] -Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -Results [9]: [channel#137, id#138, spark_grouping_id#139, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Aggregate Attributes [6]: [sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] +Results [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] (76) Exchange -Input [9]: [channel#137, id#138, spark_grouping_id#139, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -Arguments: hashpartitioning(channel#137, id#138, spark_grouping_id#139, 5), ENSURE_REQUIREMENTS, [id=#152] +Input [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Arguments: hashpartitioning(channel#133, id#134, spark_grouping_id#135, 5), ENSURE_REQUIREMENTS, [id=#148] (77) HashAggregate [codegen id : 21] -Input [9]: [channel#137, id#138, spark_grouping_id#139, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -Keys [3]: [channel#137, id#138, spark_grouping_id#139] +Input [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Keys [3]: [channel#133, id#134, spark_grouping_id#135] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] -Aggregate Attributes [3]: [sum(sales#41)#153, sum(returns#42)#154, sum(profit#43)#155] -Results [5]: [channel#137, id#138, sum(sales#41)#153 AS sales#156, sum(returns#42)#154 AS returns#157, sum(profit#43)#155 AS profit#158] +Aggregate Attributes [3]: [sum(sales#41)#149, sum(returns#42)#150, sum(profit#43)#151] +Results [5]: [channel#133, id#134, sum(sales#41)#149 AS sales#152, sum(returns#42)#150 AS returns#153, sum(profit#43)#151 AS profit#154] (78) TakeOrderedAndProject -Input [5]: [channel#137, id#138, sales#156, returns#157, profit#158] -Arguments: 100, [channel#137 ASC NULLS FIRST, id#138 ASC NULLS FIRST], [channel#137, id#138, sales#156, returns#157, profit#158] +Input [5]: [channel#133, id#134, sales#152, returns#153, profit#154] +Arguments: 100, [channel#133 ASC NULLS FIRST, id#134 ASC NULLS FIRST], [channel#133, id#134, sales#152, returns#153, profit#154] ===== Subqueries ===== @@ -454,12 +454,12 @@ ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#67] +Output [1]: [d_date_sk#22] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt index e0cd0729e0e32..1c18ab278209b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] (3) Filter [codegen id : 1] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 1] Output [2]: [i_item_sk#1, i_manufact_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (30) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#26, _w0#27, avg_quarterly_sales#29] -Condition : (isnotnull(avg_quarterly_sales#29) AND ((avg_quarterly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_quarterly_sales#29) AND (avg_quarterly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (31) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#26, avg_quarterly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt index 3bf06577e3212..3c41db90ad1aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,reference,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] (3) Filter [codegen id : 4] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] -Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,reference,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 4] Output [2]: [i_item_sk#1, i_manufact_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manufact_id#5, specifiedwindowfra (30) Filter [codegen id : 7] Input [4]: [i_manufact_id#5, sum_sales#26, _w0#27, avg_quarterly_sales#29] -Condition : (isnotnull(avg_quarterly_sales#29) AND ((avg_quarterly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_quarterly_sales#29) AND (avg_quarterly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_quarterly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (31) Project [codegen id : 7] Output [3]: [i_manufact_id#5, sum_sales#26, avg_quarterly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt index b149bdd3e1e3f..c8058eeee7ca1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt @@ -188,7 +188,7 @@ Input [4]: [sold_date_sk#12, customer_sk#13, item_sk#14, d_date_sk#21] Output [3]: [i_item_sk#25, i_class#26, i_category#27] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women ), EqualTo(i_class,maternity ), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women), EqualTo(i_class,maternity), IsNotNull(i_item_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 6] @@ -196,7 +196,7 @@ Input [3]: [i_item_sk#25, i_class#26, i_category#27] (29) Filter [codegen id : 6] Input [3]: [i_item_sk#25, i_class#26, i_category#27] -Condition : ((((isnotnull(i_category#27) AND isnotnull(i_class#26)) AND (i_category#27 = Women )) AND (i_class#26 = maternity )) AND isnotnull(i_item_sk#25)) +Condition : ((((isnotnull(i_category#27) AND isnotnull(i_class#26)) AND (i_category#27 = Women)) AND (i_class#26 = maternity)) AND isnotnull(i_item_sk#25)) (30) Project [codegen id : 6] Output [1]: [i_item_sk#25] @@ -297,39 +297,39 @@ Input [3]: [ss_customer_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] Condition : isnotnull(ss_customer_sk#33) (51) Scan parquet default.date_dim -Output [2]: [d_date_sk#37, d_month_seq#38] +Output [2]: [d_date_sk#21, d_month_seq#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#37, d_month_seq#38] +Input [2]: [d_date_sk#21, d_month_seq#37] (53) Filter [codegen id : 12] -Input [2]: [d_date_sk#37, d_month_seq#38] -Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= Subquery scalar-subquery#39, [id=#40])) AND (d_month_seq#38 <= Subquery scalar-subquery#41, [id=#42])) AND isnotnull(d_date_sk#37)) +Input [2]: [d_date_sk#21, d_month_seq#37] +Condition : (((isnotnull(d_month_seq#37) AND (d_month_seq#37 >= Subquery scalar-subquery#38, [id=#39])) AND (d_month_seq#37 <= Subquery scalar-subquery#40, [id=#41])) AND isnotnull(d_date_sk#21)) (54) Project [codegen id : 12] -Output [1]: [d_date_sk#37] -Input [2]: [d_date_sk#37, d_month_seq#38] +Output [1]: [d_date_sk#21] +Input [2]: [d_date_sk#21, d_month_seq#37] (55) BroadcastExchange -Input [1]: [d_date_sk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#43] +Input [1]: [d_date_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (56) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ss_sold_date_sk#35] -Right keys [1]: [d_date_sk#37] +Right keys [1]: [d_date_sk#21] Join condition: None (57) Project [codegen id : 13] Output [2]: [ss_customer_sk#33, ss_ext_sales_price#34] -Input [4]: [ss_customer_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35, d_date_sk#37] +Input [4]: [ss_customer_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35, d_date_sk#21] (58) Exchange Input [2]: [ss_customer_sk#33, ss_ext_sales_price#34] -Arguments: hashpartitioning(ss_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#44] +Arguments: hashpartitioning(ss_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#43] (59) Sort [codegen id : 14] Input [2]: [ss_customer_sk#33, ss_ext_sales_price#34] @@ -348,37 +348,37 @@ Input [3]: [c_customer_sk#30, ss_customer_sk#33, ss_ext_sales_price#34] Input [2]: [c_customer_sk#30, ss_ext_sales_price#34] Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#34))] -Aggregate Attributes [1]: [sum#45] -Results [2]: [c_customer_sk#30, sum#46] +Aggregate Attributes [1]: [sum#44] +Results [2]: [c_customer_sk#30, sum#45] (63) HashAggregate [codegen id : 15] -Input [2]: [c_customer_sk#30, sum#46] +Input [2]: [c_customer_sk#30, sum#45] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#34))#47] -Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#47,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#48] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#34))#46] +Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#46,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#47] (64) HashAggregate [codegen id : 15] -Input [1]: [segment#48] -Keys [1]: [segment#48] +Input [1]: [segment#47] +Keys [1]: [segment#47] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#49] -Results [2]: [segment#48, count#50] +Aggregate Attributes [1]: [count#48] +Results [2]: [segment#47, count#49] (65) Exchange -Input [2]: [segment#48, count#50] -Arguments: hashpartitioning(segment#48, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [2]: [segment#47, count#49] +Arguments: hashpartitioning(segment#47, 5), ENSURE_REQUIREMENTS, [id=#50] (66) HashAggregate [codegen id : 16] -Input [2]: [segment#48, count#50] -Keys [1]: [segment#48] +Input [2]: [segment#47, count#49] +Keys [1]: [segment#47] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#52] -Results [3]: [segment#48, count(1)#52 AS num_customers#53, (segment#48 * 50) AS segment_base#54] +Aggregate Attributes [1]: [count(1)#51] +Results [3]: [segment#47, count(1)#51 AS num_customers#52, (segment#47 * 50) AS segment_base#53] (67) TakeOrderedAndProject -Input [3]: [segment#48, num_customers#53, segment_base#54] -Arguments: 100, [segment#48 ASC NULLS FIRST, num_customers#53 ASC NULLS FIRST], [segment#48, num_customers#53, segment_base#54] +Input [3]: [segment#47, num_customers#52, segment_base#53] +Arguments: 100, [segment#47 ASC NULLS FIRST, num_customers#52 ASC NULLS FIRST], [segment#47, num_customers#52, segment_base#53] ===== Subqueries ===== @@ -396,9 +396,9 @@ ReusedExchange (69) (69) ReusedExchange [Reuses operator id: 55] -Output [1]: [d_date_sk#37] +Output [1]: [d_date_sk#21] -Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#39, [id=#40] +Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#38, [id=#39] * HashAggregate (76) +- Exchange (75) +- * HashAggregate (74) @@ -409,42 +409,42 @@ Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (70) Scan parquet default.date_dim -Output [3]: [d_month_seq#55, d_year#56, d_moy#57] +Output [3]: [d_month_seq#37, d_year#22, d_moy#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (71) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#55, d_year#56, d_moy#57] +Input [3]: [d_month_seq#37, d_year#22, d_moy#23] (72) Filter [codegen id : 1] -Input [3]: [d_month_seq#55, d_year#56, d_moy#57] -Condition : (((isnotnull(d_year#56) AND isnotnull(d_moy#57)) AND (d_year#56 = 1998)) AND (d_moy#57 = 12)) +Input [3]: [d_month_seq#37, d_year#22, d_moy#23] +Condition : (((isnotnull(d_year#22) AND isnotnull(d_moy#23)) AND (d_year#22 = 1998)) AND (d_moy#23 = 12)) (73) Project [codegen id : 1] -Output [1]: [(d_month_seq#55 + 1) AS (d_month_seq + 1)#58] -Input [3]: [d_month_seq#55, d_year#56, d_moy#57] +Output [1]: [(d_month_seq#37 + 1) AS (d_month_seq + 1)#54] +Input [3]: [d_month_seq#37, d_year#22, d_moy#23] (74) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 1)#58] -Keys [1]: [(d_month_seq + 1)#58] +Input [1]: [(d_month_seq + 1)#54] +Keys [1]: [(d_month_seq + 1)#54] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#58] +Results [1]: [(d_month_seq + 1)#54] (75) Exchange -Input [1]: [(d_month_seq + 1)#58] -Arguments: hashpartitioning((d_month_seq + 1)#58, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [1]: [(d_month_seq + 1)#54] +Arguments: hashpartitioning((d_month_seq + 1)#54, 5), ENSURE_REQUIREMENTS, [id=#55] (76) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#58] -Keys [1]: [(d_month_seq + 1)#58] +Input [1]: [(d_month_seq + 1)#54] +Keys [1]: [(d_month_seq + 1)#54] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#58] +Results [1]: [(d_month_seq + 1)#54] -Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#40, [id=#41] * HashAggregate (83) +- Exchange (82) +- * HashAggregate (81) @@ -455,39 +455,39 @@ Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (77) Scan parquet default.date_dim -Output [3]: [d_month_seq#60, d_year#61, d_moy#62] +Output [3]: [d_month_seq#37, d_year#22, d_moy#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (78) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#60, d_year#61, d_moy#62] +Input [3]: [d_month_seq#37, d_year#22, d_moy#23] (79) Filter [codegen id : 1] -Input [3]: [d_month_seq#60, d_year#61, d_moy#62] -Condition : (((isnotnull(d_year#61) AND isnotnull(d_moy#62)) AND (d_year#61 = 1998)) AND (d_moy#62 = 12)) +Input [3]: [d_month_seq#37, d_year#22, d_moy#23] +Condition : (((isnotnull(d_year#22) AND isnotnull(d_moy#23)) AND (d_year#22 = 1998)) AND (d_moy#23 = 12)) (80) Project [codegen id : 1] -Output [1]: [(d_month_seq#60 + 3) AS (d_month_seq + 3)#63] -Input [3]: [d_month_seq#60, d_year#61, d_moy#62] +Output [1]: [(d_month_seq#37 + 3) AS (d_month_seq + 3)#56] +Input [3]: [d_month_seq#37, d_year#22, d_moy#23] (81) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 3)#63] -Keys [1]: [(d_month_seq + 3)#63] +Input [1]: [(d_month_seq + 3)#56] +Keys [1]: [(d_month_seq + 3)#56] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#63] +Results [1]: [(d_month_seq + 3)#56] (82) Exchange -Input [1]: [(d_month_seq + 3)#63] -Arguments: hashpartitioning((d_month_seq + 3)#63, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [1]: [(d_month_seq + 3)#56] +Arguments: hashpartitioning((d_month_seq + 3)#56, 5), ENSURE_REQUIREMENTS, [id=#57] (83) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#63] -Keys [1]: [(d_month_seq + 3)#63] +Input [1]: [(d_month_seq + 3)#56] +Keys [1]: [(d_month_seq + 3)#56] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#63] +Results [1]: [(d_month_seq + 3)#56] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index d0638d7606bb5..b14ae55d75093 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -109,7 +109,7 @@ Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] Output [3]: [i_item_sk#14, i_class#15, i_category#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women ), EqualTo(i_class,maternity ), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women), EqualTo(i_class,maternity), IsNotNull(i_item_sk)] ReadSchema: struct (11) ColumnarToRow [codegen id : 3] @@ -117,7 +117,7 @@ Input [3]: [i_item_sk#14, i_class#15, i_category#16] (12) Filter [codegen id : 3] Input [3]: [i_item_sk#14, i_class#15, i_category#16] -Condition : ((((isnotnull(i_category#16) AND isnotnull(i_class#15)) AND (i_category#16 = Women )) AND (i_class#15 = maternity )) AND isnotnull(i_item_sk#14)) +Condition : ((((isnotnull(i_category#16) AND isnotnull(i_class#15)) AND (i_category#16 = Women)) AND (i_class#15 = maternity)) AND isnotnull(i_item_sk#14)) (13) Project [codegen id : 3] Output [1]: [i_item_sk#14] @@ -295,75 +295,75 @@ Output [3]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28] Input [7]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28, ca_county#32, ca_state#33, s_county#35, s_state#36] (51) Scan parquet default.date_dim -Output [2]: [d_date_sk#38, d_month_seq#39] +Output [2]: [d_date_sk#18, d_month_seq#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 10] -Input [2]: [d_date_sk#38, d_month_seq#39] +Input [2]: [d_date_sk#18, d_month_seq#38] (53) Filter [codegen id : 10] -Input [2]: [d_date_sk#38, d_month_seq#39] -Condition : (((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= Subquery scalar-subquery#40, [id=#41])) AND (d_month_seq#39 <= Subquery scalar-subquery#42, [id=#43])) AND isnotnull(d_date_sk#38)) +Input [2]: [d_date_sk#18, d_month_seq#38] +Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= Subquery scalar-subquery#39, [id=#40])) AND (d_month_seq#38 <= Subquery scalar-subquery#41, [id=#42])) AND isnotnull(d_date_sk#18)) (54) Project [codegen id : 10] -Output [1]: [d_date_sk#38] -Input [2]: [d_date_sk#38, d_month_seq#39] +Output [1]: [d_date_sk#18] +Input [2]: [d_date_sk#18, d_month_seq#38] (55) BroadcastExchange -Input [1]: [d_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] +Input [1]: [d_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#43] (56) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#38] +Right keys [1]: [d_date_sk#18] Join condition: None (57) Project [codegen id : 11] Output [2]: [c_customer_sk#22, ss_ext_sales_price#27] -Input [4]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#38] +Input [4]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#18] (58) HashAggregate [codegen id : 11] Input [2]: [c_customer_sk#22, ss_ext_sales_price#27] Keys [1]: [c_customer_sk#22] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum#45] -Results [2]: [c_customer_sk#22, sum#46] +Aggregate Attributes [1]: [sum#44] +Results [2]: [c_customer_sk#22, sum#45] (59) Exchange -Input [2]: [c_customer_sk#22, sum#46] -Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [2]: [c_customer_sk#22, sum#45] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#46] (60) HashAggregate [codegen id : 12] -Input [2]: [c_customer_sk#22, sum#46] +Input [2]: [c_customer_sk#22, sum#45] Keys [1]: [c_customer_sk#22] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#48] -Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#48,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#49] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#47] +Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#47,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#48] (61) HashAggregate [codegen id : 12] -Input [1]: [segment#49] -Keys [1]: [segment#49] +Input [1]: [segment#48] +Keys [1]: [segment#48] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#50] -Results [2]: [segment#49, count#51] +Aggregate Attributes [1]: [count#49] +Results [2]: [segment#48, count#50] (62) Exchange -Input [2]: [segment#49, count#51] -Arguments: hashpartitioning(segment#49, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [2]: [segment#48, count#50] +Arguments: hashpartitioning(segment#48, 5), ENSURE_REQUIREMENTS, [id=#51] (63) HashAggregate [codegen id : 13] -Input [2]: [segment#49, count#51] -Keys [1]: [segment#49] +Input [2]: [segment#48, count#50] +Keys [1]: [segment#48] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#53] -Results [3]: [segment#49, count(1)#53 AS num_customers#54, (segment#49 * 50) AS segment_base#55] +Aggregate Attributes [1]: [count(1)#52] +Results [3]: [segment#48, count(1)#52 AS num_customers#53, (segment#48 * 50) AS segment_base#54] (64) TakeOrderedAndProject -Input [3]: [segment#49, num_customers#54, segment_base#55] -Arguments: 100, [segment#49 ASC NULLS FIRST, num_customers#54 ASC NULLS FIRST], [segment#49, num_customers#54, segment_base#55] +Input [3]: [segment#48, num_customers#53, segment_base#54] +Arguments: 100, [segment#48 ASC NULLS FIRST, num_customers#53 ASC NULLS FIRST], [segment#48, num_customers#53, segment_base#54] ===== Subqueries ===== @@ -381,9 +381,9 @@ ReusedExchange (66) (66) ReusedExchange [Reuses operator id: 55] -Output [1]: [d_date_sk#38] +Output [1]: [d_date_sk#18] -Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#40, [id=#41] +Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#39, [id=#40] * HashAggregate (73) +- Exchange (72) +- * HashAggregate (71) @@ -394,42 +394,42 @@ Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (67) Scan parquet default.date_dim -Output [3]: [d_month_seq#56, d_year#57, d_moy#58] +Output [3]: [d_month_seq#38, d_year#19, d_moy#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (68) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#56, d_year#57, d_moy#58] +Input [3]: [d_month_seq#38, d_year#19, d_moy#20] (69) Filter [codegen id : 1] -Input [3]: [d_month_seq#56, d_year#57, d_moy#58] -Condition : (((isnotnull(d_year#57) AND isnotnull(d_moy#58)) AND (d_year#57 = 1998)) AND (d_moy#58 = 12)) +Input [3]: [d_month_seq#38, d_year#19, d_moy#20] +Condition : (((isnotnull(d_year#19) AND isnotnull(d_moy#20)) AND (d_year#19 = 1998)) AND (d_moy#20 = 12)) (70) Project [codegen id : 1] -Output [1]: [(d_month_seq#56 + 1) AS (d_month_seq + 1)#59] -Input [3]: [d_month_seq#56, d_year#57, d_moy#58] +Output [1]: [(d_month_seq#38 + 1) AS (d_month_seq + 1)#55] +Input [3]: [d_month_seq#38, d_year#19, d_moy#20] (71) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 1)#59] -Keys [1]: [(d_month_seq + 1)#59] +Input [1]: [(d_month_seq + 1)#55] +Keys [1]: [(d_month_seq + 1)#55] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#59] +Results [1]: [(d_month_seq + 1)#55] (72) Exchange -Input [1]: [(d_month_seq + 1)#59] -Arguments: hashpartitioning((d_month_seq + 1)#59, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [1]: [(d_month_seq + 1)#55] +Arguments: hashpartitioning((d_month_seq + 1)#55, 5), ENSURE_REQUIREMENTS, [id=#56] (73) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#59] -Keys [1]: [(d_month_seq + 1)#59] +Input [1]: [(d_month_seq + 1)#55] +Keys [1]: [(d_month_seq + 1)#55] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#59] +Results [1]: [(d_month_seq + 1)#55] -Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#42, [id=#43] +Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (80) +- Exchange (79) +- * HashAggregate (78) @@ -440,39 +440,39 @@ Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (74) Scan parquet default.date_dim -Output [3]: [d_month_seq#61, d_year#62, d_moy#63] +Output [3]: [d_month_seq#38, d_year#19, d_moy#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (75) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#61, d_year#62, d_moy#63] +Input [3]: [d_month_seq#38, d_year#19, d_moy#20] (76) Filter [codegen id : 1] -Input [3]: [d_month_seq#61, d_year#62, d_moy#63] -Condition : (((isnotnull(d_year#62) AND isnotnull(d_moy#63)) AND (d_year#62 = 1998)) AND (d_moy#63 = 12)) +Input [3]: [d_month_seq#38, d_year#19, d_moy#20] +Condition : (((isnotnull(d_year#19) AND isnotnull(d_moy#20)) AND (d_year#19 = 1998)) AND (d_moy#20 = 12)) (77) Project [codegen id : 1] -Output [1]: [(d_month_seq#61 + 3) AS (d_month_seq + 3)#64] -Input [3]: [d_month_seq#61, d_year#62, d_moy#63] +Output [1]: [(d_month_seq#38 + 3) AS (d_month_seq + 3)#57] +Input [3]: [d_month_seq#38, d_year#19, d_moy#20] (78) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 3)#64] -Keys [1]: [(d_month_seq + 3)#64] +Input [1]: [(d_month_seq + 3)#57] +Keys [1]: [(d_month_seq + 3)#57] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#64] +Results [1]: [(d_month_seq + 3)#57] (79) Exchange -Input [1]: [(d_month_seq + 3)#64] -Arguments: hashpartitioning((d_month_seq + 3)#64, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [1]: [(d_month_seq + 3)#57] +Arguments: hashpartitioning((d_month_seq + 3)#57, 5), ENSURE_REQUIREMENTS, [id=#58] (80) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#64] -Keys [1]: [(d_month_seq + 3)#64] +Input [1]: [(d_month_seq + 3)#57] +Keys [1]: [(d_month_seq + 3)#57] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#64] +Results [1]: [(d_month_seq + 3)#57] 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 173125b197079..f0be099379418 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 @@ -213,11 +213,7 @@ Arguments: [rank(d_year#7, d_moy#8) windowspecdefinition(i_category#16, i_brand# (35) Filter [codegen id : 12] Input [8]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27] -<<<<<<< HEAD Condition : ((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#25)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) -======= -Condition : (((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#25)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#27)) ->>>>>>> regen (36) Exchange Input [8]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27] @@ -233,23 +229,22 @@ Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] (39) HashAggregate [codegen id : 21] Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] Keys [5]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33] -Functions [1]: [sum(UnscaledValue(cs_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#35))#36] -Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#35))#36,17,2) AS sum_sales#37] +Functions [1]: [sum(UnscaledValue(cs_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#3))#35] +Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#3))#35,17,2) AS sum_sales#36] (40) Exchange -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#37] (41) Sort [codegen id : 22] -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 (42) Window -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] -Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] -<<<<<<< HEAD (43) Project [codegen id : 23] Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] @@ -261,30 +256,12 @@ Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#38 + 1), (45) Sort [codegen id : 24] Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#38 + 1) ASC NULLS FIRST], false, 0 -======= -(43) Filter [codegen id : 23] -Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] -Condition : isnotnull(rn#39) - -(44) Project [codegen id : 23] -Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] -Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] - -(45) Exchange -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] - -(46) Sort [codegen id : 24] -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] -Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 ->>>>>>> regen (46) SortMergeJoin [codegen id : 25] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#39 + 1)] +Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#38 + 1)] Join condition: None -<<<<<<< HEAD (47) Project [codegen id : 25] Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] @@ -311,45 +288,12 @@ Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (rn#46 - 1), (53) Sort [codegen id : 36] Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 -======= -(48) Project [codegen id : 25] -Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] -Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] - -(49) ReusedExchange [Reuses operator id: 40] -Output [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] - -(50) Sort [codegen id : 34] -Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 - -(51) Window -Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] -Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#41, i_brand#42, cc_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#41, i_brand#42, cc_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] - -(52) Filter [codegen id : 35] -Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] -Condition : isnotnull(rn#47) - -(53) Project [codegen id : 35] -Output [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] -Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] - -(54) Exchange -Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] -Arguments: hashpartitioning(i_category#41, i_brand#42, cc_name#43, (rn#47 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] - -(55) Sort [codegen id : 36] -Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, (rn#47 - 1) ASC NULLS FIRST], false, 0 ->>>>>>> regen (54) SortMergeJoin [codegen id : 37] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#41, i_brand#42, cc_name#43, (rn#47 - 1)] +Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (rn#46 - 1)] Join condition: None -<<<<<<< HEAD (55) Project [codegen id : 37] Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#48, sum_sales#45 AS nsum#49] Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] @@ -357,15 +301,6 @@ Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales (56) TakeOrderedAndProject Input [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] -======= -(57) Project [codegen id : 37] -Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#49, sum_sales#46 AS nsum#50] -Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] - -(58) TakeOrderedAndProject -Input [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] ->>>>>>> regen ===== Subqueries ===== 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 d5b269b02d199..1eb7dbe89d6e1 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 @@ -194,11 +194,7 @@ Arguments: [rank(d_year#11, d_moy#12) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [8]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26] -<<<<<<< HEAD Condition : ((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#24)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) -======= -Condition : (((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#24)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#26)) ->>>>>>> regen (33) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] @@ -206,23 +202,22 @@ Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] (34) HashAggregate [codegen id : 13] 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] +Functions [1]: [sum(UnscaledValue(cs_sales_price#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#33] +Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#33,17,2) AS sum_sales#34] (35) 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#34] +Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#35] (36) Sort [codegen id : 14] -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#34] 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 (37) 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#34] +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#36], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] -<<<<<<< HEAD (38) Project [codegen id : 15] Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34, rn#36] @@ -230,26 +225,12 @@ Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sale (39) BroadcastExchange Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#37] -======= -(38) Filter [codegen id : 15] -Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] -Condition : isnotnull(rn#37) - -(39) Project [codegen id : 15] -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] - -(40) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] + 1)),false), [id=#38] ->>>>>>> regen (40) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -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#36 + 1)] Join condition: None -<<<<<<< HEAD (41) Project [codegen id : 23] Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34] Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] @@ -272,41 +253,12 @@ Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sale (46) BroadcastExchange Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#45] -======= -(42) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35] -Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] - -(43) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] - -(44) Sort [codegen id : 21] -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 - -(45) 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] - -(46) Filter [codegen id : 22] -Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] -Condition : isnotnull(rn#45) - -(47) Project [codegen id : 22] -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] - -(48) BroadcastExchange -Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] - 1)),false), [id=#46] ->>>>>>> regen (47) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -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#44 - 1)] Join condition: None -<<<<<<< HEAD (48) Project [codegen id : 23] Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#34 AS psum#46, sum_sales#43 AS nsum#47] Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34, i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] @@ -314,15 +266,6 @@ Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales (49) TakeOrderedAndProject Input [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, 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#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, psum#46, nsum#47] -======= -(50) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] -Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] - -(51) TakeOrderedAndProject -Input [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, 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#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, psum#47, nsum#48] ->>>>>>> regen ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index b339df0707d2f..ee3abb39ed053 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -110,30 +110,30 @@ Input [2]: [d_date_sk#4, d_date#5] Condition : isnotnull(d_date_sk#4) (7) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#6, d_week_seq#7] +Input [2]: [d_date#5, d_week_seq#6] (9) Filter [codegen id : 1] -Input [2]: [d_date#6, d_week_seq#7] -Condition : (isnotnull(d_week_seq#7) AND (d_week_seq#7 = Subquery scalar-subquery#8, [id=#9])) +Input [2]: [d_date#5, d_week_seq#6] +Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = Subquery scalar-subquery#7, [id=#8])) (10) Project [codegen id : 1] -Output [1]: [d_date#6] -Input [2]: [d_date#6, d_week_seq#7] +Output [1]: [d_date#5 AS d_date#5#9] +Input [2]: [d_date#5, d_week_seq#6] (11) BroadcastExchange -Input [1]: [d_date#6] +Input [1]: [d_date#5#9] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] (12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [d_date#5] -Right keys [1]: [d_date#6] +Right keys [1]: [d_date#5#9] Join condition: None (13) Project [codegen id : 2] @@ -218,237 +218,237 @@ Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Condition : isnotnull(cs_item_sk#21) (30) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_date#25] +Output [2]: [d_date_sk#4, d_date#5] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#24, d_date#25] +Input [2]: [d_date_sk#4, d_date#5] (32) Filter [codegen id : 6] -Input [2]: [d_date_sk#24, d_date#25] -Condition : isnotnull(d_date_sk#24) +Input [2]: [d_date_sk#4, d_date#5] +Condition : isnotnull(d_date_sk#4) (33) Scan parquet default.date_dim -Output [2]: [d_date#26, d_week_seq#27] +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (34) ColumnarToRow [codegen id : 5] -Input [2]: [d_date#26, d_week_seq#27] +Input [2]: [d_date#5, d_week_seq#6] (35) Filter [codegen id : 5] -Input [2]: [d_date#26, d_week_seq#27] -Condition : (isnotnull(d_week_seq#27) AND (d_week_seq#27 = ReusedSubquery Subquery scalar-subquery#8, [id=#9])) +Input [2]: [d_date#5, d_week_seq#6] +Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) (36) Project [codegen id : 5] -Output [1]: [d_date#26] -Input [2]: [d_date#26, d_week_seq#27] +Output [1]: [d_date#5 AS d_date#5#24] +Input [2]: [d_date#5, d_week_seq#6] (37) BroadcastExchange -Input [1]: [d_date#26] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#28] +Input [1]: [d_date#5#24] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#25] (38) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [d_date#25] -Right keys [1]: [d_date#26] +Left keys [1]: [d_date#5] +Right keys [1]: [d_date#5#24] Join condition: None (39) Project [codegen id : 6] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_date#25] +Output [1]: [d_date_sk#4] +Input [2]: [d_date_sk#4, d_date#5] (40) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] +Input [1]: [d_date_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] +Right keys [1]: [d_date_sk#4] Join condition: None (42) Project [codegen id : 8] Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] +Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#4] (43) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#30, i_item_id#31] +Output [2]: [i_item_sk#12, i_item_id#13] (44) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#30] +Right keys [1]: [i_item_sk#12] Join condition: None (45) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#22, i_item_id#31] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#30, i_item_id#31] +Output [2]: [cs_ext_sales_price#22, i_item_id#13] +Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#12, i_item_id#13] (46) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#22, i_item_id#31] -Keys [1]: [i_item_id#31] +Input [2]: [cs_ext_sales_price#22, i_item_id#13] +Keys [1]: [i_item_id#13] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#31, sum#33] +Aggregate Attributes [1]: [sum#27] +Results [2]: [i_item_id#13, sum#28] (47) Exchange -Input [2]: [i_item_id#31, sum#33] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [2]: [i_item_id#13, sum#28] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#29] (48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#31, sum#33] -Keys [1]: [i_item_id#31] +Input [2]: [i_item_id#13, sum#28] +Keys [1]: [i_item_id#13] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#35] -Results [2]: [i_item_id#31 AS item_id#36, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#35,17,2) AS cs_item_rev#37] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] +Results [2]: [i_item_id#13 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] (49) Filter [codegen id : 9] -Input [2]: [item_id#36, cs_item_rev#37] -Condition : isnotnull(cs_item_rev#37) +Input [2]: [item_id#31, cs_item_rev#32] +Condition : isnotnull(cs_item_rev#32) (50) BroadcastExchange -Input [2]: [item_id#36, cs_item_rev#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#38] +Input [2]: [item_id#31, cs_item_rev#32] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] (51) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#36] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) +Right keys [1]: [item_id#31] +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) (52) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#37] -Input [4]: [item_id#19, ss_item_rev#20, item_id#36, cs_item_rev#37] +Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] +Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] (53) Scan parquet default.web_sales -Output [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] +Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] (55) Filter [codegen id : 13] -Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] -Condition : isnotnull(ws_item_sk#39) +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#34) (56) Scan parquet default.date_dim -Output [2]: [d_date_sk#42, d_date#43] +Output [2]: [d_date_sk#4, d_date#5] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 11] -Input [2]: [d_date_sk#42, d_date#43] +Input [2]: [d_date_sk#4, d_date#5] (58) Filter [codegen id : 11] -Input [2]: [d_date_sk#42, d_date#43] -Condition : isnotnull(d_date_sk#42) +Input [2]: [d_date_sk#4, d_date#5] +Condition : isnotnull(d_date_sk#4) (59) Scan parquet default.date_dim -Output [2]: [d_date#44, d_week_seq#45] +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (60) ColumnarToRow [codegen id : 10] -Input [2]: [d_date#44, d_week_seq#45] +Input [2]: [d_date#5, d_week_seq#6] (61) Filter [codegen id : 10] -Input [2]: [d_date#44, d_week_seq#45] -Condition : (isnotnull(d_week_seq#45) AND (d_week_seq#45 = ReusedSubquery Subquery scalar-subquery#8, [id=#9])) +Input [2]: [d_date#5, d_week_seq#6] +Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) (62) Project [codegen id : 10] -Output [1]: [d_date#44] -Input [2]: [d_date#44, d_week_seq#45] +Output [1]: [d_date#5 AS d_date#5#37] +Input [2]: [d_date#5, d_week_seq#6] (63) BroadcastExchange -Input [1]: [d_date#44] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#46] +Input [1]: [d_date#5#37] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#38] (64) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_date#43] -Right keys [1]: [d_date#44] +Left keys [1]: [d_date#5] +Right keys [1]: [d_date#5#37] Join condition: None (65) Project [codegen id : 11] -Output [1]: [d_date_sk#42] -Input [2]: [d_date_sk#42, d_date#43] +Output [1]: [d_date_sk#4] +Input [2]: [d_date_sk#4, d_date#5] (66) BroadcastExchange -Input [1]: [d_date_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] +Input [1]: [d_date_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] (67) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#41] -Right keys [1]: [d_date_sk#42] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#4] Join condition: None (68) Project [codegen id : 13] -Output [2]: [ws_item_sk#39, ws_ext_sales_price#40] -Input [4]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41, d_date_sk#42] +Output [2]: [ws_item_sk#34, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#4] (69) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#48, i_item_id#49] +Output [2]: [i_item_sk#12, i_item_id#13] (70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [i_item_sk#48] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#12] Join condition: None (71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#40, i_item_id#49] -Input [4]: [ws_item_sk#39, ws_ext_sales_price#40, i_item_sk#48, i_item_id#49] +Output [2]: [ws_ext_sales_price#35, i_item_id#13] +Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, i_item_sk#12, i_item_id#13] (72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#40, i_item_id#49] -Keys [1]: [i_item_id#49] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#40))] -Aggregate Attributes [1]: [sum#50] -Results [2]: [i_item_id#49, sum#51] +Input [2]: [ws_ext_sales_price#35, i_item_id#13] +Keys [1]: [i_item_id#13] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#40] +Results [2]: [i_item_id#13, sum#41] (73) Exchange -Input [2]: [i_item_id#49, sum#51] -Arguments: hashpartitioning(i_item_id#49, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [2]: [i_item_id#13, sum#41] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#42] (74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#49, sum#51] -Keys [1]: [i_item_id#49] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#40))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#40))#53] -Results [2]: [i_item_id#49 AS item_id#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#40))#53,17,2) AS ws_item_rev#55] +Input [2]: [i_item_id#13, sum#41] +Keys [1]: [i_item_id#13] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] +Results [2]: [i_item_id#13 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] (75) Filter [codegen id : 14] -Input [2]: [item_id#54, ws_item_rev#55] -Condition : isnotnull(ws_item_rev#55) +Input [2]: [item_id#44, ws_item_rev#45] +Condition : isnotnull(ws_item_rev#45) (76) BroadcastExchange -Input [2]: [item_id#54, ws_item_rev#55] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] +Input [2]: [item_id#44, ws_item_rev#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] (77) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#54] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) +Right keys [1]: [item_id#44] +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) (78) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#57, cs_item_rev#37, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#37 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#58, ws_item_rev#55, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#55 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#59, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#60] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#37, item_id#54, ws_item_rev#55] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] +Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] (79) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] ===== Subqueries ===== -Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#8, [id=#9] +Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#7, [id=#8] * Project (83) +- * Filter (82) +- * ColumnarToRow (81) @@ -456,25 +456,25 @@ Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery (80) Scan parquet default.date_dim -Output [2]: [d_date#61, d_week_seq#62] +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct (81) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#61, d_week_seq#62] +Input [2]: [d_date#5, d_week_seq#6] (82) Filter [codegen id : 1] -Input [2]: [d_date#61, d_week_seq#62] -Condition : (isnotnull(d_date#61) AND (d_date#61 = 2000-01-03)) +Input [2]: [d_date#5, d_week_seq#6] +Condition : (isnotnull(d_date#5) AND (d_date#5 = 10959)) (83) Project [codegen id : 1] -Output [1]: [d_week_seq#62] -Input [2]: [d_date#61, d_week_seq#62] +Output [1]: [d_week_seq#6] +Input [2]: [d_date#5, d_week_seq#6] -Subquery:2 Hosting operator id = 35 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] +Subquery:2 Hosting operator id = 35 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] -Subquery:3 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] +Subquery:3 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index 1f3e6853a3c41..dd37b8801e957 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -137,30 +137,30 @@ Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) (13) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (14) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#9, d_week_seq#10] +Input [2]: [d_date#8, d_week_seq#9] (15) Filter [codegen id : 2] -Input [2]: [d_date#9, d_week_seq#10] -Condition : (isnotnull(d_week_seq#10) AND (d_week_seq#10 = Subquery scalar-subquery#11, [id=#12])) +Input [2]: [d_date#8, d_week_seq#9] +Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = Subquery scalar-subquery#10, [id=#11])) (16) Project [codegen id : 2] -Output [1]: [d_date#9] -Input [2]: [d_date#9, d_week_seq#10] +Output [1]: [d_date#8 AS d_date#8#12] +Input [2]: [d_date#8, d_week_seq#9] (17) BroadcastExchange -Input [1]: [d_date#9] +Input [1]: [d_date#8#12] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] (18) BroadcastHashJoin [codegen id : 3] Left keys [1]: [d_date#8] -Right keys [1]: [d_date#9] +Right keys [1]: [d_date#8#12] Join condition: None (19) Project [codegen id : 3] @@ -218,237 +218,237 @@ Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Condition : isnotnull(cs_item_sk#21) (30) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#24, i_item_id#25] +Output [2]: [i_item_sk#4, i_item_id#5] (31) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#24] +Right keys [1]: [i_item_sk#4] Join condition: None (32) Project [codegen id : 8] -Output [3]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25] -Input [5]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_sk#24, i_item_id#25] +Output [3]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#5] +Input [5]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_sk#4, i_item_id#5] (33) Scan parquet default.date_dim -Output [2]: [d_date_sk#26, d_date#27] +Output [2]: [d_date_sk#7, d_date#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#26, d_date#27] +Input [2]: [d_date_sk#7, d_date#8] (35) Filter [codegen id : 7] -Input [2]: [d_date_sk#26, d_date#27] -Condition : isnotnull(d_date_sk#26) +Input [2]: [d_date_sk#7, d_date#8] +Condition : isnotnull(d_date_sk#7) (36) Scan parquet default.date_dim -Output [2]: [d_date#28, d_week_seq#29] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [2]: [d_date#28, d_week_seq#29] +Input [2]: [d_date#8, d_week_seq#9] (38) Filter [codegen id : 6] -Input [2]: [d_date#28, d_week_seq#29] -Condition : (isnotnull(d_week_seq#29) AND (d_week_seq#29 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) +Input [2]: [d_date#8, d_week_seq#9] +Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) (39) Project [codegen id : 6] -Output [1]: [d_date#28] -Input [2]: [d_date#28, d_week_seq#29] +Output [1]: [d_date#8 AS d_date#8#24] +Input [2]: [d_date#8, d_week_seq#9] (40) BroadcastExchange -Input [1]: [d_date#28] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#30] +Input [1]: [d_date#8#24] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#25] (41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [d_date#27] -Right keys [1]: [d_date#28] +Left keys [1]: [d_date#8] +Right keys [1]: [d_date#8#24] Join condition: None (42) Project [codegen id : 7] -Output [1]: [d_date_sk#26] -Input [2]: [d_date_sk#26, d_date#27] +Output [1]: [d_date_sk#7] +Input [2]: [d_date_sk#7, d_date#8] (43) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (44) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#26] +Right keys [1]: [d_date_sk#7] Join condition: None (45) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#22, i_item_id#25] -Input [4]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25, d_date_sk#26] +Output [2]: [cs_ext_sales_price#22, i_item_id#5] +Input [4]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#5, d_date_sk#7] (46) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#22, i_item_id#25] -Keys [1]: [i_item_id#25] +Input [2]: [cs_ext_sales_price#22, i_item_id#5] +Keys [1]: [i_item_id#5] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#25, sum#33] +Aggregate Attributes [1]: [sum#27] +Results [2]: [i_item_id#5, sum#28] (47) Exchange -Input [2]: [i_item_id#25, sum#33] -Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [2]: [i_item_id#5, sum#28] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#29] (48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#25, sum#33] -Keys [1]: [i_item_id#25] +Input [2]: [i_item_id#5, sum#28] +Keys [1]: [i_item_id#5] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#35] -Results [2]: [i_item_id#25 AS item_id#36, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#35,17,2) AS cs_item_rev#37] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] +Results [2]: [i_item_id#5 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] (49) Filter [codegen id : 9] -Input [2]: [item_id#36, cs_item_rev#37] -Condition : isnotnull(cs_item_rev#37) +Input [2]: [item_id#31, cs_item_rev#32] +Condition : isnotnull(cs_item_rev#32) (50) BroadcastExchange -Input [2]: [item_id#36, cs_item_rev#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#38] +Input [2]: [item_id#31, cs_item_rev#32] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] (51) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#36] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) +Right keys [1]: [item_id#31] +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) (52) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#37] -Input [4]: [item_id#19, ss_item_rev#20, item_id#36, cs_item_rev#37] +Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] +Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] (53) Scan parquet default.web_sales -Output [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] +Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] (55) Filter [codegen id : 13] -Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] -Condition : isnotnull(ws_item_sk#39) +Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#34) (56) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#42, i_item_id#43] +Output [2]: [i_item_sk#4, i_item_id#5] (57) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [i_item_sk#42] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#4] Join condition: None (58) Project [codegen id : 13] -Output [3]: [ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_id#43] -Input [5]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_sk#42, i_item_id#43] +Output [3]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#5] +Input [5]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_sk#4, i_item_id#5] (59) Scan parquet default.date_dim -Output [2]: [d_date_sk#44, d_date#45] +Output [2]: [d_date_sk#7, d_date#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#44, d_date#45] +Input [2]: [d_date_sk#7, d_date#8] (61) Filter [codegen id : 12] -Input [2]: [d_date_sk#44, d_date#45] -Condition : isnotnull(d_date_sk#44) +Input [2]: [d_date_sk#7, d_date#8] +Condition : isnotnull(d_date_sk#7) (62) Scan parquet default.date_dim -Output [2]: [d_date#46, d_week_seq#47] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (63) ColumnarToRow [codegen id : 11] -Input [2]: [d_date#46, d_week_seq#47] +Input [2]: [d_date#8, d_week_seq#9] (64) Filter [codegen id : 11] -Input [2]: [d_date#46, d_week_seq#47] -Condition : (isnotnull(d_week_seq#47) AND (d_week_seq#47 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) +Input [2]: [d_date#8, d_week_seq#9] +Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) (65) Project [codegen id : 11] -Output [1]: [d_date#46] -Input [2]: [d_date#46, d_week_seq#47] +Output [1]: [d_date#8 AS d_date#8#37] +Input [2]: [d_date#8, d_week_seq#9] (66) BroadcastExchange -Input [1]: [d_date#46] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#48] +Input [1]: [d_date#8#37] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#38] (67) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [d_date#45] -Right keys [1]: [d_date#46] +Left keys [1]: [d_date#8] +Right keys [1]: [d_date#8#37] Join condition: None (68) Project [codegen id : 12] -Output [1]: [d_date_sk#44] -Input [2]: [d_date_sk#44, d_date#45] +Output [1]: [d_date_sk#7] +Input [2]: [d_date_sk#7, d_date#8] (69) BroadcastExchange -Input [1]: [d_date_sk#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] (70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#41] -Right keys [1]: [d_date_sk#44] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#7] Join condition: None (71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#40, i_item_id#43] -Input [4]: [ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_id#43, d_date_sk#44] +Output [2]: [ws_ext_sales_price#35, i_item_id#5] +Input [4]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#5, d_date_sk#7] (72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#40, i_item_id#43] -Keys [1]: [i_item_id#43] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#40))] -Aggregate Attributes [1]: [sum#50] -Results [2]: [i_item_id#43, sum#51] +Input [2]: [ws_ext_sales_price#35, i_item_id#5] +Keys [1]: [i_item_id#5] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#40] +Results [2]: [i_item_id#5, sum#41] (73) Exchange -Input [2]: [i_item_id#43, sum#51] -Arguments: hashpartitioning(i_item_id#43, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [2]: [i_item_id#5, sum#41] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#42] (74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#43, sum#51] -Keys [1]: [i_item_id#43] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#40))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#40))#53] -Results [2]: [i_item_id#43 AS item_id#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#40))#53,17,2) AS ws_item_rev#55] +Input [2]: [i_item_id#5, sum#41] +Keys [1]: [i_item_id#5] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] +Results [2]: [i_item_id#5 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] (75) Filter [codegen id : 14] -Input [2]: [item_id#54, ws_item_rev#55] -Condition : isnotnull(ws_item_rev#55) +Input [2]: [item_id#44, ws_item_rev#45] +Condition : isnotnull(ws_item_rev#45) (76) BroadcastExchange -Input [2]: [item_id#54, ws_item_rev#55] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] +Input [2]: [item_id#44, ws_item_rev#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] (77) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#54] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) +Right keys [1]: [item_id#44] +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) (78) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#57, cs_item_rev#37, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#37 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#58, ws_item_rev#55, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#55 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#59, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#60] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#37, item_id#54, ws_item_rev#55] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] +Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] (79) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] ===== Subqueries ===== -Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#10, [id=#11] * Project (83) +- * Filter (82) +- * ColumnarToRow (81) @@ -456,25 +456,25 @@ Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquer (80) Scan parquet default.date_dim -Output [2]: [d_date#61, d_week_seq#62] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct (81) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#61, d_week_seq#62] +Input [2]: [d_date#8, d_week_seq#9] (82) Filter [codegen id : 1] -Input [2]: [d_date#61, d_week_seq#62] -Condition : (isnotnull(d_date#61) AND (d_date#61 = 2000-01-03)) +Input [2]: [d_date#8, d_week_seq#9] +Condition : (isnotnull(d_date#8) AND (d_date#8 = 10959)) (83) Project [codegen id : 1] -Output [1]: [d_week_seq#62] -Input [2]: [d_date#61, d_week_seq#62] +Output [1]: [d_week_seq#9] +Input [2]: [d_date#8, d_week_seq#9] -Subquery:2 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] +Subquery:2 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] -Subquery:3 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] +Subquery:3 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] 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..fb57c1b00ecf3 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))#61, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67] +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))#61,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63,17,2) AS tue_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67,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#37, s_store_id#38] 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#37, s_store_id#38] (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#37, s_store_id#38] +Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) (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#37, s_store_id#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#68] (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#37] 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#38] +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#37, s_store_id#38] (34) Scan parquet default.date_dim -Output [2]: [d_month_seq#75, d_week_seq#76] +Output [2]: [d_month_seq#69, d_week_seq#70] 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#69, d_week_seq#70] (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#69, d_week_seq#70] +Condition : (((isnotnull(d_month_seq#69) AND (d_month_seq#69 >= 1224)) AND (d_month_seq#69 <= 1235)) AND isnotnull(d_week_seq#70)) (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#70] +Input [2]: [d_month_seq#69, d_week_seq#70] (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#70] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] (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#70] 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#72, s_store_id#38 AS s_store_id2#73, sun_sales#30 AS sun_sales2#74, mon_sales#31 AS mon_sales2#75, tue_sales#32 AS tue_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 [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#38, d_week_seq#70] (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#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_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] (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#73, (d_week_seq2#72 - 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#74)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#82, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#75)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#83, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#76)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#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 [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#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] (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)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales2)#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_sales2)#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/q59/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt index cf22f178dd33f..fb57c1b00ecf3 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))#61, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67] +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))#61,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63,17,2) AS tue_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67,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#37, s_store_id#38] 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#37, s_store_id#38] (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#37, s_store_id#38] +Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) (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#37, s_store_id#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#68] (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#37] 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#38] +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#37, s_store_id#38] (34) Scan parquet default.date_dim -Output [2]: [d_month_seq#75, d_week_seq#76] +Output [2]: [d_month_seq#69, d_week_seq#70] 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#69, d_week_seq#70] (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#69, d_week_seq#70] +Condition : (((isnotnull(d_month_seq#69) AND (d_month_seq#69 >= 1224)) AND (d_month_seq#69 <= 1235)) AND isnotnull(d_week_seq#70)) (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#70] +Input [2]: [d_month_seq#69, d_week_seq#70] (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#70] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] (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#70] 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#72, s_store_id#38 AS s_store_id2#73, sun_sales#30 AS sun_sales2#74, mon_sales#31 AS mon_sales2#75, tue_sales#32 AS tue_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 [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#38, d_week_seq#70] (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#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_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] (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#73, (d_week_seq2#72 - 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#74)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#82, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#75)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#83, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#76)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#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 [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#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] (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)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales2)#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_sales2)#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/q61.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt index 410fd9bc3d4e7..23807f0561551 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt @@ -286,100 +286,100 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#32] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#32,17,2) AS promotions#33] (48) Scan parquet default.store_sales -Output [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] (50) Filter [codegen id : 13] -Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_store_sk#36) AND isnotnull(ss_customer_sk#35)) AND isnotnull(ss_item_sk#34)) +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] +Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#39] +Output [1]: [d_date_sk#8] (52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#38] -Right keys [1]: [d_date_sk#39] +Left keys [1]: [ss_sold_date_sk#6] +Right keys [1]: [d_date_sk#8] Join condition: None (53) Project [codegen id : 13] -Output [4]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37] -Input [6]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38, d_date_sk#39] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [i_item_sk#40] +Output [1]: [i_item_sk#12] (55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#34] -Right keys [1]: [i_item_sk#40] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#12] Join condition: None (56) Project [codegen id : 13] -Output [3]: [ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37] -Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, i_item_sk#40] +Output [3]: [ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, i_item_sk#12] (57) ReusedExchange [Reuses operator id: 29] -Output [1]: [s_store_sk#41] +Output [1]: [s_store_sk#20] (58) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#36] -Right keys [1]: [s_store_sk#41] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#20] Join condition: None (59) Project [codegen id : 13] -Output [2]: [ss_customer_sk#35, ss_ext_sales_price#37] -Input [4]: [ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, s_store_sk#41] +Output [2]: [ss_customer_sk#2, ss_ext_sales_price#5] +Input [4]: [ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, s_store_sk#20] (60) ReusedExchange [Reuses operator id: 42] -Output [1]: [c_customer_sk#42] +Output [1]: [c_customer_sk#23] (61) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#35] -Right keys [1]: [c_customer_sk#42] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#23] Join condition: None (62) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#37] -Input [3]: [ss_customer_sk#35, ss_ext_sales_price#37, c_customer_sk#42] +Output [1]: [ss_ext_sales_price#5] +Input [3]: [ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#23] (63) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#37] +Input [1]: [ss_ext_sales_price#5] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#37))] -Aggregate Attributes [1]: [sum#43] -Results [1]: [sum#44] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum#34] +Results [1]: [sum#35] (64) Exchange -Input [1]: [sum#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45] +Input [1]: [sum#35] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#36] (65) HashAggregate [codegen id : 14] -Input [1]: [sum#44] +Input [1]: [sum#35] Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#37))#46] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#37))#46,17,2) AS total#47] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#37] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#37,17,2) AS total#38] (66) BroadcastExchange -Input [1]: [total#47] -Arguments: IdentityBroadcastMode, [id=#48] +Input [1]: [total#38] +Arguments: IdentityBroadcastMode, [id=#39] (67) BroadcastNestedLoopJoin [codegen id : 15] Join condition: None (68) Project [codegen id : 15] -Output [3]: [promotions#33, total#47, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#47 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#49] -Input [2]: [promotions#33, total#47] +Output [3]: [promotions#33, total#38, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#38 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] +Input [2]: [promotions#33, total#38] (69) Sort [codegen id : 15] -Input [3]: [promotions#33, total#47, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#49] -Arguments: [promotions#33 ASC NULLS FIRST, total#47 ASC NULLS FIRST], true, 0 +Input [3]: [promotions#33, total#38, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] +Arguments: [promotions#33 ASC NULLS FIRST, total#38 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -390,6 +390,6 @@ ReusedExchange (70) (70) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt index e0bee37e047cb..c0fad04ce4026 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt @@ -289,112 +289,112 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#32] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#32,17,2) AS promotions#33] (48) Scan parquet default.store_sales -Output [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] (50) Filter [codegen id : 13] -Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_store_sk#36) AND isnotnull(ss_customer_sk#35)) AND isnotnull(ss_item_sk#34)) +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] +Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [s_store_sk#39] +Output [1]: [s_store_sk#8] (52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#36] -Right keys [1]: [s_store_sk#39] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#8] Join condition: None (53) Project [codegen id : 13] -Output [4]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37, ss_sold_date_sk#38] -Input [6]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38, s_store_sk#39] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] (54) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#40] +Output [1]: [d_date_sk#16] (55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] +Left keys [1]: [ss_sold_date_sk#6] +Right keys [1]: [d_date_sk#16] Join condition: None (56) Project [codegen id : 13] -Output [3]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37] -Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37, ss_sold_date_sk#38, d_date_sk#40] +Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#16] (57) ReusedExchange [Reuses operator id: 28] -Output [2]: [c_customer_sk#41, c_current_addr_sk#42] +Output [2]: [c_customer_sk#20, c_current_addr_sk#21] (58) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#35] -Right keys [1]: [c_customer_sk#41] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#20] Join condition: None (59) Project [codegen id : 13] -Output [3]: [ss_item_sk#34, ss_ext_sales_price#37, c_current_addr_sk#42] -Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37, c_customer_sk#41, c_current_addr_sk#42] +Output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#21] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#20, c_current_addr_sk#21] (60) ReusedExchange [Reuses operator id: 35] -Output [1]: [ca_address_sk#43] +Output [1]: [ca_address_sk#23] (61) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [c_current_addr_sk#42] -Right keys [1]: [ca_address_sk#43] +Left keys [1]: [c_current_addr_sk#21] +Right keys [1]: [ca_address_sk#23] Join condition: None (62) Project [codegen id : 13] -Output [2]: [ss_item_sk#34, ss_ext_sales_price#37] -Input [4]: [ss_item_sk#34, ss_ext_sales_price#37, c_current_addr_sk#42, ca_address_sk#43] +Output [2]: [ss_item_sk#1, ss_ext_sales_price#5] +Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#21, ca_address_sk#23] (63) ReusedExchange [Reuses operator id: 42] -Output [1]: [i_item_sk#44] +Output [1]: [i_item_sk#26] (64) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#34] -Right keys [1]: [i_item_sk#44] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#26] Join condition: None (65) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#37] -Input [3]: [ss_item_sk#34, ss_ext_sales_price#37, i_item_sk#44] +Output [1]: [ss_ext_sales_price#5] +Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#26] (66) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#37] +Input [1]: [ss_ext_sales_price#5] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#37))] -Aggregate Attributes [1]: [sum#45] -Results [1]: [sum#46] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum#34] +Results [1]: [sum#35] (67) Exchange -Input [1]: [sum#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#47] +Input [1]: [sum#35] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#36] (68) HashAggregate [codegen id : 14] -Input [1]: [sum#46] +Input [1]: [sum#35] Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#37))#48] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#37))#48,17,2) AS total#49] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#37] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#37,17,2) AS total#38] (69) BroadcastExchange -Input [1]: [total#49] -Arguments: IdentityBroadcastMode, [id=#50] +Input [1]: [total#38] +Arguments: IdentityBroadcastMode, [id=#39] (70) BroadcastNestedLoopJoin [codegen id : 15] Join condition: None (71) Project [codegen id : 15] -Output [3]: [promotions#33, total#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#49 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#51] -Input [2]: [promotions#33, total#49] +Output [3]: [promotions#33, total#38, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#38 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] +Input [2]: [promotions#33, total#38] (72) Sort [codegen id : 15] -Input [3]: [promotions#33, total#49, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#51] -Arguments: [promotions#33 ASC NULLS FIRST, total#49 ASC NULLS FIRST], true, 0 +Input [3]: [promotions#33, total#38, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] +Arguments: [promotions#33 ASC NULLS FIRST, total#38 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -405,6 +405,6 @@ ReusedExchange (73) (73) ReusedExchange [Reuses operator id: 22] Output [1]: [d_date_sk#16] -Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt index 2b8dbaddacad8..3533986d86b84 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,refernece ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,refernece,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] (3) Filter [codegen id : 1] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,refernece ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,refernece,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 1] Output [2]: [i_item_sk#1, i_manager_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manager_id#5, specifiedwindowfram (30) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#26, _w0#27, avg_monthly_sales#29] -Condition : (isnotnull(avg_monthly_sales#29) AND ((avg_monthly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_monthly_sales#29) AND (avg_monthly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (31) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#26, avg_monthly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt index efb27d8cd8b11..622b073a4ae10 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt @@ -37,7 +37,7 @@ TakeOrderedAndProject (32) Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,refernece ,self-help ])),In(i_brand, [scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 ])),And(And(In(i_category, [Women ,Music ,Men ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(And(In(i_category, [Books,Children,Electronics]),In(i_class, [personal,portable,refernece,self-help])),In(i_brand, [scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8])),And(And(In(i_category, [Women,Music,Men]),In(i_class, [accessories,classical,fragrances,pants])),In(i_brand, [amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -45,7 +45,7 @@ Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] (3) Filter [codegen id : 4] Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] -Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,refernece ,self-help )) AND i_brand#2 IN (scholaramalgamalg #6 ,scholaramalgamalg #7 ,exportiunivamalg #8 ,scholaramalgamalg #8 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) +Condition : ((((i_category#4 IN (Books,Children,Electronics) AND i_class#3 IN (personal,portable,refernece,self-help)) AND i_brand#2 IN (scholaramalgamalg #6,scholaramalgamalg #7,exportiunivamalg #8,scholaramalgamalg #8)) OR ((i_category#4 IN (Women,Music,Men) AND i_class#3 IN (accessories,classical,fragrances,pants)) AND i_brand#2 IN (amalgimporto #9,edu packscholar #9,exportiimporto #9,importoamalg #9))) AND isnotnull(i_item_sk#1)) (4) Project [codegen id : 4] Output [2]: [i_item_sk#1, i_manager_id#5] @@ -169,7 +169,7 @@ Arguments: [avg(_w0#27) windowspecdefinition(i_manager_id#5, specifiedwindowfram (30) Filter [codegen id : 7] Input [4]: [i_manager_id#5, sum_sales#26, _w0#27, avg_monthly_sales#29] -Condition : (isnotnull(avg_monthly_sales#29) AND ((avg_monthly_sales#29 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_monthly_sales#29) AND (avg_monthly_sales#29 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#26 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#29 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (31) Project [codegen id : 7] Output [3]: [i_manager_id#5, sum_sales#26, avg_monthly_sales#29] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt index 931e72bba7182..5d74065179f44 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt @@ -741,7 +741,7 @@ Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt# Output [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple ,burlywood ,indian ,spring ,floral ,medium ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple,burlywood,indian,spring,floral,medium]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct (121) ColumnarToRow [codegen id : 40] @@ -749,7 +749,7 @@ Input [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] (122) Filter [codegen id : 40] Input [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] -Condition : ((((((isnotnull(i_current_price#94) AND i_color#95 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#94 >= 64.00)) AND (i_current_price#94 <= 74.00)) AND (i_current_price#94 >= 65.00)) AND (i_current_price#94 <= 79.00)) AND isnotnull(i_item_sk#93)) +Condition : ((((((isnotnull(i_current_price#94) AND i_color#95 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#94 >= 64.00)) AND (i_current_price#94 <= 74.00)) AND (i_current_price#94 >= 65.00)) AND (i_current_price#94 <= 79.00)) AND isnotnull(i_item_sk#93)) (123) Project [codegen id : 40] Output [2]: [i_item_sk#93, i_product_name#96] @@ -795,353 +795,353 @@ Input [17]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_str Arguments: [item_sk#112 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, store_zip#114 ASC NULLS FIRST], false, 0 (132) Scan parquet default.store_sales -Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#140), dynamicpruningexpression(ss_sold_date_sk#140 IN dynamicpruning#141)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#129)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (133) ColumnarToRow [codegen id : 44] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] (134) Filter [codegen id : 44] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Condition : (((((((isnotnull(ss_item_sk#129) AND isnotnull(ss_ticket_number#136)) AND isnotnull(ss_store_sk#134)) AND isnotnull(ss_customer_sk#130)) AND isnotnull(ss_cdemo_sk#131)) AND isnotnull(ss_promo_sk#135)) AND isnotnull(ss_hdemo_sk#132)) AND isnotnull(ss_addr_sk#133)) +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) (135) Exchange -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Arguments: hashpartitioning(cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint), 5), ENSURE_REQUIREMENTS, [id=#142] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: hashpartitioning(cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint), 5), ENSURE_REQUIREMENTS, [id=#130] (136) Sort [codegen id : 45] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Arguments: [cast(ss_item_sk#129 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#136 as bigint) ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: [cast(ss_item_sk#1 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#8 as bigint) ASC NULLS FIRST], false, 0 (137) ReusedExchange [Reuses operator id: 10] -Output [2]: [sr_item_sk#143, sr_ticket_number#144] +Output [2]: [sr_item_sk#15, sr_ticket_number#16] (138) Sort [codegen id : 47] -Input [2]: [sr_item_sk#143, sr_ticket_number#144] -Arguments: [sr_item_sk#143 ASC NULLS FIRST, sr_ticket_number#144 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#15, sr_ticket_number#16] +Arguments: [sr_item_sk#15 ASC NULLS FIRST, sr_ticket_number#16 ASC NULLS FIRST], false, 0 (139) SortMergeJoin [codegen id : 56] -Left keys [2]: [cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint)] -Right keys [2]: [sr_item_sk#143, sr_ticket_number#144] +Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] +Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] Join condition: None (140) Project [codegen id : 56] -Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, sr_item_sk#143, sr_ticket_number#144] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] (141) ReusedExchange [Reuses operator id: 33] -Output [1]: [cs_item_sk#145] +Output [1]: [cs_item_sk#19] (142) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_item_sk#129] -Right keys [1]: [cs_item_sk#145] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [cs_item_sk#19] Join condition: None (143) Project [codegen id : 56] -Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, cs_item_sk#145] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] (144) Scan parquet default.date_dim -Output [2]: [d_date_sk#146, d_year#147] +Output [2]: [d_date_sk#43, d_year#44] 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 (145) ColumnarToRow [codegen id : 54] -Input [2]: [d_date_sk#146, d_year#147] +Input [2]: [d_date_sk#43, d_year#44] (146) Filter [codegen id : 54] -Input [2]: [d_date_sk#146, d_year#147] -Condition : ((isnotnull(d_year#147) AND (d_year#147 = 2000)) AND isnotnull(d_date_sk#146)) +Input [2]: [d_date_sk#43, d_year#44] +Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#43)) (147) BroadcastExchange -Input [2]: [d_date_sk#146, d_year#147] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#148] +Input [2]: [d_date_sk#43, d_year#44] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#131] (148) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_sold_date_sk#140] -Right keys [1]: [d_date_sk#146] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#43] Join condition: None (149) Project [codegen id : 56] -Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147] -Input [13]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, d_date_sk#146, d_year#147] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#43, d_year#44] (150) ReusedExchange [Reuses operator id: 45] -Output [3]: [s_store_sk#149, s_store_name#150, s_zip#151] +Output [3]: [s_store_sk#46, s_store_name#47, s_zip#48] (151) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_store_sk#134] -Right keys [1]: [s_store_sk#149] +Left keys [1]: [ss_store_sk#6] +Right keys [1]: [s_store_sk#46] Join condition: None (152) Project [codegen id : 56] -Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] -Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_sk#149, s_store_name#150, s_zip#151] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_sk#46, s_store_name#47, s_zip#48] (153) Exchange -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] -Arguments: hashpartitioning(ss_customer_sk#130, 5), ENSURE_REQUIREMENTS, [id=#152] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#132] (154) Sort [codegen id : 57] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] -Arguments: [ss_customer_sk#130 ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (155) ReusedExchange [Reuses operator id: 53] -Output [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Output [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] (156) Sort [codegen id : 59] -Input [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] -Arguments: [c_customer_sk#153 ASC NULLS FIRST], false, 0 +Input [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Arguments: [c_customer_sk#51 ASC NULLS FIRST], false, 0 (157) SortMergeJoin [codegen id : 62] -Left keys [1]: [ss_customer_sk#130] -Right keys [1]: [c_customer_sk#153] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#51] Join condition: None (158) Project [codegen id : 62] -Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] -Input [18]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] (159) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#159, d_year#160] +Output [2]: [d_date_sk#133, d_year#134] (160) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_sales_date_sk#158] -Right keys [1]: [d_date_sk#159] +Left keys [1]: [c_first_sales_date_sk#56] +Right keys [1]: [d_date_sk#133] Join condition: None (161) Project [codegen id : 62] -Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160] -Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158, d_date_sk#159, d_year#160] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56, d_date_sk#133, d_year#134] (162) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#161, d_year#162] +Output [2]: [d_date_sk#135, d_year#136] (163) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_shipto_date_sk#157] -Right keys [1]: [d_date_sk#161] +Left keys [1]: [c_first_shipto_date_sk#55] +Right keys [1]: [d_date_sk#135] Join condition: None (164) Project [codegen id : 62] -Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160, d_date_sk#161, d_year#162] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134, d_date_sk#135, d_year#136] (165) Exchange -Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Arguments: hashpartitioning(ss_cdemo_sk#131, 5), ENSURE_REQUIREMENTS, [id=#163] +Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Arguments: hashpartitioning(ss_cdemo_sk#3, 5), ENSURE_REQUIREMENTS, [id=#137] (166) Sort [codegen id : 63] -Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Arguments: [ss_cdemo_sk#131 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Arguments: [ss_cdemo_sk#3 ASC NULLS FIRST], false, 0 (167) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#164, cd_marital_status#165] +Output [2]: [cd_demo_sk#64, cd_marital_status#65] (168) Sort [codegen id : 65] -Input [2]: [cd_demo_sk#164, cd_marital_status#165] -Arguments: [cd_demo_sk#164 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#64, cd_marital_status#65] +Arguments: [cd_demo_sk#64 ASC NULLS FIRST], false, 0 (169) SortMergeJoin [codegen id : 66] -Left keys [1]: [ss_cdemo_sk#131] -Right keys [1]: [cd_demo_sk#164] +Left keys [1]: [ss_cdemo_sk#3] +Right keys [1]: [cd_demo_sk#64] Join condition: None (170) Project [codegen id : 66] -Output [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] -Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_demo_sk#164, cd_marital_status#165] +Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_demo_sk#64, cd_marital_status#65] (171) Exchange -Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] -Arguments: hashpartitioning(c_current_cdemo_sk#154, 5), ENSURE_REQUIREMENTS, [id=#166] +Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] +Arguments: hashpartitioning(c_current_cdemo_sk#52, 5), ENSURE_REQUIREMENTS, [id=#138] (172) Sort [codegen id : 67] -Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] -Arguments: [c_current_cdemo_sk#154 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] +Arguments: [c_current_cdemo_sk#52 ASC NULLS FIRST], false, 0 (173) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#167, cd_marital_status#168] +Output [2]: [cd_demo_sk#139, cd_marital_status#140] (174) Sort [codegen id : 69] -Input [2]: [cd_demo_sk#167, cd_marital_status#168] -Arguments: [cd_demo_sk#167 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#139, cd_marital_status#140] +Arguments: [cd_demo_sk#139 ASC NULLS FIRST], false, 0 (175) SortMergeJoin [codegen id : 73] -Left keys [1]: [c_current_cdemo_sk#154] -Right keys [1]: [cd_demo_sk#167] -Join condition: NOT (cd_marital_status#165 = cd_marital_status#168) +Left keys [1]: [c_current_cdemo_sk#52] +Right keys [1]: [cd_demo_sk#139] +Join condition: NOT (cd_marital_status#65 = cd_marital_status#140) (176) Project [codegen id : 73] -Output [14]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Input [18]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165, cd_demo_sk#167, cd_marital_status#168] +Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65, cd_demo_sk#139, cd_marital_status#140] (177) ReusedExchange [Reuses operator id: 84] -Output [1]: [p_promo_sk#169] +Output [1]: [p_promo_sk#70] (178) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_promo_sk#135] -Right keys [1]: [p_promo_sk#169] +Left keys [1]: [ss_promo_sk#7] +Right keys [1]: [p_promo_sk#70] Join condition: None (179) Project [codegen id : 73] -Output [13]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, p_promo_sk#169] +Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, p_promo_sk#70] (180) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#170, hd_income_band_sk#171] +Output [2]: [hd_demo_sk#72, hd_income_band_sk#73] (181) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_hdemo_sk#132] -Right keys [1]: [hd_demo_sk#170] +Left keys [1]: [ss_hdemo_sk#4] +Right keys [1]: [hd_demo_sk#72] Join condition: None (182) Project [codegen id : 73] -Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171] -Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_demo_sk#170, hd_income_band_sk#171] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_demo_sk#72, hd_income_band_sk#73] (183) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#172, hd_income_band_sk#173] +Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] (184) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [c_current_hdemo_sk#155] -Right keys [1]: [hd_demo_sk#172] +Left keys [1]: [c_current_hdemo_sk#53] +Right keys [1]: [hd_demo_sk#141] Join condition: None (185) Project [codegen id : 73] -Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] -Input [15]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_demo_sk#172, hd_income_band_sk#173] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_demo_sk#141, hd_income_band_sk#142] (186) Exchange -Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] -Arguments: hashpartitioning(ss_addr_sk#133, 5), ENSURE_REQUIREMENTS, [id=#174] +Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] +Arguments: hashpartitioning(ss_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#143] (187) Sort [codegen id : 74] -Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] -Arguments: [ss_addr_sk#133 ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] +Arguments: [ss_addr_sk#5 ASC NULLS FIRST], false, 0 (188) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Output [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] (189) Sort [codegen id : 76] -Input [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -Arguments: [ca_address_sk#175 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Arguments: [ca_address_sk#78 ASC NULLS FIRST], false, 0 (190) SortMergeJoin [codegen id : 77] -Left keys [1]: [ss_addr_sk#133] -Right keys [1]: [ca_address_sk#175] +Left keys [1]: [ss_addr_sk#5] +Right keys [1]: [ca_address_sk#78] Join condition: None (191) Project [codegen id : 77] -Output [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -Input [18]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] (192) Exchange -Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -Arguments: hashpartitioning(c_current_addr_sk#156, 5), ENSURE_REQUIREMENTS, [id=#180] +Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Arguments: hashpartitioning(c_current_addr_sk#54, 5), ENSURE_REQUIREMENTS, [id=#144] (193) Sort [codegen id : 78] -Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -Arguments: [c_current_addr_sk#156 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Arguments: [c_current_addr_sk#54 ASC NULLS FIRST], false, 0 (194) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Output [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] (195) Sort [codegen id : 80] -Input [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] -Arguments: [ca_address_sk#181 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Arguments: [ca_address_sk#145 ASC NULLS FIRST], false, 0 (196) SortMergeJoin [codegen id : 84] -Left keys [1]: [c_current_addr_sk#156] -Right keys [1]: [ca_address_sk#181] +Left keys [1]: [c_current_addr_sk#54] +Right keys [1]: [ca_address_sk#145] Join condition: None (197) Project [codegen id : 84] -Output [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] -Input [21]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] (198) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#186] +Output [1]: [ib_income_band_sk#90] (199) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#171] -Right keys [1]: [ib_income_band_sk#186] +Left keys [1]: [hd_income_band_sk#73] +Right keys [1]: [ib_income_band_sk#90] Join condition: None (200) Project [codegen id : 84] -Output [18]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] -Input [20]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#186] +Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#90] (201) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#187] +Output [1]: [ib_income_band_sk#150] (202) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#173] -Right keys [1]: [ib_income_band_sk#187] +Left keys [1]: [hd_income_band_sk#142] +Right keys [1]: [ib_income_band_sk#150] Join condition: None (203) Project [codegen id : 84] -Output [17]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] -Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#187] +Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#150] (204) ReusedExchange [Reuses operator id: 124] -Output [2]: [i_item_sk#188, i_product_name#189] +Output [2]: [i_item_sk#93, i_product_name#96] (205) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#129] -Right keys [1]: [i_item_sk#188] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#93] Join condition: None (206) Project [codegen id : 84] -Output [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] -Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] +Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] (207) HashAggregate [codegen id : 84] -Input [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] -Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#137)), partial_sum(UnscaledValue(ss_list_price#138)), partial_sum(UnscaledValue(ss_coupon_amt#139))] -Aggregate Attributes [4]: [count#190, sum#191, sum#192, sum#193] -Results [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] +Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count#151, sum#152, sum#153, sum#154] +Results [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] (208) Exchange -Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] -Arguments: hashpartitioning(i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, 5), ENSURE_REQUIREMENTS, [id=#198] +Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] +Arguments: hashpartitioning(i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, 5), ENSURE_REQUIREMENTS, [id=#159] (209) HashAggregate [codegen id : 85] -Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] -Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#137)), sum(UnscaledValue(ss_list_price#138)), sum(UnscaledValue(ss_coupon_amt#139))] -Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#137))#200, sum(UnscaledValue(ss_list_price#138))#201, sum(UnscaledValue(ss_coupon_amt#139))#202] -Results [8]: [i_item_sk#188 AS item_sk#203, s_store_name#150 AS store_name#204, s_zip#151 AS store_zip#205, d_year#147 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#137))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#138))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#139))#202,17,2) AS s3#210] +Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] +Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count(1)#160, sum(UnscaledValue(ss_wholesale_cost#9))#161, sum(UnscaledValue(ss_list_price#10))#162, sum(UnscaledValue(ss_coupon_amt#11))#163] +Results [8]: [i_item_sk#93 AS item_sk#164, s_store_name#47 AS store_name#165, s_zip#48 AS store_zip#166, d_year#44 AS syear#167, count(1)#160 AS cnt#168, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#161,17,2) AS s1#169, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#162,17,2) AS s2#170, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#163,17,2) AS s3#171] (210) Exchange -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] +Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] +Arguments: hashpartitioning(item_sk#164, store_name#165, store_zip#166, 5), ENSURE_REQUIREMENTS, [id=#172] (211) Sort [codegen id : 86] -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] +Arguments: [item_sk#164 ASC NULLS FIRST, store_name#165 ASC NULLS FIRST, store_zip#166 ASC NULLS FIRST], false, 0 (212) SortMergeJoin [codegen id : 87] Left keys [3]: [item_sk#112, store_name#113, store_zip#114] -Right keys [3]: [item_sk#203, store_name#204, store_zip#205] -Join condition: (cnt#207 <= cnt#124) +Right keys [3]: [item_sk#164, store_name#165, store_zip#166] +Join condition: (cnt#168 <= cnt#124) (213) Project [codegen id : 87] -Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] -Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] +Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] (214) Exchange -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] +Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#173] (215) Sort [codegen id : 88] -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] +Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1152,11 +1152,11 @@ ReusedExchange (216) (216) ReusedExchange [Reuses operator id: 39] Output [2]: [d_date_sk#43, d_year#44] -Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#140 IN dynamicpruning#141 +Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#129 ReusedExchange (217) (217) ReusedExchange [Reuses operator id: 147] -Output [2]: [d_date_sk#146, d_year#147] +Output [2]: [d_date_sk#43, d_year#44] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index a866c8aa9b486..9cc3415e6e146 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt @@ -651,7 +651,7 @@ Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt# Output [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple ,burlywood ,indian ,spring ,floral ,medium ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple,burlywood,indian,spring,floral,medium]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct (105) ColumnarToRow [codegen id : 24] @@ -659,7 +659,7 @@ Input [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] (106) Filter [codegen id : 24] Input [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] -Condition : ((((((isnotnull(i_current_price#88) AND i_color#89 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#88 >= 64.00)) AND (i_current_price#88 <= 74.00)) AND (i_current_price#88 >= 65.00)) AND (i_current_price#88 <= 79.00)) AND isnotnull(i_item_sk#87)) +Condition : ((((((isnotnull(i_current_price#88) AND i_color#89 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#88 >= 64.00)) AND (i_current_price#88 <= 74.00)) AND (i_current_price#88 >= 65.00)) AND (i_current_price#88 <= 79.00)) AND isnotnull(i_item_sk#87)) (107) Project [codegen id : 24] Output [2]: [i_item_sk#87, i_product_name#90] @@ -701,323 +701,323 @@ Input [17]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_str Arguments: [item_sk#105 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, store_zip#107 ASC NULLS FIRST], false, 0 (115) Scan parquet default.store_sales -Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#133), dynamicpruningexpression(ss_sold_date_sk#133 IN dynamicpruning#134)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#122)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (116) ColumnarToRow [codegen id : 27] -Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] (117) Filter [codegen id : 27] -Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Condition : (((((((isnotnull(ss_item_sk#122) AND isnotnull(ss_ticket_number#129)) AND isnotnull(ss_store_sk#127)) AND isnotnull(ss_customer_sk#123)) AND isnotnull(ss_cdemo_sk#124)) AND isnotnull(ss_promo_sk#128)) AND isnotnull(ss_hdemo_sk#125)) AND isnotnull(ss_addr_sk#126)) +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) (118) BroadcastExchange -Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#135] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#123] (119) Scan parquet default.store_returns -Output [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Output [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct (120) ColumnarToRow -Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] (121) Filter -Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] -Condition : (isnotnull(sr_item_sk#136) AND isnotnull(sr_ticket_number#137)) +Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Condition : (isnotnull(sr_item_sk#15) AND isnotnull(sr_ticket_number#16)) (122) Project -Output [2]: [sr_item_sk#136, sr_ticket_number#137] -Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Output [2]: [sr_item_sk#15, sr_ticket_number#16] +Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] (123) BroadcastHashJoin [codegen id : 28] -Left keys [2]: [cast(ss_item_sk#122 as bigint), cast(ss_ticket_number#129 as bigint)] -Right keys [2]: [sr_item_sk#136, sr_ticket_number#137] +Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] +Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] Join condition: None (124) Project [codegen id : 28] -Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, sr_item_sk#136, sr_ticket_number#137] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] (125) Exchange -Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Arguments: hashpartitioning(ss_item_sk#122, 5), ENSURE_REQUIREMENTS, [id=#139] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#124] (126) Sort [codegen id : 29] -Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Arguments: [ss_item_sk#122 ASC NULLS FIRST], false, 0 +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (127) ReusedExchange [Reuses operator id: 28] -Output [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] +Output [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] (128) HashAggregate [codegen id : 35] -Input [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] -Keys [1]: [cs_item_sk#140] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#144)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#144))#148, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149] -Results [3]: [cs_item_sk#140, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#144))#148,17,2) AS sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] +Input [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] +Keys [1]: [cs_item_sk#19] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#21)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#21))#128, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129] +Results [3]: [cs_item_sk#19, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#21))#128,17,2) AS sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] (129) Filter [codegen id : 35] -Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] -Condition : (isnotnull(sum(cs_ext_list_price#144)#150) AND (cast(sum(cs_ext_list_price#144)#150 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151)), DecimalType(21,2), true))) +Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] +Condition : (isnotnull(sum(cs_ext_list_price#21)#130) AND (cast(sum(cs_ext_list_price#21)#130 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131)), DecimalType(21,2), true))) (130) Project [codegen id : 35] -Output [1]: [cs_item_sk#140] -Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] +Output [1]: [cs_item_sk#19] +Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] (131) Sort [codegen id : 35] -Input [1]: [cs_item_sk#140] -Arguments: [cs_item_sk#140 ASC NULLS FIRST], false, 0 +Input [1]: [cs_item_sk#19] +Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 (132) SortMergeJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#122] -Right keys [1]: [cs_item_sk#140] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [cs_item_sk#19] Join condition: None (133) Project [codegen id : 51] -Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, cs_item_sk#140] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] (134) Scan parquet default.date_dim -Output [2]: [d_date_sk#152, d_year#153] +Output [2]: [d_date_sk#42, d_year#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (135) ColumnarToRow [codegen id : 36] -Input [2]: [d_date_sk#152, d_year#153] +Input [2]: [d_date_sk#42, d_year#43] (136) Filter [codegen id : 36] -Input [2]: [d_date_sk#152, d_year#153] -Condition : ((isnotnull(d_year#153) AND (d_year#153 = 2000)) AND isnotnull(d_date_sk#152)) +Input [2]: [d_date_sk#42, d_year#43] +Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2000)) AND isnotnull(d_date_sk#42)) (137) BroadcastExchange -Input [2]: [d_date_sk#152, d_year#153] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#154] +Input [2]: [d_date_sk#42, d_year#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#132] (138) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_sold_date_sk#133] -Right keys [1]: [d_date_sk#152] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#42] Join condition: None (139) Project [codegen id : 51] -Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153] -Input [13]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, d_date_sk#152, d_year#153] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#42, d_year#43] (140) ReusedExchange [Reuses operator id: 44] -Output [3]: [s_store_sk#155, s_store_name#156, s_zip#157] +Output [3]: [s_store_sk#45, s_store_name#46, s_zip#47] (141) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_store_sk#127] -Right keys [1]: [s_store_sk#155] +Left keys [1]: [ss_store_sk#6] +Right keys [1]: [s_store_sk#45] Join condition: None (142) Project [codegen id : 51] -Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157] -Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_sk#155, s_store_name#156, s_zip#157] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_sk#45, s_store_name#46, s_zip#47] (143) ReusedExchange [Reuses operator id: 50] -Output [6]: [c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] +Output [6]: [c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] (144) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_customer_sk#123] -Right keys [1]: [c_customer_sk#158] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#49] Join condition: None (145) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] -Input [18]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] (146) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#164, d_year#165] +Output [2]: [d_date_sk#133, d_year#134] (147) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_sales_date_sk#163] -Right keys [1]: [d_date_sk#164] +Left keys [1]: [c_first_sales_date_sk#54] +Right keys [1]: [d_date_sk#133] Join condition: None (148) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165] -Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163, d_date_sk#164, d_year#165] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54, d_date_sk#133, d_year#134] (149) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#166, d_year#167] +Output [2]: [d_date_sk#135, d_year#136] (150) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_shipto_date_sk#162] -Right keys [1]: [d_date_sk#166] +Left keys [1]: [c_first_shipto_date_sk#53] +Right keys [1]: [d_date_sk#135] Join condition: None (151) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] -Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165, d_date_sk#166, d_year#167] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134, d_date_sk#135, d_year#136] (152) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#168, cd_marital_status#169] +Output [2]: [cd_demo_sk#61, cd_marital_status#62] (153) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_cdemo_sk#124] -Right keys [1]: [cd_demo_sk#168] +Left keys [1]: [ss_cdemo_sk#3] +Right keys [1]: [cd_demo_sk#61] Join condition: None (154) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169] -Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_demo_sk#168, cd_marital_status#169] +Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_demo_sk#61, cd_marital_status#62] (155) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#170, cd_marital_status#171] +Output [2]: [cd_demo_sk#137, cd_marital_status#138] (156) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_cdemo_sk#159] -Right keys [1]: [cd_demo_sk#170] -Join condition: NOT (cd_marital_status#169 = cd_marital_status#171) +Left keys [1]: [c_current_cdemo_sk#50] +Right keys [1]: [cd_demo_sk#137] +Join condition: NOT (cd_marital_status#62 = cd_marital_status#138) (157) Project [codegen id : 51] -Output [14]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] -Input [18]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169, cd_demo_sk#170, cd_marital_status#171] +Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62, cd_demo_sk#137, cd_marital_status#138] (158) ReusedExchange [Reuses operator id: 74] -Output [1]: [p_promo_sk#172] +Output [1]: [p_promo_sk#66] (159) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_promo_sk#128] -Right keys [1]: [p_promo_sk#172] +Left keys [1]: [ss_promo_sk#7] +Right keys [1]: [p_promo_sk#66] Join condition: None (160) Project [codegen id : 51] -Output [13]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] -Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, p_promo_sk#172] +Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, p_promo_sk#66] (161) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#173, hd_income_band_sk#174] +Output [2]: [hd_demo_sk#68, hd_income_band_sk#69] (162) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_hdemo_sk#125] -Right keys [1]: [hd_demo_sk#173] +Left keys [1]: [ss_hdemo_sk#4] +Right keys [1]: [hd_demo_sk#68] Join condition: None (163) Project [codegen id : 51] -Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174] -Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_demo_sk#173, hd_income_band_sk#174] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_demo_sk#68, hd_income_band_sk#69] (164) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#175, hd_income_band_sk#176] +Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] (165) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_hdemo_sk#160] -Right keys [1]: [hd_demo_sk#175] +Left keys [1]: [c_current_hdemo_sk#51] +Right keys [1]: [hd_demo_sk#139] Join condition: None (166) Project [codegen id : 51] -Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176] -Input [15]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_demo_sk#175, hd_income_band_sk#176] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140] +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_demo_sk#139, hd_income_band_sk#140] (167) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] +Output [5]: [ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] (168) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_addr_sk#126] -Right keys [1]: [ca_address_sk#177] +Left keys [1]: [ss_addr_sk#5] +Right keys [1]: [ca_address_sk#73] Join condition: None (169) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] -Input [18]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] +Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] (170) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] (171) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_addr_sk#161] -Right keys [1]: [ca_address_sk#182] +Left keys [1]: [c_current_addr_sk#52] +Right keys [1]: [ca_address_sk#141] Join condition: None (172) Project [codegen id : 51] -Output [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] -Input [21]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] (173) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#187] +Output [1]: [ib_income_band_sk#84] (174) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#174] -Right keys [1]: [ib_income_band_sk#187] +Left keys [1]: [hd_income_band_sk#69] +Right keys [1]: [ib_income_band_sk#84] Join condition: None (175) Project [codegen id : 51] -Output [18]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] -Input [20]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#187] +Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#84] (176) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#188] +Output [1]: [ib_income_band_sk#146] (177) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#176] -Right keys [1]: [ib_income_band_sk#188] +Left keys [1]: [hd_income_band_sk#140] +Right keys [1]: [ib_income_band_sk#146] Join condition: None (178) Project [codegen id : 51] -Output [17]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] -Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#188] +Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#146] (179) ReusedExchange [Reuses operator id: 108] -Output [2]: [i_item_sk#189, i_product_name#190] +Output [2]: [i_item_sk#87, i_product_name#90] (180) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#122] -Right keys [1]: [i_item_sk#189] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#87] Join condition: None (181) Project [codegen id : 51] -Output [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] -Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] (182) HashAggregate [codegen id : 51] -Input [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] -Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#130)), partial_sum(UnscaledValue(ss_list_price#131)), partial_sum(UnscaledValue(ss_coupon_amt#132))] -Aggregate Attributes [4]: [count#191, sum#192, sum#193, sum#194] -Results [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] +Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count#147, sum#148, sum#149, sum#150] +Results [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] (183) HashAggregate [codegen id : 51] -Input [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] -Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#130)), sum(UnscaledValue(ss_list_price#131)), sum(UnscaledValue(ss_coupon_amt#132))] -Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#130))#200, sum(UnscaledValue(ss_list_price#131))#201, sum(UnscaledValue(ss_coupon_amt#132))#202] -Results [8]: [i_item_sk#189 AS item_sk#203, s_store_name#156 AS store_name#204, s_zip#157 AS store_zip#205, d_year#153 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#130))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#131))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#132))#202,17,2) AS s3#210] +Input [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] +Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count(1)#155, sum(UnscaledValue(ss_wholesale_cost#9))#156, sum(UnscaledValue(ss_list_price#10))#157, sum(UnscaledValue(ss_coupon_amt#11))#158] +Results [8]: [i_item_sk#87 AS item_sk#159, s_store_name#46 AS store_name#160, s_zip#47 AS store_zip#161, d_year#43 AS syear#162, count(1)#155 AS cnt#163, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#156,17,2) AS s1#164, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#157,17,2) AS s2#165, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#158,17,2) AS s3#166] (184) Exchange -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] +Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] +Arguments: hashpartitioning(item_sk#159, store_name#160, store_zip#161, 5), ENSURE_REQUIREMENTS, [id=#167] (185) Sort [codegen id : 52] -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] +Arguments: [item_sk#159 ASC NULLS FIRST, store_name#160 ASC NULLS FIRST, store_zip#161 ASC NULLS FIRST], false, 0 (186) SortMergeJoin [codegen id : 53] Left keys [3]: [item_sk#105, store_name#106, store_zip#107] -Right keys [3]: [item_sk#203, store_name#204, store_zip#205] -Join condition: (cnt#207 <= cnt#117) +Right keys [3]: [item_sk#159, store_name#160, store_zip#161] +Join condition: (cnt#163 <= cnt#117) (187) Project [codegen id : 53] -Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] -Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] +Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] (188) Exchange -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] +Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#168] (189) Sort [codegen id : 54] -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] +Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1028,11 +1028,11 @@ ReusedExchange (190) (190) ReusedExchange [Reuses operator id: 38] Output [2]: [d_date_sk#42, d_year#43] -Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#133 IN dynamicpruning#134 +Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#122 ReusedExchange (191) (191) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#152, d_year#153] +Output [2]: [d_date_sk#42, d_year#43] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt index 84e49ab9373e4..41c2ffcc75e7a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt @@ -130,135 +130,135 @@ Input [4]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17 Condition : isnotnull(ss_store_sk#15) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#6] (19) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#17] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#6] Join condition: None (20) Project [codegen id : 4] Output [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] -Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#18] +Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#6] (21) HashAggregate [codegen id : 4] Input [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] +Aggregate Attributes [1]: [sum#18] +Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] (22) Exchange -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] -Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] +Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#20] (23) HashAggregate [codegen id : 5] -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#22] -Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#22,17,2) AS revenue#23] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#21] +Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#21,17,2) AS revenue#22] (24) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#15, revenue#23] +Input [2]: [ss_store_sk#15, revenue#22] Keys [1]: [ss_store_sk#15] -Functions [1]: [partial_avg(revenue#23)] -Aggregate Attributes [2]: [sum#24, count#25] -Results [3]: [ss_store_sk#15, sum#26, count#27] +Functions [1]: [partial_avg(revenue#22)] +Aggregate Attributes [2]: [sum#23, count#24] +Results [3]: [ss_store_sk#15, sum#25, count#26] (25) Exchange -Input [3]: [ss_store_sk#15, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [ss_store_sk#15, sum#25, count#26] +Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#27] (26) HashAggregate [codegen id : 6] -Input [3]: [ss_store_sk#15, sum#26, count#27] +Input [3]: [ss_store_sk#15, sum#25, count#26] Keys [1]: [ss_store_sk#15] -Functions [1]: [avg(revenue#23)] -Aggregate Attributes [1]: [avg(revenue#23)#29] -Results [2]: [ss_store_sk#15, avg(revenue#23)#29 AS ave#30] +Functions [1]: [avg(revenue#22)] +Aggregate Attributes [1]: [avg(revenue#22)#28] +Results [2]: [ss_store_sk#15, avg(revenue#22)#28 AS ave#29] (27) BroadcastExchange -Input [2]: [ss_store_sk#15, ave#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [2]: [ss_store_sk#15, ave#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] (28) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] Right keys [1]: [ss_store_sk#15] -Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#30)), DecimalType(23,7), true)) +Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#29)), DecimalType(23,7), true)) (29) Project [codegen id : 8] Output [3]: [ss_store_sk#2, ss_item_sk#1, revenue#13] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#30] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#29] (30) Scan parquet default.store -Output [2]: [s_store_sk#32, s_store_name#33] +Output [2]: [s_store_sk#31, s_store_name#32] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#32, s_store_name#33] +Input [2]: [s_store_sk#31, s_store_name#32] (32) Filter [codegen id : 7] -Input [2]: [s_store_sk#32, s_store_name#33] -Condition : isnotnull(s_store_sk#32) +Input [2]: [s_store_sk#31, s_store_name#32] +Condition : isnotnull(s_store_sk#31) (33) BroadcastExchange -Input [2]: [s_store_sk#32, s_store_name#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [2]: [s_store_sk#31, s_store_name#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] (34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#32] +Right keys [1]: [s_store_sk#31] Join condition: None (35) Project [codegen id : 8] -Output [3]: [ss_item_sk#1, revenue#13, s_store_name#33] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#32, s_store_name#33] +Output [3]: [ss_item_sk#1, revenue#13, s_store_name#32] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#31, s_store_name#32] (36) Exchange -Input [3]: [ss_item_sk#1, revenue#13, s_store_name#33] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [ss_item_sk#1, revenue#13, s_store_name#32] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#34] (37) Sort [codegen id : 9] -Input [3]: [ss_item_sk#1, revenue#13, s_store_name#33] +Input [3]: [ss_item_sk#1, revenue#13, s_store_name#32] Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (38) Scan parquet default.item -Output [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Output [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 10] -Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] (40) Filter [codegen id : 10] -Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] -Condition : isnotnull(i_item_sk#36) +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Condition : isnotnull(i_item_sk#35) (41) Exchange -Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] -Arguments: hashpartitioning(i_item_sk#36, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Arguments: hashpartitioning(i_item_sk#35, 5), ENSURE_REQUIREMENTS, [id=#40] (42) Sort [codegen id : 11] -Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] -Arguments: [i_item_sk#36 ASC NULLS FIRST], false, 0 +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Arguments: [i_item_sk#35 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 12] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#36] +Right keys [1]: [i_item_sk#35] Join condition: None (44) Project [codegen id : 12] -Output [6]: [s_store_name#33, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] -Input [8]: [ss_item_sk#1, revenue#13, s_store_name#33, i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Output [6]: [s_store_name#32, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [8]: [ss_item_sk#1, revenue#13, s_store_name#32, i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] (45) TakeOrderedAndProject -Input [6]: [s_store_name#33, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] -Arguments: 100, [s_store_name#33 ASC NULLS FIRST, i_item_desc#37 ASC NULLS FIRST], [s_store_name#33, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Input [6]: [s_store_name#32, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Arguments: 100, [s_store_name#32 ASC NULLS FIRST, i_item_desc#36 ASC NULLS FIRST], [s_store_name#32, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt index 45c7c051601c5..52de9873db590 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt @@ -181,65 +181,65 @@ Input [4]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26 Condition : isnotnull(ss_store_sk#24) (30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#8] (31) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#8] Join condition: None (32) Project [codegen id : 6] Output [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] -Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#27] +Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#8] (33) HashAggregate [codegen id : 6] Input [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#28] -Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] +Aggregate Attributes [1]: [sum#27] +Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] (34) Exchange -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] -Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] +Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#29] (35) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] -Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS revenue#32] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#30] +Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#30,17,2) AS revenue#31] (36) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#24, revenue#32] +Input [2]: [ss_store_sk#24, revenue#31] Keys [1]: [ss_store_sk#24] -Functions [1]: [partial_avg(revenue#32)] -Aggregate Attributes [2]: [sum#33, count#34] -Results [3]: [ss_store_sk#24, sum#35, count#36] +Functions [1]: [partial_avg(revenue#31)] +Aggregate Attributes [2]: [sum#32, count#33] +Results [3]: [ss_store_sk#24, sum#34, count#35] (37) Exchange -Input [3]: [ss_store_sk#24, sum#35, count#36] -Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [ss_store_sk#24, sum#34, count#35] +Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#36] (38) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#24, sum#35, count#36] +Input [3]: [ss_store_sk#24, sum#34, count#35] Keys [1]: [ss_store_sk#24] -Functions [1]: [avg(revenue#32)] -Aggregate Attributes [1]: [avg(revenue#32)#38] -Results [2]: [ss_store_sk#24, avg(revenue#32)#38 AS ave#39] +Functions [1]: [avg(revenue#31)] +Aggregate Attributes [1]: [avg(revenue#31)#37] +Results [2]: [ss_store_sk#24, avg(revenue#31)#37 AS ave#38] (39) BroadcastExchange -Input [2]: [ss_store_sk#24, ave#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] +Input [2]: [ss_store_sk#24, ave#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#24] -Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#39)), DecimalType(23,7), true)) +Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#38)), DecimalType(23,7), true)) (41) Project [codegen id : 9] Output [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#39] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#38] (42) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt index 8c342961cf970..51298b80bbbbe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt @@ -221,70 +221,70 @@ Input [7]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_qu Condition : ((isnotnull(cs_warehouse_sk#176) AND isnotnull(cs_sold_time_sk#174)) AND isnotnull(cs_ship_mode_sk#175)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [sm_ship_mode_sk#181] +Output [1]: [sm_ship_mode_sk#9] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_mode_sk#175] -Right keys [1]: [sm_ship_mode_sk#181] +Right keys [1]: [sm_ship_mode_sk#9] Join condition: None (38) Project [codegen id : 11] Output [6]: [cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180] -Input [8]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, sm_ship_mode_sk#181] +Input [8]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, sm_ship_mode_sk#9] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [t_time_sk#182] +Output [1]: [t_time_sk#12] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_time_sk#174] -Right keys [1]: [t_time_sk#182] +Right keys [1]: [t_time_sk#12] Join condition: None (41) Project [codegen id : 11] Output [5]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180] -Input [7]: [cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, t_time_sk#182] +Input [7]: [cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, t_time_sk#12] (42) ReusedExchange [Reuses operator id: 21] -Output [3]: [d_date_sk#183, d_year#184, d_moy#185] +Output [3]: [d_date_sk#15, d_year#16, d_moy#17] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#180] -Right keys [1]: [d_date_sk#183] +Right keys [1]: [d_date_sk#15] Join condition: None (44) Project [codegen id : 11] -Output [6]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#184, d_moy#185] -Input [8]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, d_date_sk#183, d_year#184, d_moy#185] +Output [6]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#16, d_moy#17] +Input [8]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, d_date_sk#15, d_year#16, d_moy#17] (45) ReusedExchange [Reuses operator id: 27] -Output [7]: [w_warehouse_sk#186, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192] +Output [7]: [w_warehouse_sk#19, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25] (46) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_warehouse_sk#176] -Right keys [1]: [w_warehouse_sk#186] +Right keys [1]: [w_warehouse_sk#19] Join condition: None (47) Project [codegen id : 11] -Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, d_moy#185] -Input [13]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#184, d_moy#185, w_warehouse_sk#186, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192] +Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, d_moy#17] +Input [13]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#16, d_moy#17, w_warehouse_sk#19, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25] (48) HashAggregate [codegen id : 11] -Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, d_moy#185] -Keys [7]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184] -Functions [24]: [partial_sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240] -Results [55]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] +Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, d_moy#17] +Keys [7]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16] +Functions [24]: [partial_sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [48]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228] +Results [55]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] (49) Exchange -Input [55]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] -Arguments: hashpartitioning(w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, 5), ENSURE_REQUIREMENTS, [id=#289] +Input [55]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] +Arguments: hashpartitioning(w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#277] (50) HashAggregate [codegen id : 12] -Input [55]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] -Keys [7]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184] -Functions [24]: [sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313] -Results [32]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, DHL,BARIAN AS ship_carriers#314, d_year#184 AS year#315, sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_sales#316, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_sales#317, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_sales#318, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_sales#319, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_sales#320, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_sales#321, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_sales#322, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_sales#323, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_sales#324, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_sales#325, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_sales#326, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_sales#327, sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302 AS jan_net#328, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303 AS feb_net#329, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304 AS mar_net#330, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305 AS apr_net#331, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306 AS may_net#332, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307 AS jun_net#333, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308 AS jul_net#334, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309 AS aug_net#335, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310 AS sep_net#336, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311 AS oct_net#337, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312 AS nov_net#338, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313 AS dec_net#339] +Input [55]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] +Keys [7]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16] +Functions [24]: [sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301] +Results [32]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, DHL,BARIAN AS ship_carriers#302, d_year#16 AS year#303, sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278 AS jan_sales#304, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279 AS feb_sales#305, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280 AS mar_sales#306, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281 AS apr_sales#307, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282 AS may_sales#308, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283 AS jun_sales#309, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284 AS jul_sales#310, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285 AS aug_sales#311, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286 AS sep_sales#312, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287 AS oct_sales#313, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288 AS nov_sales#314, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289 AS dec_sales#315, sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_net#316, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_net#317, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_net#318, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_net#319, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_net#320, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_net#321, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_net#322, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_net#323, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_net#324, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_net#325, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_net#326, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_net#327] (51) Union @@ -292,23 +292,23 @@ Results [32]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county Input [32]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#150, feb_sales#151, mar_sales#152, apr_sales#153, may_sales#154, jun_sales#155, jul_sales#156, aug_sales#157, sep_sales#158, oct_sales#159, nov_sales#160, dec_sales#161, jan_net#162, feb_net#163, mar_net#164, apr_net#165, may_net#166, jun_net#167, jul_net#168, aug_net#169, sep_net#170, oct_net#171, nov_net#172, dec_net#173] Keys [8]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149] Functions [36]: [partial_sum(jan_sales#150), partial_sum(feb_sales#151), partial_sum(mar_sales#152), partial_sum(apr_sales#153), partial_sum(may_sales#154), partial_sum(jun_sales#155), partial_sum(jul_sales#156), partial_sum(aug_sales#157), partial_sum(sep_sales#158), partial_sum(oct_sales#159), partial_sum(nov_sales#160), partial_sum(dec_sales#161), partial_sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(jan_net#162), partial_sum(feb_net#163), partial_sum(mar_net#164), partial_sum(apr_net#165), partial_sum(may_net#166), partial_sum(jun_net#167), partial_sum(jul_net#168), partial_sum(aug_net#169), partial_sum(sep_net#170), partial_sum(oct_net#171), partial_sum(nov_net#172), partial_sum(dec_net#173)] -Aggregate Attributes [72]: [sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411] -Results [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] +Aggregate Attributes [72]: [sum#328, isEmpty#329, sum#330, isEmpty#331, sum#332, isEmpty#333, sum#334, isEmpty#335, sum#336, isEmpty#337, sum#338, isEmpty#339, sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399] +Results [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] (53) Exchange -Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] -Arguments: hashpartitioning(w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#484] +Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] +Arguments: hashpartitioning(w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#472] (54) HashAggregate [codegen id : 14] -Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] +Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] Keys [8]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149] Functions [36]: [sum(jan_sales#150), sum(feb_sales#151), sum(mar_sales#152), sum(apr_sales#153), sum(may_sales#154), sum(jun_sales#155), sum(jul_sales#156), sum(aug_sales#157), sum(sep_sales#158), sum(oct_sales#159), sum(nov_sales#160), sum(dec_sales#161), sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(jan_net#162), sum(feb_net#163), sum(mar_net#164), sum(apr_net#165), sum(may_net#166), sum(jun_net#167), sum(jul_net#168), sum(aug_net#169), sum(sep_net#170), sum(oct_net#171), sum(nov_net#172), sum(dec_net#173)] -Aggregate Attributes [36]: [sum(jan_sales#150)#485, sum(feb_sales#151)#486, sum(mar_sales#152)#487, sum(apr_sales#153)#488, sum(may_sales#154)#489, sum(jun_sales#155)#490, sum(jul_sales#156)#491, sum(aug_sales#157)#492, sum(sep_sales#158)#493, sum(oct_sales#159)#494, sum(nov_sales#160)#495, sum(dec_sales#161)#496, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508, sum(jan_net#162)#509, sum(feb_net#163)#510, sum(mar_net#164)#511, sum(apr_net#165)#512, sum(may_net#166)#513, sum(jun_net#167)#514, sum(jul_net#168)#515, sum(aug_net#169)#516, sum(sep_net#170)#517, sum(oct_net#171)#518, sum(nov_net#172)#519, sum(dec_net#173)#520] -Results [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum(jan_sales#150)#485 AS jan_sales#521, sum(feb_sales#151)#486 AS feb_sales#522, sum(mar_sales#152)#487 AS mar_sales#523, sum(apr_sales#153)#488 AS apr_sales#524, sum(may_sales#154)#489 AS may_sales#525, sum(jun_sales#155)#490 AS jun_sales#526, sum(jul_sales#156)#491 AS jul_sales#527, sum(aug_sales#157)#492 AS aug_sales#528, sum(sep_sales#158)#493 AS sep_sales#529, sum(oct_sales#159)#494 AS oct_sales#530, sum(nov_sales#160)#495 AS nov_sales#531, sum(dec_sales#161)#496 AS dec_sales#532, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497 AS jan_sales_per_sq_foot#533, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498 AS feb_sales_per_sq_foot#534, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499 AS mar_sales_per_sq_foot#535, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500 AS apr_sales_per_sq_foot#536, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501 AS may_sales_per_sq_foot#537, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502 AS jun_sales_per_sq_foot#538, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503 AS jul_sales_per_sq_foot#539, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504 AS aug_sales_per_sq_foot#540, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505 AS sep_sales_per_sq_foot#541, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506 AS oct_sales_per_sq_foot#542, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507 AS nov_sales_per_sq_foot#543, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508 AS dec_sales_per_sq_foot#544, sum(jan_net#162)#509 AS jan_net#545, sum(feb_net#163)#510 AS feb_net#546, sum(mar_net#164)#511 AS mar_net#547, sum(apr_net#165)#512 AS apr_net#548, sum(may_net#166)#513 AS may_net#549, sum(jun_net#167)#514 AS jun_net#550, sum(jul_net#168)#515 AS jul_net#551, sum(aug_net#169)#516 AS aug_net#552, sum(sep_net#170)#517 AS sep_net#553, sum(oct_net#171)#518 AS oct_net#554, sum(nov_net#172)#519 AS nov_net#555, sum(dec_net#173)#520 AS dec_net#556] +Aggregate Attributes [36]: [sum(jan_sales#150)#473, sum(feb_sales#151)#474, sum(mar_sales#152)#475, sum(apr_sales#153)#476, sum(may_sales#154)#477, sum(jun_sales#155)#478, sum(jul_sales#156)#479, sum(aug_sales#157)#480, sum(sep_sales#158)#481, sum(oct_sales#159)#482, sum(nov_sales#160)#483, sum(dec_sales#161)#484, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496, sum(jan_net#162)#497, sum(feb_net#163)#498, sum(mar_net#164)#499, sum(apr_net#165)#500, sum(may_net#166)#501, sum(jun_net#167)#502, sum(jul_net#168)#503, sum(aug_net#169)#504, sum(sep_net#170)#505, sum(oct_net#171)#506, sum(nov_net#172)#507, sum(dec_net#173)#508] +Results [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum(jan_sales#150)#473 AS jan_sales#509, sum(feb_sales#151)#474 AS feb_sales#510, sum(mar_sales#152)#475 AS mar_sales#511, sum(apr_sales#153)#476 AS apr_sales#512, sum(may_sales#154)#477 AS may_sales#513, sum(jun_sales#155)#478 AS jun_sales#514, sum(jul_sales#156)#479 AS jul_sales#515, sum(aug_sales#157)#480 AS aug_sales#516, sum(sep_sales#158)#481 AS sep_sales#517, sum(oct_sales#159)#482 AS oct_sales#518, sum(nov_sales#160)#483 AS nov_sales#519, sum(dec_sales#161)#484 AS dec_sales#520, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485 AS jan_sales_per_sq_foot#521, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486 AS feb_sales_per_sq_foot#522, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487 AS mar_sales_per_sq_foot#523, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488 AS apr_sales_per_sq_foot#524, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489 AS may_sales_per_sq_foot#525, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490 AS jun_sales_per_sq_foot#526, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491 AS jul_sales_per_sq_foot#527, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492 AS aug_sales_per_sq_foot#528, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493 AS sep_sales_per_sq_foot#529, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494 AS oct_sales_per_sq_foot#530, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495 AS nov_sales_per_sq_foot#531, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496 AS dec_sales_per_sq_foot#532, sum(jan_net#162)#497 AS jan_net#533, sum(feb_net#163)#498 AS feb_net#534, sum(mar_net#164)#499 AS mar_net#535, sum(apr_net#165)#500 AS apr_net#536, sum(may_net#166)#501 AS may_net#537, sum(jun_net#167)#502 AS jun_net#538, sum(jul_net#168)#503 AS jul_net#539, sum(aug_net#169)#504 AS aug_net#540, sum(sep_net#170)#505 AS sep_net#541, sum(oct_net#171)#506 AS oct_net#542, sum(nov_net#172)#507 AS nov_net#543, sum(dec_net#173)#508 AS dec_net#544] (55) TakeOrderedAndProject -Input [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, may_sales_per_sq_foot#537, jun_sales_per_sq_foot#538, jul_sales_per_sq_foot#539, aug_sales_per_sq_foot#540, sep_sales_per_sq_foot#541, oct_sales_per_sq_foot#542, nov_sales_per_sq_foot#543, dec_sales_per_sq_foot#544, jan_net#545, feb_net#546, mar_net#547, apr_net#548, may_net#549, jun_net#550, jul_net#551, aug_net#552, sep_net#553, oct_net#554, nov_net#555, dec_net#556] -Arguments: 100, [w_warehouse_name#20 ASC NULLS FIRST], [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, ... 20 more fields] +Input [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, may_sales_per_sq_foot#525, jun_sales_per_sq_foot#526, jul_sales_per_sq_foot#527, aug_sales_per_sq_foot#528, sep_sales_per_sq_foot#529, oct_sales_per_sq_foot#530, nov_sales_per_sq_foot#531, dec_sales_per_sq_foot#532, jan_net#533, feb_net#534, mar_net#535, apr_net#536, may_net#537, jun_net#538, jul_net#539, aug_net#540, sep_net#541, oct_net#542, nov_net#543, dec_net#544] +Arguments: 100, [w_warehouse_name#20 ASC NULLS FIRST], [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, ... 20 more fields] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt index 832965c1aaa31..3d44b22396486 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt @@ -221,70 +221,70 @@ Input [7]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_qu Condition : ((isnotnull(cs_warehouse_sk#176) AND isnotnull(cs_sold_time_sk#174)) AND isnotnull(cs_ship_mode_sk#175)) (36) ReusedExchange [Reuses operator id: 7] -Output [7]: [w_warehouse_sk#181, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187] +Output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_warehouse_sk#176] -Right keys [1]: [w_warehouse_sk#181] +Right keys [1]: [w_warehouse_sk#9] Join condition: None (38) Project [codegen id : 11] -Output [12]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187] -Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_sk#181, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187] +Output [12]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] (39) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#188, d_year#189, d_moy#190] +Output [3]: [d_date_sk#17, d_year#18, d_moy#19] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#180] -Right keys [1]: [d_date_sk#188] +Right keys [1]: [d_date_sk#17] Join condition: None (41) Project [codegen id : 11] -Output [13]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] -Input [15]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_date_sk#188, d_year#189, d_moy#190] +Output [13]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] +Input [15]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_date_sk#17, d_year#18, d_moy#19] (42) ReusedExchange [Reuses operator id: 20] -Output [1]: [t_time_sk#191] +Output [1]: [t_time_sk#21] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_time_sk#174] -Right keys [1]: [t_time_sk#191] +Right keys [1]: [t_time_sk#21] Join condition: None (44) Project [codegen id : 11] -Output [12]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] -Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190, t_time_sk#191] +Output [12]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] +Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19, t_time_sk#21] (45) ReusedExchange [Reuses operator id: 27] -Output [1]: [sm_ship_mode_sk#192] +Output [1]: [sm_ship_mode_sk#24] (46) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_mode_sk#175] -Right keys [1]: [sm_ship_mode_sk#192] +Right keys [1]: [sm_ship_mode_sk#24] Join condition: None (47) Project [codegen id : 11] -Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] -Input [13]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190, sm_ship_mode_sk#192] +Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] +Input [13]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#24] (48) HashAggregate [codegen id : 11] -Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] -Keys [7]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189] -Functions [24]: [partial_sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240] -Results [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] +Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] +Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18] +Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [48]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228] +Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] (49) Exchange -Input [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] -Arguments: hashpartitioning(w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, 5), ENSURE_REQUIREMENTS, [id=#289] +Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#277] (50) HashAggregate [codegen id : 12] -Input [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] -Keys [7]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189] -Functions [24]: [sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313] -Results [32]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, DHL,BARIAN AS ship_carriers#314, d_year#189 AS year#315, sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_sales#316, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_sales#317, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_sales#318, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_sales#319, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_sales#320, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_sales#321, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_sales#322, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_sales#323, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_sales#324, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_sales#325, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_sales#326, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_sales#327, sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302 AS jan_net#328, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303 AS feb_net#329, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304 AS mar_net#330, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305 AS apr_net#331, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306 AS may_net#332, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307 AS jun_net#333, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308 AS jul_net#334, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309 AS aug_net#335, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310 AS sep_net#336, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311 AS oct_net#337, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312 AS nov_net#338, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313 AS dec_net#339] +Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] +Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18] +Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301] +Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, DHL,BARIAN AS ship_carriers#302, d_year#18 AS year#303, sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278 AS jan_sales#304, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279 AS feb_sales#305, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280 AS mar_sales#306, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281 AS apr_sales#307, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282 AS may_sales#308, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283 AS jun_sales#309, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284 AS jul_sales#310, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285 AS aug_sales#311, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286 AS sep_sales#312, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287 AS oct_sales#313, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288 AS nov_sales#314, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289 AS dec_sales#315, sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_net#316, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_net#317, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_net#318, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_net#319, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_net#320, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_net#321, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_net#322, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_net#323, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_net#324, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_net#325, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_net#326, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_net#327] (51) Union @@ -292,23 +292,23 @@ Results [32]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#150, feb_sales#151, mar_sales#152, apr_sales#153, may_sales#154, jun_sales#155, jul_sales#156, aug_sales#157, sep_sales#158, oct_sales#159, nov_sales#160, dec_sales#161, jan_net#162, feb_net#163, mar_net#164, apr_net#165, may_net#166, jun_net#167, jul_net#168, aug_net#169, sep_net#170, oct_net#171, nov_net#172, dec_net#173] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149] Functions [36]: [partial_sum(jan_sales#150), partial_sum(feb_sales#151), partial_sum(mar_sales#152), partial_sum(apr_sales#153), partial_sum(may_sales#154), partial_sum(jun_sales#155), partial_sum(jul_sales#156), partial_sum(aug_sales#157), partial_sum(sep_sales#158), partial_sum(oct_sales#159), partial_sum(nov_sales#160), partial_sum(dec_sales#161), partial_sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(jan_net#162), partial_sum(feb_net#163), partial_sum(mar_net#164), partial_sum(apr_net#165), partial_sum(may_net#166), partial_sum(jun_net#167), partial_sum(jul_net#168), partial_sum(aug_net#169), partial_sum(sep_net#170), partial_sum(oct_net#171), partial_sum(nov_net#172), partial_sum(dec_net#173)] -Aggregate Attributes [72]: [sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411] -Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] +Aggregate Attributes [72]: [sum#328, isEmpty#329, sum#330, isEmpty#331, sum#332, isEmpty#333, sum#334, isEmpty#335, sum#336, isEmpty#337, sum#338, isEmpty#339, sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399] +Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] (53) Exchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#484] +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#472] (54) HashAggregate [codegen id : 14] -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149] Functions [36]: [sum(jan_sales#150), sum(feb_sales#151), sum(mar_sales#152), sum(apr_sales#153), sum(may_sales#154), sum(jun_sales#155), sum(jul_sales#156), sum(aug_sales#157), sum(sep_sales#158), sum(oct_sales#159), sum(nov_sales#160), sum(dec_sales#161), sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(jan_net#162), sum(feb_net#163), sum(mar_net#164), sum(apr_net#165), sum(may_net#166), sum(jun_net#167), sum(jul_net#168), sum(aug_net#169), sum(sep_net#170), sum(oct_net#171), sum(nov_net#172), sum(dec_net#173)] -Aggregate Attributes [36]: [sum(jan_sales#150)#485, sum(feb_sales#151)#486, sum(mar_sales#152)#487, sum(apr_sales#153)#488, sum(may_sales#154)#489, sum(jun_sales#155)#490, sum(jul_sales#156)#491, sum(aug_sales#157)#492, sum(sep_sales#158)#493, sum(oct_sales#159)#494, sum(nov_sales#160)#495, sum(dec_sales#161)#496, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508, sum(jan_net#162)#509, sum(feb_net#163)#510, sum(mar_net#164)#511, sum(apr_net#165)#512, sum(may_net#166)#513, sum(jun_net#167)#514, sum(jul_net#168)#515, sum(aug_net#169)#516, sum(sep_net#170)#517, sum(oct_net#171)#518, sum(nov_net#172)#519, sum(dec_net#173)#520] -Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum(jan_sales#150)#485 AS jan_sales#521, sum(feb_sales#151)#486 AS feb_sales#522, sum(mar_sales#152)#487 AS mar_sales#523, sum(apr_sales#153)#488 AS apr_sales#524, sum(may_sales#154)#489 AS may_sales#525, sum(jun_sales#155)#490 AS jun_sales#526, sum(jul_sales#156)#491 AS jul_sales#527, sum(aug_sales#157)#492 AS aug_sales#528, sum(sep_sales#158)#493 AS sep_sales#529, sum(oct_sales#159)#494 AS oct_sales#530, sum(nov_sales#160)#495 AS nov_sales#531, sum(dec_sales#161)#496 AS dec_sales#532, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497 AS jan_sales_per_sq_foot#533, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498 AS feb_sales_per_sq_foot#534, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499 AS mar_sales_per_sq_foot#535, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500 AS apr_sales_per_sq_foot#536, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501 AS may_sales_per_sq_foot#537, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502 AS jun_sales_per_sq_foot#538, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503 AS jul_sales_per_sq_foot#539, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504 AS aug_sales_per_sq_foot#540, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505 AS sep_sales_per_sq_foot#541, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506 AS oct_sales_per_sq_foot#542, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507 AS nov_sales_per_sq_foot#543, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508 AS dec_sales_per_sq_foot#544, sum(jan_net#162)#509 AS jan_net#545, sum(feb_net#163)#510 AS feb_net#546, sum(mar_net#164)#511 AS mar_net#547, sum(apr_net#165)#512 AS apr_net#548, sum(may_net#166)#513 AS may_net#549, sum(jun_net#167)#514 AS jun_net#550, sum(jul_net#168)#515 AS jul_net#551, sum(aug_net#169)#516 AS aug_net#552, sum(sep_net#170)#517 AS sep_net#553, sum(oct_net#171)#518 AS oct_net#554, sum(nov_net#172)#519 AS nov_net#555, sum(dec_net#173)#520 AS dec_net#556] +Aggregate Attributes [36]: [sum(jan_sales#150)#473, sum(feb_sales#151)#474, sum(mar_sales#152)#475, sum(apr_sales#153)#476, sum(may_sales#154)#477, sum(jun_sales#155)#478, sum(jul_sales#156)#479, sum(aug_sales#157)#480, sum(sep_sales#158)#481, sum(oct_sales#159)#482, sum(nov_sales#160)#483, sum(dec_sales#161)#484, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496, sum(jan_net#162)#497, sum(feb_net#163)#498, sum(mar_net#164)#499, sum(apr_net#165)#500, sum(may_net#166)#501, sum(jun_net#167)#502, sum(jul_net#168)#503, sum(aug_net#169)#504, sum(sep_net#170)#505, sum(oct_net#171)#506, sum(nov_net#172)#507, sum(dec_net#173)#508] +Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum(jan_sales#150)#473 AS jan_sales#509, sum(feb_sales#151)#474 AS feb_sales#510, sum(mar_sales#152)#475 AS mar_sales#511, sum(apr_sales#153)#476 AS apr_sales#512, sum(may_sales#154)#477 AS may_sales#513, sum(jun_sales#155)#478 AS jun_sales#514, sum(jul_sales#156)#479 AS jul_sales#515, sum(aug_sales#157)#480 AS aug_sales#516, sum(sep_sales#158)#481 AS sep_sales#517, sum(oct_sales#159)#482 AS oct_sales#518, sum(nov_sales#160)#483 AS nov_sales#519, sum(dec_sales#161)#484 AS dec_sales#520, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485 AS jan_sales_per_sq_foot#521, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486 AS feb_sales_per_sq_foot#522, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487 AS mar_sales_per_sq_foot#523, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488 AS apr_sales_per_sq_foot#524, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489 AS may_sales_per_sq_foot#525, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490 AS jun_sales_per_sq_foot#526, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491 AS jul_sales_per_sq_foot#527, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492 AS aug_sales_per_sq_foot#528, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493 AS sep_sales_per_sq_foot#529, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494 AS oct_sales_per_sq_foot#530, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495 AS nov_sales_per_sq_foot#531, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496 AS dec_sales_per_sq_foot#532, sum(jan_net#162)#497 AS jan_net#533, sum(feb_net#163)#498 AS feb_net#534, sum(mar_net#164)#499 AS mar_net#535, sum(apr_net#165)#500 AS apr_net#536, sum(may_net#166)#501 AS may_net#537, sum(jun_net#167)#502 AS jun_net#538, sum(jul_net#168)#503 AS jul_net#539, sum(aug_net#169)#504 AS aug_net#540, sum(sep_net#170)#505 AS sep_net#541, sum(oct_net#171)#506 AS oct_net#542, sum(nov_net#172)#507 AS nov_net#543, sum(dec_net#173)#508 AS dec_net#544] (55) TakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, may_sales_per_sq_foot#537, jun_sales_per_sq_foot#538, jul_sales_per_sq_foot#539, aug_sales_per_sq_foot#540, sep_sales_per_sq_foot#541, oct_sales_per_sq_foot#542, nov_sales_per_sq_foot#543, dec_sales_per_sq_foot#544, jan_net#545, feb_net#546, mar_net#547, apr_net#548, may_net#549, jun_net#550, jul_net#551, aug_net#552, sep_net#553, oct_net#554, nov_net#555, dec_net#556] -Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, ... 20 more fields] +Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, may_sales_per_sq_foot#525, jun_sales_per_sq_foot#526, jul_sales_per_sq_foot#527, aug_sales_per_sq_foot#528, sep_sales_per_sq_foot#529, oct_sales_per_sq_foot#530, nov_sales_per_sq_foot#531, dec_sales_per_sq_foot#532, jan_net#533, feb_net#534, mar_net#535, apr_net#536, may_net#537, jun_net#538, jul_net#539, aug_net#540, sep_net#541, oct_net#542, nov_net#543, dec_net#544] +Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, ... 20 more fields] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt index b6a609ec193b4..744fe4b5a594d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt @@ -230,55 +230,55 @@ Input [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_ Arguments: [ss_addr_sk#12 ASC NULLS FIRST], false, 0 (41) ReusedExchange [Reuses operator id: 9] -Output [2]: [ca_address_sk#32, ca_city#33] +Output [2]: [ca_address_sk#6, ca_city#7] (42) Sort [codegen id : 13] -Input [2]: [ca_address_sk#32, ca_city#33] -Arguments: [ca_address_sk#32 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#6, ca_city#7] +Arguments: [ca_address_sk#6 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 14] Left keys [1]: [ss_addr_sk#12] -Right keys [1]: [ca_address_sk#32] +Right keys [1]: [ca_address_sk#6] Join condition: None (44) Project [codegen id : 14] -Output [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#33] -Input [8]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_address_sk#32, ca_city#33] +Output [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#7] +Input [8]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_address_sk#6, ca_city#7] (45) HashAggregate [codegen id : 14] -Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#33] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33] +Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#7] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#15)), partial_sum(UnscaledValue(ss_ext_list_price#16)), partial_sum(UnscaledValue(ss_ext_tax#17))] -Aggregate Attributes [3]: [sum#34, sum#35, sum#36] -Results [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33, sum#37, sum#38, sum#39] +Aggregate Attributes [3]: [sum#32, sum#33, sum#34] +Results [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#35, sum#36, sum#37] (46) HashAggregate [codegen id : 14] -Input [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33, sum#37, sum#38, sum#39] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33] +Input [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#35, sum#36, sum#37] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#15)), sum(UnscaledValue(ss_ext_list_price#16)), sum(UnscaledValue(ss_ext_tax#17))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#15))#40, sum(UnscaledValue(ss_ext_list_price#16))#41, sum(UnscaledValue(ss_ext_tax#17))#42] -Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#33 AS bought_city#43, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#40,17,2) AS extended_price#44, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#16))#41,17,2) AS list_price#45, MakeDecimal(sum(UnscaledValue(ss_ext_tax#17))#42,17,2) AS extended_tax#46] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#15))#38, sum(UnscaledValue(ss_ext_list_price#16))#39, sum(UnscaledValue(ss_ext_tax#17))#40] +Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#7 AS bought_city#41, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#38,17,2) AS extended_price#42, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#16))#39,17,2) AS list_price#43, MakeDecimal(sum(UnscaledValue(ss_ext_tax#17))#40,17,2) AS extended_tax#44] (47) Exchange -Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#43, extended_price#44, list_price#45, extended_tax#46] -Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#41, extended_price#42, list_price#43, extended_tax#44] +Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#45] (48) Sort [codegen id : 15] -Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#43, extended_price#44, list_price#45, extended_tax#46] +Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#41, extended_price#42, list_price#43, extended_tax#44] Arguments: [ss_customer_sk#10 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 16] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#10] -Join condition: NOT (ca_city#7 = bought_city#43) +Join condition: NOT (ca_city#7 = bought_city#41) (50) Project [codegen id : 16] -Output [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#43, ss_ticket_number#14, extended_price#44, extended_tax#46, list_price#45] -Input [10]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#43, extended_price#44, list_price#45, extended_tax#46] +Output [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#41, ss_ticket_number#14, extended_price#42, extended_tax#44, list_price#43] +Input [10]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#41, extended_price#42, list_price#43, extended_tax#44] (51) TakeOrderedAndProject -Input [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#43, ss_ticket_number#14, extended_price#44, extended_tax#46, list_price#45] -Arguments: 100, [c_last_name#4 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#43, ss_ticket_number#14, extended_price#44, extended_tax#46, list_price#45] +Input [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#41, ss_ticket_number#14, extended_price#42, extended_tax#44, list_price#43] +Arguments: 100, [c_last_name#4 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#41, ss_ticket_number#14, extended_price#42, extended_tax#44, list_price#43] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt index 101e4f272f98a..1a9fafda6250b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt @@ -225,20 +225,20 @@ Output [8]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#3 Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_customer_sk#39, c_current_addr_sk#40, c_first_name#41, c_last_name#42] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#44, ca_city#45] +Output [2]: [ca_address_sk#22, ca_city#23] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#40] -Right keys [1]: [ca_address_sk#44] -Join condition: NOT (ca_city#45 = bought_city#35) +Right keys [1]: [ca_address_sk#22] +Join condition: NOT (ca_city#23 = bought_city#35) (42) Project [codegen id : 8] -Output [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#44, ca_city#45] +Output [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#22, ca_city#23] (43) TakeOrderedAndProject -Input [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt index a0532462eff64..97d7a81baf2a7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt @@ -139,20 +139,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#8] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] +Right keys [1]: [d_date_sk#8] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#13] -Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] +Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] (23) Exchange Input [1]: [ws_bill_customer_sk#13] -Arguments: hashpartitioning(ws_bill_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#16] +Arguments: hashpartitioning(ws_bill_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#15] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#13] @@ -164,38 +164,38 @@ Right keys [1]: [ws_bill_customer_sk#13] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#19] +Output [1]: [d_date_sk#8] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#19] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#8] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#17] -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] +Output [1]: [cs_ship_customer_sk#16] +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] (31) Exchange -Input [1]: [cs_ship_customer_sk#17] -Arguments: hashpartitioning(cs_ship_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [cs_ship_customer_sk#16] +Arguments: hashpartitioning(cs_ship_customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#18] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#17] -Arguments: [cs_ship_customer_sk#17 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#16] +Arguments: [cs_ship_customer_sk#16 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#17] +Right keys [1]: [cs_ship_customer_sk#16] Join condition: None (34) Project [codegen id : 13] @@ -203,84 +203,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (35) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] +Output [2]: [ca_address_sk#19, ca_state#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 12] -Input [2]: [ca_address_sk#21, ca_state#22] +Input [2]: [ca_address_sk#19, ca_state#20] (37) Filter [codegen id : 12] -Input [2]: [ca_address_sk#21, ca_state#22] -Condition : (ca_state#22 IN (KY,GA,NM) AND isnotnull(ca_address_sk#21)) +Input [2]: [ca_address_sk#19, ca_state#20] +Condition : (ca_state#20 IN (KY,GA,NM) AND isnotnull(ca_address_sk#19)) (38) Project [codegen id : 12] -Output [1]: [ca_address_sk#21] -Input [2]: [ca_address_sk#21, ca_state#22] +Output [1]: [ca_address_sk#19] +Input [2]: [ca_address_sk#19, ca_state#20] (39) BroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [ca_address_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#19] Join condition: None (41) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] (42) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (43) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (44) ColumnarToRow -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] (45) Filter -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] -Condition : isnotnull(cd_demo_sk#25) +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Condition : isnotnull(cd_demo_sk#23) (46) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#23] Join condition: None (47) Project [codegen id : 14] -Output [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] (48) HashAggregate [codegen id : 14] -Input [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] -Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#31] -Results [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] +Aggregate Attributes [1]: [count#29] +Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] (49) Exchange -Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] +Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, 5), ENSURE_REQUIREMENTS, [id=#31] (50) HashAggregate [codegen id : 15] -Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] -Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] +Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#34] -Results [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#34 AS cnt1#35, cd_purchase_estimate#29, count(1)#34 AS cnt2#36, cd_credit_rating#30, count(1)#34 AS cnt3#37] +Aggregate Attributes [1]: [count(1)#32] +Results [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#32 AS cnt1#33, cd_purchase_estimate#27, count(1)#32 AS cnt2#34, cd_credit_rating#28, count(1)#32 AS cnt3#35] (51) TakeOrderedAndProject -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#35, cd_purchase_estimate#29, cnt2#36, cd_credit_rating#30, cnt3#37] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#35, cd_purchase_estimate#29, cnt2#36, cd_credit_rating#30, cnt3#37] +Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#33, cd_purchase_estimate#27, cnt2#34, cd_credit_rating#28, cnt3#35] +Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#33, cd_purchase_estimate#27, cnt2#34, cd_credit_rating#28, cnt3#35] ===== Subqueries ===== @@ -293,6 +293,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index d0b3c2231d997..b5307abaf688c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -122,20 +122,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#7] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#7] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#12] -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] @@ -143,34 +143,34 @@ Right keys [1]: [ws_bill_customer_sk#12] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#7] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#18] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#7] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#16] -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#18] +Output [1]: [cs_ship_customer_sk#15] +Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [cs_ship_customer_sk#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#16] +Right keys [1]: [cs_ship_customer_sk#15] Join condition: None (29) Project [codegen id : 9] @@ -178,84 +178,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (30) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +Output [2]: [ca_address_sk#18, ca_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_state#21] +Input [2]: [ca_address_sk#18, ca_state#19] (32) Filter [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : (ca_state#21 IN (KY,GA,NM) AND isnotnull(ca_address_sk#20)) +Input [2]: [ca_address_sk#18, ca_state#19] +Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) (33) Project [codegen id : 7] -Output [1]: [ca_address_sk#20] -Input [2]: [ca_address_sk#20, ca_state#21] +Output [1]: [ca_address_sk#18] +Input [2]: [ca_address_sk#18, ca_state#19] (34) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [ca_address_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#18] Join condition: None (36) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] (37) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] (39) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Condition : isnotnull(cd_demo_sk#23) +Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Condition : isnotnull(cd_demo_sk#21) (40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#21] Join condition: None (42) Project [codegen id : 9] -Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Output [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] (43) HashAggregate [codegen id : 9] -Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#30] -Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#31] +Aggregate Attributes [1]: [count#28] +Results [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] (44) Exchange -Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#31] -Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] +Arguments: hashpartitioning(cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, 5), ENSURE_REQUIREMENTS, [id=#30] (45) HashAggregate [codegen id : 10] -Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#31] -Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] +Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#33] -Results [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#33 AS cnt1#34, cd_purchase_estimate#27, count(1)#33 AS cnt2#35, cd_credit_rating#28, count(1)#33 AS cnt3#36] +Aggregate Attributes [1]: [count(1)#31] +Results [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, count(1)#31 AS cnt1#32, cd_purchase_estimate#25, count(1)#31 AS cnt2#33, cd_credit_rating#26, count(1)#31 AS cnt3#34] (46) TakeOrderedAndProject -Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#34, cd_purchase_estimate#27, cnt2#35, cd_credit_rating#28, cnt3#36] -Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#34, cd_purchase_estimate#27, cnt2#35, cd_credit_rating#28, cnt3#36] +Input [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#32, cd_purchase_estimate#25, cnt2#33, cd_credit_rating#26, cnt3#34] +Arguments: 100, [cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_education_status#24 ASC NULLS FIRST, cd_purchase_estimate#25 ASC NULLS FIRST, cd_credit_rating#26 ASC NULLS FIRST], [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#32, cd_purchase_estimate#25, cnt2#33, cd_credit_rating#26, cnt3#34] ===== Subqueries ===== @@ -268,6 +268,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt index b138d059eaecb..ab2cc784a5f22 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnul Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt index 5f6002b84f411..9c914d56f46a2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt @@ -54,7 +54,7 @@ Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnul Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -62,7 +62,7 @@ Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_stat (6) Filter [codegen id : 1] Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] -Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College)) AND isnotnull(cd_demo_sk#10)) (7) Project [codegen id : 1] Output [1]: [cd_demo_sk#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index ddce6bf2e14f4..d31dbc3498ead 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -109,109 +109,109 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales -Output [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] (16) Filter [codegen id : 4] -Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] -Condition : isnotnull(ss_store_sk#11) +Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_store_sk#1) (17) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#5] (18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] Join condition: None (19) Project [codegen id : 4] -Output [2]: [ss_store_sk#11, ss_net_profit#12] -Input [4]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13, d_date_sk#14] +Output [2]: [ss_store_sk#1, ss_net_profit#2] +Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] (20) Scan parquet default.store -Output [2]: [s_store_sk#15, s_state#16] +Output [2]: [s_store_sk#8, s_state#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#15, s_state#16] +Input [2]: [s_store_sk#8, s_state#10] (22) Filter [codegen id : 3] -Input [2]: [s_store_sk#15, s_state#16] -Condition : isnotnull(s_store_sk#15) +Input [2]: [s_store_sk#8, s_state#10] +Condition : isnotnull(s_store_sk#8) (23) BroadcastExchange -Input [2]: [s_store_sk#15, s_state#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] +Input [2]: [s_store_sk#8, s_state#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] (24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#11] -Right keys [1]: [s_store_sk#15] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#8] Join condition: None (25) Project [codegen id : 4] -Output [2]: [ss_net_profit#12, s_state#16] -Input [4]: [ss_store_sk#11, ss_net_profit#12, s_store_sk#15, s_state#16] +Output [2]: [ss_net_profit#2, s_state#10] +Input [4]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_state#10] (26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#12, s_state#16] -Keys [1]: [s_state#16] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#12))] -Aggregate Attributes [1]: [sum#18] -Results [2]: [s_state#16, sum#19] +Input [2]: [ss_net_profit#2, s_state#10] +Keys [1]: [s_state#10] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum#12] +Results [2]: [s_state#10, sum#13] (27) Exchange -Input [2]: [s_state#16, sum#19] -Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [2]: [s_state#10, sum#13] +Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] (28) HashAggregate [codegen id : 5] -Input [2]: [s_state#16, sum#19] -Keys [1]: [s_state#16] -Functions [1]: [sum(UnscaledValue(ss_net_profit#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#12))#21] -Results [3]: [s_state#16 AS s_state#22, s_state#16, MakeDecimal(sum(UnscaledValue(ss_net_profit#12))#21,17,2) AS _w2#23] +Input [2]: [s_state#10, sum#13] +Keys [1]: [s_state#10] +Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#15] +Results [3]: [s_state#10 AS s_state#16, s_state#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#15,17,2) AS _w2#17] (29) Exchange -Input [3]: [s_state#22, s_state#16, _w2#23] -Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [s_state#16, s_state#10, _w2#17] +Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#18] (30) Sort [codegen id : 6] -Input [3]: [s_state#22, s_state#16, _w2#23] -Arguments: [s_state#16 ASC NULLS FIRST, _w2#23 DESC NULLS LAST], false, 0 +Input [3]: [s_state#16, s_state#10, _w2#17] +Arguments: [s_state#10 ASC NULLS FIRST, _w2#17 DESC NULLS LAST], false, 0 (31) Window -Input [3]: [s_state#22, s_state#16, _w2#23] -Arguments: [rank(_w2#23) windowspecdefinition(s_state#16, _w2#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#25], [s_state#16], [_w2#23 DESC NULLS LAST] +Input [3]: [s_state#16, s_state#10, _w2#17] +Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#10], [_w2#17 DESC NULLS LAST] (32) Filter [codegen id : 7] -Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] -Condition : (isnotnull(ranking#25) AND (ranking#25 <= 5)) +Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] -Output [1]: [s_state#22] -Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] +Output [1]: [s_state#16] +Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] (34) BroadcastExchange -Input [1]: [s_state#22] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#26] +Input [1]: [s_state#16] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#20] (35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#10] -Right keys [1]: [s_state#22] +Right keys [1]: [s_state#16] Join condition: None (36) BroadcastExchange Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] @@ -224,45 +224,45 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_county#9, s_state#10 (39) Expand [codegen id : 9] Input [3]: [ss_net_profit#2, s_state#10, s_county#9] -Arguments: [List(ss_net_profit#2, s_state#10, s_county#9, 0), List(ss_net_profit#2, s_state#10, null, 1), List(ss_net_profit#2, null, null, 3)], [ss_net_profit#2, s_state#28, s_county#29, spark_grouping_id#30] +Arguments: [List(ss_net_profit#2, s_state#10, s_county#9, 0), List(ss_net_profit#2, s_state#10, null, 1), List(ss_net_profit#2, null, null, 3)], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] (40) HashAggregate [codegen id : 9] -Input [4]: [ss_net_profit#2, s_state#28, s_county#29, spark_grouping_id#30] -Keys [3]: [s_state#28, s_county#29, spark_grouping_id#30] +Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] +Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#31] -Results [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] +Aggregate Attributes [1]: [sum#25] +Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] (41) Exchange -Input [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] -Arguments: hashpartitioning(s_state#28, s_county#29, spark_grouping_id#30, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, [id=#27] (42) HashAggregate [codegen id : 10] -Input [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] -Keys [3]: [s_state#28, s_county#29, spark_grouping_id#30] +Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#34] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS total_sum#35, s_state#28, s_county#29, (cast((shiftright(spark_grouping_id#30, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint)) AS lochierarchy#36, (cast((shiftright(spark_grouping_id#30, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint)) AS _w1#37, CASE WHEN (cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint) = 0) THEN s_state#28 END AS _w2#38, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS _w3#39] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#28] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS total_sum#29, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS _w3#33] (43) Exchange -Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] -Arguments: hashpartitioning(_w1#37, _w2#38, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] +Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, [id=#34] (44) Sort [codegen id : 11] -Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] -Arguments: [_w1#37 ASC NULLS FIRST, _w2#38 ASC NULLS FIRST, _w3#39 DESC NULLS LAST], false, 0 +Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] +Arguments: [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w3#33 DESC NULLS LAST], false, 0 (45) Window -Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] -Arguments: [rank(_w3#39) windowspecdefinition(_w1#37, _w2#38, _w3#39 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [_w1#37, _w2#38], [_w3#39 DESC NULLS LAST] +Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] +Arguments: [rank(_w3#33) windowspecdefinition(_w1#31, _w2#32, _w3#33 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#35], [_w1#31, _w2#32], [_w3#33 DESC NULLS LAST] (46) Project [codegen id : 12] -Output [5]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] -Input [8]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39, rank_within_parent#41] +Output [5]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] +Input [8]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33, rank_within_parent#35] (47) TakeOrderedAndProject -Input [5]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] -Arguments: 100, [lochierarchy#36 DESC NULLS LAST, CASE WHEN (lochierarchy#36 = 0) THEN s_state#28 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] +Input [5]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] +Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#35 ASC NULLS FIRST], [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] ===== Subqueries ===== @@ -273,6 +273,6 @@ ReusedExchange (48) (48) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#5] -Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index dbdfc72f92624..26fec145f4211 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -109,109 +109,109 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales -Output [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] (16) Filter [codegen id : 4] -Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] -Condition : isnotnull(ss_store_sk#11) +Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_store_sk#1) (17) Scan parquet default.store -Output [2]: [s_store_sk#14, s_state#15] +Output [2]: [s_store_sk#8, s_state#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#14, s_state#15] +Input [2]: [s_store_sk#8, s_state#10] (19) Filter [codegen id : 2] -Input [2]: [s_store_sk#14, s_state#15] -Condition : isnotnull(s_store_sk#14) +Input [2]: [s_store_sk#8, s_state#10] +Condition : isnotnull(s_store_sk#8) (20) BroadcastExchange -Input [2]: [s_store_sk#14, s_state#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#16] +Input [2]: [s_store_sk#8, s_state#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] (21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#11] -Right keys [1]: [s_store_sk#14] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#8] Join condition: None (22) Project [codegen id : 4] -Output [3]: [ss_net_profit#12, ss_sold_date_sk#13, s_state#15] -Input [5]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13, s_store_sk#14, s_state#15] +Output [3]: [ss_net_profit#2, ss_sold_date_sk#3, s_state#10] +Input [5]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, s_store_sk#8, s_state#10] (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#17] +Output [1]: [d_date_sk#5] (24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#13] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] Join condition: None (25) Project [codegen id : 4] -Output [2]: [ss_net_profit#12, s_state#15] -Input [4]: [ss_net_profit#12, ss_sold_date_sk#13, s_state#15, d_date_sk#17] +Output [2]: [ss_net_profit#2, s_state#10] +Input [4]: [ss_net_profit#2, ss_sold_date_sk#3, s_state#10, d_date_sk#5] (26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#12, s_state#15] -Keys [1]: [s_state#15] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#12))] -Aggregate Attributes [1]: [sum#18] -Results [2]: [s_state#15, sum#19] +Input [2]: [ss_net_profit#2, s_state#10] +Keys [1]: [s_state#10] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum#12] +Results [2]: [s_state#10, sum#13] (27) Exchange -Input [2]: [s_state#15, sum#19] -Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [2]: [s_state#10, sum#13] +Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] (28) HashAggregate [codegen id : 5] -Input [2]: [s_state#15, sum#19] -Keys [1]: [s_state#15] -Functions [1]: [sum(UnscaledValue(ss_net_profit#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#12))#21] -Results [3]: [s_state#15 AS s_state#22, s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#12))#21,17,2) AS _w2#23] +Input [2]: [s_state#10, sum#13] +Keys [1]: [s_state#10] +Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#15] +Results [3]: [s_state#10 AS s_state#16, s_state#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#15,17,2) AS _w2#17] (29) Exchange -Input [3]: [s_state#22, s_state#15, _w2#23] -Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [s_state#16, s_state#10, _w2#17] +Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#18] (30) Sort [codegen id : 6] -Input [3]: [s_state#22, s_state#15, _w2#23] -Arguments: [s_state#15 ASC NULLS FIRST, _w2#23 DESC NULLS LAST], false, 0 +Input [3]: [s_state#16, s_state#10, _w2#17] +Arguments: [s_state#10 ASC NULLS FIRST, _w2#17 DESC NULLS LAST], false, 0 (31) Window -Input [3]: [s_state#22, s_state#15, _w2#23] -Arguments: [rank(_w2#23) windowspecdefinition(s_state#15, _w2#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#25], [s_state#15], [_w2#23 DESC NULLS LAST] +Input [3]: [s_state#16, s_state#10, _w2#17] +Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#10], [_w2#17 DESC NULLS LAST] (32) Filter [codegen id : 7] -Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] -Condition : (isnotnull(ranking#25) AND (ranking#25 <= 5)) +Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] -Output [1]: [s_state#22] -Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] +Output [1]: [s_state#16] +Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] (34) BroadcastExchange -Input [1]: [s_state#22] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#26] +Input [1]: [s_state#16] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#20] (35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#10] -Right keys [1]: [s_state#22] +Right keys [1]: [s_state#16] Join condition: None (36) BroadcastExchange Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] @@ -224,45 +224,45 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_county#9, s_state#10 (39) Expand [codegen id : 9] Input [3]: [ss_net_profit#2, s_state#10, s_county#9] -Arguments: [List(ss_net_profit#2, s_state#10, s_county#9, 0), List(ss_net_profit#2, s_state#10, null, 1), List(ss_net_profit#2, null, null, 3)], [ss_net_profit#2, s_state#28, s_county#29, spark_grouping_id#30] +Arguments: [List(ss_net_profit#2, s_state#10, s_county#9, 0), List(ss_net_profit#2, s_state#10, null, 1), List(ss_net_profit#2, null, null, 3)], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] (40) HashAggregate [codegen id : 9] -Input [4]: [ss_net_profit#2, s_state#28, s_county#29, spark_grouping_id#30] -Keys [3]: [s_state#28, s_county#29, spark_grouping_id#30] +Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] +Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#31] -Results [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] +Aggregate Attributes [1]: [sum#25] +Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] (41) Exchange -Input [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] -Arguments: hashpartitioning(s_state#28, s_county#29, spark_grouping_id#30, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, [id=#27] (42) HashAggregate [codegen id : 10] -Input [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] -Keys [3]: [s_state#28, s_county#29, spark_grouping_id#30] +Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#34] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS total_sum#35, s_state#28, s_county#29, (cast((shiftright(spark_grouping_id#30, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint)) AS lochierarchy#36, (cast((shiftright(spark_grouping_id#30, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint)) AS _w1#37, CASE WHEN (cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint) = 0) THEN s_state#28 END AS _w2#38, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS _w3#39] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#28] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS total_sum#29, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS _w3#33] (43) Exchange -Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] -Arguments: hashpartitioning(_w1#37, _w2#38, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] +Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, [id=#34] (44) Sort [codegen id : 11] -Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] -Arguments: [_w1#37 ASC NULLS FIRST, _w2#38 ASC NULLS FIRST, _w3#39 DESC NULLS LAST], false, 0 +Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] +Arguments: [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w3#33 DESC NULLS LAST], false, 0 (45) Window -Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] -Arguments: [rank(_w3#39) windowspecdefinition(_w1#37, _w2#38, _w3#39 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [_w1#37, _w2#38], [_w3#39 DESC NULLS LAST] +Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] +Arguments: [rank(_w3#33) windowspecdefinition(_w1#31, _w2#32, _w3#33 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#35], [_w1#31, _w2#32], [_w3#33 DESC NULLS LAST] (46) Project [codegen id : 12] -Output [5]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] -Input [8]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39, rank_within_parent#41] +Output [5]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] +Input [8]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33, rank_within_parent#35] (47) TakeOrderedAndProject -Input [5]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] -Arguments: 100, [lochierarchy#36 DESC NULLS LAST, CASE WHEN (lochierarchy#36 = 0) THEN s_state#28 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] +Input [5]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] +Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#35 ASC NULLS FIRST], [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] ===== Subqueries ===== @@ -273,6 +273,6 @@ ReusedExchange (48) (48) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#5] -Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt index 274d0c02b5583..3daa8b66851f7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt @@ -127,43 +127,43 @@ Input [4]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_da Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_sold_time_sk#18)) (19) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#11] (20) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#11] Join condition: None (21) Project [codegen id : 5] -Output [3]: [cs_ext_sales_price#20 AS ext_price#23, cs_item_sk#19 AS sold_item_sk#24, cs_sold_time_sk#18 AS time_sk#25] -Input [5]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#22] +Output [3]: [cs_ext_sales_price#20 AS ext_price#22, cs_item_sk#19 AS sold_item_sk#23, cs_sold_time_sk#18 AS time_sk#24] +Input [5]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#11] (22) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Output [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 7] -Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] (24) Filter [codegen id : 7] -Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : (isnotnull(ss_item_sk#27) AND isnotnull(ss_sold_time_sk#26)) +Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Condition : (isnotnull(ss_item_sk#26) AND isnotnull(ss_sold_time_sk#25)) (25) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#30] +Output [1]: [d_date_sk#11] (26) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#29] -Right keys [1]: [d_date_sk#30] +Left keys [1]: [ss_sold_date_sk#28] +Right keys [1]: [d_date_sk#11] Join condition: None (27) Project [codegen id : 7] -Output [3]: [ss_ext_sales_price#28 AS ext_price#31, ss_item_sk#27 AS sold_item_sk#32, ss_sold_time_sk#26 AS time_sk#33] -Input [5]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#30] +Output [3]: [ss_ext_sales_price#27 AS ext_price#29, ss_item_sk#26 AS sold_item_sk#30, ss_sold_time_sk#25 AS time_sk#31] +Input [5]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#11] (28) Union @@ -177,61 +177,61 @@ Output [4]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17] Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#15, sold_item_sk#16, time_sk#17] (31) Scan parquet default.time_dim -Output [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] +PushedFilters: [Or(EqualTo(t_meal_time,breakfast),EqualTo(t_meal_time,dinner)), IsNotNull(t_time_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] -Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] (33) Filter [codegen id : 8] -Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] -Condition : (((t_meal_time#37 = breakfast ) OR (t_meal_time#37 = dinner )) AND isnotnull(t_time_sk#34)) +Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Condition : (((t_meal_time#35 = breakfast) OR (t_meal_time#35 = dinner)) AND isnotnull(t_time_sk#32)) (34) Project [codegen id : 8] -Output [3]: [t_time_sk#34, t_hour#35, t_minute#36] -Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Output [3]: [t_time_sk#32, t_hour#33, t_minute#34] +Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] (35) BroadcastExchange -Input [3]: [t_time_sk#34, t_hour#35, t_minute#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] +Input [3]: [t_time_sk#32, t_hour#33, t_minute#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [time_sk#17] -Right keys [1]: [t_time_sk#34] +Right keys [1]: [t_time_sk#32] Join condition: None (37) Project [codegen id : 9] -Output [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#35, t_minute#36] -Input [7]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17, t_time_sk#34, t_hour#35, t_minute#36] +Output [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#33, t_minute#34] +Input [7]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17, t_time_sk#32, t_hour#33, t_minute#34] (38) HashAggregate [codegen id : 9] -Input [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#35, t_minute#36] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] +Input [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#33, t_minute#34] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] Functions [1]: [partial_sum(UnscaledValue(ext_price#15))] -Aggregate Attributes [1]: [sum#39] -Results [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] +Aggregate Attributes [1]: [sum#37] +Results [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] (39) Exchange -Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] -Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] +Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, 5), ENSURE_REQUIREMENTS, [id=#39] (40) HashAggregate [codegen id : 10] -Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] Functions [1]: [sum(UnscaledValue(ext_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#15))#42] -Results [5]: [i_brand_id#2 AS brand_id#43, i_brand#3 AS brand#44, t_hour#35, t_minute#36, MakeDecimal(sum(UnscaledValue(ext_price#15))#42,17,2) AS ext_price#45] +Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#15))#40] +Results [5]: [i_brand_id#2 AS brand_id#41, i_brand#3 AS brand#42, t_hour#33, t_minute#34, MakeDecimal(sum(UnscaledValue(ext_price#15))#40,17,2) AS ext_price#43] (41) Exchange -Input [5]: [brand_id#43, brand#44, t_hour#35, t_minute#36, ext_price#45] -Arguments: rangepartitioning(ext_price#45 DESC NULLS LAST, brand_id#43 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [5]: [brand_id#41, brand#42, t_hour#33, t_minute#34, ext_price#43] +Arguments: rangepartitioning(ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#44] (42) Sort [codegen id : 11] -Input [5]: [brand_id#43, brand#44, t_hour#35, t_minute#36, ext_price#45] -Arguments: [ext_price#45 DESC NULLS LAST, brand_id#43 ASC NULLS FIRST], true, 0 +Input [5]: [brand_id#41, brand#42, t_hour#33, t_minute#34, ext_price#43] +Arguments: [ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -244,6 +244,6 @@ Output [1]: [d_date_sk#11] Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index 274d0c02b5583..3daa8b66851f7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt @@ -127,43 +127,43 @@ Input [4]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_da Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_sold_time_sk#18)) (19) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#11] (20) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#11] Join condition: None (21) Project [codegen id : 5] -Output [3]: [cs_ext_sales_price#20 AS ext_price#23, cs_item_sk#19 AS sold_item_sk#24, cs_sold_time_sk#18 AS time_sk#25] -Input [5]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#22] +Output [3]: [cs_ext_sales_price#20 AS ext_price#22, cs_item_sk#19 AS sold_item_sk#23, cs_sold_time_sk#18 AS time_sk#24] +Input [5]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#11] (22) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Output [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 7] -Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] (24) Filter [codegen id : 7] -Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] -Condition : (isnotnull(ss_item_sk#27) AND isnotnull(ss_sold_time_sk#26)) +Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Condition : (isnotnull(ss_item_sk#26) AND isnotnull(ss_sold_time_sk#25)) (25) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#30] +Output [1]: [d_date_sk#11] (26) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#29] -Right keys [1]: [d_date_sk#30] +Left keys [1]: [ss_sold_date_sk#28] +Right keys [1]: [d_date_sk#11] Join condition: None (27) Project [codegen id : 7] -Output [3]: [ss_ext_sales_price#28 AS ext_price#31, ss_item_sk#27 AS sold_item_sk#32, ss_sold_time_sk#26 AS time_sk#33] -Input [5]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#30] +Output [3]: [ss_ext_sales_price#27 AS ext_price#29, ss_item_sk#26 AS sold_item_sk#30, ss_sold_time_sk#25 AS time_sk#31] +Input [5]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#11] (28) Union @@ -177,61 +177,61 @@ Output [4]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17] Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#15, sold_item_sk#16, time_sk#17] (31) Scan parquet default.time_dim -Output [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] +PushedFilters: [Or(EqualTo(t_meal_time,breakfast),EqualTo(t_meal_time,dinner)), IsNotNull(t_time_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] -Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] (33) Filter [codegen id : 8] -Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] -Condition : (((t_meal_time#37 = breakfast ) OR (t_meal_time#37 = dinner )) AND isnotnull(t_time_sk#34)) +Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Condition : (((t_meal_time#35 = breakfast) OR (t_meal_time#35 = dinner)) AND isnotnull(t_time_sk#32)) (34) Project [codegen id : 8] -Output [3]: [t_time_sk#34, t_hour#35, t_minute#36] -Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Output [3]: [t_time_sk#32, t_hour#33, t_minute#34] +Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] (35) BroadcastExchange -Input [3]: [t_time_sk#34, t_hour#35, t_minute#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] +Input [3]: [t_time_sk#32, t_hour#33, t_minute#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [time_sk#17] -Right keys [1]: [t_time_sk#34] +Right keys [1]: [t_time_sk#32] Join condition: None (37) Project [codegen id : 9] -Output [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#35, t_minute#36] -Input [7]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17, t_time_sk#34, t_hour#35, t_minute#36] +Output [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#33, t_minute#34] +Input [7]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17, t_time_sk#32, t_hour#33, t_minute#34] (38) HashAggregate [codegen id : 9] -Input [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#35, t_minute#36] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] +Input [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#33, t_minute#34] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] Functions [1]: [partial_sum(UnscaledValue(ext_price#15))] -Aggregate Attributes [1]: [sum#39] -Results [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] +Aggregate Attributes [1]: [sum#37] +Results [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] (39) Exchange -Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] -Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] +Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, 5), ENSURE_REQUIREMENTS, [id=#39] (40) HashAggregate [codegen id : 10] -Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] Functions [1]: [sum(UnscaledValue(ext_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#15))#42] -Results [5]: [i_brand_id#2 AS brand_id#43, i_brand#3 AS brand#44, t_hour#35, t_minute#36, MakeDecimal(sum(UnscaledValue(ext_price#15))#42,17,2) AS ext_price#45] +Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#15))#40] +Results [5]: [i_brand_id#2 AS brand_id#41, i_brand#3 AS brand#42, t_hour#33, t_minute#34, MakeDecimal(sum(UnscaledValue(ext_price#15))#40,17,2) AS ext_price#43] (41) Exchange -Input [5]: [brand_id#43, brand#44, t_hour#35, t_minute#36, ext_price#45] -Arguments: rangepartitioning(ext_price#45 DESC NULLS LAST, brand_id#43 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [5]: [brand_id#41, brand#42, t_hour#33, t_minute#34, ext_price#43] +Arguments: rangepartitioning(ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#44] (42) Sort [codegen id : 11] -Input [5]: [brand_id#43, brand#44, t_hour#35, t_minute#36, ext_price#45] -Arguments: [ext_price#45 DESC NULLS LAST, brand_id#43 ASC NULLS FIRST], true, 0 +Input [5]: [brand_id#41, brand#42, t_hour#33, t_minute#34, ext_price#43] +Arguments: [ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -244,6 +244,6 @@ Output [1]: [d_date_sk#11] Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt index 339dc5d95e4c4..04172ab353f25 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt @@ -100,7 +100,7 @@ Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotn Output [2]: [hd_demo_sk#9, hd_buy_potential#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000 ), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -108,7 +108,7 @@ Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (6) Filter [codegen id : 1] Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = >10000 )) AND isnotnull(hd_demo_sk#9)) +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = >10000)) AND isnotnull(hd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [hd_demo_sk#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt index 177b8e681608a..fe49873e2e3f3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt @@ -207,7 +207,7 @@ Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_ Output [2]: [hd_demo_sk#24, hd_buy_potential#25] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000 ), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000), IsNotNull(hd_demo_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] @@ -215,7 +215,7 @@ Input [2]: [hd_demo_sk#24, hd_buy_potential#25] (31) Filter [codegen id : 5] Input [2]: [hd_demo_sk#24, hd_buy_potential#25] -Condition : ((isnotnull(hd_buy_potential#25) AND (hd_buy_potential#25 = >10000 )) AND isnotnull(hd_demo_sk#24)) +Condition : ((isnotnull(hd_buy_potential#25) AND (hd_buy_potential#25 = >10000)) AND isnotnull(hd_demo_sk#24)) (32) Project [codegen id : 5] Output [1]: [hd_demo_sk#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt index aec6d66c98fdd..e658e11ade4a0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.sf100/explain.txt @@ -121,7 +121,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -129,7 +129,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt index 646a8fbc11a3a..b45f0513ba576 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.0)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt index 9b2ead7ea96f7..74cbccc79fdc5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt @@ -199,285 +199,285 @@ Input [2]: [customer_id#18, year_total#19] Arguments: [customer_id#18 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Output [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] (27) Filter [codegen id : 10] -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#21) +Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_customer_sk#1) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#25, d_year#26] +Output [2]: [d_date_sk#5, d_year#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#25, d_year#26] +Input [2]: [d_date_sk#5, d_year#6] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) +Input [2]: [d_date_sk#5, d_year#6] +Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2002)) AND d_year#6 IN (2001,2002)) AND isnotnull(d_date_sk#5)) (31) BroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [2]: [d_date_sk#5, d_year#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#23] -Right keys [1]: [d_date_sk#25] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] Join condition: None (33) Project [codegen id : 10] -Output [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] -Input [5]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] +Output [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] +Input [5]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6] (34) Exchange -Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] -Arguments: hashpartitioning(ss_customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] +Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#23] (35) Sort [codegen id : 11] -Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] -Arguments: [ss_customer_sk#21 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] +Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (37) Sort [codegen id : 13] -Input [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#21] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#9] Join condition: None (39) Project [codegen id : 14] -Output [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] -Input [7]: [ss_customer_sk#21, ss_net_paid#22, d_year#26, c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] +Input [7]: [ss_customer_sk#1, ss_net_paid#2, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (40) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] -Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] -Aggregate Attributes [1]: [sum#33] -Results [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#2))] +Aggregate Attributes [1]: [sum#24] +Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] (41) Exchange -Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] -Arguments: hashpartitioning(c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] +Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#26] (42) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] -Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] -Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#22))#36] -Results [4]: [c_customer_id#30 AS customer_id#37, c_first_name#31 AS customer_first_name#38, c_last_name#32 AS customer_last_name#39, MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#36,17,2) AS year_total#40] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [sum(UnscaledValue(ss_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#2))#27] +Results [4]: [c_customer_id#10 AS customer_id#28, c_first_name#11 AS customer_first_name#29, c_last_name#12 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#2))#27,17,2) AS year_total#31] (43) Exchange -Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] -Arguments: hashpartitioning(customer_id#37, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +Arguments: hashpartitioning(customer_id#28, 5), ENSURE_REQUIREMENTS, [id=#32] (44) Sort [codegen id : 16] -Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] -Arguments: [customer_id#37 ASC NULLS FIRST], false, 0 +Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +Arguments: [customer_id#28 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#37] +Right keys [1]: [customer_id#28] Join condition: None (46) 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#33, ws_net_paid#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] (48) Filter [codegen id : 19] -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#33, ws_net_paid#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_bill_customer_sk#33) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#45, d_year#46] +Output [2]: [d_date_sk#5, d_year#6] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#45] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#5] Join condition: None (51) Project [codegen id : 19] -Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] -Input [5]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] +Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] (52) Exchange -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] -Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#36] (53) Sort [codegen id : 20] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] -Arguments: [ws_bill_customer_sk#42 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (55) Sort [codegen id : 22] -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Arguments: [c_customer_sk#48 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#42] -Right keys [1]: [c_customer_sk#48] +Left keys [1]: [ws_bill_customer_sk#33] +Right keys [1]: [c_customer_sk#9] Join condition: None (57) Project [codegen id : 23] -Output [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] -Input [7]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46, c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] +Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (58) HashAggregate [codegen id : 23] -Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] -Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum#52] -Results [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum#37] +Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] (59) Exchange -Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] -Arguments: hashpartitioning(c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] +Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#39] (60) HashAggregate [codegen id : 24] -Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] -Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] -Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#55] -Results [2]: [c_customer_id#49 AS customer_id#56, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#55,17,2) AS year_total#57] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#40] +Results [2]: [c_customer_id#10 AS customer_id#41, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#40,17,2) AS year_total#42] (61) Filter [codegen id : 24] -Input [2]: [customer_id#56, year_total#57] -Condition : (isnotnull(year_total#57) AND (year_total#57 > 0.00)) +Input [2]: [customer_id#41, year_total#42] +Condition : (isnotnull(year_total#42) AND (year_total#42 > 0.00)) (62) Project [codegen id : 24] -Output [2]: [customer_id#56 AS customer_id#58, year_total#57 AS year_total#59] -Input [2]: [customer_id#56, year_total#57] +Output [2]: [customer_id#41 AS customer_id#43, year_total#42 AS year_total#44] +Input [2]: [customer_id#41, year_total#42] (63) Exchange -Input [2]: [customer_id#58, year_total#59] -Arguments: hashpartitioning(customer_id#58, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [2]: [customer_id#43, year_total#44] +Arguments: hashpartitioning(customer_id#43, 5), ENSURE_REQUIREMENTS, [id=#45] (64) Sort [codegen id : 25] -Input [2]: [customer_id#58, year_total#59] -Arguments: [customer_id#58 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#43, year_total#44] +Arguments: [customer_id#43 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#58] +Right keys [1]: [customer_id#43] Join condition: None (66) Project [codegen id : 26] -Output [7]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59] -Input [8]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, customer_id#58, year_total#59] +Output [7]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44] +Input [8]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#43, year_total#44] (67) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] (69) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_customer_sk#61) +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_bill_customer_sk#33) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#64, d_year#65] +Output [2]: [d_date_sk#5, d_year#6] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#63] -Right keys [1]: [d_date_sk#64] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#5] Join condition: None (72) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] -Input [5]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65] +Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] (73) Exchange -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] -Arguments: hashpartitioning(ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#46] (74) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] -Arguments: [ws_bill_customer_sk#61 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] +Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (76) Sort [codegen id : 31] -Input [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] -Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#61] -Right keys [1]: [c_customer_sk#67] +Left keys [1]: [ws_bill_customer_sk#33] +Right keys [1]: [c_customer_sk#9] Join condition: None (78) Project [codegen id : 32] -Output [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] -Input [7]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65, c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] +Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] +Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (79) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] -Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#62))] -Aggregate Attributes [1]: [sum#71] -Results [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum#47] +Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] (80) Exchange -Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] -Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] +Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#49] (81) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] -Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] -Functions [1]: [sum(UnscaledValue(ws_net_paid#62))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#62))#74] -Results [2]: [c_customer_id#68 AS customer_id#75, MakeDecimal(sum(UnscaledValue(ws_net_paid#62))#74,17,2) AS year_total#76] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#50] +Results [2]: [c_customer_id#10 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#50,17,2) AS year_total#52] (82) Exchange -Input [2]: [customer_id#75, year_total#76] -Arguments: hashpartitioning(customer_id#75, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [2]: [customer_id#51, year_total#52] +Arguments: hashpartitioning(customer_id#51, 5), ENSURE_REQUIREMENTS, [id=#53] (83) Sort [codegen id : 34] -Input [2]: [customer_id#75, year_total#76] -Arguments: [customer_id#75 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#51, year_total#52] +Arguments: [customer_id#51 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#75] -Join condition: (CASE WHEN (year_total#59 > 0.00) THEN CheckOverflow((promote_precision(year_total#76) / promote_precision(year_total#59)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#40) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#51] +Join condition: (CASE WHEN (year_total#44 > 0.00) THEN CheckOverflow((promote_precision(year_total#52) / promote_precision(year_total#44)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#31) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) (85) Project [codegen id : 35] -Output [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] -Input [9]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59, customer_id#75, year_total#76] +Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Input [9]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44, customer_id#51, year_total#52] (86) TakeOrderedAndProject -Input [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] -Arguments: 100, [customer_id#37 ASC NULLS FIRST, customer_id#37 ASC NULLS FIRST, customer_id#37 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39] +Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Arguments: 100, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] ===== Subqueries ===== @@ -488,15 +488,15 @@ ReusedExchange (87) (87) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#5, d_year#6] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#21 ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#25, d_year#26] +Output [2]: [d_date_sk#5, d_year#6] -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#24 +Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#21 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt index 9fccc4c4ba66d..db3a0a43be591 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt @@ -171,282 +171,282 @@ Input [2]: [customer_id#17, year_total#18] Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) (20) Scan parquet default.customer -Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] (22) Filter [codegen id : 6] -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Condition : (isnotnull(c_customer_sk#19) AND isnotnull(c_customer_id#20)) +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (23) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#19)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] (25) Filter [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_customer_sk#23) +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Condition : isnotnull(ss_customer_sk#5) (26) BroadcastExchange -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#19] -Right keys [1]: [ss_customer_sk#23] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#5] Join condition: None (28) Project [codegen id : 6] -Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25] -Input [7]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] +Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#28, d_year#29] +Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#28, d_year#29] +Input [2]: [d_date_sk#10, d_year#11] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#28, d_year#29] -Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) +Input [2]: [d_date_sk#10, d_year#11] +Condition : (((isnotnull(d_year#11) AND (d_year#11 = 2002)) AND d_year#11 IN (2001,2002)) AND isnotnull(d_date_sk#10)) (32) BroadcastExchange -Input [2]: [d_date_sk#28, d_year#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [2]: [d_date_sk#10, d_year#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#25] -Right keys [1]: [d_date_sk#28] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#10] Join condition: None (34) Project [codegen id : 6] -Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] -Input [7]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#28, d_year#29] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] +Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#10, d_year#11] (35) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] -Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum#31] -Results [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] +Aggregate Attributes [1]: [sum#22] +Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] (36) Exchange -Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] -Arguments: hashpartitioning(c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#24] (37) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] -Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] -Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#34] -Results [4]: [c_customer_id#20 AS customer_id#35, c_first_name#21 AS customer_first_name#36, c_last_name#22 AS customer_last_name#37, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#34,17,2) AS year_total#38] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#25] +Results [4]: [c_customer_id#2 AS customer_id#26, c_first_name#3 AS customer_first_name#27, c_last_name#4 AS customer_last_name#28, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#25,17,2) AS year_total#29] (38) BroadcastExchange -Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#39] +Input [4]: [customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#35] +Right keys [1]: [customer_id#26] Join condition: None (40) Scan parquet default.customer -Output [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] +Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] (42) Filter [codegen id : 10] -Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] -Condition : (isnotnull(c_customer_sk#40) AND isnotnull(c_customer_id#41)) +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (43) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] (45) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] -Condition : isnotnull(ws_bill_customer_sk#44) +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Condition : isnotnull(ws_bill_customer_sk#31) (46) BroadcastExchange -Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#40] -Right keys [1]: [ws_bill_customer_sk#44] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#31] Join condition: None (48) Project [codegen id : 10] -Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46] -Input [7]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] +Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#48, d_year#49] +Output [2]: [d_date_sk#10, d_year#11] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#46] -Right keys [1]: [d_date_sk#48] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#10] Join condition: None (51) Project [codegen id : 10] -Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] -Input [7]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46, d_date_sk#48, d_year#49] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] +Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] (52) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] -Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#45))] -Aggregate Attributes [1]: [sum#50] -Results [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] +Aggregate Attributes [1]: [sum#35] +Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] (53) Exchange -Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] -Arguments: hashpartitioning(c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#37] (54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] -Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] -Functions [1]: [sum(UnscaledValue(ws_net_paid#45))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#45))#53] -Results [2]: [c_customer_id#41 AS customer_id#54, MakeDecimal(sum(UnscaledValue(ws_net_paid#45))#53,17,2) AS year_total#55] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#38] +Results [2]: [c_customer_id#2 AS customer_id#39, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#38,17,2) AS year_total#40] (55) Filter [codegen id : 11] -Input [2]: [customer_id#54, year_total#55] -Condition : (isnotnull(year_total#55) AND (year_total#55 > 0.00)) +Input [2]: [customer_id#39, year_total#40] +Condition : (isnotnull(year_total#40) AND (year_total#40 > 0.00)) (56) Project [codegen id : 11] -Output [2]: [customer_id#54 AS customer_id#56, year_total#55 AS year_total#57] -Input [2]: [customer_id#54, year_total#55] +Output [2]: [customer_id#39 AS customer_id#41, year_total#40 AS year_total#42] +Input [2]: [customer_id#39, year_total#40] (57) BroadcastExchange -Input [2]: [customer_id#56, year_total#57] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#58] +Input [2]: [customer_id#41, year_total#42] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#43] (58) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#56] +Right keys [1]: [customer_id#41] Join condition: None (59) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57] -Input [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, customer_id#56, year_total#57] +Output [7]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42] +Input [8]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, customer_id#41, year_total#42] (60) Scan parquet default.customer -Output [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] +Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] (62) Filter [codegen id : 14] -Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] -Condition : (isnotnull(c_customer_sk#59) AND isnotnull(c_customer_id#60)) +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (63) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#19)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] (65) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] -Condition : isnotnull(ws_bill_customer_sk#63) +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Condition : isnotnull(ws_bill_customer_sk#31) (66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#59] -Right keys [1]: [ws_bill_customer_sk#63] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#31] Join condition: None (68) Project [codegen id : 14] -Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65] -Input [7]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] +Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#67, d_year#68] +Output [2]: [d_date_sk#10, d_year#11] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#67] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#10] Join condition: None (71) Project [codegen id : 14] -Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] -Input [7]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65, d_date_sk#67, d_year#68] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] +Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] (72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] -Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#64))] -Aggregate Attributes [1]: [sum#69] -Results [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] +Aggregate Attributes [1]: [sum#45] +Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] (73) Exchange -Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] -Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#47] (74) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] -Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] -Functions [1]: [sum(UnscaledValue(ws_net_paid#64))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#64))#72] -Results [2]: [c_customer_id#60 AS customer_id#73, MakeDecimal(sum(UnscaledValue(ws_net_paid#64))#72,17,2) AS year_total#74] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#48] +Results [2]: [c_customer_id#2 AS customer_id#49, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#48,17,2) AS year_total#50] (75) BroadcastExchange -Input [2]: [customer_id#73, year_total#74] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#75] +Input [2]: [customer_id#49, year_total#50] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#51] (76) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#73] -Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#57)), 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#49] +Join condition: (CASE WHEN (year_total#42 > 0.00) THEN CheckOverflow((promote_precision(year_total#50) / promote_precision(year_total#42)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#29) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END) (77) Project [codegen id : 16] -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#57, customer_id#73, year_total#74] +Output [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] +Input [9]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42, customer_id#49, year_total#50] (78) TakeOrderedAndProject -Input [3]: [customer_id#35, customer_first_name#36, customer_last_name#37] -Arguments: 100, [customer_id#35 ASC NULLS FIRST, customer_id#35 ASC NULLS FIRST, customer_id#35 ASC NULLS FIRST], [customer_id#35, customer_first_name#36, customer_last_name#37] +Input [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] +Arguments: 100, [customer_id#26 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST], [customer_id#26, customer_first_name#27, customer_last_name#28] ===== Subqueries ===== @@ -457,15 +457,15 @@ ReusedExchange (79) (79) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#10, d_year#11] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#19 ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#28, d_year#29] +Output [2]: [d_date_sk#10, d_year#11] -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#26 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#19 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt index a06e41562f8ff..9472a8935eb8b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt @@ -153,7 +153,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -161,7 +161,7 @@ Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_categor (6) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books)) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#31] +Right keys [1]: [i_item_sk#7] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#36, d_year#37] +Output [2]: [d_date_sk#14, d_year#15] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] +Right keys [1]: [d_date_sk#14] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] -Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] (42) Exchange -Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] -Condition : isnotnull(ws_item_sk#47) +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_item_sk#40) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#47] -Right keys [1]: [i_item_sk#52] +Left keys [1]: [ws_item_sk#40] +Right keys [1]: [i_item_sk#7] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] -Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#57, d_year#58] +Output [2]: [d_date_sk#14, d_year#15] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#57] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#14] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] (55) Exchange -Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] (61) Exchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] -Right keys [2]: [wr_order_number#61, wr_item_sk#60] +Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] +Right keys [2]: [wr_order_number#47, wr_item_sk#46] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] -Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] +Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#69, sum#70] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Aggregate Attributes [2]: [sum#55, sum#56] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] +Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] -Condition : isnotnull(cs_item_sk#79) +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Condition : isnotnull(cs_item_sk#1) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#79] -Right keys [1]: [i_item_sk#85] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#66] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] -Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#71, d_year#72] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#90, d_year#91] +Input [2]: [d_date_sk#71, d_year#72] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#90, d_year#91] -Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) +Input [2]: [d_date_sk#71, d_year#72] +Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) (83) BroadcastExchange -Input [2]: [d_date_sk#90, d_year#91] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] +Input [2]: [d_date_sk#71, d_year#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#83] -Right keys [1]: [d_date_sk#90] +Left keys [1]: [cs_sold_date_sk#5] +Right keys [1]: [d_date_sk#71] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (86) Exchange -Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] -Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#80, cs_item_sk#79] -Right keys [2]: [cr_order_number#95, cr_item_sk#94] +Left keys [2]: [cs_order_number#2, cs_item_sk#1] +Right keys [2]: [cr_order_number#19, cr_item_sk#18] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] -Condition : isnotnull(ss_item_sk#98) +Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Condition : isnotnull(ss_item_sk#26) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#98] -Right keys [1]: [i_item_sk#103] +Left keys [1]: [ss_item_sk#26] +Right keys [1]: [i_item_sk#66] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] -Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#108, d_year#109] +Output [2]: [d_date_sk#71, d_year#72] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#102] -Right keys [1]: [d_date_sk#108] +Left keys [1]: [ss_sold_date_sk#30] +Right keys [1]: [d_date_sk#71] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (101) Exchange -Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] -Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] -Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] +Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] -Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] -Condition : isnotnull(ws_item_sk#116) +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_item_sk#40) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#116] -Right keys [1]: [i_item_sk#121] +Left keys [1]: [ws_item_sk#40] +Right keys [1]: [i_item_sk#66] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] -Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#126, d_year#127] +Output [2]: [d_date_sk#71, d_year#72] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#120] -Right keys [1]: [d_date_sk#126] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#71] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (116) Exchange -Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] -Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] -Right keys [2]: [wr_order_number#130, wr_item_sk#129] +Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] +Right keys [2]: [wr_order_number#47, wr_item_sk#46] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] -Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] +Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#135, sum#136] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Aggregate Attributes [2]: [sum#82, sum#83] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] (127) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] -Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] -Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] +Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] (129) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] -Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] (130) Sort [codegen id : 50] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] -Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 (131) 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#86, i_class_id#87, i_category_id#88, i_manufact_id#89] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] (133) TakeOrderedAndProject -Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] -Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#71, d_year#72] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt index a06e41562f8ff..9472a8935eb8b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt @@ -153,7 +153,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -161,7 +161,7 @@ Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_categor (6) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books)) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#31] +Right keys [1]: [i_item_sk#7] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#36, d_year#37] +Output [2]: [d_date_sk#14, d_year#15] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] +Right keys [1]: [d_date_sk#14] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] -Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] (42) Exchange -Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] -Condition : isnotnull(ws_item_sk#47) +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_item_sk#40) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#47] -Right keys [1]: [i_item_sk#52] +Left keys [1]: [ws_item_sk#40] +Right keys [1]: [i_item_sk#7] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] -Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#57, d_year#58] +Output [2]: [d_date_sk#14, d_year#15] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#57] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#14] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] (55) Exchange -Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] (61) Exchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] -Right keys [2]: [wr_order_number#61, wr_item_sk#60] +Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] +Right keys [2]: [wr_order_number#47, wr_item_sk#46] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] -Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] +Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#69, sum#70] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Aggregate Attributes [2]: [sum#55, sum#56] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] +Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] -Condition : isnotnull(cs_item_sk#79) +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Condition : isnotnull(cs_item_sk#1) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#79] -Right keys [1]: [i_item_sk#85] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#66] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] -Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#71, d_year#72] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#90, d_year#91] +Input [2]: [d_date_sk#71, d_year#72] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#90, d_year#91] -Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) +Input [2]: [d_date_sk#71, d_year#72] +Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) (83) BroadcastExchange -Input [2]: [d_date_sk#90, d_year#91] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] +Input [2]: [d_date_sk#71, d_year#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#83] -Right keys [1]: [d_date_sk#90] +Left keys [1]: [cs_sold_date_sk#5] +Right keys [1]: [d_date_sk#71] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (86) Exchange -Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] -Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#80, cs_item_sk#79] -Right keys [2]: [cr_order_number#95, cr_item_sk#94] +Left keys [2]: [cs_order_number#2, cs_item_sk#1] +Right keys [2]: [cr_order_number#19, cr_item_sk#18] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] -Condition : isnotnull(ss_item_sk#98) +Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Condition : isnotnull(ss_item_sk#26) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#98] -Right keys [1]: [i_item_sk#103] +Left keys [1]: [ss_item_sk#26] +Right keys [1]: [i_item_sk#66] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] -Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#108, d_year#109] +Output [2]: [d_date_sk#71, d_year#72] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#102] -Right keys [1]: [d_date_sk#108] +Left keys [1]: [ss_sold_date_sk#30] +Right keys [1]: [d_date_sk#71] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (101) Exchange -Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] -Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] -Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] +Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] -Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] -Condition : isnotnull(ws_item_sk#116) +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_item_sk#40) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#116] -Right keys [1]: [i_item_sk#121] +Left keys [1]: [ws_item_sk#40] +Right keys [1]: [i_item_sk#66] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] -Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#126, d_year#127] +Output [2]: [d_date_sk#71, d_year#72] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#120] -Right keys [1]: [d_date_sk#126] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#71] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (116) Exchange -Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] -Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] -Right keys [2]: [wr_order_number#130, wr_item_sk#129] +Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] +Right keys [2]: [wr_order_number#47, wr_item_sk#46] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] -Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] +Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#135, sum#136] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Aggregate Attributes [2]: [sum#82, sum#83] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] (127) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] -Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] -Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] +Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] (129) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] -Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] (130) Sort [codegen id : 50] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] -Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 (131) 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#86, i_class_id#87, i_category_id#88, i_manufact_id#89] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] (133) TakeOrderedAndProject -Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] -Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#71, d_year#72] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt index 9284172139688..0092354e54be0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt @@ -134,93 +134,93 @@ Input [4]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sol Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [id=#19] (20) Scan parquet default.date_dim -Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] +Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow -Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] (22) Filter -Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Condition : isnotnull(d_date_sk#20) +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Condition : isnotnull(d_date_sk#5) (23) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] +Right keys [1]: [d_date_sk#5] Join condition: None (24) Project [codegen id : 5] -Output [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#21, d_qoy#22] -Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, d_date_sk#20, d_year#21, d_qoy#22] +Output [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#6, d_qoy#7] +Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, d_date_sk#5, d_year#6, d_qoy#7] (25) BroadcastExchange -Input [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#21, d_qoy#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#6, d_qoy#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (26) Scan parquet default.item -Output [2]: [i_item_sk#24, i_category#25] +Output [2]: [i_item_sk#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (27) ColumnarToRow -Input [2]: [i_item_sk#24, i_category#25] +Input [2]: [i_item_sk#9, i_category#10] (28) Filter -Input [2]: [i_item_sk#24, i_category#25] -Condition : isnotnull(i_item_sk#24) +Input [2]: [i_item_sk#9, i_category#10] +Condition : isnotnull(i_item_sk#9) (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#15] -Right keys [1]: [i_item_sk#24] +Right keys [1]: [i_item_sk#9] Join condition: None (30) Project [codegen id : 6] -Output [6]: [web AS channel#26, ws_ship_customer_sk#16 AS col_name#27, d_year#21, d_qoy#22, i_category#25, ws_ext_sales_price#17 AS ext_sales_price#28] -Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#21, d_qoy#22, i_item_sk#24, i_category#25] +Output [6]: [web AS channel#21, ws_ship_customer_sk#16 AS col_name#22, d_year#6, d_qoy#7, i_category#10, ws_ext_sales_price#17 AS ext_sales_price#23] +Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#6, d_qoy#7, i_item_sk#9, i_category#10] (31) Scan parquet default.catalog_sales -Output [4]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32] +Output [4]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#32)] +PartitionFilters: [isnotnull(cs_sold_date_sk#27)] PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 9] -Input [4]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32] +Input [4]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] (33) Filter [codegen id : 9] -Input [4]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32] -Condition : (isnull(cs_ship_addr_sk#29) AND isnotnull(cs_item_sk#30)) +Input [4]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +Condition : (isnull(cs_ship_addr_sk#24) AND isnotnull(cs_item_sk#25)) (34) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] (35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#32] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [cs_sold_date_sk#27] +Right keys [1]: [d_date_sk#5] Join condition: None (36) Project [codegen id : 9] -Output [5]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, d_year#34, d_qoy#35] -Input [7]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32, d_date_sk#33, d_year#34, d_qoy#35] +Output [5]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_year#6, d_qoy#7] +Input [7]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#5, d_year#6, d_qoy#7] (37) ReusedExchange [Reuses operator id: 13] -Output [2]: [i_item_sk#36, i_category#37] +Output [2]: [i_item_sk#9, i_category#10] (38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#30] -Right keys [1]: [i_item_sk#36] +Left keys [1]: [cs_item_sk#25] +Right keys [1]: [i_item_sk#9] Join condition: None (39) Project [codegen id : 9] -Output [6]: [catalog AS channel#38, cs_ship_addr_sk#29 AS col_name#39, d_year#34, d_qoy#35, i_category#37, cs_ext_sales_price#31 AS ext_sales_price#40] -Input [7]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, d_year#34, d_qoy#35, i_item_sk#36, i_category#37] +Output [6]: [catalog AS channel#28, cs_ship_addr_sk#24 AS col_name#29, d_year#6, d_qoy#7, i_category#10, cs_ext_sales_price#26 AS ext_sales_price#30] +Input [7]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_year#6, d_qoy#7, i_item_sk#9, i_category#10] (40) Union @@ -228,21 +228,21 @@ Input [7]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, d_year#34, Input [6]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, ext_sales_price#14] Keys [5]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10] Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count#41, sum#42] -Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#43, sum#44] +Aggregate Attributes [2]: [count#31, sum#32] +Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#33, sum#34] (42) Exchange -Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#43, sum#44] -Arguments: hashpartitioning(channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#33, sum#34] +Arguments: hashpartitioning(channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, 5), ENSURE_REQUIREMENTS, [id=#35] (43) HashAggregate [codegen id : 11] -Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#43, sum#44] +Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#33, sum#34] Keys [5]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count(1)#46, sum(UnscaledValue(ext_sales_price#14))#47] -Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count(1)#46 AS sales_cnt#48, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#47,17,2) AS sales_amt#49] +Aggregate Attributes [2]: [count(1)#36, sum(UnscaledValue(ext_sales_price#14))#37] +Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count(1)#36 AS sales_cnt#38, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#37,17,2) AS sales_amt#39] (44) TakeOrderedAndProject -Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#48, sales_amt#49] -Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#6 ASC NULLS FIRST, d_qoy#7 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#48, sales_amt#49] +Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#38, sales_amt#39] +Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#6 ASC NULLS FIRST, d_qoy#7 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#38, sales_amt#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt index 026c9396cd025..1250725d1782e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt @@ -124,67 +124,67 @@ Input [4]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sol Condition : (isnull(ws_ship_customer_sk#16) AND isnotnull(ws_item_sk#15)) (19) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#19, i_category#20] +Output [2]: [i_item_sk#5, i_category#6] (20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#15] -Right keys [1]: [i_item_sk#19] +Right keys [1]: [i_item_sk#5] Join condition: None (21) Project [codegen id : 6] -Output [4]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#20] -Input [6]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_item_sk#19, i_category#20] +Output [4]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#6] +Input [6]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_item_sk#5, i_category#6] (22) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] +Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] (23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#18] -Right keys [1]: [d_date_sk#21] +Right keys [1]: [d_date_sk#8] Join condition: None (24) Project [codegen id : 6] -Output [6]: [web AS channel#24, ws_ship_customer_sk#16 AS col_name#25, d_year#22, d_qoy#23, i_category#20, ws_ext_sales_price#17 AS ext_sales_price#26] -Input [7]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#20, d_date_sk#21, d_year#22, d_qoy#23] +Output [6]: [web AS channel#19, ws_ship_customer_sk#16 AS col_name#20, d_year#9, d_qoy#10, i_category#6, ws_ext_sales_price#17 AS ext_sales_price#21] +Input [7]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#6, d_date_sk#8, d_year#9, d_qoy#10] (25) Scan parquet default.catalog_sales -Output [4]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30] +Output [4]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#30)] +PartitionFilters: [isnotnull(cs_sold_date_sk#25)] PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 9] -Input [4]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30] +Input [4]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25] (27) Filter [codegen id : 9] -Input [4]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30] -Condition : (isnull(cs_ship_addr_sk#27) AND isnotnull(cs_item_sk#28)) +Input [4]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25] +Condition : (isnull(cs_ship_addr_sk#22) AND isnotnull(cs_item_sk#23)) (28) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#31, i_category#32] +Output [2]: [i_item_sk#5, i_category#6] (29) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#28] -Right keys [1]: [i_item_sk#31] +Left keys [1]: [cs_item_sk#23] +Right keys [1]: [i_item_sk#5] Join condition: None (30) Project [codegen id : 9] -Output [4]: [cs_ship_addr_sk#27, cs_ext_sales_price#29, cs_sold_date_sk#30, i_category#32] -Input [6]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30, i_item_sk#31, i_category#32] +Output [4]: [cs_ship_addr_sk#22, cs_ext_sales_price#24, cs_sold_date_sk#25, i_category#6] +Input [6]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25, i_item_sk#5, i_category#6] (31) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#30] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [cs_sold_date_sk#25] +Right keys [1]: [d_date_sk#8] Join condition: None (33) Project [codegen id : 9] -Output [6]: [catalog AS channel#36, cs_ship_addr_sk#27 AS col_name#37, d_year#34, d_qoy#35, i_category#32, cs_ext_sales_price#29 AS ext_sales_price#38] -Input [7]: [cs_ship_addr_sk#27, cs_ext_sales_price#29, cs_sold_date_sk#30, i_category#32, d_date_sk#33, d_year#34, d_qoy#35] +Output [6]: [catalog AS channel#26, cs_ship_addr_sk#22 AS col_name#27, d_year#9, d_qoy#10, i_category#6, cs_ext_sales_price#24 AS ext_sales_price#28] +Input [7]: [cs_ship_addr_sk#22, cs_ext_sales_price#24, cs_sold_date_sk#25, i_category#6, d_date_sk#8, d_year#9, d_qoy#10] (34) Union @@ -192,21 +192,21 @@ Input [7]: [cs_ship_addr_sk#27, cs_ext_sales_price#29, cs_sold_date_sk#30, i_cat Input [6]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, ext_sales_price#14] Keys [5]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6] Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count#39, sum#40] -Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#41, sum#42] +Aggregate Attributes [2]: [count#29, sum#30] +Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] (36) Exchange -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#41, sum#42] -Arguments: hashpartitioning(channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] +Arguments: hashpartitioning(channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, 5), ENSURE_REQUIREMENTS, [id=#33] (37) HashAggregate [codegen id : 11] -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#41, sum#42] +Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] Keys [5]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count(1)#44, sum(UnscaledValue(ext_sales_price#14))#45] -Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count(1)#44 AS sales_cnt#46, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#45,17,2) AS sales_amt#47] +Aggregate Attributes [2]: [count(1)#34, sum(UnscaledValue(ext_sales_price#14))#35] +Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count(1)#34 AS sales_cnt#36, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#35,17,2) AS sales_amt#37] (38) TakeOrderedAndProject -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#46, sales_amt#47] -Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#46, sales_amt#47] +Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#36, sales_amt#37] +Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#36, sales_amt#37] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt index 4b2299ca2e749..a3626dfe3293d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt @@ -209,38 +209,38 @@ Output [4]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_s Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25] (26) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#26] +Output [1]: [d_date_sk#6] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#26 as bigint)] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (28) Project [codegen id : 6] Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#26] +Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#6] (29) HashAggregate [codegen id : 6] Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] Keys [1]: [s_store_sk#25] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#27, sum#28] -Results [3]: [s_store_sk#25, sum#29, sum#30] +Aggregate Attributes [2]: [sum#26, sum#27] +Results [3]: [s_store_sk#25, sum#28, sum#29] (30) Exchange -Input [3]: [s_store_sk#25, sum#29, sum#30] -Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [s_store_sk#25, sum#28, sum#29] +Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#25, sum#29, sum#30] +Input [3]: [s_store_sk#25, sum#28, sum#29] Keys [1]: [s_store_sk#25] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] -Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] +Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] (32) BroadcastExchange -Input [3]: [s_store_sk#25, returns#34, profit_loss#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [3]: [s_store_sk#25, returns#33, profit_loss#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] @@ -248,269 +248,269 @@ Right keys [1]: [s_store_sk#25] Join condition: None (34) Project [codegen id : 8] -Output [5]: [sales#18, coalesce(returns#34, 0.00) AS returns#37, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#38, store channel AS channel#39, s_store_sk#9 AS id#40] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#34, profit_loss#35] +Output [5]: [sales#18, coalesce(returns#33, 0.00) AS returns#36, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#37, store channel AS channel#38, s_store_sk#9 AS id#39] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] +Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] +Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#45] +Output [1]: [d_date_sk#6] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#44] -Right keys [1]: [d_date_sk#45] +Left keys [1]: [cs_sold_date_sk#43] +Right keys [1]: [d_date_sk#6] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] -Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] +Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] +Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] -Keys [1]: [cs_call_center_sk#41] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] -Aggregate Attributes [2]: [sum#46, sum#47] -Results [3]: [cs_call_center_sk#41, sum#48, sum#49] +Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] +Keys [1]: [cs_call_center_sk#40] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] +Aggregate Attributes [2]: [sum#44, sum#45] +Results [3]: [cs_call_center_sk#40, sum#46, sum#47] (41) Exchange -Input [3]: [cs_call_center_sk#41, sum#48, sum#49] -Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [3]: [cs_call_center_sk#40, sum#46, sum#47] +Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] (42) HashAggregate [codegen id : 14] -Input [3]: [cs_call_center_sk#41, sum#48, sum#49] -Keys [1]: [cs_call_center_sk#41] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] -Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] +Input [3]: [cs_call_center_sk#40, sum#46, sum#47] +Keys [1]: [cs_call_center_sk#40] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] +Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] (43) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Output [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#55), dynamicpruningexpression(cr_returned_date_sk#55 IN dynamicpruning#5)] ReadSchema: struct (44) ColumnarToRow [codegen id : 12] -Input [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Input [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] (45) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#58] +Output [1]: [d_date_sk#6] (46) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cr_returned_date_sk#57] -Right keys [1]: [d_date_sk#58] +Left keys [1]: [cr_returned_date_sk#55] +Right keys [1]: [d_date_sk#6] Join condition: None (47) Project [codegen id : 12] -Output [2]: [cr_return_amount#55, cr_net_loss#56] -Input [4]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57, d_date_sk#58] +Output [2]: [cr_return_amount#53, cr_net_loss#54] +Input [4]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55, d_date_sk#6] (48) HashAggregate [codegen id : 12] -Input [2]: [cr_return_amount#55, cr_net_loss#56] +Input [2]: [cr_return_amount#53, cr_net_loss#54] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#55)), partial_sum(UnscaledValue(cr_net_loss#56))] -Aggregate Attributes [2]: [sum#59, sum#60] -Results [2]: [sum#61, sum#62] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#53)), partial_sum(UnscaledValue(cr_net_loss#54))] +Aggregate Attributes [2]: [sum#56, sum#57] +Results [2]: [sum#58, sum#59] (49) Exchange -Input [2]: [sum#61, sum#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] +Input [2]: [sum#58, sum#59] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#60] (50) HashAggregate [codegen id : 13] -Input [2]: [sum#61, sum#62] +Input [2]: [sum#58, sum#59] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#55)), sum(UnscaledValue(cr_net_loss#56))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#55))#64, sum(UnscaledValue(cr_net_loss#56))#65] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#55))#64,17,2) AS returns#66, MakeDecimal(sum(UnscaledValue(cr_net_loss#56))#65,17,2) AS profit_loss#67] +Functions [2]: [sum(UnscaledValue(cr_return_amount#53)), sum(UnscaledValue(cr_net_loss#54))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#53))#61, sum(UnscaledValue(cr_net_loss#54))#62] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#53))#61,17,2) AS returns#63, MakeDecimal(sum(UnscaledValue(cr_net_loss#54))#62,17,2) AS profit_loss#64] (51) BroadcastExchange -Input [2]: [returns#66, profit_loss#67] -Arguments: IdentityBroadcastMode, [id=#68] +Input [2]: [returns#63, profit_loss#64] +Arguments: IdentityBroadcastMode, [id=#65] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [sales#53, returns#66, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#67 as decimal(18,2)))), DecimalType(18,2), true) AS profit#69, catalog channel AS channel#70, cs_call_center_sk#41 AS id#71] -Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#66, profit_loss#67] +Output [5]: [sales#51, returns#63, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#64 as decimal(18,2)))), DecimalType(18,2), true) AS profit#66, catalog channel AS channel#67, cs_call_center_sk#40 AS id#68] +Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#63, profit_loss#64] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] -Condition : isnotnull(ws_web_page_sk#72) +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Condition : isnotnull(ws_web_page_sk#69) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#76] +Output [1]: [d_date_sk#6] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#75] -Right keys [1]: [d_date_sk#76] +Left keys [1]: [ws_sold_date_sk#72] +Right keys [1]: [d_date_sk#6] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] -Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] +Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] +Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#77] +Output [1]: [wp_web_page_sk#73] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#77] +Input [1]: [wp_web_page_sk#73] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#77] -Condition : isnotnull(wp_web_page_sk#77) +Input [1]: [wp_web_page_sk#73] +Condition : isnotnull(wp_web_page_sk#73) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#77] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] +Input [1]: [wp_web_page_sk#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#72] -Right keys [1]: [wp_web_page_sk#77] +Left keys [1]: [ws_web_page_sk#69] +Right keys [1]: [wp_web_page_sk#73] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] -Aggregate Attributes [2]: [sum#79, sum#80] -Results [3]: [wp_web_page_sk#77, sum#81, sum#82] +Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Keys [1]: [wp_web_page_sk#73] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] +Aggregate Attributes [2]: [sum#75, sum#76] +Results [3]: [wp_web_page_sk#73, sum#77, sum#78] (67) Exchange -Input [3]: [wp_web_page_sk#77, sum#81, sum#82] -Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [3]: [wp_web_page_sk#73, sum#77, sum#78] +Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#77, sum#81, sum#82] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] -Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] +Input [3]: [wp_web_page_sk#73, sum#77, sum#78] +Keys [1]: [wp_web_page_sk#73] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] +Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] -Condition : isnotnull(wr_web_page_sk#88) +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Condition : isnotnull(wr_web_page_sk#84) (72) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#92] +Output [1]: [wp_web_page_sk#88] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#88] -Right keys [1]: [cast(wp_web_page_sk#92 as bigint)] +Left keys [1]: [wr_web_page_sk#84] +Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [4]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] -Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] +Output [4]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] +Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] (75) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#93] +Output [1]: [d_date_sk#6] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#91] -Right keys [1]: [cast(d_date_sk#93 as bigint)] +Left keys [1]: [wr_returned_date_sk#87] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] -Input [5]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92, d_date_sk#93] +Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Input [5]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88, d_date_sk#6] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] -Keys [1]: [wp_web_page_sk#92] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] -Aggregate Attributes [2]: [sum#94, sum#95] -Results [3]: [wp_web_page_sk#92, sum#96, sum#97] +Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Keys [1]: [wp_web_page_sk#88] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] +Aggregate Attributes [2]: [sum#89, sum#90] +Results [3]: [wp_web_page_sk#88, sum#91, sum#92] (79) Exchange -Input [3]: [wp_web_page_sk#92, sum#96, sum#97] -Arguments: hashpartitioning(wp_web_page_sk#92, 5), ENSURE_REQUIREMENTS, [id=#98] +Input [3]: [wp_web_page_sk#88, sum#91, sum#92] +Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#92, sum#96, sum#97] -Keys [1]: [wp_web_page_sk#92] -Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] -Results [3]: [wp_web_page_sk#92, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] +Input [3]: [wp_web_page_sk#88, sum#91, sum#92] +Keys [1]: [wp_web_page_sk#88] +Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] +Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#92, returns#101, profit_loss#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] +Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#77] -Right keys [1]: [wp_web_page_sk#92] +Left keys [1]: [wp_web_page_sk#73] +Right keys [1]: [wp_web_page_sk#88] Join condition: None (83) Project [codegen id : 22] -Output [5]: [sales#86, coalesce(returns#101, 0.00) AS returns#104, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#105, web channel AS channel#106, wp_web_page_sk#77 AS id#107] -Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#92, returns#101, profit_loss#102] +Output [5]: [sales#82, coalesce(returns#96, 0.00) AS returns#99, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#100, web channel AS channel#101, wp_web_page_sk#73 AS id#102] +Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] (84) Union (85) Expand [codegen id : 23] -Input [5]: [sales#18, returns#37, profit#38, channel#39, id#40] -Arguments: [List(sales#18, returns#37, profit#38, channel#39, id#40, 0), List(sales#18, returns#37, profit#38, channel#39, null, 1), List(sales#18, returns#37, profit#38, null, null, 3)], [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] +Input [5]: [sales#18, returns#36, profit#37, channel#38, id#39] +Arguments: [List(sales#18, returns#36, profit#37, channel#38, id#39, 0), List(sales#18, returns#36, profit#37, channel#38, null, 1), List(sales#18, returns#36, profit#37, null, null, 3)], [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] (86) HashAggregate [codegen id : 23] -Input [6]: [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] -Keys [3]: [channel#108, id#109, spark_grouping_id#110] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#37), partial_sum(profit#38)] -Aggregate Attributes [6]: [sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] -Results [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Input [6]: [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] +Keys [3]: [channel#103, id#104, spark_grouping_id#105] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#36), partial_sum(profit#37)] +Aggregate Attributes [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] +Results [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] (87) Exchange -Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Arguments: hashpartitioning(channel#108, id#109, spark_grouping_id#110, 5), ENSURE_REQUIREMENTS, [id=#123] +Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Arguments: hashpartitioning(channel#103, id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, [id=#118] (88) HashAggregate [codegen id : 24] -Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Keys [3]: [channel#108, id#109, spark_grouping_id#110] -Functions [3]: [sum(sales#18), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#18)#124, sum(returns#37)#125, sum(profit#38)#126] -Results [5]: [channel#108, id#109, sum(sales#18)#124 AS sales#127, sum(returns#37)#125 AS returns#128, sum(profit#38)#126 AS profit#129] +Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Keys [3]: [channel#103, id#104, spark_grouping_id#105] +Functions [3]: [sum(sales#18), sum(returns#36), sum(profit#37)] +Aggregate Attributes [3]: [sum(sales#18)#119, sum(returns#36)#120, sum(profit#37)#121] +Results [5]: [channel#103, id#104, sum(sales#18)#119 AS sales#122, sum(returns#36)#120 AS returns#123, sum(profit#37)#121 AS profit#124] (89) TakeOrderedAndProject -Input [5]: [channel#108, id#109, sales#127, returns#128, profit#129] -Arguments: 100, [channel#108 ASC NULLS FIRST, id#109 ASC NULLS FIRST], [channel#108, id#109, sales#127, returns#128, profit#129] +Input [5]: [channel#103, id#104, sales#122, returns#123, profit#124] +Arguments: 100, [channel#103 ASC NULLS FIRST, id#104 ASC NULLS FIRST], [channel#103, id#104, sales#122, returns#123, profit#124] ===== Subqueries ===== @@ -526,14 +526,14 @@ ReusedExchange (91) (91) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#26] +Output [1]: [d_date_sk#6] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#55 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt index 618da39637e23..7a277d5362e32 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt @@ -197,320 +197,320 @@ Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_s Condition : isnotnull(sr_store_sk#20) (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#6] (24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#25 as bigint)] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (25) Project [codegen id : 6] Output [3]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22] -Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#25] +Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#6] (26) ReusedExchange [Reuses operator id: 14] -Output [1]: [s_store_sk#26] +Output [1]: [s_store_sk#25] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_store_sk#20] -Right keys [1]: [cast(s_store_sk#26 as bigint)] +Right keys [1]: [cast(s_store_sk#25 as bigint)] Join condition: None (28) Project [codegen id : 6] -Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] -Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#26] +Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] +Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#25] (29) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] -Keys [1]: [s_store_sk#26] +Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] +Keys [1]: [s_store_sk#25] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#27, sum#28] -Results [3]: [s_store_sk#26, sum#29, sum#30] +Aggregate Attributes [2]: [sum#26, sum#27] +Results [3]: [s_store_sk#25, sum#28, sum#29] (30) Exchange -Input [3]: [s_store_sk#26, sum#29, sum#30] -Arguments: hashpartitioning(s_store_sk#26, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [s_store_sk#25, sum#28, sum#29] +Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#26, sum#29, sum#30] -Keys [1]: [s_store_sk#26] +Input [3]: [s_store_sk#25, sum#28, sum#29] +Keys [1]: [s_store_sk#25] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] -Results [3]: [s_store_sk#26, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] +Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] (32) BroadcastExchange -Input [3]: [s_store_sk#26, returns#34, profit_loss#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [3]: [s_store_sk#25, returns#33, profit_loss#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] -Right keys [1]: [s_store_sk#26] +Right keys [1]: [s_store_sk#25] Join condition: None (34) Project [codegen id : 8] -Output [5]: [sales#18, coalesce(returns#34, 0.00) AS returns#37, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#38, store channel AS channel#39, s_store_sk#9 AS id#40] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#26, returns#34, profit_loss#35] +Output [5]: [sales#18, coalesce(returns#33, 0.00) AS returns#36, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#37, store channel AS channel#38, s_store_sk#9 AS id#39] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] +Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] +Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#45] +Output [1]: [d_date_sk#6] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#44] -Right keys [1]: [d_date_sk#45] +Left keys [1]: [cs_sold_date_sk#43] +Right keys [1]: [d_date_sk#6] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] -Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] +Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] +Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] -Keys [1]: [cs_call_center_sk#41] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] -Aggregate Attributes [2]: [sum#46, sum#47] -Results [3]: [cs_call_center_sk#41, sum#48, sum#49] +Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] +Keys [1]: [cs_call_center_sk#40] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] +Aggregate Attributes [2]: [sum#44, sum#45] +Results [3]: [cs_call_center_sk#40, sum#46, sum#47] (41) Exchange -Input [3]: [cs_call_center_sk#41, sum#48, sum#49] -Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [3]: [cs_call_center_sk#40, sum#46, sum#47] +Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#41, sum#48, sum#49] -Keys [1]: [cs_call_center_sk#41] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] -Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] +Input [3]: [cs_call_center_sk#40, sum#46, sum#47] +Keys [1]: [cs_call_center_sk#40] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] +Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] (43) BroadcastExchange -Input [3]: [cs_call_center_sk#41, sales#53, profit#54] -Arguments: IdentityBroadcastMode, [id=#55] +Input [3]: [cs_call_center_sk#40, sales#51, profit#52] +Arguments: IdentityBroadcastMode, [id=#53] (44) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] +Output [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#58), dynamicpruningexpression(cr_returned_date_sk#58 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] ReadSchema: struct (45) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] +Input [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] (46) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#59] +Output [1]: [d_date_sk#6] (47) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#58] -Right keys [1]: [d_date_sk#59] +Left keys [1]: [cr_returned_date_sk#56] +Right keys [1]: [d_date_sk#6] Join condition: None (48) Project [codegen id : 13] -Output [2]: [cr_return_amount#56, cr_net_loss#57] -Input [4]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58, d_date_sk#59] +Output [2]: [cr_return_amount#54, cr_net_loss#55] +Input [4]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56, d_date_sk#6] (49) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#56, cr_net_loss#57] +Input [2]: [cr_return_amount#54, cr_net_loss#55] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#56)), partial_sum(UnscaledValue(cr_net_loss#57))] -Aggregate Attributes [2]: [sum#60, sum#61] -Results [2]: [sum#62, sum#63] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#54)), partial_sum(UnscaledValue(cr_net_loss#55))] +Aggregate Attributes [2]: [sum#57, sum#58] +Results [2]: [sum#59, sum#60] (50) Exchange -Input [2]: [sum#62, sum#63] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#64] +Input [2]: [sum#59, sum#60] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] (51) HashAggregate -Input [2]: [sum#62, sum#63] +Input [2]: [sum#59, sum#60] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#56)), sum(UnscaledValue(cr_net_loss#57))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#56))#65, sum(UnscaledValue(cr_net_loss#57))#66] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#56))#65,17,2) AS returns#67, MakeDecimal(sum(UnscaledValue(cr_net_loss#57))#66,17,2) AS profit_loss#68] +Functions [2]: [sum(UnscaledValue(cr_return_amount#54)), sum(UnscaledValue(cr_net_loss#55))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#54))#62, sum(UnscaledValue(cr_net_loss#55))#63] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#54))#62,17,2) AS returns#64, MakeDecimal(sum(UnscaledValue(cr_net_loss#55))#63,17,2) AS profit_loss#65] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [sales#53, returns#67, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#68 as decimal(18,2)))), DecimalType(18,2), true) AS profit#69, catalog channel AS channel#70, cs_call_center_sk#41 AS id#71] -Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#67, profit_loss#68] +Output [5]: [sales#51, returns#64, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#65 as decimal(18,2)))), DecimalType(18,2), true) AS profit#66, catalog channel AS channel#67, cs_call_center_sk#40 AS id#68] +Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#64, profit_loss#65] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] -Condition : isnotnull(ws_web_page_sk#72) +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Condition : isnotnull(ws_web_page_sk#69) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#76] +Output [1]: [d_date_sk#6] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#75] -Right keys [1]: [d_date_sk#76] +Left keys [1]: [ws_sold_date_sk#72] +Right keys [1]: [d_date_sk#6] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] -Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] +Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] +Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#77] +Output [1]: [wp_web_page_sk#73] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#77] +Input [1]: [wp_web_page_sk#73] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#77] -Condition : isnotnull(wp_web_page_sk#77) +Input [1]: [wp_web_page_sk#73] +Condition : isnotnull(wp_web_page_sk#73) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#77] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] +Input [1]: [wp_web_page_sk#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#72] -Right keys [1]: [wp_web_page_sk#77] +Left keys [1]: [ws_web_page_sk#69] +Right keys [1]: [wp_web_page_sk#73] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] -Aggregate Attributes [2]: [sum#79, sum#80] -Results [3]: [wp_web_page_sk#77, sum#81, sum#82] +Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Keys [1]: [wp_web_page_sk#73] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] +Aggregate Attributes [2]: [sum#75, sum#76] +Results [3]: [wp_web_page_sk#73, sum#77, sum#78] (67) Exchange -Input [3]: [wp_web_page_sk#77, sum#81, sum#82] -Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [3]: [wp_web_page_sk#73, sum#77, sum#78] +Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#77, sum#81, sum#82] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] -Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] +Input [3]: [wp_web_page_sk#73, sum#77, sum#78] +Keys [1]: [wp_web_page_sk#73] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] +Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] -Condition : isnotnull(wr_web_page_sk#88) +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Condition : isnotnull(wr_web_page_sk#84) (72) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#92] +Output [1]: [d_date_sk#6] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#91] -Right keys [1]: [cast(d_date_sk#92 as bigint)] +Left keys [1]: [wr_returned_date_sk#87] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90] -Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, d_date_sk#92] +Output [3]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86] +Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, d_date_sk#6] (75) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#93] +Output [1]: [wp_web_page_sk#88] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#88] -Right keys [1]: [cast(wp_web_page_sk#93 as bigint)] +Left keys [1]: [wr_web_page_sk#84] +Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] +Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] -Keys [1]: [wp_web_page_sk#93] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] -Aggregate Attributes [2]: [sum#94, sum#95] -Results [3]: [wp_web_page_sk#93, sum#96, sum#97] +Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Keys [1]: [wp_web_page_sk#88] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] +Aggregate Attributes [2]: [sum#89, sum#90] +Results [3]: [wp_web_page_sk#88, sum#91, sum#92] (79) Exchange -Input [3]: [wp_web_page_sk#93, sum#96, sum#97] -Arguments: hashpartitioning(wp_web_page_sk#93, 5), ENSURE_REQUIREMENTS, [id=#98] +Input [3]: [wp_web_page_sk#88, sum#91, sum#92] +Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#93, sum#96, sum#97] -Keys [1]: [wp_web_page_sk#93] -Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] -Results [3]: [wp_web_page_sk#93, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] +Input [3]: [wp_web_page_sk#88, sum#91, sum#92] +Keys [1]: [wp_web_page_sk#88] +Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] +Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#93, returns#101, profit_loss#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] +Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#77] -Right keys [1]: [wp_web_page_sk#93] +Left keys [1]: [wp_web_page_sk#73] +Right keys [1]: [wp_web_page_sk#88] Join condition: None (83) Project [codegen id : 22] -Output [5]: [sales#86, coalesce(returns#101, 0.00) AS returns#104, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#105, web channel AS channel#106, wp_web_page_sk#77 AS id#107] -Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#93, returns#101, profit_loss#102] +Output [5]: [sales#82, coalesce(returns#96, 0.00) AS returns#99, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#100, web channel AS channel#101, wp_web_page_sk#73 AS id#102] +Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] (84) Union (85) Expand [codegen id : 23] -Input [5]: [sales#18, returns#37, profit#38, channel#39, id#40] -Arguments: [List(sales#18, returns#37, profit#38, channel#39, id#40, 0), List(sales#18, returns#37, profit#38, channel#39, null, 1), List(sales#18, returns#37, profit#38, null, null, 3)], [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] +Input [5]: [sales#18, returns#36, profit#37, channel#38, id#39] +Arguments: [List(sales#18, returns#36, profit#37, channel#38, id#39, 0), List(sales#18, returns#36, profit#37, channel#38, null, 1), List(sales#18, returns#36, profit#37, null, null, 3)], [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] (86) HashAggregate [codegen id : 23] -Input [6]: [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] -Keys [3]: [channel#108, id#109, spark_grouping_id#110] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#37), partial_sum(profit#38)] -Aggregate Attributes [6]: [sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] -Results [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Input [6]: [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] +Keys [3]: [channel#103, id#104, spark_grouping_id#105] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#36), partial_sum(profit#37)] +Aggregate Attributes [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] +Results [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] (87) Exchange -Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Arguments: hashpartitioning(channel#108, id#109, spark_grouping_id#110, 5), ENSURE_REQUIREMENTS, [id=#123] +Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Arguments: hashpartitioning(channel#103, id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, [id=#118] (88) HashAggregate [codegen id : 24] -Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Keys [3]: [channel#108, id#109, spark_grouping_id#110] -Functions [3]: [sum(sales#18), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#18)#124, sum(returns#37)#125, sum(profit#38)#126] -Results [5]: [channel#108, id#109, sum(sales#18)#124 AS sales#127, sum(returns#37)#125 AS returns#128, sum(profit#38)#126 AS profit#129] +Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Keys [3]: [channel#103, id#104, spark_grouping_id#105] +Functions [3]: [sum(sales#18), sum(returns#36), sum(profit#37)] +Aggregate Attributes [3]: [sum(sales#18)#119, sum(returns#36)#120, sum(profit#37)#121] +Results [5]: [channel#103, id#104, sum(sales#18)#119 AS sales#122, sum(returns#36)#120 AS returns#123, sum(profit#37)#121 AS profit#124] (89) TakeOrderedAndProject -Input [5]: [channel#108, id#109, sales#127, returns#128, profit#129] -Arguments: 100, [channel#108 ASC NULLS FIRST, id#109 ASC NULLS FIRST], [channel#108, id#109, sales#127, returns#128, profit#129] +Input [5]: [channel#103, id#104, sales#122, returns#123, profit#124] +Arguments: 100, [channel#103 ASC NULLS FIRST, id#104 ASC NULLS FIRST], [channel#103, id#104, sales#122, returns#123, profit#124] ===== Subqueries ===== @@ -526,14 +526,14 @@ ReusedExchange (91) (91) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#6] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#58 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt index b74247d7aef3f..ec22a1048c116 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt @@ -248,164 +248,164 @@ Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale Input [9]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#14, d_year#15] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#14] Join condition: None (41) Project [codegen id : 13] -Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] -Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#43, d_year#44] +Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] +Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#14, d_year#15] (42) HashAggregate [codegen id : 13] -Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] -Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] +Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [partial_sum(cs_quantity#34), partial_sum(UnscaledValue(cs_wholesale_cost#35)), partial_sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum#45, sum#46, sum#47] -Results [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] +Aggregate Attributes [3]: [sum#43, sum#44, sum#45] +Results [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] (43) Exchange -Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] -Arguments: hashpartitioning(d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] +Arguments: hashpartitioning(d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#49] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] -Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] +Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [sum(cs_quantity#34), sum(UnscaledValue(cs_wholesale_cost#35)), sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum(cs_quantity#34)#52, sum(UnscaledValue(cs_wholesale_cost#35))#53, sum(UnscaledValue(cs_sales_price#36))#54] -Results [6]: [d_year#44 AS cs_sold_year#55, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#56, sum(cs_quantity#34)#52 AS cs_qty#57, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#53,17,2) AS cs_wc#58, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#54,17,2) AS cs_sp#59] +Aggregate Attributes [3]: [sum(cs_quantity#34)#50, sum(UnscaledValue(cs_wholesale_cost#35))#51, sum(UnscaledValue(cs_sales_price#36))#52] +Results [6]: [d_year#15 AS cs_sold_year#53, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#54, sum(cs_quantity#34)#50 AS cs_qty#55, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#51,17,2) AS cs_wc#56, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#52,17,2) AS cs_sp#57] (45) Filter [codegen id : 14] -Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] -Condition : (coalesce(cs_qty#57, 0) > 0) +Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] +Condition : (coalesce(cs_qty#55, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] -Arguments: [cs_sold_year#55 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#56 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] +Arguments: [cs_sold_year#53 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#54 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56] +Right keys [3]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] (49) Scan parquet default.web_sales -Output [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Output [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] (51) Filter [codegen id : 16] -Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] -Condition : (isnotnull(ws_item_sk#60) AND isnotnull(ws_bill_customer_sk#61)) +Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Condition : (isnotnull(ws_item_sk#58) AND isnotnull(ws_bill_customer_sk#59)) (52) Exchange -Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] -Arguments: hashpartitioning(cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint), 5), ENSURE_REQUIREMENTS, [id=#67] +Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Arguments: hashpartitioning(cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint), 5), ENSURE_REQUIREMENTS, [id=#65] (53) Sort [codegen id : 17] -Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] -Arguments: [cast(ws_order_number#62 as bigint) ASC NULLS FIRST, cast(ws_item_sk#60 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Arguments: [cast(ws_order_number#60 as bigint) ASC NULLS FIRST, cast(ws_item_sk#58 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.web_returns -Output [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Output [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] (56) Filter [codegen id : 18] -Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] -Condition : (isnotnull(wr_order_number#69) AND isnotnull(wr_item_sk#68)) +Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Condition : (isnotnull(wr_order_number#67) AND isnotnull(wr_item_sk#66)) (57) Project [codegen id : 18] -Output [2]: [wr_item_sk#68, wr_order_number#69] -Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Output [2]: [wr_item_sk#66, wr_order_number#67] +Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] (58) Exchange -Input [2]: [wr_item_sk#68, wr_order_number#69] -Arguments: hashpartitioning(wr_order_number#69, wr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [2]: [wr_item_sk#66, wr_order_number#67] +Arguments: hashpartitioning(wr_order_number#67, wr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] (59) Sort [codegen id : 19] -Input [2]: [wr_item_sk#68, wr_order_number#69] -Arguments: [wr_order_number#69 ASC NULLS FIRST, wr_item_sk#68 ASC NULLS FIRST], false, 0 +Input [2]: [wr_item_sk#66, wr_order_number#67] +Arguments: [wr_order_number#67 ASC NULLS FIRST, wr_item_sk#66 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint)] -Right keys [2]: [wr_order_number#69, wr_item_sk#68] +Left keys [2]: [cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint)] +Right keys [2]: [wr_order_number#67, wr_item_sk#66] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] -Condition : isnull(wr_order_number#69) +Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] +Condition : isnull(wr_order_number#67) (62) Project [codegen id : 21] -Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] -Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] +Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#72, d_year#73] +Output [2]: [d_date_sk#14, d_year#15] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [ws_sold_date_sk#66] -Right keys [1]: [d_date_sk#72] +Left keys [1]: [ws_sold_date_sk#64] +Right keys [1]: [d_date_sk#14] Join condition: None (65) Project [codegen id : 21] -Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] -Input [8]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, d_date_sk#72, d_year#73] +Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] +Input [8]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, d_date_sk#14, d_year#15] (66) HashAggregate [codegen id : 21] -Input [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] -Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] -Functions [3]: [partial_sum(ws_quantity#63), partial_sum(UnscaledValue(ws_wholesale_cost#64)), partial_sum(UnscaledValue(ws_sales_price#65))] -Aggregate Attributes [3]: [sum#74, sum#75, sum#76] -Results [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] +Input [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] +Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] +Functions [3]: [partial_sum(ws_quantity#61), partial_sum(UnscaledValue(ws_wholesale_cost#62)), partial_sum(UnscaledValue(ws_sales_price#63))] +Aggregate Attributes [3]: [sum#70, sum#71, sum#72] +Results [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] (67) Exchange -Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] -Arguments: hashpartitioning(d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] +Arguments: hashpartitioning(d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#76] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] -Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] -Functions [3]: [sum(ws_quantity#63), sum(UnscaledValue(ws_wholesale_cost#64)), sum(UnscaledValue(ws_sales_price#65))] -Aggregate Attributes [3]: [sum(ws_quantity#63)#81, sum(UnscaledValue(ws_wholesale_cost#64))#82, sum(UnscaledValue(ws_sales_price#65))#83] -Results [6]: [d_year#73 AS ws_sold_year#84, ws_item_sk#60, ws_bill_customer_sk#61 AS ws_customer_sk#85, sum(ws_quantity#63)#81 AS ws_qty#86, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#64))#82,17,2) AS ws_wc#87, MakeDecimal(sum(UnscaledValue(ws_sales_price#65))#83,17,2) AS ws_sp#88] +Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] +Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] +Functions [3]: [sum(ws_quantity#61), sum(UnscaledValue(ws_wholesale_cost#62)), sum(UnscaledValue(ws_sales_price#63))] +Aggregate Attributes [3]: [sum(ws_quantity#61)#77, sum(UnscaledValue(ws_wholesale_cost#62))#78, sum(UnscaledValue(ws_sales_price#63))#79] +Results [6]: [d_year#15 AS ws_sold_year#80, ws_item_sk#58, ws_bill_customer_sk#59 AS ws_customer_sk#81, sum(ws_quantity#61)#77 AS ws_qty#82, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#62))#78,17,2) AS ws_wc#83, MakeDecimal(sum(UnscaledValue(ws_sales_price#63))#79,17,2) AS ws_sp#84] (69) Filter [codegen id : 22] -Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] -Condition : (coalesce(ws_qty#86, 0) > 0) +Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] +Condition : (coalesce(ws_qty#82, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] -Arguments: [ws_sold_year#84 ASC NULLS FIRST, ws_item_sk#60 ASC NULLS FIRST, ws_customer_sk#85 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] +Arguments: [ws_sold_year#80 ASC NULLS FIRST, ws_item_sk#58 ASC NULLS FIRST, ws_customer_sk#81 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85] +Right keys [3]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81] Join condition: None (72) Project [codegen id : 23] -Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#86 + cs_qty#57), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#86, 0) + coalesce(cs_qty#57, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#87, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#58, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#88, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#59, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#86, cs_qty#57] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59, ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] +Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#82 + cs_qty#55), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#82, 0) + coalesce(cs_qty#55, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#83, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#56, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#84, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#57, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#82, cs_qty#55] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57, ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] (73) TakeOrderedAndProject -Input [12]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#86, cs_qty#57] -Arguments: 100, [ratio#89 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#86 + cs_qty#57), 1) as double)), 2) ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] +Input [12]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#82, cs_qty#55] +Arguments: 100, [ratio#85 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#82 + cs_qty#55), 1) as double)), 2) ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt index c7da3a1e23c62..f5eff516f9b36 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt @@ -248,164 +248,164 @@ Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale Input [9]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_order_number#33, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, wr_item_sk#39, wr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#14, d_year#15] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#37] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#14] Join condition: None (41) Project [codegen id : 13] -Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] -Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#43, d_year#44] +Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] +Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#14, d_year#15] (42) HashAggregate [codegen id : 13] -Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] -Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] +Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [partial_sum(ws_quantity#34), partial_sum(UnscaledValue(ws_wholesale_cost#35)), partial_sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum#45, sum#46, sum#47] -Results [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] +Aggregate Attributes [3]: [sum#43, sum#44, sum#45] +Results [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] (43) Exchange -Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] -Arguments: hashpartitioning(d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] +Arguments: hashpartitioning(d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#49] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] -Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] +Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [sum(ws_quantity#34), sum(UnscaledValue(ws_wholesale_cost#35)), sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum(ws_quantity#34)#52, sum(UnscaledValue(ws_wholesale_cost#35))#53, sum(UnscaledValue(ws_sales_price#36))#54] -Results [6]: [d_year#44 AS ws_sold_year#55, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#56, sum(ws_quantity#34)#52 AS ws_qty#57, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#53,17,2) AS ws_wc#58, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#54,17,2) AS ws_sp#59] +Aggregate Attributes [3]: [sum(ws_quantity#34)#50, sum(UnscaledValue(ws_wholesale_cost#35))#51, sum(UnscaledValue(ws_sales_price#36))#52] +Results [6]: [d_year#15 AS ws_sold_year#53, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#54, sum(ws_quantity#34)#50 AS ws_qty#55, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#51,17,2) AS ws_wc#56, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#52,17,2) AS ws_sp#57] (45) Filter [codegen id : 14] -Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] -Condition : (coalesce(ws_qty#57, 0) > 0) +Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] +Condition : (coalesce(ws_qty#55, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] -Arguments: [ws_sold_year#55 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#56 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] +Arguments: [ws_sold_year#53 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#54 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56] +Right keys [3]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] (49) Scan parquet default.catalog_sales -Output [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Output [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#66), dynamicpruningexpression(cs_sold_date_sk#66 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#8)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] (51) Filter [codegen id : 16] -Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] -Condition : (isnotnull(cs_item_sk#61) AND isnotnull(cs_bill_customer_sk#60)) +Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Condition : (isnotnull(cs_item_sk#59) AND isnotnull(cs_bill_customer_sk#58)) (52) Exchange -Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] -Arguments: hashpartitioning(cs_order_number#62, cs_item_sk#61, 5), ENSURE_REQUIREMENTS, [id=#67] +Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Arguments: hashpartitioning(cs_order_number#60, cs_item_sk#59, 5), ENSURE_REQUIREMENTS, [id=#65] (53) Sort [codegen id : 17] -Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] -Arguments: [cs_order_number#62 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST], false, 0 +Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Arguments: [cs_order_number#60 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST], false, 0 (54) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Output [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] (56) Filter [codegen id : 18] -Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] -Condition : (isnotnull(cr_order_number#69) AND isnotnull(cr_item_sk#68)) +Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Condition : (isnotnull(cr_order_number#67) AND isnotnull(cr_item_sk#66)) (57) Project [codegen id : 18] -Output [2]: [cr_item_sk#68, cr_order_number#69] -Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Output [2]: [cr_item_sk#66, cr_order_number#67] +Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] (58) Exchange -Input [2]: [cr_item_sk#68, cr_order_number#69] -Arguments: hashpartitioning(cr_order_number#69, cr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [2]: [cr_item_sk#66, cr_order_number#67] +Arguments: hashpartitioning(cr_order_number#67, cr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] (59) Sort [codegen id : 19] -Input [2]: [cr_item_sk#68, cr_order_number#69] -Arguments: [cr_order_number#69 ASC NULLS FIRST, cr_item_sk#68 ASC NULLS FIRST], false, 0 +Input [2]: [cr_item_sk#66, cr_order_number#67] +Arguments: [cr_order_number#67 ASC NULLS FIRST, cr_item_sk#66 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cs_order_number#62, cs_item_sk#61] -Right keys [2]: [cr_order_number#69, cr_item_sk#68] +Left keys [2]: [cs_order_number#60, cs_item_sk#59] +Right keys [2]: [cr_order_number#67, cr_item_sk#66] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] -Condition : isnull(cr_order_number#69) +Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] +Condition : isnull(cr_order_number#67) (62) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] -Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] +Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#72, d_year#73] +Output [2]: [d_date_sk#14, d_year#15] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [cs_sold_date_sk#66] -Right keys [1]: [d_date_sk#72] +Left keys [1]: [cs_sold_date_sk#64] +Right keys [1]: [d_date_sk#14] Join condition: None (65) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] -Input [8]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, d_date_sk#72, d_year#73] +Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] +Input [8]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, d_date_sk#14, d_year#15] (66) HashAggregate [codegen id : 21] -Input [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] -Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] -Functions [3]: [partial_sum(cs_quantity#63), partial_sum(UnscaledValue(cs_wholesale_cost#64)), partial_sum(UnscaledValue(cs_sales_price#65))] -Aggregate Attributes [3]: [sum#74, sum#75, sum#76] -Results [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] +Input [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] +Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] +Functions [3]: [partial_sum(cs_quantity#61), partial_sum(UnscaledValue(cs_wholesale_cost#62)), partial_sum(UnscaledValue(cs_sales_price#63))] +Aggregate Attributes [3]: [sum#70, sum#71, sum#72] +Results [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] (67) Exchange -Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] -Arguments: hashpartitioning(d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] +Arguments: hashpartitioning(d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, 5), ENSURE_REQUIREMENTS, [id=#76] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] -Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] -Functions [3]: [sum(cs_quantity#63), sum(UnscaledValue(cs_wholesale_cost#64)), sum(UnscaledValue(cs_sales_price#65))] -Aggregate Attributes [3]: [sum(cs_quantity#63)#81, sum(UnscaledValue(cs_wholesale_cost#64))#82, sum(UnscaledValue(cs_sales_price#65))#83] -Results [6]: [d_year#73 AS cs_sold_year#84, cs_item_sk#61, cs_bill_customer_sk#60 AS cs_customer_sk#85, sum(cs_quantity#63)#81 AS cs_qty#86, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#64))#82,17,2) AS cs_wc#87, MakeDecimal(sum(UnscaledValue(cs_sales_price#65))#83,17,2) AS cs_sp#88] +Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] +Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] +Functions [3]: [sum(cs_quantity#61), sum(UnscaledValue(cs_wholesale_cost#62)), sum(UnscaledValue(cs_sales_price#63))] +Aggregate Attributes [3]: [sum(cs_quantity#61)#77, sum(UnscaledValue(cs_wholesale_cost#62))#78, sum(UnscaledValue(cs_sales_price#63))#79] +Results [6]: [d_year#15 AS cs_sold_year#80, cs_item_sk#59, cs_bill_customer_sk#58 AS cs_customer_sk#81, sum(cs_quantity#61)#77 AS cs_qty#82, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#62))#78,17,2) AS cs_wc#83, MakeDecimal(sum(UnscaledValue(cs_sales_price#63))#79,17,2) AS cs_sp#84] (69) Filter [codegen id : 22] -Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] -Condition : (coalesce(cs_qty#86, 0) > 0) +Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] +Condition : (coalesce(cs_qty#82, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] -Arguments: [cs_sold_year#84 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST, cs_customer_sk#85 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] +Arguments: [cs_sold_year#80 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST, cs_customer_sk#81 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85] +Right keys [3]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81] Join condition: None (72) Project [codegen id : 23] -Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#57 + cs_qty#86), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#57, 0) + coalesce(cs_qty#86, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#58, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#87, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#59, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#88, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, cs_qty#86] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59, cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] +Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#55 + cs_qty#82), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#55, 0) + coalesce(cs_qty#82, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#56, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#83, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#57, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#84, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, cs_qty#82] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57, cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] (73) TakeOrderedAndProject -Input [12]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, cs_qty#86] -Arguments: 100, [ratio#89 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#57 + cs_qty#86), 1) as double)), 2) ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] +Input [12]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, cs_qty#82] +Arguments: 100, [ratio#85 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#55 + cs_qty#82), 1) as double)), 2) ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt index d13b9623d1b34..48521c72281f4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt @@ -149,157 +149,157 @@ Input [1]: [ca_zip#14] Condition : (substr(ca_zip#14, 1, 5) INSET (56910,69952,63792,39371,74351,11101,25003,97189,57834,73134,62377,51200,32754,22752,86379,14171,91110,40162,98569,28709,13394,66162,25733,25782,26065,18383,51949,87343,50298,83849,33786,64528,23470,67030,46136,25280,46820,77721,99076,18426,31880,17871,98235,45748,49156,18652,72013,51622,43848,78567,41248,13695,44165,67853,54917,53179,64034,10567,71791,68908,55565,59402,64147,85816,57855,61547,27700,68100,28810,58263,15723,83933,51103,58058,90578,82276,81096,81426,96451,77556,38607,76638,18906,62971,57047,48425,35576,11928,30625,83444,73520,51650,57647,60099,30122,94983,24128,10445,41368,26233,26859,21756,24676,19849,36420,38193,58470,39127,13595,87501,24317,15455,69399,98025,81019,48033,11376,39516,67875,92712,14867,38122,29741,42961,30469,51211,56458,15559,16021,33123,33282,33515,72823,54601,76698,56240,72175,60279,20004,68806,72325,28488,43933,50412,45200,22246,78668,79777,96765,67301,73273,49448,82636,23932,47305,29839,39192,18799,61265,37125,58943,64457,88424,24610,84935,89360,68893,30431,28898,10336,90257,59166,46081,26105,96888,36634,86284,35258,39972,22927,73241,53268,24206,27385,99543,31671,14663,30903,39861,24996,63089,88086,83921,21076,67897,66708,45721,60576,25103,52867,30450,36233,30010,96576,73171,56571,56575,64544,13955,78451,43285,18119,16725,83041,76107,79994,54364,35942,56691,19769,63435,34102,18845,22744,13354,75691,45549,23968,31387,83144,13375,15765,28577,88190,19736,73650,37930,25989,83926,94898,51798,39736,22437,55253,38415,71256,18376,42029,25858,44438,19515,38935,51649,71954,15882,18767,63193,25486,49130,37126,40604,34425,17043,12305,11634,26653,94167,36446,10516,67473,66864,72425,63981,18842,22461,42666,47770,69035,70372,28587,45266,15371,15798,45375,90225,16807,31016,68014,21337,19505,50016,10144,84093,21286,19430,34322,91068,94945,72305,24671,58048,65084,28545,21195,20548,22245,77191,96976,48583,76231,15734,61810,11356,68621,68786,98359,41367,26689,69913,76614,68101,88885,50308,79077,18270,28915,29178,53672,62878,10390,14922,68341,56529,41766,68309,56616,15126,61860,97789,11489,45692,41918,72151,72550,27156,36495,70738,17879,53535,17920,68880,78890,35850,14089,58078,65164,27068,26231,13376,57665,32213,77610,87816,21309,15146,86198,91137,55307,67467,40558,94627,82136,22351,89091,20260,23006,91393,47537,62496,98294,18840,71286,81312,31029,70466,35458,14060,22685,28286,25631,19512,40081,63837,14328,35474,22152,76232,51061,86057,17183) AND isnotnull(substr(ca_zip#14, 1, 5))) (22) Scan parquet default.customer_address -Output [2]: [ca_address_sk#15, ca_zip#16] +Output [2]: [ca_address_sk#15, ca_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 5] -Input [2]: [ca_address_sk#15, ca_zip#16] +Input [2]: [ca_address_sk#15, ca_zip#14] (24) Filter [codegen id : 5] -Input [2]: [ca_address_sk#15, ca_zip#16] +Input [2]: [ca_address_sk#15, ca_zip#14] Condition : isnotnull(ca_address_sk#15) (25) Exchange -Input [2]: [ca_address_sk#15, ca_zip#16] -Arguments: hashpartitioning(ca_address_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [2]: [ca_address_sk#15, ca_zip#14] +Arguments: hashpartitioning(ca_address_sk#15, 5), ENSURE_REQUIREMENTS, [id=#16] (26) Sort [codegen id : 6] -Input [2]: [ca_address_sk#15, ca_zip#16] +Input [2]: [ca_address_sk#15, ca_zip#14] Arguments: [ca_address_sk#15 ASC NULLS FIRST], false, 0 (27) Scan parquet default.customer -Output [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] +Output [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 7] -Input [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] +Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] (29) Filter [codegen id : 7] -Input [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] -Condition : ((isnotnull(c_preferred_cust_flag#19) AND (c_preferred_cust_flag#19 = Y)) AND isnotnull(c_current_addr_sk#18)) +Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] +Condition : ((isnotnull(c_preferred_cust_flag#18) AND (c_preferred_cust_flag#18 = Y)) AND isnotnull(c_current_addr_sk#17)) (30) Project [codegen id : 7] -Output [1]: [c_current_addr_sk#18] -Input [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] +Output [1]: [c_current_addr_sk#17] +Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] (31) Exchange -Input [1]: [c_current_addr_sk#18] -Arguments: hashpartitioning(c_current_addr_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [c_current_addr_sk#17] +Arguments: hashpartitioning(c_current_addr_sk#17, 5), ENSURE_REQUIREMENTS, [id=#19] (32) Sort [codegen id : 8] -Input [1]: [c_current_addr_sk#18] -Arguments: [c_current_addr_sk#18 ASC NULLS FIRST], false, 0 +Input [1]: [c_current_addr_sk#17] +Arguments: [c_current_addr_sk#17 ASC NULLS FIRST], false, 0 (33) SortMergeJoin [codegen id : 9] Left keys [1]: [ca_address_sk#15] -Right keys [1]: [c_current_addr_sk#18] +Right keys [1]: [c_current_addr_sk#17] Join condition: None (34) Project [codegen id : 9] -Output [1]: [ca_zip#16] -Input [3]: [ca_address_sk#15, ca_zip#16, c_current_addr_sk#18] +Output [1]: [ca_zip#14] +Input [3]: [ca_address_sk#15, ca_zip#14, c_current_addr_sk#17] (35) HashAggregate [codegen id : 9] -Input [1]: [ca_zip#16] -Keys [1]: [ca_zip#16] +Input [1]: [ca_zip#14] +Keys [1]: [ca_zip#14] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#21] -Results [2]: [ca_zip#16, count#22] +Aggregate Attributes [1]: [count#20] +Results [2]: [ca_zip#14, count#21] (36) Exchange -Input [2]: [ca_zip#16, count#22] -Arguments: hashpartitioning(ca_zip#16, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [2]: [ca_zip#14, count#21] +Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, [id=#22] (37) HashAggregate [codegen id : 10] -Input [2]: [ca_zip#16, count#22] -Keys [1]: [ca_zip#16] +Input [2]: [ca_zip#14, count#21] +Keys [1]: [ca_zip#14] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#24] -Results [2]: [substr(ca_zip#16, 1, 5) AS ca_zip#25, count(1)#24 AS count(1)#26] +Aggregate Attributes [1]: [count(1)#23] +Results [2]: [substr(ca_zip#14, 1, 5) AS ca_zip#24, count(1)#23 AS count(1)#25] (38) Filter [codegen id : 10] -Input [2]: [ca_zip#25, count(1)#26] -Condition : (count(1)#26 > 10) +Input [2]: [ca_zip#24, count(1)#25] +Condition : (count(1)#25 > 10) (39) Project [codegen id : 10] -Output [1]: [ca_zip#25] -Input [2]: [ca_zip#25, count(1)#26] +Output [1]: [ca_zip#24] +Input [2]: [ca_zip#24, count(1)#25] (40) BroadcastExchange -Input [1]: [ca_zip#25] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#27] +Input [1]: [ca_zip#24] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#26] (41) BroadcastHashJoin [codegen id : 11] Left keys [2]: [coalesce(substr(ca_zip#14, 1, 5), ), isnull(substr(ca_zip#14, 1, 5))] -Right keys [2]: [coalesce(ca_zip#25, ), isnull(ca_zip#25)] +Right keys [2]: [coalesce(ca_zip#24, ), isnull(ca_zip#24)] Join condition: None (42) Project [codegen id : 11] -Output [1]: [substr(ca_zip#14, 1, 5) AS ca_zip#28] +Output [1]: [substr(ca_zip#14, 1, 5) AS ca_zip#27] Input [1]: [ca_zip#14] (43) HashAggregate [codegen id : 11] -Input [1]: [ca_zip#28] -Keys [1]: [ca_zip#28] +Input [1]: [ca_zip#27] +Keys [1]: [ca_zip#27] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#28] +Results [1]: [ca_zip#27] (44) Exchange -Input [1]: [ca_zip#28] -Arguments: hashpartitioning(ca_zip#28, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [1]: [ca_zip#27] +Arguments: hashpartitioning(ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#28] (45) HashAggregate [codegen id : 12] -Input [1]: [ca_zip#28] -Keys [1]: [ca_zip#28] +Input [1]: [ca_zip#27] +Keys [1]: [ca_zip#27] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#28] +Results [1]: [ca_zip#27] (46) Exchange -Input [1]: [ca_zip#28] -Arguments: hashpartitioning(substr(ca_zip#28, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#30] +Input [1]: [ca_zip#27] +Arguments: hashpartitioning(substr(ca_zip#27, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#29] (47) Sort [codegen id : 13] -Input [1]: [ca_zip#28] -Arguments: [substr(ca_zip#28, 1, 2) ASC NULLS FIRST], false, 0 +Input [1]: [ca_zip#27] +Arguments: [substr(ca_zip#27, 1, 2) ASC NULLS FIRST], false, 0 (48) SortMergeJoin [codegen id : 14] Left keys [1]: [substr(s_zip#11, 1, 2)] -Right keys [1]: [substr(ca_zip#28, 1, 2)] +Right keys [1]: [substr(ca_zip#27, 1, 2)] Join condition: None (49) Project [codegen id : 14] Output [2]: [ss_net_profit#2, s_store_name#10] -Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#28] +Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#27] (50) HashAggregate [codegen id : 14] Input [2]: [ss_net_profit#2, s_store_name#10] Keys [1]: [s_store_name#10] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#31] -Results [2]: [s_store_name#10, sum#32] +Aggregate Attributes [1]: [sum#30] +Results [2]: [s_store_name#10, sum#31] (51) Exchange -Input [2]: [s_store_name#10, sum#32] -Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [2]: [s_store_name#10, sum#31] +Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#32] (52) HashAggregate [codegen id : 15] -Input [2]: [s_store_name#10, sum#32] +Input [2]: [s_store_name#10, sum#31] Keys [1]: [s_store_name#10] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#34] -Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS sum(ss_net_profit)#35] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#33] +Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#33,17,2) AS sum(ss_net_profit)#34] (53) TakeOrderedAndProject -Input [2]: [s_store_name#10, sum(ss_net_profit)#35] -Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#35] +Input [2]: [s_store_name#10, sum(ss_net_profit)#34] +Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#34] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt index dbb5e1f606a8d..09e1189003ed6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt @@ -135,141 +135,141 @@ Input [1]: [ca_zip#13] Condition : (substr(ca_zip#13, 1, 5) INSET (56910,69952,63792,39371,74351,11101,25003,97189,57834,73134,62377,51200,32754,22752,86379,14171,91110,40162,98569,28709,13394,66162,25733,25782,26065,18383,51949,87343,50298,83849,33786,64528,23470,67030,46136,25280,46820,77721,99076,18426,31880,17871,98235,45748,49156,18652,72013,51622,43848,78567,41248,13695,44165,67853,54917,53179,64034,10567,71791,68908,55565,59402,64147,85816,57855,61547,27700,68100,28810,58263,15723,83933,51103,58058,90578,82276,81096,81426,96451,77556,38607,76638,18906,62971,57047,48425,35576,11928,30625,83444,73520,51650,57647,60099,30122,94983,24128,10445,41368,26233,26859,21756,24676,19849,36420,38193,58470,39127,13595,87501,24317,15455,69399,98025,81019,48033,11376,39516,67875,92712,14867,38122,29741,42961,30469,51211,56458,15559,16021,33123,33282,33515,72823,54601,76698,56240,72175,60279,20004,68806,72325,28488,43933,50412,45200,22246,78668,79777,96765,67301,73273,49448,82636,23932,47305,29839,39192,18799,61265,37125,58943,64457,88424,24610,84935,89360,68893,30431,28898,10336,90257,59166,46081,26105,96888,36634,86284,35258,39972,22927,73241,53268,24206,27385,99543,31671,14663,30903,39861,24996,63089,88086,83921,21076,67897,66708,45721,60576,25103,52867,30450,36233,30010,96576,73171,56571,56575,64544,13955,78451,43285,18119,16725,83041,76107,79994,54364,35942,56691,19769,63435,34102,18845,22744,13354,75691,45549,23968,31387,83144,13375,15765,28577,88190,19736,73650,37930,25989,83926,94898,51798,39736,22437,55253,38415,71256,18376,42029,25858,44438,19515,38935,51649,71954,15882,18767,63193,25486,49130,37126,40604,34425,17043,12305,11634,26653,94167,36446,10516,67473,66864,72425,63981,18842,22461,42666,47770,69035,70372,28587,45266,15371,15798,45375,90225,16807,31016,68014,21337,19505,50016,10144,84093,21286,19430,34322,91068,94945,72305,24671,58048,65084,28545,21195,20548,22245,77191,96976,48583,76231,15734,61810,11356,68621,68786,98359,41367,26689,69913,76614,68101,88885,50308,79077,18270,28915,29178,53672,62878,10390,14922,68341,56529,41766,68309,56616,15126,61860,97789,11489,45692,41918,72151,72550,27156,36495,70738,17879,53535,17920,68880,78890,35850,14089,58078,65164,27068,26231,13376,57665,32213,77610,87816,21309,15146,86198,91137,55307,67467,40558,94627,82136,22351,89091,20260,23006,91393,47537,62496,98294,18840,71286,81312,31029,70466,35458,14060,22685,28286,25631,19512,40081,63837,14328,35474,22152,76232,51061,86057,17183) AND isnotnull(substr(ca_zip#13, 1, 5))) (20) Scan parquet default.customer_address -Output [2]: [ca_address_sk#14, ca_zip#15] +Output [2]: [ca_address_sk#14, ca_zip#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#14, ca_zip#15] +Input [2]: [ca_address_sk#14, ca_zip#13] (22) Filter [codegen id : 4] -Input [2]: [ca_address_sk#14, ca_zip#15] +Input [2]: [ca_address_sk#14, ca_zip#13] Condition : isnotnull(ca_address_sk#14) (23) Scan parquet default.customer -Output [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Output [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 3] -Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] (25) Filter [codegen id : 3] -Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] -Condition : ((isnotnull(c_preferred_cust_flag#17) AND (c_preferred_cust_flag#17 = Y)) AND isnotnull(c_current_addr_sk#16)) +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Condition : ((isnotnull(c_preferred_cust_flag#16) AND (c_preferred_cust_flag#16 = Y)) AND isnotnull(c_current_addr_sk#15)) (26) Project [codegen id : 3] -Output [1]: [c_current_addr_sk#16] -Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Output [1]: [c_current_addr_sk#15] +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] (27) BroadcastExchange -Input [1]: [c_current_addr_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Input [1]: [c_current_addr_sk#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (28) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ca_address_sk#14] -Right keys [1]: [c_current_addr_sk#16] +Right keys [1]: [c_current_addr_sk#15] Join condition: None (29) Project [codegen id : 4] -Output [1]: [ca_zip#15] -Input [3]: [ca_address_sk#14, ca_zip#15, c_current_addr_sk#16] +Output [1]: [ca_zip#13] +Input [3]: [ca_address_sk#14, ca_zip#13, c_current_addr_sk#15] (30) HashAggregate [codegen id : 4] -Input [1]: [ca_zip#15] -Keys [1]: [ca_zip#15] +Input [1]: [ca_zip#13] +Keys [1]: [ca_zip#13] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#19] -Results [2]: [ca_zip#15, count#20] +Aggregate Attributes [1]: [count#18] +Results [2]: [ca_zip#13, count#19] (31) Exchange -Input [2]: [ca_zip#15, count#20] -Arguments: hashpartitioning(ca_zip#15, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [2]: [ca_zip#13, count#19] +Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, [id=#20] (32) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#15, count#20] -Keys [1]: [ca_zip#15] +Input [2]: [ca_zip#13, count#19] +Keys [1]: [ca_zip#13] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#22] -Results [2]: [substr(ca_zip#15, 1, 5) AS ca_zip#23, count(1)#22 AS count(1)#24] +Aggregate Attributes [1]: [count(1)#21] +Results [2]: [substr(ca_zip#13, 1, 5) AS ca_zip#22, count(1)#21 AS count(1)#23] (33) Filter [codegen id : 5] -Input [2]: [ca_zip#23, count(1)#24] -Condition : (count(1)#24 > 10) +Input [2]: [ca_zip#22, count(1)#23] +Condition : (count(1)#23 > 10) (34) Project [codegen id : 5] -Output [1]: [ca_zip#23] -Input [2]: [ca_zip#23, count(1)#24] +Output [1]: [ca_zip#22] +Input [2]: [ca_zip#22, count(1)#23] (35) BroadcastExchange -Input [1]: [ca_zip#23] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#25] +Input [1]: [ca_zip#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#24] (36) BroadcastHashJoin [codegen id : 6] Left keys [2]: [coalesce(substr(ca_zip#13, 1, 5), ), isnull(substr(ca_zip#13, 1, 5))] -Right keys [2]: [coalesce(ca_zip#23, ), isnull(ca_zip#23)] +Right keys [2]: [coalesce(ca_zip#22, ), isnull(ca_zip#22)] Join condition: None (37) Project [codegen id : 6] -Output [1]: [substr(ca_zip#13, 1, 5) AS ca_zip#26] +Output [1]: [substr(ca_zip#13, 1, 5) AS ca_zip#25] Input [1]: [ca_zip#13] (38) HashAggregate [codegen id : 6] -Input [1]: [ca_zip#26] -Keys [1]: [ca_zip#26] +Input [1]: [ca_zip#25] +Keys [1]: [ca_zip#25] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#26] +Results [1]: [ca_zip#25] (39) Exchange -Input [1]: [ca_zip#26] -Arguments: hashpartitioning(ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [1]: [ca_zip#25] +Arguments: hashpartitioning(ca_zip#25, 5), ENSURE_REQUIREMENTS, [id=#26] (40) HashAggregate [codegen id : 7] -Input [1]: [ca_zip#26] -Keys [1]: [ca_zip#26] +Input [1]: [ca_zip#25] +Keys [1]: [ca_zip#25] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#26] +Results [1]: [ca_zip#25] (41) BroadcastExchange -Input [1]: [ca_zip#26] -Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [id=#28] +Input [1]: [ca_zip#25] +Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [id=#27] (42) BroadcastHashJoin [codegen id : 8] Left keys [1]: [substr(s_zip#11, 1, 2)] -Right keys [1]: [substr(ca_zip#26, 1, 2)] +Right keys [1]: [substr(ca_zip#25, 1, 2)] Join condition: None (43) Project [codegen id : 8] Output [2]: [ss_net_profit#2, s_store_name#10] -Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#26] +Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#25] (44) HashAggregate [codegen id : 8] Input [2]: [ss_net_profit#2, s_store_name#10] Keys [1]: [s_store_name#10] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#29] -Results [2]: [s_store_name#10, sum#30] +Aggregate Attributes [1]: [sum#28] +Results [2]: [s_store_name#10, sum#29] (45) Exchange -Input [2]: [s_store_name#10, sum#30] -Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [2]: [s_store_name#10, sum#29] +Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#30] (46) HashAggregate [codegen id : 9] -Input [2]: [s_store_name#10, sum#30] +Input [2]: [s_store_name#10, sum#29] Keys [1]: [s_store_name#10] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#32] -Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#32,17,2) AS sum(ss_net_profit)#33] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] +Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS sum(ss_net_profit)#32] (47) TakeOrderedAndProject -Input [2]: [s_store_name#10, sum(ss_net_profit)#33] -Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#33] +Input [2]: [s_store_name#10, sum(ss_net_profit)#32] +Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#32] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt index bdb1a52a18f2d..6e86260f0aba8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt @@ -244,7 +244,7 @@ Input [2]: [d_date_sk#22, d_date#23] (30) Filter [codegen id : 7] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-22)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11192)) AND (d_date#23 <= 11222)) AND isnotnull(d_date_sk#22)) (31) Project [codegen id : 7] Output [1]: [d_date_sk#22] @@ -367,252 +367,252 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#61] +Output [1]: [i_item_sk#16] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#61] +Right keys [1]: [i_item_sk#16] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#61] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#16] (60) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#62] +Output [1]: [p_promo_sk#19] (61) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#62] +Right keys [1]: [p_promo_sk#19] Join condition: None (62) Project [codegen id : 19] Output [6]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#62] +Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#19] (63) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#63] +Output [1]: [d_date_sk#22] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#63] +Right keys [1]: [d_date_sk#22] Join condition: None (65) Project [codegen id : 19] Output [5]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#63] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#22] (66) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 18] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] (68) Filter [codegen id : 18] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Condition : isnotnull(cp_catalog_page_sk#64) +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Condition : isnotnull(cp_catalog_page_sk#61) (69) BroadcastExchange -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#64] +Right keys [1]: [cp_catalog_page_sk#61] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#64, cp_catalog_page_id#65] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] -Keys [1]: [cp_catalog_page_id#65] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Keys [1]: [cp_catalog_page_id#62] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Results [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] (73) Exchange -Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] -Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] -Keys [1]: [cp_catalog_page_id#65] +Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Keys [1]: [cp_catalog_page_id#62] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#81, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#82, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#83, catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#65) AS id#85] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] +Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#80, catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#62) AS id#82] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) (78) Exchange -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] (84) Exchange -Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] +Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] -Right keys [2]: [wr_item_sk#94, wr_order_number#95] +Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] +Right keys [2]: [wr_item_sk#91, wr_order_number#92] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] -Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] +Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#100] +Output [1]: [i_item_sk#16] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#86] -Right keys [1]: [i_item_sk#100] +Left keys [1]: [ws_item_sk#83] +Right keys [1]: [i_item_sk#16] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] -Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, i_item_sk#100] +Output [7]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] +Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, i_item_sk#16] (91) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#101] +Output [1]: [p_promo_sk#19] (92) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#88] -Right keys [1]: [p_promo_sk#101] +Left keys [1]: [ws_promo_sk#85] +Right keys [1]: [p_promo_sk#19] Join condition: None (93) Project [codegen id : 29] -Output [6]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] -Input [8]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, p_promo_sk#101] +Output [6]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] +Input [8]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, p_promo_sk#19] (94) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#102] +Output [1]: [d_date_sk#22] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#92] -Right keys [1]: [d_date_sk#102] +Left keys [1]: [ws_sold_date_sk#89] +Right keys [1]: [d_date_sk#22] Join condition: None (96) Project [codegen id : 29] -Output [5]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] -Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#102] +Output [5]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] +Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#22] (97) Scan parquet default.web_site -Output [2]: [web_site_sk#103, web_site_id#104] +Output [2]: [web_site_sk#97, web_site_id#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (98) ColumnarToRow [codegen id : 28] -Input [2]: [web_site_sk#103, web_site_id#104] +Input [2]: [web_site_sk#97, web_site_id#98] (99) Filter [codegen id : 28] -Input [2]: [web_site_sk#103, web_site_id#104] -Condition : isnotnull(web_site_sk#103) +Input [2]: [web_site_sk#97, web_site_id#98] +Condition : isnotnull(web_site_sk#97) (100) BroadcastExchange -Input [2]: [web_site_sk#103, web_site_id#104] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#105] +Input [2]: [web_site_sk#97, web_site_id#98] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#87] -Right keys [1]: [web_site_sk#103] +Left keys [1]: [ws_web_site_sk#84] +Right keys [1]: [web_site_sk#97] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] -Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#103, web_site_id#104] +Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] -Keys [1]: [web_site_id#104] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Results [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Keys [1]: [web_site_id#98] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] (104) Exchange -Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, [id=#116] +Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Keys [1]: [web_site_id#104] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#120, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#121, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#122, web channel AS channel#123, concat(web_site, web_site_id#104) AS id#124] +Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Keys [1]: [web_site_id#98] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] +Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#114, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#115, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#116, web channel AS channel#117, concat(web_site, web_site_id#98) AS id#118] (106) Union (107) Expand [codegen id : 31] Input [5]: [sales#42, returns#43, profit#44, channel#45, id#46] -Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] +Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] (108) HashAggregate [codegen id : 31] -Input [6]: [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] -Keys [3]: [channel#125, id#126, spark_grouping_id#127] +Input [6]: [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] +Keys [3]: [channel#119, id#120, spark_grouping_id#121] Functions [3]: [partial_sum(sales#42), partial_sum(returns#43), partial_sum(profit#44)] -Aggregate Attributes [6]: [sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] -Results [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Aggregate Attributes [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] +Results [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] (109) Exchange -Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Arguments: hashpartitioning(channel#125, id#126, spark_grouping_id#127, 5), ENSURE_REQUIREMENTS, [id=#140] +Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Arguments: hashpartitioning(channel#119, id#120, spark_grouping_id#121, 5), ENSURE_REQUIREMENTS, [id=#134] (110) HashAggregate [codegen id : 32] -Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Keys [3]: [channel#125, id#126, spark_grouping_id#127] +Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Keys [3]: [channel#119, id#120, spark_grouping_id#121] Functions [3]: [sum(sales#42), sum(returns#43), sum(profit#44)] -Aggregate Attributes [3]: [sum(sales#42)#141, sum(returns#43)#142, sum(profit#44)#143] -Results [5]: [channel#125, id#126, sum(sales#42)#141 AS sales#144, sum(returns#43)#142 AS returns#145, sum(profit#44)#143 AS profit#146] +Aggregate Attributes [3]: [sum(sales#42)#135, sum(returns#43)#136, sum(profit#44)#137] +Results [5]: [channel#119, id#120, sum(sales#42)#135 AS sales#138, sum(returns#43)#136 AS returns#139, sum(profit#44)#137 AS profit#140] (111) TakeOrderedAndProject -Input [5]: [channel#125, id#126, sales#144, returns#145, profit#146] -Arguments: 100, [channel#125 ASC NULLS FIRST, id#126 ASC NULLS FIRST], [channel#125, id#126, sales#144, returns#145, profit#146] +Input [5]: [channel#119, id#120, sales#138, returns#139, profit#140] +Arguments: 100, [channel#119 ASC NULLS FIRST, id#120 ASC NULLS FIRST], [channel#119, id#120, sales#138, returns#139, profit#140] ===== Subqueries ===== @@ -625,6 +625,6 @@ Output [1]: [d_date_sk#22] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt index aa15d27d4e562..87abdc02b21a7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt @@ -182,7 +182,7 @@ Input [2]: [d_date_sk#16, d_date#17] (16) Filter [codegen id : 5] Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-08-23)) AND (d_date#17 <= 2000-09-22)) AND isnotnull(d_date_sk#16)) +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 11192)) AND (d_date#17 <= 11222)) AND isnotnull(d_date_sk#16)) (17) Project [codegen id : 5] Output [1]: [d_date_sk#16] @@ -367,252 +367,252 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#61] +Output [1]: [d_date_sk#16] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#61] +Right keys [1]: [d_date_sk#16] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#61] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#16] (60) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] (62) Filter [codegen id : 16] -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Condition : isnotnull(cp_catalog_page_sk#61) (63) BroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#62] +Right keys [1]: [cp_catalog_page_sk#61] Join condition: None (65) Project [codegen id : 19] -Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#62, cp_catalog_page_id#63] +Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] (66) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#65] +Output [1]: [i_item_sk#22] (67) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#65] +Right keys [1]: [i_item_sk#22] Join condition: None (68) Project [codegen id : 19] -Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] -Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, i_item_sk#65] +Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, i_item_sk#22] (69) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#66] +Output [1]: [p_promo_sk#25] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#66] +Right keys [1]: [p_promo_sk#25] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] -Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, p_promo_sk#66] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, p_promo_sk#25] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] -Keys [1]: [cp_catalog_page_id#63] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Keys [1]: [cp_catalog_page_id#62] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Results [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] (73) Exchange -Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] -Arguments: hashpartitioning(cp_catalog_page_id#63, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] -Keys [1]: [cp_catalog_page_id#63] +Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Keys [1]: [cp_catalog_page_id#62] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#81, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#82, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#83, catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#63) AS id#85] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] +Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#80, catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#62) AS id#82] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) (78) Exchange -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] (84) Exchange -Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] +Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] -Right keys [2]: [wr_item_sk#94, wr_order_number#95] +Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] +Right keys [2]: [wr_item_sk#91, wr_order_number#92] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] -Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] +Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#100] +Output [1]: [d_date_sk#16] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#92] -Right keys [1]: [d_date_sk#100] +Left keys [1]: [ws_sold_date_sk#89] +Right keys [1]: [d_date_sk#16] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] -Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#100] +Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] +Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#16] (91) Scan parquet default.web_site -Output [2]: [web_site_sk#101, web_site_id#102] +Output [2]: [web_site_sk#97, web_site_id#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 26] -Input [2]: [web_site_sk#101, web_site_id#102] +Input [2]: [web_site_sk#97, web_site_id#98] (93) Filter [codegen id : 26] -Input [2]: [web_site_sk#101, web_site_id#102] -Condition : isnotnull(web_site_sk#101) +Input [2]: [web_site_sk#97, web_site_id#98] +Condition : isnotnull(web_site_sk#97) (94) BroadcastExchange -Input [2]: [web_site_sk#101, web_site_id#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] +Input [2]: [web_site_sk#97, web_site_id#98] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#87] -Right keys [1]: [web_site_sk#101] +Left keys [1]: [ws_web_site_sk#84] +Right keys [1]: [web_site_sk#97] Join condition: None (96) Project [codegen id : 29] -Output [7]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] -Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#101, web_site_id#102] +Output [7]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] (97) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#104] +Output [1]: [i_item_sk#22] (98) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#86] -Right keys [1]: [i_item_sk#104] +Left keys [1]: [ws_item_sk#83] +Right keys [1]: [i_item_sk#22] Join condition: None (99) Project [codegen id : 29] -Output [6]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] -Input [8]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, i_item_sk#104] +Output [6]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Input [8]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, i_item_sk#22] (100) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#105] +Output [1]: [p_promo_sk#25] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#88] -Right keys [1]: [p_promo_sk#105] +Left keys [1]: [ws_promo_sk#85] +Right keys [1]: [p_promo_sk#25] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] -Input [7]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, p_promo_sk#105] +Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Input [7]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, p_promo_sk#25] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] -Keys [1]: [web_site_id#102] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Results [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Keys [1]: [web_site_id#98] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] (104) Exchange -Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Arguments: hashpartitioning(web_site_id#102, 5), ENSURE_REQUIREMENTS, [id=#116] +Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Keys [1]: [web_site_id#102] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#120, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#121, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#122, web channel AS channel#123, concat(web_site, web_site_id#102) AS id#124] +Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Keys [1]: [web_site_id#98] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] +Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#114, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#115, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#116, web channel AS channel#117, concat(web_site, web_site_id#98) AS id#118] (106) Union (107) Expand [codegen id : 31] Input [5]: [sales#42, returns#43, profit#44, channel#45, id#46] -Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] +Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] (108) HashAggregate [codegen id : 31] -Input [6]: [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] -Keys [3]: [channel#125, id#126, spark_grouping_id#127] +Input [6]: [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] +Keys [3]: [channel#119, id#120, spark_grouping_id#121] Functions [3]: [partial_sum(sales#42), partial_sum(returns#43), partial_sum(profit#44)] -Aggregate Attributes [6]: [sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] -Results [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Aggregate Attributes [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] +Results [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] (109) Exchange -Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Arguments: hashpartitioning(channel#125, id#126, spark_grouping_id#127, 5), ENSURE_REQUIREMENTS, [id=#140] +Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Arguments: hashpartitioning(channel#119, id#120, spark_grouping_id#121, 5), ENSURE_REQUIREMENTS, [id=#134] (110) HashAggregate [codegen id : 32] -Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Keys [3]: [channel#125, id#126, spark_grouping_id#127] +Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Keys [3]: [channel#119, id#120, spark_grouping_id#121] Functions [3]: [sum(sales#42), sum(returns#43), sum(profit#44)] -Aggregate Attributes [3]: [sum(sales#42)#141, sum(returns#43)#142, sum(profit#44)#143] -Results [5]: [channel#125, id#126, sum(sales#42)#141 AS sales#144, sum(returns#43)#142 AS returns#145, sum(profit#44)#143 AS profit#146] +Aggregate Attributes [3]: [sum(sales#42)#135, sum(returns#43)#136, sum(profit#44)#137] +Results [5]: [channel#119, id#120, sum(sales#42)#135 AS sales#138, sum(returns#43)#136 AS returns#139, sum(profit#44)#137 AS profit#140] (111) TakeOrderedAndProject -Input [5]: [channel#125, id#126, sales#144, returns#145, profit#146] -Arguments: 100, [channel#125 ASC NULLS FIRST, id#126 ASC NULLS FIRST], [channel#125, id#126, sales#144, returns#145, profit#146] +Input [5]: [channel#119, id#120, sales#138, returns#139, profit#140] +Arguments: 100, [channel#119 ASC NULLS FIRST, id#120 ASC NULLS FIRST], [channel#119, id#120, sales#138, returns#139, profit#140] ===== Subqueries ===== @@ -625,6 +625,6 @@ Output [1]: [d_date_sk#16] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#8 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 d2d70bef8c3ee..167e142598ae8 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 @@ -170,178 +170,178 @@ Input [3]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19] Condition : isnotnull(ctr_total_return#19) (24) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] +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#23), dynamicpruningexpression(cr_returned_date_sk#23 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 (25) ColumnarToRow [codegen id : 8] -Input [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] +Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] (26) Filter [codegen id : 8] -Input [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] -Condition : isnotnull(cr_returning_addr_sk#21) +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) (27) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#24] +Output [1]: [d_date_sk#6] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cr_returned_date_sk#23] -Right keys [1]: [d_date_sk#24] +Left keys [1]: [cr_returned_date_sk#4] +Right keys [1]: [d_date_sk#6] Join condition: None (29) Project [codegen id : 8] -Output [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] -Input [5]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23, d_date_sk#24] +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] (30) Exchange -Input [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] -Arguments: hashpartitioning(cr_returning_addr_sk#21, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] +Arguments: hashpartitioning(cr_returning_addr_sk#2, 5), ENSURE_REQUIREMENTS, [id=#20] (31) Sort [codegen id : 9] -Input [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] -Arguments: [cr_returning_addr_sk#21 ASC NULLS FIRST], false, 0 +Input [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] +Arguments: [cr_returning_addr_sk#2 ASC NULLS FIRST], false, 0 (32) ReusedExchange [Reuses operator id: 16] -Output [2]: [ca_address_sk#26, ca_state#27] +Output [2]: [ca_address_sk#10, ca_state#11] (33) Sort [codegen id : 11] -Input [2]: [ca_address_sk#26, ca_state#27] -Arguments: [ca_address_sk#26 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#10, ca_state#11] +Arguments: [ca_address_sk#10 ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 12] -Left keys [1]: [cr_returning_addr_sk#21] -Right keys [1]: [ca_address_sk#26] +Left keys [1]: [cr_returning_addr_sk#2] +Right keys [1]: [ca_address_sk#10] Join condition: None (35) Project [codegen id : 12] -Output [3]: [cr_returning_customer_sk#20, cr_return_amt_inc_tax#22, ca_state#27] -Input [5]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, ca_address_sk#26, ca_state#27] +Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#11] +Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#10, ca_state#11] (36) HashAggregate [codegen id : 12] -Input [3]: [cr_returning_customer_sk#20, cr_return_amt_inc_tax#22, ca_state#27] -Keys [2]: [cr_returning_customer_sk#20, ca_state#27] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#22))] -Aggregate Attributes [1]: [sum#28] -Results [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] +Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#11] +Keys [2]: [cr_returning_customer_sk#1, ca_state#11] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] +Aggregate Attributes [1]: [sum#21] +Results [3]: [cr_returning_customer_sk#1, ca_state#11, sum#22] (37) Exchange -Input [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] -Arguments: hashpartitioning(cr_returning_customer_sk#20, ca_state#27, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [cr_returning_customer_sk#1, ca_state#11, sum#22] +Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#11, 5), ENSURE_REQUIREMENTS, [id=#23] (38) HashAggregate [codegen id : 13] -Input [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] -Keys [2]: [cr_returning_customer_sk#20, ca_state#27] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#22))#31] -Results [2]: [ca_state#27 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#22))#31,17,2) AS ctr_total_return#19] +Input [3]: [cr_returning_customer_sk#1, ca_state#11, sum#22] +Keys [2]: [cr_returning_customer_sk#1, ca_state#11] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#24] +Results [2]: [ca_state#11 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#24,17,2) AS ctr_total_return#19] (39) HashAggregate [codegen id : 13] Input [2]: [ctr_state#18, ctr_total_return#19] Keys [1]: [ctr_state#18] Functions [1]: [partial_avg(ctr_total_return#19)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [3]: [ctr_state#18, sum#34, count#35] +Aggregate Attributes [2]: [sum#25, count#26] +Results [3]: [ctr_state#18, sum#27, count#28] (40) Exchange -Input [3]: [ctr_state#18, sum#34, count#35] -Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ctr_state#18, sum#27, count#28] +Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#29] (41) HashAggregate [codegen id : 14] -Input [3]: [ctr_state#18, sum#34, count#35] +Input [3]: [ctr_state#18, sum#27, count#28] Keys [1]: [ctr_state#18] Functions [1]: [avg(ctr_total_return#19)] -Aggregate Attributes [1]: [avg(ctr_total_return#19)#37] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#37) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#38, ctr_state#18 AS ctr_state#18#39] +Aggregate Attributes [1]: [avg(ctr_total_return#19)#30] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#18 AS ctr_state#18#32] (42) Filter [codegen id : 14] -Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#38) +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) (43) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#40] +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#33] (44) BroadcastHashJoin [codegen id : 15] Left keys [1]: [ctr_state#18] -Right keys [1]: [ctr_state#18#39] -Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#38) +Right keys [1]: [ctr_state#18#32] +Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) (45) Project [codegen id : 15] Output [2]: [ctr_customer_sk#17, ctr_total_return#19] -Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] +Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] (46) Exchange Input [2]: [ctr_customer_sk#17, ctr_total_return#19] -Arguments: hashpartitioning(ctr_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#41] +Arguments: hashpartitioning(ctr_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#34] (47) Sort [codegen id : 16] Input [2]: [ctr_customer_sk#17, ctr_total_return#19] Arguments: [ctr_customer_sk#17 ASC NULLS FIRST], false, 0 (48) Scan parquet default.customer -Output [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] +Output [6]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#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_current_addr_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 18] -Input [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] +Input [6]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40] (50) Filter [codegen id : 18] -Input [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] -Condition : (isnotnull(c_customer_sk#42) AND isnotnull(c_current_addr_sk#44)) +Input [6]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40] +Condition : (isnotnull(c_customer_sk#35) AND isnotnull(c_current_addr_sk#37)) (51) Scan parquet default.customer_address -Output [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Output [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] 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 (52) ColumnarToRow [codegen id : 17] -Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Input [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] (53) Filter [codegen id : 17] -Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] -Condition : ((isnotnull(ca_state#55) AND (ca_state#55 = GA)) AND isnotnull(ca_address_sk#48)) +Input [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Condition : ((isnotnull(ca_state#11) AND (ca_state#11 = GA)) AND isnotnull(ca_address_sk#10)) (54) BroadcastExchange -Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#60] +Input [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] (55) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#44] -Right keys [1]: [ca_address_sk#48] +Left keys [1]: [c_current_addr_sk#37] +Right keys [1]: [ca_address_sk#10] Join condition: None (56) Project [codegen id : 18] -Output [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] -Input [18]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47, ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Output [16]: [c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Input [18]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40, ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] (57) Exchange -Input [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] -Arguments: hashpartitioning(c_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [16]: [c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Arguments: hashpartitioning(c_customer_sk#35, 5), ENSURE_REQUIREMENTS, [id=#52] (58) Sort [codegen id : 19] -Input [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] -Arguments: [c_customer_sk#42 ASC NULLS FIRST], false, 0 +Input [16]: [c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Arguments: [c_customer_sk#35 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 20] Left keys [1]: [ctr_customer_sk#17] -Right keys [1]: [c_customer_sk#42] +Right keys [1]: [c_customer_sk#35] Join condition: None (60) Project [codegen id : 20] -Output [16]: [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] -Input [18]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Output [16]: [c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#19] +Input [18]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] (61) TakeOrderedAndProject -Input [16]: [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] -Arguments: 100, [c_customer_id#43 ASC NULLS FIRST, c_salutation#45 ASC NULLS FIRST, c_first_name#46 ASC NULLS FIRST, c_last_name#47 ASC NULLS FIRST, ca_street_number#49 ASC NULLS FIRST, ca_street_name#50 ASC NULLS FIRST, ca_street_type#51 ASC NULLS FIRST, ca_suite_number#52 ASC NULLS FIRST, ca_city#53 ASC NULLS FIRST, ca_county#54 ASC NULLS FIRST, ca_state#55 ASC NULLS FIRST, ca_zip#56 ASC NULLS FIRST, ca_country#57 ASC NULLS FIRST, ca_gmt_offset#58 ASC NULLS FIRST, ca_location_type#59 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] +Input [16]: [c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#19] +Arguments: 100, [c_customer_id#36 ASC NULLS FIRST, c_salutation#38 ASC NULLS FIRST, c_first_name#39 ASC NULLS FIRST, c_last_name#40 ASC NULLS FIRST, ca_street_number#41 ASC NULLS FIRST, ca_street_name#42 ASC NULLS FIRST, ca_street_type#43 ASC NULLS FIRST, ca_suite_number#44 ASC NULLS FIRST, ca_city#45 ASC NULLS FIRST, ca_county#46 ASC NULLS FIRST, ca_state#11 ASC NULLS FIRST, ca_zip#47 ASC NULLS FIRST, ca_country#48 ASC NULLS FIRST, ca_gmt_offset#49 ASC NULLS FIRST, ca_location_type#50 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#19] ===== Subqueries ===== @@ -352,6 +352,6 @@ ReusedExchange (62) (62) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#23 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 24 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/q81/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt index 04371a7f43d2a..6a8f9e5264d49 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 @@ -149,154 +149,154 @@ Input [3]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18] Condition : isnotnull(ctr_total_return#18) (21) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22] +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#22), dynamicpruningexpression(cr_returned_date_sk#22 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 (22) ColumnarToRow [codegen id : 6] -Input [4]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22] +Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] (23) Filter [codegen id : 6] -Input [4]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22] -Condition : isnotnull(cr_returning_addr_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] +Condition : isnotnull(cr_returning_addr_sk#2) (24) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#23] +Output [1]: [d_date_sk#6] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#22] -Right keys [1]: [d_date_sk#23] +Left keys [1]: [cr_returned_date_sk#4] +Right keys [1]: [d_date_sk#6] Join condition: None (26) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21] -Input [5]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22, d_date_sk#23] +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] (27) ReusedExchange [Reuses operator id: 14] -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#9, ca_state#10] (28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#20] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [cr_returning_addr_sk#2] +Right keys [1]: [ca_address_sk#9] Join condition: None (29) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#19, cr_return_amt_inc_tax#21, ca_state#25] -Input [5]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, ca_address_sk#24, ca_state#25] +Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] +Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#9, ca_state#10] (30) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#19, cr_return_amt_inc_tax#21, ca_state#25] -Keys [2]: [cr_returning_customer_sk#19, ca_state#25] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#21))] -Aggregate Attributes [1]: [sum#26] -Results [3]: [cr_returning_customer_sk#19, ca_state#25, sum#27] +Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] +Keys [2]: [cr_returning_customer_sk#1, ca_state#10] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] +Aggregate Attributes [1]: [sum#19] +Results [3]: [cr_returning_customer_sk#1, ca_state#10, sum#20] (31) Exchange -Input [3]: [cr_returning_customer_sk#19, ca_state#25, sum#27] -Arguments: hashpartitioning(cr_returning_customer_sk#19, ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#20] +Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#21] (32) HashAggregate [codegen id : 7] -Input [3]: [cr_returning_customer_sk#19, ca_state#25, sum#27] -Keys [2]: [cr_returning_customer_sk#19, ca_state#25] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#21))#29] -Results [2]: [ca_state#25 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#21))#29,17,2) AS ctr_total_return#18] +Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#20] +Keys [2]: [cr_returning_customer_sk#1, ca_state#10] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#22] +Results [2]: [ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#22,17,2) AS ctr_total_return#18] (33) HashAggregate [codegen id : 7] Input [2]: [ctr_state#17, ctr_total_return#18] Keys [1]: [ctr_state#17] Functions [1]: [partial_avg(ctr_total_return#18)] -Aggregate Attributes [2]: [sum#30, count#31] -Results [3]: [ctr_state#17, sum#32, count#33] +Aggregate Attributes [2]: [sum#23, count#24] +Results [3]: [ctr_state#17, sum#25, count#26] (34) Exchange -Input [3]: [ctr_state#17, sum#32, count#33] -Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [ctr_state#17, sum#25, count#26] +Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#27] (35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#17, sum#32, count#33] +Input [3]: [ctr_state#17, sum#25, count#26] Keys [1]: [ctr_state#17] Functions [1]: [avg(ctr_total_return#18)] -Aggregate Attributes [1]: [avg(ctr_total_return#18)#35] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#35) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#36, ctr_state#17 AS ctr_state#17#37] +Aggregate Attributes [1]: [avg(ctr_total_return#18)#28] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#28) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#29, ctr_state#17 AS ctr_state#17#30] (36) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#36) +Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) (37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#38] +Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#31] (38) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#17] -Right keys [1]: [ctr_state#17#37] -Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#36) +Right keys [1]: [ctr_state#17#30] +Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) (39) Project [codegen id : 11] Output [2]: [ctr_customer_sk#16, ctr_total_return#18] -Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] +Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] (40) Scan parquet default.customer -Output [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] +Output [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] +Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] (42) Filter [codegen id : 9] -Input [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] -Condition : (isnotnull(c_customer_sk#39) AND isnotnull(c_current_addr_sk#41)) +Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34)) (43) BroadcastExchange -Input [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] +Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] (44) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#16] -Right keys [1]: [c_customer_sk#39] +Right keys [1]: [c_customer_sk#32] Join condition: None (45) Project [codegen id : 11] -Output [6]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] -Input [8]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] +Output [6]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +Input [8]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] (46) Scan parquet default.customer_address -Output [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] +Output [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] 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 (47) ColumnarToRow [codegen id : 10] -Input [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] +Input [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] (48) Filter [codegen id : 10] -Input [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] -Condition : ((isnotnull(ca_state#53) AND (ca_state#53 = GA)) AND isnotnull(ca_address_sk#46)) +Input [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] +Condition : ((isnotnull(ca_state#10) AND (ca_state#10 = GA)) AND isnotnull(ca_address_sk#9)) (49) BroadcastExchange -Input [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] +Input [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#41] -Right keys [1]: [ca_address_sk#46] +Left keys [1]: [c_current_addr_sk#34] +Right keys [1]: [ca_address_sk#9] Join condition: None (51) Project [codegen id : 11] -Output [16]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57, ctr_total_return#18] -Input [18]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] +Output [16]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48, ctr_total_return#18] +Input [18]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] (52) TakeOrderedAndProject -Input [16]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57, ctr_total_return#18] -Arguments: 100, [c_customer_id#40 ASC NULLS FIRST, c_salutation#42 ASC NULLS FIRST, c_first_name#43 ASC NULLS FIRST, c_last_name#44 ASC NULLS FIRST, ca_street_number#47 ASC NULLS FIRST, ca_street_name#48 ASC NULLS FIRST, ca_street_type#49 ASC NULLS FIRST, ca_suite_number#50 ASC NULLS FIRST, ca_city#51 ASC NULLS FIRST, ca_county#52 ASC NULLS FIRST, ca_state#53 ASC NULLS FIRST, ca_zip#54 ASC NULLS FIRST, ca_country#55 ASC NULLS FIRST, ca_gmt_offset#56 ASC NULLS FIRST, ca_location_type#57 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57, ctr_total_return#18] +Input [16]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48, ctr_total_return#18] +Arguments: 100, [c_customer_id#33 ASC NULLS FIRST, c_salutation#35 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, c_last_name#37 ASC NULLS FIRST, ca_street_number#39 ASC NULLS FIRST, ca_street_name#40 ASC NULLS FIRST, ca_street_type#41 ASC NULLS FIRST, ca_suite_number#42 ASC NULLS FIRST, ca_city#43 ASC NULLS FIRST, ca_county#44 ASC NULLS FIRST, ca_state#10 ASC NULLS FIRST, ca_zip#45 ASC NULLS FIRST, ca_country#46 ASC NULLS FIRST, ca_gmt_offset#47 ASC NULLS FIRST, ca_location_type#48 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48, ctr_total_return#18] ===== Subqueries ===== @@ -307,6 +307,6 @@ ReusedExchange (53) (53) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 21 Hosting Expression = cr_returned_date_sk#22 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 21 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/q82.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt index 6725e273a3acc..8b09336f86535 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt @@ -95,7 +95,7 @@ Input [2]: [d_date_sk#11, d_date#12] (14) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 11102)) AND (d_date#12 <= 11162)) AND isnotnull(d_date_sk#11)) (15) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt index a03333f7623cc..340fcf5fa31fd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt @@ -92,7 +92,7 @@ Input [2]: [d_date_sk#11, d_date#12] (14) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-05-25)) AND (d_date#12 <= 2000-07-24)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 11102)) AND (d_date#12 <= 11162)) AND isnotnull(d_date_sk#11)) (15) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt index 7f7c9ac72707e..0073fd978d748 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt @@ -245,7 +245,7 @@ Arguments: [wr_refunded_cdemo_sk#13 ASC NULLS FIRST, wr_returning_cdemo_sk#15 AS Output [3]: [cd_demo_sk#33, cd_marital_status#34, cd_education_status#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] ReadSchema: struct (43) ColumnarToRow [codegen id : 11] @@ -253,7 +253,7 @@ Input [3]: [cd_demo_sk#33, cd_marital_status#34, cd_education_status#35] (44) Filter [codegen id : 11] Input [3]: [cd_demo_sk#33, cd_marital_status#34, cd_education_status#35] -Condition : (((isnotnull(cd_demo_sk#33) AND isnotnull(cd_marital_status#34)) AND isnotnull(cd_education_status#35)) AND ((((cd_marital_status#34 = M) AND (cd_education_status#35 = Advanced Degree )) OR ((cd_marital_status#34 = S) AND (cd_education_status#35 = College ))) OR ((cd_marital_status#34 = W) AND (cd_education_status#35 = 2 yr Degree )))) +Condition : (((isnotnull(cd_demo_sk#33) AND isnotnull(cd_marital_status#34)) AND isnotnull(cd_education_status#35)) AND ((((cd_marital_status#34 = M) AND (cd_education_status#35 = Advanced Degree)) OR ((cd_marital_status#34 = S) AND (cd_education_status#35 = College))) OR ((cd_marital_status#34 = W) AND (cd_education_status#35 = 2 yr Degree)))) (45) BroadcastExchange Input [3]: [cd_demo_sk#33, cd_marital_status#34, cd_education_status#35] @@ -293,7 +293,7 @@ Arguments: [cast(cd_demo_sk#33 as bigint) ASC NULLS FIRST, cast(cd_demo_sk#37 as (53) SortMergeJoin [codegen id : 14] Left keys [2]: [wr_refunded_cdemo_sk#13, wr_returning_cdemo_sk#15] Right keys [2]: [cast(cd_demo_sk#33 as bigint), cast(cd_demo_sk#37 as bigint)] -Join condition: ((((((cd_marital_status#34 = M) AND (cd_education_status#35 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#34 = S) AND (cd_education_status#35 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#34 = W) AND (cd_education_status#35 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) +Join condition: ((((((cd_marital_status#34 = M) AND (cd_education_status#35 = Advanced Degree)) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#34 = S) AND (cd_education_status#35 = College)) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#34 = W) AND (cd_education_status#35 = 2 yr Degree)) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) (54) Project [codegen id : 14] Output [4]: [ws_quantity#4, wr_fee#18, wr_refunded_cash#19, r_reason_desc#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt index 74a9aa3dec985..c528cf8f91a55 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt @@ -130,7 +130,7 @@ Input [12]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, Output [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] ReadSchema: struct (18) ColumnarToRow [codegen id : 3] @@ -138,7 +138,7 @@ Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] (19) Filter [codegen id : 3] Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] -Condition : (((isnotnull(cd_demo_sk#21) AND isnotnull(cd_marital_status#22)) AND isnotnull(cd_education_status#23)) AND ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) OR ((cd_marital_status#22 = S) AND (cd_education_status#23 = College ))) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )))) +Condition : (((isnotnull(cd_demo_sk#21) AND isnotnull(cd_marital_status#22)) AND isnotnull(cd_education_status#23)) AND ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) OR ((cd_marital_status#22 = S) AND (cd_education_status#23 = College))) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)))) (20) BroadcastExchange Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] @@ -147,7 +147,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (21) BroadcastHashJoin [codegen id : 8] Left keys [1]: [wr_refunded_cdemo_sk#11] Right keys [1]: [cast(cd_demo_sk#21 as bigint)] -Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) +Join condition: ((((((cd_marital_status#22 = M) AND (cd_education_status#23 = Advanced Degree)) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#22 = S) AND (cd_education_status#23 = College)) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#22 = W) AND (cd_education_status#23 = 2 yr Degree)) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) (22) Project [codegen id : 8] Output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#12, wr_returning_cdemo_sk#13, wr_reason_sk#14, wr_fee#16, wr_refunded_cash#17, cd_marital_status#22, cd_education_status#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt index 92895cb566fd2..888fd8e520796 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.sf100/explain.txt @@ -177,11 +177,11 @@ Results [3]: [c_last_name#11, c_first_name#10, d_date#5] (23) Exchange Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#14] (24) Sort [codegen id : 8] Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 0) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 (25) Scan parquet default.catalog_sales Output [2]: [cs_bill_customer_sk#15, cs_sold_date_sk#16] @@ -254,15 +254,15 @@ Results [3]: [c_last_name#22, c_first_name#21, d_date#18] (40) Exchange Input [3]: [c_last_name#22, c_first_name#21, d_date#18] -Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18), 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 0), isnull(d_date#18), 5), ENSURE_REQUIREMENTS, [id=#24] (41) Sort [codegen id : 16] Input [3]: [c_last_name#22, c_first_name#21, d_date#18] -Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 1970-01-01) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#22, ) ASC NULLS FIRST, isnull(c_last_name#22) ASC NULLS FIRST, coalesce(c_first_name#21, ) ASC NULLS FIRST, isnull(c_first_name#21) ASC NULLS FIRST, coalesce(d_date#18, 0) ASC NULLS FIRST, isnull(d_date#18) ASC NULLS FIRST], false, 0 (42) SortMergeJoin -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 1970-01-01), isnull(d_date#18)] +Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#22, ), isnull(c_last_name#22), coalesce(c_first_name#21, ), isnull(c_first_name#21), coalesce(d_date#18, 0), isnull(d_date#18)] Join condition: None (43) HashAggregate [codegen id : 17] @@ -285,11 +285,11 @@ Results [3]: [c_last_name#11, c_first_name#10, d_date#5] (46) Exchange Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#26] +Arguments: hashpartitioning(coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5), 5), ENSURE_REQUIREMENTS, [id=#26] (47) Sort [codegen id : 19] Input [3]: [c_last_name#11, c_first_name#10, d_date#5] -Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 1970-01-01) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#11, ) ASC NULLS FIRST, isnull(c_last_name#11) ASC NULLS FIRST, coalesce(c_first_name#10, ) ASC NULLS FIRST, isnull(c_first_name#10) ASC NULLS FIRST, coalesce(d_date#5, 0) ASC NULLS FIRST, isnull(d_date#5) ASC NULLS FIRST], false, 0 (48) Scan parquet default.web_sales Output [2]: [ws_bill_customer_sk#27, ws_sold_date_sk#28] @@ -362,15 +362,15 @@ Results [3]: [c_last_name#34, c_first_name#33, d_date#30] (63) Exchange Input [3]: [c_last_name#34, c_first_name#33, d_date#30] -Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30), 5), ENSURE_REQUIREMENTS, [id=#36] +Arguments: hashpartitioning(coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 0), isnull(d_date#30), 5), ENSURE_REQUIREMENTS, [id=#36] (64) Sort [codegen id : 27] Input [3]: [c_last_name#34, c_first_name#33, d_date#30] -Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 1970-01-01) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 +Arguments: [coalesce(c_last_name#34, ) ASC NULLS FIRST, isnull(c_last_name#34) ASC NULLS FIRST, coalesce(c_first_name#33, ) ASC NULLS FIRST, isnull(c_first_name#33) ASC NULLS FIRST, coalesce(d_date#30, 0) ASC NULLS FIRST, isnull(d_date#30) ASC NULLS FIRST], false, 0 (65) SortMergeJoin -Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 1970-01-01), isnull(d_date#30)] +Left keys [6]: [coalesce(c_last_name#11, ), isnull(c_last_name#11), coalesce(c_first_name#10, ), isnull(c_first_name#10), coalesce(d_date#5, 0), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#34, ), isnull(c_last_name#34), coalesce(c_first_name#33, ), isnull(c_first_name#33), coalesce(d_date#30, 0), isnull(d_date#30)] Join condition: None (66) HashAggregate [codegen id : 28] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt index 27e16b75638a8..bc62b67957935 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/explain.txt @@ -205,11 +205,11 @@ Results [3]: [c_last_name#19, c_first_name#18, d_date#16] (32) BroadcastExchange Input [3]: [c_last_name#19, c_first_name#18, d_date#16] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 0), isnull(input[2, date, true])),false), [id=#21] (33) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 1970-01-01), isnull(d_date#16)] +Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 0), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#19, ), isnull(c_last_name#19), coalesce(c_first_name#18, ), isnull(c_first_name#18), coalesce(d_date#16, 0), isnull(d_date#16)] Join condition: None (34) HashAggregate [codegen id : 12] @@ -285,11 +285,11 @@ Results [3]: [c_last_name#28, c_first_name#27, d_date#25] (48) BroadcastExchange Input [3]: [c_last_name#28, c_first_name#27, d_date#25] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [id=#30] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 0), isnull(input[2, date, true])),false), [id=#30] (49) BroadcastHashJoin [codegen id : 12] -Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] -Right keys [6]: [coalesce(c_last_name#28, ), isnull(c_last_name#28), coalesce(c_first_name#27, ), isnull(c_first_name#27), coalesce(d_date#25, 1970-01-01), isnull(d_date#25)] +Left keys [6]: [coalesce(c_last_name#10, ), isnull(c_last_name#10), coalesce(c_first_name#9, ), isnull(c_first_name#9), coalesce(d_date#5, 0), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#28, ), isnull(c_last_name#28), coalesce(c_first_name#27, ), isnull(c_first_name#27), coalesce(d_date#25, 0), isnull(d_date#25)] Join condition: None (50) HashAggregate [codegen id : 12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt index e72928545d080..f9d73c29db3d8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt @@ -313,687 +313,687 @@ Aggregate Attributes [1]: [count(1)#19] Results [1]: [count(1)#19 AS h8_30_to_9#20] (29) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (31) Filter [codegen id : 8] -Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] -Condition : ((isnotnull(ss_hdemo_sk#22) AND isnotnull(ss_sold_time_sk#21)) AND isnotnull(ss_store_sk#23)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (32) Project [codegen id : 8] -Output [3]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23] -Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (33) Scan parquet default.time_dim -Output [3]: [t_time_sk#25, t_hour#26, t_minute#27] +Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 5] -Input [3]: [t_time_sk#25, t_hour#26, t_minute#27] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (35) Filter [codegen id : 5] -Input [3]: [t_time_sk#25, t_hour#26, t_minute#27] -Condition : ((((isnotnull(t_hour#26) AND isnotnull(t_minute#27)) AND (t_hour#26 = 9)) AND (t_minute#27 < 30)) AND isnotnull(t_time_sk#25)) +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 9)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) (36) Project [codegen id : 5] -Output [1]: [t_time_sk#25] -Input [3]: [t_time_sk#25, t_hour#26, t_minute#27] +Output [1]: [t_time_sk#5] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (37) BroadcastExchange -Input [1]: [t_time_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [t_time_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (38) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_time_sk#21] -Right keys [1]: [t_time_sk#25] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#5] Join condition: None (39) Project [codegen id : 8] -Output [2]: [ss_hdemo_sk#22, ss_store_sk#23] -Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, t_time_sk#25] +Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] (40) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#29] +Output [1]: [s_store_sk#9] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#23] -Right keys [1]: [s_store_sk#29] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#9] Join condition: None (42) Project [codegen id : 8] -Output [1]: [ss_hdemo_sk#22] -Input [3]: [ss_hdemo_sk#22, ss_store_sk#23, s_store_sk#29] +Output [1]: [ss_hdemo_sk#2] +Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] (43) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#30] +Output [1]: [hd_demo_sk#12] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_hdemo_sk#22] -Right keys [1]: [hd_demo_sk#30] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#12] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ss_hdemo_sk#22, hd_demo_sk#30] +Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#31] -Results [1]: [count#32] +Aggregate Attributes [1]: [count#22] +Results [1]: [count#23] (47) Exchange -Input [1]: [count#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] +Input [1]: [count#23] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] (48) HashAggregate [codegen id : 9] -Input [1]: [count#32] +Input [1]: [count#23] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#34] -Results [1]: [count(1)#34 AS h9_to_9_30#35] +Aggregate Attributes [1]: [count(1)#25] +Results [1]: [count(1)#25 AS h9_to_9_30#26] (49) BroadcastExchange -Input [1]: [h9_to_9_30#35] -Arguments: IdentityBroadcastMode, [id=#36] +Input [1]: [h9_to_9_30#26] +Arguments: IdentityBroadcastMode, [id=#27] (50) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (51) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 13] -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (53) Filter [codegen id : 13] -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (54) Project [codegen id : 13] -Output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (55) Scan parquet default.time_dim -Output [3]: [t_time_sk#41, t_hour#42, t_minute#43] +Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 10] -Input [3]: [t_time_sk#41, t_hour#42, t_minute#43] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (57) Filter [codegen id : 10] -Input [3]: [t_time_sk#41, t_hour#42, t_minute#43] -Condition : ((((isnotnull(t_hour#42) AND isnotnull(t_minute#43)) AND (t_hour#42 = 9)) AND (t_minute#43 >= 30)) AND isnotnull(t_time_sk#41)) +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 9)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5)) (58) Project [codegen id : 10] -Output [1]: [t_time_sk#41] -Input [3]: [t_time_sk#41, t_hour#42, t_minute#43] +Output [1]: [t_time_sk#5] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (59) BroadcastExchange -Input [1]: [t_time_sk#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] +Input [1]: [t_time_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (60) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_time_sk#37] -Right keys [1]: [t_time_sk#41] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#5] Join condition: None (61) Project [codegen id : 13] -Output [2]: [ss_hdemo_sk#38, ss_store_sk#39] -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, t_time_sk#41] +Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] (62) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#45] +Output [1]: [s_store_sk#9] (63) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#39] -Right keys [1]: [s_store_sk#45] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#9] Join condition: None (64) Project [codegen id : 13] -Output [1]: [ss_hdemo_sk#38] -Input [3]: [ss_hdemo_sk#38, ss_store_sk#39, s_store_sk#45] +Output [1]: [ss_hdemo_sk#2] +Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] (65) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#46] +Output [1]: [hd_demo_sk#12] (66) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_hdemo_sk#38] -Right keys [1]: [hd_demo_sk#46] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#12] Join condition: None (67) Project [codegen id : 13] Output: [] -Input [2]: [ss_hdemo_sk#38, hd_demo_sk#46] +Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] (68) HashAggregate [codegen id : 13] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#47] -Results [1]: [count#48] +Aggregate Attributes [1]: [count#29] +Results [1]: [count#30] (69) Exchange -Input [1]: [count#48] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] +Input [1]: [count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] (70) HashAggregate [codegen id : 14] -Input [1]: [count#48] +Input [1]: [count#30] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#50] -Results [1]: [count(1)#50 AS h9_30_to_10#51] +Aggregate Attributes [1]: [count(1)#32] +Results [1]: [count(1)#32 AS h9_30_to_10#33] (71) BroadcastExchange -Input [1]: [h9_30_to_10#51] -Arguments: IdentityBroadcastMode, [id=#52] +Input [1]: [h9_30_to_10#33] +Arguments: IdentityBroadcastMode, [id=#34] (72) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (73) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 18] -Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (75) Filter [codegen id : 18] -Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] -Condition : ((isnotnull(ss_hdemo_sk#54) AND isnotnull(ss_sold_time_sk#53)) AND isnotnull(ss_store_sk#55)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (76) Project [codegen id : 18] -Output [3]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55] -Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (77) Scan parquet default.time_dim -Output [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (78) ColumnarToRow [codegen id : 15] -Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (79) Filter [codegen id : 15] -Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] -Condition : ((((isnotnull(t_hour#58) AND isnotnull(t_minute#59)) AND (t_hour#58 = 10)) AND (t_minute#59 < 30)) AND isnotnull(t_time_sk#57)) +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 10)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) (80) Project [codegen id : 15] -Output [1]: [t_time_sk#57] -Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Output [1]: [t_time_sk#5] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (81) BroadcastExchange -Input [1]: [t_time_sk#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] +Input [1]: [t_time_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] (82) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_time_sk#53] -Right keys [1]: [t_time_sk#57] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#5] Join condition: None (83) Project [codegen id : 18] -Output [2]: [ss_hdemo_sk#54, ss_store_sk#55] -Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, t_time_sk#57] +Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] (84) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#61] +Output [1]: [s_store_sk#9] (85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#55] -Right keys [1]: [s_store_sk#61] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#9] Join condition: None (86) Project [codegen id : 18] -Output [1]: [ss_hdemo_sk#54] -Input [3]: [ss_hdemo_sk#54, ss_store_sk#55, s_store_sk#61] +Output [1]: [ss_hdemo_sk#2] +Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] (87) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#62] +Output [1]: [hd_demo_sk#12] (88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#54] -Right keys [1]: [hd_demo_sk#62] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#12] Join condition: None (89) Project [codegen id : 18] Output: [] -Input [2]: [ss_hdemo_sk#54, hd_demo_sk#62] +Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] (90) HashAggregate [codegen id : 18] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#63] -Results [1]: [count#64] +Aggregate Attributes [1]: [count#36] +Results [1]: [count#37] (91) Exchange -Input [1]: [count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] +Input [1]: [count#37] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#38] (92) HashAggregate [codegen id : 19] -Input [1]: [count#64] +Input [1]: [count#37] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#66] -Results [1]: [count(1)#66 AS h10_to_10_30#67] +Aggregate Attributes [1]: [count(1)#39] +Results [1]: [count(1)#39 AS h10_to_10_30#40] (93) BroadcastExchange -Input [1]: [h10_to_10_30#67] -Arguments: IdentityBroadcastMode, [id=#68] +Input [1]: [h10_to_10_30#40] +Arguments: IdentityBroadcastMode, [id=#41] (94) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (95) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (96) ColumnarToRow [codegen id : 23] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (97) Filter [codegen id : 23] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_hdemo_sk#70) AND isnotnull(ss_sold_time_sk#69)) AND isnotnull(ss_store_sk#71)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (98) Project [codegen id : 23] -Output [3]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (99) Scan parquet default.time_dim -Output [3]: [t_time_sk#73, t_hour#74, t_minute#75] +Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (100) ColumnarToRow [codegen id : 20] -Input [3]: [t_time_sk#73, t_hour#74, t_minute#75] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (101) Filter [codegen id : 20] -Input [3]: [t_time_sk#73, t_hour#74, t_minute#75] -Condition : ((((isnotnull(t_hour#74) AND isnotnull(t_minute#75)) AND (t_hour#74 = 10)) AND (t_minute#75 >= 30)) AND isnotnull(t_time_sk#73)) +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 10)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5)) (102) Project [codegen id : 20] -Output [1]: [t_time_sk#73] -Input [3]: [t_time_sk#73, t_hour#74, t_minute#75] +Output [1]: [t_time_sk#5] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (103) BroadcastExchange -Input [1]: [t_time_sk#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] +Input [1]: [t_time_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (104) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_sold_time_sk#69] -Right keys [1]: [t_time_sk#73] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#5] Join condition: None (105) Project [codegen id : 23] -Output [2]: [ss_hdemo_sk#70, ss_store_sk#71] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, t_time_sk#73] +Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] (106) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#77] +Output [1]: [s_store_sk#9] (107) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_store_sk#71] -Right keys [1]: [s_store_sk#77] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#9] Join condition: None (108) Project [codegen id : 23] -Output [1]: [ss_hdemo_sk#70] -Input [3]: [ss_hdemo_sk#70, ss_store_sk#71, s_store_sk#77] +Output [1]: [ss_hdemo_sk#2] +Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] (109) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#78] +Output [1]: [hd_demo_sk#12] (110) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_hdemo_sk#70] -Right keys [1]: [hd_demo_sk#78] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#12] Join condition: None (111) Project [codegen id : 23] Output: [] -Input [2]: [ss_hdemo_sk#70, hd_demo_sk#78] +Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] (112) HashAggregate [codegen id : 23] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#79] -Results [1]: [count#80] +Aggregate Attributes [1]: [count#43] +Results [1]: [count#44] (113) Exchange -Input [1]: [count#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +Input [1]: [count#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45] (114) HashAggregate [codegen id : 24] -Input [1]: [count#80] +Input [1]: [count#44] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#82] -Results [1]: [count(1)#82 AS h10_30_to_11#83] +Aggregate Attributes [1]: [count(1)#46] +Results [1]: [count(1)#46 AS h10_30_to_11#47] (115) BroadcastExchange -Input [1]: [h10_30_to_11#83] -Arguments: IdentityBroadcastMode, [id=#84] +Input [1]: [h10_30_to_11#47] +Arguments: IdentityBroadcastMode, [id=#48] (116) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (117) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (118) ColumnarToRow [codegen id : 28] -Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (119) Filter [codegen id : 28] -Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] -Condition : ((isnotnull(ss_hdemo_sk#86) AND isnotnull(ss_sold_time_sk#85)) AND isnotnull(ss_store_sk#87)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (120) Project [codegen id : 28] -Output [3]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87] -Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (121) Scan parquet default.time_dim -Output [3]: [t_time_sk#89, t_hour#90, t_minute#91] +Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (122) ColumnarToRow [codegen id : 25] -Input [3]: [t_time_sk#89, t_hour#90, t_minute#91] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (123) Filter [codegen id : 25] -Input [3]: [t_time_sk#89, t_hour#90, t_minute#91] -Condition : ((((isnotnull(t_hour#90) AND isnotnull(t_minute#91)) AND (t_hour#90 = 11)) AND (t_minute#91 < 30)) AND isnotnull(t_time_sk#89)) +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 11)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) (124) Project [codegen id : 25] -Output [1]: [t_time_sk#89] -Input [3]: [t_time_sk#89, t_hour#90, t_minute#91] +Output [1]: [t_time_sk#5] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (125) BroadcastExchange -Input [1]: [t_time_sk#89] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#92] +Input [1]: [t_time_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] (126) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_sold_time_sk#85] -Right keys [1]: [t_time_sk#89] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#5] Join condition: None (127) Project [codegen id : 28] -Output [2]: [ss_hdemo_sk#86, ss_store_sk#87] -Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, t_time_sk#89] +Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] (128) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#93] +Output [1]: [s_store_sk#9] (129) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_store_sk#87] -Right keys [1]: [s_store_sk#93] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#9] Join condition: None (130) Project [codegen id : 28] -Output [1]: [ss_hdemo_sk#86] -Input [3]: [ss_hdemo_sk#86, ss_store_sk#87, s_store_sk#93] +Output [1]: [ss_hdemo_sk#2] +Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] (131) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#94] +Output [1]: [hd_demo_sk#12] (132) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_hdemo_sk#86] -Right keys [1]: [hd_demo_sk#94] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#12] Join condition: None (133) Project [codegen id : 28] Output: [] -Input [2]: [ss_hdemo_sk#86, hd_demo_sk#94] +Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] (134) HashAggregate [codegen id : 28] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#95] -Results [1]: [count#96] +Aggregate Attributes [1]: [count#50] +Results [1]: [count#51] (135) Exchange -Input [1]: [count#96] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#97] +Input [1]: [count#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] (136) HashAggregate [codegen id : 29] -Input [1]: [count#96] +Input [1]: [count#51] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#98] -Results [1]: [count(1)#98 AS h11_to_11_30#99] +Aggregate Attributes [1]: [count(1)#53] +Results [1]: [count(1)#53 AS h11_to_11_30#54] (137) BroadcastExchange -Input [1]: [h11_to_11_30#99] -Arguments: IdentityBroadcastMode, [id=#100] +Input [1]: [h11_to_11_30#54] +Arguments: IdentityBroadcastMode, [id=#55] (138) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (139) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (140) ColumnarToRow [codegen id : 33] -Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (141) Filter [codegen id : 33] -Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] -Condition : ((isnotnull(ss_hdemo_sk#102) AND isnotnull(ss_sold_time_sk#101)) AND isnotnull(ss_store_sk#103)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (142) Project [codegen id : 33] -Output [3]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103] -Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (143) Scan parquet default.time_dim -Output [3]: [t_time_sk#105, t_hour#106, t_minute#107] +Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (144) ColumnarToRow [codegen id : 30] -Input [3]: [t_time_sk#105, t_hour#106, t_minute#107] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (145) Filter [codegen id : 30] -Input [3]: [t_time_sk#105, t_hour#106, t_minute#107] -Condition : ((((isnotnull(t_hour#106) AND isnotnull(t_minute#107)) AND (t_hour#106 = 11)) AND (t_minute#107 >= 30)) AND isnotnull(t_time_sk#105)) +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 11)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5)) (146) Project [codegen id : 30] -Output [1]: [t_time_sk#105] -Input [3]: [t_time_sk#105, t_hour#106, t_minute#107] +Output [1]: [t_time_sk#5] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (147) BroadcastExchange -Input [1]: [t_time_sk#105] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#108] +Input [1]: [t_time_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] (148) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_time_sk#101] -Right keys [1]: [t_time_sk#105] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#5] Join condition: None (149) Project [codegen id : 33] -Output [2]: [ss_hdemo_sk#102, ss_store_sk#103] -Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, t_time_sk#105] +Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] (150) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#109] +Output [1]: [s_store_sk#9] (151) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#103] -Right keys [1]: [s_store_sk#109] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#9] Join condition: None (152) Project [codegen id : 33] -Output [1]: [ss_hdemo_sk#102] -Input [3]: [ss_hdemo_sk#102, ss_store_sk#103, s_store_sk#109] +Output [1]: [ss_hdemo_sk#2] +Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] (153) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#110] +Output [1]: [hd_demo_sk#12] (154) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#102] -Right keys [1]: [hd_demo_sk#110] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#12] Join condition: None (155) Project [codegen id : 33] Output: [] -Input [2]: [ss_hdemo_sk#102, hd_demo_sk#110] +Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] (156) HashAggregate [codegen id : 33] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#111] -Results [1]: [count#112] +Aggregate Attributes [1]: [count#57] +Results [1]: [count#58] (157) Exchange -Input [1]: [count#112] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] +Input [1]: [count#58] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#59] (158) HashAggregate [codegen id : 34] -Input [1]: [count#112] +Input [1]: [count#58] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#114] -Results [1]: [count(1)#114 AS h11_30_to_12#115] +Aggregate Attributes [1]: [count(1)#60] +Results [1]: [count(1)#60 AS h11_30_to_12#61] (159) BroadcastExchange -Input [1]: [h11_30_to_12#115] -Arguments: IdentityBroadcastMode, [id=#116] +Input [1]: [h11_30_to_12#61] +Arguments: IdentityBroadcastMode, [id=#62] (160) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (161) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (162) ColumnarToRow [codegen id : 38] -Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (163) Filter [codegen id : 38] -Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] -Condition : ((isnotnull(ss_hdemo_sk#118) AND isnotnull(ss_sold_time_sk#117)) AND isnotnull(ss_store_sk#119)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (164) Project [codegen id : 38] -Output [3]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119] -Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (165) Scan parquet default.time_dim -Output [3]: [t_time_sk#121, t_hour#122, t_minute#123] +Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (166) ColumnarToRow [codegen id : 35] -Input [3]: [t_time_sk#121, t_hour#122, t_minute#123] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (167) Filter [codegen id : 35] -Input [3]: [t_time_sk#121, t_hour#122, t_minute#123] -Condition : ((((isnotnull(t_hour#122) AND isnotnull(t_minute#123)) AND (t_hour#122 = 12)) AND (t_minute#123 < 30)) AND isnotnull(t_time_sk#121)) +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 12)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) (168) Project [codegen id : 35] -Output [1]: [t_time_sk#121] -Input [3]: [t_time_sk#121, t_hour#122, t_minute#123] +Output [1]: [t_time_sk#5] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (169) BroadcastExchange -Input [1]: [t_time_sk#121] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#124] +Input [1]: [t_time_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] (170) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_sold_time_sk#117] -Right keys [1]: [t_time_sk#121] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#5] Join condition: None (171) Project [codegen id : 38] -Output [2]: [ss_hdemo_sk#118, ss_store_sk#119] -Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, t_time_sk#121] +Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] (172) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#125] +Output [1]: [s_store_sk#9] (173) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_store_sk#119] -Right keys [1]: [s_store_sk#125] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#9] Join condition: None (174) Project [codegen id : 38] -Output [1]: [ss_hdemo_sk#118] -Input [3]: [ss_hdemo_sk#118, ss_store_sk#119, s_store_sk#125] +Output [1]: [ss_hdemo_sk#2] +Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] (175) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#126] +Output [1]: [hd_demo_sk#12] (176) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_hdemo_sk#118] -Right keys [1]: [hd_demo_sk#126] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#12] Join condition: None (177) Project [codegen id : 38] Output: [] -Input [2]: [ss_hdemo_sk#118, hd_demo_sk#126] +Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] (178) HashAggregate [codegen id : 38] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#127] -Results [1]: [count#128] +Aggregate Attributes [1]: [count#64] +Results [1]: [count#65] (179) Exchange -Input [1]: [count#128] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] +Input [1]: [count#65] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] (180) HashAggregate [codegen id : 39] -Input [1]: [count#128] +Input [1]: [count#65] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#130] -Results [1]: [count(1)#130 AS h12_to_12_30#131] +Aggregate Attributes [1]: [count(1)#67] +Results [1]: [count(1)#67 AS h12_to_12_30#68] (181) BroadcastExchange -Input [1]: [h12_to_12_30#131] -Arguments: IdentityBroadcastMode, [id=#132] +Input [1]: [h12_to_12_30#68] +Arguments: IdentityBroadcastMode, [id=#69] (182) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt index 9f56c71154a66..453906c2b5f34 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt @@ -313,687 +313,687 @@ Aggregate Attributes [1]: [count(1)#19] Results [1]: [count(1)#19 AS h8_30_to_9#20] (29) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (31) Filter [codegen id : 8] -Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] -Condition : ((isnotnull(ss_hdemo_sk#22) AND isnotnull(ss_sold_time_sk#21)) AND isnotnull(ss_store_sk#23)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (32) Project [codegen id : 8] -Output [3]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23] -Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (33) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#25] +Output [1]: [hd_demo_sk#5] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_hdemo_sk#22] -Right keys [1]: [hd_demo_sk#25] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] Join condition: None (35) Project [codegen id : 8] -Output [2]: [ss_sold_time_sk#21, ss_store_sk#23] -Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, hd_demo_sk#25] +Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] (36) Scan parquet default.time_dim -Output [3]: [t_time_sk#26, t_hour#27, t_minute#28] +Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [3]: [t_time_sk#26, t_hour#27, t_minute#28] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (38) Filter [codegen id : 6] -Input [3]: [t_time_sk#26, t_hour#27, t_minute#28] -Condition : ((((isnotnull(t_hour#27) AND isnotnull(t_minute#28)) AND (t_hour#27 = 9)) AND (t_minute#28 < 30)) AND isnotnull(t_time_sk#26)) +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 9)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) (39) Project [codegen id : 6] -Output [1]: [t_time_sk#26] -Input [3]: [t_time_sk#26, t_hour#27, t_minute#28] +Output [1]: [t_time_sk#9] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (40) BroadcastExchange -Input [1]: [t_time_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] +Input [1]: [t_time_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_time_sk#21] -Right keys [1]: [t_time_sk#26] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#9] Join condition: None (42) Project [codegen id : 8] -Output [1]: [ss_store_sk#23] -Input [3]: [ss_sold_time_sk#21, ss_store_sk#23, t_time_sk#26] +Output [1]: [ss_store_sk#3] +Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] (43) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#30] +Output [1]: [s_store_sk#13] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#23] -Right keys [1]: [s_store_sk#30] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#13] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ss_store_sk#23, s_store_sk#30] +Input [2]: [ss_store_sk#3, s_store_sk#13] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#31] -Results [1]: [count#32] +Aggregate Attributes [1]: [count#22] +Results [1]: [count#23] (47) Exchange -Input [1]: [count#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] +Input [1]: [count#23] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] (48) HashAggregate [codegen id : 9] -Input [1]: [count#32] +Input [1]: [count#23] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#34] -Results [1]: [count(1)#34 AS h9_to_9_30#35] +Aggregate Attributes [1]: [count(1)#25] +Results [1]: [count(1)#25 AS h9_to_9_30#26] (49) BroadcastExchange -Input [1]: [h9_to_9_30#35] -Arguments: IdentityBroadcastMode, [id=#36] +Input [1]: [h9_to_9_30#26] +Arguments: IdentityBroadcastMode, [id=#27] (50) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (51) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 13] -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (53) Filter [codegen id : 13] -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (54) Project [codegen id : 13] -Output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (55) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#41] +Output [1]: [hd_demo_sk#5] (56) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_hdemo_sk#38] -Right keys [1]: [hd_demo_sk#41] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] Join condition: None (57) Project [codegen id : 13] -Output [2]: [ss_sold_time_sk#37, ss_store_sk#39] -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] +Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] (58) Scan parquet default.time_dim -Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] +Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 11] -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (60) Filter [codegen id : 11] -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 9)) AND (t_minute#44 >= 30)) AND isnotnull(t_time_sk#42)) +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 9)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9)) (61) Project [codegen id : 11] -Output [1]: [t_time_sk#42] -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] +Output [1]: [t_time_sk#9] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (62) BroadcastExchange -Input [1]: [t_time_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#45] +Input [1]: [t_time_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (63) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_time_sk#37] -Right keys [1]: [t_time_sk#42] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#9] Join condition: None (64) Project [codegen id : 13] -Output [1]: [ss_store_sk#39] -Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] +Output [1]: [ss_store_sk#3] +Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] (65) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#46] +Output [1]: [s_store_sk#13] (66) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#39] -Right keys [1]: [s_store_sk#46] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#13] Join condition: None (67) Project [codegen id : 13] Output: [] -Input [2]: [ss_store_sk#39, s_store_sk#46] +Input [2]: [ss_store_sk#3, s_store_sk#13] (68) HashAggregate [codegen id : 13] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#47] -Results [1]: [count#48] +Aggregate Attributes [1]: [count#29] +Results [1]: [count#30] (69) Exchange -Input [1]: [count#48] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] +Input [1]: [count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] (70) HashAggregate [codegen id : 14] -Input [1]: [count#48] +Input [1]: [count#30] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#50] -Results [1]: [count(1)#50 AS h9_30_to_10#51] +Aggregate Attributes [1]: [count(1)#32] +Results [1]: [count(1)#32 AS h9_30_to_10#33] (71) BroadcastExchange -Input [1]: [h9_30_to_10#51] -Arguments: IdentityBroadcastMode, [id=#52] +Input [1]: [h9_30_to_10#33] +Arguments: IdentityBroadcastMode, [id=#34] (72) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (73) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 18] -Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (75) Filter [codegen id : 18] -Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] -Condition : ((isnotnull(ss_hdemo_sk#54) AND isnotnull(ss_sold_time_sk#53)) AND isnotnull(ss_store_sk#55)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (76) Project [codegen id : 18] -Output [3]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55] -Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (77) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#57] +Output [1]: [hd_demo_sk#5] (78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#54] -Right keys [1]: [hd_demo_sk#57] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] Join condition: None (79) Project [codegen id : 18] -Output [2]: [ss_sold_time_sk#53, ss_store_sk#55] -Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, hd_demo_sk#57] +Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] (80) Scan parquet default.time_dim -Output [3]: [t_time_sk#58, t_hour#59, t_minute#60] +Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (81) ColumnarToRow [codegen id : 16] -Input [3]: [t_time_sk#58, t_hour#59, t_minute#60] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (82) Filter [codegen id : 16] -Input [3]: [t_time_sk#58, t_hour#59, t_minute#60] -Condition : ((((isnotnull(t_hour#59) AND isnotnull(t_minute#60)) AND (t_hour#59 = 10)) AND (t_minute#60 < 30)) AND isnotnull(t_time_sk#58)) +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 10)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) (83) Project [codegen id : 16] -Output [1]: [t_time_sk#58] -Input [3]: [t_time_sk#58, t_hour#59, t_minute#60] +Output [1]: [t_time_sk#9] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (84) BroadcastExchange -Input [1]: [t_time_sk#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] +Input [1]: [t_time_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] (85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_time_sk#53] -Right keys [1]: [t_time_sk#58] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#9] Join condition: None (86) Project [codegen id : 18] -Output [1]: [ss_store_sk#55] -Input [3]: [ss_sold_time_sk#53, ss_store_sk#55, t_time_sk#58] +Output [1]: [ss_store_sk#3] +Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] (87) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#62] +Output [1]: [s_store_sk#13] (88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#55] -Right keys [1]: [s_store_sk#62] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#13] Join condition: None (89) Project [codegen id : 18] Output: [] -Input [2]: [ss_store_sk#55, s_store_sk#62] +Input [2]: [ss_store_sk#3, s_store_sk#13] (90) HashAggregate [codegen id : 18] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#63] -Results [1]: [count#64] +Aggregate Attributes [1]: [count#36] +Results [1]: [count#37] (91) Exchange -Input [1]: [count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] +Input [1]: [count#37] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#38] (92) HashAggregate [codegen id : 19] -Input [1]: [count#64] +Input [1]: [count#37] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#66] -Results [1]: [count(1)#66 AS h10_to_10_30#67] +Aggregate Attributes [1]: [count(1)#39] +Results [1]: [count(1)#39 AS h10_to_10_30#40] (93) BroadcastExchange -Input [1]: [h10_to_10_30#67] -Arguments: IdentityBroadcastMode, [id=#68] +Input [1]: [h10_to_10_30#40] +Arguments: IdentityBroadcastMode, [id=#41] (94) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (95) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (96) ColumnarToRow [codegen id : 23] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (97) Filter [codegen id : 23] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_hdemo_sk#70) AND isnotnull(ss_sold_time_sk#69)) AND isnotnull(ss_store_sk#71)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (98) Project [codegen id : 23] -Output [3]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (99) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#73] +Output [1]: [hd_demo_sk#5] (100) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_hdemo_sk#70] -Right keys [1]: [hd_demo_sk#73] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] Join condition: None (101) Project [codegen id : 23] -Output [2]: [ss_sold_time_sk#69, ss_store_sk#71] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, hd_demo_sk#73] +Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] (102) Scan parquet default.time_dim -Output [3]: [t_time_sk#74, t_hour#75, t_minute#76] +Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (103) ColumnarToRow [codegen id : 21] -Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (104) Filter [codegen id : 21] -Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] -Condition : ((((isnotnull(t_hour#75) AND isnotnull(t_minute#76)) AND (t_hour#75 = 10)) AND (t_minute#76 >= 30)) AND isnotnull(t_time_sk#74)) +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 10)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9)) (105) Project [codegen id : 21] -Output [1]: [t_time_sk#74] -Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] +Output [1]: [t_time_sk#9] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (106) BroadcastExchange -Input [1]: [t_time_sk#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] +Input [1]: [t_time_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (107) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_sold_time_sk#69] -Right keys [1]: [t_time_sk#74] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#9] Join condition: None (108) Project [codegen id : 23] -Output [1]: [ss_store_sk#71] -Input [3]: [ss_sold_time_sk#69, ss_store_sk#71, t_time_sk#74] +Output [1]: [ss_store_sk#3] +Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] (109) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#78] +Output [1]: [s_store_sk#13] (110) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_store_sk#71] -Right keys [1]: [s_store_sk#78] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#13] Join condition: None (111) Project [codegen id : 23] Output: [] -Input [2]: [ss_store_sk#71, s_store_sk#78] +Input [2]: [ss_store_sk#3, s_store_sk#13] (112) HashAggregate [codegen id : 23] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#79] -Results [1]: [count#80] +Aggregate Attributes [1]: [count#43] +Results [1]: [count#44] (113) Exchange -Input [1]: [count#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +Input [1]: [count#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45] (114) HashAggregate [codegen id : 24] -Input [1]: [count#80] +Input [1]: [count#44] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#82] -Results [1]: [count(1)#82 AS h10_30_to_11#83] +Aggregate Attributes [1]: [count(1)#46] +Results [1]: [count(1)#46 AS h10_30_to_11#47] (115) BroadcastExchange -Input [1]: [h10_30_to_11#83] -Arguments: IdentityBroadcastMode, [id=#84] +Input [1]: [h10_30_to_11#47] +Arguments: IdentityBroadcastMode, [id=#48] (116) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (117) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (118) ColumnarToRow [codegen id : 28] -Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (119) Filter [codegen id : 28] -Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] -Condition : ((isnotnull(ss_hdemo_sk#86) AND isnotnull(ss_sold_time_sk#85)) AND isnotnull(ss_store_sk#87)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (120) Project [codegen id : 28] -Output [3]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87] -Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (121) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#89] +Output [1]: [hd_demo_sk#5] (122) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_hdemo_sk#86] -Right keys [1]: [hd_demo_sk#89] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] Join condition: None (123) Project [codegen id : 28] -Output [2]: [ss_sold_time_sk#85, ss_store_sk#87] -Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, hd_demo_sk#89] +Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] (124) Scan parquet default.time_dim -Output [3]: [t_time_sk#90, t_hour#91, t_minute#92] +Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (125) ColumnarToRow [codegen id : 26] -Input [3]: [t_time_sk#90, t_hour#91, t_minute#92] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (126) Filter [codegen id : 26] -Input [3]: [t_time_sk#90, t_hour#91, t_minute#92] -Condition : ((((isnotnull(t_hour#91) AND isnotnull(t_minute#92)) AND (t_hour#91 = 11)) AND (t_minute#92 < 30)) AND isnotnull(t_time_sk#90)) +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 11)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) (127) Project [codegen id : 26] -Output [1]: [t_time_sk#90] -Input [3]: [t_time_sk#90, t_hour#91, t_minute#92] +Output [1]: [t_time_sk#9] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (128) BroadcastExchange -Input [1]: [t_time_sk#90] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] +Input [1]: [t_time_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] (129) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_sold_time_sk#85] -Right keys [1]: [t_time_sk#90] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#9] Join condition: None (130) Project [codegen id : 28] -Output [1]: [ss_store_sk#87] -Input [3]: [ss_sold_time_sk#85, ss_store_sk#87, t_time_sk#90] +Output [1]: [ss_store_sk#3] +Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] (131) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#94] +Output [1]: [s_store_sk#13] (132) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_store_sk#87] -Right keys [1]: [s_store_sk#94] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#13] Join condition: None (133) Project [codegen id : 28] Output: [] -Input [2]: [ss_store_sk#87, s_store_sk#94] +Input [2]: [ss_store_sk#3, s_store_sk#13] (134) HashAggregate [codegen id : 28] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#95] -Results [1]: [count#96] +Aggregate Attributes [1]: [count#50] +Results [1]: [count#51] (135) Exchange -Input [1]: [count#96] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#97] +Input [1]: [count#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] (136) HashAggregate [codegen id : 29] -Input [1]: [count#96] +Input [1]: [count#51] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#98] -Results [1]: [count(1)#98 AS h11_to_11_30#99] +Aggregate Attributes [1]: [count(1)#53] +Results [1]: [count(1)#53 AS h11_to_11_30#54] (137) BroadcastExchange -Input [1]: [h11_to_11_30#99] -Arguments: IdentityBroadcastMode, [id=#100] +Input [1]: [h11_to_11_30#54] +Arguments: IdentityBroadcastMode, [id=#55] (138) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (139) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (140) ColumnarToRow [codegen id : 33] -Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (141) Filter [codegen id : 33] -Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] -Condition : ((isnotnull(ss_hdemo_sk#102) AND isnotnull(ss_sold_time_sk#101)) AND isnotnull(ss_store_sk#103)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (142) Project [codegen id : 33] -Output [3]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103] -Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (143) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#105] +Output [1]: [hd_demo_sk#5] (144) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#102] -Right keys [1]: [hd_demo_sk#105] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] Join condition: None (145) Project [codegen id : 33] -Output [2]: [ss_sold_time_sk#101, ss_store_sk#103] -Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, hd_demo_sk#105] +Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] (146) Scan parquet default.time_dim -Output [3]: [t_time_sk#106, t_hour#107, t_minute#108] +Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (147) ColumnarToRow [codegen id : 31] -Input [3]: [t_time_sk#106, t_hour#107, t_minute#108] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (148) Filter [codegen id : 31] -Input [3]: [t_time_sk#106, t_hour#107, t_minute#108] -Condition : ((((isnotnull(t_hour#107) AND isnotnull(t_minute#108)) AND (t_hour#107 = 11)) AND (t_minute#108 >= 30)) AND isnotnull(t_time_sk#106)) +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 11)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9)) (149) Project [codegen id : 31] -Output [1]: [t_time_sk#106] -Input [3]: [t_time_sk#106, t_hour#107, t_minute#108] +Output [1]: [t_time_sk#9] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (150) BroadcastExchange -Input [1]: [t_time_sk#106] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#109] +Input [1]: [t_time_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] (151) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_time_sk#101] -Right keys [1]: [t_time_sk#106] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#9] Join condition: None (152) Project [codegen id : 33] -Output [1]: [ss_store_sk#103] -Input [3]: [ss_sold_time_sk#101, ss_store_sk#103, t_time_sk#106] +Output [1]: [ss_store_sk#3] +Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] (153) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#110] +Output [1]: [s_store_sk#13] (154) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#103] -Right keys [1]: [s_store_sk#110] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#13] Join condition: None (155) Project [codegen id : 33] Output: [] -Input [2]: [ss_store_sk#103, s_store_sk#110] +Input [2]: [ss_store_sk#3, s_store_sk#13] (156) HashAggregate [codegen id : 33] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#111] -Results [1]: [count#112] +Aggregate Attributes [1]: [count#57] +Results [1]: [count#58] (157) Exchange -Input [1]: [count#112] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] +Input [1]: [count#58] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#59] (158) HashAggregate [codegen id : 34] -Input [1]: [count#112] +Input [1]: [count#58] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#114] -Results [1]: [count(1)#114 AS h11_30_to_12#115] +Aggregate Attributes [1]: [count(1)#60] +Results [1]: [count(1)#60 AS h11_30_to_12#61] (159) BroadcastExchange -Input [1]: [h11_30_to_12#115] -Arguments: IdentityBroadcastMode, [id=#116] +Input [1]: [h11_30_to_12#61] +Arguments: IdentityBroadcastMode, [id=#62] (160) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (161) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (162) ColumnarToRow [codegen id : 38] -Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (163) Filter [codegen id : 38] -Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] -Condition : ((isnotnull(ss_hdemo_sk#118) AND isnotnull(ss_sold_time_sk#117)) AND isnotnull(ss_store_sk#119)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (164) Project [codegen id : 38] -Output [3]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119] -Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (165) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#121] +Output [1]: [hd_demo_sk#5] (166) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_hdemo_sk#118] -Right keys [1]: [hd_demo_sk#121] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] Join condition: None (167) Project [codegen id : 38] -Output [2]: [ss_sold_time_sk#117, ss_store_sk#119] -Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, hd_demo_sk#121] +Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] (168) Scan parquet default.time_dim -Output [3]: [t_time_sk#122, t_hour#123, t_minute#124] +Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (169) ColumnarToRow [codegen id : 36] -Input [3]: [t_time_sk#122, t_hour#123, t_minute#124] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (170) Filter [codegen id : 36] -Input [3]: [t_time_sk#122, t_hour#123, t_minute#124] -Condition : ((((isnotnull(t_hour#123) AND isnotnull(t_minute#124)) AND (t_hour#123 = 12)) AND (t_minute#124 < 30)) AND isnotnull(t_time_sk#122)) +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 12)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) (171) Project [codegen id : 36] -Output [1]: [t_time_sk#122] -Input [3]: [t_time_sk#122, t_hour#123, t_minute#124] +Output [1]: [t_time_sk#9] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (172) BroadcastExchange -Input [1]: [t_time_sk#122] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#125] +Input [1]: [t_time_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] (173) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_sold_time_sk#117] -Right keys [1]: [t_time_sk#122] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#9] Join condition: None (174) Project [codegen id : 38] -Output [1]: [ss_store_sk#119] -Input [3]: [ss_sold_time_sk#117, ss_store_sk#119, t_time_sk#122] +Output [1]: [ss_store_sk#3] +Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] (175) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#126] +Output [1]: [s_store_sk#13] (176) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_store_sk#119] -Right keys [1]: [s_store_sk#126] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#13] Join condition: None (177) Project [codegen id : 38] Output: [] -Input [2]: [ss_store_sk#119, s_store_sk#126] +Input [2]: [ss_store_sk#3, s_store_sk#13] (178) HashAggregate [codegen id : 38] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#127] -Results [1]: [count#128] +Aggregate Attributes [1]: [count#64] +Results [1]: [count#65] (179) Exchange -Input [1]: [count#128] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] +Input [1]: [count#65] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] (180) HashAggregate [codegen id : 39] -Input [1]: [count#128] +Input [1]: [count#65] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#130] -Results [1]: [count(1)#130 AS h12_to_12_30#131] +Aggregate Attributes [1]: [count(1)#67] +Results [1]: [count(1)#67 AS h12_to_12_30#68] (181) BroadcastExchange -Input [1]: [h12_to_12_30#131] -Arguments: IdentityBroadcastMode, [id=#132] +Input [1]: [h12_to_12_30#68] +Arguments: IdentityBroadcastMode, [id=#69] (182) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt index b3d0c07cd80fe..c9ebf0e9c9426 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject (31) Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(In(i_category, [Books ,Electronics ,Sports ]),In(i_class, [computers ,stereo ,football ])),And(In(i_category, [Men ,Jewelry ,Women ]),In(i_class, [shirts ,birdal ,dresses ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(In(i_category, [Books,Electronics,Sports]),In(i_class, [computers,stereo,football])),And(In(i_category, [Men,Jewelry,Women]),In(i_class, [shirts,birdal,dresses]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 1] @@ -44,7 +44,7 @@ Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] (3) Filter [codegen id : 1] Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((i_category#4 IN (Books ,Electronics ,Sports ) AND i_class#3 IN (computers ,stereo ,football )) OR (i_category#4 IN (Men ,Jewelry ,Women ) AND i_class#3 IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) +Condition : (((i_category#4 IN (Books,Electronics,Sports) AND i_class#3 IN (computers,stereo,football)) OR (i_category#4 IN (Men,Jewelry,Women) AND i_class#3 IN (shirts,birdal,dresses))) AND isnotnull(i_item_sk#1)) (4) BroadcastExchange Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] @@ -164,7 +164,7 @@ Arguments: [avg(_w0#24) windowspecdefinition(i_category#4, i_brand#2, s_store_na (29) Filter [codegen id : 7] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, _w0#24, avg_monthly_sales#26] -Condition : (isnotnull(avg_monthly_sales#26) AND (NOT (avg_monthly_sales#26 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_monthly_sales#26) AND NOT (avg_monthly_sales#26 = 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (30) Project [codegen id : 7] Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, avg_monthly_sales#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt index f61c8e6945003..cbf6273e28568 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt @@ -36,7 +36,7 @@ TakeOrderedAndProject (31) Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(And(In(i_category, [Books ,Electronics ,Sports ]),In(i_class, [computers ,stereo ,football ])),And(In(i_category, [Men ,Jewelry ,Women ]),In(i_class, [shirts ,birdal ,dresses ]))), IsNotNull(i_item_sk)] +PushedFilters: [Or(And(In(i_category, [Books,Electronics,Sports]),In(i_class, [computers,stereo,football])),And(In(i_category, [Men,Jewelry,Women]),In(i_class, [shirts,birdal,dresses]))), IsNotNull(i_item_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 4] @@ -44,7 +44,7 @@ Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] (3) Filter [codegen id : 4] Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] -Condition : (((i_category#4 IN (Books ,Electronics ,Sports ) AND i_class#3 IN (computers ,stereo ,football )) OR (i_category#4 IN (Men ,Jewelry ,Women ) AND i_class#3 IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) +Condition : (((i_category#4 IN (Books,Electronics,Sports) AND i_class#3 IN (computers,stereo,football)) OR (i_category#4 IN (Men,Jewelry,Women) AND i_class#3 IN (shirts,birdal,dresses))) AND isnotnull(i_item_sk#1)) (4) Scan parquet default.store_sales Output [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] @@ -164,7 +164,7 @@ Arguments: [avg(_w0#24) windowspecdefinition(i_category#4, i_brand#2, s_store_na (29) Filter [codegen id : 7] Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, _w0#24, avg_monthly_sales#26] -Condition : (isnotnull(avg_monthly_sales#26) AND (NOT (avg_monthly_sales#26 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000))) +Condition : ((isnotnull(avg_monthly_sales#26) AND NOT (avg_monthly_sales#26 = 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true))) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) (30) Project [codegen id : 7] Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#16, s_company_name#17, d_moy#13, sum_sales#23, avg_monthly_sales#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt index 8736c9861a5ce..65c66eb083f55 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt @@ -82,40 +82,40 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (12) Scan parquet default.store_sales -Output [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (13) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (14) Filter [codegen id : 1] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#44) AND (ss_quantity#44 >= 1)) AND (ss_quantity#44 <= 20)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) (15) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#45] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (16) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#45] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#45))] -Aggregate Attributes [2]: [sum#47, count#48] -Results [2]: [sum#49, count#50] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#45, count#46] +Results [2]: [sum#47, count#48] (17) Exchange -Input [2]: [sum#49, count#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#51] +Input [2]: [sum#47, count#48] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] (18) HashAggregate [codegen id : 2] -Input [2]: [sum#49, count#50] +Input [2]: [sum#47, count#48] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))#52] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#45))#52 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#53] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#50] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#50 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#51] Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#6, [id=#7] * HashAggregate (25) @@ -128,40 +128,40 @@ Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (19) Scan parquet default.store_sales -Output [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (20) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (21) Filter [codegen id : 1] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] -Condition : ((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 1)) AND (ss_quantity#54 <= 20)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) (22) Project [codegen id : 1] -Output [1]: [ss_net_paid#55] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (23) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#55] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [2]: [sum#57, count#58] -Results [2]: [sum#59, count#60] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#53, count#54] +Results [2]: [sum#55, count#56] (24) Exchange -Input [2]: [sum#59, count#60] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] +Input [2]: [sum#55, count#56] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#57] (25) HashAggregate [codegen id : 2] -Input [2]: [sum#59, count#60] +Input [2]: [sum#55, count#56] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#55))#62] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#55))#62 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#63] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#58] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#58 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#59] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#9, [id=#10] * HashAggregate (32) @@ -174,40 +174,40 @@ Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (26) Scan parquet default.store_sales -Output [2]: [ss_quantity#64, ss_sold_date_sk#65] +Output [2]: [ss_quantity#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (27) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (28) Filter [codegen id : 1] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] -Condition : ((isnotnull(ss_quantity#64) AND (ss_quantity#64 >= 21)) AND (ss_quantity#64 <= 40)) +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) (29) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (30) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#66] -Results [1]: [count#67] +Aggregate Attributes [1]: [count#60] +Results [1]: [count#61] (31) Exchange -Input [1]: [count#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#68] +Input [1]: [count#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] (32) HashAggregate [codegen id : 2] -Input [1]: [count#67] +Input [1]: [count#61] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#69] -Results [1]: [count(1)#69 AS count(1)#70] +Aggregate Attributes [1]: [count(1)#63] +Results [1]: [count(1)#63 AS count(1)#64] Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] * HashAggregate (39) @@ -220,40 +220,40 @@ Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (33) Scan parquet default.store_sales -Output [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (34) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (35) Filter [codegen id : 1] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] -Condition : ((isnotnull(ss_quantity#71) AND (ss_quantity#71 >= 21)) AND (ss_quantity#71 <= 40)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) (36) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#72] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (37) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#72] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#72))] -Aggregate Attributes [2]: [sum#74, count#75] -Results [2]: [sum#76, count#77] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#65, count#66] +Results [2]: [sum#67, count#68] (38) Exchange -Input [2]: [sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] +Input [2]: [sum#67, count#68] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#69] (39) HashAggregate [codegen id : 2] -Input [2]: [sum#76, count#77] +Input [2]: [sum#67, count#68] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))#79] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#72))#79 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#80] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#70] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#70 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#71] Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#13, [id=#14] * HashAggregate (46) @@ -266,40 +266,40 @@ Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (40) Scan parquet default.store_sales -Output [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (41) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (42) Filter [codegen id : 1] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] -Condition : ((isnotnull(ss_quantity#81) AND (ss_quantity#81 >= 21)) AND (ss_quantity#81 <= 40)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) (43) Project [codegen id : 1] -Output [1]: [ss_net_paid#82] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (44) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#82] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [2]: [sum#84, count#85] -Results [2]: [sum#86, count#87] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#72, count#73] +Results [2]: [sum#74, count#75] (45) Exchange -Input [2]: [sum#86, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] +Input [2]: [sum#74, count#75] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] (46) HashAggregate [codegen id : 2] -Input [2]: [sum#86, count#87] +Input [2]: [sum#74, count#75] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#82))#89] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#82))#89 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#90] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#77] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#77 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#78] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#16, [id=#17] * HashAggregate (53) @@ -312,40 +312,40 @@ Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (47) Scan parquet default.store_sales -Output [2]: [ss_quantity#91, ss_sold_date_sk#92] +Output [2]: [ss_quantity#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (48) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (49) Filter [codegen id : 1] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] -Condition : ((isnotnull(ss_quantity#91) AND (ss_quantity#91 >= 41)) AND (ss_quantity#91 <= 60)) +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) (50) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (51) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#93] -Results [1]: [count#94] +Aggregate Attributes [1]: [count#79] +Results [1]: [count#80] (52) Exchange -Input [1]: [count#94] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] +Input [1]: [count#80] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] (53) HashAggregate [codegen id : 2] -Input [1]: [count#94] +Input [1]: [count#80] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#96] -Results [1]: [count(1)#96 AS count(1)#97] +Aggregate Attributes [1]: [count(1)#82] +Results [1]: [count(1)#82 AS count(1)#83] Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#18, [id=#19] * HashAggregate (60) @@ -358,40 +358,40 @@ Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (54) Scan parquet default.store_sales -Output [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (55) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (56) Filter [codegen id : 1] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] -Condition : ((isnotnull(ss_quantity#98) AND (ss_quantity#98 >= 41)) AND (ss_quantity#98 <= 60)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) (57) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#99] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (58) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#99] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#99))] -Aggregate Attributes [2]: [sum#101, count#102] -Results [2]: [sum#103, count#104] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#84, count#85] +Results [2]: [sum#86, count#87] (59) Exchange -Input [2]: [sum#103, count#104] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] +Input [2]: [sum#86, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] (60) HashAggregate [codegen id : 2] -Input [2]: [sum#103, count#104] +Input [2]: [sum#86, count#87] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))#106] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#99))#106 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#107] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#89] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#89 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#90] Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#20, [id=#21] * HashAggregate (67) @@ -404,40 +404,40 @@ Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (61) Scan parquet default.store_sales -Output [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (62) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (63) Filter [codegen id : 1] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] -Condition : ((isnotnull(ss_quantity#108) AND (ss_quantity#108 >= 41)) AND (ss_quantity#108 <= 60)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) (64) Project [codegen id : 1] -Output [1]: [ss_net_paid#109] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (65) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#109] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#109))] -Aggregate Attributes [2]: [sum#111, count#112] -Results [2]: [sum#113, count#114] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#91, count#92] +Results [2]: [sum#93, count#94] (66) Exchange -Input [2]: [sum#113, count#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] +Input [2]: [sum#93, count#94] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] (67) HashAggregate [codegen id : 2] -Input [2]: [sum#113, count#114] +Input [2]: [sum#93, count#94] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#109))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#109))#116] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#109))#116 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#117] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#96] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#96 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#97] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#23, [id=#24] * HashAggregate (74) @@ -450,40 +450,40 @@ Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (68) Scan parquet default.store_sales -Output [2]: [ss_quantity#118, ss_sold_date_sk#119] +Output [2]: [ss_quantity#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (69) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (70) Filter [codegen id : 1] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] -Condition : ((isnotnull(ss_quantity#118) AND (ss_quantity#118 >= 61)) AND (ss_quantity#118 <= 80)) +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) (71) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (72) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#120] -Results [1]: [count#121] +Aggregate Attributes [1]: [count#98] +Results [1]: [count#99] (73) Exchange -Input [1]: [count#121] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#122] +Input [1]: [count#99] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#100] (74) HashAggregate [codegen id : 2] -Input [1]: [count#121] +Input [1]: [count#99] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#123] -Results [1]: [count(1)#123 AS count(1)#124] +Aggregate Attributes [1]: [count(1)#101] +Results [1]: [count(1)#101 AS count(1)#102] Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#25, [id=#26] * HashAggregate (81) @@ -496,40 +496,40 @@ Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (75) Scan parquet default.store_sales -Output [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (76) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (77) Filter [codegen id : 1] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] -Condition : ((isnotnull(ss_quantity#125) AND (ss_quantity#125 >= 61)) AND (ss_quantity#125 <= 80)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) (78) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#126] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (79) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#126] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#126))] -Aggregate Attributes [2]: [sum#128, count#129] -Results [2]: [sum#130, count#131] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#103, count#104] +Results [2]: [sum#105, count#106] (80) Exchange -Input [2]: [sum#130, count#131] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#132] +Input [2]: [sum#105, count#106] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#107] (81) HashAggregate [codegen id : 2] -Input [2]: [sum#130, count#131] +Input [2]: [sum#105, count#106] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))#133] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#126))#133 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#134] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#108] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#108 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#109] Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#27, [id=#28] * HashAggregate (88) @@ -542,40 +542,40 @@ Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (82) Scan parquet default.store_sales -Output [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (83) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (84) Filter [codegen id : 1] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] -Condition : ((isnotnull(ss_quantity#135) AND (ss_quantity#135 >= 61)) AND (ss_quantity#135 <= 80)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) (85) Project [codegen id : 1] -Output [1]: [ss_net_paid#136] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (86) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#136] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#136))] -Aggregate Attributes [2]: [sum#138, count#139] -Results [2]: [sum#140, count#141] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#110, count#111] +Results [2]: [sum#112, count#113] (87) Exchange -Input [2]: [sum#140, count#141] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#142] +Input [2]: [sum#112, count#113] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] (88) HashAggregate [codegen id : 2] -Input [2]: [sum#140, count#141] +Input [2]: [sum#112, count#113] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#136))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#136))#143] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#136))#143 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#144] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#115] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#115 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#116] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#30, [id=#31] * HashAggregate (95) @@ -588,40 +588,40 @@ Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (89) Scan parquet default.store_sales -Output [2]: [ss_quantity#145, ss_sold_date_sk#146] +Output [2]: [ss_quantity#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (90) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (91) Filter [codegen id : 1] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] -Condition : ((isnotnull(ss_quantity#145) AND (ss_quantity#145 >= 81)) AND (ss_quantity#145 <= 100)) +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) (92) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (93) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#147] -Results [1]: [count#148] +Aggregate Attributes [1]: [count#117] +Results [1]: [count#118] (94) Exchange -Input [1]: [count#148] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#149] +Input [1]: [count#118] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#119] (95) HashAggregate [codegen id : 2] -Input [1]: [count#148] +Input [1]: [count#118] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#150] -Results [1]: [count(1)#150 AS count(1)#151] +Aggregate Attributes [1]: [count(1)#120] +Results [1]: [count(1)#120 AS count(1)#121] Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#32, [id=#33] * HashAggregate (102) @@ -634,40 +634,40 @@ Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (96) Scan parquet default.store_sales -Output [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (97) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (98) Filter [codegen id : 1] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] -Condition : ((isnotnull(ss_quantity#152) AND (ss_quantity#152 >= 81)) AND (ss_quantity#152 <= 100)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) (99) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#153] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (100) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#153] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#153))] -Aggregate Attributes [2]: [sum#155, count#156] -Results [2]: [sum#157, count#158] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#122, count#123] +Results [2]: [sum#124, count#125] (101) Exchange -Input [2]: [sum#157, count#158] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#159] +Input [2]: [sum#124, count#125] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#126] (102) HashAggregate [codegen id : 2] -Input [2]: [sum#157, count#158] +Input [2]: [sum#124, count#125] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))#160] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#153))#160 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#161] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#127] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#127 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#128] Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#34, [id=#35] * HashAggregate (109) @@ -680,39 +680,39 @@ Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (103) Scan parquet default.store_sales -Output [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (105) Filter [codegen id : 1] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] -Condition : ((isnotnull(ss_quantity#162) AND (ss_quantity#162 >= 81)) AND (ss_quantity#162 <= 100)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) (106) Project [codegen id : 1] -Output [1]: [ss_net_paid#163] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (107) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#163] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#163))] -Aggregate Attributes [2]: [sum#165, count#166] -Results [2]: [sum#167, count#168] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#129, count#130] +Results [2]: [sum#131, count#132] (108) Exchange -Input [2]: [sum#167, count#168] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#169] +Input [2]: [sum#131, count#132] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#133] (109) HashAggregate [codegen id : 2] -Input [2]: [sum#167, count#168] +Input [2]: [sum#131, count#132] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#163))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#163))#170] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#163))#170 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#171] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#134] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#134 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#135] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt index 8736c9861a5ce..65c66eb083f55 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt @@ -82,40 +82,40 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (12) Scan parquet default.store_sales -Output [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (13) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (14) Filter [codegen id : 1] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#44) AND (ss_quantity#44 >= 1)) AND (ss_quantity#44 <= 20)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) (15) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#45] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (16) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#45] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#45))] -Aggregate Attributes [2]: [sum#47, count#48] -Results [2]: [sum#49, count#50] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#45, count#46] +Results [2]: [sum#47, count#48] (17) Exchange -Input [2]: [sum#49, count#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#51] +Input [2]: [sum#47, count#48] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] (18) HashAggregate [codegen id : 2] -Input [2]: [sum#49, count#50] +Input [2]: [sum#47, count#48] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))#52] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#45))#52 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#53] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#50] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#50 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#51] Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#6, [id=#7] * HashAggregate (25) @@ -128,40 +128,40 @@ Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (19) Scan parquet default.store_sales -Output [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (20) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (21) Filter [codegen id : 1] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] -Condition : ((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 1)) AND (ss_quantity#54 <= 20)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) (22) Project [codegen id : 1] -Output [1]: [ss_net_paid#55] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (23) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#55] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [2]: [sum#57, count#58] -Results [2]: [sum#59, count#60] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#53, count#54] +Results [2]: [sum#55, count#56] (24) Exchange -Input [2]: [sum#59, count#60] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] +Input [2]: [sum#55, count#56] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#57] (25) HashAggregate [codegen id : 2] -Input [2]: [sum#59, count#60] +Input [2]: [sum#55, count#56] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#55))#62] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#55))#62 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#63] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#58] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#58 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#59] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#9, [id=#10] * HashAggregate (32) @@ -174,40 +174,40 @@ Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (26) Scan parquet default.store_sales -Output [2]: [ss_quantity#64, ss_sold_date_sk#65] +Output [2]: [ss_quantity#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (27) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (28) Filter [codegen id : 1] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] -Condition : ((isnotnull(ss_quantity#64) AND (ss_quantity#64 >= 21)) AND (ss_quantity#64 <= 40)) +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) (29) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (30) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#66] -Results [1]: [count#67] +Aggregate Attributes [1]: [count#60] +Results [1]: [count#61] (31) Exchange -Input [1]: [count#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#68] +Input [1]: [count#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] (32) HashAggregate [codegen id : 2] -Input [1]: [count#67] +Input [1]: [count#61] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#69] -Results [1]: [count(1)#69 AS count(1)#70] +Aggregate Attributes [1]: [count(1)#63] +Results [1]: [count(1)#63 AS count(1)#64] Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] * HashAggregate (39) @@ -220,40 +220,40 @@ Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (33) Scan parquet default.store_sales -Output [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (34) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (35) Filter [codegen id : 1] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] -Condition : ((isnotnull(ss_quantity#71) AND (ss_quantity#71 >= 21)) AND (ss_quantity#71 <= 40)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) (36) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#72] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (37) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#72] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#72))] -Aggregate Attributes [2]: [sum#74, count#75] -Results [2]: [sum#76, count#77] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#65, count#66] +Results [2]: [sum#67, count#68] (38) Exchange -Input [2]: [sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] +Input [2]: [sum#67, count#68] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#69] (39) HashAggregate [codegen id : 2] -Input [2]: [sum#76, count#77] +Input [2]: [sum#67, count#68] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))#79] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#72))#79 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#80] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#70] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#70 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#71] Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#13, [id=#14] * HashAggregate (46) @@ -266,40 +266,40 @@ Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (40) Scan parquet default.store_sales -Output [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (41) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (42) Filter [codegen id : 1] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] -Condition : ((isnotnull(ss_quantity#81) AND (ss_quantity#81 >= 21)) AND (ss_quantity#81 <= 40)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) (43) Project [codegen id : 1] -Output [1]: [ss_net_paid#82] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (44) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#82] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [2]: [sum#84, count#85] -Results [2]: [sum#86, count#87] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#72, count#73] +Results [2]: [sum#74, count#75] (45) Exchange -Input [2]: [sum#86, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] +Input [2]: [sum#74, count#75] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] (46) HashAggregate [codegen id : 2] -Input [2]: [sum#86, count#87] +Input [2]: [sum#74, count#75] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#82))#89] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#82))#89 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#90] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#77] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#77 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#78] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#16, [id=#17] * HashAggregate (53) @@ -312,40 +312,40 @@ Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (47) Scan parquet default.store_sales -Output [2]: [ss_quantity#91, ss_sold_date_sk#92] +Output [2]: [ss_quantity#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (48) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (49) Filter [codegen id : 1] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] -Condition : ((isnotnull(ss_quantity#91) AND (ss_quantity#91 >= 41)) AND (ss_quantity#91 <= 60)) +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) (50) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (51) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#93] -Results [1]: [count#94] +Aggregate Attributes [1]: [count#79] +Results [1]: [count#80] (52) Exchange -Input [1]: [count#94] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] +Input [1]: [count#80] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] (53) HashAggregate [codegen id : 2] -Input [1]: [count#94] +Input [1]: [count#80] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#96] -Results [1]: [count(1)#96 AS count(1)#97] +Aggregate Attributes [1]: [count(1)#82] +Results [1]: [count(1)#82 AS count(1)#83] Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#18, [id=#19] * HashAggregate (60) @@ -358,40 +358,40 @@ Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (54) Scan parquet default.store_sales -Output [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (55) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (56) Filter [codegen id : 1] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] -Condition : ((isnotnull(ss_quantity#98) AND (ss_quantity#98 >= 41)) AND (ss_quantity#98 <= 60)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) (57) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#99] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (58) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#99] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#99))] -Aggregate Attributes [2]: [sum#101, count#102] -Results [2]: [sum#103, count#104] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#84, count#85] +Results [2]: [sum#86, count#87] (59) Exchange -Input [2]: [sum#103, count#104] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] +Input [2]: [sum#86, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] (60) HashAggregate [codegen id : 2] -Input [2]: [sum#103, count#104] +Input [2]: [sum#86, count#87] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))#106] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#99))#106 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#107] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#89] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#89 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#90] Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#20, [id=#21] * HashAggregate (67) @@ -404,40 +404,40 @@ Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (61) Scan parquet default.store_sales -Output [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (62) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (63) Filter [codegen id : 1] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] -Condition : ((isnotnull(ss_quantity#108) AND (ss_quantity#108 >= 41)) AND (ss_quantity#108 <= 60)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) (64) Project [codegen id : 1] -Output [1]: [ss_net_paid#109] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (65) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#109] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#109))] -Aggregate Attributes [2]: [sum#111, count#112] -Results [2]: [sum#113, count#114] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#91, count#92] +Results [2]: [sum#93, count#94] (66) Exchange -Input [2]: [sum#113, count#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] +Input [2]: [sum#93, count#94] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] (67) HashAggregate [codegen id : 2] -Input [2]: [sum#113, count#114] +Input [2]: [sum#93, count#94] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#109))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#109))#116] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#109))#116 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#117] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#96] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#96 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#97] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#23, [id=#24] * HashAggregate (74) @@ -450,40 +450,40 @@ Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (68) Scan parquet default.store_sales -Output [2]: [ss_quantity#118, ss_sold_date_sk#119] +Output [2]: [ss_quantity#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (69) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (70) Filter [codegen id : 1] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] -Condition : ((isnotnull(ss_quantity#118) AND (ss_quantity#118 >= 61)) AND (ss_quantity#118 <= 80)) +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) (71) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (72) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#120] -Results [1]: [count#121] +Aggregate Attributes [1]: [count#98] +Results [1]: [count#99] (73) Exchange -Input [1]: [count#121] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#122] +Input [1]: [count#99] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#100] (74) HashAggregate [codegen id : 2] -Input [1]: [count#121] +Input [1]: [count#99] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#123] -Results [1]: [count(1)#123 AS count(1)#124] +Aggregate Attributes [1]: [count(1)#101] +Results [1]: [count(1)#101 AS count(1)#102] Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#25, [id=#26] * HashAggregate (81) @@ -496,40 +496,40 @@ Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (75) Scan parquet default.store_sales -Output [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (76) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (77) Filter [codegen id : 1] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] -Condition : ((isnotnull(ss_quantity#125) AND (ss_quantity#125 >= 61)) AND (ss_quantity#125 <= 80)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) (78) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#126] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (79) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#126] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#126))] -Aggregate Attributes [2]: [sum#128, count#129] -Results [2]: [sum#130, count#131] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#103, count#104] +Results [2]: [sum#105, count#106] (80) Exchange -Input [2]: [sum#130, count#131] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#132] +Input [2]: [sum#105, count#106] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#107] (81) HashAggregate [codegen id : 2] -Input [2]: [sum#130, count#131] +Input [2]: [sum#105, count#106] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))#133] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#126))#133 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#134] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#108] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#108 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#109] Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#27, [id=#28] * HashAggregate (88) @@ -542,40 +542,40 @@ Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (82) Scan parquet default.store_sales -Output [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (83) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (84) Filter [codegen id : 1] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] -Condition : ((isnotnull(ss_quantity#135) AND (ss_quantity#135 >= 61)) AND (ss_quantity#135 <= 80)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) (85) Project [codegen id : 1] -Output [1]: [ss_net_paid#136] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (86) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#136] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#136))] -Aggregate Attributes [2]: [sum#138, count#139] -Results [2]: [sum#140, count#141] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#110, count#111] +Results [2]: [sum#112, count#113] (87) Exchange -Input [2]: [sum#140, count#141] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#142] +Input [2]: [sum#112, count#113] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] (88) HashAggregate [codegen id : 2] -Input [2]: [sum#140, count#141] +Input [2]: [sum#112, count#113] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#136))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#136))#143] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#136))#143 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#144] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#115] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#115 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#116] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#30, [id=#31] * HashAggregate (95) @@ -588,40 +588,40 @@ Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (89) Scan parquet default.store_sales -Output [2]: [ss_quantity#145, ss_sold_date_sk#146] +Output [2]: [ss_quantity#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (90) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (91) Filter [codegen id : 1] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] -Condition : ((isnotnull(ss_quantity#145) AND (ss_quantity#145 >= 81)) AND (ss_quantity#145 <= 100)) +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) (92) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (93) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#147] -Results [1]: [count#148] +Aggregate Attributes [1]: [count#117] +Results [1]: [count#118] (94) Exchange -Input [1]: [count#148] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#149] +Input [1]: [count#118] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#119] (95) HashAggregate [codegen id : 2] -Input [1]: [count#148] +Input [1]: [count#118] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#150] -Results [1]: [count(1)#150 AS count(1)#151] +Aggregate Attributes [1]: [count(1)#120] +Results [1]: [count(1)#120 AS count(1)#121] Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#32, [id=#33] * HashAggregate (102) @@ -634,40 +634,40 @@ Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (96) Scan parquet default.store_sales -Output [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (97) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (98) Filter [codegen id : 1] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] -Condition : ((isnotnull(ss_quantity#152) AND (ss_quantity#152 >= 81)) AND (ss_quantity#152 <= 100)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) (99) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#153] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (100) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#153] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#153))] -Aggregate Attributes [2]: [sum#155, count#156] -Results [2]: [sum#157, count#158] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#122, count#123] +Results [2]: [sum#124, count#125] (101) Exchange -Input [2]: [sum#157, count#158] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#159] +Input [2]: [sum#124, count#125] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#126] (102) HashAggregate [codegen id : 2] -Input [2]: [sum#157, count#158] +Input [2]: [sum#124, count#125] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))#160] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#153))#160 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#161] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#127] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#127 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#128] Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#34, [id=#35] * HashAggregate (109) @@ -680,39 +680,39 @@ Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (103) Scan parquet default.store_sales -Output [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (105) Filter [codegen id : 1] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] -Condition : ((isnotnull(ss_quantity#162) AND (ss_quantity#162 >= 81)) AND (ss_quantity#162 <= 100)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) (106) Project [codegen id : 1] -Output [1]: [ss_net_paid#163] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (107) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#163] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#163))] -Aggregate Attributes [2]: [sum#165, count#166] -Results [2]: [sum#167, count#168] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#129, count#130] +Results [2]: [sum#131, count#132] (108) Exchange -Input [2]: [sum#167, count#168] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#169] +Input [2]: [sum#131, count#132] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#133] (109) HashAggregate [codegen id : 2] -Input [2]: [sum#167, count#168] +Input [2]: [sum#131, count#132] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#163))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#163))#170] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#163))#170 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#171] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#134] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#134 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#135] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt index 5226fce4ef512..a74b232e8c667 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt @@ -183,108 +183,108 @@ Aggregate Attributes [1]: [count(1)#17] Results [1]: [count(1)#17 AS amc#18] (29) Scan parquet default.web_sales -Output [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] (31) Filter [codegen id : 8] -Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] -Condition : ((isnotnull(ws_ship_hdemo_sk#20) AND isnotnull(ws_sold_time_sk#19)) AND isnotnull(ws_web_page_sk#21)) +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) (32) Project [codegen id : 8] -Output [3]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21] -Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] (33) ReusedExchange [Reuses operator id: 9] -Output [1]: [wp_web_page_sk#23] +Output [1]: [wp_web_page_sk#5] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#21] -Right keys [1]: [wp_web_page_sk#23] +Left keys [1]: [ws_web_page_sk#3] +Right keys [1]: [wp_web_page_sk#5] Join condition: None (35) Project [codegen id : 8] -Output [2]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20] -Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, wp_web_page_sk#23] +Output [2]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2] +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, wp_web_page_sk#5] (36) ReusedExchange [Reuses operator id: 16] -Output [1]: [hd_demo_sk#24] +Output [1]: [hd_demo_sk#8] (37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_ship_hdemo_sk#20] -Right keys [1]: [hd_demo_sk#24] +Left keys [1]: [ws_ship_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#8] Join condition: None (38) Project [codegen id : 8] -Output [1]: [ws_sold_time_sk#19] -Input [3]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, hd_demo_sk#24] +Output [1]: [ws_sold_time_sk#1] +Input [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, hd_demo_sk#8] (39) Scan parquet default.time_dim -Output [2]: [t_time_sk#25, t_hour#26] +Output [2]: [t_time_sk#11, t_hour#12] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 7] -Input [2]: [t_time_sk#25, t_hour#26] +Input [2]: [t_time_sk#11, t_hour#12] (41) Filter [codegen id : 7] -Input [2]: [t_time_sk#25, t_hour#26] -Condition : (((isnotnull(t_hour#26) AND (t_hour#26 >= 19)) AND (t_hour#26 <= 20)) AND isnotnull(t_time_sk#25)) +Input [2]: [t_time_sk#11, t_hour#12] +Condition : (((isnotnull(t_hour#12) AND (t_hour#12 >= 19)) AND (t_hour#12 <= 20)) AND isnotnull(t_time_sk#11)) (42) Project [codegen id : 7] -Output [1]: [t_time_sk#25] -Input [2]: [t_time_sk#25, t_hour#26] +Output [1]: [t_time_sk#11] +Input [2]: [t_time_sk#11, t_hour#12] (43) BroadcastExchange -Input [1]: [t_time_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] +Input [1]: [t_time_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_time_sk#19] -Right keys [1]: [t_time_sk#25] +Left keys [1]: [ws_sold_time_sk#1] +Right keys [1]: [t_time_sk#11] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ws_sold_time_sk#19, t_time_sk#25] +Input [2]: [ws_sold_time_sk#1, t_time_sk#11] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [1]: [count#29] +Aggregate Attributes [1]: [count#20] +Results [1]: [count#21] (47) Exchange -Input [1]: [count#29] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#30] +Input [1]: [count#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] (48) HashAggregate [codegen id : 9] -Input [1]: [count#29] +Input [1]: [count#21] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [1]: [count(1)#31 AS pmc#32] +Aggregate Attributes [1]: [count(1)#23] +Results [1]: [count(1)#23 AS pmc#24] (49) BroadcastExchange -Input [1]: [pmc#32] -Arguments: IdentityBroadcastMode, [id=#33] +Input [1]: [pmc#24] +Arguments: IdentityBroadcastMode, [id=#25] (50) BroadcastNestedLoopJoin [codegen id : 10] Join condition: None (51) Project [codegen id : 10] -Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#32 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#34] -Input [2]: [amc#18, pmc#32] +Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#24 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#26] +Input [2]: [amc#18, pmc#24] (52) Sort [codegen id : 10] -Input [1]: [am_pm_ratio#34] -Arguments: [am_pm_ratio#34 ASC NULLS FIRST], true, 0 +Input [1]: [am_pm_ratio#26] +Arguments: [am_pm_ratio#26 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt index e369a027040d3..070b497ac4d18 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt @@ -183,108 +183,108 @@ Aggregate Attributes [1]: [count(1)#17] Results [1]: [count(1)#17 AS amc#18] (29) Scan parquet default.web_sales -Output [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] (31) Filter [codegen id : 8] -Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] -Condition : ((isnotnull(ws_ship_hdemo_sk#20) AND isnotnull(ws_sold_time_sk#19)) AND isnotnull(ws_web_page_sk#21)) +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) (32) Project [codegen id : 8] -Output [3]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21] -Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] (33) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#23] +Output [1]: [hd_demo_sk#5] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_ship_hdemo_sk#20] -Right keys [1]: [hd_demo_sk#23] +Left keys [1]: [ws_ship_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] Join condition: None (35) Project [codegen id : 8] -Output [2]: [ws_sold_time_sk#19, ws_web_page_sk#21] -Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, hd_demo_sk#23] +Output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] (36) Scan parquet default.time_dim -Output [2]: [t_time_sk#24, t_hour#25] +Output [2]: [t_time_sk#8, t_hour#9] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [2]: [t_time_sk#24, t_hour#25] +Input [2]: [t_time_sk#8, t_hour#9] (38) Filter [codegen id : 6] -Input [2]: [t_time_sk#24, t_hour#25] -Condition : (((isnotnull(t_hour#25) AND (t_hour#25 >= 19)) AND (t_hour#25 <= 20)) AND isnotnull(t_time_sk#24)) +Input [2]: [t_time_sk#8, t_hour#9] +Condition : (((isnotnull(t_hour#9) AND (t_hour#9 >= 19)) AND (t_hour#9 <= 20)) AND isnotnull(t_time_sk#8)) (39) Project [codegen id : 6] -Output [1]: [t_time_sk#24] -Input [2]: [t_time_sk#24, t_hour#25] +Output [1]: [t_time_sk#8] +Input [2]: [t_time_sk#8, t_hour#9] (40) BroadcastExchange -Input [1]: [t_time_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [t_time_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_time_sk#19] -Right keys [1]: [t_time_sk#24] +Left keys [1]: [ws_sold_time_sk#1] +Right keys [1]: [t_time_sk#8] Join condition: None (42) Project [codegen id : 8] -Output [1]: [ws_web_page_sk#21] -Input [3]: [ws_sold_time_sk#19, ws_web_page_sk#21, t_time_sk#24] +Output [1]: [ws_web_page_sk#3] +Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#8] (43) ReusedExchange [Reuses operator id: 23] -Output [1]: [wp_web_page_sk#27] +Output [1]: [wp_web_page_sk#11] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#21] -Right keys [1]: [wp_web_page_sk#27] +Left keys [1]: [ws_web_page_sk#3] +Right keys [1]: [wp_web_page_sk#11] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ws_web_page_sk#21, wp_web_page_sk#27] +Input [2]: [ws_web_page_sk#3, wp_web_page_sk#11] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [1]: [count#29] +Aggregate Attributes [1]: [count#20] +Results [1]: [count#21] (47) Exchange -Input [1]: [count#29] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#30] +Input [1]: [count#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] (48) HashAggregate [codegen id : 9] -Input [1]: [count#29] +Input [1]: [count#21] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [1]: [count(1)#31 AS pmc#32] +Aggregate Attributes [1]: [count(1)#23] +Results [1]: [count(1)#23 AS pmc#24] (49) BroadcastExchange -Input [1]: [pmc#32] -Arguments: IdentityBroadcastMode, [id=#33] +Input [1]: [pmc#24] +Arguments: IdentityBroadcastMode, [id=#25] (50) BroadcastNestedLoopJoin [codegen id : 10] Join condition: None (51) Project [codegen id : 10] -Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#32 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#34] -Input [2]: [amc#18, pmc#32] +Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#24 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#26] +Input [2]: [amc#18, pmc#24] (52) Sort [codegen id : 10] -Input [1]: [am_pm_ratio#34] -Arguments: [am_pm_ratio#34 ASC NULLS FIRST], true, 0 +Input [1]: [am_pm_ratio#26] +Arguments: [am_pm_ratio#26 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt index 3554ce5dcb117..e38c455eb6be2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt @@ -187,7 +187,7 @@ Input [5]: [cr_call_center_sk#2, cr_net_loss#3, c_current_cdemo_sk#12, c_current Output [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown )),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree ))), IsNotNull(cd_demo_sk)] +PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 5] @@ -195,7 +195,7 @@ Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] (33) Filter [codegen id : 5] Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] -Condition : ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Unknown )) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = Advanced Degree ))) AND isnotnull(cd_demo_sk#21)) +Condition : ((((cd_marital_status#22 = M) AND (cd_education_status#23 = Unknown)) OR ((cd_marital_status#22 = W) AND (cd_education_status#23 = Advanced Degree))) AND isnotnull(cd_demo_sk#21)) (34) BroadcastExchange Input [3]: [cd_demo_sk#21, cd_marital_status#22, cd_education_status#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt index fc0da4adf7b16..c14a8421bc3b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt @@ -183,7 +183,7 @@ Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_curre Output [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown )),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree ))), IsNotNull(cd_demo_sk)] +PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 5] @@ -191,7 +191,7 @@ Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] (32) Filter [codegen id : 5] Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] -Condition : ((((cd_marital_status#24 = M) AND (cd_education_status#25 = Unknown )) OR ((cd_marital_status#24 = W) AND (cd_education_status#25 = Advanced Degree ))) AND isnotnull(cd_demo_sk#23)) +Condition : ((((cd_marital_status#24 = M) AND (cd_education_status#25 = Unknown)) OR ((cd_marital_status#24 = W) AND (cd_education_status#25 = Advanced Degree))) AND isnotnull(cd_demo_sk#23)) (33) BroadcastExchange Input [3]: [cd_demo_sk#23, cd_marital_status#24, cd_education_status#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt index 94e76ced527a0..8ae9d897a7d79 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt @@ -84,7 +84,7 @@ Input [2]: [d_date_sk#8, d_date#9] (11) Filter [codegen id : 2] Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) (12) Project [codegen id : 2] Output [1]: [d_date_sk#8] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index 46953fe184f28..bae7dd2f3a240 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -108,7 +108,7 @@ Input [2]: [d_date_sk#11, d_date#12] (16) Filter [codegen id : 2] Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10983)) AND (d_date#12 <= 11073)) AND isnotnull(d_date_sk#11)) (17) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt index 2466a4d38f21e..aec81c0d5c7aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt @@ -47,7 +47,7 @@ Input [5]: [sr_item_sk#1, sr_reason_sk#2, sr_ticket_number#3, sr_return_quantity Output [2]: [r_reason_sk#6, r_reason_desc#7] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] +PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28), IsNotNull(r_reason_sk)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] @@ -55,7 +55,7 @@ Input [2]: [r_reason_sk#6, r_reason_desc#7] (7) Filter [codegen id : 1] Input [2]: [r_reason_sk#6, r_reason_desc#7] -Condition : ((isnotnull(r_reason_desc#7) AND (r_reason_desc#7 = reason 28 )) AND isnotnull(r_reason_sk#6)) +Condition : ((isnotnull(r_reason_desc#7) AND (r_reason_desc#7 = reason 28)) AND isnotnull(r_reason_sk#6)) (8) Project [codegen id : 1] Output [1]: [r_reason_sk#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index 1f3470198cd20..1875c5418a890 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -85,7 +85,7 @@ Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, s Output [2]: [r_reason_sk#14, r_reason_desc#15] Batched: true Location [not included in comparison]/{warehouse_dir}/reason] -PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] +PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28), IsNotNull(r_reason_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 5] @@ -93,7 +93,7 @@ Input [2]: [r_reason_sk#14, r_reason_desc#15] (16) Filter [codegen id : 5] Input [2]: [r_reason_sk#14, r_reason_desc#15] -Condition : ((isnotnull(r_reason_desc#15) AND (r_reason_desc#15 = reason 28 )) AND isnotnull(r_reason_sk#14)) +Condition : ((isnotnull(r_reason_desc#15) AND (r_reason_desc#15 = reason 28)) AND isnotnull(r_reason_sk#14)) (17) Project [codegen id : 5] Output [1]: [r_reason_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index 8a5b567285939..bee7110ecd6dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -77,30 +77,30 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet default.web_sales -Output [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] +Output [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] +Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] (9) Project [codegen id : 3] -Output [2]: [ws_warehouse_sk#10, ws_order_number#11] -Input [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] +Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#10, ws_order_number#5 AS ws_order_number#5#11] +Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] (10) Exchange -Input [2]: [ws_warehouse_sk#10, ws_order_number#11] -Arguments: hashpartitioning(ws_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13] +Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] +Arguments: hashpartitioning(ws_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] (11) Sort [codegen id : 4] -Input [2]: [ws_warehouse_sk#10, ws_order_number#11] -Arguments: [ws_order_number#11 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] +Arguments: [ws_order_number#5#11 ASC NULLS FIRST], false, 0 (12) SortMergeJoin Left keys [1]: [ws_order_number#5] -Right keys [1]: [ws_order_number#11] -Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#10) +Right keys [1]: [ws_order_number#5#11] +Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#10) (13) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -108,168 +108,168 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (14) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#13] (15) Sort [codegen id : 6] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: [cast(ws_order_number#5 as bigint) ASC NULLS FIRST], false, 0 (16) Scan parquet default.web_returns -Output [2]: [wr_order_number#15, wr_returned_date_sk#16] +Output [2]: [wr_order_number#14, wr_returned_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] ReadSchema: struct (17) ColumnarToRow [codegen id : 7] -Input [2]: [wr_order_number#15, wr_returned_date_sk#16] +Input [2]: [wr_order_number#14, wr_returned_date_sk#15] (18) Project [codegen id : 7] -Output [1]: [wr_order_number#15] -Input [2]: [wr_order_number#15, wr_returned_date_sk#16] +Output [1]: [wr_order_number#14] +Input [2]: [wr_order_number#14, wr_returned_date_sk#15] (19) Exchange -Input [1]: [wr_order_number#15] -Arguments: hashpartitioning(wr_order_number#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [1]: [wr_order_number#14] +Arguments: hashpartitioning(wr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] (20) Sort [codegen id : 8] -Input [1]: [wr_order_number#15] -Arguments: [wr_order_number#15 ASC NULLS FIRST], false, 0 +Input [1]: [wr_order_number#14] +Arguments: [wr_order_number#14 ASC NULLS FIRST], false, 0 (21) SortMergeJoin Left keys [1]: [cast(ws_order_number#5 as bigint)] -Right keys [1]: [wr_order_number#15] +Right keys [1]: [wr_order_number#14] Join condition: None (22) Scan parquet default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] +Output [2]: [ca_address_sk#17, ca_state#18] 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 (23) ColumnarToRow [codegen id : 9] -Input [2]: [ca_address_sk#18, ca_state#19] +Input [2]: [ca_address_sk#17, ca_state#18] (24) Filter [codegen id : 9] -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)) +Input [2]: [ca_address_sk#17, ca_state#18] +Condition : ((isnotnull(ca_state#18) AND (ca_state#18 = IL)) AND isnotnull(ca_address_sk#17)) (25) Project [codegen id : 9] -Output [1]: [ca_address_sk#18] -Input [2]: [ca_address_sk#18, ca_state#19] +Output [1]: [ca_address_sk#17] +Input [2]: [ca_address_sk#17, ca_state#18] (26) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +Input [1]: [ca_address_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (27) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#18] +Right keys [1]: [ca_address_sk#17] Join condition: None (28) Project [codegen id : 12] Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#18] +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#17] (29) Scan parquet default.web_site -Output [2]: [web_site_sk#21, web_company_name#22] +Output [2]: [web_site_sk#20, web_company_name#21] 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)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 10] -Input [2]: [web_site_sk#21, web_company_name#22] +Input [2]: [web_site_sk#20, web_company_name#21] (31) Filter [codegen id : 10] -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)) +Input [2]: [web_site_sk#20, web_company_name#21] +Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri)) AND isnotnull(web_site_sk#20)) (32) Project [codegen id : 10] -Output [1]: [web_site_sk#21] -Input [2]: [web_site_sk#21, web_company_name#22] +Output [1]: [web_site_sk#20] +Input [2]: [web_site_sk#20, web_company_name#21] (33) BroadcastExchange -Input [1]: [web_site_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [web_site_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#21] +Right keys [1]: [web_site_sk#20] Join condition: None (35) Project [codegen id : 12] Output [4]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#21] +Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#20] (36) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_date#25] +Output [2]: [d_date_sk#23, d_date#24] 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 (37) ColumnarToRow [codegen id : 11] -Input [2]: [d_date_sk#24, d_date#25] +Input [2]: [d_date_sk#23, d_date#24] (38) Filter [codegen id : 11] -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)) +Input [2]: [d_date_sk#23, d_date#24] +Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 10623)) AND (d_date#24 <= 10683)) AND isnotnull(d_date_sk#23)) (39) Project [codegen id : 11] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_date#25] +Output [1]: [d_date_sk#23] +Input [2]: [d_date_sk#23, d_date#24] (40) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [d_date_sk#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#24] +Right keys [1]: [d_date_sk#23] Join condition: None (42) Project [codegen id : 12] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [5]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#24] +Input [5]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#23] (43) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28] -Results [3]: [ws_order_number#5, sum#29, sum#30] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] +Results [3]: [ws_order_number#5, sum#28, sum#29] (44) Exchange -Input [3]: [ws_order_number#5, sum#29, sum#30] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [ws_order_number#5, sum#28, sum#29] +Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#30] (45) HashAggregate [codegen id : 13] -Input [3]: [ws_order_number#5, sum#29, sum#30] +Input [3]: [ws_order_number#5, sum#28, sum#29] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28] -Results [3]: [ws_order_number#5, sum#29, sum#30] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] +Results [3]: [ws_order_number#5, sum#28, sum#29] (46) HashAggregate [codegen id : 13] -Input [3]: [ws_order_number#5, sum#29, sum#30] +Input [3]: [ws_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28, count(ws_order_number#5)#32] -Results [3]: [sum#29, sum#30, count#33] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] +Results [3]: [sum#28, sum#29, count#32] (47) Exchange -Input [3]: [sum#29, sum#30, count#33] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [sum#28, sum#29, count#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] (48) HashAggregate [codegen id : 14] -Input [3]: [sum#29, sum#30, count#33] +Input [3]: [sum#28, sum#29, count#32] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28, count(ws_order_number#5)#32] -Results [3]: [count(ws_order_number#5)#32 AS order count #35, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#27,17,2) AS total shipping cost #36, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#28,17,2) AS total net profit #37] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] +Results [3]: [count(ws_order_number#5)#31 AS order count #34, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#26,17,2) AS total shipping cost #35, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#27,17,2) AS total net profit #36] (49) Sort [codegen id : 14] -Input [3]: [order count #35, total shipping cost #36, total net profit #37] -Arguments: [order count #35 ASC NULLS FIRST], true, 0 +Input [3]: [order count #34, total shipping cost #35, total net profit #36] +Arguments: [order count #34 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index 042d4a3e347a2..efa09c2f625cd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -77,30 +77,30 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet default.web_sales -Output [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] +Output [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] +Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] (9) Project [codegen id : 3] -Output [2]: [ws_warehouse_sk#10, ws_order_number#11] -Input [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] +Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#10, ws_order_number#5 AS ws_order_number#5#11] +Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] (10) Exchange -Input [2]: [ws_warehouse_sk#10, ws_order_number#11] -Arguments: hashpartitioning(ws_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13] +Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] +Arguments: hashpartitioning(ws_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] (11) Sort [codegen id : 4] -Input [2]: [ws_warehouse_sk#10, ws_order_number#11] -Arguments: [ws_order_number#11 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] +Arguments: [ws_order_number#5#11 ASC NULLS FIRST], false, 0 (12) SortMergeJoin Left keys [1]: [ws_order_number#5] -Right keys [1]: [ws_order_number#11] -Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#10) +Right keys [1]: [ws_order_number#5#11] +Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#10) (13) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -108,168 +108,168 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (14) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#13] (15) Sort [codegen id : 6] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: [cast(ws_order_number#5 as bigint) ASC NULLS FIRST], false, 0 (16) Scan parquet default.web_returns -Output [2]: [wr_order_number#15, wr_returned_date_sk#16] +Output [2]: [wr_order_number#14, wr_returned_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] ReadSchema: struct (17) ColumnarToRow [codegen id : 7] -Input [2]: [wr_order_number#15, wr_returned_date_sk#16] +Input [2]: [wr_order_number#14, wr_returned_date_sk#15] (18) Project [codegen id : 7] -Output [1]: [wr_order_number#15] -Input [2]: [wr_order_number#15, wr_returned_date_sk#16] +Output [1]: [wr_order_number#14] +Input [2]: [wr_order_number#14, wr_returned_date_sk#15] (19) Exchange -Input [1]: [wr_order_number#15] -Arguments: hashpartitioning(wr_order_number#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [1]: [wr_order_number#14] +Arguments: hashpartitioning(wr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] (20) Sort [codegen id : 8] -Input [1]: [wr_order_number#15] -Arguments: [wr_order_number#15 ASC NULLS FIRST], false, 0 +Input [1]: [wr_order_number#14] +Arguments: [wr_order_number#14 ASC NULLS FIRST], false, 0 (21) SortMergeJoin Left keys [1]: [cast(ws_order_number#5 as bigint)] -Right keys [1]: [wr_order_number#15] +Right keys [1]: [wr_order_number#14] Join condition: None (22) Scan parquet default.date_dim -Output [2]: [d_date_sk#18, d_date#19] +Output [2]: [d_date_sk#17, d_date#18] 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 (23) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#18, d_date#19] +Input [2]: [d_date_sk#17, d_date#18] (24) Filter [codegen id : 9] -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)) +Input [2]: [d_date_sk#17, d_date#18] +Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 10623)) AND (d_date#18 <= 10683)) AND isnotnull(d_date_sk#17)) (25) Project [codegen id : 9] -Output [1]: [d_date_sk#18] -Input [2]: [d_date_sk#18, d_date#19] +Output [1]: [d_date_sk#17] +Input [2]: [d_date_sk#17, d_date#18] (26) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +Input [1]: [d_date_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (27) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#17] Join condition: None (28) Project [codegen id : 12] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#18] +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#17] (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] +Output [2]: [ca_address_sk#20, ca_state#21] 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 (30) ColumnarToRow [codegen id : 10] -Input [2]: [ca_address_sk#21, ca_state#22] +Input [2]: [ca_address_sk#20, ca_state#21] (31) Filter [codegen id : 10] -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)) +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : ((isnotnull(ca_state#21) AND (ca_state#21 = IL)) AND isnotnull(ca_address_sk#20)) (32) Project [codegen id : 10] -Output [1]: [ca_address_sk#21] -Input [2]: [ca_address_sk#21, ca_state#22] +Output [1]: [ca_address_sk#20] +Input [2]: [ca_address_sk#20, ca_state#21] (33) BroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [ca_address_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#20] Join condition: None (35) Project [codegen id : 12] Output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#21] +Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#20] (36) Scan parquet default.web_site -Output [2]: [web_site_sk#24, web_company_name#25] +Output [2]: [web_site_sk#23, web_company_name#24] 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)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 11] -Input [2]: [web_site_sk#24, web_company_name#25] +Input [2]: [web_site_sk#23, web_company_name#24] (38) Filter [codegen id : 11] -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)) +Input [2]: [web_site_sk#23, web_company_name#24] +Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri)) AND isnotnull(web_site_sk#23)) (39) Project [codegen id : 11] -Output [1]: [web_site_sk#24] -Input [2]: [web_site_sk#24, web_company_name#25] +Output [1]: [web_site_sk#23] +Input [2]: [web_site_sk#23, web_company_name#24] (40) BroadcastExchange -Input [1]: [web_site_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [web_site_sk#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#24] +Right keys [1]: [web_site_sk#23] Join condition: None (42) Project [codegen id : 12] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#24] +Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#23] (43) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28] -Results [3]: [ws_order_number#5, sum#29, sum#30] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] +Results [3]: [ws_order_number#5, sum#28, sum#29] (44) Exchange -Input [3]: [ws_order_number#5, sum#29, sum#30] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [ws_order_number#5, sum#28, sum#29] +Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#30] (45) HashAggregate [codegen id : 13] -Input [3]: [ws_order_number#5, sum#29, sum#30] +Input [3]: [ws_order_number#5, sum#28, sum#29] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28] -Results [3]: [ws_order_number#5, sum#29, sum#30] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] +Results [3]: [ws_order_number#5, sum#28, sum#29] (46) HashAggregate [codegen id : 13] -Input [3]: [ws_order_number#5, sum#29, sum#30] +Input [3]: [ws_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28, count(ws_order_number#5)#32] -Results [3]: [sum#29, sum#30, count#33] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] +Results [3]: [sum#28, sum#29, count#32] (47) Exchange -Input [3]: [sum#29, sum#30, count#33] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [sum#28, sum#29, count#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] (48) HashAggregate [codegen id : 14] -Input [3]: [sum#29, sum#30, count#33] +Input [3]: [sum#28, sum#29, count#32] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28, count(ws_order_number#5)#32] -Results [3]: [count(ws_order_number#5)#32 AS order count #35, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#27,17,2) AS total shipping cost #36, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#28,17,2) AS total net profit #37] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] +Results [3]: [count(ws_order_number#5)#31 AS order count #34, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#26,17,2) AS total shipping cost #35, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#27,17,2) AS total net profit #36] (49) Sort [codegen id : 14] -Input [3]: [order count #35, total shipping cost #36, total net profit #37] -Arguments: [order count #35 ASC NULLS FIRST], true, 0 +Input [3]: [order count #34, total shipping cost #35, total net profit #36] +Arguments: [order count #34 ASC NULLS FIRST], true, 0 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 ee82855f8c010..725939ef24f1b 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 @@ -92,261 +92,261 @@ Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (7) Scan parquet default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] +Output [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] +Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] (9) Filter [codegen id : 3] -Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Condition : (isnotnull(ws_order_number#10) AND isnotnull(ws_warehouse_sk#9)) +Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Condition : (isnotnull(ws_order_number#4) AND isnotnull(ws_warehouse_sk#9)) (10) Project [codegen id : 3] -Output [2]: [ws_warehouse_sk#9, ws_order_number#10] -Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] +Output [2]: [ws_warehouse_sk#9, ws_order_number#4] +Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] (11) Exchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [2]: [ws_warehouse_sk#9, ws_order_number#4] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#10] (12) Sort [codegen id : 4] -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_order_number#10 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#9, ws_order_number#4] +Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (13) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#13, ws_order_number#14] +Output [2]: [ws_warehouse_sk#11, ws_order_number#12] (14) Sort [codegen id : 6] -Input [2]: [ws_warehouse_sk#13, ws_order_number#14] -Arguments: [ws_order_number#14 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#11, ws_order_number#12] +Arguments: [ws_order_number#12 ASC NULLS FIRST], false, 0 (15) SortMergeJoin [codegen id : 7] -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) +Left keys [1]: [ws_order_number#4] +Right keys [1]: [ws_order_number#12] +Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#11) (16) Project [codegen id : 7] -Output [1]: [ws_order_number#10] -Input [4]: [ws_warehouse_sk#9, ws_order_number#10, ws_warehouse_sk#13, ws_order_number#14] +Output [1]: [ws_order_number#4 AS ws_order_number#4#13] +Input [4]: [ws_warehouse_sk#9, ws_order_number#4, ws_warehouse_sk#11, ws_order_number#12] (17) SortMergeJoin Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#10] +Right keys [1]: [ws_order_number#4#13] Join condition: None (18) Exchange Input [6]: [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] -Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#15] +Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#14] (19) Sort [codegen id : 8] Input [6]: [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] Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 (20) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#16, ws_order_number#17] +Output [2]: [ws_warehouse_sk#9, ws_order_number#4] (21) Sort [codegen id : 10] -Input [2]: [ws_warehouse_sk#16, ws_order_number#17] -Arguments: [ws_order_number#17 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#9, ws_order_number#4] +Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (22) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#18, ws_order_number#19] +Output [2]: [ws_warehouse_sk#15, ws_order_number#16] (23) 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#15, ws_order_number#16] +Arguments: [ws_order_number#16 ASC NULLS FIRST], false, 0 (24) SortMergeJoin [codegen id : 13] -Left keys [1]: [ws_order_number#17] -Right keys [1]: [ws_order_number#19] -Join condition: NOT (ws_warehouse_sk#16 = ws_warehouse_sk#18) +Left keys [1]: [ws_order_number#4] +Right keys [1]: [ws_order_number#16] +Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#15) (25) Project [codegen id : 13] -Output [1]: [ws_order_number#17] -Input [4]: [ws_warehouse_sk#16, ws_order_number#17, ws_warehouse_sk#18, ws_order_number#19] +Output [1]: [ws_order_number#4] +Input [4]: [ws_warehouse_sk#9, ws_order_number#4, ws_warehouse_sk#15, ws_order_number#16] (26) Exchange -Input [1]: [ws_order_number#17] -Arguments: hashpartitioning(cast(ws_order_number#17 as bigint), 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [ws_order_number#4] +Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#17] (27) Sort [codegen id : 14] -Input [1]: [ws_order_number#17] -Arguments: [cast(ws_order_number#17 as bigint) ASC NULLS FIRST], false, 0 +Input [1]: [ws_order_number#4] +Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 (28) Scan parquet default.web_returns -Output [2]: [wr_order_number#21, wr_returned_date_sk#22] +Output [2]: [wr_order_number#18, wr_returned_date_sk#19] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct (29) ColumnarToRow [codegen id : 15] -Input [2]: [wr_order_number#21, wr_returned_date_sk#22] +Input [2]: [wr_order_number#18, wr_returned_date_sk#19] (30) Filter [codegen id : 15] -Input [2]: [wr_order_number#21, wr_returned_date_sk#22] -Condition : isnotnull(wr_order_number#21) +Input [2]: [wr_order_number#18, wr_returned_date_sk#19] +Condition : isnotnull(wr_order_number#18) (31) Project [codegen id : 15] -Output [1]: [wr_order_number#21] -Input [2]: [wr_order_number#21, wr_returned_date_sk#22] +Output [1]: [wr_order_number#18] +Input [2]: [wr_order_number#18, wr_returned_date_sk#19] (32) Exchange -Input [1]: [wr_order_number#21] -Arguments: hashpartitioning(wr_order_number#21, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [1]: [wr_order_number#18] +Arguments: hashpartitioning(wr_order_number#18, 5), ENSURE_REQUIREMENTS, [id=#20] (33) Sort [codegen id : 16] -Input [1]: [wr_order_number#21] -Arguments: [wr_order_number#21 ASC NULLS FIRST], false, 0 +Input [1]: [wr_order_number#18] +Arguments: [wr_order_number#18 ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 17] -Left keys [1]: [cast(ws_order_number#17 as bigint)] -Right keys [1]: [wr_order_number#21] +Left keys [1]: [cast(ws_order_number#4 as bigint)] +Right keys [1]: [wr_order_number#18] Join condition: None (35) Project [codegen id : 17] -Output [1]: [wr_order_number#21] -Input [2]: [ws_order_number#17, wr_order_number#21] +Output [1]: [wr_order_number#18] +Input [2]: [ws_order_number#4, wr_order_number#18] (36) SortMergeJoin Left keys [1]: [cast(ws_order_number#4 as bigint)] -Right keys [1]: [wr_order_number#21] +Right keys [1]: [wr_order_number#18] Join condition: None (37) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +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 (38) ColumnarToRow [codegen id : 18] -Input [2]: [ca_address_sk#24, ca_state#25] +Input [2]: [ca_address_sk#21, ca_state#22] (39) Filter [codegen id : 18] -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : ((isnotnull(ca_state#25) AND (ca_state#25 = IL)) AND isnotnull(ca_address_sk#24)) +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)) (40) Project [codegen id : 18] -Output [1]: [ca_address_sk#24] -Input [2]: [ca_address_sk#24, ca_state#25] +Output [1]: [ca_address_sk#21] +Input [2]: [ca_address_sk#21, ca_state#22] (41) BroadcastExchange -Input [1]: [ca_address_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [ca_address_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (42) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#24] +Right keys [1]: [ca_address_sk#21] Join condition: None (43) Project [codegen id : 21] 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#24] +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#21] (44) Scan parquet default.web_site -Output [2]: [web_site_sk#27, web_company_name#28] +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)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct (45) ColumnarToRow [codegen id : 19] -Input [2]: [web_site_sk#27, web_company_name#28] +Input [2]: [web_site_sk#24, web_company_name#25] (46) Filter [codegen id : 19] -Input [2]: [web_site_sk#27, web_company_name#28] -Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri )) AND isnotnull(web_site_sk#27)) +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)) (47) Project [codegen id : 19] -Output [1]: [web_site_sk#27] -Input [2]: [web_site_sk#27, web_company_name#28] +Output [1]: [web_site_sk#24] +Input [2]: [web_site_sk#24, web_company_name#25] (48) BroadcastExchange -Input [1]: [web_site_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] +Input [1]: [web_site_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (49) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#27] +Right keys [1]: [web_site_sk#24] Join condition: None (50) Project [codegen id : 21] 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#27] +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#24] (51) Scan parquet default.date_dim -Output [2]: [d_date_sk#30, d_date#31] +Output [2]: [d_date_sk#27, d_date#28] 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 (52) ColumnarToRow [codegen id : 20] -Input [2]: [d_date_sk#30, d_date#31] +Input [2]: [d_date_sk#27, d_date#28] (53) Filter [codegen id : 20] -Input [2]: [d_date_sk#30, d_date#31] -Condition : (((isnotnull(d_date#31) AND (d_date#31 >= 1999-02-01)) AND (d_date#31 <= 1999-04-02)) AND isnotnull(d_date_sk#30)) +Input [2]: [d_date_sk#27, d_date#28] +Condition : (((isnotnull(d_date#28) AND (d_date#28 >= 10623)) AND (d_date#28 <= 10683)) AND isnotnull(d_date_sk#27)) (54) Project [codegen id : 20] -Output [1]: [d_date_sk#30] -Input [2]: [d_date_sk#30, d_date#31] +Output [1]: [d_date_sk#27] +Input [2]: [d_date_sk#27, d_date#28] (55) BroadcastExchange -Input [1]: [d_date_sk#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (56) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#30] +Right keys [1]: [d_date_sk#27] Join condition: None (57) Project [codegen id : 21] 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#30] +Input [5]: [ws_ship_date_sk#1, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#27] (58) HashAggregate [codegen id : 21] 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))#33, sum(UnscaledValue(ws_net_profit#6))#34] -Results [3]: [ws_order_number#4, sum#35, sum#36] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#30, sum(UnscaledValue(ws_net_profit#6))#31] +Results [3]: [ws_order_number#4, sum#32, sum#33] (59) Exchange -Input [3]: [ws_order_number#4, sum#35, sum#36] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [ws_order_number#4, sum#32, sum#33] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#34] (60) HashAggregate [codegen id : 22] -Input [3]: [ws_order_number#4, sum#35, sum#36] +Input [3]: [ws_order_number#4, sum#32, sum#33] 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))#33, sum(UnscaledValue(ws_net_profit#6))#34] -Results [3]: [ws_order_number#4, sum#35, sum#36] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#30, sum(UnscaledValue(ws_net_profit#6))#31] +Results [3]: [ws_order_number#4, sum#32, sum#33] (61) HashAggregate [codegen id : 22] -Input [3]: [ws_order_number#4, sum#35, sum#36] +Input [3]: [ws_order_number#4, sum#32, sum#33] 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))#33, sum(UnscaledValue(ws_net_profit#6))#34, count(ws_order_number#4)#38] -Results [3]: [sum#35, sum#36, count#39] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#30, sum(UnscaledValue(ws_net_profit#6))#31, count(ws_order_number#4)#35] +Results [3]: [sum#32, sum#33, count#36] (62) Exchange -Input [3]: [sum#35, sum#36, count#39] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] +Input [3]: [sum#32, sum#33, count#36] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#37] (63) HashAggregate [codegen id : 23] -Input [3]: [sum#35, sum#36, count#39] +Input [3]: [sum#32, sum#33, count#36] 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))#33, sum(UnscaledValue(ws_net_profit#6))#34, count(ws_order_number#4)#38] -Results [3]: [count(ws_order_number#4)#38 AS order count #41, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#33,17,2) AS total shipping cost #42, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#34,17,2) AS total net profit #43] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#30, sum(UnscaledValue(ws_net_profit#6))#31, 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))#30,17,2) AS total shipping cost #39, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#31,17,2) AS total net profit #40] (64) Sort [codegen id : 23] -Input [3]: [order count #41, total shipping cost #42, total net profit #43] -Arguments: [order count #41 ASC NULLS FIRST], true, 0 +Input [3]: [order count #38, total shipping cost #39, total net profit #40] +Arguments: [order count #38 ASC NULLS FIRST], true, 0 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 73f7f88aeb68d..9f15375b5cfc0 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 @@ -92,261 +92,261 @@ Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (7) Scan parquet default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] +Output [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] +Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] (9) Filter [codegen id : 3] -Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] -Condition : (isnotnull(ws_order_number#10) AND isnotnull(ws_warehouse_sk#9)) +Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Condition : (isnotnull(ws_order_number#4) AND isnotnull(ws_warehouse_sk#9)) (10) Project [codegen id : 3] -Output [2]: [ws_warehouse_sk#9, ws_order_number#10] -Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] +Output [2]: [ws_warehouse_sk#9, ws_order_number#4] +Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] (11) Exchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [2]: [ws_warehouse_sk#9, ws_order_number#4] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#10] (12) Sort [codegen id : 4] -Input [2]: [ws_warehouse_sk#9, ws_order_number#10] -Arguments: [ws_order_number#10 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#9, ws_order_number#4] +Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (13) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#13, ws_order_number#14] +Output [2]: [ws_warehouse_sk#11, ws_order_number#12] (14) Sort [codegen id : 6] -Input [2]: [ws_warehouse_sk#13, ws_order_number#14] -Arguments: [ws_order_number#14 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#11, ws_order_number#12] +Arguments: [ws_order_number#12 ASC NULLS FIRST], false, 0 (15) SortMergeJoin [codegen id : 7] -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) +Left keys [1]: [ws_order_number#4] +Right keys [1]: [ws_order_number#12] +Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#11) (16) Project [codegen id : 7] -Output [1]: [ws_order_number#10] -Input [4]: [ws_warehouse_sk#9, ws_order_number#10, ws_warehouse_sk#13, ws_order_number#14] +Output [1]: [ws_order_number#4 AS ws_order_number#4#13] +Input [4]: [ws_warehouse_sk#9, ws_order_number#4, ws_warehouse_sk#11, ws_order_number#12] (17) SortMergeJoin Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#10] +Right keys [1]: [ws_order_number#4#13] Join condition: None (18) Exchange Input [6]: [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] -Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#15] +Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#14] (19) Sort [codegen id : 8] Input [6]: [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] Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 (20) Scan parquet default.web_returns -Output [2]: [wr_order_number#16, wr_returned_date_sk#17] +Output [2]: [wr_order_number#15, wr_returned_date_sk#16] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct (21) ColumnarToRow [codegen id : 9] -Input [2]: [wr_order_number#16, wr_returned_date_sk#17] +Input [2]: [wr_order_number#15, wr_returned_date_sk#16] (22) Filter [codegen id : 9] -Input [2]: [wr_order_number#16, wr_returned_date_sk#17] -Condition : isnotnull(wr_order_number#16) +Input [2]: [wr_order_number#15, wr_returned_date_sk#16] +Condition : isnotnull(wr_order_number#15) (23) Project [codegen id : 9] -Output [1]: [wr_order_number#16] -Input [2]: [wr_order_number#16, wr_returned_date_sk#17] +Output [1]: [wr_order_number#15] +Input [2]: [wr_order_number#15, wr_returned_date_sk#16] (24) Exchange -Input [1]: [wr_order_number#16] -Arguments: hashpartitioning(wr_order_number#16, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [1]: [wr_order_number#15] +Arguments: hashpartitioning(wr_order_number#15, 5), ENSURE_REQUIREMENTS, [id=#17] (25) Sort [codegen id : 10] -Input [1]: [wr_order_number#16] -Arguments: [wr_order_number#16 ASC NULLS FIRST], false, 0 +Input [1]: [wr_order_number#15] +Arguments: [wr_order_number#15 ASC NULLS FIRST], false, 0 (26) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#19, ws_order_number#20] +Output [2]: [ws_warehouse_sk#9, ws_order_number#4] (27) Sort [codegen id : 12] -Input [2]: [ws_warehouse_sk#19, ws_order_number#20] -Arguments: [ws_order_number#20 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#9, ws_order_number#4] +Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (28) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#21, ws_order_number#22] +Output [2]: [ws_warehouse_sk#18, ws_order_number#19] (29) Sort [codegen id : 14] -Input [2]: [ws_warehouse_sk#21, ws_order_number#22] -Arguments: [ws_order_number#22 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#18, ws_order_number#19] +Arguments: [ws_order_number#19 ASC NULLS FIRST], false, 0 (30) SortMergeJoin [codegen id : 15] -Left keys [1]: [ws_order_number#20] -Right keys [1]: [ws_order_number#22] -Join condition: NOT (ws_warehouse_sk#19 = ws_warehouse_sk#21) +Left keys [1]: [ws_order_number#4] +Right keys [1]: [ws_order_number#19] +Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#18) (31) Project [codegen id : 15] -Output [1]: [ws_order_number#20] -Input [4]: [ws_warehouse_sk#19, ws_order_number#20, ws_warehouse_sk#21, ws_order_number#22] +Output [1]: [ws_order_number#4] +Input [4]: [ws_warehouse_sk#9, ws_order_number#4, ws_warehouse_sk#18, ws_order_number#19] (32) Exchange -Input [1]: [ws_order_number#20] -Arguments: hashpartitioning(cast(ws_order_number#20 as bigint), 5), ENSURE_REQUIREMENTS, [id=#23] +Input [1]: [ws_order_number#4] +Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#20] (33) Sort [codegen id : 16] -Input [1]: [ws_order_number#20] -Arguments: [cast(ws_order_number#20 as bigint) ASC NULLS FIRST], false, 0 +Input [1]: [ws_order_number#4] +Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 17] -Left keys [1]: [wr_order_number#16] -Right keys [1]: [cast(ws_order_number#20 as bigint)] +Left keys [1]: [wr_order_number#15] +Right keys [1]: [cast(ws_order_number#4 as bigint)] Join condition: None (35) Project [codegen id : 17] -Output [1]: [wr_order_number#16] -Input [2]: [wr_order_number#16, ws_order_number#20] +Output [1]: [wr_order_number#15] +Input [2]: [wr_order_number#15, ws_order_number#4] (36) SortMergeJoin Left keys [1]: [cast(ws_order_number#4 as bigint)] -Right keys [1]: [wr_order_number#16] +Right keys [1]: [wr_order_number#15] Join condition: None (37) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_date#25] +Output [2]: [d_date_sk#21, d_date#22] 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 (38) ColumnarToRow [codegen id : 18] -Input [2]: [d_date_sk#24, d_date#25] +Input [2]: [d_date_sk#21, d_date#22] (39) Filter [codegen id : 18] -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)) +Input [2]: [d_date_sk#21, d_date#22] +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 10623)) AND (d_date#22 <= 10683)) AND isnotnull(d_date_sk#21)) (40) Project [codegen id : 18] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_date#25] +Output [1]: [d_date_sk#21] +Input [2]: [d_date_sk#21, d_date#22] (41) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [d_date_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (42) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#24] +Right keys [1]: [d_date_sk#21] Join condition: None (43) Project [codegen id : 21] 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#24] +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#21] (44) Scan parquet default.customer_address -Output [2]: [ca_address_sk#27, ca_state#28] +Output [2]: [ca_address_sk#24, ca_state#25] 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 (45) ColumnarToRow [codegen id : 19] -Input [2]: [ca_address_sk#27, ca_state#28] +Input [2]: [ca_address_sk#24, ca_state#25] (46) Filter [codegen id : 19] -Input [2]: [ca_address_sk#27, ca_state#28] -Condition : ((isnotnull(ca_state#28) AND (ca_state#28 = IL)) AND isnotnull(ca_address_sk#27)) +Input [2]: [ca_address_sk#24, ca_state#25] +Condition : ((isnotnull(ca_state#25) AND (ca_state#25 = IL)) AND isnotnull(ca_address_sk#24)) (47) Project [codegen id : 19] -Output [1]: [ca_address_sk#27] -Input [2]: [ca_address_sk#27, ca_state#28] +Output [1]: [ca_address_sk#24] +Input [2]: [ca_address_sk#24, ca_state#25] (48) BroadcastExchange -Input [1]: [ca_address_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] +Input [1]: [ca_address_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (49) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#27] +Right keys [1]: [ca_address_sk#24] Join condition: None (50) Project [codegen id : 21] 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#27] +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#24] (51) Scan parquet default.web_site -Output [2]: [web_site_sk#30, web_company_name#31] +Output [2]: [web_site_sk#27, web_company_name#28] 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)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 20] -Input [2]: [web_site_sk#30, web_company_name#31] +Input [2]: [web_site_sk#27, web_company_name#28] (53) Filter [codegen id : 20] -Input [2]: [web_site_sk#30, web_company_name#31] -Condition : ((isnotnull(web_company_name#31) AND (web_company_name#31 = pri )) AND isnotnull(web_site_sk#30)) +Input [2]: [web_site_sk#27, web_company_name#28] +Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri)) AND isnotnull(web_site_sk#27)) (54) Project [codegen id : 20] -Output [1]: [web_site_sk#30] -Input [2]: [web_site_sk#30, web_company_name#31] +Output [1]: [web_site_sk#27] +Input [2]: [web_site_sk#27, web_company_name#28] (55) BroadcastExchange -Input [1]: [web_site_sk#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] +Input [1]: [web_site_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (56) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#30] +Right keys [1]: [web_site_sk#27] Join condition: None (57) Project [codegen id : 21] 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#30] +Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#27] (58) HashAggregate [codegen id : 21] 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))#33, sum(UnscaledValue(ws_net_profit#6))#34] -Results [3]: [ws_order_number#4, sum#35, sum#36] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#30, sum(UnscaledValue(ws_net_profit#6))#31] +Results [3]: [ws_order_number#4, sum#32, sum#33] (59) Exchange -Input [3]: [ws_order_number#4, sum#35, sum#36] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [ws_order_number#4, sum#32, sum#33] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#34] (60) HashAggregate [codegen id : 22] -Input [3]: [ws_order_number#4, sum#35, sum#36] +Input [3]: [ws_order_number#4, sum#32, sum#33] 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))#33, sum(UnscaledValue(ws_net_profit#6))#34] -Results [3]: [ws_order_number#4, sum#35, sum#36] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#30, sum(UnscaledValue(ws_net_profit#6))#31] +Results [3]: [ws_order_number#4, sum#32, sum#33] (61) HashAggregate [codegen id : 22] -Input [3]: [ws_order_number#4, sum#35, sum#36] +Input [3]: [ws_order_number#4, sum#32, sum#33] 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))#33, sum(UnscaledValue(ws_net_profit#6))#34, count(ws_order_number#4)#38] -Results [3]: [sum#35, sum#36, count#39] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#30, sum(UnscaledValue(ws_net_profit#6))#31, count(ws_order_number#4)#35] +Results [3]: [sum#32, sum#33, count#36] (62) Exchange -Input [3]: [sum#35, sum#36, count#39] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] +Input [3]: [sum#32, sum#33, count#36] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#37] (63) HashAggregate [codegen id : 23] -Input [3]: [sum#35, sum#36, count#39] +Input [3]: [sum#32, sum#33, count#36] 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))#33, sum(UnscaledValue(ws_net_profit#6))#34, count(ws_order_number#4)#38] -Results [3]: [count(ws_order_number#4)#38 AS order count #41, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#33,17,2) AS total shipping cost #42, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#34,17,2) AS total net profit #43] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#30, sum(UnscaledValue(ws_net_profit#6))#31, 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))#30,17,2) AS total shipping cost #39, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#31,17,2) AS total net profit #40] (64) Sort [codegen id : 23] -Input [3]: [order count #41, total shipping cost #42, total net profit #43] -Arguments: [order count #41 ASC NULLS FIRST], true, 0 +Input [3]: [order count #38, total shipping cost #39, total net profit #40] +Arguments: [order count #38 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt index c01f3465ed693..245357ef5560c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt @@ -102,16 +102,16 @@ ReadSchema: struct Input [3]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13] (16) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#5] (17) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#5] Join condition: None (18) Project [codegen id : 5] Output [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#14] +Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#5] (19) HashAggregate [codegen id : 5] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] @@ -122,45 +122,45 @@ Results [2]: [cs_bill_customer_sk#11, cs_item_sk#12] (20) Exchange Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#15] +Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#14] (21) HashAggregate [codegen id : 6] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Keys [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Functions: [] Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#11 AS customer_sk#16, cs_item_sk#12 AS item_sk#17] +Results [2]: [cs_bill_customer_sk#11 AS customer_sk#15, cs_item_sk#12 AS item_sk#16] (22) Sort [codegen id : 6] -Input [2]: [customer_sk#16, item_sk#17] -Arguments: [customer_sk#16 ASC NULLS FIRST, item_sk#17 ASC NULLS FIRST], false, 0 +Input [2]: [customer_sk#15, item_sk#16] +Arguments: [customer_sk#15 ASC NULLS FIRST, item_sk#16 ASC NULLS FIRST], false, 0 (23) SortMergeJoin Left keys [2]: [customer_sk#9, item_sk#10] -Right keys [2]: [customer_sk#16, item_sk#17] +Right keys [2]: [customer_sk#15, item_sk#16] Join condition: None (24) Project [codegen id : 7] -Output [2]: [customer_sk#9, customer_sk#16] -Input [4]: [customer_sk#9, item_sk#10, customer_sk#16, item_sk#17] +Output [2]: [customer_sk#9, customer_sk#15] +Input [4]: [customer_sk#9, item_sk#10, customer_sk#15, item_sk#16] (25) HashAggregate [codegen id : 7] -Input [2]: [customer_sk#9, customer_sk#16] +Input [2]: [customer_sk#9, customer_sk#15] Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum#18, sum#19, sum#20] -Results [3]: [sum#21, sum#22, sum#23] +Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum#17, sum#18, sum#19] +Results [3]: [sum#20, sum#21, sum#22] (26) Exchange -Input [3]: [sum#21, sum#22, sum#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [sum#20, sum#21, sum#22] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#23] (27) HashAggregate [codegen id : 8] -Input [3]: [sum#21, sum#22, sum#23] +Input [3]: [sum#20, sum#21, sum#22] Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27] -Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25 AS store_only#28, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26 AS catalog_only#29, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27 AS store_and_catalog#30] +Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26] +Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24 AS store_only#27, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25 AS catalog_only#28, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26 AS store_and_catalog#29] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt index c01f3465ed693..245357ef5560c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt @@ -102,16 +102,16 @@ ReadSchema: struct Input [3]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13] (16) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#5] (17) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#5] Join condition: None (18) Project [codegen id : 5] Output [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#14] +Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#5] (19) HashAggregate [codegen id : 5] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] @@ -122,45 +122,45 @@ Results [2]: [cs_bill_customer_sk#11, cs_item_sk#12] (20) Exchange Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#15] +Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#14] (21) HashAggregate [codegen id : 6] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Keys [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Functions: [] Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#11 AS customer_sk#16, cs_item_sk#12 AS item_sk#17] +Results [2]: [cs_bill_customer_sk#11 AS customer_sk#15, cs_item_sk#12 AS item_sk#16] (22) Sort [codegen id : 6] -Input [2]: [customer_sk#16, item_sk#17] -Arguments: [customer_sk#16 ASC NULLS FIRST, item_sk#17 ASC NULLS FIRST], false, 0 +Input [2]: [customer_sk#15, item_sk#16] +Arguments: [customer_sk#15 ASC NULLS FIRST, item_sk#16 ASC NULLS FIRST], false, 0 (23) SortMergeJoin Left keys [2]: [customer_sk#9, item_sk#10] -Right keys [2]: [customer_sk#16, item_sk#17] +Right keys [2]: [customer_sk#15, item_sk#16] Join condition: None (24) Project [codegen id : 7] -Output [2]: [customer_sk#9, customer_sk#16] -Input [4]: [customer_sk#9, item_sk#10, customer_sk#16, item_sk#17] +Output [2]: [customer_sk#9, customer_sk#15] +Input [4]: [customer_sk#9, item_sk#10, customer_sk#15, item_sk#16] (25) HashAggregate [codegen id : 7] -Input [2]: [customer_sk#9, customer_sk#16] +Input [2]: [customer_sk#9, customer_sk#15] Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum#18, sum#19, sum#20] -Results [3]: [sum#21, sum#22, sum#23] +Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum#17, sum#18, sum#19] +Results [3]: [sum#20, sum#21, sum#22] (26) Exchange -Input [3]: [sum#21, sum#22, sum#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [sum#20, sum#21, sum#22] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#23] (27) HashAggregate [codegen id : 8] -Input [3]: [sum#21, sum#22, sum#23] +Input [3]: [sum#20, sum#21, sum#22] Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27] -Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25 AS store_only#28, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26 AS catalog_only#29, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27 AS store_and_catalog#30] +Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26] +Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24 AS store_only#27, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25 AS catalog_only#28, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26 AS store_and_catalog#29] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt index 98c1bc5671cd9..7b8e4ff5516a7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt @@ -57,7 +57,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -88,7 +88,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -96,7 +96,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt index 1507b5ccbc0ae..3457f398b7dea 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt @@ -46,7 +46,7 @@ Condition : isnotnull(ss_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -54,7 +54,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -81,7 +81,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt index 2e710b7eda4b3..58d28b28dae10 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt @@ -137,52 +137,52 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#8] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] +Right keys [1]: [d_date_sk#8] Join condition: None (22) Project [codegen id : 7] -Output [1]: [ws_bill_customer_sk#13 AS customer_sk#16] -Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] +Output [1]: [ws_bill_customer_sk#13 AS customer_sk#15] +Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] (23) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] ReadSchema: struct (24) ColumnarToRow [codegen id : 9] -Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] (25) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#19] +Output [1]: [d_date_sk#8] (26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#19] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#8] Join condition: None (27) Project [codegen id : 9] -Output [1]: [cs_ship_customer_sk#17 AS customer_sk#20] -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] +Output [1]: [cs_ship_customer_sk#16 AS customer_sk#18] +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] (28) Union (29) Exchange -Input [1]: [customer_sk#16] -Arguments: hashpartitioning(customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [1]: [customer_sk#15] +Arguments: hashpartitioning(customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#19] (30) Sort [codegen id : 10] -Input [1]: [customer_sk#16] -Arguments: [customer_sk#16 ASC NULLS FIRST], false, 0 +Input [1]: [customer_sk#15] +Arguments: [customer_sk#15 ASC NULLS FIRST], false, 0 (31) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#16] +Right keys [1]: [customer_sk#15] Join condition: None (32) Project [codegen id : 12] @@ -190,84 +190,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (33) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_county#23] +Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -Input [2]: [ca_address_sk#22, ca_county#23] +Input [2]: [ca_address_sk#20, ca_county#21] (35) Filter [codegen id : 11] -Input [2]: [ca_address_sk#22, ca_county#23] -Condition : (ca_county#23 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#22)) +Input [2]: [ca_address_sk#20, ca_county#21] +Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) (36) Project [codegen id : 11] -Output [1]: [ca_address_sk#22] -Input [2]: [ca_address_sk#22, ca_county#23] +Output [1]: [ca_address_sk#20] +Input [2]: [ca_address_sk#20, ca_county#21] (37) 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#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (38) BroadcastHashJoin [codegen id : 12] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#20] Join condition: None (39) Project [codegen id : 12] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#22] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] (40) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (41) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Output [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (42) ColumnarToRow -Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] (43) Filter -Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] -Condition : isnotnull(cd_demo_sk#26) +Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Condition : isnotnull(cd_demo_sk#24) (44) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#26] +Right keys [1]: [cd_demo_sk#24] Join condition: None (45) Project [codegen id : 13] -Output [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Output [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] (46) HashAggregate [codegen id : 13] -Input [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] -Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Input [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#35] -Results [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] +Aggregate Attributes [1]: [count#33] +Results [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] (47) Exchange -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] -Arguments: hashpartitioning(cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] +Arguments: hashpartitioning(cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#35] (48) HashAggregate [codegen id : 14] -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] -Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] +Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, count(1)#38 AS cnt1#39, cd_purchase_estimate#30, count(1)#38 AS cnt2#40, cd_credit_rating#31, count(1)#38 AS cnt3#41, cd_dep_count#32, count(1)#38 AS cnt4#42, cd_dep_employed_count#33, count(1)#38 AS cnt5#43, cd_dep_college_count#34, count(1)#38 AS cnt6#44] +Aggregate Attributes [1]: [count(1)#36] +Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count(1)#36 AS cnt1#37, cd_purchase_estimate#28, count(1)#36 AS cnt2#38, cd_credit_rating#29, count(1)#36 AS cnt3#39, cd_dep_count#30, count(1)#36 AS cnt4#40, cd_dep_employed_count#31, count(1)#36 AS cnt5#41, cd_dep_college_count#32, count(1)#36 AS cnt6#42] (49) TakeOrderedAndProject -Input [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#39, cd_purchase_estimate#30, cnt2#40, cd_credit_rating#31, cnt3#41, cd_dep_count#32, cnt4#42, cd_dep_employed_count#33, cnt5#43, cd_dep_college_count#34, cnt6#44] -Arguments: 100, [cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_education_status#29 ASC NULLS FIRST, cd_purchase_estimate#30 ASC NULLS FIRST, cd_credit_rating#31 ASC NULLS FIRST, cd_dep_count#32 ASC NULLS FIRST, cd_dep_employed_count#33 ASC NULLS FIRST, cd_dep_college_count#34 ASC NULLS FIRST], [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#39, cd_purchase_estimate#30, cnt2#40, cd_credit_rating#31, cnt3#41, cd_dep_count#32, cnt4#42, cd_dep_employed_count#33, cnt5#43, cd_dep_college_count#34, cnt6#44] +Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] +Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] ===== Subqueries ===== @@ -280,6 +280,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index 62054fe96dd5c..eac6acc3543ca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -121,48 +121,48 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#7] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#7] Join condition: None (19) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#12 AS customer_sk#15] -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] +Output [1]: [ws_bill_customer_sk#12 AS customer_sk#14] +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] (20) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] (22) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#7] (23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#18] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#7] Join condition: None (24) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#16 AS customer_sk#19] -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#18] +Output [1]: [cs_ship_customer_sk#15 AS customer_sk#17] +Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] (25) Union (26) BroadcastExchange -Input [1]: [customer_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +Input [1]: [customer_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (27) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#15] +Right keys [1]: [customer_sk#14] Join condition: None (28) Project [codegen id : 9] @@ -170,84 +170,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_county#22] +Output [2]: [ca_address_sk#19, ca_county#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#21, ca_county#22] +Input [2]: [ca_address_sk#19, ca_county#20] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#21, ca_county#22] -Condition : (ca_county#22 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#21)) +Input [2]: [ca_address_sk#19, ca_county#20] +Condition : (ca_county#20 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#19)) (32) Project [codegen id : 7] -Output [1]: [ca_address_sk#21] -Input [2]: [ca_address_sk#21, ca_county#22] +Output [1]: [ca_address_sk#19] +Input [2]: [ca_address_sk#19, ca_county#20] (33) BroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [ca_address_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (34) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#19] Join condition: None (35) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] (36) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (38) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Condition : isnotnull(cd_demo_sk#24) +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#22) (39) BroadcastExchange -Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#24] +Right keys [1]: [cd_demo_sk#22] Join condition: None (41) Project [codegen id : 9] -Output [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Output [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (42) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#34] -Results [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#35] +Aggregate Attributes [1]: [count#32] +Results [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] (43) Exchange -Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#35] -Arguments: hashpartitioning(cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] +Arguments: hashpartitioning(cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#34] (44) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#35] -Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] +Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#37] -Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count(1)#37 AS cnt1#38, cd_purchase_estimate#28, count(1)#37 AS cnt2#39, cd_credit_rating#29, count(1)#37 AS cnt3#40, cd_dep_count#30, count(1)#37 AS cnt4#41, cd_dep_employed_count#31, count(1)#37 AS cnt5#42, cd_dep_college_count#32, count(1)#37 AS cnt6#43] +Aggregate Attributes [1]: [count(1)#35] +Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(1)#35 AS cnt1#36, cd_purchase_estimate#26, count(1)#35 AS cnt2#37, cd_credit_rating#27, count(1)#35 AS cnt3#38, cd_dep_count#28, count(1)#35 AS cnt4#39, cd_dep_employed_count#29, count(1)#35 AS cnt5#40, cd_dep_college_count#30, count(1)#35 AS cnt6#41] (45) TakeOrderedAndProject -Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#38, cd_purchase_estimate#28, cnt2#39, cd_credit_rating#29, cnt3#40, cd_dep_count#30, cnt4#41, cd_dep_employed_count#31, cnt5#42, cd_dep_college_count#32, cnt6#43] -Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#38, cd_purchase_estimate#28, cnt2#39, cd_credit_rating#29, cnt3#40, cd_dep_count#30, cnt4#41, cd_dep_employed_count#31, cnt5#42, cd_dep_college_count#32, cnt6#43] +Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] +Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] ===== Subqueries ===== @@ -260,6 +260,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 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 453a35d8d5175..6eb3be35a05bc 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 @@ -199,285 +199,285 @@ Input [2]: [customer_id#23, year_total#24] Arguments: [customer_id#23 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] +Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] (27) Filter [codegen id : 10] -Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] -Condition : isnotnull(ss_customer_sk#26) +Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] +Condition : isnotnull(ss_customer_sk#1) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#31, d_year#32] +Output [2]: [d_date_sk#6, d_year#7] 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 (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#31, d_year#32] +Input [2]: [d_date_sk#6, d_year#7] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#31, d_year#32] -Condition : ((isnotnull(d_year#32) AND (d_year#32 = 2002)) AND isnotnull(d_date_sk#31)) +Input [2]: [d_date_sk#6, d_year#7] +Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) (31) BroadcastExchange -Input [2]: [d_date_sk#31, d_year#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +Input [2]: [d_date_sk#6, d_year#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#29] -Right keys [1]: [d_date_sk#31] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#6] Join condition: None (33) Project [codegen id : 10] -Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Input [6]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29, d_date_sk#31, d_year#32] +Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4, d_date_sk#6, d_year#7] (34) Exchange -Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#28] (35) Sort [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (37) Sort [codegen id : 13] -Input [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] -Arguments: [c_customer_sk#35 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#35] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#10] Join condition: None (39) Project [codegen id : 14] -Output [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Input [12]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32, c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Input [12]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (40) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#43] -Results [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] +Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#29] +Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] (41) Exchange -Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] +Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, 5), ENSURE_REQUIREMENTS, [id=#31] (42) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] -Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46] -Results [5]: [c_customer_id#36 AS customer_id#47, c_first_name#37 AS customer_first_name#48, c_last_name#38 AS customer_last_name#49, c_email_address#42 AS customer_email_address#50, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46,18,2) AS year_total#51] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32] +Results [5]: [c_customer_id#11 AS customer_id#33, c_first_name#12 AS customer_first_name#34, c_last_name#13 AS customer_last_name#35, c_email_address#17 AS customer_email_address#36, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32,18,2) AS year_total#37] (43) Exchange -Input [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] -Arguments: hashpartitioning(customer_id#47, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [5]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37] +Arguments: hashpartitioning(customer_id#33, 5), ENSURE_REQUIREMENTS, [id=#38] (44) Sort [codegen id : 16] -Input [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] -Arguments: [customer_id#47 ASC NULLS FIRST], false, 0 +Input [5]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37] +Arguments: [customer_id#33 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#47] +Right keys [1]: [customer_id#33] Join condition: None (46) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56] +Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#56), dynamicpruningexpression(ws_sold_date_sk#56 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56] +Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] (48) Filter [codegen id : 19] -Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56] -Condition : isnotnull(ws_bill_customer_sk#53) +Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] +Condition : isnotnull(ws_bill_customer_sk#39) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#57, d_year#58] +Output [2]: [d_date_sk#6, d_year#7] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#56] -Right keys [1]: [d_date_sk#57] +Left keys [1]: [ws_sold_date_sk#42] +Right keys [1]: [d_date_sk#6] Join condition: None (51) Project [codegen id : 19] -Output [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] -Input [6]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56, d_date_sk#57, d_year#58] +Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Input [6]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42, d_date_sk#6, d_year#7] (52) Exchange -Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] -Arguments: hashpartitioning(ws_bill_customer_sk#53, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Arguments: hashpartitioning(ws_bill_customer_sk#39, 5), ENSURE_REQUIREMENTS, [id=#43] (53) Sort [codegen id : 20] -Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] -Arguments: [ws_bill_customer_sk#53 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Arguments: [ws_bill_customer_sk#39 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -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#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (55) Sort [codegen id : 22] -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] -Arguments: [c_customer_sk#60 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#53] -Right keys [1]: [c_customer_sk#60] +Left keys [1]: [ws_bill_customer_sk#39] +Right keys [1]: [c_customer_sk#10] Join condition: None (57) Project [codegen id : 23] -Output [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, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] -Input [12]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58, 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 [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Input [12]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (58) HashAggregate [codegen id : 23] -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, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] -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#58] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#68] -Results [9]: [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#58, sum#69] +Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#44] +Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#45] (59) Exchange -Input [9]: [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#58, sum#69] -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#58, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#45] +Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#46] (60) HashAggregate [codegen id : 24] -Input [9]: [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#58, sum#69] -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#58] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))#71] -Results [2]: [c_customer_id#61 AS customer_id#72, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))#71,18,2) AS year_total#73] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#45] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#47] +Results [2]: [c_customer_id#11 AS customer_id#48, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#47,18,2) AS year_total#49] (61) Filter [codegen id : 24] -Input [2]: [customer_id#72, year_total#73] -Condition : (isnotnull(year_total#73) AND (year_total#73 > 0.00)) +Input [2]: [customer_id#48, year_total#49] +Condition : (isnotnull(year_total#49) AND (year_total#49 > 0.00)) (62) Project [codegen id : 24] -Output [2]: [customer_id#72 AS customer_id#74, year_total#73 AS year_total#75] -Input [2]: [customer_id#72, year_total#73] +Output [2]: [customer_id#48 AS customer_id#50, year_total#49 AS year_total#51] +Input [2]: [customer_id#48, year_total#49] (63) Exchange -Input [2]: [customer_id#74, year_total#75] -Arguments: hashpartitioning(customer_id#74, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [2]: [customer_id#50, year_total#51] +Arguments: hashpartitioning(customer_id#50, 5), ENSURE_REQUIREMENTS, [id=#52] (64) Sort [codegen id : 25] -Input [2]: [customer_id#74, year_total#75] -Arguments: [customer_id#74 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#50, year_total#51] +Arguments: [customer_id#50 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#74] +Right keys [1]: [customer_id#50] Join condition: None (66) Project [codegen id : 26] -Output [8]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75] -Input [9]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, customer_id#74, year_total#75] +Output [8]: [customer_id#23, year_total#24, customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37, year_total#51] +Input [9]: [customer_id#23, year_total#24, customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37, customer_id#50, year_total#51] (67) 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#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -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#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] (69) Filter [codegen id : 28] -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#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] +Condition : isnotnull(ws_bill_customer_sk#39) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#81, d_year#82] +Output [2]: [d_date_sk#6, d_year#7] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#80] -Right keys [1]: [d_date_sk#81] +Left keys [1]: [ws_sold_date_sk#42] +Right keys [1]: [d_date_sk#6] Join condition: None (72) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] -Input [6]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80, d_date_sk#81, d_year#82] +Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Input [6]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42, d_date_sk#6, d_year#7] (73) Exchange -Input [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] -Arguments: hashpartitioning(ws_bill_customer_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Arguments: hashpartitioning(ws_bill_customer_sk#39, 5), ENSURE_REQUIREMENTS, [id=#53] (74) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] -Arguments: [ws_bill_customer_sk#77 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Arguments: [ws_bill_customer_sk#39 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#84, c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91] +Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (76) Sort [codegen id : 31] -Input [8]: [c_customer_sk#84, c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91] -Arguments: [c_customer_sk#84 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#77] -Right keys [1]: [c_customer_sk#84] +Left keys [1]: [ws_bill_customer_sk#39] +Right keys [1]: [c_customer_sk#10] Join condition: None (78) Project [codegen id : 32] -Output [10]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] -Input [12]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82, c_customer_sk#84, c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91] +Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Input [12]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (79) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] -Keys [8]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82] -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#92] -Results [9]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, sum#93] +Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#54] +Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#55] (80) Exchange -Input [9]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, sum#93] -Arguments: hashpartitioning(c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, 5), ENSURE_REQUIREMENTS, [id=#94] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#55] +Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#56] (81) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, sum#93] -Keys [8]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82] -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)))#95] -Results [2]: [c_customer_id#85 AS customer_id#96, 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)))#95,18,2) AS year_total#97] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#55] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#57] +Results [2]: [c_customer_id#11 AS customer_id#58, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#57,18,2) AS year_total#59] (82) Exchange -Input [2]: [customer_id#96, year_total#97] -Arguments: hashpartitioning(customer_id#96, 5), ENSURE_REQUIREMENTS, [id=#98] +Input [2]: [customer_id#58, year_total#59] +Arguments: hashpartitioning(customer_id#58, 5), ENSURE_REQUIREMENTS, [id=#60] (83) Sort [codegen id : 34] -Input [2]: [customer_id#96, year_total#97] -Arguments: [customer_id#96 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#58, year_total#59] +Arguments: [customer_id#58 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#96] -Join condition: (CASE WHEN (year_total#75 > 0.00) THEN CheckOverflow((promote_precision(year_total#97) / promote_precision(year_total#75)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#51) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE 0E-20 END) +Right keys [1]: [customer_id#58] +Join condition: (CASE WHEN (year_total#51 > 0.00) THEN CheckOverflow((promote_precision(year_total#59) / promote_precision(year_total#51)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#37) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE 0E-20 END) (85) Project [codegen id : 35] -Output [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] -Input [10]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75, customer_id#96, year_total#97] +Output [4]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36] +Input [10]: [customer_id#23, year_total#24, customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37, year_total#51, customer_id#58, year_total#59] (86) TakeOrderedAndProject -Input [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] -Arguments: 100, [customer_id#47 ASC NULLS FIRST, customer_first_name#48 ASC NULLS FIRST, customer_last_name#49 ASC NULLS FIRST, customer_email_address#50 ASC NULLS FIRST], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] +Input [4]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36] +Arguments: 100, [customer_id#33 ASC NULLS FIRST, customer_first_name#34 ASC NULLS FIRST, customer_last_name#35 ASC NULLS FIRST, customer_email_address#36 ASC NULLS FIRST], [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36] ===== Subqueries ===== @@ -488,15 +488,15 @@ ReusedExchange (87) (87) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#6, d_year#7] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#30 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#26 ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#31, d_year#32] +Output [2]: [d_date_sk#6, d_year#7] -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#56 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#30 +Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#26 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 17d16033bb5be..8f7b2faaa1220 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 @@ -171,282 +171,282 @@ Input [2]: [customer_id#22, year_total#23] Condition : (isnotnull(year_total#23) AND (year_total#23 > 0.00)) (20) Scan parquet default.customer -Output [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] -Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_customer_id#25)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (23) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] 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#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] (25) Filter [codegen id : 4] -Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] -Condition : isnotnull(ss_customer_sk#32) +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_customer_sk#9) (26) BroadcastExchange -Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#24] -Right keys [1]: [ss_customer_sk#32] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#9] Join condition: None (28) Project [codegen id : 6] -Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] -Input [12]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#38, d_year#39] +Output [2]: [d_date_sk#15, d_year#16] 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 (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#38, d_year#39] +Input [2]: [d_date_sk#15, d_year#16] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) +Input [2]: [d_date_sk#15, d_year#16] +Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2002)) AND isnotnull(d_date_sk#15)) (32) BroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] +Input [2]: [d_date_sk#15, d_year#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#35] -Right keys [1]: [d_date_sk#38] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#15] Join condition: None (34) Project [codegen id : 6] -Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] -Input [12]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#38, d_year#39] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#15, d_year#16] (35) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] -Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#41] -Results [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#27] +Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] (36) Exchange -Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] -Arguments: hashpartitioning(c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [id=#29] (37) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] -Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44] -Results [5]: [c_customer_id#25 AS customer_id#45, c_first_name#26 AS customer_first_name#46, c_last_name#27 AS customer_last_name#47, c_email_address#31 AS customer_email_address#48, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44,18,2) AS year_total#49] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30] +Results [5]: [c_customer_id#2 AS customer_id#31, c_first_name#3 AS customer_first_name#32, c_last_name#4 AS customer_last_name#33, c_email_address#8 AS customer_email_address#34, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30,18,2) AS year_total#35] (38) BroadcastExchange -Input [5]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] +Input [5]: [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#36] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#45] +Right keys [1]: [customer_id#31] Join condition: None (40) Scan parquet default.customer -Output [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (42) Filter [codegen id : 10] -Input [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] -Condition : (isnotnull(c_customer_sk#51) AND isnotnull(c_customer_id#52)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (43) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] +Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] (45) Filter [codegen id : 8] -Input [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] -Condition : isnotnull(ws_bill_customer_sk#59) +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Condition : isnotnull(ws_bill_customer_sk#37) (46) BroadcastExchange -Input [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#41] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#51] -Right keys [1]: [ws_bill_customer_sk#59] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#37] Join condition: None (48) Project [codegen id : 10] -Output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] -Input [12]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#64, d_year#65] +Output [2]: [d_date_sk#15, d_year#16] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#62] -Right keys [1]: [d_date_sk#64] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#15] Join condition: None (51) Project [codegen id : 10] -Output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, d_year#65] -Input [12]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62, d_date_sk#64, d_year#65] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#15, d_year#16] (52) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, d_year#65] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#66] -Results [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, sum#67] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#42] +Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#43] (53) Exchange -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, sum#67] -Arguments: hashpartitioning(c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#43] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#44] (54) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, sum#67] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))#69] -Results [2]: [c_customer_id#52 AS customer_id#70, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))#69,18,2) AS year_total#71] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#43] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45] +Results [2]: [c_customer_id#2 AS customer_id#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45,18,2) AS year_total#47] (55) Filter [codegen id : 11] -Input [2]: [customer_id#70, year_total#71] -Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.00)) +Input [2]: [customer_id#46, year_total#47] +Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) (56) Project [codegen id : 11] -Output [2]: [customer_id#70 AS customer_id#72, year_total#71 AS year_total#73] -Input [2]: [customer_id#70, year_total#71] +Output [2]: [customer_id#46 AS customer_id#48, year_total#47 AS year_total#49] +Input [2]: [customer_id#46, year_total#47] (57) BroadcastExchange -Input [2]: [customer_id#72, year_total#73] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#74] +Input [2]: [customer_id#48, year_total#49] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] (58) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#72] +Right keys [1]: [customer_id#48] Join condition: None (59) Project [codegen id : 16] -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#73] -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#72, year_total#73] +Output [8]: [customer_id#22, year_total#23, customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35, year_total#49] +Input [9]: [customer_id#22, year_total#23, customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35, customer_id#48, year_total#49] (60) Scan parquet default.customer -Output [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (62) Filter [codegen id : 14] -Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] -Condition : (isnotnull(c_customer_sk#75) AND isnotnull(c_customer_id#76)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (63) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] (65) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] -Condition : isnotnull(ws_bill_customer_sk#83) +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Condition : isnotnull(ws_bill_customer_sk#37) (66) BroadcastExchange -Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#87] +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#75] -Right keys [1]: [ws_bill_customer_sk#83] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#37] Join condition: None (68) Project [codegen id : 14] -Output [10]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] -Input [12]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#88, d_year#89] +Output [2]: [d_date_sk#15, d_year#16] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#86] -Right keys [1]: [d_date_sk#88] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#15] Join condition: None (71) Project [codegen id : 14] -Output [10]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] -Input [12]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86, d_date_sk#88, d_year#89] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#15, d_year#16] (72) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] -Keys [8]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#90] -Results [9]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, sum#91] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#52] +Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#53] (73) Exchange -Input [9]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, sum#91] -Arguments: hashpartitioning(c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#53] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#54] (74) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, sum#91] -Keys [8]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))#93] -Results [2]: [c_customer_id#76 AS customer_id#94, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))#93,18,2) AS year_total#95] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#53] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55] +Results [2]: [c_customer_id#2 AS customer_id#56, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55,18,2) AS year_total#57] (75) BroadcastExchange -Input [2]: [customer_id#94, year_total#95] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#96] +Input [2]: [customer_id#56, year_total#57] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#58] (76) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#94] -Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#95) / promote_precision(year_total#73)), 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#56] +Join condition: (CASE WHEN (year_total#49 > 0.00) THEN CheckOverflow((promote_precision(year_total#57) / promote_precision(year_total#49)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#35) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE 0E-20 END) (77) Project [codegen id : 16] -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#73, customer_id#94, year_total#95] +Output [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34] +Input [10]: [customer_id#22, year_total#23, customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35, year_total#49, customer_id#56, year_total#57] (78) 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#31, customer_first_name#32, customer_last_name#33, customer_email_address#34] +Arguments: 100, [customer_id#31 ASC NULLS FIRST, customer_first_name#32 ASC NULLS FIRST, customer_last_name#33 ASC NULLS FIRST, customer_email_address#34 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34] ===== Subqueries ===== @@ -457,15 +457,15 @@ ReusedExchange (79) (79) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#15, d_year#16] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#24 ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#38, d_year#39] +Output [2]: [d_date_sk#15, d_year#16] -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#36 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt index dd85594160c8f..15c445b6b1325 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt @@ -55,7 +55,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -86,7 +86,7 @@ Arguments: [ws_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -94,7 +94,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt index f6dd412f5c4b1..774081bc5c3f6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt @@ -44,7 +44,7 @@ Condition : isnotnull(ws_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -52,7 +52,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -79,7 +79,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt index 25baf22f2b4b5..b19d4f3aeae0f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt @@ -155,499 +155,499 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (10) ColumnarToRow [codegen id : 10] -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] (11) Filter [codegen id : 10] -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_item_sk#11) +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#12, d_year#13] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#14, d_year#15] +Input [2]: [d_date_sk#12, d_year#13] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#14, d_year#15] -Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1998)) AND (d_year#15 <= 2000)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_year#15] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (16) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (17) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (18) Project [codegen id : 10] -Output [1]: [ss_item_sk#11] -Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] +Output [1]: [ss_item_sk#1] +Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] (19) Scan parquet default.item -Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Condition : (((isnotnull(i_item_sk#15) AND isnotnull(i_brand_id#16)) AND isnotnull(i_class_id#17)) AND isnotnull(i_category_id#18)) (22) Exchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: hashpartitioning(coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] (23) Sort [codegen id : 5] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: [coalesce(i_brand_id#16, 0) ASC NULLS FIRST, isnull(i_brand_id#16) ASC NULLS FIRST, coalesce(i_class_id#17, 0) ASC NULLS FIRST, isnull(i_class_id#17) ASC NULLS FIRST, coalesce(i_category_id#18, 0) ASC NULLS FIRST, isnull(i_category_id#18) ASC NULLS FIRST], false, 0 (24) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 8] -Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] (26) Filter [codegen id : 8] -Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] -Condition : isnotnull(cs_item_sk#22) +Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +Condition : isnotnull(cs_item_sk#20) (27) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#24] +Output [1]: [d_date_sk#22] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#22] Join condition: None (29) Project [codegen id : 8] -Output [1]: [cs_item_sk#22] -Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] +Output [1]: [cs_item_sk#20] +Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#22] (30) Scan parquet default.item -Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Output [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] (32) Filter [codegen id : 7] -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -Condition : isnotnull(i_item_sk#25) +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Condition : isnotnull(i_item_sk#23) (33) BroadcastExchange -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#22] -Right keys [1]: [i_item_sk#25] +Left keys [1]: [cs_item_sk#20] +Right keys [1]: [i_item_sk#23] Join condition: None (35) Project [codegen id : 8] -Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Output [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Input [5]: [cs_item_sk#20, i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] (36) Exchange -Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: hashpartitioning(coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26), 5), ENSURE_REQUIREMENTS, [id=#28] (37) Sort [codegen id : 9] -Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 +Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: [coalesce(i_brand_id#24, 0) ASC NULLS FIRST, isnull(i_brand_id#24) ASC NULLS FIRST, coalesce(i_class_id#25, 0) ASC NULLS FIRST, isnull(i_class_id#25) ASC NULLS FIRST, coalesce(i_category_id#26, 0) ASC NULLS FIRST, isnull(i_category_id#26) ASC NULLS FIRST], false, 0 (38) SortMergeJoin -Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] -Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] +Left keys [6]: [coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18)] +Right keys [6]: [coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26)] Join condition: None (39) BroadcastExchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#11] -Right keys [1]: [i_item_sk#17] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#15] Join condition: None (41) Project [codegen id : 10] -Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] -Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [3]: [i_brand_id#16 AS brand_id#30, i_class_id#17 AS class_id#31, i_category_id#18 AS category_id#32] +Input [5]: [ss_item_sk#1, i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] (42) HashAggregate [codegen id : 10] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#30, class_id#31, category_id#32] (43) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] (44) HashAggregate [codegen id : 11] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#30, class_id#31, category_id#32] (45) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32), 5), ENSURE_REQUIREMENTS, [id=#34] (46) Sort [codegen id : 12] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: [coalesce(brand_id#30, 0) ASC NULLS FIRST, isnull(brand_id#30) ASC NULLS FIRST, coalesce(class_id#31, 0) ASC NULLS FIRST, isnull(class_id#31) ASC NULLS FIRST, coalesce(category_id#32, 0) ASC NULLS FIRST, isnull(category_id#32) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Output [2]: [ws_item_sk#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 15] -Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] (49) Filter [codegen id : 15] -Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_item_sk#37) +Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#35) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#39] +Output [1]: [d_date_sk#37] (51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#39] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#37] Join condition: None (52) Project [codegen id : 15] -Output [1]: [ws_item_sk#37] -Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] +Output [1]: [ws_item_sk#35] +Input [3]: [ws_item_sk#35, ws_sold_date_sk#36, d_date_sk#37] (53) ReusedExchange [Reuses operator id: 33] -Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] +Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] (54) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#40] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [i_item_sk#38] Join condition: None (55) Project [codegen id : 15] -Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] +Output [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Input [5]: [ws_item_sk#35, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] (56) Exchange -Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] +Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: hashpartitioning(coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41), 5), ENSURE_REQUIREMENTS, [id=#42] (57) Sort [codegen id : 16] -Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 +Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: [coalesce(i_brand_id#39, 0) ASC NULLS FIRST, isnull(i_brand_id#39) ASC NULLS FIRST, coalesce(i_class_id#40, 0) ASC NULLS FIRST, isnull(i_class_id#40) ASC NULLS FIRST, coalesce(i_category_id#41, 0) ASC NULLS FIRST, isnull(i_category_id#41) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] -Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41)] Join condition: None (59) HashAggregate [codegen id : 17] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#30, class_id#31, category_id#32] (60) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#43] (61) HashAggregate [codegen id : 18] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#30, class_id#31, category_id#32] (62) BroadcastExchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#44] (63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#32, class_id#33, category_id#34] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None (64) Project [codegen id : 19] -Output [1]: [i_item_sk#7 AS ss_item_sk#47] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] +Output [1]: [i_item_sk#7 AS ss_item_sk#45] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#30, class_id#31, category_id#32] (65) Exchange -Input [1]: [ss_item_sk#47] -Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [1]: [ss_item_sk#45] +Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] (66) Sort [codegen id : 20] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#47] +Right keys [1]: [ss_item_sk#45] Join condition: None (68) Scan parquet default.date_dim -Output [2]: [d_date_sk#49, d_week_seq#50] +Output [2]: [d_date_sk#12, d_week_seq#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 21] -Input [2]: [d_date_sk#49, d_week_seq#50] +Input [2]: [d_date_sk#12, d_week_seq#47] (70) Filter [codegen id : 21] -Input [2]: [d_date_sk#49, d_week_seq#50] -Condition : ((isnotnull(d_week_seq#50) AND (d_week_seq#50 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#49)) +Input [2]: [d_date_sk#12, d_week_seq#47] +Condition : ((isnotnull(d_week_seq#47) AND (d_week_seq#47 = Subquery scalar-subquery#48, [id=#49])) AND isnotnull(d_date_sk#12)) (71) Project [codegen id : 21] -Output [1]: [d_date_sk#49] -Input [2]: [d_date_sk#49, d_week_seq#50] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_week_seq#47] (72) BroadcastExchange -Input [1]: [d_date_sk#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] (73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#49] +Right keys [1]: [d_date_sk#12] Join condition: None (74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (75) Scan parquet default.item -Output [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (76) ColumnarToRow [codegen id : 22] -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (77) Filter [codegen id : 22] -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -Condition : (((isnotnull(i_item_sk#54) AND isnotnull(i_brand_id#55)) AND isnotnull(i_class_id#56)) AND isnotnull(i_category_id#57)) +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (78) Exchange -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -Arguments: hashpartitioning(i_item_sk#54, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#51] (79) Sort [codegen id : 23] -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -Arguments: [i_item_sk#54 ASC NULLS FIRST], false, 0 +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#45] (81) Sort [codegen id : 41] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#54] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [i_item_sk#7] +Right keys [1]: [ss_item_sk#45] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#59] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] (84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#54] +Right keys [1]: [i_item_sk#7] Join condition: None (85) Project [codegen id : 42] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (86) HashAggregate [codegen id : 42] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] -Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#60, isEmpty#61, count#62] -Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] +Aggregate Attributes [3]: [sum#53, isEmpty#54, count#55] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] (87) Exchange -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] -Arguments: hashpartitioning(i_brand_id#55, i_class_id#56, i_category_id#57, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#59] (88) HashAggregate [codegen id : 86] -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] -Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67, count(1)#68] -Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#69, count(1)#68 AS number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60, count(1)#61] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sales#62, count(1)#61 AS number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] (89) Filter [codegen id : 86] -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(32,6)) > cast(Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 as decimal(32,6)) > cast(Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) (90) Project [codegen id : 86] -Output [6]: [store AS channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70] -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] +Output [6]: [store AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] (91) Scan parquet default.store_sales -Output [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Output [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] +PartitionFilters: [isnotnull(ss_sold_date_sk#71), dynamicpruningexpression(ss_sold_date_sk#71 IN dynamicpruning#72)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 43] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] (93) Filter [codegen id : 43] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Condition : isnotnull(ss_item_sk#75) +Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Condition : isnotnull(ss_item_sk#68) (94) Exchange -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Arguments: hashpartitioning(ss_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Arguments: hashpartitioning(ss_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#73] (95) Sort [codegen id : 44] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Arguments: [ss_item_sk#75 ASC NULLS FIRST], false, 0 +Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Arguments: [ss_item_sk#68 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#45] (97) Sort [codegen id : 62] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ss_item_sk#75] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [ss_item_sk#68] +Right keys [1]: [ss_item_sk#45] Join condition: None (99) Scan parquet default.date_dim -Output [2]: [d_date_sk#81, d_week_seq#82] +Output [2]: [d_date_sk#74, d_week_seq#75] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (100) ColumnarToRow [codegen id : 63] -Input [2]: [d_date_sk#81, d_week_seq#82] +Input [2]: [d_date_sk#74, d_week_seq#75] (101) Filter [codegen id : 63] -Input [2]: [d_date_sk#81, d_week_seq#82] -Condition : ((isnotnull(d_week_seq#82) AND (d_week_seq#82 = Subquery scalar-subquery#83, [id=#84])) AND isnotnull(d_date_sk#81)) +Input [2]: [d_date_sk#74, d_week_seq#75] +Condition : ((isnotnull(d_week_seq#75) AND (d_week_seq#75 = Subquery scalar-subquery#76, [id=#77])) AND isnotnull(d_date_sk#74)) (102) Project [codegen id : 63] -Output [1]: [d_date_sk#81] -Input [2]: [d_date_sk#81, d_week_seq#82] +Output [1]: [d_date_sk#74] +Input [2]: [d_date_sk#74, d_week_seq#75] (103) BroadcastExchange -Input [1]: [d_date_sk#81] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#85] +Input [1]: [d_date_sk#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] (104) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_sold_date_sk#78] -Right keys [1]: [d_date_sk#81] +Left keys [1]: [ss_sold_date_sk#71] +Right keys [1]: [d_date_sk#74] Join condition: None (105) Project [codegen id : 84] -Output [3]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77] -Input [5]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#81] +Output [3]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70] +Input [5]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71, d_date_sk#74] (106) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] +Output [4]: [i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] (107) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#75] -Right keys [1]: [i_item_sk#86] +Left keys [1]: [ss_item_sk#68] +Right keys [1]: [i_item_sk#79] Join condition: None (108) Project [codegen id : 84] -Output [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] -Input [7]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] +Output [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#80, i_class_id#81, i_category_id#82] +Input [7]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] (109) HashAggregate [codegen id : 84] -Input [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] -Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] -Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] +Input [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#80, i_class_id#81, i_category_id#82] +Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] +Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] (110) Exchange -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] -Arguments: hashpartitioning(i_brand_id#87, i_class_id#88, i_category_id#89, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] +Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [id=#89] (111) HashAggregate [codegen id : 85] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] -Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] -Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#99, count(1)#98 AS number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] +Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90, count(1)#91] +Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sales#92, count(1)#91 AS number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] (112) Filter [codegen id : 85] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) (113) Project [codegen id : 85] -Output [6]: [store AS channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] +Output [6]: [store AS channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] (114) BroadcastExchange -Input [6]: [channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#103] +Input [6]: [channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#96] (115) BroadcastHashJoin [codegen id : 86] -Left keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] -Right keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Right keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] Join condition: None (116) TakeOrderedAndProject -Input [12]: [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Arguments: 100, [i_brand_id#55 ASC NULLS FIRST, i_class_id#56 ASC NULLS FIRST, i_category_id#57 ASC NULLS FIRST], [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Input [12]: [channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#72, [id=#73] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#65, [id=#66] * HashAggregate (139) +- Exchange (138) +- * HashAggregate (137) @@ -674,140 +674,140 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (117) Scan parquet default.store_sales -Output [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] +Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#106), dynamicpruningexpression(ss_sold_date_sk#106 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#97)] ReadSchema: struct (118) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] +Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (119) Scan parquet default.date_dim -Output [2]: [d_date_sk#108, d_year#109] +Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (120) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#108, d_year#109] +Input [2]: [d_date_sk#12, d_year#13] (121) Filter [codegen id : 1] -Input [2]: [d_date_sk#108, d_year#109] -Condition : (((isnotnull(d_year#109) AND (d_year#109 >= 1998)) AND (d_year#109 <= 2000)) AND isnotnull(d_date_sk#108)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) (122) Project [codegen id : 1] -Output [1]: [d_date_sk#108] -Input [2]: [d_date_sk#108, d_year#109] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (123) BroadcastExchange -Input [1]: [d_date_sk#108] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#110] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] (124) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#106] -Right keys [1]: [d_date_sk#108] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (125) Project [codegen id : 2] -Output [2]: [ss_quantity#104 AS quantity#111, ss_list_price#105 AS list_price#112] -Input [4]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106, d_date_sk#108] +Output [2]: [ss_quantity#2 AS quantity#99, ss_list_price#3 AS list_price#100] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (126) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] +Output [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#115), dynamicpruningexpression(cs_sold_date_sk#115 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#97)] ReadSchema: struct (127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] +Input [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21] (128) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#116] +Output [1]: [d_date_sk#103] (129) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#115] -Right keys [1]: [d_date_sk#116] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#103] Join condition: None (130) Project [codegen id : 4] -Output [2]: [cs_quantity#113 AS quantity#117, cs_list_price#114 AS list_price#118] -Input [4]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115, d_date_sk#116] +Output [2]: [cs_quantity#101 AS quantity#104, cs_list_price#102 AS list_price#105] +Input [4]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21, d_date_sk#103] (131) Scan parquet default.web_sales -Output [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] +Output [3]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#97)] ReadSchema: struct (132) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] +Input [3]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36] (133) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#122] +Output [1]: [d_date_sk#108] (134) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#121] -Right keys [1]: [d_date_sk#122] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#108] Join condition: None (135) Project [codegen id : 6] -Output [2]: [ws_quantity#119 AS quantity#123, ws_list_price#120 AS list_price#124] -Input [4]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121, d_date_sk#122] +Output [2]: [ws_quantity#106 AS quantity#109, ws_list_price#107 AS list_price#110] +Input [4]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36, d_date_sk#108] (136) Union (137) HashAggregate [codegen id : 7] -Input [2]: [quantity#111, list_price#112] +Input [2]: [quantity#99, list_price#100] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#125, count#126] -Results [2]: [sum#127, count#128] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#111, count#112] +Results [2]: [sum#113, count#114] (138) Exchange -Input [2]: [sum#127, count#128] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] +Input [2]: [sum#113, count#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] (139) HashAggregate [codegen id : 8] -Input [2]: [sum#127, count#128] +Input [2]: [sum#113, count#114] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130 AS average_sales#131] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#116] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#116 AS average_sales#117] -Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#106 IN dynamicpruning#107 +Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#97 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#108] +Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#115 IN dynamicpruning#107 +Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#97 -Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#107 +Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#97 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (141) (141) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#49] +Output [1]: [d_date_sk#12] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 ReusedExchange (142) (142) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#11 -Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#48, [id=#49] * Project (146) +- * Filter (145) +- * ColumnarToRow (144) @@ -815,33 +815,33 @@ Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquer (143) Scan parquet default.date_dim -Output [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Output [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] (145) Filter [codegen id : 1] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] -Condition : (((((isnotnull(d_year#133) AND isnotnull(d_moy#134)) AND isnotnull(d_dom#135)) AND (d_year#133 = 1999)) AND (d_moy#134 = 12)) AND (d_dom#135 = 16)) +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#13 = 1999)) AND (d_moy#118 = 12)) AND (d_dom#119 = 16)) (146) Project [codegen id : 1] -Output [1]: [d_week_seq#132] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Output [1]: [d_week_seq#47] +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] -Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] +Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#65, [id=#66] -Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#79 +Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#71 IN dynamicpruning#72 ReusedExchange (147) (147) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#81] +Output [1]: [d_date_sk#74] -Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#83, [id=#84] +Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#76, [id=#77] * Project (151) +- * Filter (150) +- * ColumnarToRow (149) @@ -849,21 +849,21 @@ Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subqu (148) Scan parquet default.date_dim -Output [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Output [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (149) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] (150) Filter [codegen id : 1] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] -Condition : (((((isnotnull(d_year#137) AND isnotnull(d_moy#138)) AND isnotnull(d_dom#139)) AND (d_year#137 = 1998)) AND (d_moy#138 = 12)) AND (d_dom#139 = 16)) +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#13 = 1998)) AND (d_moy#118 = 12)) AND (d_dom#119 = 16)) (151) Project [codegen id : 1] -Output [1]: [d_week_seq#136] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Output [1]: [d_week_seq#47] +Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index bdafb17d69d45..4c62de561c263 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -131,443 +131,443 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (10) Scan parquet default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (11) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] (12) Filter [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14)) (13) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_item_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] (15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) +Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Condition : isnotnull(cs_item_sk#15) (16) Scan parquet default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (18) Filter [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : isnotnull(i_item_sk#17) (19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [i_item_sk#17] Join condition: None (21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (22) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_year#25] +Output [2]: [d_date_sk#22, d_year#23] 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 (23) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#24, d_year#25] +Input [2]: [d_date_sk#22, d_year#23] (24) Filter [codegen id : 2] -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) +Input [2]: [d_date_sk#22, d_year#23] +Condition : (((isnotnull(d_year#23) AND (d_year#23 >= 1998)) AND (d_year#23 <= 2000)) AND isnotnull(d_date_sk#22)) (25) Project [codegen id : 2] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_year#25] +Output [1]: [d_date_sk#22] +Input [2]: [d_date_sk#22, d_year#23] (26) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (27) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#22] Join condition: None (28) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] +Output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] +Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#22] (29) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] +Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] (30) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] +Left keys [6]: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)] +Right keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] Join condition: None (31) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] (32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#11] Join condition: None (33) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Output [4]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14] +Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] (34) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#27] (35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#27] Join condition: None (36) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] +Output [3]: [i_brand_id#12 AS brand_id#28, i_class_id#13 AS class_id#29, i_category_id#14 AS category_id#30] +Input [5]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#27] (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#28, class_id#29, category_id#30] (38) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [id=#31] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#28, class_id#29, category_id#30] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Output [2]: [ws_item_sk#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_item_sk#34) +Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Condition : isnotnull(ws_item_sk#32) (43) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] +Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#36] +Left keys [1]: [ws_item_sk#32] +Right keys [1]: [i_item_sk#34] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] +Output [4]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37] +Input [6]: [ws_item_sk#32, ws_sold_date_sk#33, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] (46) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#40] +Output [1]: [d_date_sk#38] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#40] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#38] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] +Output [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +Input [5]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] (49) BroadcastExchange -Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] +Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#39] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] -Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] +Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] +Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#28, class_id#29, category_id#30] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#28, class_id#29, category_id#30] (53) BroadcastExchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#40] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#30, class_id#31, category_id#32] +Right keys [3]: [brand_id#28, class_id#29, category_id#30] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#43] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] +Output [1]: [i_item_sk#6 AS ss_item_sk#41] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] (56) BroadcastExchange -Input [1]: [ss_item_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] +Input [1]: [ss_item_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#43] +Right keys [1]: [ss_item_sk#41] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] -Condition : (((isnotnull(i_item_sk#45) AND isnotnull(i_brand_id#46)) AND isnotnull(i_class_id#47)) AND isnotnull(i_category_id#48)) +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#41] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#45] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [i_item_sk#6] +Right keys [1]: [ss_item_sk#41] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#45] +Right keys [1]: [i_item_sk#6] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (66) Scan parquet default.date_dim -Output [2]: [d_date_sk#50, d_week_seq#51] +Output [2]: [d_date_sk#27, d_week_seq#44] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [2]: [d_date_sk#50, d_week_seq#51] +Input [2]: [d_date_sk#27, d_week_seq#44] (68) Filter [codegen id : 24] -Input [2]: [d_date_sk#50, d_week_seq#51] -Condition : ((isnotnull(d_week_seq#51) AND (d_week_seq#51 = Subquery scalar-subquery#52, [id=#53])) AND isnotnull(d_date_sk#50)) +Input [2]: [d_date_sk#27, d_week_seq#44] +Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#27)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#50] -Input [2]: [d_date_sk#50, d_week_seq#51] +Output [1]: [d_date_sk#27] +Input [2]: [d_date_sk#27, d_week_seq#44] (70) BroadcastExchange -Input [1]: [d_date_sk#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#50] +Right keys [1]: [d_date_sk#27] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#27] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] -Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] +Aggregate Attributes [3]: [sum#48, isEmpty#49, count#50] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] (74) Exchange -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] -Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#54] (75) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] -Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55, count(1)#56] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sales#57, count(1)#56 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] (76) Filter [codegen id : 52] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59 as decimal(32,6)) > cast(Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) (77) Project [codegen id : 52] -Output [6]: [store AS channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] +Output [6]: [store AS channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] (78) Scan parquet default.store_sales -Output [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +Output [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] +PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] (80) Filter [codegen id : 50] -Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] -Condition : isnotnull(ss_item_sk#70) +Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] +Condition : isnotnull(ss_item_sk#63) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#41] (82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#70] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [ss_item_sk#63] +Right keys [1]: [ss_item_sk#41] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] +Output [4]: [i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] (84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#70] -Right keys [1]: [i_item_sk#75] +Left keys [1]: [ss_item_sk#63] +Right keys [1]: [i_item_sk#68] Join condition: None (85) Project [codegen id : 50] -Output [6]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] -Input [8]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] +Output [6]: [ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_brand_id#69, i_class_id#70, i_category_id#71] +Input [8]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#79, d_week_seq#80] +Output [2]: [d_date_sk#72, d_week_seq#73] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#79, d_week_seq#80] +Input [2]: [d_date_sk#72, d_week_seq#73] (88) Filter [codegen id : 49] -Input [2]: [d_date_sk#79, d_week_seq#80] -Condition : ((isnotnull(d_week_seq#80) AND (d_week_seq#80 = Subquery scalar-subquery#81, [id=#82])) AND isnotnull(d_date_sk#79)) +Input [2]: [d_date_sk#72, d_week_seq#73] +Condition : ((isnotnull(d_week_seq#73) AND (d_week_seq#73 = Subquery scalar-subquery#74, [id=#75])) AND isnotnull(d_date_sk#72)) (89) Project [codegen id : 49] -Output [1]: [d_date_sk#79] -Input [2]: [d_date_sk#79, d_week_seq#80] +Output [1]: [d_date_sk#72] +Input [2]: [d_date_sk#72, d_week_seq#73] (90) BroadcastExchange -Input [1]: [d_date_sk#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#83] +Input [1]: [d_date_sk#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] (91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#73] -Right keys [1]: [d_date_sk#79] +Left keys [1]: [ss_sold_date_sk#66] +Right keys [1]: [d_date_sk#72] Join condition: None (92) Project [codegen id : 50] -Output [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] -Input [7]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78, d_date_sk#79] +Output [5]: [ss_quantity#64, ss_list_price#65, i_brand_id#69, i_class_id#70, i_category_id#71] +Input [7]: [ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_brand_id#69, i_class_id#70, i_category_id#71, d_date_sk#72] (93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] -Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] +Input [5]: [ss_quantity#64, ss_list_price#65, i_brand_id#69, i_class_id#70, i_category_id#71] +Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#77, isEmpty#78, count#79] +Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] (94) Exchange -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] -Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] +Arguments: hashpartitioning(i_brand_id#69, i_class_id#70, i_category_id#71, 5), ENSURE_REQUIREMENTS, [id=#83] (95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] -Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] +Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84, count(1)#85] +Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sales#86, count(1)#85 AS number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] (96) Filter [codegen id : 51] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) (97) Project [codegen id : 51] -Output [6]: [store AS channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] +Output [6]: [store AS channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] +Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] (98) BroadcastExchange -Input [6]: [channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#97] +Input [6]: [channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#90] (99) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] -Right keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Right keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] Join condition: None (100) TakeOrderedAndProject -Input [12]: [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Arguments: 100, [i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Input [12]: [channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] +Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#67, [id=#68] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#60, [id=#61] * HashAggregate (123) +- Exchange (122) +- * HashAggregate (121) @@ -594,140 +594,140 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] +Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#100), dynamicpruningexpression(ss_sold_date_sk#100 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#91)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] +Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (103) Scan parquet default.date_dim -Output [2]: [d_date_sk#102, d_year#103] +Output [2]: [d_date_sk#27, d_year#92] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#102, d_year#103] +Input [2]: [d_date_sk#27, d_year#92] (105) Filter [codegen id : 1] -Input [2]: [d_date_sk#102, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 <= 2000)) AND isnotnull(d_date_sk#102)) +Input [2]: [d_date_sk#27, d_year#92] +Condition : (((isnotnull(d_year#92) AND (d_year#92 >= 1998)) AND (d_year#92 <= 2000)) AND isnotnull(d_date_sk#27)) (106) Project [codegen id : 1] -Output [1]: [d_date_sk#102] -Input [2]: [d_date_sk#102, d_year#103] +Output [1]: [d_date_sk#27] +Input [2]: [d_date_sk#27, d_year#92] (107) BroadcastExchange -Input [1]: [d_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#104] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] (108) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#100] -Right keys [1]: [d_date_sk#102] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#27] Join condition: None (109) Project [codegen id : 2] -Output [2]: [ss_quantity#98 AS quantity#105, ss_list_price#99 AS list_price#106] -Input [4]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100, d_date_sk#102] +Output [2]: [ss_quantity#2 AS quantity#94, ss_list_price#3 AS list_price#95] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#27] (110) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] +Output [3]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#109), dynamicpruningexpression(cs_sold_date_sk#109 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#91)] ReadSchema: struct (111) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] +Input [3]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16] (112) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#110] +Output [1]: [d_date_sk#98] (113) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#109] -Right keys [1]: [d_date_sk#110] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#98] Join condition: None (114) Project [codegen id : 4] -Output [2]: [cs_quantity#107 AS quantity#111, cs_list_price#108 AS list_price#112] -Input [4]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109, d_date_sk#110] +Output [2]: [cs_quantity#96 AS quantity#99, cs_list_price#97 AS list_price#100] +Input [4]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16, d_date_sk#98] (115) Scan parquet default.web_sales -Output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] +Output [3]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#115), dynamicpruningexpression(ws_sold_date_sk#115 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#91)] ReadSchema: struct (116) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] +Input [3]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33] (117) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#116] +Output [1]: [d_date_sk#103] (118) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#115] -Right keys [1]: [d_date_sk#116] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#103] Join condition: None (119) Project [codegen id : 6] -Output [2]: [ws_quantity#113 AS quantity#117, ws_list_price#114 AS list_price#118] -Input [4]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115, d_date_sk#116] +Output [2]: [ws_quantity#101 AS quantity#104, ws_list_price#102 AS list_price#105] +Input [4]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33, d_date_sk#103] (120) Union (121) HashAggregate [codegen id : 7] -Input [2]: [quantity#105, list_price#106] +Input [2]: [quantity#94, list_price#95] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#119, count#120] -Results [2]: [sum#121, count#122] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#106, count#107] +Results [2]: [sum#108, count#109] (122) Exchange -Input [2]: [sum#121, count#122] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#123] +Input [2]: [sum#108, count#109] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] (123) HashAggregate [codegen id : 8] -Input [2]: [sum#121, count#122] +Input [2]: [sum#108, count#109] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124 AS average_sales#125] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))#111] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))#111 AS average_sales#112] -Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#100 IN dynamicpruning#101 +Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#91 ReusedExchange (124) (124) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#102] +Output [1]: [d_date_sk#27] -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#109 IN dynamicpruning#101 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#91 -Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#115 IN dynamicpruning#101 +Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#91 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#50] +Output [1]: [d_date_sk#27] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 ReusedExchange (126) (126) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#27] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#10 -Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#45, [id=#46] * Project (130) +- * Filter (129) +- * ColumnarToRow (128) @@ -735,33 +735,33 @@ Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer (127) Scan parquet default.date_dim -Output [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Output [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (128) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] (129) Filter [codegen id : 1] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] -Condition : (((((isnotnull(d_year#127) AND isnotnull(d_moy#128)) AND isnotnull(d_dom#129)) AND (d_year#127 = 1999)) AND (d_moy#128 = 12)) AND (d_dom#129 = 16)) +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#92 = 1999)) AND (d_moy#113 = 12)) AND (d_dom#114 = 16)) (130) Project [codegen id : 1] -Output [1]: [d_week_seq#126] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Output [1]: [d_week_seq#44] +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] -Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] +Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#60, [id=#61] -Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 +Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67 ReusedExchange (131) (131) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#79] +Output [1]: [d_date_sk#72] -Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#81, [id=#82] +Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#74, [id=#75] * Project (135) +- * Filter (134) +- * ColumnarToRow (133) @@ -769,21 +769,21 @@ Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subque (132) Scan parquet default.date_dim -Output [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Output [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (133) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] (134) Filter [codegen id : 1] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] -Condition : (((((isnotnull(d_year#131) AND isnotnull(d_moy#132)) AND isnotnull(d_dom#133)) AND (d_year#131 = 1998)) AND (d_moy#132 = 12)) AND (d_dom#133 = 16)) +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#92 = 1998)) AND (d_moy#113 = 12)) AND (d_dom#114 = 16)) (135) Project [codegen id : 1] -Output [1]: [d_week_seq#130] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Output [1]: [d_week_seq#44] +Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] 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 43ebe6e0b4d9b..26141bdb911ca 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 @@ -252,964 +252,964 @@ Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (10) ColumnarToRow [codegen id : 10] -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] (11) Filter [codegen id : 10] -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_item_sk#11) +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#12, d_year#13] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#14, d_year#15] +Input [2]: [d_date_sk#12, d_year#13] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#14, d_year#15] -Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1999)) AND (d_year#15 <= 2001)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_year#15] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (16) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (17) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (18) Project [codegen id : 10] -Output [1]: [ss_item_sk#11] -Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] +Output [1]: [ss_item_sk#1] +Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] (19) Scan parquet default.item -Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Condition : (((isnotnull(i_item_sk#15) AND isnotnull(i_brand_id#16)) AND isnotnull(i_class_id#17)) AND isnotnull(i_category_id#18)) (22) Exchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: hashpartitioning(coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] (23) Sort [codegen id : 5] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: [coalesce(i_brand_id#16, 0) ASC NULLS FIRST, isnull(i_brand_id#16) ASC NULLS FIRST, coalesce(i_class_id#17, 0) ASC NULLS FIRST, isnull(i_class_id#17) ASC NULLS FIRST, coalesce(i_category_id#18, 0) ASC NULLS FIRST, isnull(i_category_id#18) ASC NULLS FIRST], false, 0 (24) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 8] -Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] (26) Filter [codegen id : 8] -Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] -Condition : isnotnull(cs_item_sk#22) +Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +Condition : isnotnull(cs_item_sk#20) (27) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#24] +Output [1]: [d_date_sk#22] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#22] Join condition: None (29) Project [codegen id : 8] -Output [1]: [cs_item_sk#22] -Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] +Output [1]: [cs_item_sk#20] +Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#22] (30) Scan parquet default.item -Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Output [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] (32) Filter [codegen id : 7] -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -Condition : isnotnull(i_item_sk#25) +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Condition : isnotnull(i_item_sk#23) (33) BroadcastExchange -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#22] -Right keys [1]: [i_item_sk#25] +Left keys [1]: [cs_item_sk#20] +Right keys [1]: [i_item_sk#23] Join condition: None (35) Project [codegen id : 8] -Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Output [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Input [5]: [cs_item_sk#20, i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] (36) Exchange -Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: hashpartitioning(coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26), 5), ENSURE_REQUIREMENTS, [id=#28] (37) Sort [codegen id : 9] -Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 +Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] +Arguments: [coalesce(i_brand_id#24, 0) ASC NULLS FIRST, isnull(i_brand_id#24) ASC NULLS FIRST, coalesce(i_class_id#25, 0) ASC NULLS FIRST, isnull(i_class_id#25) ASC NULLS FIRST, coalesce(i_category_id#26, 0) ASC NULLS FIRST, isnull(i_category_id#26) ASC NULLS FIRST], false, 0 (38) SortMergeJoin -Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] -Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] +Left keys [6]: [coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18)] +Right keys [6]: [coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26)] Join condition: None (39) BroadcastExchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#11] -Right keys [1]: [i_item_sk#17] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#15] Join condition: None (41) Project [codegen id : 10] -Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] -Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [3]: [i_brand_id#16 AS brand_id#30, i_class_id#17 AS class_id#31, i_category_id#18 AS category_id#32] +Input [5]: [ss_item_sk#1, i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] (42) HashAggregate [codegen id : 10] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#30, class_id#31, category_id#32] (43) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] (44) HashAggregate [codegen id : 11] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#30, class_id#31, category_id#32] (45) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32), 5), ENSURE_REQUIREMENTS, [id=#34] (46) Sort [codegen id : 12] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: [coalesce(brand_id#30, 0) ASC NULLS FIRST, isnull(brand_id#30) ASC NULLS FIRST, coalesce(class_id#31, 0) ASC NULLS FIRST, isnull(class_id#31) ASC NULLS FIRST, coalesce(category_id#32, 0) ASC NULLS FIRST, isnull(category_id#32) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Output [2]: [ws_item_sk#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 15] -Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] (49) Filter [codegen id : 15] -Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_item_sk#37) +Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#35) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#39] +Output [1]: [d_date_sk#37] (51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#39] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#37] Join condition: None (52) Project [codegen id : 15] -Output [1]: [ws_item_sk#37] -Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] +Output [1]: [ws_item_sk#35] +Input [3]: [ws_item_sk#35, ws_sold_date_sk#36, d_date_sk#37] (53) ReusedExchange [Reuses operator id: 33] -Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] +Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] (54) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#40] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [i_item_sk#38] Join condition: None (55) Project [codegen id : 15] -Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] +Output [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Input [5]: [ws_item_sk#35, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] (56) Exchange -Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] +Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: hashpartitioning(coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41), 5), ENSURE_REQUIREMENTS, [id=#42] (57) Sort [codegen id : 16] -Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 +Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: [coalesce(i_brand_id#39, 0) ASC NULLS FIRST, isnull(i_brand_id#39) ASC NULLS FIRST, coalesce(i_class_id#40, 0) ASC NULLS FIRST, isnull(i_class_id#40) ASC NULLS FIRST, coalesce(i_category_id#41, 0) ASC NULLS FIRST, isnull(i_category_id#41) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] -Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41)] Join condition: None (59) HashAggregate [codegen id : 17] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#30, class_id#31, category_id#32] (60) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#43] (61) HashAggregate [codegen id : 18] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#30, class_id#31, category_id#32] (62) BroadcastExchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#44] (63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#32, class_id#33, category_id#34] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None (64) Project [codegen id : 19] -Output [1]: [i_item_sk#7 AS ss_item_sk#47] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] +Output [1]: [i_item_sk#7 AS ss_item_sk#45] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#30, class_id#31, category_id#32] (65) Exchange -Input [1]: [ss_item_sk#47] -Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [1]: [ss_item_sk#45] +Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] (66) Sort [codegen id : 20] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#47] +Right keys [1]: [ss_item_sk#45] Join condition: None (68) Scan parquet default.date_dim -Output [3]: [d_date_sk#49, d_year#50, d_moy#51] +Output [3]: [d_date_sk#12, d_year#13, d_moy#47] 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 (69) ColumnarToRow [codegen id : 21] -Input [3]: [d_date_sk#49, d_year#50, d_moy#51] +Input [3]: [d_date_sk#12, d_year#13, d_moy#47] (70) Filter [codegen id : 21] -Input [3]: [d_date_sk#49, d_year#50, d_moy#51] -Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2000)) AND (d_moy#51 = 11)) AND isnotnull(d_date_sk#49)) +Input [3]: [d_date_sk#12, d_year#13, d_moy#47] +Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#47)) AND (d_year#13 = 2000)) AND (d_moy#47 = 11)) AND isnotnull(d_date_sk#12)) (71) Project [codegen id : 21] -Output [1]: [d_date_sk#49] -Input [3]: [d_date_sk#49, d_year#50, d_moy#51] +Output [1]: [d_date_sk#12] +Input [3]: [d_date_sk#12, d_year#13, d_moy#47] (72) BroadcastExchange -Input [1]: [d_date_sk#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] (73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#49] +Right keys [1]: [d_date_sk#12] Join condition: None (74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (75) Scan parquet default.item -Output [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 22] -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (77) Filter [codegen id : 22] -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Condition : isnotnull(i_item_sk#53) +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : isnotnull(i_item_sk#7) (78) Exchange -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: hashpartitioning(i_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#49] (79) Sort [codegen id : 23] -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [i_item_sk#53 ASC NULLS FIRST], false, 0 +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#45] (81) Sort [codegen id : 41] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#53] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [i_item_sk#7] +Right keys [1]: [ss_item_sk#45] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] (84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#53] +Right keys [1]: [i_item_sk#7] Join condition: None (85) Project [codegen id : 42] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (86) HashAggregate [codegen id : 42] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] -Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#59, isEmpty#60, count#61] -Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] +Aggregate Attributes [3]: [sum#51, isEmpty#52, count#53] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] (87) Exchange -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] -Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#57] (88) HashAggregate [codegen id : 43] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] -Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66, count(1)#67] -Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] (89) Filter [codegen id : 43] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(32,6)) > cast(Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (90) Project [codegen id : 43] -Output [6]: [store AS channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +Output [6]: [store AS channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] (91) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Output [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#77), dynamicpruningexpression(cs_sold_date_sk#77 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 44] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] (93) Filter [codegen id : 44] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Condition : isnotnull(cs_item_sk#74) +Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Condition : isnotnull(cs_item_sk#20) (94) Exchange -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Arguments: hashpartitioning(cs_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#68] (95) Sort [codegen id : 45] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Arguments: [cs_item_sk#74 ASC NULLS FIRST], false, 0 +Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Arguments: [cs_item_sk#20 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#45] (97) Sort [codegen id : 63] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [cs_item_sk#74] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [cs_item_sk#20] +Right keys [1]: [ss_item_sk#45] Join condition: None (99) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#79] +Output [1]: [d_date_sk#69] (100) BroadcastHashJoin [codegen id : 85] -Left keys [1]: [cs_sold_date_sk#77] -Right keys [1]: [d_date_sk#79] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#69] Join condition: None (101) Project [codegen id : 85] -Output [3]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76] -Input [5]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77, d_date_sk#79] +Output [3]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67] +Input [5]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21, d_date_sk#69] (102) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] +Output [4]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73] (103) BroadcastHashJoin [codegen id : 85] -Left keys [1]: [cs_item_sk#74] -Right keys [1]: [i_item_sk#80] +Left keys [1]: [cs_item_sk#20] +Right keys [1]: [i_item_sk#70] Join condition: None (104) Project [codegen id : 85] -Output [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] -Input [7]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] +Output [5]: [cs_quantity#66, cs_list_price#67, i_brand_id#71, i_class_id#72, i_category_id#73] +Input [7]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73] (105) HashAggregate [codegen id : 85] -Input [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] -Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +Input [5]: [cs_quantity#66, cs_list_price#67, i_brand_id#71, i_class_id#72, i_category_id#73] +Keys [3]: [i_brand_id#71, i_class_id#72, i_category_id#73] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#74, isEmpty#75, count#76] +Results [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] (106) Exchange -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] -Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] +Arguments: hashpartitioning(i_brand_id#71, i_class_id#72, i_category_id#73, 5), ENSURE_REQUIREMENTS, [id=#80] (107) HashAggregate [codegen id : 86] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] -Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] +Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] +Keys [3]: [i_brand_id#71, i_class_id#72, i_category_id#73] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81, count(1)#82] +Results [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sales#83, count(1)#82 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] (108) Filter [codegen id : 86] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (109) Project [codegen id : 86] -Output [6]: [catalog AS channel#96, i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] +Output [6]: [catalog AS channel#86, i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84] +Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] (110) Scan parquet default.web_sales -Output [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Output [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#100), dynamicpruningexpression(ws_sold_date_sk#100 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (111) ColumnarToRow [codegen id : 87] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] (112) Filter [codegen id : 87] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Condition : isnotnull(ws_item_sk#97) +Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Condition : isnotnull(ws_item_sk#35) (113) Exchange -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Arguments: hashpartitioning(ws_item_sk#97, 5), ENSURE_REQUIREMENTS, [id=#101] +Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Arguments: hashpartitioning(ws_item_sk#35, 5), ENSURE_REQUIREMENTS, [id=#89] (114) Sort [codegen id : 88] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Arguments: [ws_item_sk#97 ASC NULLS FIRST], false, 0 +Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Arguments: [ws_item_sk#35 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#45] (116) Sort [codegen id : 106] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +Input [1]: [ss_item_sk#45] +Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 (117) SortMergeJoin -Left keys [1]: [ws_item_sk#97] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [ss_item_sk#45] Join condition: None (118) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#102] +Output [1]: [d_date_sk#90] (119) BroadcastHashJoin [codegen id : 128] -Left keys [1]: [ws_sold_date_sk#100] -Right keys [1]: [d_date_sk#102] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#90] Join condition: None (120) Project [codegen id : 128] -Output [3]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99] -Input [5]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100, d_date_sk#102] +Output [3]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88] +Input [5]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36, d_date_sk#90] (121) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] +Output [4]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94] (122) BroadcastHashJoin [codegen id : 128] -Left keys [1]: [ws_item_sk#97] -Right keys [1]: [i_item_sk#103] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [i_item_sk#91] Join condition: None (123) Project [codegen id : 128] -Output [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] -Input [7]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] +Output [5]: [ws_quantity#87, ws_list_price#88, i_brand_id#92, i_class_id#93, i_category_id#94] +Input [7]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94] (124) HashAggregate [codegen id : 128] -Input [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] -Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#107, isEmpty#108, count#109] -Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +Input [5]: [ws_quantity#87, ws_list_price#88, i_brand_id#92, i_class_id#93, i_category_id#94] +Keys [3]: [i_brand_id#92, i_class_id#93, i_category_id#94] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#95, isEmpty#96, count#97] +Results [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] (125) Exchange -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] -Arguments: hashpartitioning(i_brand_id#104, i_class_id#105, i_category_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] +Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] +Arguments: hashpartitioning(i_brand_id#92, i_class_id#93, i_category_id#94, 5), ENSURE_REQUIREMENTS, [id=#101] (126) HashAggregate [codegen id : 129] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] -Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114, count(1)#115] -Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sales#116, count(1)#115 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] +Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] +Keys [3]: [i_brand_id#92, i_class_id#93, i_category_id#94] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102, count(1)#103] +Results [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sales#104, count(1)#103 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] (127) Filter [codegen id : 129] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (128) Project [codegen id : 129] -Output [6]: [web AS channel#119, i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] +Output [6]: [web AS channel#107, i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105] +Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] (129) Union (130) HashAggregate [codegen id : 130] -Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69] -Keys [4]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#120, isEmpty#121, sum#122] -Results [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] +Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61] +Keys [4]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] +Aggregate Attributes [3]: [sum#108, isEmpty#109, sum#110] +Results [7]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum#111, isEmpty#112, sum#113] (131) Exchange -Input [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] -Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#126] +Input [7]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum#111, isEmpty#112, sum#113] +Arguments: hashpartitioning(channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#114] (132) HashAggregate [codegen id : 131] -Input [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] -Keys [4]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#127, sum(number_sales#69)#128] -Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum(sales#68)#127 AS sum_sales#129, sum(number_sales#69)#128 AS number_sales#130] +Input [7]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum#111, isEmpty#112, sum#113] +Keys [4]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(sales#60), sum(number_sales#61)] +Aggregate Attributes [2]: [sum(sales#60)#115, sum(number_sales#61)#116] +Results [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum(sales#60)#115 AS sum_sales#117, sum(number_sales#61)#116 AS number_sales#118] (133) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, count#136] +Output [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sum#122, isEmpty#123, count#124] (134) HashAggregate [codegen id : 174] -Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, count#136] -Keys [3]: [i_brand_id#131, i_class_id#132, i_category_id#133] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139, count(1)#140] -Results [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sales#68, count(1)#140 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] +Input [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sum#122, isEmpty#123, count#124] +Keys [3]: [i_brand_id#119, i_class_id#120, i_category_id#121] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#127, count(1)#128] +Results [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#127 AS sales#60, count(1)#128 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#127 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129] (135) Filter [codegen id : 174] -Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (136) Project [codegen id : 174] -Output [6]: [store AS channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69] -Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] +Output [6]: [store AS channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sales#60, number_sales#61] +Input [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129] (137) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum#145, isEmpty#146, count#147] +Output [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sum#133, isEmpty#134, count#135] (138) HashAggregate [codegen id : 217] -Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum#145, isEmpty#146, count#147] -Keys [3]: [i_brand_id#142, i_class_id#143, i_category_id#144] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150, count(1)#151] -Results [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sales#93, count(1)#151 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] +Input [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sum#133, isEmpty#134, count#135] +Keys [3]: [i_brand_id#130, i_class_id#131, i_category_id#132] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#138, count(1)#139] +Results [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sales#83, count(1)#139 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140] (139) Filter [codegen id : 217] -Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (140) Project [codegen id : 217] -Output [6]: [catalog AS channel#96, i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94] -Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] +Output [6]: [catalog AS channel#86, i_brand_id#130, i_class_id#131, i_category_id#132, sales#83, number_sales#84] +Input [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140] (141) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum#156, isEmpty#157, count#158] +Output [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sum#144, isEmpty#145, count#146] (142) HashAggregate [codegen id : 260] -Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum#156, isEmpty#157, count#158] -Keys [3]: [i_brand_id#153, i_class_id#154, i_category_id#155] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161, count(1)#162] -Results [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161 AS sales#116, count(1)#162 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] +Input [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sum#144, isEmpty#145, count#146] +Keys [3]: [i_brand_id#141, i_class_id#142, i_category_id#143] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#149, count(1)#150] +Results [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#149 AS sales#104, count(1)#150 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151] (143) Filter [codegen id : 260] -Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (144) Project [codegen id : 260] -Output [6]: [web AS channel#119, i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117] -Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] +Output [6]: [web AS channel#107, i_brand_id#141, i_class_id#142, i_category_id#143, sales#104, number_sales#105] +Input [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151] (145) Union (146) HashAggregate [codegen id : 261] -Input [6]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69] -Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#164, isEmpty#165, sum#166] -Results [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] +Input [6]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sales#60, number_sales#61] +Keys [4]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121] +Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] +Aggregate Attributes [3]: [sum#152, isEmpty#153, sum#154] +Results [7]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sum#155, isEmpty#156, sum#157] (147) Exchange -Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] -Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, 5), ENSURE_REQUIREMENTS, [id=#170] +Input [7]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sum#155, isEmpty#156, sum#157] +Arguments: hashpartitioning(channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, 5), ENSURE_REQUIREMENTS, [id=#158] (148) HashAggregate [codegen id : 262] -Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] -Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#171, sum(number_sales#69)#172] -Results [5]: [channel#73, i_brand_id#131, i_class_id#132, sum(sales#68)#171 AS sum_sales#129, sum(number_sales#69)#172 AS number_sales#130] +Input [7]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sum#155, isEmpty#156, sum#157] +Keys [4]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121] +Functions [2]: [sum(sales#60), sum(number_sales#61)] +Aggregate Attributes [2]: [sum(sales#60)#159, sum(number_sales#61)#160] +Results [5]: [channel#65, i_brand_id#119, i_class_id#120, sum(sales#60)#159 AS sum_sales#117, sum(number_sales#61)#160 AS number_sales#118] (149) HashAggregate [codegen id : 262] -Input [5]: [channel#73, i_brand_id#131, i_class_id#132, sum_sales#129, number_sales#130] -Keys [3]: [channel#73, i_brand_id#131, i_class_id#132] -Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#173, isEmpty#174, sum#175] -Results [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] +Input [5]: [channel#65, i_brand_id#119, i_class_id#120, sum_sales#117, number_sales#118] +Keys [3]: [channel#65, i_brand_id#119, i_class_id#120] +Functions [2]: [partial_sum(sum_sales#117), partial_sum(number_sales#118)] +Aggregate Attributes [3]: [sum#161, isEmpty#162, sum#163] +Results [6]: [channel#65, i_brand_id#119, i_class_id#120, sum#164, isEmpty#165, sum#166] (150) Exchange -Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] -Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, 5), ENSURE_REQUIREMENTS, [id=#179] +Input [6]: [channel#65, i_brand_id#119, i_class_id#120, sum#164, isEmpty#165, sum#166] +Arguments: hashpartitioning(channel#65, i_brand_id#119, i_class_id#120, 5), ENSURE_REQUIREMENTS, [id=#167] (151) HashAggregate [codegen id : 263] -Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] -Keys [3]: [channel#73, i_brand_id#131, i_class_id#132] -Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#180, sum(number_sales#130)#181] -Results [6]: [channel#73, i_brand_id#131, i_class_id#132, null AS i_category_id#182, sum(sum_sales#129)#180 AS sum(sum_sales)#183, sum(number_sales#130)#181 AS sum(number_sales)#184] +Input [6]: [channel#65, i_brand_id#119, i_class_id#120, sum#164, isEmpty#165, sum#166] +Keys [3]: [channel#65, i_brand_id#119, i_class_id#120] +Functions [2]: [sum(sum_sales#117), sum(number_sales#118)] +Aggregate Attributes [2]: [sum(sum_sales#117)#168, sum(number_sales#118)#169] +Results [6]: [channel#65, i_brand_id#119, i_class_id#120, null AS i_category_id#170, sum(sum_sales#117)#168 AS sum(sum_sales)#171, sum(number_sales#118)#169 AS sum(number_sales)#172] (152) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum#188, isEmpty#189, count#190] +Output [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sum#176, isEmpty#177, count#178] (153) HashAggregate [codegen id : 306] -Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum#188, isEmpty#189, count#190] -Keys [3]: [i_brand_id#185, i_class_id#186, i_category_id#187] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193, count(1)#194] -Results [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193 AS sales#68, count(1)#194 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] +Input [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sum#176, isEmpty#177, count#178] +Keys [3]: [i_brand_id#173, i_class_id#174, i_category_id#175] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#181, count(1)#182] +Results [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sales#60, count(1)#182 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183] (154) Filter [codegen id : 306] -Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (155) Project [codegen id : 306] -Output [6]: [store AS channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69] -Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] +Output [6]: [store AS channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sales#60, number_sales#61] +Input [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183] (156) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum#199, isEmpty#200, count#201] +Output [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sum#187, isEmpty#188, count#189] (157) HashAggregate [codegen id : 349] -Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum#199, isEmpty#200, count#201] -Keys [3]: [i_brand_id#196, i_class_id#197, i_category_id#198] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204, count(1)#205] -Results [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204 AS sales#93, count(1)#205 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] +Input [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sum#187, isEmpty#188, count#189] +Keys [3]: [i_brand_id#184, i_class_id#185, i_category_id#186] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#192, count(1)#193] +Results [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sales#83, count(1)#193 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194] (158) Filter [codegen id : 349] -Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (159) Project [codegen id : 349] -Output [6]: [catalog AS channel#96, i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94] -Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] +Output [6]: [catalog AS channel#86, i_brand_id#184, i_class_id#185, i_category_id#186, sales#83, number_sales#84] +Input [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194] (160) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum#210, isEmpty#211, count#212] +Output [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sum#198, isEmpty#199, count#200] (161) HashAggregate [codegen id : 392] -Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum#210, isEmpty#211, count#212] -Keys [3]: [i_brand_id#207, i_class_id#208, i_category_id#209] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215, count(1)#216] -Results [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sales#116, count(1)#216 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] +Input [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sum#198, isEmpty#199, count#200] +Keys [3]: [i_brand_id#195, i_class_id#196, i_category_id#197] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#203, count(1)#204] +Results [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#203 AS sales#104, count(1)#204 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#203 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205] (162) Filter [codegen id : 392] -Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (163) Project [codegen id : 392] -Output [6]: [web AS channel#119, i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117] -Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] +Output [6]: [web AS channel#107, i_brand_id#195, i_class_id#196, i_category_id#197, sales#104, number_sales#105] +Input [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205] (164) Union (165) HashAggregate [codegen id : 393] -Input [6]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69] -Keys [4]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#218, isEmpty#219, sum#220] -Results [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] +Input [6]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sales#60, number_sales#61] +Keys [4]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175] +Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] +Aggregate Attributes [3]: [sum#206, isEmpty#207, sum#208] +Results [7]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sum#209, isEmpty#210, sum#211] (166) Exchange -Input [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] -Arguments: hashpartitioning(channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, 5), ENSURE_REQUIREMENTS, [id=#224] +Input [7]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sum#209, isEmpty#210, sum#211] +Arguments: hashpartitioning(channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, 5), ENSURE_REQUIREMENTS, [id=#212] (167) HashAggregate [codegen id : 394] -Input [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] -Keys [4]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#225, sum(number_sales#69)#226] -Results [4]: [channel#73, i_brand_id#185, sum(sales#68)#225 AS sum_sales#129, sum(number_sales#69)#226 AS number_sales#130] +Input [7]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sum#209, isEmpty#210, sum#211] +Keys [4]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175] +Functions [2]: [sum(sales#60), sum(number_sales#61)] +Aggregate Attributes [2]: [sum(sales#60)#213, sum(number_sales#61)#214] +Results [4]: [channel#65, i_brand_id#173, sum(sales#60)#213 AS sum_sales#117, sum(number_sales#61)#214 AS number_sales#118] (168) HashAggregate [codegen id : 394] -Input [4]: [channel#73, i_brand_id#185, sum_sales#129, number_sales#130] -Keys [2]: [channel#73, i_brand_id#185] -Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#227, isEmpty#228, sum#229] -Results [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] +Input [4]: [channel#65, i_brand_id#173, sum_sales#117, number_sales#118] +Keys [2]: [channel#65, i_brand_id#173] +Functions [2]: [partial_sum(sum_sales#117), partial_sum(number_sales#118)] +Aggregate Attributes [3]: [sum#215, isEmpty#216, sum#217] +Results [5]: [channel#65, i_brand_id#173, sum#218, isEmpty#219, sum#220] (169) Exchange -Input [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] -Arguments: hashpartitioning(channel#73, i_brand_id#185, 5), ENSURE_REQUIREMENTS, [id=#233] +Input [5]: [channel#65, i_brand_id#173, sum#218, isEmpty#219, sum#220] +Arguments: hashpartitioning(channel#65, i_brand_id#173, 5), ENSURE_REQUIREMENTS, [id=#221] (170) HashAggregate [codegen id : 395] -Input [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] -Keys [2]: [channel#73, i_brand_id#185] -Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#234, sum(number_sales#130)#235] -Results [6]: [channel#73, i_brand_id#185, null AS i_class_id#236, null AS i_category_id#237, sum(sum_sales#129)#234 AS sum(sum_sales)#238, sum(number_sales#130)#235 AS sum(number_sales)#239] +Input [5]: [channel#65, i_brand_id#173, sum#218, isEmpty#219, sum#220] +Keys [2]: [channel#65, i_brand_id#173] +Functions [2]: [sum(sum_sales#117), sum(number_sales#118)] +Aggregate Attributes [2]: [sum(sum_sales#117)#222, sum(number_sales#118)#223] +Results [6]: [channel#65, i_brand_id#173, null AS i_class_id#224, null AS i_category_id#225, sum(sum_sales#117)#222 AS sum(sum_sales)#226, sum(number_sales#118)#223 AS sum(number_sales)#227] (171) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum#243, isEmpty#244, count#245] +Output [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sum#231, isEmpty#232, count#233] (172) HashAggregate [codegen id : 438] -Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum#243, isEmpty#244, count#245] -Keys [3]: [i_brand_id#240, i_class_id#241, i_category_id#242] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248, count(1)#249] -Results [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248 AS sales#68, count(1)#249 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] +Input [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sum#231, isEmpty#232, count#233] +Keys [3]: [i_brand_id#228, i_class_id#229, i_category_id#230] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#236, count(1)#237] +Results [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#236 AS sales#60, count(1)#237 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#236 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238] (173) Filter [codegen id : 438] -Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (174) Project [codegen id : 438] -Output [6]: [store AS channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69] -Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] +Output [6]: [store AS channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sales#60, number_sales#61] +Input [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238] (175) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum#254, isEmpty#255, count#256] +Output [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sum#242, isEmpty#243, count#244] (176) HashAggregate [codegen id : 481] -Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum#254, isEmpty#255, count#256] -Keys [3]: [i_brand_id#251, i_class_id#252, i_category_id#253] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259, count(1)#260] -Results [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259 AS sales#93, count(1)#260 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] +Input [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sum#242, isEmpty#243, count#244] +Keys [3]: [i_brand_id#239, i_class_id#240, i_category_id#241] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#247, count(1)#248] +Results [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#247 AS sales#83, count(1)#248 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#247 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249] (177) Filter [codegen id : 481] -Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (178) Project [codegen id : 481] -Output [6]: [catalog AS channel#96, i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94] -Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] +Output [6]: [catalog AS channel#86, i_brand_id#239, i_class_id#240, i_category_id#241, sales#83, number_sales#84] +Input [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249] (179) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum#265, isEmpty#266, count#267] +Output [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sum#253, isEmpty#254, count#255] (180) HashAggregate [codegen id : 524] -Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum#265, isEmpty#266, count#267] -Keys [3]: [i_brand_id#262, i_class_id#263, i_category_id#264] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270, count(1)#271] -Results [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270 AS sales#116, count(1)#271 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] +Input [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sum#253, isEmpty#254, count#255] +Keys [3]: [i_brand_id#250, i_class_id#251, i_category_id#252] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#258, count(1)#259] +Results [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#258 AS sales#104, count(1)#259 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#258 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260] (181) Filter [codegen id : 524] -Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (182) Project [codegen id : 524] -Output [6]: [web AS channel#119, i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117] -Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] +Output [6]: [web AS channel#107, i_brand_id#250, i_class_id#251, i_category_id#252, sales#104, number_sales#105] +Input [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260] (183) Union (184) HashAggregate [codegen id : 525] -Input [6]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69] -Keys [4]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#273, isEmpty#274, sum#275] -Results [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] +Input [6]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sales#60, number_sales#61] +Keys [4]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230] +Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] +Aggregate Attributes [3]: [sum#261, isEmpty#262, sum#263] +Results [7]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sum#264, isEmpty#265, sum#266] (185) Exchange -Input [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] -Arguments: hashpartitioning(channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, 5), ENSURE_REQUIREMENTS, [id=#279] +Input [7]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sum#264, isEmpty#265, sum#266] +Arguments: hashpartitioning(channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, 5), ENSURE_REQUIREMENTS, [id=#267] (186) HashAggregate [codegen id : 526] -Input [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] -Keys [4]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#280, sum(number_sales#69)#281] -Results [3]: [channel#73, sum(sales#68)#280 AS sum_sales#129, sum(number_sales#69)#281 AS number_sales#130] +Input [7]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sum#264, isEmpty#265, sum#266] +Keys [4]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230] +Functions [2]: [sum(sales#60), sum(number_sales#61)] +Aggregate Attributes [2]: [sum(sales#60)#268, sum(number_sales#61)#269] +Results [3]: [channel#65, sum(sales#60)#268 AS sum_sales#117, sum(number_sales#61)#269 AS number_sales#118] (187) HashAggregate [codegen id : 526] -Input [3]: [channel#73, sum_sales#129, number_sales#130] -Keys [1]: [channel#73] -Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#282, isEmpty#283, sum#284] -Results [4]: [channel#73, sum#285, isEmpty#286, sum#287] +Input [3]: [channel#65, sum_sales#117, number_sales#118] +Keys [1]: [channel#65] +Functions [2]: [partial_sum(sum_sales#117), partial_sum(number_sales#118)] +Aggregate Attributes [3]: [sum#270, isEmpty#271, sum#272] +Results [4]: [channel#65, sum#273, isEmpty#274, sum#275] (188) Exchange -Input [4]: [channel#73, sum#285, isEmpty#286, sum#287] -Arguments: hashpartitioning(channel#73, 5), ENSURE_REQUIREMENTS, [id=#288] +Input [4]: [channel#65, sum#273, isEmpty#274, sum#275] +Arguments: hashpartitioning(channel#65, 5), ENSURE_REQUIREMENTS, [id=#276] (189) HashAggregate [codegen id : 527] -Input [4]: [channel#73, sum#285, isEmpty#286, sum#287] -Keys [1]: [channel#73] -Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#289, sum(number_sales#130)#290] -Results [6]: [channel#73, null AS i_brand_id#291, null AS i_class_id#292, null AS i_category_id#293, sum(sum_sales#129)#289 AS sum(sum_sales)#294, sum(number_sales#130)#290 AS sum(number_sales)#295] +Input [4]: [channel#65, sum#273, isEmpty#274, sum#275] +Keys [1]: [channel#65] +Functions [2]: [sum(sum_sales#117), sum(number_sales#118)] +Aggregate Attributes [2]: [sum(sum_sales#117)#277, sum(number_sales#118)#278] +Results [6]: [channel#65, null AS i_brand_id#279, null AS i_class_id#280, null AS i_category_id#281, sum(sum_sales#117)#277 AS sum(sum_sales)#282, sum(number_sales#118)#278 AS sum(number_sales)#283] (190) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum#299, isEmpty#300, count#301] +Output [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sum#287, isEmpty#288, count#289] (191) HashAggregate [codegen id : 570] -Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum#299, isEmpty#300, count#301] -Keys [3]: [i_brand_id#296, i_class_id#297, i_category_id#298] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304, count(1)#305] -Results [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304 AS sales#68, count(1)#305 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] +Input [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sum#287, isEmpty#288, count#289] +Keys [3]: [i_brand_id#284, i_class_id#285, i_category_id#286] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#292, count(1)#293] +Results [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#292 AS sales#60, count(1)#293 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#292 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294] (192) Filter [codegen id : 570] -Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (193) Project [codegen id : 570] -Output [6]: [store AS channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69] -Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] +Output [6]: [store AS channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sales#60, number_sales#61] +Input [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294] (194) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum#310, isEmpty#311, count#312] +Output [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sum#298, isEmpty#299, count#300] (195) HashAggregate [codegen id : 613] -Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum#310, isEmpty#311, count#312] -Keys [3]: [i_brand_id#307, i_class_id#308, i_category_id#309] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315, count(1)#316] -Results [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315 AS sales#93, count(1)#316 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] +Input [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sum#298, isEmpty#299, count#300] +Keys [3]: [i_brand_id#295, i_class_id#296, i_category_id#297] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#303, count(1)#304] +Results [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#303 AS sales#83, count(1)#304 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#303 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305] (196) Filter [codegen id : 613] -Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (197) Project [codegen id : 613] -Output [6]: [catalog AS channel#96, i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94] -Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] +Output [6]: [catalog AS channel#86, i_brand_id#295, i_class_id#296, i_category_id#297, sales#83, number_sales#84] +Input [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305] (198) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum#321, isEmpty#322, count#323] +Output [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sum#309, isEmpty#310, count#311] (199) HashAggregate [codegen id : 656] -Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum#321, isEmpty#322, count#323] -Keys [3]: [i_brand_id#318, i_class_id#319, i_category_id#320] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326, count(1)#327] -Results [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326 AS sales#116, count(1)#327 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] +Input [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sum#309, isEmpty#310, count#311] +Keys [3]: [i_brand_id#306, i_class_id#307, i_category_id#308] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#314, count(1)#315] +Results [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#314 AS sales#104, count(1)#315 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#314 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316] (200) Filter [codegen id : 656] -Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +Input [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) (201) Project [codegen id : 656] -Output [6]: [web AS channel#119, i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117] -Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] +Output [6]: [web AS channel#107, i_brand_id#306, i_class_id#307, i_category_id#308, sales#104, number_sales#105] +Input [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316] (202) Union (203) HashAggregate [codegen id : 657] -Input [6]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69] -Keys [4]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#329, isEmpty#330, sum#331] -Results [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] +Input [6]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sales#60, number_sales#61] +Keys [4]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286] +Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] +Aggregate Attributes [3]: [sum#317, isEmpty#318, sum#319] +Results [7]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sum#320, isEmpty#321, sum#322] (204) Exchange -Input [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] -Arguments: hashpartitioning(channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, 5), ENSURE_REQUIREMENTS, [id=#335] +Input [7]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sum#320, isEmpty#321, sum#322] +Arguments: hashpartitioning(channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, 5), ENSURE_REQUIREMENTS, [id=#323] (205) HashAggregate [codegen id : 658] -Input [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] -Keys [4]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#336, sum(number_sales#69)#337] -Results [2]: [sum(sales#68)#336 AS sum_sales#129, sum(number_sales#69)#337 AS number_sales#130] +Input [7]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sum#320, isEmpty#321, sum#322] +Keys [4]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286] +Functions [2]: [sum(sales#60), sum(number_sales#61)] +Aggregate Attributes [2]: [sum(sales#60)#324, sum(number_sales#61)#325] +Results [2]: [sum(sales#60)#324 AS sum_sales#117, sum(number_sales#61)#325 AS number_sales#118] (206) HashAggregate [codegen id : 658] -Input [2]: [sum_sales#129, number_sales#130] +Input [2]: [sum_sales#117, number_sales#118] Keys: [] -Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#338, isEmpty#339, sum#340] -Results [3]: [sum#341, isEmpty#342, sum#343] +Functions [2]: [partial_sum(sum_sales#117), partial_sum(number_sales#118)] +Aggregate Attributes [3]: [sum#326, isEmpty#327, sum#328] +Results [3]: [sum#329, isEmpty#330, sum#331] (207) Exchange -Input [3]: [sum#341, isEmpty#342, sum#343] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#344] +Input [3]: [sum#329, isEmpty#330, sum#331] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#332] (208) HashAggregate [codegen id : 659] -Input [3]: [sum#341, isEmpty#342, sum#343] +Input [3]: [sum#329, isEmpty#330, sum#331] Keys: [] -Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#345, sum(number_sales#130)#346] -Results [6]: [null AS channel#347, null AS i_brand_id#348, null AS i_class_id#349, null AS i_category_id#350, sum(sum_sales#129)#345 AS sum(sum_sales)#351, sum(number_sales#130)#346 AS sum(number_sales)#352] +Functions [2]: [sum(sum_sales#117), sum(number_sales#118)] +Aggregate Attributes [2]: [sum(sum_sales#117)#333, sum(number_sales#118)#334] +Results [6]: [null AS channel#335, null AS i_brand_id#336, null AS i_class_id#337, null AS i_category_id#338, sum(sum_sales#117)#333 AS sum(sum_sales)#339, sum(number_sales#118)#334 AS sum(number_sales)#340] (209) Union (210) HashAggregate [codegen id : 660] -Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] -Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +Keys [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Results [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] (211) Exchange -Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] -Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130, 5), ENSURE_REQUIREMENTS, [id=#353] +Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +Arguments: hashpartitioning(channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118, 5), ENSURE_REQUIREMENTS, [id=#341] (212) HashAggregate [codegen id : 661] -Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] -Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +Keys [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Results [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] (213) TakeOrderedAndProject -Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] -Arguments: 100, [channel#73 ASC NULLS FIRST, i_brand_id#54 ASC NULLS FIRST, i_class_id#55 ASC NULLS FIRST, i_category_id#56 ASC NULLS FIRST], [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +Arguments: 100, [channel#65 ASC NULLS FIRST, i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#71, [id=#72] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#63, [id=#64] * HashAggregate (240) +- Exchange (239) +- * HashAggregate (238) @@ -1240,193 +1240,193 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (214) Scan parquet default.store_sales -Output [3]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356] +Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#356), dynamicpruningexpression(ss_sold_date_sk#356 IN dynamicpruning#357)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#342)] ReadSchema: struct (215) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356] +Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (216) Scan parquet default.date_dim -Output [2]: [d_date_sk#358, d_year#359] +Output [2]: [d_date_sk#12, d_year#13] 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 (217) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#358, d_year#359] +Input [2]: [d_date_sk#12, d_year#13] (218) Filter [codegen id : 1] -Input [2]: [d_date_sk#358, d_year#359] -Condition : (((isnotnull(d_year#359) AND (d_year#359 >= 1999)) AND (d_year#359 <= 2001)) AND isnotnull(d_date_sk#358)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (219) Project [codegen id : 1] -Output [1]: [d_date_sk#358] -Input [2]: [d_date_sk#358, d_year#359] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (220) BroadcastExchange -Input [1]: [d_date_sk#358] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#360] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#343] (221) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#356] -Right keys [1]: [d_date_sk#358] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (222) Project [codegen id : 2] -Output [2]: [ss_quantity#354 AS quantity#361, ss_list_price#355 AS list_price#362] -Input [4]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356, d_date_sk#358] +Output [2]: [ss_quantity#2 AS quantity#344, ss_list_price#3 AS list_price#345] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (223) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365] +Output [3]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#365), dynamicpruningexpression(cs_sold_date_sk#365 IN dynamicpruning#366)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#346)] ReadSchema: struct (224) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365] +Input [3]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] (225) Scan parquet default.date_dim -Output [2]: [d_date_sk#367, d_year#368] +Output [2]: [d_date_sk#347, d_year#348] 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 (226) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#367, d_year#368] +Input [2]: [d_date_sk#347, d_year#348] (227) Filter [codegen id : 3] -Input [2]: [d_date_sk#367, d_year#368] -Condition : (((isnotnull(d_year#368) AND (d_year#368 >= 1998)) AND (d_year#368 <= 2000)) AND isnotnull(d_date_sk#367)) +Input [2]: [d_date_sk#347, d_year#348] +Condition : (((isnotnull(d_year#348) AND (d_year#348 >= 1998)) AND (d_year#348 <= 2000)) AND isnotnull(d_date_sk#347)) (228) Project [codegen id : 3] -Output [1]: [d_date_sk#367] -Input [2]: [d_date_sk#367, d_year#368] +Output [1]: [d_date_sk#347] +Input [2]: [d_date_sk#347, d_year#348] (229) BroadcastExchange -Input [1]: [d_date_sk#367] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#369] +Input [1]: [d_date_sk#347] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#349] (230) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#365] -Right keys [1]: [d_date_sk#367] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#347] Join condition: None (231) Project [codegen id : 4] -Output [2]: [cs_quantity#363 AS quantity#370, cs_list_price#364 AS list_price#371] -Input [4]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365, d_date_sk#367] +Output [2]: [cs_quantity#66 AS quantity#350, cs_list_price#67 AS list_price#351] +Input [4]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21, d_date_sk#347] (232) Scan parquet default.web_sales -Output [3]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374] +Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#374), dynamicpruningexpression(ws_sold_date_sk#374 IN dynamicpruning#366)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#346)] ReadSchema: struct (233) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374] +Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] (234) ReusedExchange [Reuses operator id: 229] -Output [1]: [d_date_sk#375] +Output [1]: [d_date_sk#352] (235) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#374] -Right keys [1]: [d_date_sk#375] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#352] Join condition: None (236) Project [codegen id : 6] -Output [2]: [ws_quantity#372 AS quantity#376, ws_list_price#373 AS list_price#377] -Input [4]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374, d_date_sk#375] +Output [2]: [ws_quantity#87 AS quantity#353, ws_list_price#88 AS list_price#354] +Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36, d_date_sk#352] (237) Union (238) HashAggregate [codegen id : 7] -Input [2]: [quantity#361, list_price#362] +Input [2]: [quantity#344, list_price#345] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#378, count#379] -Results [2]: [sum#380, count#381] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#344 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#345 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#355, count#356] +Results [2]: [sum#357, count#358] (239) Exchange -Input [2]: [sum#380, count#381] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#382] +Input [2]: [sum#357, count#358] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#359] (240) HashAggregate [codegen id : 8] -Input [2]: [sum#380, count#381] +Input [2]: [sum#357, count#358] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))#383] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))#383 AS average_sales#384] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#344 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#345 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#344 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#345 as decimal(12,2)))), DecimalType(18,2), true))#360] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#344 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#345 as decimal(12,2)))), DecimalType(18,2), true))#360 AS average_sales#361] -Subquery:2 Hosting operator id = 214 Hosting Expression = ss_sold_date_sk#356 IN dynamicpruning#357 +Subquery:2 Hosting operator id = 214 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#342 ReusedExchange (241) (241) ReusedExchange [Reuses operator id: 220] -Output [1]: [d_date_sk#358] +Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 223 Hosting Expression = cs_sold_date_sk#365 IN dynamicpruning#366 +Subquery:3 Hosting operator id = 223 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#346 ReusedExchange (242) (242) ReusedExchange [Reuses operator id: 229] -Output [1]: [d_date_sk#367] +Output [1]: [d_date_sk#347] -Subquery:4 Hosting operator id = 232 Hosting Expression = ws_sold_date_sk#374 IN dynamicpruning#366 +Subquery:4 Hosting operator id = 232 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#346 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (243) (243) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#49] +Output [1]: [d_date_sk#12] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 ReusedExchange (244) (244) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#11 -Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#77 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 -Subquery:13 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:13 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:14 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:14 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:15 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:15 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:16 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:16 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:17 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:17 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:18 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:18 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:19 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:19 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:20 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:20 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:21 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:21 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:22 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:22 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:23 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:23 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] -Subquery:24 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:24 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] 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 755973f79e493..4bd1e57d994ab 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -225,896 +225,896 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (10) Scan parquet default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (11) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] (12) Filter [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14)) (13) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_item_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] (15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) +Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] +Condition : isnotnull(cs_item_sk#15) (16) Scan parquet default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (18) Filter [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : isnotnull(i_item_sk#17) (19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [i_item_sk#17] Join condition: None (21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (22) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_year#25] +Output [2]: [d_date_sk#22, d_year#23] 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 (23) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#24, d_year#25] +Input [2]: [d_date_sk#22, d_year#23] (24) Filter [codegen id : 2] -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) +Input [2]: [d_date_sk#22, d_year#23] +Condition : (((isnotnull(d_year#23) AND (d_year#23 >= 1999)) AND (d_year#23 <= 2001)) AND isnotnull(d_date_sk#22)) (25) Project [codegen id : 2] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_year#25] +Output [1]: [d_date_sk#22] +Input [2]: [d_date_sk#22, d_year#23] (26) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (27) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#22] Join condition: None (28) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] +Output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] +Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#22] (29) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] +Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] (30) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] +Left keys [6]: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)] +Right keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] Join condition: None (31) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] +Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] (32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#11] Join condition: None (33) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Output [4]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14] +Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] (34) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#27] (35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#27] Join condition: None (36) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] +Output [3]: [i_brand_id#12 AS brand_id#28, i_class_id#13 AS class_id#29, i_category_id#14 AS category_id#30] +Input [5]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#27] (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#28, class_id#29, category_id#30] (38) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [id=#31] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#28, class_id#29, category_id#30] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Output [2]: [ws_item_sk#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_item_sk#34) +Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Condition : isnotnull(ws_item_sk#32) (43) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] +Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#36] +Left keys [1]: [ws_item_sk#32] +Right keys [1]: [i_item_sk#34] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] +Output [4]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37] +Input [6]: [ws_item_sk#32, ws_sold_date_sk#33, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] (46) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#40] +Output [1]: [d_date_sk#38] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#40] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#38] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] +Output [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +Input [5]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] (49) BroadcastExchange -Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] +Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#39] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] -Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] +Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] +Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#28, class_id#29, category_id#30] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Keys [3]: [brand_id#28, class_id#29, category_id#30] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#28, class_id#29, category_id#30] (53) BroadcastExchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] +Input [3]: [brand_id#28, class_id#29, category_id#30] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#40] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#30, class_id#31, category_id#32] +Right keys [3]: [brand_id#28, class_id#29, category_id#30] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#43] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] +Output [1]: [i_item_sk#6 AS ss_item_sk#41] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] (56) BroadcastExchange -Input [1]: [ss_item_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] +Input [1]: [ss_item_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#43] +Right keys [1]: [ss_item_sk#41] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] -Condition : isnotnull(i_item_sk#45) +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : isnotnull(i_item_sk#6) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#41] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#45] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [i_item_sk#6] +Right keys [1]: [ss_item_sk#41] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#45] +Right keys [1]: [i_item_sk#6] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (66) Scan parquet default.date_dim -Output [3]: [d_date_sk#50, d_year#51, d_moy#52] +Output [3]: [d_date_sk#27, d_year#44, d_moy#45] 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 (67) ColumnarToRow [codegen id : 24] -Input [3]: [d_date_sk#50, d_year#51, d_moy#52] +Input [3]: [d_date_sk#27, d_year#44, d_moy#45] (68) Filter [codegen id : 24] -Input [3]: [d_date_sk#50, d_year#51, d_moy#52] -Condition : ((((isnotnull(d_year#51) AND isnotnull(d_moy#52)) AND (d_year#51 = 2000)) AND (d_moy#52 = 11)) AND isnotnull(d_date_sk#50)) +Input [3]: [d_date_sk#27, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#27)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#50] -Input [3]: [d_date_sk#50, d_year#51, d_moy#52] +Output [1]: [d_date_sk#27] +Input [3]: [d_date_sk#27, d_year#44, d_moy#45] (70) BroadcastExchange -Input [1]: [d_date_sk#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#50] +Right keys [1]: [d_date_sk#27] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#27] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] -Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] +Aggregate Attributes [3]: [sum#47, isEmpty#48, count#49] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] (74) Exchange -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] -Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#53] (75) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] -Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] (76) Filter [codegen id : 26] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [store AS channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Output [6]: [store AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] (78) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +Output [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +Input [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] (80) Filter [codegen id : 51] -Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] -Condition : isnotnull(cs_item_sk#69) +Input [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Condition : isnotnull(cs_item_sk#15) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#41] (82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#69] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [ss_item_sk#41] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] +Output [4]: [i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] (84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#69] -Right keys [1]: [i_item_sk#73] +Left keys [1]: [cs_item_sk#15] +Right keys [1]: [i_item_sk#64] Join condition: None (85) Project [codegen id : 51] -Output [6]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76] -Input [8]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] +Output [6]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_brand_id#65, i_class_id#66, i_category_id#67] +Input [8]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] (86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#77] +Output [1]: [d_date_sk#68] (87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#72] -Right keys [1]: [d_date_sk#77] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#68] Join condition: None (88) Project [codegen id : 51] -Output [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] -Input [7]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76, d_date_sk#77] +Output [5]: [cs_quantity#62, cs_list_price#63, i_brand_id#65, i_class_id#66, i_category_id#67] +Input [7]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_brand_id#65, i_class_id#66, i_category_id#67, d_date_sk#68] (89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] -Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#78, isEmpty#79, count#80] -Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Input [5]: [cs_quantity#62, cs_list_price#63, i_brand_id#65, i_class_id#66, i_category_id#67] +Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] +Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] (90) Exchange -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] -Arguments: hashpartitioning(i_brand_id#74, i_class_id#75, i_category_id#76, 5), ENSURE_REQUIREMENTS, [id=#84] +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] +Arguments: hashpartitioning(i_brand_id#65, i_class_id#66, i_category_id#67, 5), ENSURE_REQUIREMENTS, [id=#75] (91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] -Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85, count(1)#86] -Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sales#87, count(1)#86 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] +Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76, count(1)#77] +Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sales#78, count(1)#77 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] (92) Filter [codegen id : 52] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [catalog AS channel#90, i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] +Output [6]: [catalog AS channel#81, i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79] +Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] (94) Scan parquet default.web_sales -Output [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Output [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Input [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] (96) Filter [codegen id : 77] -Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] -Condition : isnotnull(ws_item_sk#91) +Input [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Condition : isnotnull(ws_item_sk#32) (97) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#41] (98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#91] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [ws_item_sk#32] +Right keys [1]: [ss_item_sk#41] Join condition: None (99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] +Output [4]: [i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] (100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#91] -Right keys [1]: [i_item_sk#95] +Left keys [1]: [ws_item_sk#32] +Right keys [1]: [i_item_sk#84] Join condition: None (101) Project [codegen id : 77] -Output [6]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98] -Input [8]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] +Output [6]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_brand_id#85, i_class_id#86, i_category_id#87] +Input [8]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] (102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#99] +Output [1]: [d_date_sk#88] (103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#94] -Right keys [1]: [d_date_sk#99] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#88] Join condition: None (104) Project [codegen id : 77] -Output [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] -Input [7]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98, d_date_sk#99] +Output [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#85, i_class_id#86, i_category_id#87] +Input [7]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_brand_id#85, i_class_id#86, i_category_id#87, d_date_sk#88] (105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] -Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#100, isEmpty#101, count#102] -Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Input [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#85, i_class_id#86, i_category_id#87] +Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#89, isEmpty#90, count#91] +Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] (106) Exchange -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] -Arguments: hashpartitioning(i_brand_id#96, i_class_id#97, i_category_id#98, 5), ENSURE_REQUIREMENTS, [id=#106] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, 5), ENSURE_REQUIREMENTS, [id=#95] (107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] -Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] -Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#109, count(1)#108 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96, count(1)#97] +Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sales#98, count(1)#97 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] (108) Filter [codegen id : 78] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [web AS channel#112, i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] +Output [6]: [web AS channel#101, i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] (110) Union (111) HashAggregate [codegen id : 79] -Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64] -Keys [4]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#113, isEmpty#114, sum#115] -Results [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] +Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57] +Keys [4]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] +Aggregate Attributes [3]: [sum#102, isEmpty#103, sum#104] +Results [7]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum#105, isEmpty#106, sum#107] (112) Exchange -Input [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] -Arguments: hashpartitioning(channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#119] +Input [7]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum#105, isEmpty#106, sum#107] +Arguments: hashpartitioning(channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#108] (113) HashAggregate [codegen id : 80] -Input [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] -Keys [4]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#120, sum(number_sales#64)#121] -Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum(sales#63)#120 AS sum_sales#122, sum(number_sales#64)#121 AS number_sales#123] +Input [7]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum#105, isEmpty#106, sum#107] +Keys [4]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#56), sum(number_sales#57)] +Aggregate Attributes [2]: [sum(sales#56)#109, sum(number_sales#57)#110] +Results [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#56)#109 AS sum_sales#111, sum(number_sales#57)#110 AS number_sales#112] (114) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] +Output [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sum#116, isEmpty#117, count#118] (115) HashAggregate [codegen id : 106] -Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] -Keys [3]: [i_brand_id#124, i_class_id#125, i_category_id#126] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132, count(1)#133] -Results [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sales#63, count(1)#133 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] +Input [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sum#116, isEmpty#117, count#118] +Keys [3]: [i_brand_id#113, i_class_id#114, i_category_id#115] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#121, count(1)#122] +Results [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#121 AS sales#56, count(1)#122 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#121 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123] (116) Filter [codegen id : 106] -Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (117) Project [codegen id : 106] -Output [6]: [store AS channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64] -Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] +Output [6]: [store AS channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sales#56, number_sales#57] +Input [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123] (118) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] +Output [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] (119) HashAggregate [codegen id : 132] -Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] -Keys [3]: [i_brand_id#135, i_class_id#136, i_category_id#137] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143, count(1)#144] -Results [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sales#87, count(1)#144 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] +Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] +Keys [3]: [i_brand_id#124, i_class_id#125, i_category_id#126] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132, count(1)#133] +Results [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sales#78, count(1)#133 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] (120) Filter [codegen id : 132] -Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (121) Project [codegen id : 132] -Output [6]: [catalog AS channel#90, i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88] -Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] +Output [6]: [catalog AS channel#81, i_brand_id#124, i_class_id#125, i_category_id#126, sales#78, number_sales#79] +Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] (122) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum#149, isEmpty#150, count#151] +Output [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] (123) HashAggregate [codegen id : 158] -Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum#149, isEmpty#150, count#151] -Keys [3]: [i_brand_id#146, i_class_id#147, i_category_id#148] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154, count(1)#155] -Results [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154 AS sales#109, count(1)#155 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] +Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] +Keys [3]: [i_brand_id#135, i_class_id#136, i_category_id#137] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143, count(1)#144] +Results [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sales#98, count(1)#144 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] (124) Filter [codegen id : 158] -Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (125) Project [codegen id : 158] -Output [6]: [web AS channel#112, i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110] -Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] +Output [6]: [web AS channel#101, i_brand_id#135, i_class_id#136, i_category_id#137, sales#98, number_sales#99] +Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] (126) Union (127) HashAggregate [codegen id : 159] -Input [6]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64] -Keys [4]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#157, isEmpty#158, sum#159] -Results [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] +Input [6]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sales#56, number_sales#57] +Keys [4]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115] +Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] +Aggregate Attributes [3]: [sum#146, isEmpty#147, sum#148] +Results [7]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sum#149, isEmpty#150, sum#151] (128) Exchange -Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] -Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, 5), ENSURE_REQUIREMENTS, [id=#163] +Input [7]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sum#149, isEmpty#150, sum#151] +Arguments: hashpartitioning(channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, 5), ENSURE_REQUIREMENTS, [id=#152] (129) HashAggregate [codegen id : 160] -Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] -Keys [4]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#164, sum(number_sales#64)#165] -Results [5]: [channel#68, i_brand_id#124, i_class_id#125, sum(sales#63)#164 AS sum_sales#122, sum(number_sales#64)#165 AS number_sales#123] +Input [7]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sum#149, isEmpty#150, sum#151] +Keys [4]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115] +Functions [2]: [sum(sales#56), sum(number_sales#57)] +Aggregate Attributes [2]: [sum(sales#56)#153, sum(number_sales#57)#154] +Results [5]: [channel#61, i_brand_id#113, i_class_id#114, sum(sales#56)#153 AS sum_sales#111, sum(number_sales#57)#154 AS number_sales#112] (130) HashAggregate [codegen id : 160] -Input [5]: [channel#68, i_brand_id#124, i_class_id#125, sum_sales#122, number_sales#123] -Keys [3]: [channel#68, i_brand_id#124, i_class_id#125] -Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] -Aggregate Attributes [3]: [sum#166, isEmpty#167, sum#168] -Results [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] +Input [5]: [channel#61, i_brand_id#113, i_class_id#114, sum_sales#111, number_sales#112] +Keys [3]: [channel#61, i_brand_id#113, i_class_id#114] +Functions [2]: [partial_sum(sum_sales#111), partial_sum(number_sales#112)] +Aggregate Attributes [3]: [sum#155, isEmpty#156, sum#157] +Results [6]: [channel#61, i_brand_id#113, i_class_id#114, sum#158, isEmpty#159, sum#160] (131) Exchange -Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] -Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, 5), ENSURE_REQUIREMENTS, [id=#172] +Input [6]: [channel#61, i_brand_id#113, i_class_id#114, sum#158, isEmpty#159, sum#160] +Arguments: hashpartitioning(channel#61, i_brand_id#113, i_class_id#114, 5), ENSURE_REQUIREMENTS, [id=#161] (132) HashAggregate [codegen id : 161] -Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] -Keys [3]: [channel#68, i_brand_id#124, i_class_id#125] -Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] -Aggregate Attributes [2]: [sum(sum_sales#122)#173, sum(number_sales#123)#174] -Results [6]: [channel#68, i_brand_id#124, i_class_id#125, null AS i_category_id#175, sum(sum_sales#122)#173 AS sum(sum_sales)#176, sum(number_sales#123)#174 AS sum(number_sales)#177] +Input [6]: [channel#61, i_brand_id#113, i_class_id#114, sum#158, isEmpty#159, sum#160] +Keys [3]: [channel#61, i_brand_id#113, i_class_id#114] +Functions [2]: [sum(sum_sales#111), sum(number_sales#112)] +Aggregate Attributes [2]: [sum(sum_sales#111)#162, sum(number_sales#112)#163] +Results [6]: [channel#61, i_brand_id#113, i_class_id#114, null AS i_category_id#164, sum(sum_sales#111)#162 AS sum(sum_sales)#165, sum(number_sales#112)#163 AS sum(number_sales)#166] (133) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] +Output [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sum#170, isEmpty#171, count#172] (134) HashAggregate [codegen id : 187] -Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] -Keys [3]: [i_brand_id#178, i_class_id#179, i_category_id#180] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186, count(1)#187] -Results [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sales#63, count(1)#187 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] +Input [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sum#170, isEmpty#171, count#172] +Keys [3]: [i_brand_id#167, i_class_id#168, i_category_id#169] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#175, count(1)#176] +Results [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#175 AS sales#56, count(1)#176 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#175 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177] (135) Filter [codegen id : 187] -Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (136) Project [codegen id : 187] -Output [6]: [store AS channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64] -Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] +Output [6]: [store AS channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sales#56, number_sales#57] +Input [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177] (137) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] +Output [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] (138) HashAggregate [codegen id : 213] -Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] -Keys [3]: [i_brand_id#189, i_class_id#190, i_category_id#191] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197, count(1)#198] -Results [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sales#87, count(1)#198 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] +Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] +Keys [3]: [i_brand_id#178, i_class_id#179, i_category_id#180] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186, count(1)#187] +Results [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sales#78, count(1)#187 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] (139) Filter [codegen id : 213] -Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (140) Project [codegen id : 213] -Output [6]: [catalog AS channel#90, i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88] -Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] +Output [6]: [catalog AS channel#81, i_brand_id#178, i_class_id#179, i_category_id#180, sales#78, number_sales#79] +Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] (141) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum#203, isEmpty#204, count#205] +Output [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] (142) HashAggregate [codegen id : 239] -Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum#203, isEmpty#204, count#205] -Keys [3]: [i_brand_id#200, i_class_id#201, i_category_id#202] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208, count(1)#209] -Results [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208 AS sales#109, count(1)#209 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] +Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] +Keys [3]: [i_brand_id#189, i_class_id#190, i_category_id#191] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197, count(1)#198] +Results [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sales#98, count(1)#198 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] (143) Filter [codegen id : 239] -Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (144) Project [codegen id : 239] -Output [6]: [web AS channel#112, i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110] -Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] +Output [6]: [web AS channel#101, i_brand_id#189, i_class_id#190, i_category_id#191, sales#98, number_sales#99] +Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] (145) Union (146) HashAggregate [codegen id : 240] -Input [6]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64] -Keys [4]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#211, isEmpty#212, sum#213] -Results [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] +Input [6]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sales#56, number_sales#57] +Keys [4]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169] +Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] +Aggregate Attributes [3]: [sum#200, isEmpty#201, sum#202] +Results [7]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sum#203, isEmpty#204, sum#205] (147) Exchange -Input [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] -Arguments: hashpartitioning(channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, 5), ENSURE_REQUIREMENTS, [id=#217] +Input [7]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sum#203, isEmpty#204, sum#205] +Arguments: hashpartitioning(channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, 5), ENSURE_REQUIREMENTS, [id=#206] (148) HashAggregate [codegen id : 241] -Input [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] -Keys [4]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#218, sum(number_sales#64)#219] -Results [4]: [channel#68, i_brand_id#178, sum(sales#63)#218 AS sum_sales#122, sum(number_sales#64)#219 AS number_sales#123] +Input [7]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sum#203, isEmpty#204, sum#205] +Keys [4]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169] +Functions [2]: [sum(sales#56), sum(number_sales#57)] +Aggregate Attributes [2]: [sum(sales#56)#207, sum(number_sales#57)#208] +Results [4]: [channel#61, i_brand_id#167, sum(sales#56)#207 AS sum_sales#111, sum(number_sales#57)#208 AS number_sales#112] (149) HashAggregate [codegen id : 241] -Input [4]: [channel#68, i_brand_id#178, sum_sales#122, number_sales#123] -Keys [2]: [channel#68, i_brand_id#178] -Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] -Aggregate Attributes [3]: [sum#220, isEmpty#221, sum#222] -Results [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] +Input [4]: [channel#61, i_brand_id#167, sum_sales#111, number_sales#112] +Keys [2]: [channel#61, i_brand_id#167] +Functions [2]: [partial_sum(sum_sales#111), partial_sum(number_sales#112)] +Aggregate Attributes [3]: [sum#209, isEmpty#210, sum#211] +Results [5]: [channel#61, i_brand_id#167, sum#212, isEmpty#213, sum#214] (150) Exchange -Input [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] -Arguments: hashpartitioning(channel#68, i_brand_id#178, 5), ENSURE_REQUIREMENTS, [id=#226] +Input [5]: [channel#61, i_brand_id#167, sum#212, isEmpty#213, sum#214] +Arguments: hashpartitioning(channel#61, i_brand_id#167, 5), ENSURE_REQUIREMENTS, [id=#215] (151) HashAggregate [codegen id : 242] -Input [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] -Keys [2]: [channel#68, i_brand_id#178] -Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] -Aggregate Attributes [2]: [sum(sum_sales#122)#227, sum(number_sales#123)#228] -Results [6]: [channel#68, i_brand_id#178, null AS i_class_id#229, null AS i_category_id#230, sum(sum_sales#122)#227 AS sum(sum_sales)#231, sum(number_sales#123)#228 AS sum(number_sales)#232] +Input [5]: [channel#61, i_brand_id#167, sum#212, isEmpty#213, sum#214] +Keys [2]: [channel#61, i_brand_id#167] +Functions [2]: [sum(sum_sales#111), sum(number_sales#112)] +Aggregate Attributes [2]: [sum(sum_sales#111)#216, sum(number_sales#112)#217] +Results [6]: [channel#61, i_brand_id#167, null AS i_class_id#218, null AS i_category_id#219, sum(sum_sales#111)#216 AS sum(sum_sales)#220, sum(number_sales#112)#217 AS sum(number_sales)#221] (152) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] +Output [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sum#225, isEmpty#226, count#227] (153) HashAggregate [codegen id : 268] -Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] -Keys [3]: [i_brand_id#233, i_class_id#234, i_category_id#235] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241, count(1)#242] -Results [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sales#63, count(1)#242 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] +Input [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sum#225, isEmpty#226, count#227] +Keys [3]: [i_brand_id#222, i_class_id#223, i_category_id#224] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#230, count(1)#231] +Results [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sales#56, count(1)#231 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232] (154) Filter [codegen id : 268] -Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (155) Project [codegen id : 268] -Output [6]: [store AS channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64] -Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] +Output [6]: [store AS channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sales#56, number_sales#57] +Input [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232] (156) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] +Output [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] (157) HashAggregate [codegen id : 294] -Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] -Keys [3]: [i_brand_id#244, i_class_id#245, i_category_id#246] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252, count(1)#253] -Results [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sales#87, count(1)#253 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] +Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] +Keys [3]: [i_brand_id#233, i_class_id#234, i_category_id#235] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241, count(1)#242] +Results [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sales#78, count(1)#242 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] (158) Filter [codegen id : 294] -Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (159) Project [codegen id : 294] -Output [6]: [catalog AS channel#90, i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88] -Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] +Output [6]: [catalog AS channel#81, i_brand_id#233, i_class_id#234, i_category_id#235, sales#78, number_sales#79] +Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] (160) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#260] +Output [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] (161) HashAggregate [codegen id : 320] -Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#260] -Keys [3]: [i_brand_id#255, i_class_id#256, i_category_id#257] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263, count(1)#264] -Results [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263 AS sales#109, count(1)#264 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] +Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] +Keys [3]: [i_brand_id#244, i_class_id#245, i_category_id#246] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252, count(1)#253] +Results [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sales#98, count(1)#253 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] (162) Filter [codegen id : 320] -Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (163) Project [codegen id : 320] -Output [6]: [web AS channel#112, i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110] -Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] +Output [6]: [web AS channel#101, i_brand_id#244, i_class_id#245, i_category_id#246, sales#98, number_sales#99] +Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] (164) Union (165) HashAggregate [codegen id : 321] -Input [6]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64] -Keys [4]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#266, isEmpty#267, sum#268] -Results [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] +Input [6]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sales#56, number_sales#57] +Keys [4]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224] +Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] +Aggregate Attributes [3]: [sum#255, isEmpty#256, sum#257] +Results [7]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sum#258, isEmpty#259, sum#260] (166) Exchange -Input [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] -Arguments: hashpartitioning(channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, 5), ENSURE_REQUIREMENTS, [id=#272] +Input [7]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sum#258, isEmpty#259, sum#260] +Arguments: hashpartitioning(channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, 5), ENSURE_REQUIREMENTS, [id=#261] (167) HashAggregate [codegen id : 322] -Input [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] -Keys [4]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#273, sum(number_sales#64)#274] -Results [3]: [channel#68, sum(sales#63)#273 AS sum_sales#122, sum(number_sales#64)#274 AS number_sales#123] +Input [7]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sum#258, isEmpty#259, sum#260] +Keys [4]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224] +Functions [2]: [sum(sales#56), sum(number_sales#57)] +Aggregate Attributes [2]: [sum(sales#56)#262, sum(number_sales#57)#263] +Results [3]: [channel#61, sum(sales#56)#262 AS sum_sales#111, sum(number_sales#57)#263 AS number_sales#112] (168) HashAggregate [codegen id : 322] -Input [3]: [channel#68, sum_sales#122, number_sales#123] -Keys [1]: [channel#68] -Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] -Aggregate Attributes [3]: [sum#275, isEmpty#276, sum#277] -Results [4]: [channel#68, sum#278, isEmpty#279, sum#280] +Input [3]: [channel#61, sum_sales#111, number_sales#112] +Keys [1]: [channel#61] +Functions [2]: [partial_sum(sum_sales#111), partial_sum(number_sales#112)] +Aggregate Attributes [3]: [sum#264, isEmpty#265, sum#266] +Results [4]: [channel#61, sum#267, isEmpty#268, sum#269] (169) Exchange -Input [4]: [channel#68, sum#278, isEmpty#279, sum#280] -Arguments: hashpartitioning(channel#68, 5), ENSURE_REQUIREMENTS, [id=#281] +Input [4]: [channel#61, sum#267, isEmpty#268, sum#269] +Arguments: hashpartitioning(channel#61, 5), ENSURE_REQUIREMENTS, [id=#270] (170) HashAggregate [codegen id : 323] -Input [4]: [channel#68, sum#278, isEmpty#279, sum#280] -Keys [1]: [channel#68] -Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] -Aggregate Attributes [2]: [sum(sum_sales#122)#282, sum(number_sales#123)#283] -Results [6]: [channel#68, null AS i_brand_id#284, null AS i_class_id#285, null AS i_category_id#286, sum(sum_sales#122)#282 AS sum(sum_sales)#287, sum(number_sales#123)#283 AS sum(number_sales)#288] +Input [4]: [channel#61, sum#267, isEmpty#268, sum#269] +Keys [1]: [channel#61] +Functions [2]: [sum(sum_sales#111), sum(number_sales#112)] +Aggregate Attributes [2]: [sum(sum_sales#111)#271, sum(number_sales#112)#272] +Results [6]: [channel#61, null AS i_brand_id#273, null AS i_class_id#274, null AS i_category_id#275, sum(sum_sales#111)#271 AS sum(sum_sales)#276, sum(number_sales#112)#272 AS sum(number_sales)#277] (171) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] +Output [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sum#281, isEmpty#282, count#283] (172) HashAggregate [codegen id : 349] -Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] -Keys [3]: [i_brand_id#289, i_class_id#290, i_category_id#291] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297, count(1)#298] -Results [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sales#63, count(1)#298 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] +Input [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sum#281, isEmpty#282, count#283] +Keys [3]: [i_brand_id#278, i_class_id#279, i_category_id#280] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#286, count(1)#287] +Results [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#286 AS sales#56, count(1)#287 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#286 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288] (173) Filter [codegen id : 349] -Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (174) Project [codegen id : 349] -Output [6]: [store AS channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64] -Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] +Output [6]: [store AS channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sales#56, number_sales#57] +Input [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288] (175) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] +Output [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] (176) HashAggregate [codegen id : 375] -Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] -Keys [3]: [i_brand_id#300, i_class_id#301, i_category_id#302] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308, count(1)#309] -Results [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sales#87, count(1)#309 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] +Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] +Keys [3]: [i_brand_id#289, i_class_id#290, i_category_id#291] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297, count(1)#298] +Results [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sales#78, count(1)#298 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] (177) Filter [codegen id : 375] -Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (178) Project [codegen id : 375] -Output [6]: [catalog AS channel#90, i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88] -Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] +Output [6]: [catalog AS channel#81, i_brand_id#289, i_class_id#290, i_category_id#291, sales#78, number_sales#79] +Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] (179) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum#314, isEmpty#315, count#316] +Output [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] (180) HashAggregate [codegen id : 401] -Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum#314, isEmpty#315, count#316] -Keys [3]: [i_brand_id#311, i_class_id#312, i_category_id#313] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319, count(1)#320] -Results [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319 AS sales#109, count(1)#320 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] +Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] +Keys [3]: [i_brand_id#300, i_class_id#301, i_category_id#302] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308, count(1)#309] +Results [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sales#98, count(1)#309 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] (181) Filter [codegen id : 401] -Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) (182) Project [codegen id : 401] -Output [6]: [web AS channel#112, i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110] -Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] +Output [6]: [web AS channel#101, i_brand_id#300, i_class_id#301, i_category_id#302, sales#98, number_sales#99] +Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] (183) Union (184) HashAggregate [codegen id : 402] -Input [6]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64] -Keys [4]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#322, isEmpty#323, sum#324] -Results [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] +Input [6]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sales#56, number_sales#57] +Keys [4]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280] +Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] +Aggregate Attributes [3]: [sum#311, isEmpty#312, sum#313] +Results [7]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sum#314, isEmpty#315, sum#316] (185) Exchange -Input [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] -Arguments: hashpartitioning(channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, 5), ENSURE_REQUIREMENTS, [id=#328] +Input [7]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sum#314, isEmpty#315, sum#316] +Arguments: hashpartitioning(channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, 5), ENSURE_REQUIREMENTS, [id=#317] (186) HashAggregate [codegen id : 403] -Input [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] -Keys [4]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#329, sum(number_sales#64)#330] -Results [2]: [sum(sales#63)#329 AS sum_sales#122, sum(number_sales#64)#330 AS number_sales#123] +Input [7]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sum#314, isEmpty#315, sum#316] +Keys [4]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280] +Functions [2]: [sum(sales#56), sum(number_sales#57)] +Aggregate Attributes [2]: [sum(sales#56)#318, sum(number_sales#57)#319] +Results [2]: [sum(sales#56)#318 AS sum_sales#111, sum(number_sales#57)#319 AS number_sales#112] (187) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#122, number_sales#123] +Input [2]: [sum_sales#111, number_sales#112] Keys: [] -Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] -Aggregate Attributes [3]: [sum#331, isEmpty#332, sum#333] -Results [3]: [sum#334, isEmpty#335, sum#336] +Functions [2]: [partial_sum(sum_sales#111), partial_sum(number_sales#112)] +Aggregate Attributes [3]: [sum#320, isEmpty#321, sum#322] +Results [3]: [sum#323, isEmpty#324, sum#325] (188) Exchange -Input [3]: [sum#334, isEmpty#335, sum#336] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#337] +Input [3]: [sum#323, isEmpty#324, sum#325] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#326] (189) HashAggregate [codegen id : 404] -Input [3]: [sum#334, isEmpty#335, sum#336] +Input [3]: [sum#323, isEmpty#324, sum#325] Keys: [] -Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] -Aggregate Attributes [2]: [sum(sum_sales#122)#338, sum(number_sales#123)#339] -Results [6]: [null AS channel#340, null AS i_brand_id#341, null AS i_class_id#342, null AS i_category_id#343, sum(sum_sales#122)#338 AS sum(sum_sales)#344, sum(number_sales#123)#339 AS sum(number_sales)#345] +Functions [2]: [sum(sum_sales#111), sum(number_sales#112)] +Aggregate Attributes [2]: [sum(sum_sales#111)#327, sum(number_sales#112)#328] +Results [6]: [null AS channel#329, null AS i_brand_id#330, null AS i_class_id#331, null AS i_category_id#332, sum(sum_sales#111)#327 AS sum(sum_sales)#333, sum(number_sales#112)#328 AS sum(number_sales)#334] (190) Union (191) HashAggregate [codegen id : 405] -Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] -Keys [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Keys [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Results [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] (192) Exchange -Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] -Arguments: hashpartitioning(channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123, 5), ENSURE_REQUIREMENTS, [id=#346] +Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Arguments: hashpartitioning(channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112, 5), ENSURE_REQUIREMENTS, [id=#335] (193) HashAggregate [codegen id : 406] -Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] -Keys [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Keys [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Results [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] (194) TakeOrderedAndProject -Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] -Arguments: 100, [channel#68 ASC NULLS FIRST, i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Arguments: 100, [channel#61 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#66, [id=#67] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#59, [id=#60] * HashAggregate (221) +- Exchange (220) +- * HashAggregate (219) @@ -1145,193 +1145,193 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (195) Scan parquet default.store_sales -Output [3]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349] +Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#349), dynamicpruningexpression(ss_sold_date_sk#349 IN dynamicpruning#350)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#336)] ReadSchema: struct (196) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349] +Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (197) Scan parquet default.date_dim -Output [2]: [d_date_sk#351, d_year#352] +Output [2]: [d_date_sk#27, d_year#44] 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 (198) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#351, d_year#352] +Input [2]: [d_date_sk#27, d_year#44] (199) Filter [codegen id : 1] -Input [2]: [d_date_sk#351, d_year#352] -Condition : (((isnotnull(d_year#352) AND (d_year#352 >= 1999)) AND (d_year#352 <= 2001)) AND isnotnull(d_date_sk#351)) +Input [2]: [d_date_sk#27, d_year#44] +Condition : (((isnotnull(d_year#44) AND (d_year#44 >= 1999)) AND (d_year#44 <= 2001)) AND isnotnull(d_date_sk#27)) (200) Project [codegen id : 1] -Output [1]: [d_date_sk#351] -Input [2]: [d_date_sk#351, d_year#352] +Output [1]: [d_date_sk#27] +Input [2]: [d_date_sk#27, d_year#44] (201) BroadcastExchange -Input [1]: [d_date_sk#351] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#353] +Input [1]: [d_date_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#337] (202) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#349] -Right keys [1]: [d_date_sk#351] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#27] Join condition: None (203) Project [codegen id : 2] -Output [2]: [ss_quantity#347 AS quantity#354, ss_list_price#348 AS list_price#355] -Input [4]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349, d_date_sk#351] +Output [2]: [ss_quantity#2 AS quantity#338, ss_list_price#3 AS list_price#339] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#27] (204) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358] +Output [3]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#358), dynamicpruningexpression(cs_sold_date_sk#358 IN dynamicpruning#359)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#340)] ReadSchema: struct (205) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358] +Input [3]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] (206) Scan parquet default.date_dim -Output [2]: [d_date_sk#360, d_year#361] +Output [2]: [d_date_sk#341, d_year#342] 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 (207) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#360, d_year#361] +Input [2]: [d_date_sk#341, d_year#342] (208) Filter [codegen id : 3] -Input [2]: [d_date_sk#360, d_year#361] -Condition : (((isnotnull(d_year#361) AND (d_year#361 >= 1998)) AND (d_year#361 <= 2000)) AND isnotnull(d_date_sk#360)) +Input [2]: [d_date_sk#341, d_year#342] +Condition : (((isnotnull(d_year#342) AND (d_year#342 >= 1998)) AND (d_year#342 <= 2000)) AND isnotnull(d_date_sk#341)) (209) Project [codegen id : 3] -Output [1]: [d_date_sk#360] -Input [2]: [d_date_sk#360, d_year#361] +Output [1]: [d_date_sk#341] +Input [2]: [d_date_sk#341, d_year#342] (210) BroadcastExchange -Input [1]: [d_date_sk#360] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#362] +Input [1]: [d_date_sk#341] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#343] (211) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#358] -Right keys [1]: [d_date_sk#360] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#341] Join condition: None (212) Project [codegen id : 4] -Output [2]: [cs_quantity#356 AS quantity#363, cs_list_price#357 AS list_price#364] -Input [4]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358, d_date_sk#360] +Output [2]: [cs_quantity#62 AS quantity#344, cs_list_price#63 AS list_price#345] +Input [4]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, d_date_sk#341] (213) Scan parquet default.web_sales -Output [3]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367] +Output [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#367), dynamicpruningexpression(ws_sold_date_sk#367 IN dynamicpruning#359)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#340)] ReadSchema: struct (214) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367] +Input [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] (215) ReusedExchange [Reuses operator id: 210] -Output [1]: [d_date_sk#368] +Output [1]: [d_date_sk#346] (216) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#367] -Right keys [1]: [d_date_sk#368] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#346] Join condition: None (217) Project [codegen id : 6] -Output [2]: [ws_quantity#365 AS quantity#369, ws_list_price#366 AS list_price#370] -Input [4]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367, d_date_sk#368] +Output [2]: [ws_quantity#82 AS quantity#347, ws_list_price#83 AS list_price#348] +Input [4]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, d_date_sk#346] (218) Union (219) HashAggregate [codegen id : 7] -Input [2]: [quantity#354, list_price#355] +Input [2]: [quantity#338, list_price#339] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#371, count#372] -Results [2]: [sum#373, count#374] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#338 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#339 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#349, count#350] +Results [2]: [sum#351, count#352] (220) Exchange -Input [2]: [sum#373, count#374] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#375] +Input [2]: [sum#351, count#352] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#353] (221) HashAggregate [codegen id : 8] -Input [2]: [sum#373, count#374] +Input [2]: [sum#351, count#352] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))#376] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))#376 AS average_sales#377] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#338 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#339 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#338 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#339 as decimal(12,2)))), DecimalType(18,2), true))#354] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#338 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#339 as decimal(12,2)))), DecimalType(18,2), true))#354 AS average_sales#355] -Subquery:2 Hosting operator id = 195 Hosting Expression = ss_sold_date_sk#349 IN dynamicpruning#350 +Subquery:2 Hosting operator id = 195 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#336 ReusedExchange (222) (222) ReusedExchange [Reuses operator id: 201] -Output [1]: [d_date_sk#351] +Output [1]: [d_date_sk#27] -Subquery:3 Hosting operator id = 204 Hosting Expression = cs_sold_date_sk#358 IN dynamicpruning#359 +Subquery:3 Hosting operator id = 204 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#340 ReusedExchange (223) (223) ReusedExchange [Reuses operator id: 210] -Output [1]: [d_date_sk#360] +Output [1]: [d_date_sk#341] -Subquery:4 Hosting operator id = 213 Hosting Expression = ws_sold_date_sk#367 IN dynamicpruning#359 +Subquery:4 Hosting operator id = 213 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#340 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (224) (224) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#50] +Output [1]: [d_date_sk#27] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 ReusedExchange (225) (225) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#27] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#10 -Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 -Subquery:13 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:13 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:14 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:14 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:15 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:15 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:16 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:16 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:17 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:17 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:18 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:18 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:19 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:19 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:20 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:20 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:21 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:21 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:22 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:22 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:23 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:23 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] -Subquery:24 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:24 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] 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 56fa48006d894..15f74a7802b27 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 @@ -393,484 +393,484 @@ Aggregate Attributes [7]: [avg(agg1#37)#73, avg(agg2#38)#74, avg(agg3#39)#75, av Results [11]: [i_item_id#20, ca_country#34, ca_state#33, ca_county#32, avg(agg1#37)#73 AS agg1#80, avg(agg2#38)#74 AS agg2#81, avg(agg3#39)#75 AS agg3#82, avg(agg4#40)#76 AS agg4#83, avg(agg5#41)#77 AS agg5#84, avg(agg6#42)#78 AS agg6#85, avg(agg7#43)#79 AS agg7#86] (52) ReusedExchange [Reuses operator id: unknown] -Output [8]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94] +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#20] (53) Sort [codegen id : 19] -Input [8]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94] -Arguments: [cs_bill_customer_sk#87 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#20] +Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (54) Scan parquet default.customer -Output [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] +Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 21] -Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] (56) Filter [codegen id : 21] -Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] -Condition : (((c_birth_month#98 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#95)) AND isnotnull(c_current_cdemo_sk#96)) AND isnotnull(c_current_addr_sk#97)) +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) (57) Project [codegen id : 21] -Output [4]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#99] -Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] +Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] (58) Scan parquet default.customer_address -Output [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Output [3]: [ca_address_sk#31, ca_state#33, ca_country#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] (60) Filter [codegen id : 20] -Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] -Condition : (ca_state#101 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#100)) +Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Condition : (ca_state#33 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#31)) (61) BroadcastExchange -Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] +Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#87] (62) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [c_current_addr_sk#97] -Right keys [1]: [ca_address_sk#100] +Left keys [1]: [c_current_addr_sk#25] +Right keys [1]: [ca_address_sk#31] Join condition: None (63) Project [codegen id : 21] -Output [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] -Input [7]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#99, ca_address_sk#100, ca_state#101, ca_country#102] +Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34] +Input [7]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31, ca_state#33, ca_country#34] (64) Exchange -Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] -Arguments: hashpartitioning(c_current_cdemo_sk#96, 5), ENSURE_REQUIREMENTS, [id=#104] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34] +Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#88] (65) Sort [codegen id : 22] -Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] -Arguments: [c_current_cdemo_sk#96 ASC NULLS FIRST], false, 0 +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34] +Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 (66) ReusedExchange [Reuses operator id: 35] -Output [1]: [cd_demo_sk#105] +Output [1]: [cd_demo_sk#89] (67) Sort [codegen id : 24] -Input [1]: [cd_demo_sk#105] -Arguments: [cd_demo_sk#105 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#89] +Arguments: [cd_demo_sk#89 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 25] -Left keys [1]: [c_current_cdemo_sk#96] -Right keys [1]: [cd_demo_sk#105] +Left keys [1]: [c_current_cdemo_sk#24] +Right keys [1]: [cd_demo_sk#89] Join condition: None (69) Project [codegen id : 25] -Output [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] -Input [6]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102, cd_demo_sk#105] +Output [4]: [c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] +Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34, cd_demo_sk#89] (70) Exchange -Input [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] -Arguments: hashpartitioning(c_customer_sk#95, 5), ENSURE_REQUIREMENTS, [id=#106] +Input [4]: [c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#90] (71) Sort [codegen id : 26] -Input [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] -Arguments: [c_customer_sk#95 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] +Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 (72) SortMergeJoin [codegen id : 27] -Left keys [1]: [cs_bill_customer_sk#87] -Right keys [1]: [c_customer_sk#95] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#23] Join condition: None (73) Project [codegen id : 27] -Output [10]: [i_item_id#94, ca_country#102, ca_state#101, cast(cs_quantity#88 as decimal(12,2)) AS agg1#37, cast(cs_list_price#89 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#91 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#90 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#92 as decimal(12,2)) AS agg5#41, cast(c_birth_year#99 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#93 as decimal(12,2)) AS agg7#43] -Input [12]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94, c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Output [10]: [i_item_id#20, ca_country#34, ca_state#33, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] +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#20, c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] (74) HashAggregate [codegen id : 27] -Input [10]: [i_item_id#94, ca_country#102, ca_state#101, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] -Keys [3]: [i_item_id#94, ca_country#102, ca_state#101] +Input [10]: [i_item_id#20, ca_country#34, ca_state#33, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [3]: [i_item_id#20, ca_country#34, ca_state#33] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] -Results [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] +Aggregate Attributes [14]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100, sum#101, count#102, sum#103, count#104] +Results [17]: [i_item_id#20, ca_country#34, ca_state#33, 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] (75) Exchange -Input [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] -Arguments: hashpartitioning(i_item_id#94, ca_country#102, ca_state#101, 5), ENSURE_REQUIREMENTS, [id=#135] +Input [17]: [i_item_id#20, ca_country#34, ca_state#33, 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] +Arguments: hashpartitioning(i_item_id#20, ca_country#34, ca_state#33, 5), ENSURE_REQUIREMENTS, [id=#119] (76) HashAggregate [codegen id : 28] -Input [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] -Keys [3]: [i_item_id#94, ca_country#102, ca_state#101] +Input [17]: [i_item_id#20, ca_country#34, ca_state#33, 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] +Keys [3]: [i_item_id#20, ca_country#34, ca_state#33] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#136, avg(agg2#38)#137, avg(agg3#39)#138, avg(agg4#40)#139, avg(agg5#41)#140, avg(agg6#42)#141, avg(agg7#43)#142] -Results [11]: [i_item_id#94, ca_country#102, ca_state#101, null AS county#143, avg(agg1#37)#136 AS agg1#144, avg(agg2#38)#137 AS agg2#145, avg(agg3#39)#138 AS agg3#146, avg(agg4#40)#139 AS agg4#147, avg(agg5#41)#140 AS agg5#148, avg(agg6#42)#141 AS agg6#149, avg(agg7#43)#142 AS agg7#150] +Aggregate Attributes [7]: [avg(agg1#37)#120, avg(agg2#38)#121, avg(agg3#39)#122, avg(agg4#40)#123, avg(agg5#41)#124, avg(agg6#42)#125, avg(agg7#43)#126] +Results [11]: [i_item_id#20, ca_country#34, ca_state#33, null AS county#127, avg(agg1#37)#120 AS agg1#128, avg(agg2#38)#121 AS agg2#129, avg(agg3#39)#122 AS agg3#130, avg(agg4#40)#123 AS agg4#131, avg(agg5#41)#124 AS agg5#132, avg(agg6#42)#125 AS agg6#133, avg(agg7#43)#126 AS agg7#134] (77) ReusedExchange [Reuses operator id: unknown] -Output [8]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158] +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#20] (78) Sort [codegen id : 33] -Input [8]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158] -Arguments: [cs_bill_customer_sk#151 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#20] +Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer -Output [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] +Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (80) ColumnarToRow [codegen id : 35] -Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] (81) Filter [codegen id : 35] -Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] -Condition : (((c_birth_month#162 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#159)) AND isnotnull(c_current_cdemo_sk#160)) AND isnotnull(c_current_addr_sk#161)) +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) (82) Project [codegen id : 35] -Output [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_year#163] -Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] +Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] (83) Scan parquet default.customer_address -Output [3]: [ca_address_sk#164, ca_state#165, ca_country#166] +Output [3]: [ca_address_sk#31, ca_state#33, ca_country#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (84) ColumnarToRow [codegen id : 34] -Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] +Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] (85) Filter [codegen id : 34] -Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] -Condition : (ca_state#165 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#164)) +Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Condition : (ca_state#33 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#31)) (86) Project [codegen id : 34] -Output [2]: [ca_address_sk#164, ca_country#166] -Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] +Output [2]: [ca_address_sk#31, ca_country#34] +Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] (87) BroadcastExchange -Input [2]: [ca_address_sk#164, ca_country#166] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#167] +Input [2]: [ca_address_sk#31, ca_country#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#135] (88) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [c_current_addr_sk#161] -Right keys [1]: [ca_address_sk#164] +Left keys [1]: [c_current_addr_sk#25] +Right keys [1]: [ca_address_sk#31] Join condition: None (89) Project [codegen id : 35] -Output [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] -Input [6]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_year#163, ca_address_sk#164, ca_country#166] +Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34] +Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31, ca_country#34] (90) Exchange -Input [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] -Arguments: hashpartitioning(c_current_cdemo_sk#160, 5), ENSURE_REQUIREMENTS, [id=#168] +Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34] +Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#136] (91) Sort [codegen id : 36] -Input [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] -Arguments: [c_current_cdemo_sk#160 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34] +Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 (92) ReusedExchange [Reuses operator id: 35] -Output [1]: [cd_demo_sk#169] +Output [1]: [cd_demo_sk#137] (93) Sort [codegen id : 38] -Input [1]: [cd_demo_sk#169] -Arguments: [cd_demo_sk#169 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#137] +Arguments: [cd_demo_sk#137 ASC NULLS FIRST], false, 0 (94) SortMergeJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#160] -Right keys [1]: [cd_demo_sk#169] +Left keys [1]: [c_current_cdemo_sk#24] +Right keys [1]: [cd_demo_sk#137] Join condition: None (95) Project [codegen id : 39] -Output [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] -Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166, cd_demo_sk#169] +Output [3]: [c_customer_sk#23, c_birth_year#27, ca_country#34] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34, cd_demo_sk#137] (96) Exchange -Input [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] -Arguments: hashpartitioning(c_customer_sk#159, 5), ENSURE_REQUIREMENTS, [id=#170] +Input [3]: [c_customer_sk#23, c_birth_year#27, ca_country#34] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#138] (97) Sort [codegen id : 40] -Input [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] -Arguments: [c_customer_sk#159 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#23, c_birth_year#27, ca_country#34] +Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 41] -Left keys [1]: [cs_bill_customer_sk#151] -Right keys [1]: [c_customer_sk#159] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#23] Join condition: None (99) Project [codegen id : 41] -Output [9]: [i_item_id#158, ca_country#166, cast(cs_quantity#152 as decimal(12,2)) AS agg1#37, cast(cs_list_price#153 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#155 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#154 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#156 as decimal(12,2)) AS agg5#41, cast(c_birth_year#163 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#157 as decimal(12,2)) AS agg7#43] -Input [11]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158, c_customer_sk#159, c_birth_year#163, ca_country#166] +Output [9]: [i_item_id#20, ca_country#34, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] +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#20, c_customer_sk#23, c_birth_year#27, ca_country#34] (100) HashAggregate [codegen id : 41] -Input [9]: [i_item_id#158, ca_country#166, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] -Keys [2]: [i_item_id#158, ca_country#166] +Input [9]: [i_item_id#20, ca_country#34, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [2]: [i_item_id#20, ca_country#34] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178, sum#179, count#180, sum#181, count#182, sum#183, count#184] -Results [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] +Aggregate Attributes [14]: [sum#139, count#140, sum#141, count#142, sum#143, count#144, sum#145, count#146, sum#147, count#148, sum#149, count#150, sum#151, count#152] +Results [16]: [i_item_id#20, ca_country#34, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] (101) Exchange -Input [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] -Arguments: hashpartitioning(i_item_id#158, ca_country#166, 5), ENSURE_REQUIREMENTS, [id=#199] +Input [16]: [i_item_id#20, ca_country#34, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] +Arguments: hashpartitioning(i_item_id#20, ca_country#34, 5), ENSURE_REQUIREMENTS, [id=#167] (102) HashAggregate [codegen id : 42] -Input [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] -Keys [2]: [i_item_id#158, ca_country#166] +Input [16]: [i_item_id#20, ca_country#34, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] +Keys [2]: [i_item_id#20, ca_country#34] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#200, avg(agg2#38)#201, avg(agg3#39)#202, avg(agg4#40)#203, avg(agg5#41)#204, avg(agg6#42)#205, avg(agg7#43)#206] -Results [11]: [i_item_id#158, ca_country#166, null AS ca_state#207, null AS county#208, avg(agg1#37)#200 AS agg1#209, avg(agg2#38)#201 AS agg2#210, avg(agg3#39)#202 AS agg3#211, avg(agg4#40)#203 AS agg4#212, avg(agg5#41)#204 AS agg5#213, avg(agg6#42)#205 AS agg6#214, avg(agg7#43)#206 AS agg7#215] +Aggregate Attributes [7]: [avg(agg1#37)#168, avg(agg2#38)#169, avg(agg3#39)#170, avg(agg4#40)#171, avg(agg5#41)#172, avg(agg6#42)#173, avg(agg7#43)#174] +Results [11]: [i_item_id#20, ca_country#34, null AS ca_state#175, null AS county#176, avg(agg1#37)#168 AS agg1#177, avg(agg2#38)#169 AS agg2#178, avg(agg3#39)#170 AS agg3#179, avg(agg4#40)#171 AS agg4#180, avg(agg5#41)#172 AS agg5#181, avg(agg6#42)#173 AS agg6#182, avg(agg7#43)#174 AS agg7#183] (103) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] +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#224), dynamicpruningexpression(cs_sold_date_sk#224 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 (104) ColumnarToRow [codegen id : 49] -Input [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] +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] (105) Filter [codegen id : 49] -Input [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] -Condition : ((isnotnull(cs_bill_cdemo_sk#217) AND isnotnull(cs_bill_customer_sk#216)) AND isnotnull(cs_item_sk#218)) +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)) (106) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#225, cd_dep_count#226] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (107) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_cdemo_sk#217] -Right keys [1]: [cd_demo_sk#225] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (108) Project [codegen id : 49] -Output [9]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_dep_count#226] -Input [11]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_demo_sk#225, cd_dep_count#226] +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] (109) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#227] +Output [1]: [d_date_sk#16] (110) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_sold_date_sk#224] -Right keys [1]: [d_date_sk#227] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (111) Project [codegen id : 49] -Output [8]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226] -Input [10]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_dep_count#226, d_date_sk#227] +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] (112) Scan parquet default.customer -Output [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] +Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (113) ColumnarToRow [codegen id : 46] -Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] (114) Filter [codegen id : 46] -Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] -Condition : (((c_birth_month#231 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#228)) AND isnotnull(c_current_cdemo_sk#229)) AND isnotnull(c_current_addr_sk#230)) +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) (115) Project [codegen id : 46] -Output [4]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_year#232] -Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] +Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] (116) Scan parquet default.customer_address -Output [2]: [ca_address_sk#233, ca_state#234] +Output [2]: [ca_address_sk#31, ca_state#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (117) ColumnarToRow [codegen id : 45] -Input [2]: [ca_address_sk#233, ca_state#234] +Input [2]: [ca_address_sk#31, ca_state#33] (118) Filter [codegen id : 45] -Input [2]: [ca_address_sk#233, ca_state#234] -Condition : (ca_state#234 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#233)) +Input [2]: [ca_address_sk#31, ca_state#33] +Condition : (ca_state#33 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#31)) (119) Project [codegen id : 45] -Output [1]: [ca_address_sk#233] -Input [2]: [ca_address_sk#233, ca_state#234] +Output [1]: [ca_address_sk#31] +Input [2]: [ca_address_sk#31, ca_state#33] (120) BroadcastExchange -Input [1]: [ca_address_sk#233] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#235] +Input [1]: [ca_address_sk#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#184] (121) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [c_current_addr_sk#230] -Right keys [1]: [ca_address_sk#233] +Left keys [1]: [c_current_addr_sk#25] +Right keys [1]: [ca_address_sk#31] Join condition: None (122) Project [codegen id : 46] -Output [3]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232] -Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_year#232, ca_address_sk#233] +Output [3]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31] (123) BroadcastExchange -Input [3]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#236] +Input [3]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#185] (124) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#237] +Output [1]: [cd_demo_sk#186] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (125) ColumnarToRow -Input [1]: [cd_demo_sk#237] +Input [1]: [cd_demo_sk#186] (126) Filter -Input [1]: [cd_demo_sk#237] -Condition : isnotnull(cd_demo_sk#237) +Input [1]: [cd_demo_sk#186] +Condition : isnotnull(cd_demo_sk#186) (127) BroadcastHashJoin [codegen id : 47] -Left keys [1]: [c_current_cdemo_sk#229] -Right keys [1]: [cd_demo_sk#237] +Left keys [1]: [c_current_cdemo_sk#24] +Right keys [1]: [cd_demo_sk#186] Join condition: None (128) Project [codegen id : 47] -Output [2]: [c_customer_sk#228, c_birth_year#232] -Input [4]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232, cd_demo_sk#237] +Output [2]: [c_customer_sk#23, c_birth_year#27] +Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, cd_demo_sk#186] (129) BroadcastExchange -Input [2]: [c_customer_sk#228, c_birth_year#232] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#238] +Input [2]: [c_customer_sk#23, c_birth_year#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#187] (130) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_customer_sk#216] -Right keys [1]: [c_customer_sk#228] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#23] Join condition: None (131) Project [codegen id : 49] -Output [8]: [cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_birth_year#232] -Input [10]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_customer_sk#228, c_birth_year#232] +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#27] +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#23, c_birth_year#27] (132) ReusedExchange [Reuses operator id: 21] -Output [2]: [i_item_sk#239, i_item_id#240] +Output [2]: [i_item_sk#19, i_item_id#20] (133) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_item_sk#218] -Right keys [1]: [i_item_sk#239] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#19] Join condition: None (134) Project [codegen id : 49] -Output [8]: [i_item_id#240, cast(cs_quantity#219 as decimal(12,2)) AS agg1#37, cast(cs_list_price#220 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#222 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#221 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#223 as decimal(12,2)) AS agg5#41, cast(c_birth_year#232 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#226 as decimal(12,2)) AS agg7#43] -Input [10]: [cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_birth_year#232, i_item_sk#239, i_item_id#240] +Output [8]: [i_item_id#20, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] +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#27, i_item_sk#19, i_item_id#20] (135) HashAggregate [codegen id : 49] -Input [8]: [i_item_id#240, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] -Keys [1]: [i_item_id#240] +Input [8]: [i_item_id#20, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [1]: [i_item_id#20] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#241, count#242, sum#243, count#244, sum#245, count#246, sum#247, count#248, sum#249, count#250, sum#251, count#252, sum#253, count#254] -Results [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] +Aggregate Attributes [14]: [sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201] +Results [15]: [i_item_id#20, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215] (136) Exchange -Input [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] -Arguments: hashpartitioning(i_item_id#240, 5), ENSURE_REQUIREMENTS, [id=#269] +Input [15]: [i_item_id#20, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215] +Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, [id=#216] (137) HashAggregate [codegen id : 50] -Input [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] -Keys [1]: [i_item_id#240] +Input [15]: [i_item_id#20, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215] +Keys [1]: [i_item_id#20] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#270, avg(agg2#38)#271, avg(agg3#39)#272, avg(agg4#40)#273, avg(agg5#41)#274, avg(agg6#42)#275, avg(agg7#43)#276] -Results [11]: [i_item_id#240, null AS ca_country#277, null AS ca_state#278, null AS county#279, avg(agg1#37)#270 AS agg1#280, avg(agg2#38)#271 AS agg2#281, avg(agg3#39)#272 AS agg3#282, avg(agg4#40)#273 AS agg4#283, avg(agg5#41)#274 AS agg5#284, avg(agg6#42)#275 AS agg6#285, avg(agg7#43)#276 AS agg7#286] +Aggregate Attributes [7]: [avg(agg1#37)#217, avg(agg2#38)#218, avg(agg3#39)#219, avg(agg4#40)#220, avg(agg5#41)#221, avg(agg6#42)#222, avg(agg7#43)#223] +Results [11]: [i_item_id#20, null AS ca_country#224, null AS ca_state#225, null AS county#226, avg(agg1#37)#217 AS agg1#227, avg(agg2#38)#218 AS agg2#228, avg(agg3#39)#219 AS agg3#229, avg(agg4#40)#220 AS agg4#230, avg(agg5#41)#221 AS agg5#231, avg(agg6#42)#222 AS agg6#232, avg(agg7#43)#223 AS agg7#233] (138) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] +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#295), dynamicpruningexpression(cs_sold_date_sk#295 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 (139) ColumnarToRow [codegen id : 57] -Input [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] +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] (140) Filter [codegen id : 57] -Input [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] -Condition : ((isnotnull(cs_bill_cdemo_sk#288) AND isnotnull(cs_bill_customer_sk#287)) AND isnotnull(cs_item_sk#289)) +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)) (141) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#296, cd_dep_count#297] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (142) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_cdemo_sk#288] -Right keys [1]: [cd_demo_sk#296] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (143) Project [codegen id : 57] -Output [9]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_dep_count#297] -Input [11]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_demo_sk#296, cd_dep_count#297] +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] (144) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#298] +Output [1]: [d_date_sk#16] (145) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_sold_date_sk#295] -Right keys [1]: [d_date_sk#298] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (146) Project [codegen id : 57] -Output [8]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297] -Input [10]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_dep_count#297, d_date_sk#298] +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] (147) Scan parquet default.item -Output [1]: [i_item_sk#299] +Output [1]: [i_item_sk#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (148) ColumnarToRow [codegen id : 53] -Input [1]: [i_item_sk#299] +Input [1]: [i_item_sk#19] (149) Filter [codegen id : 53] -Input [1]: [i_item_sk#299] -Condition : isnotnull(i_item_sk#299) +Input [1]: [i_item_sk#19] +Condition : isnotnull(i_item_sk#19) (150) BroadcastExchange -Input [1]: [i_item_sk#299] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#300] +Input [1]: [i_item_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#234] (151) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_item_sk#289] -Right keys [1]: [i_item_sk#299] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#19] Join condition: None (152) Project [codegen id : 57] -Output [7]: [cs_bill_customer_sk#287, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297] -Input [9]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297, i_item_sk#299] +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#19] (153) ReusedExchange [Reuses operator id: 129] -Output [2]: [c_customer_sk#301, c_birth_year#302] +Output [2]: [c_customer_sk#23, c_birth_year#27] (154) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_customer_sk#287] -Right keys [1]: [c_customer_sk#301] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#23] Join condition: None (155) Project [codegen id : 57] -Output [7]: [cast(cs_quantity#290 as decimal(12,2)) AS agg1#37, cast(cs_list_price#291 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#293 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#292 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#294 as decimal(12,2)) AS agg5#41, cast(c_birth_year#302 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#297 as decimal(12,2)) AS agg7#43] -Input [9]: [cs_bill_customer_sk#287, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297, c_customer_sk#301, c_birth_year#302] +Output [7]: [cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] +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#23, c_birth_year#27] (156) HashAggregate [codegen id : 57] Input [7]: [agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] Keys: [] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#303, count#304, sum#305, count#306, sum#307, count#308, sum#309, count#310, sum#311, count#312, sum#313, count#314, sum#315, count#316] -Results [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] +Aggregate Attributes [14]: [sum#235, count#236, sum#237, count#238, sum#239, count#240, sum#241, count#242, sum#243, count#244, sum#245, count#246, sum#247, count#248] +Results [14]: [sum#249, count#250, sum#251, count#252, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262] (157) Exchange -Input [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#331] +Input [14]: [sum#249, count#250, sum#251, count#252, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#263] (158) HashAggregate [codegen id : 58] -Input [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] +Input [14]: [sum#249, count#250, sum#251, count#252, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262] Keys: [] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#332, avg(agg2#38)#333, avg(agg3#39)#334, avg(agg4#40)#335, avg(agg5#41)#336, avg(agg6#42)#337, avg(agg7#43)#338] -Results [11]: [null AS i_item_id#339, null AS ca_country#340, null AS ca_state#341, null AS county#342, avg(agg1#37)#332 AS agg1#343, avg(agg2#38)#333 AS agg2#344, avg(agg3#39)#334 AS agg3#345, avg(agg4#40)#335 AS agg4#346, avg(agg5#41)#336 AS agg5#347, avg(agg6#42)#337 AS agg6#348, avg(agg7#43)#338 AS agg7#349] +Aggregate Attributes [7]: [avg(agg1#37)#264, avg(agg2#38)#265, avg(agg3#39)#266, avg(agg4#40)#267, avg(agg5#41)#268, avg(agg6#42)#269, avg(agg7#43)#270] +Results [11]: [null AS i_item_id#271, null AS ca_country#272, null AS ca_state#273, null AS county#274, avg(agg1#37)#264 AS agg1#275, avg(agg2#38)#265 AS agg2#276, avg(agg3#39)#266 AS agg3#277, avg(agg4#40)#267 AS agg4#278, avg(agg5#41)#268 AS agg5#279, avg(agg6#42)#269 AS agg6#280, avg(agg7#43)#270 AS agg7#281] (159) Union @@ -887,8 +887,8 @@ ReusedExchange (161) (161) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#16] -Subquery:2 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#224 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 138 Hosting Expression = cs_sold_date_sk#295 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 138 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 8e3c5958d2831..0cb9a331b406a 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 @@ -366,492 +366,492 @@ Aggregate Attributes [7]: [avg(agg1#35)#71, avg(agg2#36)#72, avg(agg3#37)#73, av Results [11]: [i_item_id#33, ca_country#27, ca_state#26, ca_county#25, 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] (46) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93] +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#93), dynamicpruningexpression(cs_sold_date_sk#93 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 (47) ColumnarToRow [codegen id : 15] -Input [9]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93] +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] (48) Filter [codegen id : 15] -Input [9]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93] -Condition : ((isnotnull(cs_bill_cdemo_sk#86) AND isnotnull(cs_bill_customer_sk#85)) AND isnotnull(cs_item_sk#87)) +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)) (49) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#94, cd_dep_count#95] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (50) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_cdemo_sk#86] -Right keys [1]: [cd_demo_sk#94] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (51) Project [codegen id : 15] -Output [9]: [cs_bill_customer_sk#85, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95] -Input [11]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_demo_sk#94, cd_dep_count#95] +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] (52) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#96, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99] +Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] (53) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_customer_sk#85] -Right keys [1]: [c_customer_sk#96] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#16] Join condition: None (54) Project [codegen id : 15] -Output [11]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99] -Input [13]: [cs_bill_customer_sk#85, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_customer_sk#96, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99] +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] (55) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#100] +Output [1]: [cd_demo_sk#85] (56) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_cdemo_sk#97] -Right keys [1]: [cd_demo_sk#100] +Left keys [1]: [c_current_cdemo_sk#17] +Right keys [1]: [cd_demo_sk#85] Join condition: None (57) Project [codegen id : 15] -Output [10]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_addr_sk#98, c_birth_year#99] -Input [12]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99, cd_demo_sk#100] +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#85] (58) Scan parquet default.customer_address -Output [3]: [ca_address_sk#101, ca_state#102, ca_country#103] +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, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 12] -Input [3]: [ca_address_sk#101, ca_state#102, ca_country#103] +Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] (60) Filter [codegen id : 12] -Input [3]: [ca_address_sk#101, ca_state#102, ca_country#103] -Condition : (ca_state#102 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#101)) +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)) (61) BroadcastExchange -Input [3]: [ca_address_sk#101, ca_state#102, ca_country#103] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#104] +Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#86] (62) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_addr_sk#98] -Right keys [1]: [ca_address_sk#101] +Left keys [1]: [c_current_addr_sk#18] +Right keys [1]: [ca_address_sk#24] Join condition: None (63) Project [codegen id : 15] -Output [11]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103] -Input [13]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_addr_sk#98, c_birth_year#99, ca_address_sk#101, ca_state#102, ca_country#103] +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] (64) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#105] +Output [1]: [d_date_sk#29] (65) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#93] -Right keys [1]: [d_date_sk#105] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#29] Join condition: None (66) Project [codegen id : 15] -Output [10]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103] -Input [12]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103, d_date_sk#105] +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] (67) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#106, i_item_id#107] +Output [2]: [i_item_sk#32, i_item_id#33] (68) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_item_sk#87] -Right keys [1]: [i_item_sk#106] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#32] Join condition: None (69) Project [codegen id : 15] -Output [10]: [i_item_id#107, ca_country#103, ca_state#102, cast(cs_quantity#88 as decimal(12,2)) AS agg1#35, cast(cs_list_price#89 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#91 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#90 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#92 as decimal(12,2)) AS agg5#39, cast(c_birth_year#99 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#95 as decimal(12,2)) AS agg7#41] -Input [12]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103, i_item_sk#106, i_item_id#107] +Output [10]: [i_item_id#33, ca_country#27, ca_state#26, 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#20 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] +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#32, i_item_id#33] (70) HashAggregate [codegen id : 15] -Input [10]: [i_item_id#107, ca_country#103, ca_state#102, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [3]: [i_item_id#107, ca_country#103, ca_state#102] +Input [10]: [i_item_id#33, ca_country#27, ca_state#26, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [3]: [i_item_id#33, ca_country#27, ca_state#26] 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#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117, sum#118, count#119, sum#120, count#121] -Results [17]: [i_item_id#107, ca_country#103, ca_state#102, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] +Aggregate Attributes [14]: [sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100] +Results [17]: [i_item_id#33, ca_country#27, ca_state#26, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] (71) Exchange -Input [17]: [i_item_id#107, ca_country#103, ca_state#102, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] -Arguments: hashpartitioning(i_item_id#107, ca_country#103, ca_state#102, 5), ENSURE_REQUIREMENTS, [id=#136] +Input [17]: [i_item_id#33, ca_country#27, ca_state#26, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] +Arguments: hashpartitioning(i_item_id#33, ca_country#27, ca_state#26, 5), ENSURE_REQUIREMENTS, [id=#115] (72) HashAggregate [codegen id : 16] -Input [17]: [i_item_id#107, ca_country#103, ca_state#102, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] -Keys [3]: [i_item_id#107, ca_country#103, ca_state#102] +Input [17]: [i_item_id#33, ca_country#27, ca_state#26, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] +Keys [3]: [i_item_id#33, ca_country#27, ca_state#26] 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)#137, avg(agg2#36)#138, avg(agg3#37)#139, avg(agg4#38)#140, avg(agg5#39)#141, avg(agg6#40)#142, avg(agg7#41)#143] -Results [11]: [i_item_id#107, ca_country#103, ca_state#102, null AS county#144, avg(agg1#35)#137 AS agg1#145, avg(agg2#36)#138 AS agg2#146, avg(agg3#37)#139 AS agg3#147, avg(agg4#38)#140 AS agg4#148, avg(agg5#39)#141 AS agg5#149, avg(agg6#40)#142 AS agg6#150, avg(agg7#41)#143 AS agg7#151] +Aggregate Attributes [7]: [avg(agg1#35)#116, avg(agg2#36)#117, avg(agg3#37)#118, avg(agg4#38)#119, avg(agg5#39)#120, avg(agg6#40)#121, avg(agg7#41)#122] +Results [11]: [i_item_id#33, ca_country#27, ca_state#26, null AS county#123, avg(agg1#35)#116 AS agg1#124, avg(agg2#36)#117 AS agg2#125, avg(agg3#37)#118 AS agg3#126, avg(agg4#38)#119 AS agg4#127, avg(agg5#39)#120 AS agg5#128, avg(agg6#40)#121 AS agg6#129, avg(agg7#41)#122 AS agg7#130] (73) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160] +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#160), dynamicpruningexpression(cs_sold_date_sk#160 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 (74) ColumnarToRow [codegen id : 23] -Input [9]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160] +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] (75) Filter [codegen id : 23] -Input [9]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160] -Condition : ((isnotnull(cs_bill_cdemo_sk#153) AND isnotnull(cs_bill_customer_sk#152)) AND isnotnull(cs_item_sk#154)) +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)) (76) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#161, cd_dep_count#162] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (77) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_cdemo_sk#153] -Right keys [1]: [cd_demo_sk#161] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (78) Project [codegen id : 23] -Output [9]: [cs_bill_customer_sk#152, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162] -Input [11]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_demo_sk#161, cd_dep_count#162] +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] (79) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#163, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] +Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] (80) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#152] -Right keys [1]: [c_customer_sk#163] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#16] Join condition: None (81) Project [codegen id : 23] -Output [11]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] -Input [13]: [cs_bill_customer_sk#152, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_customer_sk#163, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] +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] (82) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#167] +Output [1]: [cd_demo_sk#131] (83) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_cdemo_sk#164] -Right keys [1]: [cd_demo_sk#167] +Left keys [1]: [c_current_cdemo_sk#17] +Right keys [1]: [cd_demo_sk#131] Join condition: None (84) Project [codegen id : 23] -Output [10]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_addr_sk#165, c_birth_year#166] -Input [12]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166, cd_demo_sk#167] +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#131] (85) Scan parquet default.customer_address -Output [3]: [ca_address_sk#168, ca_state#169, ca_country#170] +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, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (86) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] +Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] (87) Filter [codegen id : 20] -Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] -Condition : (ca_state#169 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#168)) +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)) (88) Project [codegen id : 20] -Output [2]: [ca_address_sk#168, ca_country#170] -Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] +Output [2]: [ca_address_sk#24, ca_country#27] +Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] (89) BroadcastExchange -Input [2]: [ca_address_sk#168, ca_country#170] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#171] +Input [2]: [ca_address_sk#24, ca_country#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#132] (90) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_addr_sk#165] -Right keys [1]: [ca_address_sk#168] +Left keys [1]: [c_current_addr_sk#18] +Right keys [1]: [ca_address_sk#24] Join condition: None (91) Project [codegen id : 23] -Output [10]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_birth_year#166, ca_country#170] -Input [12]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_addr_sk#165, c_birth_year#166, ca_address_sk#168, ca_country#170] +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] (92) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#172] +Output [1]: [d_date_sk#29] (93) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_sold_date_sk#160] -Right keys [1]: [d_date_sk#172] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#29] Join condition: None (94) Project [codegen id : 23] -Output [9]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cd_dep_count#162, c_birth_year#166, ca_country#170] -Input [11]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_birth_year#166, ca_country#170, d_date_sk#172] +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] (95) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#173, i_item_id#174] +Output [2]: [i_item_sk#32, i_item_id#33] (96) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#154] -Right keys [1]: [i_item_sk#173] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#32] Join condition: None (97) Project [codegen id : 23] -Output [9]: [i_item_id#174, ca_country#170, cast(cs_quantity#155 as decimal(12,2)) AS agg1#35, cast(cs_list_price#156 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#158 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#157 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#159 as decimal(12,2)) AS agg5#39, cast(c_birth_year#166 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#162 as decimal(12,2)) AS agg7#41] -Input [11]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cd_dep_count#162, c_birth_year#166, ca_country#170, i_item_sk#173, i_item_id#174] +Output [9]: [i_item_id#33, ca_country#27, 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#20 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] +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#32, i_item_id#33] (98) HashAggregate [codegen id : 23] -Input [9]: [i_item_id#174, ca_country#170, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [2]: [i_item_id#174, ca_country#170] +Input [9]: [i_item_id#33, ca_country#27, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [2]: [i_item_id#33, ca_country#27] 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#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 [16]: [i_item_id#174, ca_country#170, 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] +Aggregate Attributes [14]: [sum#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140, sum#141, count#142, sum#143, count#144, sum#145, count#146] +Results [16]: [i_item_id#33, ca_country#27, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160] (99) Exchange -Input [16]: [i_item_id#174, ca_country#170, 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#174, ca_country#170, 5), ENSURE_REQUIREMENTS, [id=#203] +Input [16]: [i_item_id#33, ca_country#27, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160] +Arguments: hashpartitioning(i_item_id#33, ca_country#27, 5), ENSURE_REQUIREMENTS, [id=#161] (100) HashAggregate [codegen id : 24] -Input [16]: [i_item_id#174, ca_country#170, 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 [2]: [i_item_id#174, ca_country#170] +Input [16]: [i_item_id#33, ca_country#27, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160] +Keys [2]: [i_item_id#33, ca_country#27] 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)#204, avg(agg2#36)#205, avg(agg3#37)#206, avg(agg4#38)#207, avg(agg5#39)#208, avg(agg6#40)#209, avg(agg7#41)#210] -Results [11]: [i_item_id#174, ca_country#170, null AS ca_state#211, null AS county#212, avg(agg1#35)#204 AS agg1#213, avg(agg2#36)#205 AS agg2#214, avg(agg3#37)#206 AS agg3#215, avg(agg4#38)#207 AS agg4#216, avg(agg5#39)#208 AS agg5#217, avg(agg6#40)#209 AS agg6#218, avg(agg7#41)#210 AS agg7#219] +Aggregate Attributes [7]: [avg(agg1#35)#162, avg(agg2#36)#163, avg(agg3#37)#164, avg(agg4#38)#165, avg(agg5#39)#166, avg(agg6#40)#167, avg(agg7#41)#168] +Results [11]: [i_item_id#33, ca_country#27, null AS ca_state#169, null AS county#170, avg(agg1#35)#162 AS agg1#171, avg(agg2#36)#163 AS agg2#172, avg(agg3#37)#164 AS agg3#173, avg(agg4#38)#165 AS agg4#174, avg(agg5#39)#166 AS agg5#175, avg(agg6#40)#167 AS agg6#176, avg(agg7#41)#168 AS agg7#177] (101) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228] +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#228), dynamicpruningexpression(cs_sold_date_sk#228 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 (102) ColumnarToRow [codegen id : 31] -Input [9]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228] +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] (103) Filter [codegen id : 31] -Input [9]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228] -Condition : ((isnotnull(cs_bill_cdemo_sk#221) AND isnotnull(cs_bill_customer_sk#220)) AND isnotnull(cs_item_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] +Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) (104) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#229, cd_dep_count#230] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (105) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_cdemo_sk#221] -Right keys [1]: [cd_demo_sk#229] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (106) Project [codegen id : 31] -Output [9]: [cs_bill_customer_sk#220, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230] -Input [11]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_demo_sk#229, cd_dep_count#230] +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] (107) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#231, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234] +Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] (108) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_customer_sk#220] -Right keys [1]: [c_customer_sk#231] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#16] Join condition: None (109) Project [codegen id : 31] -Output [11]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234] -Input [13]: [cs_bill_customer_sk#220, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_customer_sk#231, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234] +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] (110) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#235] +Output [1]: [cd_demo_sk#178] (111) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_cdemo_sk#232] -Right keys [1]: [cd_demo_sk#235] +Left keys [1]: [c_current_cdemo_sk#17] +Right keys [1]: [cd_demo_sk#178] Join condition: None (112) Project [codegen id : 31] -Output [10]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_addr_sk#233, c_birth_year#234] -Input [12]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234, cd_demo_sk#235] +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#178] (113) Scan parquet default.customer_address -Output [2]: [ca_address_sk#236, ca_state#237] +Output [2]: [ca_address_sk#24, ca_state#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (114) ColumnarToRow [codegen id : 28] -Input [2]: [ca_address_sk#236, ca_state#237] +Input [2]: [ca_address_sk#24, ca_state#26] (115) Filter [codegen id : 28] -Input [2]: [ca_address_sk#236, ca_state#237] -Condition : (ca_state#237 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#236)) +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)) (116) Project [codegen id : 28] -Output [1]: [ca_address_sk#236] -Input [2]: [ca_address_sk#236, ca_state#237] +Output [1]: [ca_address_sk#24] +Input [2]: [ca_address_sk#24, ca_state#26] (117) BroadcastExchange -Input [1]: [ca_address_sk#236] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#238] +Input [1]: [ca_address_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#179] (118) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_addr_sk#233] -Right keys [1]: [ca_address_sk#236] +Left keys [1]: [c_current_addr_sk#18] +Right keys [1]: [ca_address_sk#24] Join condition: None (119) Project [codegen id : 31] -Output [9]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_birth_year#234] -Input [11]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_addr_sk#233, c_birth_year#234, ca_address_sk#236] +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] (120) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#239] +Output [1]: [d_date_sk#29] (121) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_sold_date_sk#228] -Right keys [1]: [d_date_sk#239] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#29] Join condition: None (122) Project [codegen id : 31] -Output [8]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cd_dep_count#230, c_birth_year#234] -Input [10]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_birth_year#234, d_date_sk#239] +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] (123) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#240, i_item_id#241] +Output [2]: [i_item_sk#32, i_item_id#33] (124) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_item_sk#222] -Right keys [1]: [i_item_sk#240] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#32] Join condition: None (125) Project [codegen id : 31] -Output [8]: [i_item_id#241, cast(cs_quantity#223 as decimal(12,2)) AS agg1#35, cast(cs_list_price#224 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#226 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#225 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#227 as decimal(12,2)) AS agg5#39, cast(c_birth_year#234 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#230 as decimal(12,2)) AS agg7#41] -Input [10]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cd_dep_count#230, c_birth_year#234, i_item_sk#240, i_item_id#241] +Output [8]: [i_item_id#33, 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#20 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#20, i_item_sk#32, i_item_id#33] (126) HashAggregate [codegen id : 31] -Input [8]: [i_item_id#241, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [1]: [i_item_id#241] +Input [8]: [i_item_id#33, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [1]: [i_item_id#33] 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#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] -Results [15]: [i_item_id#241, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] +Aggregate Attributes [14]: [sum#180, count#181, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193] +Results [15]: [i_item_id#33, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] (127) Exchange -Input [15]: [i_item_id#241, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] -Arguments: hashpartitioning(i_item_id#241, 5), ENSURE_REQUIREMENTS, [id=#270] +Input [15]: [i_item_id#33, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] +Arguments: hashpartitioning(i_item_id#33, 5), ENSURE_REQUIREMENTS, [id=#208] (128) HashAggregate [codegen id : 32] -Input [15]: [i_item_id#241, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] -Keys [1]: [i_item_id#241] +Input [15]: [i_item_id#33, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] +Keys [1]: [i_item_id#33] 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)#271, avg(agg2#36)#272, avg(agg3#37)#273, avg(agg4#38)#274, avg(agg5#39)#275, avg(agg6#40)#276, avg(agg7#41)#277] -Results [11]: [i_item_id#241, null AS ca_country#278, null AS ca_state#279, null AS county#280, avg(agg1#35)#271 AS agg1#281, avg(agg2#36)#272 AS agg2#282, avg(agg3#37)#273 AS agg3#283, avg(agg4#38)#274 AS agg4#284, avg(agg5#39)#275 AS agg5#285, avg(agg6#40)#276 AS agg6#286, avg(agg7#41)#277 AS agg7#287] +Aggregate Attributes [7]: [avg(agg1#35)#209, avg(agg2#36)#210, avg(agg3#37)#211, avg(agg4#38)#212, avg(agg5#39)#213, avg(agg6#40)#214, avg(agg7#41)#215] +Results [11]: [i_item_id#33, null AS ca_country#216, null AS ca_state#217, null AS county#218, avg(agg1#35)#209 AS agg1#219, avg(agg2#36)#210 AS agg2#220, avg(agg3#37)#211 AS agg3#221, avg(agg4#38)#212 AS agg4#222, avg(agg5#39)#213 AS agg5#223, avg(agg6#40)#214 AS agg6#224, avg(agg7#41)#215 AS agg7#225] (129) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296] +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#296), dynamicpruningexpression(cs_sold_date_sk#296 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 (130) ColumnarToRow [codegen id : 39] -Input [9]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296] +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] (131) Filter [codegen id : 39] -Input [9]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296] -Condition : ((isnotnull(cs_bill_cdemo_sk#289) AND isnotnull(cs_bill_customer_sk#288)) AND isnotnull(cs_item_sk#290)) +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)) (132) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#297, cd_dep_count#298] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (133) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_cdemo_sk#289] -Right keys [1]: [cd_demo_sk#297] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (134) Project [codegen id : 39] -Output [9]: [cs_bill_customer_sk#288, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298] -Input [11]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_demo_sk#297, cd_dep_count#298] +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] (135) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#299, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302] +Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] (136) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_customer_sk#288] -Right keys [1]: [c_customer_sk#299] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#16] Join condition: None (137) Project [codegen id : 39] -Output [11]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302] -Input [13]: [cs_bill_customer_sk#288, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_customer_sk#299, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302] +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] (138) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#303] +Output [1]: [cd_demo_sk#226] (139) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#300] -Right keys [1]: [cd_demo_sk#303] +Left keys [1]: [c_current_cdemo_sk#17] +Right keys [1]: [cd_demo_sk#226] Join condition: None (140) Project [codegen id : 39] -Output [10]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_addr_sk#301, c_birth_year#302] -Input [12]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302, cd_demo_sk#303] +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#226] (141) ReusedExchange [Reuses operator id: 117] -Output [1]: [ca_address_sk#304] +Output [1]: [ca_address_sk#24] (142) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_addr_sk#301] -Right keys [1]: [ca_address_sk#304] +Left keys [1]: [c_current_addr_sk#18] +Right keys [1]: [ca_address_sk#24] Join condition: None (143) Project [codegen id : 39] -Output [9]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_birth_year#302] -Input [11]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_addr_sk#301, c_birth_year#302, ca_address_sk#304] +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] (144) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#305] +Output [1]: [d_date_sk#29] (145) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_sold_date_sk#296] -Right keys [1]: [d_date_sk#305] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#29] Join condition: None (146) Project [codegen id : 39] -Output [8]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cd_dep_count#298, c_birth_year#302] -Input [10]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_birth_year#302, d_date_sk#305] +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] (147) Scan parquet default.item -Output [1]: [i_item_sk#306] +Output [1]: [i_item_sk#32] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (148) ColumnarToRow [codegen id : 38] -Input [1]: [i_item_sk#306] +Input [1]: [i_item_sk#32] (149) Filter [codegen id : 38] -Input [1]: [i_item_sk#306] -Condition : isnotnull(i_item_sk#306) +Input [1]: [i_item_sk#32] +Condition : isnotnull(i_item_sk#32) (150) BroadcastExchange -Input [1]: [i_item_sk#306] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#307] +Input [1]: [i_item_sk#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#227] (151) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_item_sk#290] -Right keys [1]: [i_item_sk#306] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#32] Join condition: None (152) Project [codegen id : 39] -Output [7]: [cast(cs_quantity#291 as decimal(12,2)) AS agg1#35, cast(cs_list_price#292 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#294 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#293 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#295 as decimal(12,2)) AS agg5#39, cast(c_birth_year#302 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#298 as decimal(12,2)) AS agg7#41] -Input [9]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cd_dep_count#298, c_birth_year#302, i_item_sk#306] +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#20 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] +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#32] (153) HashAggregate [codegen id : 39] 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#308, count#309, sum#310, count#311, sum#312, count#313, sum#314, count#315, sum#316, count#317, sum#318, count#319, sum#320, count#321] -Results [14]: [sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335] +Aggregate Attributes [14]: [sum#228, count#229, sum#230, count#231, sum#232, count#233, sum#234, count#235, sum#236, count#237, sum#238, count#239, sum#240, count#241] +Results [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] (154) Exchange -Input [14]: [sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#336] +Input [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#256] (155) HashAggregate [codegen id : 40] -Input [14]: [sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335] +Input [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] 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)#337, avg(agg2#36)#338, avg(agg3#37)#339, avg(agg4#38)#340, avg(agg5#39)#341, avg(agg6#40)#342, avg(agg7#41)#343] -Results [11]: [null AS i_item_id#344, null AS ca_country#345, null AS ca_state#346, null AS county#347, avg(agg1#35)#337 AS agg1#348, avg(agg2#36)#338 AS agg2#349, avg(agg3#37)#339 AS agg3#350, avg(agg4#38)#340 AS agg4#351, avg(agg5#39)#341 AS agg5#352, avg(agg6#40)#342 AS agg6#353, avg(agg7#41)#343 AS agg7#354] +Aggregate Attributes [7]: [avg(agg1#35)#257, avg(agg2#36)#258, avg(agg3#37)#259, avg(agg4#38)#260, avg(agg5#39)#261, avg(agg6#40)#262, avg(agg7#41)#263] +Results [11]: [null AS i_item_id#264, null AS ca_country#265, null AS ca_state#266, null AS county#267, avg(agg1#35)#257 AS agg1#268, avg(agg2#36)#258 AS agg2#269, avg(agg3#37)#259 AS agg3#270, avg(agg4#38)#260 AS agg4#271, avg(agg5#39)#261 AS agg5#272, avg(agg6#40)#262 AS agg6#273, avg(agg7#41)#263 AS agg7#274] (156) Union @@ -868,12 +868,12 @@ ReusedExchange (158) (158) ReusedExchange [Reuses operator id: 34] Output [1]: [d_date_sk#29] -Subquery:2 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#93 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#160 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 101 Hosting Expression = cs_sold_date_sk#228 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 101 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 129 Hosting Expression = cs_sold_date_sk#296 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 129 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/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt index a74ad3e4b8b77..8bc09f9457fe7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt @@ -55,7 +55,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -86,7 +86,7 @@ Arguments: [cs_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -94,7 +94,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt index c6a345be29c9d..49b35aacfcf2f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt @@ -44,7 +44,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -52,7 +52,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -79,7 +79,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] 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 f5a7a9135cf29..014b89cecd475 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 @@ -198,116 +198,116 @@ Aggregate Attributes [1]: [avg(qoh#24)#29] Results [5]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, avg(qoh#24)#29 AS qoh#30] (31) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#31, i_brand#32, i_class#33, i_category#34, sum#35, count#36] +Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#31, count#32] (32) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#31, i_brand#32, i_class#33, i_category#34, sum#35, count#36] -Keys [4]: [i_product_name#31, i_brand#32, i_class#33, i_category#34] -Functions [1]: [avg(inv_quantity_on_hand#37)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#37)#38] -Results [4]: [i_product_name#31, i_brand#32, i_class#33, avg(inv_quantity_on_hand#37)#38 AS qoh#24] +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#31, count#32] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#33] +Results [4]: [i_product_name#16, i_brand#13, i_class#14, avg(inv_quantity_on_hand#3)#33 AS qoh#24] (33) HashAggregate [codegen id : 16] -Input [4]: [i_product_name#31, i_brand#32, i_class#33, qoh#24] -Keys [3]: [i_product_name#31, i_brand#32, i_class#33] +Input [4]: [i_product_name#16, i_brand#13, i_class#14, qoh#24] +Keys [3]: [i_product_name#16, i_brand#13, i_class#14] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#39, count#40] -Results [5]: [i_product_name#31, i_brand#32, i_class#33, sum#41, count#42] +Aggregate Attributes [2]: [sum#34, count#35] +Results [5]: [i_product_name#16, i_brand#13, i_class#14, sum#36, count#37] (34) Exchange -Input [5]: [i_product_name#31, i_brand#32, i_class#33, sum#41, count#42] -Arguments: hashpartitioning(i_product_name#31, i_brand#32, i_class#33, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#36, count#37] +Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, 5), ENSURE_REQUIREMENTS, [id=#38] (35) HashAggregate [codegen id : 17] -Input [5]: [i_product_name#31, i_brand#32, i_class#33, sum#41, count#42] -Keys [3]: [i_product_name#31, i_brand#32, i_class#33] +Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#36, count#37] +Keys [3]: [i_product_name#16, i_brand#13, i_class#14] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#44] -Results [5]: [i_product_name#31, i_brand#32, i_class#33, null AS i_category#45, avg(qoh#24)#44 AS qoh#46] +Aggregate Attributes [1]: [avg(qoh#24)#39] +Results [5]: [i_product_name#16, i_brand#13, i_class#14, null AS i_category#40, avg(qoh#24)#39 AS qoh#41] (36) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#47, i_brand#48, i_class#49, i_category#50, sum#51, count#52] +Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#42, count#43] (37) HashAggregate [codegen id : 25] -Input [6]: [i_product_name#47, i_brand#48, i_class#49, i_category#50, sum#51, count#52] -Keys [4]: [i_product_name#47, i_brand#48, i_class#49, i_category#50] -Functions [1]: [avg(inv_quantity_on_hand#53)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#53)#54] -Results [3]: [i_product_name#47, i_brand#48, avg(inv_quantity_on_hand#53)#54 AS qoh#24] +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#42, count#43] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#44] +Results [3]: [i_product_name#16, i_brand#13, avg(inv_quantity_on_hand#3)#44 AS qoh#24] (38) HashAggregate [codegen id : 25] -Input [3]: [i_product_name#47, i_brand#48, qoh#24] -Keys [2]: [i_product_name#47, i_brand#48] +Input [3]: [i_product_name#16, i_brand#13, qoh#24] +Keys [2]: [i_product_name#16, i_brand#13] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#55, count#56] -Results [4]: [i_product_name#47, i_brand#48, sum#57, count#58] +Aggregate Attributes [2]: [sum#45, count#46] +Results [4]: [i_product_name#16, i_brand#13, sum#47, count#48] (39) Exchange -Input [4]: [i_product_name#47, i_brand#48, sum#57, count#58] -Arguments: hashpartitioning(i_product_name#47, i_brand#48, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [4]: [i_product_name#16, i_brand#13, sum#47, count#48] +Arguments: hashpartitioning(i_product_name#16, i_brand#13, 5), ENSURE_REQUIREMENTS, [id=#49] (40) HashAggregate [codegen id : 26] -Input [4]: [i_product_name#47, i_brand#48, sum#57, count#58] -Keys [2]: [i_product_name#47, i_brand#48] +Input [4]: [i_product_name#16, i_brand#13, sum#47, count#48] +Keys [2]: [i_product_name#16, i_brand#13] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#60] -Results [5]: [i_product_name#47, i_brand#48, null AS i_class#61, null AS i_category#62, avg(qoh#24)#60 AS qoh#63] +Aggregate Attributes [1]: [avg(qoh#24)#50] +Results [5]: [i_product_name#16, i_brand#13, null AS i_class#51, null AS i_category#52, avg(qoh#24)#50 AS qoh#53] (41) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#64, i_brand#65, i_class#66, i_category#67, sum#68, count#69] +Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#54, count#55] (42) HashAggregate [codegen id : 34] -Input [6]: [i_product_name#64, i_brand#65, i_class#66, i_category#67, sum#68, count#69] -Keys [4]: [i_product_name#64, i_brand#65, i_class#66, i_category#67] -Functions [1]: [avg(inv_quantity_on_hand#70)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#70)#71] -Results [2]: [i_product_name#64, avg(inv_quantity_on_hand#70)#71 AS qoh#24] +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#54, count#55] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#56] +Results [2]: [i_product_name#16, avg(inv_quantity_on_hand#3)#56 AS qoh#24] (43) HashAggregate [codegen id : 34] -Input [2]: [i_product_name#64, qoh#24] -Keys [1]: [i_product_name#64] +Input [2]: [i_product_name#16, qoh#24] +Keys [1]: [i_product_name#16] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#72, count#73] -Results [3]: [i_product_name#64, sum#74, count#75] +Aggregate Attributes [2]: [sum#57, count#58] +Results [3]: [i_product_name#16, sum#59, count#60] (44) Exchange -Input [3]: [i_product_name#64, sum#74, count#75] -Arguments: hashpartitioning(i_product_name#64, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [3]: [i_product_name#16, sum#59, count#60] +Arguments: hashpartitioning(i_product_name#16, 5), ENSURE_REQUIREMENTS, [id=#61] (45) HashAggregate [codegen id : 35] -Input [3]: [i_product_name#64, sum#74, count#75] -Keys [1]: [i_product_name#64] +Input [3]: [i_product_name#16, sum#59, count#60] +Keys [1]: [i_product_name#16] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#77] -Results [5]: [i_product_name#64, null AS i_brand#78, null AS i_class#79, null AS i_category#80, avg(qoh#24)#77 AS qoh#81] +Aggregate Attributes [1]: [avg(qoh#24)#62] +Results [5]: [i_product_name#16, null AS i_brand#63, null AS i_class#64, null AS i_category#65, avg(qoh#24)#62 AS qoh#66] (46) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#82, i_brand#83, i_class#84, i_category#85, sum#86, count#87] +Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#67, count#68] (47) HashAggregate [codegen id : 43] -Input [6]: [i_product_name#82, i_brand#83, i_class#84, i_category#85, sum#86, count#87] -Keys [4]: [i_product_name#82, i_brand#83, i_class#84, i_category#85] -Functions [1]: [avg(inv_quantity_on_hand#88)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#88)#89] -Results [1]: [avg(inv_quantity_on_hand#88)#89 AS qoh#24] +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#67, count#68] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#69] +Results [1]: [avg(inv_quantity_on_hand#3)#69 AS qoh#24] (48) HashAggregate [codegen id : 43] Input [1]: [qoh#24] Keys: [] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#90, count#91] -Results [2]: [sum#92, count#93] +Aggregate Attributes [2]: [sum#70, count#71] +Results [2]: [sum#72, count#73] (49) Exchange -Input [2]: [sum#92, count#93] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] +Input [2]: [sum#72, count#73] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#74] (50) HashAggregate [codegen id : 44] -Input [2]: [sum#92, count#93] +Input [2]: [sum#72, count#73] Keys: [] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#95] -Results [5]: [null AS i_product_name#96, null AS i_brand#97, null AS i_class#98, null AS i_category#99, avg(qoh#24)#95 AS qoh#100] +Aggregate Attributes [1]: [avg(qoh#24)#75] +Results [5]: [null AS i_product_name#76, null AS i_brand#77, null AS i_class#78, null AS i_category#79, avg(qoh#24)#75 AS qoh#80] (51) Union 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 527d2eb5e86a4..74dd076851c1d 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 @@ -183,116 +183,116 @@ Aggregate Attributes [1]: [avg(qoh#23)#28] Results [5]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, avg(qoh#23)#28 AS qoh#29] (28) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#30, i_brand#31, i_class#32, i_category#33, sum#34, count#35] +Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#30, count#31] (29) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#30, i_brand#31, i_class#32, i_category#33, sum#34, count#35] -Keys [4]: [i_product_name#30, i_brand#31, i_class#32, i_category#33] -Functions [1]: [avg(inv_quantity_on_hand#36)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#36)#37] -Results [4]: [i_product_name#30, i_brand#31, i_class#32, avg(inv_quantity_on_hand#36)#37 AS qoh#23] +Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#30, count#31] +Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#32] +Results [4]: [i_product_name#13, i_brand#10, i_class#11, avg(inv_quantity_on_hand#3)#32 AS qoh#23] (30) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#30, i_brand#31, i_class#32, qoh#23] -Keys [3]: [i_product_name#30, i_brand#31, i_class#32] +Input [4]: [i_product_name#13, i_brand#10, i_class#11, qoh#23] +Keys [3]: [i_product_name#13, i_brand#10, i_class#11] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#38, count#39] -Results [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] +Aggregate Attributes [2]: [sum#33, count#34] +Results [5]: [i_product_name#13, i_brand#10, i_class#11, sum#35, count#36] (31) Exchange -Input [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] -Arguments: hashpartitioning(i_product_name#30, i_brand#31, i_class#32, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [5]: [i_product_name#13, i_brand#10, i_class#11, sum#35, count#36] +Arguments: hashpartitioning(i_product_name#13, i_brand#10, i_class#11, 5), ENSURE_REQUIREMENTS, [id=#37] (32) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] -Keys [3]: [i_product_name#30, i_brand#31, i_class#32] +Input [5]: [i_product_name#13, i_brand#10, i_class#11, sum#35, count#36] +Keys [3]: [i_product_name#13, i_brand#10, i_class#11] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#43] -Results [5]: [i_product_name#30, i_brand#31, i_class#32, null AS i_category#44, avg(qoh#23)#43 AS qoh#45] +Aggregate Attributes [1]: [avg(qoh#23)#38] +Results [5]: [i_product_name#13, i_brand#10, i_class#11, null AS i_category#39, avg(qoh#23)#38 AS qoh#40] (33) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#46, i_brand#47, i_class#48, i_category#49, sum#50, count#51] +Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#41, count#42] (34) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#46, i_brand#47, i_class#48, i_category#49, sum#50, count#51] -Keys [4]: [i_product_name#46, i_brand#47, i_class#48, i_category#49] -Functions [1]: [avg(inv_quantity_on_hand#52)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#52)#53] -Results [3]: [i_product_name#46, i_brand#47, avg(inv_quantity_on_hand#52)#53 AS qoh#23] +Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#41, count#42] +Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#43] +Results [3]: [i_product_name#13, i_brand#10, avg(inv_quantity_on_hand#3)#43 AS qoh#23] (35) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#46, i_brand#47, qoh#23] -Keys [2]: [i_product_name#46, i_brand#47] +Input [3]: [i_product_name#13, i_brand#10, qoh#23] +Keys [2]: [i_product_name#13, i_brand#10] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#54, count#55] -Results [4]: [i_product_name#46, i_brand#47, sum#56, count#57] +Aggregate Attributes [2]: [sum#44, count#45] +Results [4]: [i_product_name#13, i_brand#10, sum#46, count#47] (36) Exchange -Input [4]: [i_product_name#46, i_brand#47, sum#56, count#57] -Arguments: hashpartitioning(i_product_name#46, i_brand#47, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [4]: [i_product_name#13, i_brand#10, sum#46, count#47] +Arguments: hashpartitioning(i_product_name#13, i_brand#10, 5), ENSURE_REQUIREMENTS, [id=#48] (37) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#46, i_brand#47, sum#56, count#57] -Keys [2]: [i_product_name#46, i_brand#47] +Input [4]: [i_product_name#13, i_brand#10, sum#46, count#47] +Keys [2]: [i_product_name#13, i_brand#10] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#59] -Results [5]: [i_product_name#46, i_brand#47, null AS i_class#60, null AS i_category#61, avg(qoh#23)#59 AS qoh#62] +Aggregate Attributes [1]: [avg(qoh#23)#49] +Results [5]: [i_product_name#13, i_brand#10, null AS i_class#50, null AS i_category#51, avg(qoh#23)#49 AS qoh#52] (38) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, sum#67, count#68] +Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#53, count#54] (39) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, sum#67, count#68] -Keys [4]: [i_product_name#63, i_brand#64, i_class#65, i_category#66] -Functions [1]: [avg(inv_quantity_on_hand#69)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#69)#70] -Results [2]: [i_product_name#63, avg(inv_quantity_on_hand#69)#70 AS qoh#23] +Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#53, count#54] +Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#55] +Results [2]: [i_product_name#13, avg(inv_quantity_on_hand#3)#55 AS qoh#23] (40) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#63, qoh#23] -Keys [1]: [i_product_name#63] +Input [2]: [i_product_name#13, qoh#23] +Keys [1]: [i_product_name#13] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#71, count#72] -Results [3]: [i_product_name#63, sum#73, count#74] +Aggregate Attributes [2]: [sum#56, count#57] +Results [3]: [i_product_name#13, sum#58, count#59] (41) Exchange -Input [3]: [i_product_name#63, sum#73, count#74] -Arguments: hashpartitioning(i_product_name#63, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [3]: [i_product_name#13, sum#58, count#59] +Arguments: hashpartitioning(i_product_name#13, 5), ENSURE_REQUIREMENTS, [id=#60] (42) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#63, sum#73, count#74] -Keys [1]: [i_product_name#63] +Input [3]: [i_product_name#13, sum#58, count#59] +Keys [1]: [i_product_name#13] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#76] -Results [5]: [i_product_name#63, null AS i_brand#77, null AS i_class#78, null AS i_category#79, avg(qoh#23)#76 AS qoh#80] +Aggregate Attributes [1]: [avg(qoh#23)#61] +Results [5]: [i_product_name#13, null AS i_brand#62, null AS i_class#63, null AS i_category#64, avg(qoh#23)#61 AS qoh#65] (43) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#81, i_brand#82, i_class#83, i_category#84, sum#85, count#86] +Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#66, count#67] (44) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#81, i_brand#82, i_class#83, i_category#84, sum#85, count#86] -Keys [4]: [i_product_name#81, i_brand#82, i_class#83, i_category#84] -Functions [1]: [avg(inv_quantity_on_hand#87)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#87)#88] -Results [1]: [avg(inv_quantity_on_hand#87)#88 AS qoh#23] +Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#66, count#67] +Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#68] +Results [1]: [avg(inv_quantity_on_hand#3)#68 AS qoh#23] (45) HashAggregate [codegen id : 28] Input [1]: [qoh#23] Keys: [] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#89, count#90] -Results [2]: [sum#91, count#92] +Aggregate Attributes [2]: [sum#69, count#70] +Results [2]: [sum#71, count#72] (46) Exchange -Input [2]: [sum#91, count#92] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] +Input [2]: [sum#71, count#72] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#73] (47) HashAggregate [codegen id : 29] -Input [2]: [sum#91, count#92] +Input [2]: [sum#71, count#72] Keys: [] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#94] -Results [5]: [null AS i_product_name#95, null AS i_brand#96, null AS i_class#97, null AS i_category#98, avg(qoh#23)#94 AS qoh#99] +Aggregate Attributes [1]: [avg(qoh#23)#74] +Results [5]: [null AS i_product_name#75, null AS i_brand#76, null AS i_class#77, null AS i_category#78, avg(qoh#23)#74 AS qoh#79] (48) Union 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 1c0ed5c8c188b..d4d0015a9fec0 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 @@ -157,7 +157,7 @@ Input [12]: [s_store_sk#1, s_store_name#2, s_state#4, ca_state#8, c_customer_sk# 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_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 4] @@ -165,7 +165,7 @@ Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, (26) Filter [codegen id : 4] 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_color#27) AND (i_color#27 = pale )) AND isnotnull(i_item_sk#24)) +Condition : ((isnotnull(i_color#27) AND (i_color#27 = pale)) AND isnotnull(i_item_sk#24)) (27) BroadcastExchange Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] @@ -329,224 +329,224 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (50) Scan parquet default.store -Output [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] +Output [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (51) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] +Input [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] (52) Filter [codegen id : 1] -Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] -Condition : (((isnotnull(s_market_id#54) AND (s_market_id#54 = 8)) AND isnotnull(s_store_sk#52)) AND isnotnull(s_zip#56)) +Input [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] +Condition : (((isnotnull(s_market_id#3) AND (s_market_id#3 = 8)) AND isnotnull(s_store_sk#1)) AND isnotnull(s_zip#5)) (53) Project [codegen id : 1] -Output [4]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56] -Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] +Output [4]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5] +Input [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] (54) BroadcastExchange -Input [4]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56] -Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#57] +Input [4]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5] +Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#52] (55) Scan parquet default.customer_address -Output [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] +Output [4]: [ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (56) ColumnarToRow -Input [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] +Input [4]: [ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] (57) Filter -Input [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] -Condition : ((isnotnull(ca_address_sk#58) AND isnotnull(ca_country#61)) AND isnotnull(ca_zip#60)) +Input [4]: [ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] +Condition : ((isnotnull(ca_address_sk#7) AND isnotnull(ca_country#10)) AND isnotnull(ca_zip#9)) (58) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [s_zip#56] -Right keys [1]: [ca_zip#60] +Left keys [1]: [s_zip#5] +Right keys [1]: [ca_zip#9] Join condition: None (59) Project [codegen id : 2] -Output [6]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61] -Input [8]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56, ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] +Output [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10] +Input [8]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5, ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] (60) BroadcastExchange -Input [6]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61] -Arguments: HashedRelationBroadcastMode(List(input[3, int, true], upper(input[5, string, true])),false), [id=#62] +Input [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10] +Arguments: HashedRelationBroadcastMode(List(input[3, int, true], upper(input[5, string, true])),false), [id=#53] (61) Scan parquet default.customer -Output [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] +Output [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct (62) ColumnarToRow -Input [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] +Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] (63) Filter -Input [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] -Condition : ((isnotnull(c_customer_sk#63) AND isnotnull(c_current_addr_sk#64)) AND isnotnull(c_birth_country#67)) +Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] +Condition : ((isnotnull(c_customer_sk#12) AND isnotnull(c_current_addr_sk#13)) AND isnotnull(c_birth_country#16)) (64) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ca_address_sk#58, upper(ca_country#61)] -Right keys [2]: [c_current_addr_sk#64, c_birth_country#67] +Left keys [2]: [ca_address_sk#7, upper(ca_country#10)] +Right keys [2]: [c_current_addr_sk#13, c_birth_country#16] Join condition: None (65) Project [codegen id : 3] -Output [7]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66] -Input [11]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61, c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] +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] +Input [11]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10, c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] (66) BroadcastExchange -Input [7]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#68] +Input [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] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#54] (67) Scan parquet default.store_sales -Output [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] +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 (68) ColumnarToRow -Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] +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] (69) Filter -Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] -Condition : (((isnotnull(ss_ticket_number#72) AND isnotnull(ss_item_sk#69)) AND isnotnull(ss_store_sk#71)) AND isnotnull(ss_customer_sk#70)) +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)) (70) Project -Output [5]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73] -Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] +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] (71) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [s_store_sk#52, c_customer_sk#63] -Right keys [2]: [ss_store_sk#71, ss_customer_sk#70] +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 (72) Project [codegen id : 4] -Output [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] -Input [12]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73] +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] (73) Exchange -Input [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] -Arguments: hashpartitioning(ss_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] +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=#55] (74) Sort [codegen id : 5] -Input [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] -Arguments: [ss_item_sk#69 ASC NULLS FIRST], false, 0 +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 (75) Scan parquet default.item -Output [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +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 (76) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] (77) Filter [codegen id : 6] -Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Condition : isnotnull(i_item_sk#76) +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) (78) Exchange -Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Arguments: hashpartitioning(i_item_sk#76, 5), ENSURE_REQUIREMENTS, [id=#82] +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=#56] (79) Sort [codegen id : 7] -Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Arguments: [i_item_sk#76 ASC NULLS FIRST], false, 0 +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 (80) SortMergeJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#69] -Right keys [1]: [i_item_sk#76] +Left keys [1]: [ss_item_sk#18] +Right keys [1]: [i_item_sk#24] Join condition: None (81) Project [codegen id : 8] -Output [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Input [14]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +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] (82) Exchange -Input [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Arguments: hashpartitioning(cast(ss_ticket_number#72 as bigint), cast(ss_item_sk#69 as bigint), 5), ENSURE_REQUIREMENTS, [id=#83] +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(cast(ss_ticket_number#21 as bigint), cast(ss_item_sk#18 as bigint), 5), ENSURE_REQUIREMENTS, [id=#57] (83) Sort [codegen id : 9] -Input [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Arguments: [cast(ss_ticket_number#72 as bigint) ASC NULLS FIRST, cast(ss_item_sk#69 as bigint) 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: [cast(ss_ticket_number#21 as bigint) ASC NULLS FIRST, cast(ss_item_sk#18 as bigint) ASC NULLS FIRST], false, 0 (84) Scan parquet default.store_returns -Output [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] +Output [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (85) ColumnarToRow [codegen id : 10] -Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] +Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] (86) Filter [codegen id : 10] -Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] -Condition : (isnotnull(sr_ticket_number#85) AND isnotnull(sr_item_sk#84)) +Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) (87) Project [codegen id : 10] -Output [2]: [sr_item_sk#84, sr_ticket_number#85] -Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] +Output [2]: [sr_item_sk#32, sr_ticket_number#33] +Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] (88) Exchange -Input [2]: [sr_item_sk#84, sr_ticket_number#85] -Arguments: hashpartitioning(sr_ticket_number#85, sr_item_sk#84, 5), ENSURE_REQUIREMENTS, [id=#87] +Input [2]: [sr_item_sk#32, sr_ticket_number#33] +Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#58] (89) Sort [codegen id : 11] -Input [2]: [sr_item_sk#84, sr_ticket_number#85] -Arguments: [sr_ticket_number#85 ASC NULLS FIRST, sr_item_sk#84 ASC NULLS FIRST], false, 0 +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 (90) SortMergeJoin [codegen id : 12] -Left keys [2]: [cast(ss_ticket_number#72 as bigint), cast(ss_item_sk#69 as bigint)] -Right keys [2]: [sr_ticket_number#85, sr_item_sk#84] +Left keys [2]: [cast(ss_ticket_number#21 as bigint), cast(ss_item_sk#18 as bigint)] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (91) Project [codegen id : 12] -Output [11]: [ss_net_paid#73, s_store_name#53, s_state#55, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, c_first_name#65, c_last_name#66, ca_state#59] -Input [15]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, sr_item_sk#84, sr_ticket_number#85] +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] (92) HashAggregate [codegen id : 12] -Input [11]: [ss_net_paid#73, s_store_name#53, s_state#55, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, c_first_name#65, c_last_name#66, ca_state#59] -Keys [10]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#73))] -Aggregate Attributes [1]: [sum#88] -Results [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] +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#59] +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#60] (93) Exchange -Input [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] -Arguments: hashpartitioning(c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, 5), ENSURE_REQUIREMENTS, [id=#90] +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#60] +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=#61] (94) HashAggregate [codegen id : 13] -Input [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] -Keys [10]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78] -Functions [1]: [sum(UnscaledValue(ss_net_paid#73))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#73))#91] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#73))#91,17,2) AS netpaid#40] +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#60] +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))#62] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#62,17,2) AS netpaid#40] (95) HashAggregate [codegen id : 13] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#92, count#93] -Results [2]: [sum#94, count#95] +Aggregate Attributes [2]: [sum#63, count#64] +Results [2]: [sum#65, count#66] (96) Exchange -Input [2]: [sum#94, count#95] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#96] +Input [2]: [sum#65, count#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] (97) HashAggregate [codegen id : 14] -Input [2]: [sum#94, count#95] +Input [2]: [sum#65, count#66] Keys: [] Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#97] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#97)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#98] +Aggregate Attributes [1]: [avg(netpaid#40)#68] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#68)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#69] 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 7259a3d829613..6835ff84f5bc7 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 @@ -146,7 +146,7 @@ Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_stor 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_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] @@ -154,7 +154,7 @@ Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, (24) Filter [codegen id : 6] 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_color#21) AND (i_color#21 = pale )) AND isnotnull(i_item_sk#18)) +Condition : ((isnotnull(i_color#21) AND (i_color#21 = pale)) AND isnotnull(i_item_sk#18)) (25) BroadcastExchange Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] @@ -326,212 +326,212 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (50) Scan parquet default.store_sales -Output [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] +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 : 1] -Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] +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 : 1] -Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] -Condition : (((isnotnull(ss_ticket_number#55) AND isnotnull(ss_item_sk#52)) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_customer_sk#53)) +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 : 1] -Output [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] -Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] +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) Exchange -Input [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] -Arguments: hashpartitioning(cast(ss_ticket_number#55 as bigint), cast(ss_item_sk#52 as bigint), 5), ENSURE_REQUIREMENTS, [id=#58] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#52] (55) Sort [codegen id : 2] -Input [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] -Arguments: [cast(ss_ticket_number#55 as bigint) ASC NULLS FIRST, cast(ss_item_sk#52 as bigint) 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: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (56) Scan parquet default.store_returns -Output [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] +Output [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] (58) Filter [codegen id : 3] -Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] -Condition : (isnotnull(sr_ticket_number#60) AND isnotnull(sr_item_sk#59)) +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Condition : (isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#8)) (59) Project [codegen id : 3] -Output [2]: [sr_item_sk#59, sr_ticket_number#60] -Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] +Output [2]: [sr_item_sk#8, sr_ticket_number#9] +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] (60) Exchange -Input [2]: [sr_item_sk#59, sr_ticket_number#60] -Arguments: hashpartitioning(sr_ticket_number#60, sr_item_sk#59, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [2]: [sr_item_sk#8, sr_ticket_number#9] +Arguments: hashpartitioning(sr_ticket_number#9, sr_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#53] (61) Sort [codegen id : 4] -Input [2]: [sr_item_sk#59, sr_ticket_number#60] -Arguments: [sr_ticket_number#60 ASC NULLS FIRST, sr_item_sk#59 ASC NULLS FIRST], false, 0 +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 (62) SortMergeJoin [codegen id : 9] -Left keys [2]: [cast(ss_ticket_number#55 as bigint), cast(ss_item_sk#52 as bigint)] -Right keys [2]: [sr_ticket_number#60, sr_item_sk#59] +Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] +Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] Join condition: None (63) Project [codegen id : 9] -Output [4]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_net_paid#56] -Input [7]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, sr_item_sk#59, sr_ticket_number#60] +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] (64) Scan parquet default.store -Output [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] +Output [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (65) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] (66) Filter [codegen id : 5] -Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] -Condition : (((isnotnull(s_market_id#65) AND (s_market_id#65 = 8)) AND isnotnull(s_store_sk#63)) AND isnotnull(s_zip#67)) +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Condition : (((isnotnull(s_market_id#14) AND (s_market_id#14 = 8)) AND isnotnull(s_store_sk#12)) AND isnotnull(s_zip#16)) (67) Project [codegen id : 5] -Output [4]: [s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] -Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] +Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] (68) BroadcastExchange -Input [4]: [s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#68] +Input [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] (69) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#54] -Right keys [1]: [s_store_sk#63] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#12] Join condition: None (70) Project [codegen id : 9] -Output [6]: [ss_item_sk#52, ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67] -Input [8]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_net_paid#56, s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] +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] (71) Scan parquet default.item -Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +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 (72) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] (73) Filter [codegen id : 6] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Condition : isnotnull(i_item_sk#69) +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) (74) BroadcastExchange -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] +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=#55] (75) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#52] -Right keys [1]: [i_item_sk#69] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#18] Join condition: None (76) Project [codegen id : 9] -Output [10]: [ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Input [12]: [ss_item_sk#52, ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +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] (77) Scan parquet default.customer -Output [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +Output [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct (78) ColumnarToRow [codegen id : 7] -Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +Input [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] (79) Filter [codegen id : 7] -Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] -Condition : ((isnotnull(c_customer_sk#76) AND isnotnull(c_current_addr_sk#77)) AND isnotnull(c_birth_country#80)) +Input [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] +Condition : ((isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#26)) AND isnotnull(c_birth_country#29)) (80) BroadcastExchange -Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] +Input [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#56] (81) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#53] -Right keys [1]: [c_customer_sk#76] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#25] Join condition: None (82) Project [codegen id : 9] -Output [13]: [ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] -Input [15]: [ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +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] (83) Scan parquet default.customer_address -Output [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] +Output [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (84) ColumnarToRow [codegen id : 8] -Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] +Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] (85) Filter [codegen id : 8] -Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] -Condition : ((isnotnull(ca_address_sk#82) AND isnotnull(ca_country#85)) AND isnotnull(ca_zip#84)) +Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] +Condition : ((isnotnull(ca_address_sk#31) AND isnotnull(ca_country#34)) AND isnotnull(ca_zip#33)) (86) BroadcastExchange -Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] -Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [id=#86] +Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [id=#57] (87) BroadcastHashJoin [codegen id : 9] -Left keys [3]: [c_current_addr_sk#77, c_birth_country#80, s_zip#67] -Right keys [3]: [ca_address_sk#82, upper(ca_country#85), ca_zip#84] +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 (88) Project [codegen id : 9] -Output [11]: [ss_net_paid#56, s_store_name#64, s_state#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#78, c_last_name#79, ca_state#83] -Input [17]: [ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80, ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] +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] (89) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#56, s_store_name#64, s_state#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#78, c_last_name#79, ca_state#83] -Keys [10]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#56))] -Aggregate Attributes [1]: [sum#87] -Results [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] +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#58] +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#59] (90) Exchange -Input [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] -Arguments: hashpartitioning(c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#89] +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#59] +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=#60] (91) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] -Keys [10]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] -Functions [1]: [sum(UnscaledValue(ss_net_paid#56))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#56))#90] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#56))#90,17,2) AS netpaid#40] +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#59] +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))#61] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#61,17,2) AS netpaid#40] (92) HashAggregate [codegen id : 10] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#91, count#92] -Results [2]: [sum#93, count#94] +Aggregate Attributes [2]: [sum#62, count#63] +Results [2]: [sum#64, count#65] (93) Exchange -Input [2]: [sum#93, count#94] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] +Input [2]: [sum#64, count#65] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] (94) HashAggregate [codegen id : 11] -Input [2]: [sum#93, count#94] +Input [2]: [sum#64, count#65] Keys: [] Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#96] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#96)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#97] +Aggregate Attributes [1]: [avg(netpaid#40)#67] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#67)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#68] 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 1fb9533024b2f..6c48953be5aeb 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +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#61), dynamicpruningexpression(ss_sold_date_sk#61 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 (34) ColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +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] (35) Filter [codegen id : 11] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] -Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) +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)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#62] +Output [1]: [cd_demo_sk#10] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#55] -Right keys [1]: [cd_demo_sk#62] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (38) Project [codegen id : 11] -Output [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] -Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, cd_demo_sk#62] +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] (39) Scan parquet default.store -Output [2]: [s_store_sk#63, s_state#64] +Output [2]: [s_store_sk#18, s_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 8] -Input [2]: [s_store_sk#63, s_state#64] +Input [2]: [s_store_sk#18, s_state#19] (41) Filter [codegen id : 8] -Input [2]: [s_store_sk#63, s_state#64] -Condition : ((isnotnull(s_state#64) AND (s_state#64 = TN)) AND isnotnull(s_store_sk#63)) +Input [2]: [s_store_sk#18, s_state#19] +Condition : ((isnotnull(s_state#19) AND (s_state#19 = TN)) AND isnotnull(s_store_sk#18)) (42) Project [codegen id : 8] -Output [1]: [s_store_sk#63] -Input [2]: [s_store_sk#63, s_state#64] +Output [1]: [s_store_sk#18] +Input [2]: [s_store_sk#18, s_state#19] (43) BroadcastExchange -Input [1]: [s_store_sk#63] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] +Input [1]: [s_store_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] (44) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#56] -Right keys [1]: [s_store_sk#63] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None (45) Project [codegen id : 11] -Output [6]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] -Input [8]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, s_store_sk#63] +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#18] (46) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#66] +Output [1]: [d_date_sk#15] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#61] -Right keys [1]: [d_date_sk#66] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (48) Project [codegen id : 11] -Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] -Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#66] +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] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#67, i_item_id#68] +Output [2]: [i_item_sk#21, i_item_id#22] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#67] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#21] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] -Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] +Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] +Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#68] +Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#22] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] -Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] +Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] (53) Exchange -Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Keys [1]: [i_item_id#68] +Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Keys [1]: [i_item_id#22] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] -Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] +Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] +Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] (55) Scan parquet default.store_sales -Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +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#103), dynamicpruningexpression(ss_sold_date_sk#103 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 (56) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +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] (57) Filter [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) +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)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#104] +Output [1]: [cd_demo_sk#10] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#97] -Right keys [1]: [cd_demo_sk#104] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (60) Project [codegen id : 17] -Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] +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] (61) ReusedExchange [Reuses operator id: 43] -Output [1]: [s_store_sk#105] +Output [1]: [s_store_sk#18] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#98] -Right keys [1]: [s_store_sk#105] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, s_store_sk#105] +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#18] (64) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#106] +Output [1]: [d_date_sk#15] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#103] -Right keys [1]: [d_date_sk#106] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [7]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#106] +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] (67) Scan parquet default.item -Output [1]: [i_item_sk#107] +Output [1]: [i_item_sk#21] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#107] +Input [1]: [i_item_sk#21] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#107] -Condition : isnotnull(i_item_sk#107) +Input [1]: [i_item_sk#21] +Condition : isnotnull(i_item_sk#21) (70) BroadcastExchange -Input [1]: [i_item_sk#107] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] +Input [1]: [i_item_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#96] -Right keys [1]: [i_item_sk#107] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#21] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] -Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] +Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] +Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] -Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] +Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] (74) Exchange -Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] +Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] -Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] +Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] +Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#15] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 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 f05f4e9f2e43e..04c79a5413ad9 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +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#61), dynamicpruningexpression(ss_sold_date_sk#61 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 (34) ColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +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] (35) Filter [codegen id : 11] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] -Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) +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)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#62] +Output [1]: [cd_demo_sk#10] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#55] -Right keys [1]: [cd_demo_sk#62] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (38) Project [codegen id : 11] -Output [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] -Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, cd_demo_sk#62] +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] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#63] +Output [1]: [d_date_sk#15] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#61] -Right keys [1]: [d_date_sk#63] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (41) Project [codegen id : 11] -Output [6]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] -Input [8]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#63] +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] (42) Scan parquet default.store -Output [2]: [s_store_sk#64, s_state#65] +Output [2]: [s_store_sk#18, s_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (43) ColumnarToRow [codegen id : 9] -Input [2]: [s_store_sk#64, s_state#65] +Input [2]: [s_store_sk#18, s_state#19] (44) Filter [codegen id : 9] -Input [2]: [s_store_sk#64, s_state#65] -Condition : ((isnotnull(s_state#65) AND (s_state#65 = TN)) AND isnotnull(s_store_sk#64)) +Input [2]: [s_store_sk#18, s_state#19] +Condition : ((isnotnull(s_state#19) AND (s_state#19 = TN)) AND isnotnull(s_store_sk#18)) (45) Project [codegen id : 9] -Output [1]: [s_store_sk#64] -Input [2]: [s_store_sk#64, s_state#65] +Output [1]: [s_store_sk#18] +Input [2]: [s_store_sk#18, s_state#19] (46) BroadcastExchange -Input [1]: [s_store_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#66] +Input [1]: [s_store_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#56] -Right keys [1]: [s_store_sk#64] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None (48) Project [codegen id : 11] -Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] -Input [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, s_store_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_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#18] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#67, i_item_id#68] +Output [2]: [i_item_sk#21, i_item_id#22] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#67] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#21] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] -Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] +Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] +Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#68] +Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#22] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] -Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] +Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] (53) Exchange -Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Keys [1]: [i_item_id#68] +Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Keys [1]: [i_item_id#22] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] -Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] +Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] +Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] (55) Scan parquet default.store_sales -Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +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#103), dynamicpruningexpression(ss_sold_date_sk#103 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 (56) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +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] (57) Filter [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) +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)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#104] +Output [1]: [cd_demo_sk#10] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#97] -Right keys [1]: [cd_demo_sk#104] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (60) Project [codegen id : 17] -Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] +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] (61) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#105] +Output [1]: [d_date_sk#15] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#103] -Right keys [1]: [d_date_sk#105] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#105] +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] (64) ReusedExchange [Reuses operator id: 46] -Output [1]: [s_store_sk#106] +Output [1]: [s_store_sk#18] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#98] -Right keys [1]: [s_store_sk#106] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#106] +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#18] (67) Scan parquet default.item -Output [1]: [i_item_sk#107] +Output [1]: [i_item_sk#21] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#107] +Input [1]: [i_item_sk#21] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#107] -Condition : isnotnull(i_item_sk#107) +Input [1]: [i_item_sk#21] +Condition : isnotnull(i_item_sk#21) (70) BroadcastExchange -Input [1]: [i_item_sk#107] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] +Input [1]: [i_item_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#96] -Right keys [1]: [i_item_sk#107] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#21] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] -Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] +Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] +Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] -Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] +Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] (74) Exchange -Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] +Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] -Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] +Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] +Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#15] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 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/q34.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt index f6151f4d79ec1..104340f2fd6db 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.sf100/explain.txt @@ -121,7 +121,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -129,7 +129,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt index e4d14f842f5a9..f106fba199355 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt @@ -118,7 +118,7 @@ Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, Output [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), IsNotNull(hd_dep_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,unknown)), GreaterThan(hd_vehicle_count,0), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -126,7 +126,7 @@ Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#14, hd_buy_potential#15, hd_dep_count#16, hd_vehicle_count#17] -Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000 ) OR (hd_buy_potential#15 = unknown ))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) +Condition : (((((isnotnull(hd_vehicle_count#17) AND isnotnull(hd_dep_count#16)) AND ((hd_buy_potential#15 = >10000) OR (hd_buy_potential#15 = unknown))) AND (hd_vehicle_count#17 > 0)) AND ((cast(hd_dep_count#16 as double) / cast(hd_vehicle_count#17 as double)) > 1.2)) AND isnotnull(hd_demo_sk#14)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt index f7fff421817c0..0a861f4ce8e6d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt @@ -145,20 +145,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#17] +Output [1]: [d_date_sk#10] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#17] +Right keys [1]: [d_date_sk#10] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] -Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] +Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] (23) Exchange Input [1]: [ws_bill_customer_sk#15] -Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] +Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] @@ -170,38 +170,38 @@ Right keys [1]: [ws_bill_customer_sk#15] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] +Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] +Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#21] +Output [1]: [d_date_sk#10] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#21] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#10] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#19] -Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] +Output [1]: [cs_ship_customer_sk#18] +Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] (31) Exchange -Input [1]: [cs_ship_customer_sk#19] -Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [cs_ship_customer_sk#18] +Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#19] -Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#18] +Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#19] +Right keys [1]: [cs_ship_customer_sk#18] Join condition: None (34) Filter [codegen id : 12] @@ -214,103 +214,103 @@ Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2 (36) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#23] +Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#21] (37) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 (38) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#22, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 14] -Input [2]: [ca_address_sk#24, ca_state#25] +Input [2]: [ca_address_sk#22, ca_state#23] (40) Filter [codegen id : 14] -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : isnotnull(ca_address_sk#24) +Input [2]: [ca_address_sk#22, ca_state#23] +Condition : isnotnull(ca_address_sk#22) (41) Exchange -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [ca_address_sk#22, ca_state#23] +Arguments: hashpartitioning(ca_address_sk#22, 5), ENSURE_REQUIREMENTS, [id=#24] (42) Sort [codegen id : 15] -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#22, ca_state#23] +Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#24] +Right keys [1]: [ca_address_sk#22] Join condition: None (44) Project [codegen id : 16] -Output [2]: [c_current_cdemo_sk#4, ca_state#25] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24, ca_state#25] +Output [2]: [c_current_cdemo_sk#4, ca_state#23] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] (45) Exchange -Input [2]: [c_current_cdemo_sk#4, ca_state#25] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [2]: [c_current_cdemo_sk#4, ca_state#23] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#25] (46) Sort [codegen id : 17] -Input [2]: [c_current_cdemo_sk#4, ca_state#25] +Input [2]: [c_current_cdemo_sk#4, ca_state#23] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 (47) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 18] -Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] (49) Filter [codegen id : 18] -Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Condition : isnotnull(cd_demo_sk#28) +Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Condition : isnotnull(cd_demo_sk#26) (50) Exchange -Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Arguments: hashpartitioning(cd_demo_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] (51) Sort [codegen id : 19] -Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 +Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Arguments: [cd_demo_sk#26 ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#28] +Right keys [1]: [cd_demo_sk#26] Join condition: None (53) Project [codegen id : 20] -Output [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Input [8]: [c_current_cdemo_sk#4, ca_state#25, cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] (54) HashAggregate [codegen id : 20] -Input [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#31), partial_max(cd_dep_count#31), partial_sum(cd_dep_count#31), partial_avg(cd_dep_employed_count#32), partial_max(cd_dep_employed_count#32), partial_sum(cd_dep_employed_count#32), partial_avg(cd_dep_college_count#33), partial_max(cd_dep_college_count#33), partial_sum(cd_dep_college_count#33)] -Aggregate Attributes [13]: [count#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47] -Results [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] +Input [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#29), partial_max(cd_dep_count#29), partial_sum(cd_dep_count#29), partial_avg(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_sum(cd_dep_employed_count#30), partial_avg(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_sum(cd_dep_college_count#31)] +Aggregate Attributes [13]: [count#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45] +Results [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] (55) Exchange -Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] -Arguments: hashpartitioning(ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] +Arguments: hashpartitioning(ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#59] (56) HashAggregate [codegen id : 21] -Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] -Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Functions [10]: [count(1), avg(cd_dep_count#31), max(cd_dep_count#31), sum(cd_dep_count#31), avg(cd_dep_employed_count#32), max(cd_dep_employed_count#32), sum(cd_dep_employed_count#32), avg(cd_dep_college_count#33), max(cd_dep_college_count#33), sum(cd_dep_college_count#33)] -Aggregate Attributes [10]: [count(1)#62, avg(cd_dep_count#31)#63, max(cd_dep_count#31)#64, sum(cd_dep_count#31)#65, avg(cd_dep_employed_count#32)#66, max(cd_dep_employed_count#32)#67, sum(cd_dep_employed_count#32)#68, avg(cd_dep_college_count#33)#69, max(cd_dep_college_count#33)#70, sum(cd_dep_college_count#33)#71] -Results [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, count(1)#62 AS cnt1#72, avg(cd_dep_count#31)#63 AS avg(cd_dep_count)#73, max(cd_dep_count#31)#64 AS max(cd_dep_count)#74, sum(cd_dep_count#31)#65 AS sum(cd_dep_count)#75, cd_dep_employed_count#32, count(1)#62 AS cnt2#76, avg(cd_dep_employed_count#32)#66 AS avg(cd_dep_employed_count)#77, max(cd_dep_employed_count#32)#67 AS max(cd_dep_employed_count)#78, sum(cd_dep_employed_count#32)#68 AS sum(cd_dep_employed_count)#79, cd_dep_college_count#33, count(1)#62 AS cnt3#80, avg(cd_dep_college_count#33)#69 AS avg(cd_dep_college_count)#81, max(cd_dep_college_count#33)#70 AS max(cd_dep_college_count)#82, sum(cd_dep_college_count#33)#71 AS sum(cd_dep_college_count)#83] +Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] +Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Functions [10]: [count(1), avg(cd_dep_count#29), max(cd_dep_count#29), sum(cd_dep_count#29), avg(cd_dep_employed_count#30), max(cd_dep_employed_count#30), sum(cd_dep_employed_count#30), avg(cd_dep_college_count#31), max(cd_dep_college_count#31), sum(cd_dep_college_count#31)] +Aggregate Attributes [10]: [count(1)#60, avg(cd_dep_count#29)#61, max(cd_dep_count#29)#62, sum(cd_dep_count#29)#63, avg(cd_dep_employed_count#30)#64, max(cd_dep_employed_count#30)#65, sum(cd_dep_employed_count#30)#66, avg(cd_dep_college_count#31)#67, max(cd_dep_college_count#31)#68, sum(cd_dep_college_count#31)#69] +Results [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, count(1)#60 AS cnt1#70, avg(cd_dep_count#29)#61 AS avg(cd_dep_count)#71, max(cd_dep_count#29)#62 AS max(cd_dep_count)#72, sum(cd_dep_count#29)#63 AS sum(cd_dep_count)#73, cd_dep_employed_count#30, count(1)#60 AS cnt2#74, avg(cd_dep_employed_count#30)#64 AS avg(cd_dep_employed_count)#75, max(cd_dep_employed_count#30)#65 AS max(cd_dep_employed_count)#76, sum(cd_dep_employed_count#30)#66 AS sum(cd_dep_employed_count)#77, cd_dep_college_count#31, count(1)#60 AS cnt3#78, avg(cd_dep_college_count#31)#67 AS avg(cd_dep_college_count)#79, max(cd_dep_college_count#31)#68 AS max(cd_dep_college_count)#80, sum(cd_dep_college_count#31)#69 AS sum(cd_dep_college_count)#81] (57) TakeOrderedAndProject -Input [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cnt1#72, avg(cd_dep_count)#73, max(cd_dep_count)#74, sum(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, avg(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, sum(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, avg(cd_dep_college_count)#81, max(cd_dep_college_count)#82, sum(cd_dep_college_count)#83] -Arguments: 100, [ca_state#25 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cnt1#72, avg(cd_dep_count)#73, max(cd_dep_count)#74, sum(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, avg(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, sum(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, avg(cd_dep_college_count)#81, max(cd_dep_college_count)#82, sum(cd_dep_college_count)#83] +Input [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#70, avg(cd_dep_count)#71, max(cd_dep_count)#72, sum(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, avg(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, sum(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, avg(cd_dep_college_count)#79, max(cd_dep_college_count)#80, sum(cd_dep_college_count)#81] +Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#70, avg(cd_dep_count)#71, max(cd_dep_count)#72, sum(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, avg(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, sum(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, avg(cd_dep_college_count)#79, max(cd_dep_college_count)#80, sum(cd_dep_college_count)#81] ===== Subqueries ===== @@ -323,6 +323,6 @@ Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index 642bf989159d3..12c8230c7e3bc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -122,20 +122,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#9] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#9] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#16] +Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] @@ -143,34 +143,34 @@ Right keys [1]: [ws_bill_customer_sk#14] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#20] +Output [1]: [d_date_sk#9] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#20] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#9] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#18] -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#20] +Output [1]: [cs_ship_customer_sk#17] +Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [cs_ship_customer_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#17] Join condition: None (29) Filter [codegen id : 9] @@ -182,80 +182,80 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_state#23] +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#22, ca_state#23] +Input [2]: [ca_address_sk#20, ca_state#21] (33) Filter [codegen id : 7] -Input [2]: [ca_address_sk#22, ca_state#23] -Condition : isnotnull(ca_address_sk#22) +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) (34) BroadcastExchange -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] +Input [2]: [ca_address_sk#20, ca_state#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] (35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#20] Join condition: None (36) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#23] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] +Output [2]: [c_current_cdemo_sk#4, ca_state#21] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] (37) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (39) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#25) +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Condition : isnotnull(cd_demo_sk#23) (40) BroadcastExchange -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#23] Join condition: None (42) Project [codegen id : 9] -Output [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (43) HashAggregate [codegen id : 9] -Input [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#28), partial_max(cd_dep_count#28), partial_sum(cd_dep_count#28), partial_avg(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_sum(cd_dep_employed_count#29), partial_avg(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_sum(cd_dep_college_count#30)] -Aggregate Attributes [13]: [count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] -Results [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] +Input [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] +Aggregate Attributes [13]: [count#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42] +Results [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] (44) Exchange -Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] -Arguments: hashpartitioning(ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] +Arguments: hashpartitioning(ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [id=#56] (45) HashAggregate [codegen id : 10] -Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] -Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [10]: [count(1), avg(cd_dep_count#28), max(cd_dep_count#28), sum(cd_dep_count#28), avg(cd_dep_employed_count#29), max(cd_dep_employed_count#29), sum(cd_dep_employed_count#29), avg(cd_dep_college_count#30), max(cd_dep_college_count#30), sum(cd_dep_college_count#30)] -Aggregate Attributes [10]: [count(1)#59, avg(cd_dep_count#28)#60, max(cd_dep_count#28)#61, sum(cd_dep_count#28)#62, avg(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, sum(cd_dep_employed_count#29)#65, avg(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, sum(cd_dep_college_count#30)#68] -Results [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, count(1)#59 AS cnt1#69, avg(cd_dep_count#28)#60 AS avg(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, sum(cd_dep_count#28)#62 AS sum(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, avg(cd_dep_employed_count#29)#63 AS avg(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, sum(cd_dep_employed_count#29)#65 AS sum(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, avg(cd_dep_college_count#30)#66 AS avg(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, sum(cd_dep_college_count#30)#68 AS sum(cd_dep_college_count)#80] +Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] +Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] +Aggregate Attributes [10]: [count(1)#57, avg(cd_dep_count#26)#58, max(cd_dep_count#26)#59, sum(cd_dep_count#26)#60, avg(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, sum(cd_dep_employed_count#27)#63, avg(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, sum(cd_dep_college_count#28)#66] +Results [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, count(1)#57 AS cnt1#67, avg(cd_dep_count#26)#58 AS avg(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, sum(cd_dep_count#26)#60 AS sum(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, avg(cd_dep_employed_count#27)#61 AS avg(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, sum(cd_dep_employed_count#27)#63 AS sum(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, avg(cd_dep_college_count#28)#64 AS avg(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, sum(cd_dep_college_count#28)#66 AS sum(cd_dep_college_count)#78] (46) TakeOrderedAndProject -Input [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] -Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] +Input [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] +Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] ===== Subqueries ===== @@ -268,6 +268,6 @@ Output [1]: [d_date_sk#9] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt index df75bec72326d..1c1b8ad01abfa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt @@ -142,52 +142,52 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#8] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] +Right keys [1]: [d_date_sk#8] Join condition: None (22) Project [codegen id : 7] -Output [1]: [ws_bill_customer_sk#13 AS customsk#16] -Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] +Output [1]: [ws_bill_customer_sk#13 AS customsk#15] +Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] (23) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] ReadSchema: struct (24) ColumnarToRow [codegen id : 9] -Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] (25) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#19] +Output [1]: [d_date_sk#8] (26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#19] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#8] Join condition: None (27) Project [codegen id : 9] -Output [1]: [cs_ship_customer_sk#17 AS customsk#20] -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] +Output [1]: [cs_ship_customer_sk#16 AS customsk#18] +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] (28) Union (29) Exchange -Input [1]: [customsk#16] -Arguments: hashpartitioning(customsk#16, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [1]: [customsk#15] +Arguments: hashpartitioning(customsk#15, 5), ENSURE_REQUIREMENTS, [id=#19] (30) Sort [codegen id : 10] -Input [1]: [customsk#16] -Arguments: [customsk#16 ASC NULLS FIRST], false, 0 +Input [1]: [customsk#15] +Arguments: [customsk#15 ASC NULLS FIRST], false, 0 (31) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#16] +Right keys [1]: [customsk#15] Join condition: None (32) Project [codegen id : 11] @@ -196,103 +196,103 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (33) Exchange Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: hashpartitioning(c_current_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#22] +Arguments: hashpartitioning(c_current_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#20] (34) Sort [codegen id : 12] Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: [c_current_addr_sk#3 ASC NULLS FIRST], false, 0 (35) Scan parquet default.customer_address -Output [2]: [ca_address_sk#23, ca_state#24] +Output [2]: [ca_address_sk#21, ca_state#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 13] -Input [2]: [ca_address_sk#23, ca_state#24] +Input [2]: [ca_address_sk#21, ca_state#22] (37) Filter [codegen id : 13] -Input [2]: [ca_address_sk#23, ca_state#24] -Condition : isnotnull(ca_address_sk#23) +Input [2]: [ca_address_sk#21, ca_state#22] +Condition : isnotnull(ca_address_sk#21) (38) Exchange -Input [2]: [ca_address_sk#23, ca_state#24] -Arguments: hashpartitioning(ca_address_sk#23, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [2]: [ca_address_sk#21, ca_state#22] +Arguments: hashpartitioning(ca_address_sk#21, 5), ENSURE_REQUIREMENTS, [id=#23] (39) Sort [codegen id : 14] -Input [2]: [ca_address_sk#23, ca_state#24] -Arguments: [ca_address_sk#23 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#21, ca_state#22] +Arguments: [ca_address_sk#21 ASC NULLS FIRST], false, 0 (40) SortMergeJoin [codegen id : 15] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#23] +Right keys [1]: [ca_address_sk#21] Join condition: None (41) Project [codegen id : 15] -Output [2]: [c_current_cdemo_sk#2, ca_state#24] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#23, ca_state#24] +Output [2]: [c_current_cdemo_sk#2, ca_state#22] +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21, ca_state#22] (42) Exchange -Input [2]: [c_current_cdemo_sk#2, ca_state#24] -Arguments: hashpartitioning(c_current_cdemo_sk#2, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [c_current_cdemo_sk#2, ca_state#22] +Arguments: hashpartitioning(c_current_cdemo_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] (43) Sort [codegen id : 16] -Input [2]: [c_current_cdemo_sk#2, ca_state#24] +Input [2]: [c_current_cdemo_sk#2, ca_state#22] Arguments: [c_current_cdemo_sk#2 ASC NULLS FIRST], false, 0 (44) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (45) ColumnarToRow [codegen id : 17] -Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (46) Filter [codegen id : 17] -Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Condition : isnotnull(cd_demo_sk#27) +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#25) (47) Exchange -Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Arguments: hashpartitioning(cd_demo_sk#27, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: hashpartitioning(cd_demo_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] (48) Sort [codegen id : 18] -Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Arguments: [cd_demo_sk#27 ASC NULLS FIRST], false, 0 +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [cd_demo_sk#25 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 19] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#27] +Right keys [1]: [cd_demo_sk#25] Join condition: None (50) Project [codegen id : 19] -Output [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Input [8]: [c_current_cdemo_sk#2, ca_state#24, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Output [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (51) HashAggregate [codegen id : 19] -Input [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Keys [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#30), partial_max(cd_dep_count#30), partial_sum(cd_dep_count#30), partial_avg(cd_dep_employed_count#31), partial_max(cd_dep_employed_count#31), partial_sum(cd_dep_employed_count#31), partial_avg(cd_dep_college_count#32), partial_max(cd_dep_college_count#32), partial_sum(cd_dep_college_count#32)] -Aggregate Attributes [13]: [count#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42, sum#43, count#44, max#45, sum#46] -Results [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] +Input [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#28), partial_max(cd_dep_count#28), partial_sum(cd_dep_count#28), partial_avg(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_sum(cd_dep_employed_count#29), partial_avg(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_sum(cd_dep_college_count#30)] +Aggregate Attributes [13]: [count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] +Results [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] (52) Exchange -Input [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] -Arguments: hashpartitioning(ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] +Arguments: hashpartitioning(ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] (53) HashAggregate [codegen id : 20] -Input [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] -Keys [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Functions [10]: [count(1), avg(cd_dep_count#30), max(cd_dep_count#30), sum(cd_dep_count#30), avg(cd_dep_employed_count#31), max(cd_dep_employed_count#31), sum(cd_dep_employed_count#31), avg(cd_dep_college_count#32), max(cd_dep_college_count#32), sum(cd_dep_college_count#32)] -Aggregate Attributes [10]: [count(1)#61, avg(cd_dep_count#30)#62, max(cd_dep_count#30)#63, sum(cd_dep_count#30)#64, avg(cd_dep_employed_count#31)#65, max(cd_dep_employed_count#31)#66, sum(cd_dep_employed_count#31)#67, avg(cd_dep_college_count#32)#68, max(cd_dep_college_count#32)#69, sum(cd_dep_college_count#32)#70] -Results [18]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, count(1)#61 AS cnt1#71, avg(cd_dep_count#30)#62 AS avg(cd_dep_count)#72, max(cd_dep_count#30)#63 AS max(cd_dep_count)#73, sum(cd_dep_count#30)#64 AS sum(cd_dep_count)#74, cd_dep_employed_count#31, count(1)#61 AS cnt2#75, avg(cd_dep_employed_count#31)#65 AS avg(cd_dep_employed_count)#76, max(cd_dep_employed_count#31)#66 AS max(cd_dep_employed_count)#77, sum(cd_dep_employed_count#31)#67 AS sum(cd_dep_employed_count)#78, cd_dep_college_count#32, count(1)#61 AS cnt3#79, avg(cd_dep_college_count#32)#68 AS avg(cd_dep_college_count)#80, max(cd_dep_college_count#32)#69 AS max(cd_dep_college_count)#81, sum(cd_dep_college_count#32)#70 AS sum(cd_dep_college_count)#82] +Input [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] +Keys [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Functions [10]: [count(1), avg(cd_dep_count#28), max(cd_dep_count#28), sum(cd_dep_count#28), avg(cd_dep_employed_count#29), max(cd_dep_employed_count#29), sum(cd_dep_employed_count#29), avg(cd_dep_college_count#30), max(cd_dep_college_count#30), sum(cd_dep_college_count#30)] +Aggregate Attributes [10]: [count(1)#59, avg(cd_dep_count#28)#60, max(cd_dep_count#28)#61, sum(cd_dep_count#28)#62, avg(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, sum(cd_dep_employed_count#29)#65, avg(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, sum(cd_dep_college_count#30)#68] +Results [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, count(1)#59 AS cnt1#69, avg(cd_dep_count#28)#60 AS avg(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, sum(cd_dep_count#28)#62 AS sum(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, avg(cd_dep_employed_count#29)#63 AS avg(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, sum(cd_dep_employed_count#29)#65 AS sum(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, avg(cd_dep_college_count#30)#66 AS avg(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, sum(cd_dep_college_count#30)#68 AS sum(cd_dep_college_count)#80] (54) TakeOrderedAndProject -Input [18]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cnt1#71, avg(cd_dep_count)#72, max(cd_dep_count)#73, sum(cd_dep_count)#74, cd_dep_employed_count#31, cnt2#75, avg(cd_dep_employed_count)#76, max(cd_dep_employed_count)#77, sum(cd_dep_employed_count)#78, cd_dep_college_count#32, cnt3#79, avg(cd_dep_college_count)#80, max(cd_dep_college_count)#81, sum(cd_dep_college_count)#82] -Arguments: 100, [ca_state#24 ASC NULLS FIRST, cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cnt1#71, avg(cd_dep_count)#72, max(cd_dep_count)#73, sum(cd_dep_count)#74, cd_dep_employed_count#31, cnt2#75, avg(cd_dep_employed_count)#76, max(cd_dep_employed_count)#77, sum(cd_dep_employed_count)#78, cd_dep_college_count#32, cnt3#79, avg(cd_dep_college_count)#80, max(cd_dep_college_count)#81, sum(cd_dep_college_count)#82] +Input [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] +Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] ===== Subqueries ===== @@ -305,6 +305,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index 57710f427f0f8..ffb07d398de71 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -120,48 +120,48 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#7] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#7] Join condition: None (19) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#12 AS customsk#15] -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] +Output [1]: [ws_bill_customer_sk#12 AS customsk#14] +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] (20) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] (22) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#7] (23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#18] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#7] Join condition: None (24) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#16 AS customsk#19] -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#18] +Output [1]: [cs_ship_customer_sk#15 AS customsk#17] +Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] (25) Union (26) BroadcastExchange -Input [1]: [customsk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +Input [1]: [customsk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (27) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#15] +Right keys [1]: [customsk#14] Join condition: None (28) Project [codegen id : 9] @@ -169,80 +169,80 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] +Output [2]: [ca_address_sk#19, ca_state#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#21, ca_state#22] +Input [2]: [ca_address_sk#19, ca_state#20] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#21, ca_state#22] -Condition : isnotnull(ca_address_sk#21) +Input [2]: [ca_address_sk#19, ca_state#20] +Condition : isnotnull(ca_address_sk#19) (32) BroadcastExchange -Input [2]: [ca_address_sk#21, ca_state#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] +Input [2]: [ca_address_sk#19, ca_state#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (33) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#19] Join condition: None (34) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, ca_state#22] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21, ca_state#22] +Output [2]: [c_current_cdemo_sk#2, ca_state#20] +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19, ca_state#20] (35) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] (37) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Condition : isnotnull(cd_demo_sk#24) +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Condition : isnotnull(cd_demo_sk#22) (38) BroadcastExchange -Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] (39) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#24] +Right keys [1]: [cd_demo_sk#22] Join condition: None (40) Project [codegen id : 9] -Output [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Output [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Input [8]: [c_current_cdemo_sk#2, ca_state#20, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] (41) HashAggregate [codegen id : 9] -Input [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Keys [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#27), partial_max(cd_dep_count#27), partial_sum(cd_dep_count#27), partial_avg(cd_dep_employed_count#28), partial_max(cd_dep_employed_count#28), partial_sum(cd_dep_employed_count#28), partial_avg(cd_dep_college_count#29), partial_max(cd_dep_college_count#29), partial_sum(cd_dep_college_count#29)] -Aggregate Attributes [13]: [count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Results [19]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] +Input [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_dep_count#25), partial_sum(cd_dep_count#25), partial_avg(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_sum(cd_dep_employed_count#26), partial_avg(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_sum(cd_dep_college_count#27)] +Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] +Results [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] (42) Exchange -Input [19]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -Arguments: hashpartitioning(ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +Arguments: hashpartitioning(ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [id=#55] (43) HashAggregate [codegen id : 10] -Input [19]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -Keys [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Functions [10]: [count(1), avg(cd_dep_count#27), max(cd_dep_count#27), sum(cd_dep_count#27), avg(cd_dep_employed_count#28), max(cd_dep_employed_count#28), sum(cd_dep_employed_count#28), avg(cd_dep_college_count#29), max(cd_dep_college_count#29), sum(cd_dep_college_count#29)] -Aggregate Attributes [10]: [count(1)#58, avg(cd_dep_count#27)#59, max(cd_dep_count#27)#60, sum(cd_dep_count#27)#61, avg(cd_dep_employed_count#28)#62, max(cd_dep_employed_count#28)#63, sum(cd_dep_employed_count#28)#64, avg(cd_dep_college_count#29)#65, max(cd_dep_college_count#29)#66, sum(cd_dep_college_count#29)#67] -Results [18]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, count(1)#58 AS cnt1#68, avg(cd_dep_count#27)#59 AS avg(cd_dep_count)#69, max(cd_dep_count#27)#60 AS max(cd_dep_count)#70, sum(cd_dep_count#27)#61 AS sum(cd_dep_count)#71, cd_dep_employed_count#28, count(1)#58 AS cnt2#72, avg(cd_dep_employed_count#28)#62 AS avg(cd_dep_employed_count)#73, max(cd_dep_employed_count#28)#63 AS max(cd_dep_employed_count)#74, sum(cd_dep_employed_count#28)#64 AS sum(cd_dep_employed_count)#75, cd_dep_college_count#29, count(1)#58 AS cnt3#76, avg(cd_dep_college_count#29)#65 AS avg(cd_dep_college_count)#77, max(cd_dep_college_count#29)#66 AS max(cd_dep_college_count)#78, sum(cd_dep_college_count#29)#67 AS sum(cd_dep_college_count)#79] +Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_dep_count#25), avg(cd_dep_employed_count#26), max(cd_dep_employed_count#26), sum(cd_dep_employed_count#26), avg(cd_dep_college_count#27), max(cd_dep_college_count#27), sum(cd_dep_college_count#27)] +Aggregate Attributes [10]: [count(1)#56, avg(cd_dep_count#25)#57, max(cd_dep_count#25)#58, sum(cd_dep_count#25)#59, avg(cd_dep_employed_count#26)#60, max(cd_dep_employed_count#26)#61, sum(cd_dep_employed_count#26)#62, avg(cd_dep_college_count#27)#63, max(cd_dep_college_count#27)#64, sum(cd_dep_college_count#27)#65] +Results [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, count(1)#56 AS cnt1#66, avg(cd_dep_count#25)#57 AS avg(cd_dep_count)#67, max(cd_dep_count#25)#58 AS max(cd_dep_count)#68, sum(cd_dep_count#25)#59 AS sum(cd_dep_count)#69, cd_dep_employed_count#26, count(1)#56 AS cnt2#70, avg(cd_dep_employed_count#26)#60 AS avg(cd_dep_employed_count)#71, max(cd_dep_employed_count#26)#61 AS max(cd_dep_employed_count)#72, sum(cd_dep_employed_count#26)#62 AS sum(cd_dep_employed_count)#73, cd_dep_college_count#27, count(1)#56 AS cnt3#74, avg(cd_dep_college_count#27)#63 AS avg(cd_dep_college_count)#75, max(cd_dep_college_count#27)#64 AS max(cd_dep_college_count)#76, sum(cd_dep_college_count#27)#65 AS sum(cd_dep_college_count)#77] (44) TakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#28, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#29, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79] -Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_dep_count#27 ASC NULLS FIRST, cd_dep_employed_count#28 ASC NULLS FIRST, cd_dep_college_count#29 ASC NULLS FIRST], [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#28, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#29, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79] +Input [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] +Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] ===== Subqueries ===== @@ -255,6 +255,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 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 79037befbd6b5..9f594df6754ba 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 @@ -169,60 +169,60 @@ Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#22, sum(UnscaledV Results [6]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#22,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#23,17,2))), DecimalType(37,20), true) as decimal(38,20)) AS gross_margin#24, i_category#15, i_class#14, 0 AS t_category#25, 0 AS t_class#26, 0 AS lochierarchy#27] (27) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#28, i_class#29, sum#30, sum#31] +Output [4]: [i_category#15, i_class#14, sum#28, sum#29] (28) HashAggregate [codegen id : 10] -Input [4]: [i_category#28, i_class#29, sum#30, sum#31] -Keys [2]: [i_category#28, i_class#29] -Functions [2]: [sum(UnscaledValue(ss_net_profit#32)), sum(UnscaledValue(ss_ext_sales_price#33))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#32))#34, sum(UnscaledValue(ss_ext_sales_price#33))#35] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#32))#34,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#33))#35,17,2) AS ss_ext_sales_price#37, i_category#28] +Input [4]: [i_category#15, i_class#14, sum#28, sum#29] +Keys [2]: [i_category#15, i_class#14] +Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#30, sum(UnscaledValue(ss_ext_sales_price#3))#31] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#30,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#31,17,2) AS ss_ext_sales_price#33, i_category#15] (29) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#36, ss_ext_sales_price#37, i_category#28] -Keys [1]: [i_category#28] -Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] -Aggregate Attributes [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] -Results [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] +Input [3]: [ss_net_profit#32, ss_ext_sales_price#33, i_category#15] +Keys [1]: [i_category#15] +Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] +Aggregate Attributes [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] +Results [5]: [i_category#15, sum#38, isEmpty#39, sum#40, isEmpty#41] (30) Exchange -Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [5]: [i_category#15, sum#38, isEmpty#39, sum#40, isEmpty#41] +Arguments: hashpartitioning(i_category#15, 5), ENSURE_REQUIREMENTS, [id=#42] (31) HashAggregate [codegen id : 11] -Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] -Keys [1]: [i_category#28] -Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] -Aggregate Attributes [2]: [sum(ss_net_profit#36)#47, sum(ss_ext_sales_price#37)#48] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#47) / promote_precision(sum(ss_ext_sales_price#37)#48)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#49, i_category#28, null AS i_class#50, 0 AS t_category#51, 1 AS t_class#52, 1 AS lochierarchy#53] +Input [5]: [i_category#15, sum#38, isEmpty#39, sum#40, isEmpty#41] +Keys [1]: [i_category#15] +Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] +Aggregate Attributes [2]: [sum(ss_net_profit#32)#43, sum(ss_ext_sales_price#33)#44] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#43) / promote_precision(sum(ss_ext_sales_price#33)#44)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#45, i_category#15, null AS i_class#46, 0 AS t_category#47, 1 AS t_class#48, 1 AS lochierarchy#49] (32) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#54, i_class#55, sum#56, sum#57] +Output [4]: [i_category#15, i_class#14, sum#50, sum#51] (33) HashAggregate [codegen id : 16] -Input [4]: [i_category#54, i_class#55, sum#56, sum#57] -Keys [2]: [i_category#54, i_class#55] -Functions [2]: [sum(UnscaledValue(ss_net_profit#58)), sum(UnscaledValue(ss_ext_sales_price#59))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#58))#60, sum(UnscaledValue(ss_ext_sales_price#59))#61] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#58))#60,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#59))#61,17,2) AS ss_ext_sales_price#37] +Input [4]: [i_category#15, i_class#14, sum#50, sum#51] +Keys [2]: [i_category#15, i_class#14] +Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#52, sum(UnscaledValue(ss_ext_sales_price#3))#53] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#52,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#53,17,2) AS ss_ext_sales_price#33] (34) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#36, ss_ext_sales_price#37] +Input [2]: [ss_net_profit#32, ss_ext_sales_price#33] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] -Aggregate Attributes [4]: [sum#62, isEmpty#63, sum#64, isEmpty#65] -Results [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] +Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] +Aggregate Attributes [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] +Results [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] (35) Exchange -Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#70] +Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] (36) HashAggregate [codegen id : 17] -Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] +Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] Keys: [] -Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] -Aggregate Attributes [2]: [sum(ss_net_profit#36)#71, sum(ss_ext_sales_price#37)#72] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#71) / promote_precision(sum(ss_ext_sales_price#37)#72)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#73, null AS i_category#74, null AS i_class#75, 1 AS t_category#76, 1 AS t_class#77, 2 AS lochierarchy#78] +Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] +Aggregate Attributes [2]: [sum(ss_net_profit#32)#63, sum(ss_ext_sales_price#33)#64] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#63) / promote_precision(sum(ss_ext_sales_price#33)#64)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#65, null AS i_category#66, null AS i_class#67, 1 AS t_category#68, 1 AS t_class#69, 2 AS lochierarchy#70] (37) Union @@ -235,34 +235,34 @@ Results [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class (39) Exchange Input [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27] -Arguments: hashpartitioning(gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#79] +Arguments: hashpartitioning(gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#71] (40) HashAggregate [codegen id : 19] Input [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27] Keys [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27] Functions: [] Aggregate Attributes: [] -Results [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#15 END AS _w0#80] +Results [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#15 END AS _w0#72] (41) Exchange -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80] -Arguments: hashpartitioning(lochierarchy#27, _w0#80, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72] +Arguments: hashpartitioning(lochierarchy#27, _w0#72, 5), ENSURE_REQUIREMENTS, [id=#73] (42) Sort [codegen id : 20] -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80] -Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#80 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72] +Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 (43) Window -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80] -Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#80, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#82], [lochierarchy#27, _w0#80], [gross_margin#24 ASC NULLS FIRST] +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72] +Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#72, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#74], [lochierarchy#27, _w0#72], [gross_margin#24 ASC NULLS FIRST] (44) Project [codegen id : 21] -Output [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#82] -Input [6]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80, rank_within_parent#82] +Output [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#74] +Input [6]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72, rank_within_parent#74] (45) TakeOrderedAndProject -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#82] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#82 ASC NULLS FIRST], [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#82] +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#74] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#74 ASC NULLS FIRST], [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#74] ===== Subqueries ===== 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 bc2931129868b..5e58913557f22 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 @@ -169,60 +169,60 @@ Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#22, sum(UnscaledV Results [6]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#22,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#23,17,2))), DecimalType(37,20), true) as decimal(38,20)) AS gross_margin#24, i_category#12, i_class#11, 0 AS t_category#25, 0 AS t_class#26, 0 AS lochierarchy#27] (27) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#28, i_class#29, sum#30, sum#31] +Output [4]: [i_category#12, i_class#11, sum#28, sum#29] (28) HashAggregate [codegen id : 10] -Input [4]: [i_category#28, i_class#29, sum#30, sum#31] -Keys [2]: [i_category#28, i_class#29] -Functions [2]: [sum(UnscaledValue(ss_net_profit#32)), sum(UnscaledValue(ss_ext_sales_price#33))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#32))#34, sum(UnscaledValue(ss_ext_sales_price#33))#35] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#32))#34,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#33))#35,17,2) AS ss_ext_sales_price#37, i_category#28] +Input [4]: [i_category#12, i_class#11, sum#28, sum#29] +Keys [2]: [i_category#12, i_class#11] +Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#30, sum(UnscaledValue(ss_ext_sales_price#3))#31] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#30,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#31,17,2) AS ss_ext_sales_price#33, i_category#12] (29) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#36, ss_ext_sales_price#37, i_category#28] -Keys [1]: [i_category#28] -Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] -Aggregate Attributes [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] -Results [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] +Input [3]: [ss_net_profit#32, ss_ext_sales_price#33, i_category#12] +Keys [1]: [i_category#12] +Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] +Aggregate Attributes [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] +Results [5]: [i_category#12, sum#38, isEmpty#39, sum#40, isEmpty#41] (30) Exchange -Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [5]: [i_category#12, sum#38, isEmpty#39, sum#40, isEmpty#41] +Arguments: hashpartitioning(i_category#12, 5), ENSURE_REQUIREMENTS, [id=#42] (31) HashAggregate [codegen id : 11] -Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] -Keys [1]: [i_category#28] -Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] -Aggregate Attributes [2]: [sum(ss_net_profit#36)#47, sum(ss_ext_sales_price#37)#48] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#47) / promote_precision(sum(ss_ext_sales_price#37)#48)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#49, i_category#28, null AS i_class#50, 0 AS t_category#51, 1 AS t_class#52, 1 AS lochierarchy#53] +Input [5]: [i_category#12, sum#38, isEmpty#39, sum#40, isEmpty#41] +Keys [1]: [i_category#12] +Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] +Aggregate Attributes [2]: [sum(ss_net_profit#32)#43, sum(ss_ext_sales_price#33)#44] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#43) / promote_precision(sum(ss_ext_sales_price#33)#44)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#45, i_category#12, null AS i_class#46, 0 AS t_category#47, 1 AS t_class#48, 1 AS lochierarchy#49] (32) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#54, i_class#55, sum#56, sum#57] +Output [4]: [i_category#12, i_class#11, sum#50, sum#51] (33) HashAggregate [codegen id : 16] -Input [4]: [i_category#54, i_class#55, sum#56, sum#57] -Keys [2]: [i_category#54, i_class#55] -Functions [2]: [sum(UnscaledValue(ss_net_profit#58)), sum(UnscaledValue(ss_ext_sales_price#59))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#58))#60, sum(UnscaledValue(ss_ext_sales_price#59))#61] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#58))#60,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#59))#61,17,2) AS ss_ext_sales_price#37] +Input [4]: [i_category#12, i_class#11, sum#50, sum#51] +Keys [2]: [i_category#12, i_class#11] +Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#52, sum(UnscaledValue(ss_ext_sales_price#3))#53] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#52,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#53,17,2) AS ss_ext_sales_price#33] (34) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#36, ss_ext_sales_price#37] +Input [2]: [ss_net_profit#32, ss_ext_sales_price#33] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] -Aggregate Attributes [4]: [sum#62, isEmpty#63, sum#64, isEmpty#65] -Results [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] +Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] +Aggregate Attributes [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] +Results [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] (35) Exchange -Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#70] +Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] (36) HashAggregate [codegen id : 17] -Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] +Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] Keys: [] -Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] -Aggregate Attributes [2]: [sum(ss_net_profit#36)#71, sum(ss_ext_sales_price#37)#72] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#71) / promote_precision(sum(ss_ext_sales_price#37)#72)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#73, null AS i_category#74, null AS i_class#75, 1 AS t_category#76, 1 AS t_class#77, 2 AS lochierarchy#78] +Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] +Aggregate Attributes [2]: [sum(ss_net_profit#32)#63, sum(ss_ext_sales_price#33)#64] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#63) / promote_precision(sum(ss_ext_sales_price#33)#64)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#65, null AS i_category#66, null AS i_class#67, 1 AS t_category#68, 1 AS t_class#69, 2 AS lochierarchy#70] (37) Union @@ -235,34 +235,34 @@ Results [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class (39) Exchange Input [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27] -Arguments: hashpartitioning(gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#79] +Arguments: hashpartitioning(gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#71] (40) HashAggregate [codegen id : 19] Input [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27] Keys [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27] Functions: [] Aggregate Attributes: [] -Results [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#12 END AS _w0#80] +Results [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#12 END AS _w0#72] (41) Exchange -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80] -Arguments: hashpartitioning(lochierarchy#27, _w0#80, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72] +Arguments: hashpartitioning(lochierarchy#27, _w0#72, 5), ENSURE_REQUIREMENTS, [id=#73] (42) Sort [codegen id : 20] -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80] -Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#80 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72] +Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 (43) Window -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80] -Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#80, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#82], [lochierarchy#27, _w0#80], [gross_margin#24 ASC NULLS FIRST] +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72] +Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#72, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#74], [lochierarchy#27, _w0#72], [gross_margin#24 ASC NULLS FIRST] (44) Project [codegen id : 21] -Output [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#82] -Input [6]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80, rank_within_parent#82] +Output [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#74] +Input [6]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72, rank_within_parent#74] (45) TakeOrderedAndProject -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#82] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#82 ASC NULLS FIRST], [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#82] +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#74] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#74 ASC NULLS FIRST], [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#74] ===== Subqueries ===== 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 20d80b4503755..a0737e88bd760 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 @@ -213,11 +213,7 @@ Arguments: [rank(d_year#7, d_moy#8) windowspecdefinition(i_category#17, i_brand# (35) Filter [codegen id : 12] Input [9]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28] -<<<<<<< HEAD Condition : ((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#26)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#26 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) -======= -Condition : (((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#26)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#26 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#23 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#28)) ->>>>>>> regen (36) Exchange Input [9]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28] @@ -233,23 +229,22 @@ Output [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_ye (39) HashAggregate [codegen id : 21] Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum#36] Keys [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35] -Functions [1]: [sum(UnscaledValue(ss_sales_price#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#37))#38] -Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#37))#38,17,2) AS sum_sales#39] +Functions [1]: [sum(UnscaledValue(ss_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#3))#37] +Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#3))#37,17,2) AS sum_sales#38] (40) Exchange -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#39] (41) Sort [codegen id : 22] -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST], false, 0 (42) Window -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] -<<<<<<< HEAD (43) Project [codegen id : 23] Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38, rn#40] @@ -261,30 +256,12 @@ Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_compan (45) Sort [codegen id : 24] Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#40 + 1) ASC NULLS FIRST], false, 0 -======= -(43) Filter [codegen id : 23] -Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] -Condition : isnotnull(rn#41) - -(44) Project [codegen id : 23] -Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] -Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] - -(45) Exchange -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1), 5), ENSURE_REQUIREMENTS, [id=#42] - -(46) Sort [codegen id : 24] -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#41 + 1) ASC NULLS FIRST], false, 0 ->>>>>>> regen (46) SortMergeJoin [codegen id : 25] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1)] +Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1)] Join condition: None -<<<<<<< HEAD (47) Project [codegen id : 25] Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38] Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] @@ -311,45 +288,12 @@ Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_compan (53) Sort [codegen id : 36] Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (rn#49 - 1) ASC NULLS FIRST], false, 0 -======= -(48) Project [codegen id : 25] -Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39] -Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] - -(49) ReusedExchange [Reuses operator id: 40] -Output [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] - -(50) Sort [codegen id : 34] -Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] -Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST], false, 0 - -(51) Window -Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] -Arguments: [rank(d_year#47, d_moy#48) windowspecdefinition(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#50], [i_category#43, i_brand#44, s_store_name#45, s_company_name#46], [d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST] - -(52) Filter [codegen id : 35] -Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] -Condition : isnotnull(rn#50) - -(53) Project [codegen id : 35] -Output [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] -Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] - -(54) Exchange -Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] -Arguments: hashpartitioning(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1), 5), ENSURE_REQUIREMENTS, [id=#51] - -(55) Sort [codegen id : 36] -Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] -Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, (rn#50 - 1) ASC NULLS FIRST], false, 0 ->>>>>>> regen (54) SortMergeJoin [codegen id : 37] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1)] +Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1)] Join condition: None -<<<<<<< HEAD (55) Project [codegen id : 37] Output [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#38 AS psum#51, sum_sales#48 AS nsum#52] Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] @@ -357,15 +301,6 @@ Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_ye (56) TakeOrderedAndProject Input [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] -======= -(57) Project [codegen id : 37] -Output [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#39 AS psum#52, sum_sales#49 AS nsum#53] -Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39, i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] - -(58) TakeOrderedAndProject -Input [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] ->>>>>>> regen ===== Subqueries ===== 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 9017cd21a3873..b0a231fc01379 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 @@ -194,11 +194,7 @@ Arguments: [rank(d_year#11, d_moy#12) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [9]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27] -<<<<<<< HEAD Condition : ((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#25)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) -======= -Condition : (((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#25)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#27)) ->>>>>>> regen (33) ReusedExchange [Reuses operator id: unknown] Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum#34] @@ -206,23 +202,22 @@ Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_ye (34) HashAggregate [codegen id : 13] 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] +Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#35] +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#6))#35,17,2) AS sum_sales#36] (35) 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#36] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#37] (36) Sort [codegen id : 14] -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#36] 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 (37) 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#36] +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#38], [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] -<<<<<<< HEAD (38) Project [codegen id : 15] Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] @@ -230,26 +225,12 @@ Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_yea (39) BroadcastExchange Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] 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=#39] -======= -(38) Filter [codegen id : 15] -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] -Condition : isnotnull(rn#39) - -(39) Project [codegen id : 15] -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] - -(40) BroadcastExchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] + 1)),false), [id=#40] ->>>>>>> regen (40) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -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#38 + 1)] Join condition: None -<<<<<<< HEAD (41) Project [codegen id : 23] Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] @@ -272,41 +253,12 @@ Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_yea (46) BroadcastExchange Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] 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=#48] -======= -(42) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] -Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] - -(43) ReusedExchange [Reuses operator id: 35] -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] - -(44) Sort [codegen id : 21] -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 - -(45) 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] - -(46) Filter [codegen id : 22] -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] -Condition : isnotnull(rn#48) - -(47) Project [codegen id : 22] -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] - -(48) BroadcastExchange -Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, true] - 1)),false), [id=#49] ->>>>>>> regen (47) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -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#47 - 1)] Join condition: None -<<<<<<< HEAD (48) Project [codegen id : 23] Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#49, sum_sales#46 AS nsum#50] Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] @@ -314,15 +266,6 @@ Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year (49) TakeOrderedAndProject Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, 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#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#22, psum#49, nsum#50] -======= -(50) Project [codegen id : 23] -Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] -Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] - -(51) TakeOrderedAndProject -Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, 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#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#22, psum#50, nsum#51] ->>>>>>> regen ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt index a1c7e4bfb35dc..57e57951851aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt @@ -250,225 +250,225 @@ Output [5]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_net_profit#47, cs_sold_date_sk#48] (36) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#49] +Output [1]: [d_date_sk#8] (37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_sold_date_sk#48] -Right keys [1]: [d_date_sk#49] +Right keys [1]: [d_date_sk#8] Join condition: None (38) Project [codegen id : 12] Output [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#49] +Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#8] (39) Exchange Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#50] +Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#49] (40) Sort [codegen id : 13] Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] Arguments: [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST], false, 0 (41) Scan parquet default.catalog_returns -Output [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Output [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (42) ColumnarToRow [codegen id : 14] -Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] (43) Filter [codegen id : 14] -Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] -Condition : (((isnotnull(cr_return_amount#54) AND (cr_return_amount#54 > 10000.00)) AND isnotnull(cr_order_number#52)) AND isnotnull(cr_item_sk#51)) +Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Condition : (((isnotnull(cr_return_amount#53) AND (cr_return_amount#53 > 10000.00)) AND isnotnull(cr_order_number#51)) AND isnotnull(cr_item_sk#50)) (44) Project [codegen id : 14] -Output [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] -Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Output [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] (45) Exchange -Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] -Arguments: hashpartitioning(cr_order_number#52, cr_item_sk#51, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#55] (46) Sort [codegen id : 15] -Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] -Arguments: [cr_order_number#52 ASC NULLS FIRST, cr_item_sk#51 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Arguments: [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 16] Left keys [2]: [cs_order_number#44, cs_item_sk#43] -Right keys [2]: [cr_order_number#52, cr_item_sk#51] +Right keys [2]: [cr_order_number#51, cr_item_sk#50] Join condition: None (48) Project [codegen id : 16] -Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] -Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] +Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] (49) HashAggregate [codegen id : 16] -Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] +Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] Keys [1]: [cs_item_sk#43] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#53, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] -Results [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#52, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#56, sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] +Results [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] (50) Exchange -Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#68] (51) HashAggregate [codegen id : 17] -Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] Keys [1]: [cs_item_sk#43] -Functions [4]: [sum(coalesce(cr_return_quantity#53, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#53, 0))#70, sum(coalesce(cs_quantity#45, 0))#71, sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73] -Results [3]: [cs_item_sk#43 AS item#74, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#53, 0))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#71 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#75, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#76] +Functions [4]: [sum(coalesce(cr_return_quantity#52, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#52, 0))#69, sum(coalesce(cs_quantity#45, 0))#70, sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72] +Results [3]: [cs_item_sk#43 AS item#73, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#52, 0))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#70 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#74, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#75] (52) Exchange -Input [3]: [item#74, return_ratio#75, currency_ratio#76] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#77] +Input [3]: [item#73, return_ratio#74, currency_ratio#75] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] (53) Sort [codegen id : 18] -Input [3]: [item#74, return_ratio#75, currency_ratio#76] -Arguments: [return_ratio#75 ASC NULLS FIRST], false, 0 +Input [3]: [item#73, return_ratio#74, currency_ratio#75] +Arguments: [return_ratio#74 ASC NULLS FIRST], false, 0 (54) Window -Input [3]: [item#74, return_ratio#75, currency_ratio#76] -Arguments: [rank(return_ratio#75) windowspecdefinition(return_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#78], [return_ratio#75 ASC NULLS FIRST] +Input [3]: [item#73, return_ratio#74, currency_ratio#75] +Arguments: [rank(return_ratio#74) windowspecdefinition(return_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#77], [return_ratio#74 ASC NULLS FIRST] (55) Sort [codegen id : 19] -Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] -Arguments: [currency_ratio#76 ASC NULLS FIRST], false, 0 +Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] +Arguments: [currency_ratio#75 ASC NULLS FIRST], false, 0 (56) Window -Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] -Arguments: [rank(currency_ratio#76) windowspecdefinition(currency_ratio#76 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#79], [currency_ratio#76 ASC NULLS FIRST] +Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] +Arguments: [rank(currency_ratio#75) windowspecdefinition(currency_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#78], [currency_ratio#75 ASC NULLS FIRST] (57) Filter [codegen id : 20] -Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] -Condition : ((return_rank#78 <= 10) OR (currency_rank#79 <= 10)) +Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] +Condition : ((return_rank#77 <= 10) OR (currency_rank#78 <= 10)) (58) Project [codegen id : 20] -Output [5]: [catalog AS channel#80, item#74, return_ratio#75, return_rank#78, currency_rank#79] -Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] +Output [5]: [catalog AS channel#79, item#73, return_ratio#74, return_rank#77, currency_rank#78] +Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] (59) Scan parquet default.store_sales -Output [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Output [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_sold_date_sk#85 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 22] -Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] (61) Filter [codegen id : 22] -Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] -Condition : (((((((isnotnull(ss_net_profit#85) AND isnotnull(ss_net_paid#84)) AND isnotnull(ss_quantity#83)) AND (ss_net_profit#85 > 1.00)) AND (ss_net_paid#84 > 0.00)) AND (ss_quantity#83 > 0)) AND isnotnull(ss_ticket_number#82)) AND isnotnull(ss_item_sk#81)) +Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Condition : (((((((isnotnull(ss_net_profit#84) AND isnotnull(ss_net_paid#83)) AND isnotnull(ss_quantity#82)) AND (ss_net_profit#84 > 1.00)) AND (ss_net_paid#83 > 0.00)) AND (ss_quantity#82 > 0)) AND isnotnull(ss_ticket_number#81)) AND isnotnull(ss_item_sk#80)) (62) Project [codegen id : 22] -Output [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86] -Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Output [5]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85] +Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] (63) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#87] +Output [1]: [d_date_sk#8] (64) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ss_sold_date_sk#86] -Right keys [1]: [d_date_sk#87] +Left keys [1]: [ss_sold_date_sk#85] +Right keys [1]: [d_date_sk#8] Join condition: None (65) Project [codegen id : 22] -Output [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] -Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86, d_date_sk#87] +Output [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] +Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85, d_date_sk#8] (66) Exchange -Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] -Arguments: hashpartitioning(cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint), 5), ENSURE_REQUIREMENTS, [id=#88] +Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] +Arguments: hashpartitioning(cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint), 5), ENSURE_REQUIREMENTS, [id=#86] (67) Sort [codegen id : 23] -Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] -Arguments: [cast(ss_ticket_number#82 as bigint) ASC NULLS FIRST, cast(ss_item_sk#81 as bigint) ASC NULLS FIRST], false, 0 +Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] +Arguments: [cast(ss_ticket_number#81 as bigint) ASC NULLS FIRST, cast(ss_item_sk#80 as bigint) ASC NULLS FIRST], false, 0 (68) Scan parquet default.store_returns -Output [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Output [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 24] -Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] (70) Filter [codegen id : 24] -Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] -Condition : (((isnotnull(sr_return_amt#92) AND (sr_return_amt#92 > 10000.00)) AND isnotnull(sr_ticket_number#90)) AND isnotnull(sr_item_sk#89)) +Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Condition : (((isnotnull(sr_return_amt#90) AND (sr_return_amt#90 > 10000.00)) AND isnotnull(sr_ticket_number#88)) AND isnotnull(sr_item_sk#87)) (71) Project [codegen id : 24] -Output [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] -Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Output [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] (72) Exchange -Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] -Arguments: hashpartitioning(sr_ticket_number#90, sr_item_sk#89, 5), ENSURE_REQUIREMENTS, [id=#94] +Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Arguments: hashpartitioning(sr_ticket_number#88, sr_item_sk#87, 5), ENSURE_REQUIREMENTS, [id=#92] (73) Sort [codegen id : 25] -Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] -Arguments: [sr_ticket_number#90 ASC NULLS FIRST, sr_item_sk#89 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Arguments: [sr_ticket_number#88 ASC NULLS FIRST, sr_item_sk#87 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 26] -Left keys [2]: [cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint)] -Right keys [2]: [sr_ticket_number#90, sr_item_sk#89] +Left keys [2]: [cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint)] +Right keys [2]: [sr_ticket_number#88, sr_item_sk#87] Join condition: None (75) Project [codegen id : 26] -Output [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] -Input [8]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Output [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] +Input [8]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] (76) HashAggregate [codegen id : 26] -Input [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] -Keys [1]: [ss_item_sk#81] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#91, 0)), partial_sum(coalesce(ss_quantity#83, 0)), partial_sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#95, sum#96, sum#97, isEmpty#98, sum#99, isEmpty#100] -Results [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] +Input [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] +Keys [1]: [ss_item_sk#80] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#89, 0)), partial_sum(coalesce(ss_quantity#82, 0)), partial_sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#93, sum#94, sum#95, isEmpty#96, sum#97, isEmpty#98] +Results [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] (77) Exchange -Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] -Arguments: hashpartitioning(ss_item_sk#81, 5), ENSURE_REQUIREMENTS, [id=#107] +Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Arguments: hashpartitioning(ss_item_sk#80, 5), ENSURE_REQUIREMENTS, [id=#105] (78) HashAggregate [codegen id : 27] -Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] -Keys [1]: [ss_item_sk#81] -Functions [4]: [sum(coalesce(sr_return_quantity#91, 0)), sum(coalesce(ss_quantity#83, 0)), sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#91, 0))#108, sum(coalesce(ss_quantity#83, 0))#109, sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110, sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111] -Results [3]: [ss_item_sk#81 AS item#112, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#91, 0))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#83, 0))#109 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#113, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#114] +Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Keys [1]: [ss_item_sk#80] +Functions [4]: [sum(coalesce(sr_return_quantity#89, 0)), sum(coalesce(ss_quantity#82, 0)), sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#89, 0))#106, sum(coalesce(ss_quantity#82, 0))#107, sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108, sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109] +Results [3]: [ss_item_sk#80 AS item#110, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#89, 0))#106 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#82, 0))#107 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#111, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#112] (79) Exchange -Input [3]: [item#112, return_ratio#113, currency_ratio#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] +Input [3]: [item#110, return_ratio#111, currency_ratio#112] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] (80) Sort [codegen id : 28] -Input [3]: [item#112, return_ratio#113, currency_ratio#114] -Arguments: [return_ratio#113 ASC NULLS FIRST], false, 0 +Input [3]: [item#110, return_ratio#111, currency_ratio#112] +Arguments: [return_ratio#111 ASC NULLS FIRST], false, 0 (81) Window -Input [3]: [item#112, return_ratio#113, currency_ratio#114] -Arguments: [rank(return_ratio#113) windowspecdefinition(return_ratio#113 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#116], [return_ratio#113 ASC NULLS FIRST] +Input [3]: [item#110, return_ratio#111, currency_ratio#112] +Arguments: [rank(return_ratio#111) windowspecdefinition(return_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#114], [return_ratio#111 ASC NULLS FIRST] (82) Sort [codegen id : 29] -Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] -Arguments: [currency_ratio#114 ASC NULLS FIRST], false, 0 +Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] +Arguments: [currency_ratio#112 ASC NULLS FIRST], false, 0 (83) Window -Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] -Arguments: [rank(currency_ratio#114) windowspecdefinition(currency_ratio#114 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#117], [currency_ratio#114 ASC NULLS FIRST] +Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] +Arguments: [rank(currency_ratio#112) windowspecdefinition(currency_ratio#112 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#115], [currency_ratio#112 ASC NULLS FIRST] (84) Filter [codegen id : 30] -Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] -Condition : ((return_rank#116 <= 10) OR (currency_rank#117 <= 10)) +Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] +Condition : ((return_rank#114 <= 10) OR (currency_rank#115 <= 10)) (85) Project [codegen id : 30] -Output [5]: [store AS channel#118, item#112, return_ratio#113, return_rank#116, currency_rank#117] -Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] +Output [5]: [store AS channel#116, item#110, return_ratio#111, return_rank#114, currency_rank#115] +Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] (86) Union @@ -481,7 +481,7 @@ Results [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_ran (88) Exchange Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] -Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#119] +Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#117] (89) HashAggregate [codegen id : 32] Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] @@ -505,6 +505,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#85 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index 74b789e4a7a65..ada8c0415c672 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -260,170 +260,170 @@ Output [6]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, Input [9]: [cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_item_sk#49, cr_order_number#50, cr_return_quantity#51, cr_return_amount#52] (40) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#54] +Output [1]: [d_date_sk#14] (41) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#47] -Right keys [1]: [d_date_sk#54] +Right keys [1]: [d_date_sk#14] Join condition: None (42) Project [codegen id : 10] Output [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] -Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#54] +Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#14] (43) HashAggregate [codegen id : 10] Input [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] Keys [1]: [cs_item_sk#42] Functions [4]: [partial_sum(coalesce(cr_return_quantity#51, 0)), partial_sum(coalesce(cs_quantity#44, 0)), partial_sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Results [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Aggregate Attributes [6]: [sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] +Results [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] (44) Exchange -Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#67] +Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] +Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#66] (45) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] Keys [1]: [cs_item_sk#42] Functions [4]: [sum(coalesce(cr_return_quantity#51, 0)), sum(coalesce(cs_quantity#44, 0)), sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#68, sum(coalesce(cs_quantity#44, 0))#69, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71] -Results [3]: [cs_item_sk#42 AS item#72, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#68 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#69 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#73, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#74] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#67, sum(coalesce(cs_quantity#44, 0))#68, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70] +Results [3]: [cs_item_sk#42 AS item#71, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#67 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#68 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#72, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#73] (46) Exchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#75] +Input [3]: [item#71, return_ratio#72, currency_ratio#73] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#74] (47) Sort [codegen id : 12] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [return_ratio#73 ASC NULLS FIRST], false, 0 +Input [3]: [item#71, return_ratio#72, currency_ratio#73] +Arguments: [return_ratio#72 ASC NULLS FIRST], false, 0 (48) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#76], [return_ratio#73 ASC NULLS FIRST] +Input [3]: [item#71, return_ratio#72, currency_ratio#73] +Arguments: [rank(return_ratio#72) windowspecdefinition(return_ratio#72 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#72 ASC NULLS FIRST] (49) Sort [codegen id : 13] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 +Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] +Arguments: [currency_ratio#73 ASC NULLS FIRST], false, 0 (50) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#77], [currency_ratio#74 ASC NULLS FIRST] +Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] +Arguments: [rank(currency_ratio#73) windowspecdefinition(currency_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#73 ASC NULLS FIRST] (51) Filter [codegen id : 14] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] -Condition : ((return_rank#76 <= 10) OR (currency_rank#77 <= 10)) +Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] +Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) (52) Project [codegen id : 14] -Output [5]: [catalog AS channel#78, item#72, return_ratio#73, return_rank#76, currency_rank#77] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] +Output [5]: [catalog AS channel#77, item#71, return_ratio#72, return_rank#75, currency_rank#76] +Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] (53) Scan parquet default.store_sales -Output [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Output [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#84), dynamicpruningexpression(ss_sold_date_sk#84 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#83), dynamicpruningexpression(ss_sold_date_sk#83 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 15] -Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] (55) Filter [codegen id : 15] -Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] -Condition : (((((((isnotnull(ss_net_profit#83) AND isnotnull(ss_net_paid#82)) AND isnotnull(ss_quantity#81)) AND (ss_net_profit#83 > 1.00)) AND (ss_net_paid#82 > 0.00)) AND (ss_quantity#81 > 0)) AND isnotnull(ss_ticket_number#80)) AND isnotnull(ss_item_sk#79)) +Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Condition : (((((((isnotnull(ss_net_profit#82) AND isnotnull(ss_net_paid#81)) AND isnotnull(ss_quantity#80)) AND (ss_net_profit#82 > 1.00)) AND (ss_net_paid#81 > 0.00)) AND (ss_quantity#80 > 0)) AND isnotnull(ss_ticket_number#79)) AND isnotnull(ss_item_sk#78)) (56) Project [codegen id : 15] -Output [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] -Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Output [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] +Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] (57) BroadcastExchange -Input [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#85] +Input [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#84] (58) Scan parquet default.store_returns -Output [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Output [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (59) ColumnarToRow -Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] (60) Filter -Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] -Condition : (((isnotnull(sr_return_amt#89) AND (sr_return_amt#89 > 10000.00)) AND isnotnull(sr_ticket_number#87)) AND isnotnull(sr_item_sk#86)) +Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Condition : (((isnotnull(sr_return_amt#88) AND (sr_return_amt#88 > 10000.00)) AND isnotnull(sr_ticket_number#86)) AND isnotnull(sr_item_sk#85)) (61) Project -Output [4]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] -Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Output [4]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] +Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] (62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [cast(ss_ticket_number#80 as bigint), cast(ss_item_sk#79 as bigint)] -Right keys [2]: [sr_ticket_number#87, sr_item_sk#86] +Left keys [2]: [cast(ss_ticket_number#79 as bigint), cast(ss_item_sk#78 as bigint)] +Right keys [2]: [sr_ticket_number#86, sr_item_sk#85] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89] -Input [9]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] +Output [6]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88] +Input [9]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] (64) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#91] +Output [1]: [d_date_sk#14] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#84] -Right keys [1]: [d_date_sk#91] +Left keys [1]: [ss_sold_date_sk#83] +Right keys [1]: [d_date_sk#14] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] -Input [7]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89, d_date_sk#91] +Output [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] +Input [7]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88, d_date_sk#14] (67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] -Keys [1]: [ss_item_sk#79] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#88, 0)), partial_sum(coalesce(ss_quantity#81, 0)), partial_sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Results [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Input [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] +Keys [1]: [ss_item_sk#78] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#87, 0)), partial_sum(coalesce(ss_quantity#80, 0)), partial_sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Results [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] (68) Exchange -Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] -Arguments: hashpartitioning(ss_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#104] +Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Arguments: hashpartitioning(ss_item_sk#78, 5), ENSURE_REQUIREMENTS, [id=#102] (69) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] -Keys [1]: [ss_item_sk#79] -Functions [4]: [sum(coalesce(sr_return_quantity#88, 0)), sum(coalesce(ss_quantity#81, 0)), sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#88, 0))#105, sum(coalesce(ss_quantity#81, 0))#106, sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107, sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108] -Results [3]: [ss_item_sk#79 AS item#109, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#88, 0))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#81, 0))#106 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#110, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#111] +Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Keys [1]: [ss_item_sk#78] +Functions [4]: [sum(coalesce(sr_return_quantity#87, 0)), sum(coalesce(ss_quantity#80, 0)), sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#87, 0))#103, sum(coalesce(ss_quantity#80, 0))#104, sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105, sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106] +Results [3]: [ss_item_sk#78 AS item#107, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#87, 0))#103 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#80, 0))#104 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#108, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#109] (70) Exchange -Input [3]: [item#109, return_ratio#110, currency_ratio#111] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#112] +Input [3]: [item#107, return_ratio#108, currency_ratio#109] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] (71) Sort [codegen id : 19] -Input [3]: [item#109, return_ratio#110, currency_ratio#111] -Arguments: [return_ratio#110 ASC NULLS FIRST], false, 0 +Input [3]: [item#107, return_ratio#108, currency_ratio#109] +Arguments: [return_ratio#108 ASC NULLS FIRST], false, 0 (72) Window -Input [3]: [item#109, return_ratio#110, currency_ratio#111] -Arguments: [rank(return_ratio#110) windowspecdefinition(return_ratio#110 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#113], [return_ratio#110 ASC NULLS FIRST] +Input [3]: [item#107, return_ratio#108, currency_ratio#109] +Arguments: [rank(return_ratio#108) windowspecdefinition(return_ratio#108 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#111], [return_ratio#108 ASC NULLS FIRST] (73) Sort [codegen id : 20] -Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] -Arguments: [currency_ratio#111 ASC NULLS FIRST], false, 0 +Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] +Arguments: [currency_ratio#109 ASC NULLS FIRST], false, 0 (74) Window -Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] -Arguments: [rank(currency_ratio#111) windowspecdefinition(currency_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#114], [currency_ratio#111 ASC NULLS FIRST] +Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] +Arguments: [rank(currency_ratio#109) windowspecdefinition(currency_ratio#109 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#112], [currency_ratio#109 ASC NULLS FIRST] (75) Filter [codegen id : 21] -Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] -Condition : ((return_rank#113 <= 10) OR (currency_rank#114 <= 10)) +Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] +Condition : ((return_rank#111 <= 10) OR (currency_rank#112 <= 10)) (76) Project [codegen id : 21] -Output [5]: [store AS channel#115, item#109, return_ratio#110, return_rank#113, currency_rank#114] -Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] +Output [5]: [store AS channel#113, item#107, return_ratio#108, return_rank#111, currency_rank#112] +Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] (77) Union @@ -436,7 +436,7 @@ Results [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_ran (79) Exchange Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] -Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#116] +Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#114] (80) HashAggregate [codegen id : 23] Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] @@ -460,6 +460,6 @@ Output [1]: [d_date_sk#14] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#84 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#83 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt index 957e2d9a015b4..bcf47f68f2e59 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 @@ -163,7 +163,6 @@ Arguments: hashpartitioning(item_sk#13, 5), ENSURE_REQUIREMENTS, [id=#17] Input [4]: [item_sk#13, d_date#6, sumws#14, rk#16] Arguments: [item_sk#13 ASC NULLS FIRST], false, 0 -<<<<<<< HEAD (20) ReusedExchange [Reuses operator id: unknown] Output [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] @@ -185,64 +184,31 @@ Arguments: hashpartitioning(item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#22] (25) Sort [codegen id : 12] Input [3]: [item_sk#18, sumws#20, rk#21] -======= -(21) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] - -(22) Sort [codegen id : 10] -Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] -Arguments: [ws_item_sk#21 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST], false, 0 - -(23) Window -Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] -Arguments: [row_number() windowspecdefinition(ws_item_sk#21, d_date#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#22], [ws_item_sk#21], [d_date#19 ASC NULLS FIRST] - -(24) Filter [codegen id : 11] -Input [5]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21, rk#22] -Condition : isnotnull(rk#22) - -(25) Project [codegen id : 11] -Output [3]: [item_sk#18, sumws#20, rk#22] -Input [5]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21, rk#22] - -(26) Exchange -Input [3]: [item_sk#18, sumws#20, rk#22] -Arguments: hashpartitioning(item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#23] - -(27) Sort [codegen id : 12] -Input [3]: [item_sk#18, sumws#20, rk#22] ->>>>>>> regen Arguments: [item_sk#18 ASC NULLS FIRST], false, 0 (26) SortMergeJoin [codegen id : 13] Left keys [1]: [item_sk#13] Right keys [1]: [item_sk#18] -Join condition: (rk#16 >= rk#22) +Join condition: (rk#16 >= rk#21) (27) Project [codegen id : 13] Output [4]: [item_sk#13, d_date#6, sumws#14, sumws#20] -Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#18, sumws#20, rk#22] +Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#18, sumws#20, rk#21] (28) HashAggregate [codegen id : 13] Input [4]: [item_sk#13, d_date#6, sumws#14, sumws#20] Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [partial_sum(sumws#20)] -Aggregate Attributes [2]: [sum#24, isEmpty#25] -Results [5]: [item_sk#13, d_date#6, sumws#14, sum#26, isEmpty#27] +Aggregate Attributes [2]: [sum#23, isEmpty#24] +Results [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] -<<<<<<< HEAD (29) HashAggregate [codegen id : 13] Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] -======= -(31) HashAggregate [codegen id : 13] -Input [5]: [item_sk#13, d_date#6, sumws#14, sum#26, isEmpty#27] ->>>>>>> regen Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [sum(sumws#20)] -Aggregate Attributes [1]: [sum(sumws#20)#28] -Results [3]: [item_sk#13, d_date#6, sum(sumws#20)#28 AS cume_sales#29] +Aggregate Attributes [1]: [sum(sumws#20)#27] +Results [3]: [item_sk#13, d_date#6, sum(sumws#20)#27 AS cume_sales#28] -<<<<<<< HEAD (30) Exchange Input [3]: [item_sk#13, d_date#6, cume_sales#28] Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#29] @@ -253,25 +219,12 @@ Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 (32) Scan parquet default.store_sales Output [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] -======= -(32) Exchange -Input [3]: [item_sk#13, d_date#6, cume_sales#29] -Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#30] - -(33) Sort [codegen id : 14] -Input [3]: [item_sk#13, d_date#6, cume_sales#29] -Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 - -(34) Scan parquet default.store_sales -Output [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] ->>>>>>> regen Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#33), dynamicpruningexpression(ss_sold_date_sk#33 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -<<<<<<< HEAD (33) ColumnarToRow [codegen id : 16] Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] @@ -386,137 +339,12 @@ Arguments: hashpartitioning(item_sk#39, d_date#34, 5), ENSURE_REQUIREMENTS, [id= (58) Sort [codegen id : 28] Input [3]: [item_sk#39, d_date#34, cume_sales#54] Arguments: [item_sk#39 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 -======= -(35) ColumnarToRow [codegen id : 16] -Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] - -(36) Filter [codegen id : 16] -Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] -Condition : isnotnull(ss_item_sk#31) - -(37) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#34, d_date#35] - -(38) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ss_sold_date_sk#33] -Right keys [1]: [d_date_sk#34] -Join condition: None - -(39) Project [codegen id : 16] -Output [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] -Input [5]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33, d_date_sk#34, d_date#35] - -(40) HashAggregate [codegen id : 16] -Input [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] -Keys [2]: [ss_item_sk#31, d_date#35] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#32))] -Aggregate Attributes [1]: [sum#36] -Results [3]: [ss_item_sk#31, d_date#35, sum#37] - -(41) Exchange -Input [3]: [ss_item_sk#31, d_date#35, sum#37] -Arguments: hashpartitioning(ss_item_sk#31, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#38] - -(42) HashAggregate [codegen id : 17] -Input [3]: [ss_item_sk#31, d_date#35, sum#37] -Keys [2]: [ss_item_sk#31, d_date#35] -Functions [1]: [sum(UnscaledValue(ss_sales_price#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#32))#39] -Results [4]: [ss_item_sk#31 AS item_sk#40, d_date#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#32))#39,17,2) AS sumss#41, ss_item_sk#31] - -(43) Exchange -Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] -Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#42] - -(44) Sort [codegen id : 18] -Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 - -(45) Window -Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] -Arguments: [row_number() windowspecdefinition(ss_item_sk#31, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [ss_item_sk#31], [d_date#35 ASC NULLS FIRST] - -(46) Filter [codegen id : 19] -Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] -Condition : isnotnull(rk#43) - -(47) Project [codegen id : 19] -Output [4]: [item_sk#40, d_date#35, sumss#41, rk#43] -Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] - -(48) Exchange -Input [4]: [item_sk#40, d_date#35, sumss#41, rk#43] -Arguments: hashpartitioning(item_sk#40, 5), ENSURE_REQUIREMENTS, [id=#44] - -(49) Sort [codegen id : 20] -Input [4]: [item_sk#40, d_date#35, sumss#41, rk#43] -Arguments: [item_sk#40 ASC NULLS FIRST], false, 0 - -(50) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] - -(51) Sort [codegen id : 24] -Input [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] -Arguments: [ss_item_sk#48 ASC NULLS FIRST, d_date#46 ASC NULLS FIRST], false, 0 - -(52) Window -Input [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] -Arguments: [row_number() windowspecdefinition(ss_item_sk#48, d_date#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [ss_item_sk#48], [d_date#46 ASC NULLS FIRST] - -(53) Filter [codegen id : 25] -Input [5]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48, rk#49] -Condition : isnotnull(rk#49) - -(54) Project [codegen id : 25] -Output [3]: [item_sk#45, sumss#47, rk#49] -Input [5]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48, rk#49] - -(55) Exchange -Input [3]: [item_sk#45, sumss#47, rk#49] -Arguments: hashpartitioning(item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#50] - -(56) Sort [codegen id : 26] -Input [3]: [item_sk#45, sumss#47, rk#49] -Arguments: [item_sk#45 ASC NULLS FIRST], false, 0 - -(57) SortMergeJoin [codegen id : 27] -Left keys [1]: [item_sk#40] -Right keys [1]: [item_sk#45] -Join condition: (rk#43 >= rk#49) - -(58) Project [codegen id : 27] -Output [4]: [item_sk#40, d_date#35, sumss#41, sumss#47] -Input [7]: [item_sk#40, d_date#35, sumss#41, rk#43, item_sk#45, sumss#47, rk#49] - -(59) HashAggregate [codegen id : 27] -Input [4]: [item_sk#40, d_date#35, sumss#41, sumss#47] -Keys [3]: [item_sk#40, d_date#35, sumss#41] -Functions [1]: [partial_sum(sumss#47)] -Aggregate Attributes [2]: [sum#51, isEmpty#52] -Results [5]: [item_sk#40, d_date#35, sumss#41, sum#53, isEmpty#54] - -(60) HashAggregate [codegen id : 27] -Input [5]: [item_sk#40, d_date#35, sumss#41, sum#53, isEmpty#54] -Keys [3]: [item_sk#40, d_date#35, sumss#41] -Functions [1]: [sum(sumss#47)] -Aggregate Attributes [1]: [sum(sumss#47)#55] -Results [3]: [item_sk#40, d_date#35, sum(sumss#47)#55 AS cume_sales#56] - -(61) Exchange -Input [3]: [item_sk#40, d_date#35, cume_sales#56] -Arguments: hashpartitioning(item_sk#40, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#57] - -(62) Sort [codegen id : 28] -Input [3]: [item_sk#40, d_date#35, cume_sales#56] -Arguments: [item_sk#40 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 ->>>>>>> regen (59) SortMergeJoin Left keys [2]: [item_sk#13, d_date#6] -Right keys [2]: [item_sk#40, d_date#35] +Right keys [2]: [item_sk#39, d_date#34] Join condition: None -<<<<<<< HEAD (60) Filter [codegen id : 29] Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END) @@ -582,81 +410,6 @@ Condition : ((isnotnull(web_cumulative#73) AND isnotnull(store_cumulative#74)) A (74) 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] -======= -(64) Filter [codegen id : 29] -Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END) - -(65) Project [codegen id : 29] -Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END AS item_sk#58, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#35 END AS d_date#59, cume_sales#29 AS web_sales#60, cume_sales#56 AS store_sales#61] -Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] - -(66) Exchange -Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Arguments: hashpartitioning(item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#62] - -(67) Sort [codegen id : 30] -Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Arguments: [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], false, 0 - -(68) Window -Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Arguments: [row_number() windowspecdefinition(item_sk#58, d_date#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#63], [item_sk#58], [d_date#59 ASC NULLS FIRST] - -(69) Filter [codegen id : 31] -Input [5]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63] -Condition : isnotnull(rk#63) - -(70) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] - -(71) Sort [codegen id : 61] -Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] -Arguments: [item_sk#64 ASC NULLS FIRST, d_date#65 ASC NULLS FIRST], false, 0 - -(72) Window -Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] -Arguments: [row_number() windowspecdefinition(item_sk#64, d_date#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#68], [item_sk#64], [d_date#65 ASC NULLS FIRST] - -(73) Filter [codegen id : 62] -Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] -Condition : isnotnull(rk#68) - -(74) Project [codegen id : 62] -Output [4]: [item_sk#64, web_sales#66, store_sales#67, rk#68] -Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] - -(75) SortMergeJoin [codegen id : 63] -Left keys [1]: [item_sk#58] -Right keys [1]: [item_sk#64] -Join condition: (rk#63 >= rk#68) - -(76) Project [codegen id : 63] -Output [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] -Input [9]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63, item_sk#64, web_sales#66, store_sales#67, rk#68] - -(77) HashAggregate [codegen id : 63] -Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] -Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Functions [2]: [partial_max(web_sales#66), partial_max(store_sales#67)] -Aggregate Attributes [2]: [max#69, max#70] -Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#71, max#72] - -(78) HashAggregate [codegen id : 63] -Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#71, max#72] -Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Functions [2]: [max(web_sales#66), max(store_sales#67)] -Aggregate Attributes [2]: [max(web_sales#66)#73, max(store_sales#67)#74] -Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max(web_sales#66)#73 AS web_cumulative#75, max(store_sales#67)#74 AS store_cumulative#76] - -(79) Filter [codegen id : 63] -Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] -Condition : ((isnotnull(web_cumulative#75) AND isnotnull(store_cumulative#76)) AND (web_cumulative#75 > store_cumulative#76)) - -(80) TakeOrderedAndProject -Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] -Arguments: 100, [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] ->>>>>>> regen ===== Subqueries ===== @@ -667,10 +420,6 @@ ReusedExchange (75) (75) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#5, d_date#6] -<<<<<<< HEAD Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#4 -======= -Subquery:2 Hosting operator id = 34 Hosting Expression = ss_sold_date_sk#33 IN dynamicpruning#4 ->>>>>>> regen 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 471fe83a95901..8aaaf48a8162c 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 @@ -152,7 +152,6 @@ Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS F Output [4]: [item_sk#13, d_date#6, sumws#14, rk#16] Input [5]: [item_sk#13, d_date#6, sumws#14, ws_item_sk#1, rk#16] -<<<<<<< HEAD (18) ReusedExchange [Reuses operator id: unknown] Output [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] @@ -171,68 +170,34 @@ Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1, rk#20] (22) BroadcastExchange Input [3]: [item_sk#17, sumws#19, rk#20] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] -======= -(19) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] - -(20) Sort [codegen id : 8] -Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] -Arguments: [ws_item_sk#20 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 - -(21) Window -Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] -Arguments: [row_number() windowspecdefinition(ws_item_sk#20, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#21], [ws_item_sk#20], [d_date#18 ASC NULLS FIRST] - -(22) Filter [codegen id : 9] -Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20, rk#21] -Condition : isnotnull(rk#21) - -(23) Project [codegen id : 9] -Output [3]: [item_sk#17, sumws#19, rk#21] -Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20, rk#21] - -(24) BroadcastExchange -Input [3]: [item_sk#17, sumws#19, rk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] ->>>>>>> regen (23) BroadcastHashJoin [codegen id : 10] Left keys [1]: [item_sk#13] Right keys [1]: [item_sk#17] -Join condition: (rk#16 >= rk#21) +Join condition: (rk#16 >= rk#20) (24) Project [codegen id : 10] Output [4]: [item_sk#13, d_date#6, sumws#14, sumws#19] -Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#17, sumws#19, rk#21] +Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#17, sumws#19, rk#20] (25) HashAggregate [codegen id : 10] Input [4]: [item_sk#13, d_date#6, sumws#14, sumws#19] Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [partial_sum(sumws#19)] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +Aggregate Attributes [2]: [sum#22, isEmpty#23] +Results [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] -<<<<<<< HEAD (26) Exchange Input [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] Arguments: hashpartitioning(item_sk#13, d_date#6, sumws#14, 5), ENSURE_REQUIREMENTS, [id=#26] (27) HashAggregate [codegen id : 11] Input [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] -======= -(28) Exchange -Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] -Arguments: hashpartitioning(item_sk#13, d_date#6, sumws#14, 5), ENSURE_REQUIREMENTS, [id=#27] - -(29) HashAggregate [codegen id : 11] -Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] ->>>>>>> regen Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [sum(sumws#19)] -Aggregate Attributes [1]: [sum(sumws#19)#28] -Results [3]: [item_sk#13, d_date#6, sum(sumws#19)#28 AS cume_sales#29] +Aggregate Attributes [1]: [sum(sumws#19)#27] +Results [3]: [item_sk#13, d_date#6, sum(sumws#19)#27 AS cume_sales#28] -<<<<<<< HEAD (28) Exchange Input [3]: [item_sk#13, d_date#6, cume_sales#28] Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#29] @@ -243,25 +208,12 @@ Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 (30) Scan parquet default.store_sales Output [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] -======= -(30) Exchange -Input [3]: [item_sk#13, d_date#6, cume_sales#29] -Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#30] - -(31) Sort [codegen id : 12] -Input [3]: [item_sk#13, d_date#6, cume_sales#29] -Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 - -(32) Scan parquet default.store_sales -Output [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] ->>>>>>> regen Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#33), dynamicpruningexpression(ss_sold_date_sk#33 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -<<<<<<< HEAD (31) ColumnarToRow [codegen id : 14] Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] @@ -368,129 +320,12 @@ Arguments: hashpartitioning(item_sk#39, d_date#34, 5), ENSURE_REQUIREMENTS, [id= (54) Sort [codegen id : 24] Input [3]: [item_sk#39, d_date#34, cume_sales#54] Arguments: [item_sk#39 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 -======= -(33) ColumnarToRow [codegen id : 14] -Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] - -(34) Filter [codegen id : 14] -Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] -Condition : isnotnull(ss_item_sk#31) - -(35) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#34, d_date#35] - -(36) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_sold_date_sk#33] -Right keys [1]: [d_date_sk#34] -Join condition: None - -(37) Project [codegen id : 14] -Output [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] -Input [5]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33, d_date_sk#34, d_date#35] - -(38) HashAggregate [codegen id : 14] -Input [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] -Keys [2]: [ss_item_sk#31, d_date#35] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#32))] -Aggregate Attributes [1]: [sum#36] -Results [3]: [ss_item_sk#31, d_date#35, sum#37] - -(39) Exchange -Input [3]: [ss_item_sk#31, d_date#35, sum#37] -Arguments: hashpartitioning(ss_item_sk#31, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#38] - -(40) HashAggregate [codegen id : 15] -Input [3]: [ss_item_sk#31, d_date#35, sum#37] -Keys [2]: [ss_item_sk#31, d_date#35] -Functions [1]: [sum(UnscaledValue(ss_sales_price#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#32))#39] -Results [4]: [ss_item_sk#31 AS item_sk#40, d_date#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#32))#39,17,2) AS sumss#41, ss_item_sk#31] - -(41) Exchange -Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] -Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#42] - -(42) Sort [codegen id : 16] -Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 - -(43) Window -Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] -Arguments: [row_number() windowspecdefinition(ss_item_sk#31, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [ss_item_sk#31], [d_date#35 ASC NULLS FIRST] - -(44) Filter [codegen id : 22] -Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] -Condition : isnotnull(rk#43) - -(45) Project [codegen id : 22] -Output [4]: [item_sk#40, d_date#35, sumss#41, rk#43] -Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] - -(46) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] - -(47) Sort [codegen id : 20] -Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], false, 0 - -(48) Window -Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] -Arguments: [row_number() windowspecdefinition(ss_item_sk#47, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [ss_item_sk#47], [d_date#45 ASC NULLS FIRST] - -(49) Filter [codegen id : 21] -Input [5]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47, rk#48] -Condition : isnotnull(rk#48) - -(50) Project [codegen id : 21] -Output [3]: [item_sk#44, sumss#46, rk#48] -Input [5]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47, rk#48] - -(51) BroadcastExchange -Input [3]: [item_sk#44, sumss#46, rk#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] - -(52) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [item_sk#40] -Right keys [1]: [item_sk#44] -Join condition: (rk#43 >= rk#48) - -(53) Project [codegen id : 22] -Output [4]: [item_sk#40, d_date#35, sumss#41, sumss#46] -Input [7]: [item_sk#40, d_date#35, sumss#41, rk#43, item_sk#44, sumss#46, rk#48] - -(54) HashAggregate [codegen id : 22] -Input [4]: [item_sk#40, d_date#35, sumss#41, sumss#46] -Keys [3]: [item_sk#40, d_date#35, sumss#41] -Functions [1]: [partial_sum(sumss#46)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] - -(55) Exchange -Input [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] -Arguments: hashpartitioning(item_sk#40, d_date#35, sumss#41, 5), ENSURE_REQUIREMENTS, [id=#54] - -(56) HashAggregate [codegen id : 23] -Input [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] -Keys [3]: [item_sk#40, d_date#35, sumss#41] -Functions [1]: [sum(sumss#46)] -Aggregate Attributes [1]: [sum(sumss#46)#55] -Results [3]: [item_sk#40, d_date#35, sum(sumss#46)#55 AS cume_sales#56] - -(57) Exchange -Input [3]: [item_sk#40, d_date#35, cume_sales#56] -Arguments: hashpartitioning(item_sk#40, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#57] - -(58) Sort [codegen id : 24] -Input [3]: [item_sk#40, d_date#35, cume_sales#56] -Arguments: [item_sk#40 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 ->>>>>>> regen (55) SortMergeJoin Left keys [2]: [item_sk#13, d_date#6] -Right keys [2]: [item_sk#40, d_date#35] +Right keys [2]: [item_sk#39, d_date#34] Join condition: None -<<<<<<< HEAD (56) Filter [codegen id : 25] Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END) @@ -560,85 +395,6 @@ Condition : ((isnotnull(web_cumulative#74) AND isnotnull(store_cumulative#75)) A (71) 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] -======= -(60) Filter [codegen id : 25] -Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END) - -(61) Project [codegen id : 25] -Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END AS item_sk#58, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#35 END AS d_date#59, cume_sales#29 AS web_sales#60, cume_sales#56 AS store_sales#61] -Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] - -(62) Exchange -Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Arguments: hashpartitioning(item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#62] - -(63) Sort [codegen id : 26] -Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Arguments: [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], false, 0 - -(64) Window -Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Arguments: [row_number() windowspecdefinition(item_sk#58, d_date#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#63], [item_sk#58], [d_date#59 ASC NULLS FIRST] - -(65) Filter [codegen id : 54] -Input [5]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63] -Condition : isnotnull(rk#63) - -(66) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] - -(67) Sort [codegen id : 52] -Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] -Arguments: [item_sk#64 ASC NULLS FIRST, d_date#65 ASC NULLS FIRST], false, 0 - -(68) Window -Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] -Arguments: [row_number() windowspecdefinition(item_sk#64, d_date#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#68], [item_sk#64], [d_date#65 ASC NULLS FIRST] - -(69) Filter [codegen id : 53] -Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] -Condition : isnotnull(rk#68) - -(70) Project [codegen id : 53] -Output [4]: [item_sk#64, web_sales#66, store_sales#67, rk#68] -Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] - -(71) BroadcastExchange -Input [4]: [item_sk#64, web_sales#66, store_sales#67, rk#68] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] - -(72) BroadcastHashJoin [codegen id : 54] -Left keys [1]: [item_sk#58] -Right keys [1]: [item_sk#64] -Join condition: (rk#63 >= rk#68) - -(73) Project [codegen id : 54] -Output [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] -Input [9]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63, item_sk#64, web_sales#66, store_sales#67, rk#68] - -(74) HashAggregate [codegen id : 54] -Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] -Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Functions [2]: [partial_max(web_sales#66), partial_max(store_sales#67)] -Aggregate Attributes [2]: [max#70, max#71] -Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#72, max#73] - -(75) HashAggregate [codegen id : 54] -Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#72, max#73] -Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Functions [2]: [max(web_sales#66), max(store_sales#67)] -Aggregate Attributes [2]: [max(web_sales#66)#74, max(store_sales#67)#75] -Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max(web_sales#66)#74 AS web_cumulative#76, max(store_sales#67)#75 AS store_cumulative#77] - -(76) Filter [codegen id : 54] -Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] -Condition : ((isnotnull(web_cumulative#76) AND isnotnull(store_cumulative#77)) AND (web_cumulative#76 > store_cumulative#77)) - -(77) TakeOrderedAndProject -Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] -Arguments: 100, [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] ->>>>>>> regen ===== Subqueries ===== @@ -649,10 +405,6 @@ ReusedExchange (72) (72) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#5, d_date#6] -<<<<<<< HEAD Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#4 -======= -Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#33 IN dynamicpruning#4 ->>>>>>> regen 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 8d2776513f90b..ed0399519f579 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 @@ -213,11 +213,7 @@ Arguments: [rank(d_year#7, d_moy#8) windowspecdefinition(i_category#16, i_brand# (35) Filter [codegen id : 12] Input [8]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27] -<<<<<<< HEAD Condition : ((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#25)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) -======= -Condition : (((((isnotnull(d_year#7) AND isnotnull(avg_monthly_sales#25)) AND (d_year#7 = 1999)) AND (avg_monthly_sales#25 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#27)) ->>>>>>> regen (36) Exchange Input [8]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27] @@ -233,23 +229,22 @@ Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] (39) HashAggregate [codegen id : 21] Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] Keys [5]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33] -Functions [1]: [sum(UnscaledValue(cs_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#35))#36] -Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#35))#36,17,2) AS sum_sales#37] +Functions [1]: [sum(UnscaledValue(cs_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#3))#35] +Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#3))#35,17,2) AS sum_sales#36] (40) Exchange -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#37] (41) Sort [codegen id : 22] -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 (42) Window -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] -Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] -<<<<<<< HEAD (43) Project [codegen id : 23] Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] @@ -261,30 +256,12 @@ Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#38 + 1), (45) Sort [codegen id : 24] Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#38 + 1) ASC NULLS FIRST], false, 0 -======= -(43) Filter [codegen id : 23] -Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] -Condition : isnotnull(rn#39) - -(44) Project [codegen id : 23] -Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] -Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] - -(45) Exchange -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] - -(46) Sort [codegen id : 24] -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] -Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 ->>>>>>> regen (46) SortMergeJoin [codegen id : 25] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#39 + 1)] +Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#38 + 1)] Join condition: None -<<<<<<< HEAD (47) Project [codegen id : 25] Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] @@ -311,45 +288,12 @@ Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (rn#46 - 1), (53) Sort [codegen id : 36] Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 -======= -(48) Project [codegen id : 25] -Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] -Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] - -(49) ReusedExchange [Reuses operator id: 40] -Output [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] - -(50) Sort [codegen id : 34] -Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 - -(51) Window -Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] -Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#41, i_brand#42, cc_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#41, i_brand#42, cc_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] - -(52) Filter [codegen id : 35] -Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] -Condition : isnotnull(rn#47) - -(53) Project [codegen id : 35] -Output [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] -Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] - -(54) Exchange -Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] -Arguments: hashpartitioning(i_category#41, i_brand#42, cc_name#43, (rn#47 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] - -(55) Sort [codegen id : 36] -Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, (rn#47 - 1) ASC NULLS FIRST], false, 0 ->>>>>>> regen (54) SortMergeJoin [codegen id : 37] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#41, i_brand#42, cc_name#43, (rn#47 - 1)] +Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (rn#46 - 1)] Join condition: None -<<<<<<< HEAD (55) Project [codegen id : 37] Output [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#48, sum_sales#45 AS nsum#49] Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] @@ -357,15 +301,6 @@ Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales (56) TakeOrderedAndProject Input [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 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#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] -======= -(57) Project [codegen id : 37] -Output [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#49, sum_sales#46 AS nsum#50] -Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] - -(58) TakeOrderedAndProject -Input [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] ->>>>>>> regen ===== Subqueries ===== 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 f1956e47f453e..c347bbbe3b4ad 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 @@ -194,11 +194,7 @@ Arguments: [rank(d_year#11, d_moy#12) windowspecdefinition(i_category#3, i_brand (32) Filter [codegen id : 23] Input [8]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26] -<<<<<<< HEAD Condition : ((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#24)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) -======= -Condition : (((((isnotnull(d_year#11) AND isnotnull(avg_monthly_sales#24)) AND (d_year#11 = 1999)) AND (avg_monthly_sales#24 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true), false)) / promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(38,16), true) > 0.1000000000000000)) AND isnotnull(rn#26)) ->>>>>>> regen (33) ReusedExchange [Reuses operator id: unknown] Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] @@ -206,23 +202,22 @@ Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] (34) HashAggregate [codegen id : 13] 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] +Functions [1]: [sum(UnscaledValue(cs_sales_price#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#33] +Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#33,17,2) AS sum_sales#34] (35) 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#34] +Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#35] (36) Sort [codegen id : 14] -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#34] 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 (37) 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#34] +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#36], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] -<<<<<<< HEAD (38) Project [codegen id : 15] Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34, rn#36] @@ -230,26 +225,12 @@ Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sale (39) BroadcastExchange Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#37] -======= -(38) Filter [codegen id : 15] -Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35, rn#37] -Condition : isnotnull(rn#37) - -(39) Project [codegen id : 15] -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] - -(40) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] + 1)),false), [id=#38] ->>>>>>> regen (40) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -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#36 + 1)] Join condition: None -<<<<<<< HEAD (41) Project [codegen id : 23] Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34] Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] @@ -272,41 +253,12 @@ Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sale (46) BroadcastExchange Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#45] -======= -(42) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35] -Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] - -(43) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] - -(44) Sort [codegen id : 21] -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 - -(45) 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] - -(46) Filter [codegen id : 22] -Input [7]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44, rn#45] -Condition : isnotnull(rn#45) - -(47) Project [codegen id : 22] -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] - -(48) BroadcastExchange -Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, true] - 1)),false), [id=#46] ->>>>>>> regen (47) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -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#44 - 1)] Join condition: None -<<<<<<< HEAD (48) Project [codegen id : 23] Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#34 AS psum#46, sum_sales#43 AS nsum#47] Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34, i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] @@ -314,15 +266,6 @@ Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales (49) TakeOrderedAndProject Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, 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#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#21, psum#46, nsum#47] -======= -(50) Project [codegen id : 23] -Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] -Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] - -(51) TakeOrderedAndProject -Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, 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#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#21, psum#47, nsum#48] ->>>>>>> regen ===== Subqueries ===== 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 ef8088e415cc5..e04148fad545e 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 @@ -173,7 +173,7 @@ Input [2]: [d_date_sk#25, d_date#26] (18) Filter [codegen id : 4] Input [2]: [d_date_sk#25, d_date#26] -Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 1998-08-04)) AND (d_date#26 <= 1998-08-18)) AND isnotnull(d_date_sk#25)) +Condition : (((isnotnull(d_date#26) AND (d_date#26 >= 10442)) AND (d_date#26 <= 10456)) AND isnotnull(d_date_sk#25)) (19) Project [codegen id : 4] Output [1]: [d_date_sk#25] @@ -278,165 +278,165 @@ Output [6]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, Input [8]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] (41) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#70] +Output [1]: [d_date_sk#25] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#70] +Right keys [1]: [d_date_sk#25] Join condition: None (43) Project [codegen id : 11] Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#70] +Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#25] (44) HashAggregate [codegen id : 11] Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] -Results [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] +Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] +Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] (45) Exchange -Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] -Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] +Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] -Results [5]: [catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#68) AS id#85, MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS returns#87, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#88] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] +Results [5]: [catalog channel AS channel#83, concat(catalog_page, cp_catalog_page_id#68) AS id#84, MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#85, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS returns#86, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#87] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Condition : isnotnull(ws_web_site_sk#89) +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Condition : isnotnull(ws_web_site_sk#88) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] (53) Exchange -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] -Arguments: hashpartitioning(wr_item_sk#99, wr_order_number#100, 5), ENSURE_REQUIREMENTS, [id=#104] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Arguments: hashpartitioning(wr_item_sk#98, wr_order_number#99, 5), ENSURE_REQUIREMENTS, [id=#103] (54) Sort [codegen id : 15] -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] -Arguments: [wr_item_sk#99 ASC NULLS FIRST, wr_order_number#100 ASC NULLS FIRST], false, 0 +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Arguments: [wr_item_sk#98 ASC NULLS FIRST, wr_order_number#99 ASC NULLS FIRST], false, 0 (55) Scan parquet default.web_sales -Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 16] -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] (57) Filter [codegen id : 16] -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] -Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) (58) Project [codegen id : 16] -Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] (59) Exchange -Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] -Arguments: hashpartitioning(cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint), 5), ENSURE_REQUIREMENTS, [id=#109] +Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Arguments: hashpartitioning(cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint), 5), ENSURE_REQUIREMENTS, [id=#106] (60) Sort [codegen id : 17] -Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] -Arguments: [cast(ws_item_sk#105 as bigint) ASC NULLS FIRST, cast(ws_order_number#107 as bigint) ASC NULLS FIRST], false, 0 +Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Arguments: [cast(ws_item_sk#104 as bigint) ASC NULLS FIRST, cast(ws_order_number#105 as bigint) ASC NULLS FIRST], false, 0 (61) SortMergeJoin [codegen id : 18] -Left keys [2]: [wr_item_sk#99, wr_order_number#100] -Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] +Left keys [2]: [wr_item_sk#98, wr_order_number#99] +Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None (62) Project [codegen id : 18] -Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#110, wr_returned_date_sk#103 AS date_sk#111, 0.00 AS sales_price#112, 0.00 AS profit#113, wr_return_amt#101 AS return_amt#114, wr_net_loss#102 AS net_loss#115] -Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#107, wr_returned_date_sk#102 AS date_sk#108, 0.00 AS sales_price#109, 0.00 AS profit#110, wr_return_amt#100 AS return_amt#111, wr_net_loss#101 AS net_loss#112] +Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] (63) Union (64) Scan parquet default.web_site -Output [2]: [web_site_sk#116, web_site_id#117] +Output [2]: [web_site_sk#113, web_site_id#114] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 19] -Input [2]: [web_site_sk#116, web_site_id#117] +Input [2]: [web_site_sk#113, web_site_id#114] (66) Filter [codegen id : 19] -Input [2]: [web_site_sk#116, web_site_id#117] -Condition : isnotnull(web_site_sk#116) +Input [2]: [web_site_sk#113, web_site_id#114] +Condition : isnotnull(web_site_sk#113) (67) BroadcastExchange -Input [2]: [web_site_sk#116, web_site_id#117] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] +Input [2]: [web_site_sk#113, web_site_id#114] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] (68) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [wsr_web_site_sk#93] -Right keys [1]: [web_site_sk#116] +Left keys [1]: [wsr_web_site_sk#92] +Right keys [1]: [web_site_sk#113] Join condition: None (69) Project [codegen id : 21] -Output [6]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Input [8]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] +Output [6]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] +Input [8]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#113, web_site_id#114] (70) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#119] +Output [1]: [d_date_sk#25] (71) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [date_sk#94] -Right keys [1]: [cast(d_date_sk#119 as bigint)] +Left keys [1]: [date_sk#93] +Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None (72) Project [codegen id : 21] -Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Input [7]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117, d_date_sk#119] +Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] +Input [7]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114, d_date_sk#25] (73) HashAggregate [codegen id : 21] -Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Keys [1]: [web_site_id#117] -Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] -Aggregate Attributes [4]: [sum#120, sum#121, sum#122, sum#123] -Results [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] +Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] +Keys [1]: [web_site_id#114] +Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] +Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] +Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] (74) Exchange -Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] -Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#128] +Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] +Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] (75) HashAggregate [codegen id : 22] -Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] -Keys [1]: [web_site_id#117] -Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#129, sum(UnscaledValue(return_amt#97))#130, sum(UnscaledValue(profit#96))#131, sum(UnscaledValue(net_loss#98))#132] -Results [5]: [web channel AS channel#133, concat(web_site, web_site_id#117) AS id#134, MakeDecimal(sum(UnscaledValue(sales_price#95))#129,17,2) AS sales#135, MakeDecimal(sum(UnscaledValue(return_amt#97))#130,17,2) AS returns#136, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#131,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#132,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#137] +Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] +Keys [1]: [web_site_id#114] +Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#125, sum(UnscaledValue(return_amt#96))#126, sum(UnscaledValue(profit#95))#127, sum(UnscaledValue(net_loss#97))#128] +Results [5]: [web channel AS channel#129, concat(web_site, web_site_id#114) AS id#130, MakeDecimal(sum(UnscaledValue(sales_price#94))#125,17,2) AS sales#131, MakeDecimal(sum(UnscaledValue(return_amt#96))#126,17,2) AS returns#132, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#133] (76) Union @@ -444,99 +444,99 @@ Results [5]: [web channel AS channel#133, concat(web_site, web_site_id#117) AS i Input [5]: [channel#41, id#42, sales#43, returns#44, profit#45] Keys [2]: [channel#41, id#42] Functions [3]: [partial_sum(sales#43), partial_sum(returns#44), partial_sum(profit#45)] -Aggregate Attributes [6]: [sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Results [8]: [channel#41, id#42, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] +Aggregate Attributes [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Results [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] (78) Exchange -Input [8]: [channel#41, id#42, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] -Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#150] +Input [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#146] (79) HashAggregate [codegen id : 24] -Input [8]: [channel#41, id#42, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] +Input [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#45)] -Aggregate Attributes [3]: [sum(sales#43)#151, sum(returns#44)#152, sum(profit#45)#153] -Results [5]: [channel#41, id#42, cast(sum(sales#43)#151 as decimal(37,2)) AS sales#154, cast(sum(returns#44)#152 as decimal(37,2)) AS returns#155, cast(sum(profit#45)#153 as decimal(38,2)) AS profit#156] +Aggregate Attributes [3]: [sum(sales#43)#147, sum(returns#44)#148, sum(profit#45)#149] +Results [5]: [channel#41, id#42, cast(sum(sales#43)#147 as decimal(37,2)) AS sales#150, cast(sum(returns#44)#148 as decimal(37,2)) AS returns#151, cast(sum(profit#45)#149 as decimal(38,2)) AS profit#152] (80) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] +Output [8]: [channel#41, id#42, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] (81) HashAggregate [codegen id : 48] -Input [8]: [channel#41, id#42, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] +Input [8]: [channel#41, id#42, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#163)] -Aggregate Attributes [3]: [sum(sales#43)#164, sum(returns#44)#165, sum(profit#163)#166] -Results [4]: [channel#41, sum(sales#43)#164 AS sales#167, sum(returns#44)#165 AS returns#168, sum(profit#163)#166 AS profit#169] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#159)] +Aggregate Attributes [3]: [sum(sales#43)#160, sum(returns#44)#161, sum(profit#159)#162] +Results [4]: [channel#41, sum(sales#43)#160 AS sales#163, sum(returns#44)#161 AS returns#164, sum(profit#159)#162 AS profit#165] (82) HashAggregate [codegen id : 48] -Input [4]: [channel#41, sales#167, returns#168, profit#169] +Input [4]: [channel#41, sales#163, returns#164, profit#165] Keys [1]: [channel#41] -Functions [3]: [partial_sum(sales#167), partial_sum(returns#168), partial_sum(profit#169)] -Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] -Results [7]: [channel#41, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +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#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] (83) Exchange -Input [7]: [channel#41, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] -Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#182] +Input [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] +Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#178] (84) HashAggregate [codegen id : 49] -Input [7]: [channel#41, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Input [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] Keys [1]: [channel#41] -Functions [3]: [sum(sales#167), sum(returns#168), sum(profit#169)] -Aggregate Attributes [3]: [sum(sales#167)#183, sum(returns#168)#184, sum(profit#169)#185] -Results [5]: [channel#41, null AS id#186, sum(sales#167)#183 AS sum(sales)#187, sum(returns#168)#184 AS sum(returns)#188, sum(profit#169)#185 AS sum(profit)#189] +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#41, 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] (85) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#190, isEmpty#191, sum#192, isEmpty#193, sum#194, isEmpty#195] +Output [8]: [channel#41, id#42, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] (86) HashAggregate [codegen id : 73] -Input [8]: [channel#41, id#42, sum#190, isEmpty#191, sum#192, isEmpty#193, sum#194, isEmpty#195] +Input [8]: [channel#41, id#42, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#196)] -Aggregate Attributes [3]: [sum(sales#43)#197, sum(returns#44)#198, sum(profit#196)#199] -Results [3]: [sum(sales#43)#197 AS sales#167, sum(returns#44)#198 AS returns#168, sum(profit#196)#199 AS profit#169] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#192)] +Aggregate Attributes [3]: [sum(sales#43)#193, sum(returns#44)#194, sum(profit#192)#195] +Results [3]: [sum(sales#43)#193 AS sales#163, sum(returns#44)#194 AS returns#164, sum(profit#192)#195 AS profit#165] (87) HashAggregate [codegen id : 73] -Input [3]: [sales#167, returns#168, profit#169] +Input [3]: [sales#163, returns#164, profit#165] Keys: [] -Functions [3]: [partial_sum(sales#167), partial_sum(returns#168), partial_sum(profit#169)] -Aggregate Attributes [6]: [sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205] -Results [6]: [sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211] +Functions [3]: [partial_sum(sales#163), partial_sum(returns#164), partial_sum(profit#165)] +Aggregate Attributes [6]: [sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201] +Results [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] (88) Exchange -Input [6]: [sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#212] +Input [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#208] (89) HashAggregate [codegen id : 74] -Input [6]: [sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211] +Input [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] Keys: [] -Functions [3]: [sum(sales#167), sum(returns#168), sum(profit#169)] -Aggregate Attributes [3]: [sum(sales#167)#213, sum(returns#168)#214, sum(profit#169)#215] -Results [5]: [null AS channel#216, null AS id#217, sum(sales#167)#213 AS sum(sales)#218, sum(returns#168)#214 AS sum(returns)#219, sum(profit#169)#215 AS sum(profit)#220] +Functions [3]: [sum(sales#163), sum(returns#164), sum(profit#165)] +Aggregate Attributes [3]: [sum(sales#163)#209, sum(returns#164)#210, sum(profit#165)#211] +Results [5]: [null AS channel#212, null AS id#213, sum(sales#163)#209 AS sum(sales)#214, sum(returns#164)#210 AS sum(returns)#215, sum(profit#165)#211 AS sum(profit)#216] (90) Union (91) HashAggregate [codegen id : 75] -Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] -Keys [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Keys [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Results [5]: [channel#41, id#42, sales#150, returns#151, profit#152] (92) Exchange -Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] -Arguments: hashpartitioning(channel#41, id#42, sales#154, returns#155, profit#156, 5), ENSURE_REQUIREMENTS, [id=#221] +Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Arguments: hashpartitioning(channel#41, id#42, sales#150, returns#151, profit#152, 5), ENSURE_REQUIREMENTS, [id=#217] (93) HashAggregate [codegen id : 76] -Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] -Keys [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Keys [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Results [5]: [channel#41, id#42, sales#150, returns#151, profit#152] (94) TakeOrderedAndProject -Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] -Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#154, returns#155, profit#156] +Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#150, returns#151, profit#152] ===== Subqueries ===== @@ -554,12 +554,12 @@ ReusedExchange (96) (96) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#70] +Output [1]: [d_date_sk#25] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 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 c6b64d59abf82..36b1ff63b2065 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 @@ -143,7 +143,7 @@ Input [2]: [d_date_sk#22, d_date#23] (12) Filter [codegen id : 3] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 10442)) AND (d_date#23 <= 10456)) AND isnotnull(d_date_sk#22)) (13) Project [codegen id : 3] Output [1]: [d_date_sk#22] @@ -248,180 +248,180 @@ Input [4]: [cr_catalog_page_sk#57, cr_return_amount#58, cr_net_loss#59, cr_retur (34) Union (35) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#67] +Output [1]: [d_date_sk#22] (36) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#67] +Right keys [1]: [d_date_sk#22] Join condition: None (37) Project [codegen id : 11] Output [5]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56] -Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#67] +Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#22] (38) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Output [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] (40) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] -Condition : isnotnull(cp_catalog_page_sk#68) +Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Condition : isnotnull(cp_catalog_page_sk#67) (41) BroadcastExchange -Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#70] +Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#69] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [page_sk#51] -Right keys [1]: [cp_catalog_page_sk#68] +Right keys [1]: [cp_catalog_page_sk#67] Join condition: None (43) Project [codegen id : 11] -Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] -Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#68, cp_catalog_page_id#69] +Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] +Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] (44) HashAggregate [codegen id : 11] -Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] -Keys [1]: [cp_catalog_page_id#69] +Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] +Keys [1]: [cp_catalog_page_id#68] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] -Results [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] +Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] +Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] (45) Exchange -Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] -Arguments: hashpartitioning(cp_catalog_page_id#69, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] -Keys [1]: [cp_catalog_page_id#69] +Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Keys [1]: [cp_catalog_page_id#68] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] -Results [5]: [catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#69) AS id#85, MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS returns#87, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#88] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] +Results [5]: [catalog channel AS channel#83, concat(catalog_page, cp_catalog_page_id#68) AS id#84, MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#85, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS returns#86, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#87] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Condition : isnotnull(ws_web_site_sk#89) +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Condition : isnotnull(ws_web_site_sk#88) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] (53) BroadcastExchange -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#104] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#103] (54) Scan parquet default.web_sales -Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (55) ColumnarToRow -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] (56) Filter -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] -Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) (57) Project -Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] (58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#99, wr_order_number#100] -Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] +Left keys [2]: [wr_item_sk#98, wr_order_number#99] +Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None (59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#109, wr_returned_date_sk#103 AS date_sk#110, 0.00 AS sales_price#111, 0.00 AS profit#112, wr_return_amt#101 AS return_amt#113, wr_net_loss#102 AS net_loss#114] -Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#106, wr_returned_date_sk#102 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#100 AS return_amt#110, wr_net_loss#101 AS net_loss#111] +Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] (60) Union (61) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#115] +Output [1]: [d_date_sk#22] (62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#94] -Right keys [1]: [cast(d_date_sk#115 as bigint)] +Left keys [1]: [date_sk#93] +Right keys [1]: [cast(d_date_sk#22 as bigint)] Join condition: None (63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98] -Input [7]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, d_date_sk#115] +Output [5]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97] +Input [7]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, d_date_sk#22] (64) Scan parquet default.web_site -Output [2]: [web_site_sk#116, web_site_id#117] +Output [2]: [web_site_sk#112, web_site_id#113] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#116, web_site_id#117] +Input [2]: [web_site_sk#112, web_site_id#113] (66) Filter [codegen id : 17] -Input [2]: [web_site_sk#116, web_site_id#117] -Condition : isnotnull(web_site_sk#116) +Input [2]: [web_site_sk#112, web_site_id#113] +Condition : isnotnull(web_site_sk#112) (67) BroadcastExchange -Input [2]: [web_site_sk#116, web_site_id#117] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] +Input [2]: [web_site_sk#112, web_site_id#113] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#114] (68) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#93] -Right keys [1]: [web_site_sk#116] +Left keys [1]: [wsr_web_site_sk#92] +Right keys [1]: [web_site_sk#112] Join condition: None (69) Project [codegen id : 18] -Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Input [7]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] +Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] +Input [7]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#112, web_site_id#113] (70) HashAggregate [codegen id : 18] -Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Keys [1]: [web_site_id#117] -Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] -Aggregate Attributes [4]: [sum#119, sum#120, sum#121, sum#122] -Results [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] +Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] +Keys [1]: [web_site_id#113] +Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] +Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] +Results [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] (71) Exchange -Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] -Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#127] +Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] +Arguments: hashpartitioning(web_site_id#113, 5), ENSURE_REQUIREMENTS, [id=#123] (72) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] -Keys [1]: [web_site_id#117] -Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#128, sum(UnscaledValue(return_amt#97))#129, sum(UnscaledValue(profit#96))#130, sum(UnscaledValue(net_loss#98))#131] -Results [5]: [web channel AS channel#132, concat(web_site, web_site_id#117) AS id#133, MakeDecimal(sum(UnscaledValue(sales_price#95))#128,17,2) AS sales#134, MakeDecimal(sum(UnscaledValue(return_amt#97))#129,17,2) AS returns#135, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#130,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#131,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#136] +Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] +Keys [1]: [web_site_id#113] +Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#124, sum(UnscaledValue(return_amt#96))#125, sum(UnscaledValue(profit#95))#126, sum(UnscaledValue(net_loss#97))#127] +Results [5]: [web channel AS channel#128, concat(web_site, web_site_id#113) AS id#129, MakeDecimal(sum(UnscaledValue(sales_price#94))#124,17,2) AS sales#130, MakeDecimal(sum(UnscaledValue(return_amt#96))#125,17,2) AS returns#131, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#126,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#127,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#132] (73) Union @@ -429,99 +429,99 @@ Results [5]: [web channel AS channel#132, concat(web_site, web_site_id#117) AS i Input [5]: [channel#41, id#42, sales#43, returns#44, profit#45] Keys [2]: [channel#41, id#42] Functions [3]: [partial_sum(sales#43), partial_sum(returns#44), partial_sum(profit#45)] -Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] -Results [8]: [channel#41, id#42, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Aggregate Attributes [6]: [sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] +Results [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] (75) Exchange -Input [8]: [channel#41, id#42, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#149] +Input [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#145] (76) HashAggregate [codegen id : 21] -Input [8]: [channel#41, id#42, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Input [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#45)] -Aggregate Attributes [3]: [sum(sales#43)#150, sum(returns#44)#151, sum(profit#45)#152] -Results [5]: [channel#41, id#42, cast(sum(sales#43)#150 as decimal(37,2)) AS sales#153, cast(sum(returns#44)#151 as decimal(37,2)) AS returns#154, cast(sum(profit#45)#152 as decimal(38,2)) AS profit#155] +Aggregate Attributes [3]: [sum(sales#43)#146, sum(returns#44)#147, sum(profit#45)#148] +Results [5]: [channel#41, id#42, cast(sum(sales#43)#146 as decimal(37,2)) AS sales#149, cast(sum(returns#44)#147 as decimal(37,2)) AS returns#150, cast(sum(profit#45)#148 as decimal(38,2)) AS profit#151] (77) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Output [8]: [channel#41, id#42, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] (78) HashAggregate [codegen id : 42] -Input [8]: [channel#41, id#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Input [8]: [channel#41, id#42, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#162)] -Aggregate Attributes [3]: [sum(sales#43)#163, sum(returns#44)#164, sum(profit#162)#165] -Results [4]: [channel#41, sum(sales#43)#163 AS sales#166, sum(returns#44)#164 AS returns#167, sum(profit#162)#165 AS profit#168] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#158)] +Aggregate Attributes [3]: [sum(sales#43)#159, sum(returns#44)#160, sum(profit#158)#161] +Results [4]: [channel#41, sum(sales#43)#159 AS sales#162, sum(returns#44)#160 AS returns#163, sum(profit#158)#161 AS profit#164] (79) HashAggregate [codegen id : 42] -Input [4]: [channel#41, sales#166, returns#167, profit#168] +Input [4]: [channel#41, sales#162, returns#163, profit#164] Keys [1]: [channel#41] -Functions [3]: [partial_sum(sales#166), partial_sum(returns#167), partial_sum(profit#168)] -Aggregate Attributes [6]: [sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174] -Results [7]: [channel#41, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] +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#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] (80) Exchange -Input [7]: [channel#41, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] -Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#181] +Input [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#177] (81) HashAggregate [codegen id : 43] -Input [7]: [channel#41, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] +Input [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] Keys [1]: [channel#41] -Functions [3]: [sum(sales#166), sum(returns#167), sum(profit#168)] -Aggregate Attributes [3]: [sum(sales#166)#182, sum(returns#167)#183, sum(profit#168)#184] -Results [5]: [channel#41, null AS id#185, sum(sales#166)#182 AS sum(sales)#186, sum(returns#167)#183 AS sum(returns)#187, sum(profit#168)#184 AS sum(profit)#188] +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#41, 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] (82) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194] +Output [8]: [channel#41, id#42, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] (83) HashAggregate [codegen id : 64] -Input [8]: [channel#41, id#42, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194] +Input [8]: [channel#41, id#42, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#195)] -Aggregate Attributes [3]: [sum(sales#43)#196, sum(returns#44)#197, sum(profit#195)#198] -Results [3]: [sum(sales#43)#196 AS sales#166, sum(returns#44)#197 AS returns#167, sum(profit#195)#198 AS profit#168] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#191)] +Aggregate Attributes [3]: [sum(sales#43)#192, sum(returns#44)#193, sum(profit#191)#194] +Results [3]: [sum(sales#43)#192 AS sales#162, sum(returns#44)#193 AS returns#163, sum(profit#191)#194 AS profit#164] (84) HashAggregate [codegen id : 64] -Input [3]: [sales#166, returns#167, profit#168] +Input [3]: [sales#162, returns#163, profit#164] Keys: [] -Functions [3]: [partial_sum(sales#166), partial_sum(returns#167), partial_sum(profit#168)] -Aggregate Attributes [6]: [sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204] -Results [6]: [sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210] +Functions [3]: [partial_sum(sales#162), partial_sum(returns#163), partial_sum(profit#164)] +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] (85) Exchange -Input [6]: [sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#211] +Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#207] (86) HashAggregate [codegen id : 65] -Input [6]: [sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210] +Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Keys: [] -Functions [3]: [sum(sales#166), sum(returns#167), sum(profit#168)] -Aggregate Attributes [3]: [sum(sales#166)#212, sum(returns#167)#213, sum(profit#168)#214] -Results [5]: [null AS channel#215, null AS id#216, sum(sales#166)#212 AS sum(sales)#217, sum(returns#167)#213 AS sum(returns)#218, sum(profit#168)#214 AS sum(profit)#219] +Functions [3]: [sum(sales#162), sum(returns#163), sum(profit#164)] +Aggregate Attributes [3]: [sum(sales#162)#208, sum(returns#163)#209, sum(profit#164)#210] +Results [5]: [null AS channel#211, null AS id#212, sum(sales#162)#208 AS sum(sales)#213, sum(returns#163)#209 AS sum(returns)#214, sum(profit#164)#210 AS sum(profit)#215] (87) Union (88) HashAggregate [codegen id : 66] -Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] -Keys [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Keys [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Results [5]: [channel#41, id#42, sales#149, returns#150, profit#151] (89) Exchange -Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] -Arguments: hashpartitioning(channel#41, id#42, sales#153, returns#154, profit#155, 5), ENSURE_REQUIREMENTS, [id=#220] +Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Arguments: hashpartitioning(channel#41, id#42, sales#149, returns#150, profit#151, 5), ENSURE_REQUIREMENTS, [id=#216] (90) HashAggregate [codegen id : 67] -Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] -Keys [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Keys [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Results [5]: [channel#41, id#42, sales#149, returns#150, profit#151] (91) TakeOrderedAndProject -Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] -Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#153, returns#154, profit#155] +Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#149, returns#150, profit#151] ===== Subqueries ===== @@ -539,12 +539,12 @@ ReusedExchange (93) (93) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#67] +Output [1]: [d_date_sk#22] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 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 3847146fa8fa8..c096a28386361 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 @@ -741,7 +741,7 @@ Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt# Output [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple ,burlywood ,indian ,spring ,floral ,medium ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple,burlywood,indian,spring,floral,medium]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct (121) ColumnarToRow [codegen id : 40] @@ -749,7 +749,7 @@ Input [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] (122) Filter [codegen id : 40] Input [4]: [i_item_sk#93, i_current_price#94, i_color#95, i_product_name#96] -Condition : ((((((isnotnull(i_current_price#94) AND i_color#95 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#94 >= 64.00)) AND (i_current_price#94 <= 74.00)) AND (i_current_price#94 >= 65.00)) AND (i_current_price#94 <= 79.00)) AND isnotnull(i_item_sk#93)) +Condition : ((((((isnotnull(i_current_price#94) AND i_color#95 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#94 >= 64.00)) AND (i_current_price#94 <= 74.00)) AND (i_current_price#94 >= 65.00)) AND (i_current_price#94 <= 79.00)) AND isnotnull(i_item_sk#93)) (123) Project [codegen id : 40] Output [2]: [i_item_sk#93, i_product_name#96] @@ -795,353 +795,353 @@ Input [17]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_str Arguments: [item_sk#112 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, store_zip#114 ASC NULLS FIRST], false, 0 (132) Scan parquet default.store_sales -Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#140), dynamicpruningexpression(ss_sold_date_sk#140 IN dynamicpruning#141)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#129)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (133) ColumnarToRow [codegen id : 44] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] (134) Filter [codegen id : 44] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Condition : (((((((isnotnull(ss_item_sk#129) AND isnotnull(ss_ticket_number#136)) AND isnotnull(ss_store_sk#134)) AND isnotnull(ss_customer_sk#130)) AND isnotnull(ss_cdemo_sk#131)) AND isnotnull(ss_promo_sk#135)) AND isnotnull(ss_hdemo_sk#132)) AND isnotnull(ss_addr_sk#133)) +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) (135) Exchange -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Arguments: hashpartitioning(cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint), 5), ENSURE_REQUIREMENTS, [id=#142] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: hashpartitioning(cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint), 5), ENSURE_REQUIREMENTS, [id=#130] (136) Sort [codegen id : 45] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Arguments: [cast(ss_item_sk#129 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#136 as bigint) ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: [cast(ss_item_sk#1 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#8 as bigint) ASC NULLS FIRST], false, 0 (137) ReusedExchange [Reuses operator id: 10] -Output [2]: [sr_item_sk#143, sr_ticket_number#144] +Output [2]: [sr_item_sk#15, sr_ticket_number#16] (138) Sort [codegen id : 47] -Input [2]: [sr_item_sk#143, sr_ticket_number#144] -Arguments: [sr_item_sk#143 ASC NULLS FIRST, sr_ticket_number#144 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#15, sr_ticket_number#16] +Arguments: [sr_item_sk#15 ASC NULLS FIRST, sr_ticket_number#16 ASC NULLS FIRST], false, 0 (139) SortMergeJoin [codegen id : 56] -Left keys [2]: [cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint)] -Right keys [2]: [sr_item_sk#143, sr_ticket_number#144] +Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] +Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] Join condition: None (140) Project [codegen id : 56] -Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, sr_item_sk#143, sr_ticket_number#144] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] (141) ReusedExchange [Reuses operator id: 33] -Output [1]: [cs_item_sk#145] +Output [1]: [cs_item_sk#19] (142) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_item_sk#129] -Right keys [1]: [cs_item_sk#145] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [cs_item_sk#19] Join condition: None (143) Project [codegen id : 56] -Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, cs_item_sk#145] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] (144) Scan parquet default.date_dim -Output [2]: [d_date_sk#146, d_year#147] +Output [2]: [d_date_sk#43, d_year#44] 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 (145) ColumnarToRow [codegen id : 54] -Input [2]: [d_date_sk#146, d_year#147] +Input [2]: [d_date_sk#43, d_year#44] (146) Filter [codegen id : 54] -Input [2]: [d_date_sk#146, d_year#147] -Condition : ((isnotnull(d_year#147) AND (d_year#147 = 2000)) AND isnotnull(d_date_sk#146)) +Input [2]: [d_date_sk#43, d_year#44] +Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#43)) (147) BroadcastExchange -Input [2]: [d_date_sk#146, d_year#147] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#148] +Input [2]: [d_date_sk#43, d_year#44] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#131] (148) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_sold_date_sk#140] -Right keys [1]: [d_date_sk#146] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#43] Join condition: None (149) Project [codegen id : 56] -Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147] -Input [13]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, d_date_sk#146, d_year#147] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#43, d_year#44] (150) ReusedExchange [Reuses operator id: 45] -Output [3]: [s_store_sk#149, s_store_name#150, s_zip#151] +Output [3]: [s_store_sk#46, s_store_name#47, s_zip#48] (151) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_store_sk#134] -Right keys [1]: [s_store_sk#149] +Left keys [1]: [ss_store_sk#6] +Right keys [1]: [s_store_sk#46] Join condition: None (152) Project [codegen id : 56] -Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] -Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_sk#149, s_store_name#150, s_zip#151] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_sk#46, s_store_name#47, s_zip#48] (153) Exchange -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] -Arguments: hashpartitioning(ss_customer_sk#130, 5), ENSURE_REQUIREMENTS, [id=#152] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#132] (154) Sort [codegen id : 57] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] -Arguments: [ss_customer_sk#130 ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (155) ReusedExchange [Reuses operator id: 53] -Output [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Output [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] (156) Sort [codegen id : 59] -Input [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] -Arguments: [c_customer_sk#153 ASC NULLS FIRST], false, 0 +Input [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Arguments: [c_customer_sk#51 ASC NULLS FIRST], false, 0 (157) SortMergeJoin [codegen id : 62] -Left keys [1]: [ss_customer_sk#130] -Right keys [1]: [c_customer_sk#153] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#51] Join condition: None (158) Project [codegen id : 62] -Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] -Input [18]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] (159) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#159, d_year#160] +Output [2]: [d_date_sk#133, d_year#134] (160) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_sales_date_sk#158] -Right keys [1]: [d_date_sk#159] +Left keys [1]: [c_first_sales_date_sk#56] +Right keys [1]: [d_date_sk#133] Join condition: None (161) Project [codegen id : 62] -Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160] -Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158, d_date_sk#159, d_year#160] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56, d_date_sk#133, d_year#134] (162) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#161, d_year#162] +Output [2]: [d_date_sk#135, d_year#136] (163) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_shipto_date_sk#157] -Right keys [1]: [d_date_sk#161] +Left keys [1]: [c_first_shipto_date_sk#55] +Right keys [1]: [d_date_sk#135] Join condition: None (164) Project [codegen id : 62] -Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160, d_date_sk#161, d_year#162] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134, d_date_sk#135, d_year#136] (165) Exchange -Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Arguments: hashpartitioning(ss_cdemo_sk#131, 5), ENSURE_REQUIREMENTS, [id=#163] +Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Arguments: hashpartitioning(ss_cdemo_sk#3, 5), ENSURE_REQUIREMENTS, [id=#137] (166) Sort [codegen id : 63] -Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Arguments: [ss_cdemo_sk#131 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Arguments: [ss_cdemo_sk#3 ASC NULLS FIRST], false, 0 (167) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#164, cd_marital_status#165] +Output [2]: [cd_demo_sk#64, cd_marital_status#65] (168) Sort [codegen id : 65] -Input [2]: [cd_demo_sk#164, cd_marital_status#165] -Arguments: [cd_demo_sk#164 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#64, cd_marital_status#65] +Arguments: [cd_demo_sk#64 ASC NULLS FIRST], false, 0 (169) SortMergeJoin [codegen id : 66] -Left keys [1]: [ss_cdemo_sk#131] -Right keys [1]: [cd_demo_sk#164] +Left keys [1]: [ss_cdemo_sk#3] +Right keys [1]: [cd_demo_sk#64] Join condition: None (170) Project [codegen id : 66] -Output [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] -Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_demo_sk#164, cd_marital_status#165] +Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_demo_sk#64, cd_marital_status#65] (171) Exchange -Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] -Arguments: hashpartitioning(c_current_cdemo_sk#154, 5), ENSURE_REQUIREMENTS, [id=#166] +Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] +Arguments: hashpartitioning(c_current_cdemo_sk#52, 5), ENSURE_REQUIREMENTS, [id=#138] (172) Sort [codegen id : 67] -Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] -Arguments: [c_current_cdemo_sk#154 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] +Arguments: [c_current_cdemo_sk#52 ASC NULLS FIRST], false, 0 (173) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#167, cd_marital_status#168] +Output [2]: [cd_demo_sk#139, cd_marital_status#140] (174) Sort [codegen id : 69] -Input [2]: [cd_demo_sk#167, cd_marital_status#168] -Arguments: [cd_demo_sk#167 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#139, cd_marital_status#140] +Arguments: [cd_demo_sk#139 ASC NULLS FIRST], false, 0 (175) SortMergeJoin [codegen id : 73] -Left keys [1]: [c_current_cdemo_sk#154] -Right keys [1]: [cd_demo_sk#167] -Join condition: NOT (cd_marital_status#165 = cd_marital_status#168) +Left keys [1]: [c_current_cdemo_sk#52] +Right keys [1]: [cd_demo_sk#139] +Join condition: NOT (cd_marital_status#65 = cd_marital_status#140) (176) Project [codegen id : 73] -Output [14]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Input [18]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165, cd_demo_sk#167, cd_marital_status#168] +Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65, cd_demo_sk#139, cd_marital_status#140] (177) ReusedExchange [Reuses operator id: 84] -Output [1]: [p_promo_sk#169] +Output [1]: [p_promo_sk#70] (178) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_promo_sk#135] -Right keys [1]: [p_promo_sk#169] +Left keys [1]: [ss_promo_sk#7] +Right keys [1]: [p_promo_sk#70] Join condition: None (179) Project [codegen id : 73] -Output [13]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, p_promo_sk#169] +Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, p_promo_sk#70] (180) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#170, hd_income_band_sk#171] +Output [2]: [hd_demo_sk#72, hd_income_band_sk#73] (181) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_hdemo_sk#132] -Right keys [1]: [hd_demo_sk#170] +Left keys [1]: [ss_hdemo_sk#4] +Right keys [1]: [hd_demo_sk#72] Join condition: None (182) Project [codegen id : 73] -Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171] -Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_demo_sk#170, hd_income_band_sk#171] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_demo_sk#72, hd_income_band_sk#73] (183) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#172, hd_income_band_sk#173] +Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] (184) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [c_current_hdemo_sk#155] -Right keys [1]: [hd_demo_sk#172] +Left keys [1]: [c_current_hdemo_sk#53] +Right keys [1]: [hd_demo_sk#141] Join condition: None (185) Project [codegen id : 73] -Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] -Input [15]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_demo_sk#172, hd_income_band_sk#173] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_demo_sk#141, hd_income_band_sk#142] (186) Exchange -Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] -Arguments: hashpartitioning(ss_addr_sk#133, 5), ENSURE_REQUIREMENTS, [id=#174] +Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] +Arguments: hashpartitioning(ss_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#143] (187) Sort [codegen id : 74] -Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] -Arguments: [ss_addr_sk#133 ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] +Arguments: [ss_addr_sk#5 ASC NULLS FIRST], false, 0 (188) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Output [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] (189) Sort [codegen id : 76] -Input [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -Arguments: [ca_address_sk#175 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Arguments: [ca_address_sk#78 ASC NULLS FIRST], false, 0 (190) SortMergeJoin [codegen id : 77] -Left keys [1]: [ss_addr_sk#133] -Right keys [1]: [ca_address_sk#175] +Left keys [1]: [ss_addr_sk#5] +Right keys [1]: [ca_address_sk#78] Join condition: None (191) Project [codegen id : 77] -Output [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -Input [18]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] (192) Exchange -Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -Arguments: hashpartitioning(c_current_addr_sk#156, 5), ENSURE_REQUIREMENTS, [id=#180] +Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Arguments: hashpartitioning(c_current_addr_sk#54, 5), ENSURE_REQUIREMENTS, [id=#144] (193) Sort [codegen id : 78] -Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -Arguments: [c_current_addr_sk#156 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Arguments: [c_current_addr_sk#54 ASC NULLS FIRST], false, 0 (194) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Output [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] (195) Sort [codegen id : 80] -Input [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] -Arguments: [ca_address_sk#181 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Arguments: [ca_address_sk#145 ASC NULLS FIRST], false, 0 (196) SortMergeJoin [codegen id : 84] -Left keys [1]: [c_current_addr_sk#156] -Right keys [1]: [ca_address_sk#181] +Left keys [1]: [c_current_addr_sk#54] +Right keys [1]: [ca_address_sk#145] Join condition: None (197) Project [codegen id : 84] -Output [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] -Input [21]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] (198) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#186] +Output [1]: [ib_income_band_sk#90] (199) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#171] -Right keys [1]: [ib_income_band_sk#186] +Left keys [1]: [hd_income_band_sk#73] +Right keys [1]: [ib_income_band_sk#90] Join condition: None (200) Project [codegen id : 84] -Output [18]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] -Input [20]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#186] +Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#90] (201) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#187] +Output [1]: [ib_income_band_sk#150] (202) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#173] -Right keys [1]: [ib_income_band_sk#187] +Left keys [1]: [hd_income_band_sk#142] +Right keys [1]: [ib_income_band_sk#150] Join condition: None (203) Project [codegen id : 84] -Output [17]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] -Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#187] +Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#150] (204) ReusedExchange [Reuses operator id: 124] -Output [2]: [i_item_sk#188, i_product_name#189] +Output [2]: [i_item_sk#93, i_product_name#96] (205) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#129] -Right keys [1]: [i_item_sk#188] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#93] Join condition: None (206) Project [codegen id : 84] -Output [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] -Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] +Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] (207) HashAggregate [codegen id : 84] -Input [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] -Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#137)), partial_sum(UnscaledValue(ss_list_price#138)), partial_sum(UnscaledValue(ss_coupon_amt#139))] -Aggregate Attributes [4]: [count#190, sum#191, sum#192, sum#193] -Results [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] +Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count#151, sum#152, sum#153, sum#154] +Results [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] (208) Exchange -Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] -Arguments: hashpartitioning(i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, 5), ENSURE_REQUIREMENTS, [id=#198] +Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] +Arguments: hashpartitioning(i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, 5), ENSURE_REQUIREMENTS, [id=#159] (209) HashAggregate [codegen id : 85] -Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] -Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#137)), sum(UnscaledValue(ss_list_price#138)), sum(UnscaledValue(ss_coupon_amt#139))] -Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#137))#200, sum(UnscaledValue(ss_list_price#138))#201, sum(UnscaledValue(ss_coupon_amt#139))#202] -Results [8]: [i_item_sk#188 AS item_sk#203, s_store_name#150 AS store_name#204, s_zip#151 AS store_zip#205, d_year#147 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#137))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#138))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#139))#202,17,2) AS s3#210] +Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] +Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count(1)#160, sum(UnscaledValue(ss_wholesale_cost#9))#161, sum(UnscaledValue(ss_list_price#10))#162, sum(UnscaledValue(ss_coupon_amt#11))#163] +Results [8]: [i_item_sk#93 AS item_sk#164, s_store_name#47 AS store_name#165, s_zip#48 AS store_zip#166, d_year#44 AS syear#167, count(1)#160 AS cnt#168, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#161,17,2) AS s1#169, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#162,17,2) AS s2#170, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#163,17,2) AS s3#171] (210) Exchange -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] +Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] +Arguments: hashpartitioning(item_sk#164, store_name#165, store_zip#166, 5), ENSURE_REQUIREMENTS, [id=#172] (211) Sort [codegen id : 86] -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] +Arguments: [item_sk#164 ASC NULLS FIRST, store_name#165 ASC NULLS FIRST, store_zip#166 ASC NULLS FIRST], false, 0 (212) SortMergeJoin [codegen id : 87] Left keys [3]: [item_sk#112, store_name#113, store_zip#114] -Right keys [3]: [item_sk#203, store_name#204, store_zip#205] -Join condition: (cnt#207 <= cnt#124) +Right keys [3]: [item_sk#164, store_name#165, store_zip#166] +Join condition: (cnt#168 <= cnt#124) (213) Project [codegen id : 87] -Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] -Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] +Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] (214) Exchange -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#208 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] +Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#169 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#173] (215) Sort [codegen id : 88] -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#208 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] +Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#169 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1152,11 +1152,11 @@ ReusedExchange (216) (216) ReusedExchange [Reuses operator id: 39] Output [2]: [d_date_sk#43, d_year#44] -Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#140 IN dynamicpruning#141 +Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#129 ReusedExchange (217) (217) ReusedExchange [Reuses operator id: 147] -Output [2]: [d_date_sk#146, d_year#147] +Output [2]: [d_date_sk#43, d_year#44] 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 0115f4ab8a63a..9424b7f1e2225 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 @@ -651,7 +651,7 @@ Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt# Output [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple ,burlywood ,indian ,spring ,floral ,medium ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [purple,burlywood,indian,spring,floral,medium]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] ReadSchema: struct (105) ColumnarToRow [codegen id : 24] @@ -659,7 +659,7 @@ Input [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] (106) Filter [codegen id : 24] Input [4]: [i_item_sk#87, i_current_price#88, i_color#89, i_product_name#90] -Condition : ((((((isnotnull(i_current_price#88) AND i_color#89 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#88 >= 64.00)) AND (i_current_price#88 <= 74.00)) AND (i_current_price#88 >= 65.00)) AND (i_current_price#88 <= 79.00)) AND isnotnull(i_item_sk#87)) +Condition : ((((((isnotnull(i_current_price#88) AND i_color#89 IN (purple,burlywood,indian,spring,floral,medium)) AND (i_current_price#88 >= 64.00)) AND (i_current_price#88 <= 74.00)) AND (i_current_price#88 >= 65.00)) AND (i_current_price#88 <= 79.00)) AND isnotnull(i_item_sk#87)) (107) Project [codegen id : 24] Output [2]: [i_item_sk#87, i_product_name#90] @@ -701,323 +701,323 @@ Input [17]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_str Arguments: [item_sk#105 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, store_zip#107 ASC NULLS FIRST], false, 0 (115) Scan parquet default.store_sales -Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#133), dynamicpruningexpression(ss_sold_date_sk#133 IN dynamicpruning#134)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#122)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (116) ColumnarToRow [codegen id : 27] -Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] (117) Filter [codegen id : 27] -Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Condition : (((((((isnotnull(ss_item_sk#122) AND isnotnull(ss_ticket_number#129)) AND isnotnull(ss_store_sk#127)) AND isnotnull(ss_customer_sk#123)) AND isnotnull(ss_cdemo_sk#124)) AND isnotnull(ss_promo_sk#128)) AND isnotnull(ss_hdemo_sk#125)) AND isnotnull(ss_addr_sk#126)) +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) (118) BroadcastExchange -Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#135] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#123] (119) Scan parquet default.store_returns -Output [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Output [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct (120) ColumnarToRow -Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] (121) Filter -Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] -Condition : (isnotnull(sr_item_sk#136) AND isnotnull(sr_ticket_number#137)) +Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Condition : (isnotnull(sr_item_sk#15) AND isnotnull(sr_ticket_number#16)) (122) Project -Output [2]: [sr_item_sk#136, sr_ticket_number#137] -Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Output [2]: [sr_item_sk#15, sr_ticket_number#16] +Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] (123) BroadcastHashJoin [codegen id : 28] -Left keys [2]: [cast(ss_item_sk#122 as bigint), cast(ss_ticket_number#129 as bigint)] -Right keys [2]: [sr_item_sk#136, sr_ticket_number#137] +Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] +Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] Join condition: None (124) Project [codegen id : 28] -Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, sr_item_sk#136, sr_ticket_number#137] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] (125) Exchange -Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Arguments: hashpartitioning(ss_item_sk#122, 5), ENSURE_REQUIREMENTS, [id=#139] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#124] (126) Sort [codegen id : 29] -Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Arguments: [ss_item_sk#122 ASC NULLS FIRST], false, 0 +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (127) ReusedExchange [Reuses operator id: 28] -Output [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] +Output [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] (128) HashAggregate [codegen id : 35] -Input [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] -Keys [1]: [cs_item_sk#140] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#144)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#144))#148, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149] -Results [3]: [cs_item_sk#140, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#144))#148,17,2) AS sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] +Input [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] +Keys [1]: [cs_item_sk#19] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#21)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#21))#128, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129] +Results [3]: [cs_item_sk#19, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#21))#128,17,2) AS sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] (129) Filter [codegen id : 35] -Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] -Condition : (isnotnull(sum(cs_ext_list_price#144)#150) AND (cast(sum(cs_ext_list_price#144)#150 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151)), DecimalType(21,2), true))) +Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] +Condition : (isnotnull(sum(cs_ext_list_price#21)#130) AND (cast(sum(cs_ext_list_price#21)#130 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131)), DecimalType(21,2), true))) (130) Project [codegen id : 35] -Output [1]: [cs_item_sk#140] -Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] +Output [1]: [cs_item_sk#19] +Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] (131) Sort [codegen id : 35] -Input [1]: [cs_item_sk#140] -Arguments: [cs_item_sk#140 ASC NULLS FIRST], false, 0 +Input [1]: [cs_item_sk#19] +Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 (132) SortMergeJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#122] -Right keys [1]: [cs_item_sk#140] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [cs_item_sk#19] Join condition: None (133) Project [codegen id : 51] -Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, cs_item_sk#140] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] (134) Scan parquet default.date_dim -Output [2]: [d_date_sk#152, d_year#153] +Output [2]: [d_date_sk#42, d_year#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (135) ColumnarToRow [codegen id : 36] -Input [2]: [d_date_sk#152, d_year#153] +Input [2]: [d_date_sk#42, d_year#43] (136) Filter [codegen id : 36] -Input [2]: [d_date_sk#152, d_year#153] -Condition : ((isnotnull(d_year#153) AND (d_year#153 = 2000)) AND isnotnull(d_date_sk#152)) +Input [2]: [d_date_sk#42, d_year#43] +Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2000)) AND isnotnull(d_date_sk#42)) (137) BroadcastExchange -Input [2]: [d_date_sk#152, d_year#153] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#154] +Input [2]: [d_date_sk#42, d_year#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#132] (138) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_sold_date_sk#133] -Right keys [1]: [d_date_sk#152] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#42] Join condition: None (139) Project [codegen id : 51] -Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153] -Input [13]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, d_date_sk#152, d_year#153] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#42, d_year#43] (140) ReusedExchange [Reuses operator id: 44] -Output [3]: [s_store_sk#155, s_store_name#156, s_zip#157] +Output [3]: [s_store_sk#45, s_store_name#46, s_zip#47] (141) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_store_sk#127] -Right keys [1]: [s_store_sk#155] +Left keys [1]: [ss_store_sk#6] +Right keys [1]: [s_store_sk#45] Join condition: None (142) Project [codegen id : 51] -Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157] -Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_sk#155, s_store_name#156, s_zip#157] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_sk#45, s_store_name#46, s_zip#47] (143) ReusedExchange [Reuses operator id: 50] -Output [6]: [c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] +Output [6]: [c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] (144) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_customer_sk#123] -Right keys [1]: [c_customer_sk#158] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#49] Join condition: None (145) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] -Input [18]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] (146) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#164, d_year#165] +Output [2]: [d_date_sk#133, d_year#134] (147) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_sales_date_sk#163] -Right keys [1]: [d_date_sk#164] +Left keys [1]: [c_first_sales_date_sk#54] +Right keys [1]: [d_date_sk#133] Join condition: None (148) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165] -Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163, d_date_sk#164, d_year#165] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54, d_date_sk#133, d_year#134] (149) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#166, d_year#167] +Output [2]: [d_date_sk#135, d_year#136] (150) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_shipto_date_sk#162] -Right keys [1]: [d_date_sk#166] +Left keys [1]: [c_first_shipto_date_sk#53] +Right keys [1]: [d_date_sk#135] Join condition: None (151) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] -Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165, d_date_sk#166, d_year#167] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134, d_date_sk#135, d_year#136] (152) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#168, cd_marital_status#169] +Output [2]: [cd_demo_sk#61, cd_marital_status#62] (153) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_cdemo_sk#124] -Right keys [1]: [cd_demo_sk#168] +Left keys [1]: [ss_cdemo_sk#3] +Right keys [1]: [cd_demo_sk#61] Join condition: None (154) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169] -Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_demo_sk#168, cd_marital_status#169] +Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_demo_sk#61, cd_marital_status#62] (155) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#170, cd_marital_status#171] +Output [2]: [cd_demo_sk#137, cd_marital_status#138] (156) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_cdemo_sk#159] -Right keys [1]: [cd_demo_sk#170] -Join condition: NOT (cd_marital_status#169 = cd_marital_status#171) +Left keys [1]: [c_current_cdemo_sk#50] +Right keys [1]: [cd_demo_sk#137] +Join condition: NOT (cd_marital_status#62 = cd_marital_status#138) (157) Project [codegen id : 51] -Output [14]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] -Input [18]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169, cd_demo_sk#170, cd_marital_status#171] +Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62, cd_demo_sk#137, cd_marital_status#138] (158) ReusedExchange [Reuses operator id: 74] -Output [1]: [p_promo_sk#172] +Output [1]: [p_promo_sk#66] (159) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_promo_sk#128] -Right keys [1]: [p_promo_sk#172] +Left keys [1]: [ss_promo_sk#7] +Right keys [1]: [p_promo_sk#66] Join condition: None (160) Project [codegen id : 51] -Output [13]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] -Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, p_promo_sk#172] +Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, p_promo_sk#66] (161) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#173, hd_income_band_sk#174] +Output [2]: [hd_demo_sk#68, hd_income_band_sk#69] (162) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_hdemo_sk#125] -Right keys [1]: [hd_demo_sk#173] +Left keys [1]: [ss_hdemo_sk#4] +Right keys [1]: [hd_demo_sk#68] Join condition: None (163) Project [codegen id : 51] -Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174] -Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_demo_sk#173, hd_income_band_sk#174] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_demo_sk#68, hd_income_band_sk#69] (164) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#175, hd_income_band_sk#176] +Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] (165) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_hdemo_sk#160] -Right keys [1]: [hd_demo_sk#175] +Left keys [1]: [c_current_hdemo_sk#51] +Right keys [1]: [hd_demo_sk#139] Join condition: None (166) Project [codegen id : 51] -Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176] -Input [15]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_demo_sk#175, hd_income_band_sk#176] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140] +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_demo_sk#139, hd_income_band_sk#140] (167) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] +Output [5]: [ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] (168) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_addr_sk#126] -Right keys [1]: [ca_address_sk#177] +Left keys [1]: [ss_addr_sk#5] +Right keys [1]: [ca_address_sk#73] Join condition: None (169) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] -Input [18]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] +Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] (170) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] (171) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_addr_sk#161] -Right keys [1]: [ca_address_sk#182] +Left keys [1]: [c_current_addr_sk#52] +Right keys [1]: [ca_address_sk#141] Join condition: None (172) Project [codegen id : 51] -Output [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] -Input [21]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] (173) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#187] +Output [1]: [ib_income_band_sk#84] (174) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#174] -Right keys [1]: [ib_income_band_sk#187] +Left keys [1]: [hd_income_band_sk#69] +Right keys [1]: [ib_income_band_sk#84] Join condition: None (175) Project [codegen id : 51] -Output [18]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] -Input [20]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#187] +Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#84] (176) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#188] +Output [1]: [ib_income_band_sk#146] (177) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#176] -Right keys [1]: [ib_income_band_sk#188] +Left keys [1]: [hd_income_band_sk#140] +Right keys [1]: [ib_income_band_sk#146] Join condition: None (178) Project [codegen id : 51] -Output [17]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] -Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#188] +Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#146] (179) ReusedExchange [Reuses operator id: 108] -Output [2]: [i_item_sk#189, i_product_name#190] +Output [2]: [i_item_sk#87, i_product_name#90] (180) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#122] -Right keys [1]: [i_item_sk#189] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#87] Join condition: None (181) Project [codegen id : 51] -Output [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] -Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] (182) HashAggregate [codegen id : 51] -Input [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] -Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#130)), partial_sum(UnscaledValue(ss_list_price#131)), partial_sum(UnscaledValue(ss_coupon_amt#132))] -Aggregate Attributes [4]: [count#191, sum#192, sum#193, sum#194] -Results [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] +Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count#147, sum#148, sum#149, sum#150] +Results [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] (183) HashAggregate [codegen id : 51] -Input [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] -Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#130)), sum(UnscaledValue(ss_list_price#131)), sum(UnscaledValue(ss_coupon_amt#132))] -Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#130))#200, sum(UnscaledValue(ss_list_price#131))#201, sum(UnscaledValue(ss_coupon_amt#132))#202] -Results [8]: [i_item_sk#189 AS item_sk#203, s_store_name#156 AS store_name#204, s_zip#157 AS store_zip#205, d_year#153 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#130))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#131))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#132))#202,17,2) AS s3#210] +Input [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] +Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count(1)#155, sum(UnscaledValue(ss_wholesale_cost#9))#156, sum(UnscaledValue(ss_list_price#10))#157, sum(UnscaledValue(ss_coupon_amt#11))#158] +Results [8]: [i_item_sk#87 AS item_sk#159, s_store_name#46 AS store_name#160, s_zip#47 AS store_zip#161, d_year#43 AS syear#162, count(1)#155 AS cnt#163, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#156,17,2) AS s1#164, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#157,17,2) AS s2#165, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#158,17,2) AS s3#166] (184) Exchange -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] +Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] +Arguments: hashpartitioning(item_sk#159, store_name#160, store_zip#161, 5), ENSURE_REQUIREMENTS, [id=#167] (185) Sort [codegen id : 52] -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] +Arguments: [item_sk#159 ASC NULLS FIRST, store_name#160 ASC NULLS FIRST, store_zip#161 ASC NULLS FIRST], false, 0 (186) SortMergeJoin [codegen id : 53] Left keys [3]: [item_sk#105, store_name#106, store_zip#107] -Right keys [3]: [item_sk#203, store_name#204, store_zip#205] -Join condition: (cnt#207 <= cnt#117) +Right keys [3]: [item_sk#159, store_name#160, store_zip#161] +Join condition: (cnt#163 <= cnt#117) (187) Project [codegen id : 53] -Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] -Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] +Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] (188) Exchange -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#208 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] +Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#164 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#168] (189) Sort [codegen id : 54] -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#208 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] +Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#164 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1028,11 +1028,11 @@ ReusedExchange (190) (190) ReusedExchange [Reuses operator id: 38] Output [2]: [d_date_sk#42, d_year#43] -Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#133 IN dynamicpruning#134 +Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#122 ReusedExchange (191) (191) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#152, d_year#153] +Output [2]: [d_date_sk#42, d_year#43] 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 13bcda00cb11d..6d9adf1d38106 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 @@ -206,234 +206,234 @@ Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 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))#28 as decimal(38,2)) AS sumsales#29] (29) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, s_store_id#37, sum#38, isEmpty#39] +Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#30, isEmpty#31] (30) HashAggregate [codegen id : 16] -Input [10]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, s_store_id#37, sum#38, isEmpty#39] -Keys [8]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, s_store_id#37] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#40 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#41 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#40 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#41 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#42] -Results [8]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#40 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#41 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#42 AS sumsales#43] +Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#30, isEmpty#31] +Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] +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))#32] +Results [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#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))#32 AS sumsales#33] (31) HashAggregate [codegen id : 16] -Input [8]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sumsales#43] -Keys [7]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36] -Functions [1]: [partial_sum(sumsales#43)] -Aggregate Attributes [2]: [sum#44, isEmpty#45] -Results [9]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sum#46, isEmpty#47] +Input [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, sumsales#33] +Keys [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10] +Functions [1]: [partial_sum(sumsales#33)] +Aggregate Attributes [2]: [sum#34, isEmpty#35] +Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, sum#36, isEmpty#37] (32) Exchange -Input [9]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sum#46, isEmpty#47] -Arguments: hashpartitioning(i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, sum#36, isEmpty#37] +Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, 5), ENSURE_REQUIREMENTS, [id=#38] (33) HashAggregate [codegen id : 17] -Input [9]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sum#46, isEmpty#47] -Keys [7]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36] -Functions [1]: [sum(sumsales#43)] -Aggregate Attributes [1]: [sum(sumsales#43)#49] -Results [9]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, null AS s_store_id#50, sum(sumsales#43)#49 AS sumsales#51] +Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, sum#36, isEmpty#37] +Keys [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10] +Functions [1]: [sum(sumsales#33)] +Aggregate Attributes [1]: [sum(sumsales#33)#39] +Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, null AS s_store_id#40, sum(sumsales#33)#39 AS sumsales#41] (34) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] +Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#42, isEmpty#43] (35) HashAggregate [codegen id : 25] -Input [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] -Keys [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#62 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#63 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#62 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#63 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#64] -Results [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#62 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#63 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#64 AS sumsales#43] +Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#42, isEmpty#43] +Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] +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))#44] +Results [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, 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))#44 AS sumsales#33] (36) HashAggregate [codegen id : 25] -Input [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sumsales#43] -Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] -Functions [1]: [partial_sum(sumsales#43)] -Aggregate Attributes [2]: [sum#65, isEmpty#66] -Results [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Input [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, sumsales#33] +Keys [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11] +Functions [1]: [partial_sum(sumsales#33)] +Aggregate Attributes [2]: [sum#45, isEmpty#46] +Results [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, sum#47, isEmpty#48] (37) Exchange -Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] -Arguments: hashpartitioning(i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, 5), ENSURE_REQUIREMENTS, [id=#49] (38) HashAggregate [codegen id : 26] -Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] -Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] -Functions [1]: [sum(sumsales#43)] -Aggregate Attributes [1]: [sum(sumsales#43)#70] -Results [9]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, null AS d_moy#71, null AS s_store_id#72, sum(sumsales#43)#70 AS sumsales#73] +Input [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, sum#47, isEmpty#48] +Keys [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11] +Functions [1]: [sum(sumsales#33)] +Aggregate Attributes [1]: [sum(sumsales#33)#50] +Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, null AS d_moy#51, null AS s_store_id#52, sum(sumsales#33)#50 AS sumsales#53] (39) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81, sum#82, isEmpty#83] +Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#54, isEmpty#55] (40) HashAggregate [codegen id : 34] -Input [10]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81, sum#82, isEmpty#83] -Keys [8]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#84 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#85 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#84 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#86] -Results [6]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#84 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#86 AS sumsales#43] +Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#54, isEmpty#55] +Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] +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))#56] +Results [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#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))#56 AS sumsales#33] (41) HashAggregate [codegen id : 34] -Input [6]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sumsales#43] -Keys [5]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78] -Functions [1]: [partial_sum(sumsales#43)] -Aggregate Attributes [2]: [sum#87, isEmpty#88] -Results [7]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum#89, isEmpty#90] +Input [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, sumsales#33] +Keys [5]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9] +Functions [1]: [partial_sum(sumsales#33)] +Aggregate Attributes [2]: [sum#57, isEmpty#58] +Results [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, sum#59, isEmpty#60] (42) Exchange -Input [7]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum#89, isEmpty#90] -Arguments: hashpartitioning(i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, sum#59, isEmpty#60] +Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#61] (43) HashAggregate [codegen id : 35] -Input [7]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum#89, isEmpty#90] -Keys [5]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78] -Functions [1]: [sum(sumsales#43)] -Aggregate Attributes [1]: [sum(sumsales#43)#92] -Results [9]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, null AS d_qoy#93, null AS d_moy#94, null AS s_store_id#95, sum(sumsales#43)#92 AS sumsales#96] +Input [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, sum#59, isEmpty#60] +Keys [5]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9] +Functions [1]: [sum(sumsales#33)] +Aggregate Attributes [1]: [sum(sumsales#33)#62] +Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, null AS d_qoy#63, null AS d_moy#64, null AS s_store_id#65, sum(sumsales#33)#62 AS sumsales#66] (44) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, d_year#101, d_qoy#102, d_moy#103, s_store_id#104, sum#105, isEmpty#106] +Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#67, isEmpty#68] (45) HashAggregate [codegen id : 43] -Input [10]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, d_year#101, d_qoy#102, d_moy#103, s_store_id#104, sum#105, isEmpty#106] -Keys [8]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, d_year#101, d_qoy#102, d_moy#103, s_store_id#104] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#107 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#108 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#107 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#108 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#109] -Results [5]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#107 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#108 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#109 AS sumsales#43] +Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#67, isEmpty#68] +Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] +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))#69] +Results [5]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, 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))#69 AS sumsales#33] (46) HashAggregate [codegen id : 43] -Input [5]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sumsales#43] -Keys [4]: [i_category#97, i_class#98, i_brand#99, i_product_name#100] -Functions [1]: [partial_sum(sumsales#43)] -Aggregate Attributes [2]: [sum#110, isEmpty#111] -Results [6]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sum#112, isEmpty#113] +Input [5]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, sumsales#33] +Keys [4]: [i_category#20, i_class#19, i_brand#18, i_product_name#21] +Functions [1]: [partial_sum(sumsales#33)] +Aggregate Attributes [2]: [sum#70, isEmpty#71] +Results [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, sum#72, isEmpty#73] (47) Exchange -Input [6]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sum#112, isEmpty#113] -Arguments: hashpartitioning(i_category#97, i_class#98, i_brand#99, i_product_name#100, 5), ENSURE_REQUIREMENTS, [id=#114] +Input [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, sum#72, isEmpty#73] +Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, i_product_name#21, 5), ENSURE_REQUIREMENTS, [id=#74] (48) HashAggregate [codegen id : 44] -Input [6]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sum#112, isEmpty#113] -Keys [4]: [i_category#97, i_class#98, i_brand#99, i_product_name#100] -Functions [1]: [sum(sumsales#43)] -Aggregate Attributes [1]: [sum(sumsales#43)#115] -Results [9]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, null AS d_year#116, null AS d_qoy#117, null AS d_moy#118, null AS s_store_id#119, sum(sumsales#43)#115 AS sumsales#120] +Input [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, sum#72, isEmpty#73] +Keys [4]: [i_category#20, i_class#19, i_brand#18, i_product_name#21] +Functions [1]: [sum(sumsales#33)] +Aggregate Attributes [1]: [sum(sumsales#33)#75] +Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, null AS d_year#76, null AS d_qoy#77, null AS d_moy#78, null AS s_store_id#79, sum(sumsales#33)#75 AS sumsales#80] (49) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#121, i_class#122, i_brand#123, i_product_name#124, d_year#125, d_qoy#126, d_moy#127, s_store_id#128, sum#129, isEmpty#130] +Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#81, isEmpty#82] (50) HashAggregate [codegen id : 52] -Input [10]: [i_category#121, i_class#122, i_brand#123, i_product_name#124, d_year#125, d_qoy#126, d_moy#127, s_store_id#128, sum#129, isEmpty#130] -Keys [8]: [i_category#121, i_class#122, i_brand#123, i_product_name#124, d_year#125, d_qoy#126, d_moy#127, s_store_id#128] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#131 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#132 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#131 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#132 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#133] -Results [4]: [i_category#121, i_class#122, i_brand#123, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#131 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#132 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#133 AS sumsales#43] +Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#81, isEmpty#82] +Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] +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))#83] +Results [4]: [i_category#20, i_class#19, i_brand#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))#83 AS sumsales#33] (51) HashAggregate [codegen id : 52] -Input [4]: [i_category#121, i_class#122, i_brand#123, sumsales#43] -Keys [3]: [i_category#121, i_class#122, i_brand#123] -Functions [1]: [partial_sum(sumsales#43)] -Aggregate Attributes [2]: [sum#134, isEmpty#135] -Results [5]: [i_category#121, i_class#122, i_brand#123, sum#136, isEmpty#137] +Input [4]: [i_category#20, i_class#19, i_brand#18, sumsales#33] +Keys [3]: [i_category#20, i_class#19, i_brand#18] +Functions [1]: [partial_sum(sumsales#33)] +Aggregate Attributes [2]: [sum#84, isEmpty#85] +Results [5]: [i_category#20, i_class#19, i_brand#18, sum#86, isEmpty#87] (52) Exchange -Input [5]: [i_category#121, i_class#122, i_brand#123, sum#136, isEmpty#137] -Arguments: hashpartitioning(i_category#121, i_class#122, i_brand#123, 5), ENSURE_REQUIREMENTS, [id=#138] +Input [5]: [i_category#20, i_class#19, i_brand#18, sum#86, isEmpty#87] +Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, 5), ENSURE_REQUIREMENTS, [id=#88] (53) HashAggregate [codegen id : 53] -Input [5]: [i_category#121, i_class#122, i_brand#123, sum#136, isEmpty#137] -Keys [3]: [i_category#121, i_class#122, i_brand#123] -Functions [1]: [sum(sumsales#43)] -Aggregate Attributes [1]: [sum(sumsales#43)#139] -Results [9]: [i_category#121, i_class#122, i_brand#123, null AS i_product_name#140, null AS d_year#141, null AS d_qoy#142, null AS d_moy#143, null AS s_store_id#144, sum(sumsales#43)#139 AS sumsales#145] +Input [5]: [i_category#20, i_class#19, i_brand#18, sum#86, isEmpty#87] +Keys [3]: [i_category#20, i_class#19, i_brand#18] +Functions [1]: [sum(sumsales#33)] +Aggregate Attributes [1]: [sum(sumsales#33)#89] +Results [9]: [i_category#20, i_class#19, i_brand#18, null AS i_product_name#90, null AS d_year#91, null AS d_qoy#92, null AS d_moy#93, null AS s_store_id#94, sum(sumsales#33)#89 AS sumsales#95] (54) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sum#154, isEmpty#155] +Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#96, isEmpty#97] (55) HashAggregate [codegen id : 61] -Input [10]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sum#154, isEmpty#155] -Keys [8]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#156 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#157 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#156 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#157 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#158] -Results [3]: [i_category#146, i_class#147, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#156 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#157 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#158 AS sumsales#43] +Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#96, isEmpty#97] +Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] +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))#98] +Results [3]: [i_category#20, i_class#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))#98 AS sumsales#33] (56) HashAggregate [codegen id : 61] -Input [3]: [i_category#146, i_class#147, sumsales#43] -Keys [2]: [i_category#146, i_class#147] -Functions [1]: [partial_sum(sumsales#43)] -Aggregate Attributes [2]: [sum#159, isEmpty#160] -Results [4]: [i_category#146, i_class#147, sum#161, isEmpty#162] +Input [3]: [i_category#20, i_class#19, sumsales#33] +Keys [2]: [i_category#20, i_class#19] +Functions [1]: [partial_sum(sumsales#33)] +Aggregate Attributes [2]: [sum#99, isEmpty#100] +Results [4]: [i_category#20, i_class#19, sum#101, isEmpty#102] (57) Exchange -Input [4]: [i_category#146, i_class#147, sum#161, isEmpty#162] -Arguments: hashpartitioning(i_category#146, i_class#147, 5), ENSURE_REQUIREMENTS, [id=#163] +Input [4]: [i_category#20, i_class#19, sum#101, isEmpty#102] +Arguments: hashpartitioning(i_category#20, i_class#19, 5), ENSURE_REQUIREMENTS, [id=#103] (58) HashAggregate [codegen id : 62] -Input [4]: [i_category#146, i_class#147, sum#161, isEmpty#162] -Keys [2]: [i_category#146, i_class#147] -Functions [1]: [sum(sumsales#43)] -Aggregate Attributes [1]: [sum(sumsales#43)#164] -Results [9]: [i_category#146, i_class#147, null AS i_brand#165, null AS i_product_name#166, null AS d_year#167, null AS d_qoy#168, null AS d_moy#169, null AS s_store_id#170, sum(sumsales#43)#164 AS sumsales#171] +Input [4]: [i_category#20, i_class#19, sum#101, isEmpty#102] +Keys [2]: [i_category#20, i_class#19] +Functions [1]: [sum(sumsales#33)] +Aggregate Attributes [1]: [sum(sumsales#33)#104] +Results [9]: [i_category#20, i_class#19, null AS i_brand#105, null AS i_product_name#106, null AS d_year#107, null AS d_qoy#108, null AS d_moy#109, null AS s_store_id#110, sum(sumsales#33)#104 AS sumsales#111] (59) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179, sum#180, isEmpty#181] +Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#112, isEmpty#113] (60) HashAggregate [codegen id : 70] -Input [10]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179, sum#180, isEmpty#181] -Keys [8]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#182 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#183 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#182 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#183 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#184] -Results [2]: [i_category#172, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#182 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#183 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#184 AS sumsales#43] +Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#112, isEmpty#113] +Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] +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))#114] +Results [2]: [i_category#20, 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))#114 AS sumsales#33] (61) HashAggregate [codegen id : 70] -Input [2]: [i_category#172, sumsales#43] -Keys [1]: [i_category#172] -Functions [1]: [partial_sum(sumsales#43)] -Aggregate Attributes [2]: [sum#185, isEmpty#186] -Results [3]: [i_category#172, sum#187, isEmpty#188] +Input [2]: [i_category#20, sumsales#33] +Keys [1]: [i_category#20] +Functions [1]: [partial_sum(sumsales#33)] +Aggregate Attributes [2]: [sum#115, isEmpty#116] +Results [3]: [i_category#20, sum#117, isEmpty#118] (62) Exchange -Input [3]: [i_category#172, sum#187, isEmpty#188] -Arguments: hashpartitioning(i_category#172, 5), ENSURE_REQUIREMENTS, [id=#189] +Input [3]: [i_category#20, sum#117, isEmpty#118] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#119] (63) HashAggregate [codegen id : 71] -Input [3]: [i_category#172, sum#187, isEmpty#188] -Keys [1]: [i_category#172] -Functions [1]: [sum(sumsales#43)] -Aggregate Attributes [1]: [sum(sumsales#43)#190] -Results [9]: [i_category#172, null AS i_class#191, null AS i_brand#192, null AS i_product_name#193, null AS d_year#194, null AS d_qoy#195, null AS d_moy#196, null AS s_store_id#197, sum(sumsales#43)#190 AS sumsales#198] +Input [3]: [i_category#20, sum#117, isEmpty#118] +Keys [1]: [i_category#20] +Functions [1]: [sum(sumsales#33)] +Aggregate Attributes [1]: [sum(sumsales#33)#120] +Results [9]: [i_category#20, null AS i_class#121, null AS i_brand#122, null AS i_product_name#123, null AS d_year#124, null AS d_qoy#125, null AS d_moy#126, null AS s_store_id#127, sum(sumsales#33)#120 AS sumsales#128] (64) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#199, i_class#200, i_brand#201, i_product_name#202, d_year#203, d_qoy#204, d_moy#205, s_store_id#206, sum#207, isEmpty#208] +Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#129, isEmpty#130] (65) HashAggregate [codegen id : 79] -Input [10]: [i_category#199, i_class#200, i_brand#201, i_product_name#202, d_year#203, d_qoy#204, d_moy#205, s_store_id#206, sum#207, isEmpty#208] -Keys [8]: [i_category#199, i_class#200, i_brand#201, i_product_name#202, d_year#203, d_qoy#204, d_moy#205, s_store_id#206] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#209 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#210 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#209 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#210 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#211] -Results [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#209 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#210 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#211 AS sumsales#43] +Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#129, isEmpty#130] +Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] +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))#131] +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))#131 AS sumsales#33] (66) HashAggregate [codegen id : 79] -Input [1]: [sumsales#43] +Input [1]: [sumsales#33] Keys: [] -Functions [1]: [partial_sum(sumsales#43)] -Aggregate Attributes [2]: [sum#212, isEmpty#213] -Results [2]: [sum#214, isEmpty#215] +Functions [1]: [partial_sum(sumsales#33)] +Aggregate Attributes [2]: [sum#132, isEmpty#133] +Results [2]: [sum#134, isEmpty#135] (67) Exchange -Input [2]: [sum#214, isEmpty#215] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#216] +Input [2]: [sum#134, isEmpty#135] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#136] (68) HashAggregate [codegen id : 80] -Input [2]: [sum#214, isEmpty#215] +Input [2]: [sum#134, isEmpty#135] Keys: [] -Functions [1]: [sum(sumsales#43)] -Aggregate Attributes [1]: [sum(sumsales#43)#217] -Results [9]: [null AS i_category#218, null AS i_class#219, null AS i_brand#220, null AS i_product_name#221, null AS d_year#222, null AS d_qoy#223, null AS d_moy#224, null AS s_store_id#225, sum(sumsales#43)#217 AS sumsales#226] +Functions [1]: [sum(sumsales#33)] +Aggregate Attributes [1]: [sum(sumsales#33)#137] +Results [9]: [null AS i_category#138, null AS i_class#139, null AS i_brand#140, null AS i_product_name#141, null AS d_year#142, null AS d_qoy#143, null AS d_moy#144, null AS s_store_id#145, sum(sumsales#33)#137 AS sumsales#146] (69) Union (70) Exchange Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#227] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#147] (71) Sort [codegen id : 81] Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29] @@ -441,15 +441,15 @@ Arguments: [i_category#20 ASC NULLS FIRST, sumsales#29 DESC NULLS LAST], false, (72) Window Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29] -Arguments: [rank(sumsales#29) windowspecdefinition(i_category#20, sumsales#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#228], [i_category#20], [sumsales#29 DESC NULLS LAST] +Arguments: [rank(sumsales#29) windowspecdefinition(i_category#20, sumsales#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#148], [i_category#20], [sumsales#29 DESC NULLS LAST] (73) Filter [codegen id : 82] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#228] -Condition : (isnotnull(rk#228) AND (rk#228 <= 100)) +Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] +Condition : (isnotnull(rk#148) AND (rk#148 <= 100)) (74) TakeOrderedAndProject -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#228] -Arguments: 100, [i_category#20 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#18 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#14 ASC NULLS FIRST, sumsales#29 ASC NULLS FIRST, rk#228 ASC NULLS FIRST], [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#228] +Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] +Arguments: 100, [i_category#20 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#18 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#14 ASC NULLS FIRST, sumsales#29 ASC NULLS FIRST, rk#148 ASC NULLS FIRST], [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] ===== Subqueries ===== 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 ffa5e96fad67e..ae6b3ff4d7542 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 @@ -191,234 +191,234 @@ Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 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))#27 as decimal(38,2)) AS sumsales#28] (26) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, s_store_id#36, sum#37, isEmpty#38] +Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#29, isEmpty#30] (27) HashAggregate [codegen id : 10] -Input [10]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, s_store_id#36, sum#37, isEmpty#38] -Keys [8]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, s_store_id#36] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#39 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#40 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#39 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#40 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#41] -Results [8]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#39 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#40 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#41 AS sumsales#42] +Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#29, isEmpty#30] +Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] +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))#31] +Results [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#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))#31 AS sumsales#32] (28) HashAggregate [codegen id : 10] -Input [8]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sumsales#42] -Keys [7]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35] -Functions [1]: [partial_sum(sumsales#42)] -Aggregate Attributes [2]: [sum#43, isEmpty#44] -Results [9]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sum#45, isEmpty#46] +Input [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, sumsales#32] +Keys [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10] +Functions [1]: [partial_sum(sumsales#32)] +Aggregate Attributes [2]: [sum#33, isEmpty#34] +Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, sum#35, isEmpty#36] (29) Exchange -Input [9]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sum#45, isEmpty#46] -Arguments: hashpartitioning(i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, sum#35, isEmpty#36] +Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, 5), ENSURE_REQUIREMENTS, [id=#37] (30) HashAggregate [codegen id : 11] -Input [9]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sum#45, isEmpty#46] -Keys [7]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35] -Functions [1]: [sum(sumsales#42)] -Aggregate Attributes [1]: [sum(sumsales#42)#48] -Results [9]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, null AS s_store_id#49, sum(sumsales#42)#48 AS sumsales#50] +Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, sum#35, isEmpty#36] +Keys [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10] +Functions [1]: [sum(sumsales#32)] +Aggregate Attributes [1]: [sum(sumsales#32)#38] +Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, null AS s_store_id#39, sum(sumsales#32)#38 AS sumsales#40] (31) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, d_moy#57, s_store_id#58, sum#59, isEmpty#60] +Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#41, isEmpty#42] (32) HashAggregate [codegen id : 16] -Input [10]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, d_moy#57, s_store_id#58, sum#59, isEmpty#60] -Keys [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, d_moy#57, s_store_id#58] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#61 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#62 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#61 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#62 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#63] -Results [7]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#61 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#62 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#63 AS sumsales#42] +Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#41, isEmpty#42] +Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] +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))#43] +Results [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, 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))#43 AS sumsales#32] (33) HashAggregate [codegen id : 16] -Input [7]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sumsales#42] -Keys [6]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56] -Functions [1]: [partial_sum(sumsales#42)] -Aggregate Attributes [2]: [sum#64, isEmpty#65] -Results [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sum#66, isEmpty#67] +Input [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, sumsales#32] +Keys [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11] +Functions [1]: [partial_sum(sumsales#32)] +Aggregate Attributes [2]: [sum#44, isEmpty#45] +Results [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, sum#46, isEmpty#47] (34) Exchange -Input [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sum#66, isEmpty#67] -Arguments: hashpartitioning(i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, sum#46, isEmpty#47] +Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, 5), ENSURE_REQUIREMENTS, [id=#48] (35) HashAggregate [codegen id : 17] -Input [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sum#66, isEmpty#67] -Keys [6]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56] -Functions [1]: [sum(sumsales#42)] -Aggregate Attributes [1]: [sum(sumsales#42)#69] -Results [9]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, null AS d_moy#70, null AS s_store_id#71, sum(sumsales#42)#69 AS sumsales#72] +Input [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, sum#46, isEmpty#47] +Keys [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11] +Functions [1]: [sum(sumsales#32)] +Aggregate Attributes [1]: [sum(sumsales#32)#49] +Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, null AS d_moy#50, null AS s_store_id#51, sum(sumsales#32)#49 AS sumsales#52] (36) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] +Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#53, isEmpty#54] (37) HashAggregate [codegen id : 22] -Input [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] -Keys [8]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#83 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#84 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#83 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#85] -Results [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#83 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#85 AS sumsales#42] +Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#53, isEmpty#54] +Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] +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))#55] +Results [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#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))#55 AS sumsales#32] (38) HashAggregate [codegen id : 22] -Input [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sumsales#42] -Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] -Functions [1]: [partial_sum(sumsales#42)] -Aggregate Attributes [2]: [sum#86, isEmpty#87] -Results [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Input [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, sumsales#32] +Keys [5]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9] +Functions [1]: [partial_sum(sumsales#32)] +Aggregate Attributes [2]: [sum#56, isEmpty#57] +Results [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, sum#58, isEmpty#59] (39) Exchange -Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] -Arguments: hashpartitioning(i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, sum#58, isEmpty#59] +Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#60] (40) HashAggregate [codegen id : 23] -Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] -Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] -Functions [1]: [sum(sumsales#42)] -Aggregate Attributes [1]: [sum(sumsales#42)#91] -Results [9]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, null AS d_qoy#92, null AS d_moy#93, null AS s_store_id#94, sum(sumsales#42)#91 AS sumsales#95] +Input [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, sum#58, isEmpty#59] +Keys [5]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9] +Functions [1]: [sum(sumsales#32)] +Aggregate Attributes [1]: [sum(sumsales#32)#61] +Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, null AS d_qoy#62, null AS d_moy#63, null AS s_store_id#64, sum(sumsales#32)#61 AS sumsales#65] (41) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, d_year#100, d_qoy#101, d_moy#102, s_store_id#103, sum#104, isEmpty#105] +Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#66, isEmpty#67] (42) HashAggregate [codegen id : 28] -Input [10]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, d_year#100, d_qoy#101, d_moy#102, s_store_id#103, sum#104, isEmpty#105] -Keys [8]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, d_year#100, d_qoy#101, d_moy#102, s_store_id#103] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#106 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#107 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#106 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#107 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#108] -Results [5]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#106 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#107 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#108 AS sumsales#42] +Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#66, isEmpty#67] +Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] +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))#68] +Results [5]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, 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))#68 AS sumsales#32] (43) HashAggregate [codegen id : 28] -Input [5]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sumsales#42] -Keys [4]: [i_category#96, i_class#97, i_brand#98, i_product_name#99] -Functions [1]: [partial_sum(sumsales#42)] -Aggregate Attributes [2]: [sum#109, isEmpty#110] -Results [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] +Input [5]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, sumsales#32] +Keys [4]: [i_category#19, i_class#18, i_brand#17, i_product_name#20] +Functions [1]: [partial_sum(sumsales#32)] +Aggregate Attributes [2]: [sum#69, isEmpty#70] +Results [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, sum#71, isEmpty#72] (44) Exchange -Input [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] -Arguments: hashpartitioning(i_category#96, i_class#97, i_brand#98, i_product_name#99, 5), ENSURE_REQUIREMENTS, [id=#113] +Input [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, sum#71, isEmpty#72] +Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, i_product_name#20, 5), ENSURE_REQUIREMENTS, [id=#73] (45) HashAggregate [codegen id : 29] -Input [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] -Keys [4]: [i_category#96, i_class#97, i_brand#98, i_product_name#99] -Functions [1]: [sum(sumsales#42)] -Aggregate Attributes [1]: [sum(sumsales#42)#114] -Results [9]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, null AS d_year#115, null AS d_qoy#116, null AS d_moy#117, null AS s_store_id#118, sum(sumsales#42)#114 AS sumsales#119] +Input [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, sum#71, isEmpty#72] +Keys [4]: [i_category#19, i_class#18, i_brand#17, i_product_name#20] +Functions [1]: [sum(sumsales#32)] +Aggregate Attributes [1]: [sum(sumsales#32)#74] +Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, null AS d_year#75, null AS d_qoy#76, null AS d_moy#77, null AS s_store_id#78, sum(sumsales#32)#74 AS sumsales#79] (46) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#120, i_class#121, i_brand#122, i_product_name#123, d_year#124, d_qoy#125, d_moy#126, s_store_id#127, sum#128, isEmpty#129] +Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#80, isEmpty#81] (47) HashAggregate [codegen id : 34] -Input [10]: [i_category#120, i_class#121, i_brand#122, i_product_name#123, d_year#124, d_qoy#125, d_moy#126, s_store_id#127, sum#128, isEmpty#129] -Keys [8]: [i_category#120, i_class#121, i_brand#122, i_product_name#123, d_year#124, d_qoy#125, d_moy#126, s_store_id#127] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#130 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#131 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#130 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#131 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#132] -Results [4]: [i_category#120, i_class#121, i_brand#122, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#130 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#131 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#132 AS sumsales#42] +Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#80, isEmpty#81] +Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] +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))#82] +Results [4]: [i_category#19, i_class#18, i_brand#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))#82 AS sumsales#32] (48) HashAggregate [codegen id : 34] -Input [4]: [i_category#120, i_class#121, i_brand#122, sumsales#42] -Keys [3]: [i_category#120, i_class#121, i_brand#122] -Functions [1]: [partial_sum(sumsales#42)] -Aggregate Attributes [2]: [sum#133, isEmpty#134] -Results [5]: [i_category#120, i_class#121, i_brand#122, sum#135, isEmpty#136] +Input [4]: [i_category#19, i_class#18, i_brand#17, sumsales#32] +Keys [3]: [i_category#19, i_class#18, i_brand#17] +Functions [1]: [partial_sum(sumsales#32)] +Aggregate Attributes [2]: [sum#83, isEmpty#84] +Results [5]: [i_category#19, i_class#18, i_brand#17, sum#85, isEmpty#86] (49) Exchange -Input [5]: [i_category#120, i_class#121, i_brand#122, sum#135, isEmpty#136] -Arguments: hashpartitioning(i_category#120, i_class#121, i_brand#122, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [5]: [i_category#19, i_class#18, i_brand#17, sum#85, isEmpty#86] +Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, 5), ENSURE_REQUIREMENTS, [id=#87] (50) HashAggregate [codegen id : 35] -Input [5]: [i_category#120, i_class#121, i_brand#122, sum#135, isEmpty#136] -Keys [3]: [i_category#120, i_class#121, i_brand#122] -Functions [1]: [sum(sumsales#42)] -Aggregate Attributes [1]: [sum(sumsales#42)#138] -Results [9]: [i_category#120, i_class#121, i_brand#122, null AS i_product_name#139, null AS d_year#140, null AS d_qoy#141, null AS d_moy#142, null AS s_store_id#143, sum(sumsales#42)#138 AS sumsales#144] +Input [5]: [i_category#19, i_class#18, i_brand#17, sum#85, isEmpty#86] +Keys [3]: [i_category#19, i_class#18, i_brand#17] +Functions [1]: [sum(sumsales#32)] +Aggregate Attributes [1]: [sum(sumsales#32)#88] +Results [9]: [i_category#19, i_class#18, i_brand#17, null AS i_product_name#89, null AS d_year#90, null AS d_qoy#91, null AS d_moy#92, null AS s_store_id#93, sum(sumsales#32)#88 AS sumsales#94] (51) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#145, i_class#146, i_brand#147, i_product_name#148, d_year#149, d_qoy#150, d_moy#151, s_store_id#152, sum#153, isEmpty#154] +Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#95, isEmpty#96] (52) HashAggregate [codegen id : 40] -Input [10]: [i_category#145, i_class#146, i_brand#147, i_product_name#148, d_year#149, d_qoy#150, d_moy#151, s_store_id#152, sum#153, isEmpty#154] -Keys [8]: [i_category#145, i_class#146, i_brand#147, i_product_name#148, d_year#149, d_qoy#150, d_moy#151, s_store_id#152] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#155 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#156 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#155 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#156 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#157] -Results [3]: [i_category#145, i_class#146, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#155 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#156 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#157 AS sumsales#42] +Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#95, isEmpty#96] +Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] +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))#97] +Results [3]: [i_category#19, i_class#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))#97 AS sumsales#32] (53) HashAggregate [codegen id : 40] -Input [3]: [i_category#145, i_class#146, sumsales#42] -Keys [2]: [i_category#145, i_class#146] -Functions [1]: [partial_sum(sumsales#42)] -Aggregate Attributes [2]: [sum#158, isEmpty#159] -Results [4]: [i_category#145, i_class#146, sum#160, isEmpty#161] +Input [3]: [i_category#19, i_class#18, sumsales#32] +Keys [2]: [i_category#19, i_class#18] +Functions [1]: [partial_sum(sumsales#32)] +Aggregate Attributes [2]: [sum#98, isEmpty#99] +Results [4]: [i_category#19, i_class#18, sum#100, isEmpty#101] (54) Exchange -Input [4]: [i_category#145, i_class#146, sum#160, isEmpty#161] -Arguments: hashpartitioning(i_category#145, i_class#146, 5), ENSURE_REQUIREMENTS, [id=#162] +Input [4]: [i_category#19, i_class#18, sum#100, isEmpty#101] +Arguments: hashpartitioning(i_category#19, i_class#18, 5), ENSURE_REQUIREMENTS, [id=#102] (55) HashAggregate [codegen id : 41] -Input [4]: [i_category#145, i_class#146, sum#160, isEmpty#161] -Keys [2]: [i_category#145, i_class#146] -Functions [1]: [sum(sumsales#42)] -Aggregate Attributes [1]: [sum(sumsales#42)#163] -Results [9]: [i_category#145, i_class#146, null AS i_brand#164, null AS i_product_name#165, null AS d_year#166, null AS d_qoy#167, null AS d_moy#168, null AS s_store_id#169, sum(sumsales#42)#163 AS sumsales#170] +Input [4]: [i_category#19, i_class#18, sum#100, isEmpty#101] +Keys [2]: [i_category#19, i_class#18] +Functions [1]: [sum(sumsales#32)] +Aggregate Attributes [1]: [sum(sumsales#32)#103] +Results [9]: [i_category#19, i_class#18, null AS i_brand#104, null AS i_product_name#105, null AS d_year#106, null AS d_qoy#107, null AS d_moy#108, null AS s_store_id#109, sum(sumsales#32)#103 AS sumsales#110] (56) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#171, i_class#172, i_brand#173, i_product_name#174, d_year#175, d_qoy#176, d_moy#177, s_store_id#178, sum#179, isEmpty#180] +Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#111, isEmpty#112] (57) HashAggregate [codegen id : 46] -Input [10]: [i_category#171, i_class#172, i_brand#173, i_product_name#174, d_year#175, d_qoy#176, d_moy#177, s_store_id#178, sum#179, isEmpty#180] -Keys [8]: [i_category#171, i_class#172, i_brand#173, i_product_name#174, d_year#175, d_qoy#176, d_moy#177, s_store_id#178] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#181 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#182 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#181 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#182 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#183] -Results [2]: [i_category#171, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#181 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#182 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#183 AS sumsales#42] +Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#111, isEmpty#112] +Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] +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))#113] +Results [2]: [i_category#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))#113 AS sumsales#32] (58) HashAggregate [codegen id : 46] -Input [2]: [i_category#171, sumsales#42] -Keys [1]: [i_category#171] -Functions [1]: [partial_sum(sumsales#42)] -Aggregate Attributes [2]: [sum#184, isEmpty#185] -Results [3]: [i_category#171, sum#186, isEmpty#187] +Input [2]: [i_category#19, sumsales#32] +Keys [1]: [i_category#19] +Functions [1]: [partial_sum(sumsales#32)] +Aggregate Attributes [2]: [sum#114, isEmpty#115] +Results [3]: [i_category#19, sum#116, isEmpty#117] (59) Exchange -Input [3]: [i_category#171, sum#186, isEmpty#187] -Arguments: hashpartitioning(i_category#171, 5), ENSURE_REQUIREMENTS, [id=#188] +Input [3]: [i_category#19, sum#116, isEmpty#117] +Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, [id=#118] (60) HashAggregate [codegen id : 47] -Input [3]: [i_category#171, sum#186, isEmpty#187] -Keys [1]: [i_category#171] -Functions [1]: [sum(sumsales#42)] -Aggregate Attributes [1]: [sum(sumsales#42)#189] -Results [9]: [i_category#171, null AS i_class#190, null AS i_brand#191, null AS i_product_name#192, null AS d_year#193, null AS d_qoy#194, null AS d_moy#195, null AS s_store_id#196, sum(sumsales#42)#189 AS sumsales#197] +Input [3]: [i_category#19, sum#116, isEmpty#117] +Keys [1]: [i_category#19] +Functions [1]: [sum(sumsales#32)] +Aggregate Attributes [1]: [sum(sumsales#32)#119] +Results [9]: [i_category#19, null AS i_class#120, null AS i_brand#121, null AS i_product_name#122, null AS d_year#123, null AS d_qoy#124, null AS d_moy#125, null AS s_store_id#126, sum(sumsales#32)#119 AS sumsales#127] (61) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205, sum#206, isEmpty#207] +Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#128, isEmpty#129] (62) HashAggregate [codegen id : 52] -Input [10]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205, sum#206, isEmpty#207] -Keys [8]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205] -Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#208 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#209 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#208 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#209 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#210] -Results [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#208 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#209 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#210 AS sumsales#42] +Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#128, isEmpty#129] +Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] +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))#130] +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))#130 AS sumsales#32] (63) HashAggregate [codegen id : 52] -Input [1]: [sumsales#42] +Input [1]: [sumsales#32] Keys: [] -Functions [1]: [partial_sum(sumsales#42)] -Aggregate Attributes [2]: [sum#211, isEmpty#212] -Results [2]: [sum#213, isEmpty#214] +Functions [1]: [partial_sum(sumsales#32)] +Aggregate Attributes [2]: [sum#131, isEmpty#132] +Results [2]: [sum#133, isEmpty#134] (64) Exchange -Input [2]: [sum#213, isEmpty#214] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#215] +Input [2]: [sum#133, isEmpty#134] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#135] (65) HashAggregate [codegen id : 53] -Input [2]: [sum#213, isEmpty#214] +Input [2]: [sum#133, isEmpty#134] Keys: [] -Functions [1]: [sum(sumsales#42)] -Aggregate Attributes [1]: [sum(sumsales#42)#216] -Results [9]: [null AS i_category#217, null AS i_class#218, null AS i_brand#219, null AS i_product_name#220, null AS d_year#221, null AS d_qoy#222, null AS d_moy#223, null AS s_store_id#224, sum(sumsales#42)#216 AS sumsales#225] +Functions [1]: [sum(sumsales#32)] +Aggregate Attributes [1]: [sum(sumsales#32)#136] +Results [9]: [null AS i_category#137, null AS i_class#138, null AS i_brand#139, null AS i_product_name#140, null AS d_year#141, null AS d_qoy#142, null AS d_moy#143, null AS s_store_id#144, sum(sumsales#32)#136 AS sumsales#145] (66) Union (67) Exchange Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, [id=#226] +Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, [id=#146] (68) Sort [codegen id : 54] Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28] @@ -426,15 +426,15 @@ Arguments: [i_category#19 ASC NULLS FIRST, sumsales#28 DESC NULLS LAST], false, (69) Window Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28] -Arguments: [rank(sumsales#28) windowspecdefinition(i_category#19, sumsales#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#227], [i_category#19], [sumsales#28 DESC NULLS LAST] +Arguments: [rank(sumsales#28) windowspecdefinition(i_category#19, sumsales#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#147], [i_category#19], [sumsales#28 DESC NULLS LAST] (70) Filter [codegen id : 55] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#227] -Condition : (isnotnull(rk#227) AND (rk#227 <= 100)) +Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] +Condition : (isnotnull(rk#147) AND (rk#147 <= 100)) (71) TakeOrderedAndProject -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#227] -Arguments: 100, [i_category#19 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_product_name#20 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#14 ASC NULLS FIRST, sumsales#28 ASC NULLS FIRST, rk#227 ASC NULLS FIRST], [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#227] +Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] +Arguments: 100, [i_category#19 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_product_name#20 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#14 ASC NULLS FIRST, sumsales#28 ASC NULLS FIRST, rk#147 ASC NULLS FIRST], [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] ===== Subqueries ===== 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 d99363440b2bf..3c65529504320 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 @@ -122,109 +122,109 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales -Output [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] (16) Filter [codegen id : 4] -Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] -Condition : isnotnull(ss_store_sk#11) +Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_store_sk#1) (17) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#5] (18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] Join condition: None (19) Project [codegen id : 4] -Output [2]: [ss_store_sk#11, ss_net_profit#12] -Input [4]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13, d_date_sk#14] +Output [2]: [ss_store_sk#1, ss_net_profit#2] +Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] (20) Scan parquet default.store -Output [2]: [s_store_sk#15, s_state#16] +Output [2]: [s_store_sk#8, s_state#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#15, s_state#16] +Input [2]: [s_store_sk#8, s_state#10] (22) Filter [codegen id : 3] -Input [2]: [s_store_sk#15, s_state#16] -Condition : isnotnull(s_store_sk#15) +Input [2]: [s_store_sk#8, s_state#10] +Condition : isnotnull(s_store_sk#8) (23) BroadcastExchange -Input [2]: [s_store_sk#15, s_state#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] +Input [2]: [s_store_sk#8, s_state#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] (24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#11] -Right keys [1]: [s_store_sk#15] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#8] Join condition: None (25) Project [codegen id : 4] -Output [2]: [ss_net_profit#12, s_state#16] -Input [4]: [ss_store_sk#11, ss_net_profit#12, s_store_sk#15, s_state#16] +Output [2]: [ss_net_profit#2, s_state#10] +Input [4]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_state#10] (26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#12, s_state#16] -Keys [1]: [s_state#16] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#12))] -Aggregate Attributes [1]: [sum#18] -Results [2]: [s_state#16, sum#19] +Input [2]: [ss_net_profit#2, s_state#10] +Keys [1]: [s_state#10] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum#12] +Results [2]: [s_state#10, sum#13] (27) Exchange -Input [2]: [s_state#16, sum#19] -Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [2]: [s_state#10, sum#13] +Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] (28) HashAggregate [codegen id : 5] -Input [2]: [s_state#16, sum#19] -Keys [1]: [s_state#16] -Functions [1]: [sum(UnscaledValue(ss_net_profit#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#12))#21] -Results [3]: [s_state#16 AS s_state#22, s_state#16, MakeDecimal(sum(UnscaledValue(ss_net_profit#12))#21,17,2) AS _w2#23] +Input [2]: [s_state#10, sum#13] +Keys [1]: [s_state#10] +Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#15] +Results [3]: [s_state#10 AS s_state#16, s_state#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#15,17,2) AS _w2#17] (29) Exchange -Input [3]: [s_state#22, s_state#16, _w2#23] -Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [s_state#16, s_state#10, _w2#17] +Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#18] (30) Sort [codegen id : 6] -Input [3]: [s_state#22, s_state#16, _w2#23] -Arguments: [s_state#16 ASC NULLS FIRST, _w2#23 DESC NULLS LAST], false, 0 +Input [3]: [s_state#16, s_state#10, _w2#17] +Arguments: [s_state#10 ASC NULLS FIRST, _w2#17 DESC NULLS LAST], false, 0 (31) Window -Input [3]: [s_state#22, s_state#16, _w2#23] -Arguments: [rank(_w2#23) windowspecdefinition(s_state#16, _w2#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#25], [s_state#16], [_w2#23 DESC NULLS LAST] +Input [3]: [s_state#16, s_state#10, _w2#17] +Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#10], [_w2#17 DESC NULLS LAST] (32) Filter [codegen id : 7] -Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] -Condition : (isnotnull(ranking#25) AND (ranking#25 <= 5)) +Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] -Output [1]: [s_state#22] -Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] +Output [1]: [s_state#16] +Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] (34) BroadcastExchange -Input [1]: [s_state#22] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#26] +Input [1]: [s_state#16] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#20] (35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#10] -Right keys [1]: [s_state#22] +Right keys [1]: [s_state#16] Join condition: None (36) BroadcastExchange Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] @@ -239,115 +239,115 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_county#9, s_state#10 Input [3]: [ss_net_profit#2, s_county#9, s_state#10] Keys [2]: [s_state#10, s_county#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#28] -Results [3]: [s_state#10, s_county#9, sum#29] +Aggregate Attributes [1]: [sum#22] +Results [3]: [s_state#10, s_county#9, sum#23] (40) Exchange -Input [3]: [s_state#10, s_county#9, sum#29] -Arguments: hashpartitioning(s_state#10, s_county#9, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [s_state#10, s_county#9, sum#23] +Arguments: hashpartitioning(s_state#10, s_county#9, 5), ENSURE_REQUIREMENTS, [id=#24] (41) HashAggregate [codegen id : 10] -Input [3]: [s_state#10, s_county#9, sum#29] +Input [3]: [s_state#10, s_county#9, sum#23] Keys [2]: [s_state#10, s_county#9] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) as decimal(27,2)) AS total_sum#32, s_state#10, s_county#9, 0 AS g_state#33, 0 AS g_county#34, 0 AS lochierarchy#35] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) as decimal(27,2)) AS total_sum#26, s_state#10, s_county#9, 0 AS g_state#27, 0 AS g_county#28, 0 AS lochierarchy#29] (42) ReusedExchange [Reuses operator id: unknown] -Output [3]: [s_state#36, s_county#37, sum#38] +Output [3]: [s_state#10, s_county#9, sum#30] (43) HashAggregate [codegen id : 20] -Input [3]: [s_state#36, s_county#37, sum#38] -Keys [2]: [s_state#36, s_county#37] -Functions [1]: [sum(UnscaledValue(ss_net_profit#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#39))#40] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#39))#40,17,2) AS total_sum#41, s_state#36] +Input [3]: [s_state#10, s_county#9, sum#30] +Keys [2]: [s_state#10, s_county#9] +Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS total_sum#32, s_state#10] (44) HashAggregate [codegen id : 20] -Input [2]: [total_sum#41, s_state#36] -Keys [1]: [s_state#36] -Functions [1]: [partial_sum(total_sum#41)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [s_state#36, sum#44, isEmpty#45] +Input [2]: [total_sum#32, s_state#10] +Keys [1]: [s_state#10] +Functions [1]: [partial_sum(total_sum#32)] +Aggregate Attributes [2]: [sum#33, isEmpty#34] +Results [3]: [s_state#10, sum#35, isEmpty#36] (45) Exchange -Input [3]: [s_state#36, sum#44, isEmpty#45] -Arguments: hashpartitioning(s_state#36, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [s_state#10, sum#35, isEmpty#36] +Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#37] (46) HashAggregate [codegen id : 21] -Input [3]: [s_state#36, sum#44, isEmpty#45] -Keys [1]: [s_state#36] -Functions [1]: [sum(total_sum#41)] -Aggregate Attributes [1]: [sum(total_sum#41)#47] -Results [6]: [sum(total_sum#41)#47 AS total_sum#48, s_state#36, null AS s_county#49, 0 AS g_state#50, 1 AS g_county#51, 1 AS lochierarchy#52] +Input [3]: [s_state#10, sum#35, isEmpty#36] +Keys [1]: [s_state#10] +Functions [1]: [sum(total_sum#32)] +Aggregate Attributes [1]: [sum(total_sum#32)#38] +Results [6]: [sum(total_sum#32)#38 AS total_sum#39, s_state#10, null AS s_county#40, 0 AS g_state#41, 1 AS g_county#42, 1 AS lochierarchy#43] (47) ReusedExchange [Reuses operator id: unknown] -Output [3]: [s_state#53, s_county#54, sum#55] +Output [3]: [s_state#10, s_county#9, sum#44] (48) HashAggregate [codegen id : 31] -Input [3]: [s_state#53, s_county#54, sum#55] -Keys [2]: [s_state#53, s_county#54] -Functions [1]: [sum(UnscaledValue(ss_net_profit#56))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#56))#57] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#56))#57,17,2) AS total_sum#41] +Input [3]: [s_state#10, s_county#9, sum#44] +Keys [2]: [s_state#10, s_county#9] +Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#45] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#45,17,2) AS total_sum#32] (49) HashAggregate [codegen id : 31] -Input [1]: [total_sum#41] +Input [1]: [total_sum#32] Keys: [] -Functions [1]: [partial_sum(total_sum#41)] -Aggregate Attributes [2]: [sum#58, isEmpty#59] -Results [2]: [sum#60, isEmpty#61] +Functions [1]: [partial_sum(total_sum#32)] +Aggregate Attributes [2]: [sum#46, isEmpty#47] +Results [2]: [sum#48, isEmpty#49] (50) Exchange -Input [2]: [sum#60, isEmpty#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] +Input [2]: [sum#48, isEmpty#49] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#50] (51) HashAggregate [codegen id : 32] -Input [2]: [sum#60, isEmpty#61] +Input [2]: [sum#48, isEmpty#49] Keys: [] -Functions [1]: [sum(total_sum#41)] -Aggregate Attributes [1]: [sum(total_sum#41)#63] -Results [6]: [sum(total_sum#41)#63 AS total_sum#64, null AS s_state#65, null AS s_county#66, 1 AS g_state#67, 1 AS g_county#68, 2 AS lochierarchy#69] +Functions [1]: [sum(total_sum#32)] +Aggregate Attributes [1]: [sum(total_sum#32)#51] +Results [6]: [sum(total_sum#32)#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] (52) Union (53) HashAggregate [codegen id : 33] -Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] -Keys [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Keys [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Results [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] (54) Exchange -Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] -Arguments: hashpartitioning(total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Arguments: hashpartitioning(total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29, 5), ENSURE_REQUIREMENTS, [id=#58] (55) HashAggregate [codegen id : 34] -Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] -Keys [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Keys [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, CASE WHEN (g_county#34 = 0) THEN s_state#10 END AS _w0#71] +Results [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, CASE WHEN (g_county#28 = 0) THEN s_state#10 END AS _w0#59] (56) Exchange -Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] -Arguments: hashpartitioning(lochierarchy#35, _w0#71, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] +Arguments: hashpartitioning(lochierarchy#29, _w0#59, 5), ENSURE_REQUIREMENTS, [id=#60] (57) Sort [codegen id : 35] -Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] -Arguments: [lochierarchy#35 ASC NULLS FIRST, _w0#71 ASC NULLS FIRST, total_sum#32 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] +Arguments: [lochierarchy#29 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST], false, 0 (58) Window -Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] -Arguments: [rank(total_sum#32) windowspecdefinition(lochierarchy#35, _w0#71, total_sum#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#73], [lochierarchy#35, _w0#71], [total_sum#32 DESC NULLS LAST] +Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] +Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#29, _w0#59, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#29, _w0#59], [total_sum#26 DESC NULLS LAST] (59) Project [codegen id : 36] -Output [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] -Input [6]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71, rank_within_parent#73] +Output [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] +Input [6]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59, rank_within_parent#61] (60) TakeOrderedAndProject -Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] -Arguments: 100, [lochierarchy#35 DESC NULLS LAST, CASE WHEN (lochierarchy#35 = 0) THEN s_state#10 END ASC NULLS FIRST, rank_within_parent#73 ASC NULLS FIRST], [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] +Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] +Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#10 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] ===== Subqueries ===== @@ -358,6 +358,6 @@ ReusedExchange (61) (61) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#5] -Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#3 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 159494182ba83..406acb0e0a27f 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 @@ -122,109 +122,109 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales -Output [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] (16) Filter [codegen id : 4] -Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] -Condition : isnotnull(ss_store_sk#11) +Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_store_sk#1) (17) Scan parquet default.store -Output [2]: [s_store_sk#14, s_state#15] +Output [2]: [s_store_sk#8, s_state#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#14, s_state#15] +Input [2]: [s_store_sk#8, s_state#10] (19) Filter [codegen id : 2] -Input [2]: [s_store_sk#14, s_state#15] -Condition : isnotnull(s_store_sk#14) +Input [2]: [s_store_sk#8, s_state#10] +Condition : isnotnull(s_store_sk#8) (20) BroadcastExchange -Input [2]: [s_store_sk#14, s_state#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#16] +Input [2]: [s_store_sk#8, s_state#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] (21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#11] -Right keys [1]: [s_store_sk#14] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#8] Join condition: None (22) Project [codegen id : 4] -Output [3]: [ss_net_profit#12, ss_sold_date_sk#13, s_state#15] -Input [5]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13, s_store_sk#14, s_state#15] +Output [3]: [ss_net_profit#2, ss_sold_date_sk#3, s_state#10] +Input [5]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, s_store_sk#8, s_state#10] (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#17] +Output [1]: [d_date_sk#5] (24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#13] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] Join condition: None (25) Project [codegen id : 4] -Output [2]: [ss_net_profit#12, s_state#15] -Input [4]: [ss_net_profit#12, ss_sold_date_sk#13, s_state#15, d_date_sk#17] +Output [2]: [ss_net_profit#2, s_state#10] +Input [4]: [ss_net_profit#2, ss_sold_date_sk#3, s_state#10, d_date_sk#5] (26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#12, s_state#15] -Keys [1]: [s_state#15] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#12))] -Aggregate Attributes [1]: [sum#18] -Results [2]: [s_state#15, sum#19] +Input [2]: [ss_net_profit#2, s_state#10] +Keys [1]: [s_state#10] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum#12] +Results [2]: [s_state#10, sum#13] (27) Exchange -Input [2]: [s_state#15, sum#19] -Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [2]: [s_state#10, sum#13] +Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] (28) HashAggregate [codegen id : 5] -Input [2]: [s_state#15, sum#19] -Keys [1]: [s_state#15] -Functions [1]: [sum(UnscaledValue(ss_net_profit#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#12))#21] -Results [3]: [s_state#15 AS s_state#22, s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#12))#21,17,2) AS _w2#23] +Input [2]: [s_state#10, sum#13] +Keys [1]: [s_state#10] +Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#15] +Results [3]: [s_state#10 AS s_state#16, s_state#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#15,17,2) AS _w2#17] (29) Exchange -Input [3]: [s_state#22, s_state#15, _w2#23] -Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [s_state#16, s_state#10, _w2#17] +Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#18] (30) Sort [codegen id : 6] -Input [3]: [s_state#22, s_state#15, _w2#23] -Arguments: [s_state#15 ASC NULLS FIRST, _w2#23 DESC NULLS LAST], false, 0 +Input [3]: [s_state#16, s_state#10, _w2#17] +Arguments: [s_state#10 ASC NULLS FIRST, _w2#17 DESC NULLS LAST], false, 0 (31) Window -Input [3]: [s_state#22, s_state#15, _w2#23] -Arguments: [rank(_w2#23) windowspecdefinition(s_state#15, _w2#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#25], [s_state#15], [_w2#23 DESC NULLS LAST] +Input [3]: [s_state#16, s_state#10, _w2#17] +Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#10], [_w2#17 DESC NULLS LAST] (32) Filter [codegen id : 7] -Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] -Condition : (isnotnull(ranking#25) AND (ranking#25 <= 5)) +Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] -Output [1]: [s_state#22] -Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] +Output [1]: [s_state#16] +Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] (34) BroadcastExchange -Input [1]: [s_state#22] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#26] +Input [1]: [s_state#16] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#20] (35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#10] -Right keys [1]: [s_state#22] +Right keys [1]: [s_state#16] Join condition: None (36) BroadcastExchange Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] @@ -239,115 +239,115 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_county#9, s_state#10 Input [3]: [ss_net_profit#2, s_county#9, s_state#10] Keys [2]: [s_state#10, s_county#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#28] -Results [3]: [s_state#10, s_county#9, sum#29] +Aggregate Attributes [1]: [sum#22] +Results [3]: [s_state#10, s_county#9, sum#23] (40) Exchange -Input [3]: [s_state#10, s_county#9, sum#29] -Arguments: hashpartitioning(s_state#10, s_county#9, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [s_state#10, s_county#9, sum#23] +Arguments: hashpartitioning(s_state#10, s_county#9, 5), ENSURE_REQUIREMENTS, [id=#24] (41) HashAggregate [codegen id : 10] -Input [3]: [s_state#10, s_county#9, sum#29] +Input [3]: [s_state#10, s_county#9, sum#23] Keys [2]: [s_state#10, s_county#9] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) as decimal(27,2)) AS total_sum#32, s_state#10, s_county#9, 0 AS g_state#33, 0 AS g_county#34, 0 AS lochierarchy#35] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) as decimal(27,2)) AS total_sum#26, s_state#10, s_county#9, 0 AS g_state#27, 0 AS g_county#28, 0 AS lochierarchy#29] (42) ReusedExchange [Reuses operator id: unknown] -Output [3]: [s_state#36, s_county#37, sum#38] +Output [3]: [s_state#10, s_county#9, sum#30] (43) HashAggregate [codegen id : 20] -Input [3]: [s_state#36, s_county#37, sum#38] -Keys [2]: [s_state#36, s_county#37] -Functions [1]: [sum(UnscaledValue(ss_net_profit#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#39))#40] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#39))#40,17,2) AS total_sum#41, s_state#36] +Input [3]: [s_state#10, s_county#9, sum#30] +Keys [2]: [s_state#10, s_county#9] +Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS total_sum#32, s_state#10] (44) HashAggregate [codegen id : 20] -Input [2]: [total_sum#41, s_state#36] -Keys [1]: [s_state#36] -Functions [1]: [partial_sum(total_sum#41)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [s_state#36, sum#44, isEmpty#45] +Input [2]: [total_sum#32, s_state#10] +Keys [1]: [s_state#10] +Functions [1]: [partial_sum(total_sum#32)] +Aggregate Attributes [2]: [sum#33, isEmpty#34] +Results [3]: [s_state#10, sum#35, isEmpty#36] (45) Exchange -Input [3]: [s_state#36, sum#44, isEmpty#45] -Arguments: hashpartitioning(s_state#36, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [s_state#10, sum#35, isEmpty#36] +Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#37] (46) HashAggregate [codegen id : 21] -Input [3]: [s_state#36, sum#44, isEmpty#45] -Keys [1]: [s_state#36] -Functions [1]: [sum(total_sum#41)] -Aggregate Attributes [1]: [sum(total_sum#41)#47] -Results [6]: [sum(total_sum#41)#47 AS total_sum#48, s_state#36, null AS s_county#49, 0 AS g_state#50, 1 AS g_county#51, 1 AS lochierarchy#52] +Input [3]: [s_state#10, sum#35, isEmpty#36] +Keys [1]: [s_state#10] +Functions [1]: [sum(total_sum#32)] +Aggregate Attributes [1]: [sum(total_sum#32)#38] +Results [6]: [sum(total_sum#32)#38 AS total_sum#39, s_state#10, null AS s_county#40, 0 AS g_state#41, 1 AS g_county#42, 1 AS lochierarchy#43] (47) ReusedExchange [Reuses operator id: unknown] -Output [3]: [s_state#53, s_county#54, sum#55] +Output [3]: [s_state#10, s_county#9, sum#44] (48) HashAggregate [codegen id : 31] -Input [3]: [s_state#53, s_county#54, sum#55] -Keys [2]: [s_state#53, s_county#54] -Functions [1]: [sum(UnscaledValue(ss_net_profit#56))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#56))#57] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#56))#57,17,2) AS total_sum#41] +Input [3]: [s_state#10, s_county#9, sum#44] +Keys [2]: [s_state#10, s_county#9] +Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#45] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#45,17,2) AS total_sum#32] (49) HashAggregate [codegen id : 31] -Input [1]: [total_sum#41] +Input [1]: [total_sum#32] Keys: [] -Functions [1]: [partial_sum(total_sum#41)] -Aggregate Attributes [2]: [sum#58, isEmpty#59] -Results [2]: [sum#60, isEmpty#61] +Functions [1]: [partial_sum(total_sum#32)] +Aggregate Attributes [2]: [sum#46, isEmpty#47] +Results [2]: [sum#48, isEmpty#49] (50) Exchange -Input [2]: [sum#60, isEmpty#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] +Input [2]: [sum#48, isEmpty#49] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#50] (51) HashAggregate [codegen id : 32] -Input [2]: [sum#60, isEmpty#61] +Input [2]: [sum#48, isEmpty#49] Keys: [] -Functions [1]: [sum(total_sum#41)] -Aggregate Attributes [1]: [sum(total_sum#41)#63] -Results [6]: [sum(total_sum#41)#63 AS total_sum#64, null AS s_state#65, null AS s_county#66, 1 AS g_state#67, 1 AS g_county#68, 2 AS lochierarchy#69] +Functions [1]: [sum(total_sum#32)] +Aggregate Attributes [1]: [sum(total_sum#32)#51] +Results [6]: [sum(total_sum#32)#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] (52) Union (53) HashAggregate [codegen id : 33] -Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] -Keys [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Keys [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Results [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] (54) Exchange -Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] -Arguments: hashpartitioning(total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Arguments: hashpartitioning(total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29, 5), ENSURE_REQUIREMENTS, [id=#58] (55) HashAggregate [codegen id : 34] -Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] -Keys [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Keys [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, CASE WHEN (g_county#34 = 0) THEN s_state#10 END AS _w0#71] +Results [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, CASE WHEN (g_county#28 = 0) THEN s_state#10 END AS _w0#59] (56) Exchange -Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] -Arguments: hashpartitioning(lochierarchy#35, _w0#71, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] +Arguments: hashpartitioning(lochierarchy#29, _w0#59, 5), ENSURE_REQUIREMENTS, [id=#60] (57) Sort [codegen id : 35] -Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] -Arguments: [lochierarchy#35 ASC NULLS FIRST, _w0#71 ASC NULLS FIRST, total_sum#32 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] +Arguments: [lochierarchy#29 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST], false, 0 (58) Window -Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] -Arguments: [rank(total_sum#32) windowspecdefinition(lochierarchy#35, _w0#71, total_sum#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#73], [lochierarchy#35, _w0#71], [total_sum#32 DESC NULLS LAST] +Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] +Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#29, _w0#59, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#29, _w0#59], [total_sum#26 DESC NULLS LAST] (59) Project [codegen id : 36] -Output [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] -Input [6]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71, rank_within_parent#73] +Output [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] +Input [6]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59, rank_within_parent#61] (60) TakeOrderedAndProject -Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] -Arguments: 100, [lochierarchy#35 DESC NULLS LAST, CASE WHEN (lochierarchy#35 = 0) THEN s_state#10 END ASC NULLS FIRST, rank_within_parent#73 ASC NULLS FIRST], [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] +Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] +Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#10 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] ===== Subqueries ===== @@ -358,6 +358,6 @@ ReusedExchange (61) (61) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#5] -Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt index fe8ceb415f571..0d90342fc9bbf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt @@ -100,7 +100,7 @@ Condition : ((((isnotnull(cs_quantity#7) AND isnotnull(cs_item_sk#4)) AND isnotn Output [2]: [hd_demo_sk#9, hd_buy_potential#10] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000 ), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -108,7 +108,7 @@ Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (6) Filter [codegen id : 1] Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = 1001-5000 )) AND isnotnull(hd_demo_sk#9)) +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = 1001-5000)) AND isnotnull(hd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [hd_demo_sk#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt index 0e67565e856ad..c897c78880bc2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt @@ -207,7 +207,7 @@ Input [11]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_ Output [2]: [hd_demo_sk#24, hd_buy_potential#25] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000 ), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000), IsNotNull(hd_demo_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] @@ -215,7 +215,7 @@ Input [2]: [hd_demo_sk#24, hd_buy_potential#25] (31) Filter [codegen id : 5] Input [2]: [hd_demo_sk#24, hd_buy_potential#25] -Condition : ((isnotnull(hd_buy_potential#25) AND (hd_buy_potential#25 = 1001-5000 )) AND isnotnull(hd_demo_sk#24)) +Condition : ((isnotnull(hd_buy_potential#25) AND (hd_buy_potential#25 = 1001-5000)) AND isnotnull(hd_demo_sk#24)) (32) Project [codegen id : 5] Output [1]: [hd_demo_sk#24] 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 7eea8040043d6..035f89d3a04f7 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 @@ -199,285 +199,285 @@ Input [2]: [customer_id#18, year_total#19] Arguments: [customer_id#18 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Output [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] (27) Filter [codegen id : 10] -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#21) +Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_customer_sk#1) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#25, d_year#26] +Output [2]: [d_date_sk#5, d_year#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#25, d_year#26] +Input [2]: [d_date_sk#5, d_year#6] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) +Input [2]: [d_date_sk#5, d_year#6] +Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2002)) AND d_year#6 IN (2001,2002)) AND isnotnull(d_date_sk#5)) (31) BroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [2]: [d_date_sk#5, d_year#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#23] -Right keys [1]: [d_date_sk#25] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] Join condition: None (33) Project [codegen id : 10] -Output [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] -Input [5]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] +Output [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] +Input [5]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6] (34) Exchange -Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] -Arguments: hashpartitioning(ss_customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] +Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#23] (35) Sort [codegen id : 11] -Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] -Arguments: [ss_customer_sk#21 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] +Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (37) Sort [codegen id : 13] -Input [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#21] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#9] Join condition: None (39) Project [codegen id : 14] -Output [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] -Input [7]: [ss_customer_sk#21, ss_net_paid#22, d_year#26, c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] +Input [7]: [ss_customer_sk#1, ss_net_paid#2, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (40) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] -Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] -Aggregate Attributes [1]: [sum#33] -Results [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#2))] +Aggregate Attributes [1]: [sum#24] +Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] (41) Exchange -Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] -Arguments: hashpartitioning(c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] +Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#26] (42) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] -Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] -Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#22))#36] -Results [4]: [c_customer_id#30 AS customer_id#37, c_first_name#31 AS customer_first_name#38, c_last_name#32 AS customer_last_name#39, MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#36,17,2) AS year_total#40] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [sum(UnscaledValue(ss_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#2))#27] +Results [4]: [c_customer_id#10 AS customer_id#28, c_first_name#11 AS customer_first_name#29, c_last_name#12 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#2))#27,17,2) AS year_total#31] (43) Exchange -Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] -Arguments: hashpartitioning(customer_id#37, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +Arguments: hashpartitioning(customer_id#28, 5), ENSURE_REQUIREMENTS, [id=#32] (44) Sort [codegen id : 16] -Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] -Arguments: [customer_id#37 ASC NULLS FIRST], false, 0 +Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +Arguments: [customer_id#28 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#37] +Right keys [1]: [customer_id#28] Join condition: None (46) 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#33, ws_net_paid#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] (48) Filter [codegen id : 19] -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#33, ws_net_paid#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_bill_customer_sk#33) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#45, d_year#46] +Output [2]: [d_date_sk#5, d_year#6] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#45] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#5] Join condition: None (51) Project [codegen id : 19] -Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] -Input [5]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] +Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] (52) Exchange -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] -Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#36] (53) Sort [codegen id : 20] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] -Arguments: [ws_bill_customer_sk#42 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (55) Sort [codegen id : 22] -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Arguments: [c_customer_sk#48 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#42] -Right keys [1]: [c_customer_sk#48] +Left keys [1]: [ws_bill_customer_sk#33] +Right keys [1]: [c_customer_sk#9] Join condition: None (57) Project [codegen id : 23] -Output [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] -Input [7]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46, c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] +Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (58) HashAggregate [codegen id : 23] -Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] -Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum#52] -Results [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum#37] +Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] (59) Exchange -Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] -Arguments: hashpartitioning(c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] +Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#39] (60) HashAggregate [codegen id : 24] -Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] -Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] -Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#55] -Results [2]: [c_customer_id#49 AS customer_id#56, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#55,17,2) AS year_total#57] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#40] +Results [2]: [c_customer_id#10 AS customer_id#41, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#40,17,2) AS year_total#42] (61) Filter [codegen id : 24] -Input [2]: [customer_id#56, year_total#57] -Condition : (isnotnull(year_total#57) AND (year_total#57 > 0.00)) +Input [2]: [customer_id#41, year_total#42] +Condition : (isnotnull(year_total#42) AND (year_total#42 > 0.00)) (62) Project [codegen id : 24] -Output [2]: [customer_id#56 AS customer_id#58, year_total#57 AS year_total#59] -Input [2]: [customer_id#56, year_total#57] +Output [2]: [customer_id#41 AS customer_id#43, year_total#42 AS year_total#44] +Input [2]: [customer_id#41, year_total#42] (63) Exchange -Input [2]: [customer_id#58, year_total#59] -Arguments: hashpartitioning(customer_id#58, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [2]: [customer_id#43, year_total#44] +Arguments: hashpartitioning(customer_id#43, 5), ENSURE_REQUIREMENTS, [id=#45] (64) Sort [codegen id : 25] -Input [2]: [customer_id#58, year_total#59] -Arguments: [customer_id#58 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#43, year_total#44] +Arguments: [customer_id#43 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#58] +Right keys [1]: [customer_id#43] Join condition: None (66) Project [codegen id : 26] -Output [7]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59] -Input [8]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, customer_id#58, year_total#59] +Output [7]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44] +Input [8]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#43, year_total#44] (67) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] (69) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_customer_sk#61) +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_bill_customer_sk#33) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#64, d_year#65] +Output [2]: [d_date_sk#5, d_year#6] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#63] -Right keys [1]: [d_date_sk#64] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#5] Join condition: None (72) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] -Input [5]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65] +Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] (73) Exchange -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] -Arguments: hashpartitioning(ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#46] (74) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] -Arguments: [ws_bill_customer_sk#61 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] +Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (76) Sort [codegen id : 31] -Input [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] -Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#61] -Right keys [1]: [c_customer_sk#67] +Left keys [1]: [ws_bill_customer_sk#33] +Right keys [1]: [c_customer_sk#9] Join condition: None (78) Project [codegen id : 32] -Output [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] -Input [7]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65, c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] +Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] +Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (79) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] -Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#62))] -Aggregate Attributes [1]: [sum#71] -Results [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum#47] +Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] (80) Exchange -Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] -Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] +Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#49] (81) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] -Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] -Functions [1]: [sum(UnscaledValue(ws_net_paid#62))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#62))#74] -Results [2]: [c_customer_id#68 AS customer_id#75, MakeDecimal(sum(UnscaledValue(ws_net_paid#62))#74,17,2) AS year_total#76] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#50] +Results [2]: [c_customer_id#10 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#50,17,2) AS year_total#52] (82) Exchange -Input [2]: [customer_id#75, year_total#76] -Arguments: hashpartitioning(customer_id#75, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [2]: [customer_id#51, year_total#52] +Arguments: hashpartitioning(customer_id#51, 5), ENSURE_REQUIREMENTS, [id=#53] (83) Sort [codegen id : 34] -Input [2]: [customer_id#75, year_total#76] -Arguments: [customer_id#75 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#51, year_total#52] +Arguments: [customer_id#51 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#75] -Join condition: (CASE WHEN (year_total#59 > 0.00) THEN CheckOverflow((promote_precision(year_total#76) / promote_precision(year_total#59)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#40) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#51] +Join condition: (CASE WHEN (year_total#44 > 0.00) THEN CheckOverflow((promote_precision(year_total#52) / promote_precision(year_total#44)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#31) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) (85) Project [codegen id : 35] -Output [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] -Input [9]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59, customer_id#75, year_total#76] +Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Input [9]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44, customer_id#51, year_total#52] (86) TakeOrderedAndProject -Input [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] -Arguments: 100, [customer_first_name#38 ASC NULLS FIRST, customer_id#37 ASC NULLS FIRST, customer_last_name#39 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39] +Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Arguments: 100, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] ===== Subqueries ===== @@ -488,15 +488,15 @@ ReusedExchange (87) (87) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#5, d_year#6] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#21 ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#25, d_year#26] +Output [2]: [d_date_sk#5, d_year#6] -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#24 +Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#21 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 3ee6b56189888..44a902cf6da07 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 @@ -171,282 +171,282 @@ Input [2]: [customer_id#17, year_total#18] Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) (20) Scan parquet default.customer -Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] (22) Filter [codegen id : 6] -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Condition : (isnotnull(c_customer_sk#19) AND isnotnull(c_customer_id#20)) +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (23) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#19)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] (25) Filter [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_customer_sk#23) +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Condition : isnotnull(ss_customer_sk#5) (26) BroadcastExchange -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#19] -Right keys [1]: [ss_customer_sk#23] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#5] Join condition: None (28) Project [codegen id : 6] -Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25] -Input [7]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] +Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#28, d_year#29] +Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#28, d_year#29] +Input [2]: [d_date_sk#10, d_year#11] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#28, d_year#29] -Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) +Input [2]: [d_date_sk#10, d_year#11] +Condition : (((isnotnull(d_year#11) AND (d_year#11 = 2002)) AND d_year#11 IN (2001,2002)) AND isnotnull(d_date_sk#10)) (32) BroadcastExchange -Input [2]: [d_date_sk#28, d_year#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [2]: [d_date_sk#10, d_year#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#25] -Right keys [1]: [d_date_sk#28] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#10] Join condition: None (34) Project [codegen id : 6] -Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] -Input [7]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#28, d_year#29] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] +Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#10, d_year#11] (35) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] -Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum#31] -Results [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] +Aggregate Attributes [1]: [sum#22] +Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] (36) Exchange -Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] -Arguments: hashpartitioning(c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#24] (37) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] -Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] -Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#34] -Results [4]: [c_customer_id#20 AS customer_id#35, c_first_name#21 AS customer_first_name#36, c_last_name#22 AS customer_last_name#37, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#34,17,2) AS year_total#38] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#25] +Results [4]: [c_customer_id#2 AS customer_id#26, c_first_name#3 AS customer_first_name#27, c_last_name#4 AS customer_last_name#28, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#25,17,2) AS year_total#29] (38) BroadcastExchange -Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#39] +Input [4]: [customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#35] +Right keys [1]: [customer_id#26] Join condition: None (40) Scan parquet default.customer -Output [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] +Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] (42) Filter [codegen id : 10] -Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] -Condition : (isnotnull(c_customer_sk#40) AND isnotnull(c_customer_id#41)) +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (43) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] (45) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] -Condition : isnotnull(ws_bill_customer_sk#44) +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Condition : isnotnull(ws_bill_customer_sk#31) (46) BroadcastExchange -Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#40] -Right keys [1]: [ws_bill_customer_sk#44] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#31] Join condition: None (48) Project [codegen id : 10] -Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46] -Input [7]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] +Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#48, d_year#49] +Output [2]: [d_date_sk#10, d_year#11] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#46] -Right keys [1]: [d_date_sk#48] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#10] Join condition: None (51) Project [codegen id : 10] -Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] -Input [7]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46, d_date_sk#48, d_year#49] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] +Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] (52) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] -Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#45))] -Aggregate Attributes [1]: [sum#50] -Results [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] +Aggregate Attributes [1]: [sum#35] +Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] (53) Exchange -Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] -Arguments: hashpartitioning(c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#37] (54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] -Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] -Functions [1]: [sum(UnscaledValue(ws_net_paid#45))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#45))#53] -Results [2]: [c_customer_id#41 AS customer_id#54, MakeDecimal(sum(UnscaledValue(ws_net_paid#45))#53,17,2) AS year_total#55] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#38] +Results [2]: [c_customer_id#2 AS customer_id#39, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#38,17,2) AS year_total#40] (55) Filter [codegen id : 11] -Input [2]: [customer_id#54, year_total#55] -Condition : (isnotnull(year_total#55) AND (year_total#55 > 0.00)) +Input [2]: [customer_id#39, year_total#40] +Condition : (isnotnull(year_total#40) AND (year_total#40 > 0.00)) (56) Project [codegen id : 11] -Output [2]: [customer_id#54 AS customer_id#56, year_total#55 AS year_total#57] -Input [2]: [customer_id#54, year_total#55] +Output [2]: [customer_id#39 AS customer_id#41, year_total#40 AS year_total#42] +Input [2]: [customer_id#39, year_total#40] (57) BroadcastExchange -Input [2]: [customer_id#56, year_total#57] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#58] +Input [2]: [customer_id#41, year_total#42] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#43] (58) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#56] +Right keys [1]: [customer_id#41] Join condition: None (59) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57] -Input [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, customer_id#56, year_total#57] +Output [7]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42] +Input [8]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, customer_id#41, year_total#42] (60) Scan parquet default.customer -Output [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] +Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] (62) Filter [codegen id : 14] -Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] -Condition : (isnotnull(c_customer_sk#59) AND isnotnull(c_customer_id#60)) +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (63) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#19)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] (65) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] -Condition : isnotnull(ws_bill_customer_sk#63) +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Condition : isnotnull(ws_bill_customer_sk#31) (66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#59] -Right keys [1]: [ws_bill_customer_sk#63] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#31] Join condition: None (68) Project [codegen id : 14] -Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65] -Input [7]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] +Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#67, d_year#68] +Output [2]: [d_date_sk#10, d_year#11] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#67] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#10] Join condition: None (71) Project [codegen id : 14] -Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] -Input [7]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65, d_date_sk#67, d_year#68] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] +Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] (72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] -Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#64))] -Aggregate Attributes [1]: [sum#69] -Results [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] +Aggregate Attributes [1]: [sum#45] +Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] (73) Exchange -Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] -Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#47] (74) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] -Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] -Functions [1]: [sum(UnscaledValue(ws_net_paid#64))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#64))#72] -Results [2]: [c_customer_id#60 AS customer_id#73, MakeDecimal(sum(UnscaledValue(ws_net_paid#64))#72,17,2) AS year_total#74] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#48] +Results [2]: [c_customer_id#2 AS customer_id#49, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#48,17,2) AS year_total#50] (75) BroadcastExchange -Input [2]: [customer_id#73, year_total#74] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#75] +Input [2]: [customer_id#49, year_total#50] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#51] (76) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#73] -Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#57)), 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#49] +Join condition: (CASE WHEN (year_total#42 > 0.00) THEN CheckOverflow((promote_precision(year_total#50) / promote_precision(year_total#42)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#29) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END) (77) Project [codegen id : 16] -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#57, customer_id#73, year_total#74] +Output [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] +Input [9]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42, customer_id#49, year_total#50] (78) 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#26, customer_first_name#27, customer_last_name#28] +Arguments: 100, [customer_first_name#27 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST, customer_last_name#28 ASC NULLS FIRST], [customer_id#26, customer_first_name#27, customer_last_name#28] ===== Subqueries ===== @@ -457,15 +457,15 @@ ReusedExchange (79) (79) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#10, d_year#11] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#19 ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#28, d_year#29] +Output [2]: [d_date_sk#10, d_year#11] -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#26 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#19 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 56998d9197835..955dde96eb177 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 @@ -153,7 +153,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -161,7 +161,7 @@ Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_categor (6) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books)) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#31] +Right keys [1]: [i_item_sk#7] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#36, d_year#37] +Output [2]: [d_date_sk#14, d_year#15] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] +Right keys [1]: [d_date_sk#14] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] -Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] (42) Exchange -Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] -Condition : isnotnull(ws_item_sk#47) +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_item_sk#40) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#47] -Right keys [1]: [i_item_sk#52] +Left keys [1]: [ws_item_sk#40] +Right keys [1]: [i_item_sk#7] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] -Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#57, d_year#58] +Output [2]: [d_date_sk#14, d_year#15] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#57] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#14] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] (55) Exchange -Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] (61) Exchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] -Right keys [2]: [wr_order_number#61, wr_item_sk#60] +Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] +Right keys [2]: [wr_order_number#47, wr_item_sk#46] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] -Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] +Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#69, sum#70] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Aggregate Attributes [2]: [sum#55, sum#56] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] +Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] -Condition : isnotnull(cs_item_sk#79) +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Condition : isnotnull(cs_item_sk#1) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#79] -Right keys [1]: [i_item_sk#85] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#66] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] -Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#71, d_year#72] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#90, d_year#91] +Input [2]: [d_date_sk#71, d_year#72] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#90, d_year#91] -Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) +Input [2]: [d_date_sk#71, d_year#72] +Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) (83) BroadcastExchange -Input [2]: [d_date_sk#90, d_year#91] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] +Input [2]: [d_date_sk#71, d_year#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#83] -Right keys [1]: [d_date_sk#90] +Left keys [1]: [cs_sold_date_sk#5] +Right keys [1]: [d_date_sk#71] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (86) Exchange -Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] -Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#80, cs_item_sk#79] -Right keys [2]: [cr_order_number#95, cr_item_sk#94] +Left keys [2]: [cs_order_number#2, cs_item_sk#1] +Right keys [2]: [cr_order_number#19, cr_item_sk#18] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] -Condition : isnotnull(ss_item_sk#98) +Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Condition : isnotnull(ss_item_sk#26) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#98] -Right keys [1]: [i_item_sk#103] +Left keys [1]: [ss_item_sk#26] +Right keys [1]: [i_item_sk#66] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] -Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#108, d_year#109] +Output [2]: [d_date_sk#71, d_year#72] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#102] -Right keys [1]: [d_date_sk#108] +Left keys [1]: [ss_sold_date_sk#30] +Right keys [1]: [d_date_sk#71] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (101) Exchange -Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] -Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] -Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] +Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] -Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] -Condition : isnotnull(ws_item_sk#116) +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_item_sk#40) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#116] -Right keys [1]: [i_item_sk#121] +Left keys [1]: [ws_item_sk#40] +Right keys [1]: [i_item_sk#66] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] -Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#126, d_year#127] +Output [2]: [d_date_sk#71, d_year#72] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#120] -Right keys [1]: [d_date_sk#126] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#71] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (116) Exchange -Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] -Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] -Right keys [2]: [wr_order_number#130, wr_item_sk#129] +Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] +Right keys [2]: [wr_order_number#47, wr_item_sk#46] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] -Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] +Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#135, sum#136] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Aggregate Attributes [2]: [sum#82, sum#83] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] (127) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] -Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] -Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] +Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] (129) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] -Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] (130) Sort [codegen id : 50] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] -Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 (131) 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#86, i_class_id#87, i_category_id#88, i_manufact_id#89] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] (133) TakeOrderedAndProject -Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] -Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST, sales_amt_diff#150 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST, sales_amt_diff#97 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#71, d_year#72] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 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 56998d9197835..955dde96eb177 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 @@ -153,7 +153,7 @@ Condition : isnotnull(cs_item_sk#1) Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -161,7 +161,7 @@ Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_categor (6) Filter [codegen id : 1] Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] -Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books)) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) (7) Project [codegen id : 1] Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#31] +Right keys [1]: [i_item_sk#7] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#36, d_year#37] +Output [2]: [d_date_sk#14, d_year#15] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] +Right keys [1]: [d_date_sk#14] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] -Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] (42) Exchange -Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] -Condition : isnotnull(ws_item_sk#47) +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_item_sk#40) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#47] -Right keys [1]: [i_item_sk#52] +Left keys [1]: [ws_item_sk#40] +Right keys [1]: [i_item_sk#7] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] -Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#57, d_year#58] +Output [2]: [d_date_sk#14, d_year#15] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#57] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#14] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] (55) Exchange -Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] (61) Exchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] -Right keys [2]: [wr_order_number#61, wr_item_sk#60] +Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] +Right keys [2]: [wr_order_number#47, wr_item_sk#46] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] -Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] +Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#69, sum#70] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Aggregate Attributes [2]: [sum#55, sum#56] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] +Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] -Condition : isnotnull(cs_item_sk#79) +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Condition : isnotnull(cs_item_sk#1) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#79] -Right keys [1]: [i_item_sk#85] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#66] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] -Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#71, d_year#72] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#90, d_year#91] +Input [2]: [d_date_sk#71, d_year#72] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#90, d_year#91] -Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) +Input [2]: [d_date_sk#71, d_year#72] +Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) (83) BroadcastExchange -Input [2]: [d_date_sk#90, d_year#91] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] +Input [2]: [d_date_sk#71, d_year#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#83] -Right keys [1]: [d_date_sk#90] +Left keys [1]: [cs_sold_date_sk#5] +Right keys [1]: [d_date_sk#71] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (86) Exchange -Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] -Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#80, cs_item_sk#79] -Right keys [2]: [cr_order_number#95, cr_item_sk#94] +Left keys [2]: [cs_order_number#2, cs_item_sk#1] +Right keys [2]: [cr_order_number#19, cr_item_sk#18] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] -Condition : isnotnull(ss_item_sk#98) +Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Condition : isnotnull(ss_item_sk#26) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#98] -Right keys [1]: [i_item_sk#103] +Left keys [1]: [ss_item_sk#26] +Right keys [1]: [i_item_sk#66] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] -Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#108, d_year#109] +Output [2]: [d_date_sk#71, d_year#72] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#102] -Right keys [1]: [d_date_sk#108] +Left keys [1]: [ss_sold_date_sk#30] +Right keys [1]: [d_date_sk#71] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (101) Exchange -Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] -Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] -Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] +Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] -Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] -Condition : isnotnull(ws_item_sk#116) +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_item_sk#40) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#116] -Right keys [1]: [i_item_sk#121] +Left keys [1]: [ws_item_sk#40] +Right keys [1]: [i_item_sk#66] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] -Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#126, d_year#127] +Output [2]: [d_date_sk#71, d_year#72] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#120] -Right keys [1]: [d_date_sk#126] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#71] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (116) Exchange -Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] -Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] -Right keys [2]: [wr_order_number#130, wr_item_sk#129] +Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] +Right keys [2]: [wr_order_number#47, wr_item_sk#46] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] -Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] +Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#135, sum#136] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Aggregate Attributes [2]: [sum#82, sum#83] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] (127) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] -Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] -Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] +Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] (129) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] -Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] (130) Sort [codegen id : 50] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] -Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 (131) 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#86, i_class_id#87, i_category_id#88, i_manufact_id#89] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] (133) TakeOrderedAndProject -Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] -Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST, sales_amt_diff#150 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST, sales_amt_diff#97 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#71, d_year#72] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 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 422443509b417..e6ade3880e2d4 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 @@ -222,38 +222,38 @@ Output [4]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_s Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25] (26) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#26] +Output [1]: [d_date_sk#6] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#26 as bigint)] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (28) Project [codegen id : 6] Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#26] +Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#6] (29) HashAggregate [codegen id : 6] Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] Keys [1]: [s_store_sk#25] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#27, sum#28] -Results [3]: [s_store_sk#25, sum#29, sum#30] +Aggregate Attributes [2]: [sum#26, sum#27] +Results [3]: [s_store_sk#25, sum#28, sum#29] (30) Exchange -Input [3]: [s_store_sk#25, sum#29, sum#30] -Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [s_store_sk#25, sum#28, sum#29] +Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#25, sum#29, sum#30] +Input [3]: [s_store_sk#25, sum#28, sum#29] Keys [1]: [s_store_sk#25] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] -Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] +Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] (32) BroadcastExchange -Input [3]: [s_store_sk#25, returns#34, profit_loss#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [3]: [s_store_sk#25, returns#33, profit_loss#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] @@ -261,341 +261,341 @@ Right keys [1]: [s_store_sk#25] Join condition: None (34) Project [codegen id : 8] -Output [5]: [store channel AS channel#37, s_store_sk#9 AS id#38, sales#18, coalesce(returns#34, 0.00) AS returns#39, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#40] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#34, profit_loss#35] +Output [5]: [store channel AS channel#36, s_store_sk#9 AS id#37, sales#18, coalesce(returns#33, 0.00) AS returns#38, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#39] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] +Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] +Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#45] +Output [1]: [d_date_sk#6] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#44] -Right keys [1]: [d_date_sk#45] +Left keys [1]: [cs_sold_date_sk#43] +Right keys [1]: [d_date_sk#6] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] -Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] +Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] +Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] -Keys [1]: [cs_call_center_sk#41] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] -Aggregate Attributes [2]: [sum#46, sum#47] -Results [3]: [cs_call_center_sk#41, sum#48, sum#49] +Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] +Keys [1]: [cs_call_center_sk#40] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] +Aggregate Attributes [2]: [sum#44, sum#45] +Results [3]: [cs_call_center_sk#40, sum#46, sum#47] (41) Exchange -Input [3]: [cs_call_center_sk#41, sum#48, sum#49] -Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [3]: [cs_call_center_sk#40, sum#46, sum#47] +Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] (42) HashAggregate [codegen id : 14] -Input [3]: [cs_call_center_sk#41, sum#48, sum#49] -Keys [1]: [cs_call_center_sk#41] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] -Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] +Input [3]: [cs_call_center_sk#40, sum#46, sum#47] +Keys [1]: [cs_call_center_sk#40] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] +Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] (43) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Output [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#55), dynamicpruningexpression(cr_returned_date_sk#55 IN dynamicpruning#5)] ReadSchema: struct (44) ColumnarToRow [codegen id : 12] -Input [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Input [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] (45) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#58] +Output [1]: [d_date_sk#6] (46) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cr_returned_date_sk#57] -Right keys [1]: [d_date_sk#58] +Left keys [1]: [cr_returned_date_sk#55] +Right keys [1]: [d_date_sk#6] Join condition: None (47) Project [codegen id : 12] -Output [2]: [cr_return_amount#55, cr_net_loss#56] -Input [4]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57, d_date_sk#58] +Output [2]: [cr_return_amount#53, cr_net_loss#54] +Input [4]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55, d_date_sk#6] (48) HashAggregate [codegen id : 12] -Input [2]: [cr_return_amount#55, cr_net_loss#56] +Input [2]: [cr_return_amount#53, cr_net_loss#54] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#55)), partial_sum(UnscaledValue(cr_net_loss#56))] -Aggregate Attributes [2]: [sum#59, sum#60] -Results [2]: [sum#61, sum#62] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#53)), partial_sum(UnscaledValue(cr_net_loss#54))] +Aggregate Attributes [2]: [sum#56, sum#57] +Results [2]: [sum#58, sum#59] (49) Exchange -Input [2]: [sum#61, sum#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] +Input [2]: [sum#58, sum#59] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#60] (50) HashAggregate [codegen id : 13] -Input [2]: [sum#61, sum#62] +Input [2]: [sum#58, sum#59] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#55)), sum(UnscaledValue(cr_net_loss#56))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#55))#64, sum(UnscaledValue(cr_net_loss#56))#65] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#55))#64,17,2) AS returns#66, MakeDecimal(sum(UnscaledValue(cr_net_loss#56))#65,17,2) AS profit_loss#67] +Functions [2]: [sum(UnscaledValue(cr_return_amount#53)), sum(UnscaledValue(cr_net_loss#54))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#53))#61, sum(UnscaledValue(cr_net_loss#54))#62] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#53))#61,17,2) AS returns#63, MakeDecimal(sum(UnscaledValue(cr_net_loss#54))#62,17,2) AS profit_loss#64] (51) BroadcastExchange -Input [2]: [returns#66, profit_loss#67] -Arguments: IdentityBroadcastMode, [id=#68] +Input [2]: [returns#63, profit_loss#64] +Arguments: IdentityBroadcastMode, [id=#65] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#69, cs_call_center_sk#41 AS id#70, sales#53, returns#66, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#67 as decimal(18,2)))), DecimalType(18,2), true) AS profit#71] -Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#66, profit_loss#67] +Output [5]: [catalog channel AS channel#66, cs_call_center_sk#40 AS id#67, sales#51, returns#63, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#64 as decimal(18,2)))), DecimalType(18,2), true) AS profit#68] +Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#63, profit_loss#64] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] -Condition : isnotnull(ws_web_page_sk#72) +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Condition : isnotnull(ws_web_page_sk#69) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#76] +Output [1]: [d_date_sk#6] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#75] -Right keys [1]: [d_date_sk#76] +Left keys [1]: [ws_sold_date_sk#72] +Right keys [1]: [d_date_sk#6] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] -Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] +Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] +Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#77] +Output [1]: [wp_web_page_sk#73] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#77] +Input [1]: [wp_web_page_sk#73] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#77] -Condition : isnotnull(wp_web_page_sk#77) +Input [1]: [wp_web_page_sk#73] +Condition : isnotnull(wp_web_page_sk#73) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#77] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] +Input [1]: [wp_web_page_sk#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#72] -Right keys [1]: [wp_web_page_sk#77] +Left keys [1]: [ws_web_page_sk#69] +Right keys [1]: [wp_web_page_sk#73] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] -Aggregate Attributes [2]: [sum#79, sum#80] -Results [3]: [wp_web_page_sk#77, sum#81, sum#82] +Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Keys [1]: [wp_web_page_sk#73] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] +Aggregate Attributes [2]: [sum#75, sum#76] +Results [3]: [wp_web_page_sk#73, sum#77, sum#78] (67) Exchange -Input [3]: [wp_web_page_sk#77, sum#81, sum#82] -Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [3]: [wp_web_page_sk#73, sum#77, sum#78] +Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#77, sum#81, sum#82] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] -Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] +Input [3]: [wp_web_page_sk#73, sum#77, sum#78] +Keys [1]: [wp_web_page_sk#73] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] +Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] -Condition : isnotnull(wr_web_page_sk#88) +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Condition : isnotnull(wr_web_page_sk#84) (72) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#92] +Output [1]: [wp_web_page_sk#88] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#88] -Right keys [1]: [cast(wp_web_page_sk#92 as bigint)] +Left keys [1]: [wr_web_page_sk#84] +Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [4]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] -Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] +Output [4]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] +Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] (75) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#93] +Output [1]: [d_date_sk#6] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#91] -Right keys [1]: [cast(d_date_sk#93 as bigint)] +Left keys [1]: [wr_returned_date_sk#87] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] -Input [5]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92, d_date_sk#93] +Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Input [5]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88, d_date_sk#6] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] -Keys [1]: [wp_web_page_sk#92] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] -Aggregate Attributes [2]: [sum#94, sum#95] -Results [3]: [wp_web_page_sk#92, sum#96, sum#97] +Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Keys [1]: [wp_web_page_sk#88] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] +Aggregate Attributes [2]: [sum#89, sum#90] +Results [3]: [wp_web_page_sk#88, sum#91, sum#92] (79) Exchange -Input [3]: [wp_web_page_sk#92, sum#96, sum#97] -Arguments: hashpartitioning(wp_web_page_sk#92, 5), ENSURE_REQUIREMENTS, [id=#98] +Input [3]: [wp_web_page_sk#88, sum#91, sum#92] +Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#92, sum#96, sum#97] -Keys [1]: [wp_web_page_sk#92] -Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] -Results [3]: [wp_web_page_sk#92, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] +Input [3]: [wp_web_page_sk#88, sum#91, sum#92] +Keys [1]: [wp_web_page_sk#88] +Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] +Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#92, returns#101, profit_loss#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] +Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#77] -Right keys [1]: [wp_web_page_sk#92] +Left keys [1]: [wp_web_page_sk#73] +Right keys [1]: [wp_web_page_sk#88] Join condition: None (83) Project [codegen id : 22] -Output [5]: [web channel AS channel#104, wp_web_page_sk#77 AS id#105, sales#86, coalesce(returns#101, 0.00) AS returns#106, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#107] -Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#92, returns#101, profit_loss#102] +Output [5]: [web channel AS channel#99, wp_web_page_sk#73 AS id#100, sales#82, coalesce(returns#96, 0.00) AS returns#101, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#102] +Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] (84) Union (85) HashAggregate [codegen id : 23] -Input [5]: [channel#37, id#38, sales#18, returns#39, profit#40] -Keys [2]: [channel#37, id#38] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#39), partial_sum(profit#40)] -Aggregate Attributes [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -Results [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Input [5]: [channel#36, id#37, sales#18, returns#38, profit#39] +Keys [2]: [channel#36, id#37] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#38), partial_sum(profit#39)] +Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Results [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] (86) Exchange -Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [id=#120] +Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Arguments: hashpartitioning(channel#36, id#37, 5), ENSURE_REQUIREMENTS, [id=#115] (87) HashAggregate [codegen id : 24] -Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#40)] -Aggregate Attributes [3]: [sum(sales#18)#121, sum(returns#39)#122, sum(profit#40)#123] -Results [5]: [channel#37, id#38, cast(sum(sales#18)#121 as decimal(37,2)) AS sales#124, cast(sum(returns#39)#122 as decimal(37,2)) AS returns#125, cast(sum(profit#40)#123 as decimal(38,2)) AS profit#126] +Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#36, id#37] +Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#39)] +Aggregate Attributes [3]: [sum(sales#18)#116, sum(returns#38)#117, sum(profit#39)#118] +Results [5]: [channel#36, id#37, cast(sum(sales#18)#116 as decimal(37,2)) AS sales#119, cast(sum(returns#38)#117 as decimal(37,2)) AS returns#120, cast(sum(profit#39)#118 as decimal(38,2)) AS profit#121] (88) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Output [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] (89) HashAggregate [codegen id : 48] -Input [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#133)] -Aggregate Attributes [3]: [sum(sales#18)#134, sum(returns#39)#135, sum(profit#133)#136] -Results [4]: [channel#37, sum(sales#18)#134 AS sales#137, sum(returns#39)#135 AS returns#138, sum(profit#133)#136 AS profit#139] +Input [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] +Keys [2]: [channel#36, id#37] +Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#128)] +Aggregate Attributes [3]: [sum(sales#18)#129, sum(returns#38)#130, sum(profit#128)#131] +Results [4]: [channel#36, sum(sales#18)#129 AS sales#132, sum(returns#38)#130 AS returns#133, sum(profit#128)#131 AS profit#134] (90) HashAggregate [codegen id : 48] -Input [4]: [channel#37, sales#137, returns#138, profit#139] -Keys [1]: [channel#37] -Functions [3]: [partial_sum(sales#137), partial_sum(returns#138), partial_sum(profit#139)] -Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -Results [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Input [4]: [channel#36, sales#132, returns#133, profit#134] +Keys [1]: [channel#36] +Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] +Aggregate Attributes [6]: [sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +Results [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] (91) Exchange -Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [id=#152] +Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Arguments: hashpartitioning(channel#36, 5), ENSURE_REQUIREMENTS, [id=#147] (92) HashAggregate [codegen id : 49] -Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -Keys [1]: [channel#37] -Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] -Aggregate Attributes [3]: [sum(sales#137)#153, sum(returns#138)#154, sum(profit#139)#155] -Results [5]: [channel#37, null AS id#156, sum(sales#137)#153 AS sales#157, sum(returns#138)#154 AS returns#158, sum(profit#139)#155 AS profit#159] +Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Keys [1]: [channel#36] +Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] +Aggregate Attributes [3]: [sum(sales#132)#148, sum(returns#133)#149, sum(profit#134)#150] +Results [5]: [channel#36, null AS id#151, sum(sales#132)#148 AS sales#152, sum(returns#133)#149 AS returns#153, sum(profit#134)#150 AS profit#154] (93) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +Output [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] (94) HashAggregate [codegen id : 73] -Input [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#166)] -Aggregate Attributes [3]: [sum(sales#18)#167, sum(returns#39)#168, sum(profit#166)#169] -Results [3]: [sum(sales#18)#167 AS sales#137, sum(returns#39)#168 AS returns#138, sum(profit#166)#169 AS profit#139] +Input [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] +Keys [2]: [channel#36, id#37] +Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#161)] +Aggregate Attributes [3]: [sum(sales#18)#162, sum(returns#38)#163, sum(profit#161)#164] +Results [3]: [sum(sales#18)#162 AS sales#132, sum(returns#38)#163 AS returns#133, sum(profit#161)#164 AS profit#134] (95) HashAggregate [codegen id : 73] -Input [3]: [sales#137, returns#138, profit#139] +Input [3]: [sales#132, returns#133, profit#134] Keys: [] -Functions [3]: [partial_sum(sales#137), partial_sum(returns#138), partial_sum(profit#139)] -Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] -Results [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] +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] (96) Exchange -Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#182] +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#177] (97) HashAggregate [codegen id : 74] -Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] Keys: [] -Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] -Aggregate Attributes [3]: [sum(sales#137)#183, sum(returns#138)#184, sum(profit#139)#185] -Results [5]: [null AS channel#186, null AS id#187, sum(sales#137)#183 AS sales#188, sum(returns#138)#184 AS returns#189, sum(profit#139)#185 AS profit#190] +Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] +Aggregate Attributes [3]: [sum(sales#132)#178, sum(returns#133)#179, sum(profit#134)#180] +Results [5]: [null AS channel#181, null AS id#182, sum(sales#132)#178 AS sales#183, sum(returns#133)#179 AS returns#184, sum(profit#134)#180 AS profit#185] (98) Union (99) HashAggregate [codegen id : 75] -Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] -Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] (100) Exchange -Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] -Arguments: hashpartitioning(channel#37, id#38, sales#124, returns#125, profit#126, 5), ENSURE_REQUIREMENTS, [id=#191] +Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Arguments: hashpartitioning(channel#36, id#37, sales#119, returns#120, profit#121, 5), ENSURE_REQUIREMENTS, [id=#186] (101) HashAggregate [codegen id : 76] -Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] -Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] (102) TakeOrderedAndProject -Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] -Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#124, returns#125, profit#126] +Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Arguments: 100, [channel#36 ASC NULLS FIRST, id#37 ASC NULLS FIRST], [channel#36, id#37, sales#119, returns#120, profit#121] ===== Subqueries ===== @@ -611,14 +611,14 @@ ReusedExchange (104) (104) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#26] +Output [1]: [d_date_sk#6] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#55 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 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 705277b65ff0a..0ec2ed7cca356 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 @@ -210,392 +210,392 @@ Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_s Condition : isnotnull(sr_store_sk#20) (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#6] (24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#25 as bigint)] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (25) Project [codegen id : 6] Output [3]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22] -Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#25] +Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#6] (26) ReusedExchange [Reuses operator id: 14] -Output [1]: [s_store_sk#26] +Output [1]: [s_store_sk#25] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_store_sk#20] -Right keys [1]: [cast(s_store_sk#26 as bigint)] +Right keys [1]: [cast(s_store_sk#25 as bigint)] Join condition: None (28) Project [codegen id : 6] -Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] -Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#26] +Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] +Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#25] (29) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] -Keys [1]: [s_store_sk#26] +Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] +Keys [1]: [s_store_sk#25] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#27, sum#28] -Results [3]: [s_store_sk#26, sum#29, sum#30] +Aggregate Attributes [2]: [sum#26, sum#27] +Results [3]: [s_store_sk#25, sum#28, sum#29] (30) Exchange -Input [3]: [s_store_sk#26, sum#29, sum#30] -Arguments: hashpartitioning(s_store_sk#26, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [s_store_sk#25, sum#28, sum#29] +Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#26, sum#29, sum#30] -Keys [1]: [s_store_sk#26] +Input [3]: [s_store_sk#25, sum#28, sum#29] +Keys [1]: [s_store_sk#25] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] -Results [3]: [s_store_sk#26, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] +Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] (32) BroadcastExchange -Input [3]: [s_store_sk#26, returns#34, profit_loss#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [3]: [s_store_sk#25, returns#33, profit_loss#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] -Right keys [1]: [s_store_sk#26] +Right keys [1]: [s_store_sk#25] Join condition: None (34) Project [codegen id : 8] -Output [5]: [store channel AS channel#37, s_store_sk#9 AS id#38, sales#18, coalesce(returns#34, 0.00) AS returns#39, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#40] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#26, returns#34, profit_loss#35] +Output [5]: [store channel AS channel#36, s_store_sk#9 AS id#37, sales#18, coalesce(returns#33, 0.00) AS returns#38, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#39] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] +Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] +Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#45] +Output [1]: [d_date_sk#6] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#44] -Right keys [1]: [d_date_sk#45] +Left keys [1]: [cs_sold_date_sk#43] +Right keys [1]: [d_date_sk#6] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] -Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] +Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] +Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] -Keys [1]: [cs_call_center_sk#41] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] -Aggregate Attributes [2]: [sum#46, sum#47] -Results [3]: [cs_call_center_sk#41, sum#48, sum#49] +Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] +Keys [1]: [cs_call_center_sk#40] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] +Aggregate Attributes [2]: [sum#44, sum#45] +Results [3]: [cs_call_center_sk#40, sum#46, sum#47] (41) Exchange -Input [3]: [cs_call_center_sk#41, sum#48, sum#49] -Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [3]: [cs_call_center_sk#40, sum#46, sum#47] +Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#41, sum#48, sum#49] -Keys [1]: [cs_call_center_sk#41] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] -Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] +Input [3]: [cs_call_center_sk#40, sum#46, sum#47] +Keys [1]: [cs_call_center_sk#40] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] +Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] (43) BroadcastExchange -Input [3]: [cs_call_center_sk#41, sales#53, profit#54] -Arguments: IdentityBroadcastMode, [id=#55] +Input [3]: [cs_call_center_sk#40, sales#51, profit#52] +Arguments: IdentityBroadcastMode, [id=#53] (44) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] +Output [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#58), dynamicpruningexpression(cr_returned_date_sk#58 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] ReadSchema: struct (45) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] +Input [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] (46) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#59] +Output [1]: [d_date_sk#6] (47) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#58] -Right keys [1]: [d_date_sk#59] +Left keys [1]: [cr_returned_date_sk#56] +Right keys [1]: [d_date_sk#6] Join condition: None (48) Project [codegen id : 13] -Output [2]: [cr_return_amount#56, cr_net_loss#57] -Input [4]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58, d_date_sk#59] +Output [2]: [cr_return_amount#54, cr_net_loss#55] +Input [4]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56, d_date_sk#6] (49) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#56, cr_net_loss#57] +Input [2]: [cr_return_amount#54, cr_net_loss#55] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#56)), partial_sum(UnscaledValue(cr_net_loss#57))] -Aggregate Attributes [2]: [sum#60, sum#61] -Results [2]: [sum#62, sum#63] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#54)), partial_sum(UnscaledValue(cr_net_loss#55))] +Aggregate Attributes [2]: [sum#57, sum#58] +Results [2]: [sum#59, sum#60] (50) Exchange -Input [2]: [sum#62, sum#63] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#64] +Input [2]: [sum#59, sum#60] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] (51) HashAggregate -Input [2]: [sum#62, sum#63] +Input [2]: [sum#59, sum#60] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#56)), sum(UnscaledValue(cr_net_loss#57))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#56))#65, sum(UnscaledValue(cr_net_loss#57))#66] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#56))#65,17,2) AS returns#67, MakeDecimal(sum(UnscaledValue(cr_net_loss#57))#66,17,2) AS profit_loss#68] +Functions [2]: [sum(UnscaledValue(cr_return_amount#54)), sum(UnscaledValue(cr_net_loss#55))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#54))#62, sum(UnscaledValue(cr_net_loss#55))#63] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#54))#62,17,2) AS returns#64, MakeDecimal(sum(UnscaledValue(cr_net_loss#55))#63,17,2) AS profit_loss#65] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#69, cs_call_center_sk#41 AS id#70, sales#53, returns#67, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#68 as decimal(18,2)))), DecimalType(18,2), true) AS profit#71] -Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#67, profit_loss#68] +Output [5]: [catalog channel AS channel#66, cs_call_center_sk#40 AS id#67, sales#51, returns#64, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#65 as decimal(18,2)))), DecimalType(18,2), true) AS profit#68] +Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#64, profit_loss#65] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] -Condition : isnotnull(ws_web_page_sk#72) +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Condition : isnotnull(ws_web_page_sk#69) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#76] +Output [1]: [d_date_sk#6] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#75] -Right keys [1]: [d_date_sk#76] +Left keys [1]: [ws_sold_date_sk#72] +Right keys [1]: [d_date_sk#6] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] -Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] +Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] +Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#77] +Output [1]: [wp_web_page_sk#73] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#77] +Input [1]: [wp_web_page_sk#73] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#77] -Condition : isnotnull(wp_web_page_sk#77) +Input [1]: [wp_web_page_sk#73] +Condition : isnotnull(wp_web_page_sk#73) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#77] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] +Input [1]: [wp_web_page_sk#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#72] -Right keys [1]: [wp_web_page_sk#77] +Left keys [1]: [ws_web_page_sk#69] +Right keys [1]: [wp_web_page_sk#73] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] -Aggregate Attributes [2]: [sum#79, sum#80] -Results [3]: [wp_web_page_sk#77, sum#81, sum#82] +Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Keys [1]: [wp_web_page_sk#73] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] +Aggregate Attributes [2]: [sum#75, sum#76] +Results [3]: [wp_web_page_sk#73, sum#77, sum#78] (67) Exchange -Input [3]: [wp_web_page_sk#77, sum#81, sum#82] -Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [3]: [wp_web_page_sk#73, sum#77, sum#78] +Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#77, sum#81, sum#82] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] -Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] +Input [3]: [wp_web_page_sk#73, sum#77, sum#78] +Keys [1]: [wp_web_page_sk#73] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] +Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] -Condition : isnotnull(wr_web_page_sk#88) +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Condition : isnotnull(wr_web_page_sk#84) (72) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#92] +Output [1]: [d_date_sk#6] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#91] -Right keys [1]: [cast(d_date_sk#92 as bigint)] +Left keys [1]: [wr_returned_date_sk#87] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90] -Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, d_date_sk#92] +Output [3]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86] +Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, d_date_sk#6] (75) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#93] +Output [1]: [wp_web_page_sk#88] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#88] -Right keys [1]: [cast(wp_web_page_sk#93 as bigint)] +Left keys [1]: [wr_web_page_sk#84] +Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] +Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] -Keys [1]: [wp_web_page_sk#93] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] -Aggregate Attributes [2]: [sum#94, sum#95] -Results [3]: [wp_web_page_sk#93, sum#96, sum#97] +Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Keys [1]: [wp_web_page_sk#88] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] +Aggregate Attributes [2]: [sum#89, sum#90] +Results [3]: [wp_web_page_sk#88, sum#91, sum#92] (79) Exchange -Input [3]: [wp_web_page_sk#93, sum#96, sum#97] -Arguments: hashpartitioning(wp_web_page_sk#93, 5), ENSURE_REQUIREMENTS, [id=#98] +Input [3]: [wp_web_page_sk#88, sum#91, sum#92] +Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#93, sum#96, sum#97] -Keys [1]: [wp_web_page_sk#93] -Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] -Results [3]: [wp_web_page_sk#93, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] +Input [3]: [wp_web_page_sk#88, sum#91, sum#92] +Keys [1]: [wp_web_page_sk#88] +Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] +Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#93, returns#101, profit_loss#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] +Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#77] -Right keys [1]: [wp_web_page_sk#93] +Left keys [1]: [wp_web_page_sk#73] +Right keys [1]: [wp_web_page_sk#88] Join condition: None (83) Project [codegen id : 22] -Output [5]: [web channel AS channel#104, wp_web_page_sk#77 AS id#105, sales#86, coalesce(returns#101, 0.00) AS returns#106, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#107] -Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#93, returns#101, profit_loss#102] +Output [5]: [web channel AS channel#99, wp_web_page_sk#73 AS id#100, sales#82, coalesce(returns#96, 0.00) AS returns#101, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#102] +Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] (84) Union (85) HashAggregate [codegen id : 23] -Input [5]: [channel#37, id#38, sales#18, returns#39, profit#40] -Keys [2]: [channel#37, id#38] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#39), partial_sum(profit#40)] -Aggregate Attributes [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -Results [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Input [5]: [channel#36, id#37, sales#18, returns#38, profit#39] +Keys [2]: [channel#36, id#37] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#38), partial_sum(profit#39)] +Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Results [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] (86) Exchange -Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [id=#120] +Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Arguments: hashpartitioning(channel#36, id#37, 5), ENSURE_REQUIREMENTS, [id=#115] (87) HashAggregate [codegen id : 24] -Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#40)] -Aggregate Attributes [3]: [sum(sales#18)#121, sum(returns#39)#122, sum(profit#40)#123] -Results [5]: [channel#37, id#38, cast(sum(sales#18)#121 as decimal(37,2)) AS sales#124, cast(sum(returns#39)#122 as decimal(37,2)) AS returns#125, cast(sum(profit#40)#123 as decimal(38,2)) AS profit#126] +Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#36, id#37] +Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#39)] +Aggregate Attributes [3]: [sum(sales#18)#116, sum(returns#38)#117, sum(profit#39)#118] +Results [5]: [channel#36, id#37, cast(sum(sales#18)#116 as decimal(37,2)) AS sales#119, cast(sum(returns#38)#117 as decimal(37,2)) AS returns#120, cast(sum(profit#39)#118 as decimal(38,2)) AS profit#121] (88) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Output [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] (89) HashAggregate [codegen id : 48] -Input [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#133)] -Aggregate Attributes [3]: [sum(sales#18)#134, sum(returns#39)#135, sum(profit#133)#136] -Results [4]: [channel#37, sum(sales#18)#134 AS sales#137, sum(returns#39)#135 AS returns#138, sum(profit#133)#136 AS profit#139] +Input [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] +Keys [2]: [channel#36, id#37] +Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#128)] +Aggregate Attributes [3]: [sum(sales#18)#129, sum(returns#38)#130, sum(profit#128)#131] +Results [4]: [channel#36, sum(sales#18)#129 AS sales#132, sum(returns#38)#130 AS returns#133, sum(profit#128)#131 AS profit#134] (90) HashAggregate [codegen id : 48] -Input [4]: [channel#37, sales#137, returns#138, profit#139] -Keys [1]: [channel#37] -Functions [3]: [partial_sum(sales#137), partial_sum(returns#138), partial_sum(profit#139)] -Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -Results [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Input [4]: [channel#36, sales#132, returns#133, profit#134] +Keys [1]: [channel#36] +Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] +Aggregate Attributes [6]: [sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +Results [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] (91) Exchange -Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [id=#152] +Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Arguments: hashpartitioning(channel#36, 5), ENSURE_REQUIREMENTS, [id=#147] (92) HashAggregate [codegen id : 49] -Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -Keys [1]: [channel#37] -Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] -Aggregate Attributes [3]: [sum(sales#137)#153, sum(returns#138)#154, sum(profit#139)#155] -Results [5]: [channel#37, null AS id#156, sum(sales#137)#153 AS sales#157, sum(returns#138)#154 AS returns#158, sum(profit#139)#155 AS profit#159] +Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Keys [1]: [channel#36] +Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] +Aggregate Attributes [3]: [sum(sales#132)#148, sum(returns#133)#149, sum(profit#134)#150] +Results [5]: [channel#36, null AS id#151, sum(sales#132)#148 AS sales#152, sum(returns#133)#149 AS returns#153, sum(profit#134)#150 AS profit#154] (93) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +Output [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] (94) HashAggregate [codegen id : 73] -Input [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#166)] -Aggregate Attributes [3]: [sum(sales#18)#167, sum(returns#39)#168, sum(profit#166)#169] -Results [3]: [sum(sales#18)#167 AS sales#137, sum(returns#39)#168 AS returns#138, sum(profit#166)#169 AS profit#139] +Input [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] +Keys [2]: [channel#36, id#37] +Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#161)] +Aggregate Attributes [3]: [sum(sales#18)#162, sum(returns#38)#163, sum(profit#161)#164] +Results [3]: [sum(sales#18)#162 AS sales#132, sum(returns#38)#163 AS returns#133, sum(profit#161)#164 AS profit#134] (95) HashAggregate [codegen id : 73] -Input [3]: [sales#137, returns#138, profit#139] +Input [3]: [sales#132, returns#133, profit#134] Keys: [] -Functions [3]: [partial_sum(sales#137), partial_sum(returns#138), partial_sum(profit#139)] -Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] -Results [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] +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] (96) Exchange -Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#182] +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#177] (97) HashAggregate [codegen id : 74] -Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] Keys: [] -Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] -Aggregate Attributes [3]: [sum(sales#137)#183, sum(returns#138)#184, sum(profit#139)#185] -Results [5]: [null AS channel#186, null AS id#187, sum(sales#137)#183 AS sales#188, sum(returns#138)#184 AS returns#189, sum(profit#139)#185 AS profit#190] +Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] +Aggregate Attributes [3]: [sum(sales#132)#178, sum(returns#133)#179, sum(profit#134)#180] +Results [5]: [null AS channel#181, null AS id#182, sum(sales#132)#178 AS sales#183, sum(returns#133)#179 AS returns#184, sum(profit#134)#180 AS profit#185] (98) Union (99) HashAggregate [codegen id : 75] -Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] -Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] (100) Exchange -Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] -Arguments: hashpartitioning(channel#37, id#38, sales#124, returns#125, profit#126, 5), ENSURE_REQUIREMENTS, [id=#191] +Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Arguments: hashpartitioning(channel#36, id#37, sales#119, returns#120, profit#121, 5), ENSURE_REQUIREMENTS, [id=#186] (101) HashAggregate [codegen id : 76] -Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] -Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] (102) TakeOrderedAndProject -Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] -Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#124, returns#125, profit#126] +Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Arguments: 100, [channel#36 ASC NULLS FIRST, id#37 ASC NULLS FIRST], [channel#36, id#37, sales#119, returns#120, profit#121] ===== Subqueries ===== @@ -611,14 +611,14 @@ ReusedExchange (104) (104) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#6] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#58 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt index a12ab50a89f43..0bd6422fffa5a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt @@ -248,164 +248,164 @@ Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale Input [9]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#14, d_year#15] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#14] Join condition: None (41) Project [codegen id : 13] -Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] -Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#43, d_year#44] +Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] +Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#14, d_year#15] (42) HashAggregate [codegen id : 13] -Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] -Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] +Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [partial_sum(cs_quantity#34), partial_sum(UnscaledValue(cs_wholesale_cost#35)), partial_sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum#45, sum#46, sum#47] -Results [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] +Aggregate Attributes [3]: [sum#43, sum#44, sum#45] +Results [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] (43) Exchange -Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] -Arguments: hashpartitioning(d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] +Arguments: hashpartitioning(d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#49] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] -Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] +Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [sum(cs_quantity#34), sum(UnscaledValue(cs_wholesale_cost#35)), sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum(cs_quantity#34)#52, sum(UnscaledValue(cs_wholesale_cost#35))#53, sum(UnscaledValue(cs_sales_price#36))#54] -Results [6]: [d_year#44 AS cs_sold_year#55, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#56, sum(cs_quantity#34)#52 AS cs_qty#57, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#53,17,2) AS cs_wc#58, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#54,17,2) AS cs_sp#59] +Aggregate Attributes [3]: [sum(cs_quantity#34)#50, sum(UnscaledValue(cs_wholesale_cost#35))#51, sum(UnscaledValue(cs_sales_price#36))#52] +Results [6]: [d_year#15 AS cs_sold_year#53, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#54, sum(cs_quantity#34)#50 AS cs_qty#55, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#51,17,2) AS cs_wc#56, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#52,17,2) AS cs_sp#57] (45) Filter [codegen id : 14] -Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] -Condition : (coalesce(cs_qty#57, 0) > 0) +Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] +Condition : (coalesce(cs_qty#55, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] -Arguments: [cs_sold_year#55 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#56 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] +Arguments: [cs_sold_year#53 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#54 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56] +Right keys [3]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] (49) Scan parquet default.web_sales -Output [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Output [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] (51) Filter [codegen id : 16] -Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] -Condition : (isnotnull(ws_item_sk#60) AND isnotnull(ws_bill_customer_sk#61)) +Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Condition : (isnotnull(ws_item_sk#58) AND isnotnull(ws_bill_customer_sk#59)) (52) Exchange -Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] -Arguments: hashpartitioning(cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint), 5), ENSURE_REQUIREMENTS, [id=#67] +Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Arguments: hashpartitioning(cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint), 5), ENSURE_REQUIREMENTS, [id=#65] (53) Sort [codegen id : 17] -Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] -Arguments: [cast(ws_order_number#62 as bigint) ASC NULLS FIRST, cast(ws_item_sk#60 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Arguments: [cast(ws_order_number#60 as bigint) ASC NULLS FIRST, cast(ws_item_sk#58 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.web_returns -Output [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Output [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] (56) Filter [codegen id : 18] -Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] -Condition : (isnotnull(wr_order_number#69) AND isnotnull(wr_item_sk#68)) +Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Condition : (isnotnull(wr_order_number#67) AND isnotnull(wr_item_sk#66)) (57) Project [codegen id : 18] -Output [2]: [wr_item_sk#68, wr_order_number#69] -Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Output [2]: [wr_item_sk#66, wr_order_number#67] +Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] (58) Exchange -Input [2]: [wr_item_sk#68, wr_order_number#69] -Arguments: hashpartitioning(wr_order_number#69, wr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [2]: [wr_item_sk#66, wr_order_number#67] +Arguments: hashpartitioning(wr_order_number#67, wr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] (59) Sort [codegen id : 19] -Input [2]: [wr_item_sk#68, wr_order_number#69] -Arguments: [wr_order_number#69 ASC NULLS FIRST, wr_item_sk#68 ASC NULLS FIRST], false, 0 +Input [2]: [wr_item_sk#66, wr_order_number#67] +Arguments: [wr_order_number#67 ASC NULLS FIRST, wr_item_sk#66 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint)] -Right keys [2]: [wr_order_number#69, wr_item_sk#68] +Left keys [2]: [cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint)] +Right keys [2]: [wr_order_number#67, wr_item_sk#66] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] -Condition : isnull(wr_order_number#69) +Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] +Condition : isnull(wr_order_number#67) (62) Project [codegen id : 21] -Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] -Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] +Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#72, d_year#73] +Output [2]: [d_date_sk#14, d_year#15] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [ws_sold_date_sk#66] -Right keys [1]: [d_date_sk#72] +Left keys [1]: [ws_sold_date_sk#64] +Right keys [1]: [d_date_sk#14] Join condition: None (65) Project [codegen id : 21] -Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] -Input [8]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, d_date_sk#72, d_year#73] +Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] +Input [8]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, d_date_sk#14, d_year#15] (66) HashAggregate [codegen id : 21] -Input [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] -Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] -Functions [3]: [partial_sum(ws_quantity#63), partial_sum(UnscaledValue(ws_wholesale_cost#64)), partial_sum(UnscaledValue(ws_sales_price#65))] -Aggregate Attributes [3]: [sum#74, sum#75, sum#76] -Results [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] +Input [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] +Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] +Functions [3]: [partial_sum(ws_quantity#61), partial_sum(UnscaledValue(ws_wholesale_cost#62)), partial_sum(UnscaledValue(ws_sales_price#63))] +Aggregate Attributes [3]: [sum#70, sum#71, sum#72] +Results [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] (67) Exchange -Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] -Arguments: hashpartitioning(d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] +Arguments: hashpartitioning(d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#76] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] -Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] -Functions [3]: [sum(ws_quantity#63), sum(UnscaledValue(ws_wholesale_cost#64)), sum(UnscaledValue(ws_sales_price#65))] -Aggregate Attributes [3]: [sum(ws_quantity#63)#81, sum(UnscaledValue(ws_wholesale_cost#64))#82, sum(UnscaledValue(ws_sales_price#65))#83] -Results [6]: [d_year#73 AS ws_sold_year#84, ws_item_sk#60, ws_bill_customer_sk#61 AS ws_customer_sk#85, sum(ws_quantity#63)#81 AS ws_qty#86, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#64))#82,17,2) AS ws_wc#87, MakeDecimal(sum(UnscaledValue(ws_sales_price#65))#83,17,2) AS ws_sp#88] +Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] +Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] +Functions [3]: [sum(ws_quantity#61), sum(UnscaledValue(ws_wholesale_cost#62)), sum(UnscaledValue(ws_sales_price#63))] +Aggregate Attributes [3]: [sum(ws_quantity#61)#77, sum(UnscaledValue(ws_wholesale_cost#62))#78, sum(UnscaledValue(ws_sales_price#63))#79] +Results [6]: [d_year#15 AS ws_sold_year#80, ws_item_sk#58, ws_bill_customer_sk#59 AS ws_customer_sk#81, sum(ws_quantity#61)#77 AS ws_qty#82, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#62))#78,17,2) AS ws_wc#83, MakeDecimal(sum(UnscaledValue(ws_sales_price#63))#79,17,2) AS ws_sp#84] (69) Filter [codegen id : 22] -Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] -Condition : (coalesce(ws_qty#86, 0) > 0) +Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] +Condition : (coalesce(ws_qty#82, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] -Arguments: [ws_sold_year#84 ASC NULLS FIRST, ws_item_sk#60 ASC NULLS FIRST, ws_customer_sk#85 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] +Arguments: [ws_sold_year#80 ASC NULLS FIRST, ws_item_sk#58 ASC NULLS FIRST, ws_customer_sk#81 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85] +Right keys [3]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81] Join condition: None (72) Project [codegen id : 23] -Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#86 + cs_qty#57), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#86, 0) + coalesce(cs_qty#57, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#87, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#58, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#88, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#59, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59, ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] +Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#82 + cs_qty#55), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#82, 0) + coalesce(cs_qty#55, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#83, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#56, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#84, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#57, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57, ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] (73) TakeOrderedAndProject -Input [13]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, ratio#89 ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] +Input [13]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, ratio#85 ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt index b14ab47d28afe..2d84cacd5d09c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt @@ -248,164 +248,164 @@ Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale Input [9]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_order_number#33, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, wr_item_sk#39, wr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#14, d_year#15] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#37] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#14] Join condition: None (41) Project [codegen id : 13] -Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] -Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#43, d_year#44] +Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] +Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#14, d_year#15] (42) HashAggregate [codegen id : 13] -Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] -Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] +Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [partial_sum(ws_quantity#34), partial_sum(UnscaledValue(ws_wholesale_cost#35)), partial_sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum#45, sum#46, sum#47] -Results [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] +Aggregate Attributes [3]: [sum#43, sum#44, sum#45] +Results [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] (43) Exchange -Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] -Arguments: hashpartitioning(d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] +Arguments: hashpartitioning(d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#49] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] -Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] +Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [sum(ws_quantity#34), sum(UnscaledValue(ws_wholesale_cost#35)), sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum(ws_quantity#34)#52, sum(UnscaledValue(ws_wholesale_cost#35))#53, sum(UnscaledValue(ws_sales_price#36))#54] -Results [6]: [d_year#44 AS ws_sold_year#55, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#56, sum(ws_quantity#34)#52 AS ws_qty#57, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#53,17,2) AS ws_wc#58, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#54,17,2) AS ws_sp#59] +Aggregate Attributes [3]: [sum(ws_quantity#34)#50, sum(UnscaledValue(ws_wholesale_cost#35))#51, sum(UnscaledValue(ws_sales_price#36))#52] +Results [6]: [d_year#15 AS ws_sold_year#53, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#54, sum(ws_quantity#34)#50 AS ws_qty#55, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#51,17,2) AS ws_wc#56, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#52,17,2) AS ws_sp#57] (45) Filter [codegen id : 14] -Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] -Condition : (coalesce(ws_qty#57, 0) > 0) +Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] +Condition : (coalesce(ws_qty#55, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] -Arguments: [ws_sold_year#55 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#56 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] +Arguments: [ws_sold_year#53 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#54 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56] +Right keys [3]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] (49) Scan parquet default.catalog_sales -Output [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Output [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#66), dynamicpruningexpression(cs_sold_date_sk#66 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#8)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] (51) Filter [codegen id : 16] -Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] -Condition : (isnotnull(cs_item_sk#61) AND isnotnull(cs_bill_customer_sk#60)) +Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Condition : (isnotnull(cs_item_sk#59) AND isnotnull(cs_bill_customer_sk#58)) (52) Exchange -Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] -Arguments: hashpartitioning(cs_order_number#62, cs_item_sk#61, 5), ENSURE_REQUIREMENTS, [id=#67] +Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Arguments: hashpartitioning(cs_order_number#60, cs_item_sk#59, 5), ENSURE_REQUIREMENTS, [id=#65] (53) Sort [codegen id : 17] -Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] -Arguments: [cs_order_number#62 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST], false, 0 +Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Arguments: [cs_order_number#60 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST], false, 0 (54) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Output [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] (56) Filter [codegen id : 18] -Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] -Condition : (isnotnull(cr_order_number#69) AND isnotnull(cr_item_sk#68)) +Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Condition : (isnotnull(cr_order_number#67) AND isnotnull(cr_item_sk#66)) (57) Project [codegen id : 18] -Output [2]: [cr_item_sk#68, cr_order_number#69] -Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Output [2]: [cr_item_sk#66, cr_order_number#67] +Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] (58) Exchange -Input [2]: [cr_item_sk#68, cr_order_number#69] -Arguments: hashpartitioning(cr_order_number#69, cr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [2]: [cr_item_sk#66, cr_order_number#67] +Arguments: hashpartitioning(cr_order_number#67, cr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] (59) Sort [codegen id : 19] -Input [2]: [cr_item_sk#68, cr_order_number#69] -Arguments: [cr_order_number#69 ASC NULLS FIRST, cr_item_sk#68 ASC NULLS FIRST], false, 0 +Input [2]: [cr_item_sk#66, cr_order_number#67] +Arguments: [cr_order_number#67 ASC NULLS FIRST, cr_item_sk#66 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cs_order_number#62, cs_item_sk#61] -Right keys [2]: [cr_order_number#69, cr_item_sk#68] +Left keys [2]: [cs_order_number#60, cs_item_sk#59] +Right keys [2]: [cr_order_number#67, cr_item_sk#66] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] -Condition : isnull(cr_order_number#69) +Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] +Condition : isnull(cr_order_number#67) (62) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] -Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] +Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#72, d_year#73] +Output [2]: [d_date_sk#14, d_year#15] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [cs_sold_date_sk#66] -Right keys [1]: [d_date_sk#72] +Left keys [1]: [cs_sold_date_sk#64] +Right keys [1]: [d_date_sk#14] Join condition: None (65) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] -Input [8]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, d_date_sk#72, d_year#73] +Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] +Input [8]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, d_date_sk#14, d_year#15] (66) HashAggregate [codegen id : 21] -Input [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] -Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] -Functions [3]: [partial_sum(cs_quantity#63), partial_sum(UnscaledValue(cs_wholesale_cost#64)), partial_sum(UnscaledValue(cs_sales_price#65))] -Aggregate Attributes [3]: [sum#74, sum#75, sum#76] -Results [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] +Input [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] +Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] +Functions [3]: [partial_sum(cs_quantity#61), partial_sum(UnscaledValue(cs_wholesale_cost#62)), partial_sum(UnscaledValue(cs_sales_price#63))] +Aggregate Attributes [3]: [sum#70, sum#71, sum#72] +Results [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] (67) Exchange -Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] -Arguments: hashpartitioning(d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] +Arguments: hashpartitioning(d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, 5), ENSURE_REQUIREMENTS, [id=#76] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] -Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] -Functions [3]: [sum(cs_quantity#63), sum(UnscaledValue(cs_wholesale_cost#64)), sum(UnscaledValue(cs_sales_price#65))] -Aggregate Attributes [3]: [sum(cs_quantity#63)#81, sum(UnscaledValue(cs_wholesale_cost#64))#82, sum(UnscaledValue(cs_sales_price#65))#83] -Results [6]: [d_year#73 AS cs_sold_year#84, cs_item_sk#61, cs_bill_customer_sk#60 AS cs_customer_sk#85, sum(cs_quantity#63)#81 AS cs_qty#86, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#64))#82,17,2) AS cs_wc#87, MakeDecimal(sum(UnscaledValue(cs_sales_price#65))#83,17,2) AS cs_sp#88] +Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] +Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] +Functions [3]: [sum(cs_quantity#61), sum(UnscaledValue(cs_wholesale_cost#62)), sum(UnscaledValue(cs_sales_price#63))] +Aggregate Attributes [3]: [sum(cs_quantity#61)#77, sum(UnscaledValue(cs_wholesale_cost#62))#78, sum(UnscaledValue(cs_sales_price#63))#79] +Results [6]: [d_year#15 AS cs_sold_year#80, cs_item_sk#59, cs_bill_customer_sk#58 AS cs_customer_sk#81, sum(cs_quantity#61)#77 AS cs_qty#82, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#62))#78,17,2) AS cs_wc#83, MakeDecimal(sum(UnscaledValue(cs_sales_price#63))#79,17,2) AS cs_sp#84] (69) Filter [codegen id : 22] -Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] -Condition : (coalesce(cs_qty#86, 0) > 0) +Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] +Condition : (coalesce(cs_qty#82, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] -Arguments: [cs_sold_year#84 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST, cs_customer_sk#85 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] +Arguments: [cs_sold_year#80 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST, cs_customer_sk#81 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85] +Right keys [3]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81] Join condition: None (72) Project [codegen id : 23] -Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#57 + cs_qty#86), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#57, 0) + coalesce(cs_qty#86, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#58, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#87, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#59, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#88, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59, cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] +Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#55 + cs_qty#82), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#55, 0) + coalesce(cs_qty#82, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#56, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#83, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#57, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#84, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57, cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] (73) TakeOrderedAndProject -Input [13]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, ratio#89 ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] +Input [13]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, ratio#85 ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#8 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 a5d47ede475ca..e192ab8d637de 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 @@ -257,7 +257,7 @@ Input [2]: [d_date_sk#22, d_date#23] (30) Filter [codegen id : 7] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-09-03)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 10442)) AND (d_date#23 <= 10472)) AND isnotnull(d_date_sk#22)) (31) Project [codegen id : 7] Output [1]: [d_date_sk#22] @@ -380,224 +380,224 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#61] +Output [1]: [i_item_sk#16] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#61] +Right keys [1]: [i_item_sk#16] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#61] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#16] (60) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#62] +Output [1]: [p_promo_sk#19] (61) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#62] +Right keys [1]: [p_promo_sk#19] Join condition: None (62) Project [codegen id : 19] Output [6]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#62] +Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#19] (63) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#63] +Output [1]: [d_date_sk#22] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#63] +Right keys [1]: [d_date_sk#22] Join condition: None (65) Project [codegen id : 19] Output [5]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#63] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#22] (66) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 18] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] (68) Filter [codegen id : 18] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Condition : isnotnull(cp_catalog_page_sk#64) +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Condition : isnotnull(cp_catalog_page_sk#61) (69) BroadcastExchange -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#64] +Right keys [1]: [cp_catalog_page_sk#61] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#64, cp_catalog_page_id#65] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] -Keys [1]: [cp_catalog_page_id#65] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Keys [1]: [cp_catalog_page_id#62] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Results [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] (73) Exchange -Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] -Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] -Keys [1]: [cp_catalog_page_id#65] +Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Keys [1]: [cp_catalog_page_id#62] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] -Results [5]: [catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#65) AS id#82, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#83, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#84, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#85] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] +Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_page_id#62) AS id#79, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#80, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#81, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#82] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) (78) Exchange -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] (84) Exchange -Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] +Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] -Right keys [2]: [wr_item_sk#94, wr_order_number#95] +Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] +Right keys [2]: [wr_item_sk#91, wr_order_number#92] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] -Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] +Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#100] +Output [1]: [i_item_sk#16] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#86] -Right keys [1]: [i_item_sk#100] +Left keys [1]: [ws_item_sk#83] +Right keys [1]: [i_item_sk#16] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] -Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, i_item_sk#100] +Output [7]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] +Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, i_item_sk#16] (91) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#101] +Output [1]: [p_promo_sk#19] (92) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#88] -Right keys [1]: [p_promo_sk#101] +Left keys [1]: [ws_promo_sk#85] +Right keys [1]: [p_promo_sk#19] Join condition: None (93) Project [codegen id : 29] -Output [6]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] -Input [8]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, p_promo_sk#101] +Output [6]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] +Input [8]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, p_promo_sk#19] (94) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#102] +Output [1]: [d_date_sk#22] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#92] -Right keys [1]: [d_date_sk#102] +Left keys [1]: [ws_sold_date_sk#89] +Right keys [1]: [d_date_sk#22] Join condition: None (96) Project [codegen id : 29] -Output [5]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] -Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#102] +Output [5]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] +Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#22] (97) Scan parquet default.web_site -Output [2]: [web_site_sk#103, web_site_id#104] +Output [2]: [web_site_sk#97, web_site_id#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (98) ColumnarToRow [codegen id : 28] -Input [2]: [web_site_sk#103, web_site_id#104] +Input [2]: [web_site_sk#97, web_site_id#98] (99) Filter [codegen id : 28] -Input [2]: [web_site_sk#103, web_site_id#104] -Condition : isnotnull(web_site_sk#103) +Input [2]: [web_site_sk#97, web_site_id#98] +Condition : isnotnull(web_site_sk#97) (100) BroadcastExchange -Input [2]: [web_site_sk#103, web_site_id#104] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#105] +Input [2]: [web_site_sk#97, web_site_id#98] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#87] -Right keys [1]: [web_site_sk#103] +Left keys [1]: [ws_web_site_sk#84] +Right keys [1]: [web_site_sk#97] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] -Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#103, web_site_id#104] +Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] -Keys [1]: [web_site_id#104] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Results [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Keys [1]: [web_site_id#98] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] (104) Exchange -Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, [id=#116] +Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Keys [1]: [web_site_id#104] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] -Results [5]: [web channel AS channel#120, concat(web_site, web_site_id#104) AS id#121, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#122, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#123, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#124] +Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Keys [1]: [web_site_id#98] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] +Results [5]: [web channel AS channel#114, concat(web_site, web_site_id#98) AS id#115, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#116, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#117, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#118] (106) Union @@ -605,99 +605,99 @@ Results [5]: [web channel AS channel#120, concat(web_site, web_site_id#104) AS i Input [5]: [channel#42, id#43, sales#44, returns#45, profit#46] Keys [2]: [channel#42, id#43] Functions [3]: [partial_sum(sales#44), partial_sum(returns#45), partial_sum(profit#46)] -Aggregate Attributes [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Results [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] +Results [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] (108) Exchange -Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] -Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#131] (109) HashAggregate [codegen id : 32] -Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#138, sum(returns#45)#139, sum(profit#46)#140] -Results [5]: [channel#42, id#43, cast(sum(sales#44)#138 as decimal(37,2)) AS sales#141, cast(sum(returns#45)#139 as decimal(38,2)) AS returns#142, cast(sum(profit#46)#140 as decimal(38,2)) AS profit#143] +Aggregate Attributes [3]: [sum(sales#44)#132, sum(returns#45)#133, sum(profit#46)#134] +Results [5]: [channel#42, id#43, cast(sum(sales#44)#132 as decimal(37,2)) AS sales#135, cast(sum(returns#45)#133 as decimal(38,2)) AS returns#136, cast(sum(profit#46)#134 as decimal(38,2)) AS profit#137] (110) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] +Output [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] (111) HashAggregate [codegen id : 64] -Input [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] +Input [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#150, sum(returns#45)#151, sum(profit#46)#152] -Results [4]: [channel#42, sum(sales#44)#150 AS sales#153, sum(returns#45)#151 AS returns#154, sum(profit#46)#152 AS profit#155] +Aggregate Attributes [3]: [sum(sales#44)#144, sum(returns#45)#145, sum(profit#46)#146] +Results [4]: [channel#42, sum(sales#44)#144 AS sales#147, sum(returns#45)#145 AS returns#148, sum(profit#46)#146 AS profit#149] (112) HashAggregate [codegen id : 64] -Input [4]: [channel#42, sales#153, returns#154, profit#155] +Input [4]: [channel#42, sales#147, returns#148, profit#149] Keys [1]: [channel#42] -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#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] +Aggregate Attributes [6]: [sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] +Results [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] (113) Exchange -Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] -Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#168] +Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#162] (114) HashAggregate [codegen id : 65] -Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] Keys [1]: [channel#42] -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#42, null AS id#172, sum(sales#153)#169 AS sales#173, sum(returns#154)#170 AS returns#174, sum(profit#155)#171 AS profit#175] +Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] +Aggregate Attributes [3]: [sum(sales#147)#163, sum(returns#148)#164, sum(profit#149)#165] +Results [5]: [channel#42, null AS id#166, sum(sales#147)#163 AS sales#167, sum(returns#148)#164 AS returns#168, sum(profit#149)#165 AS profit#169] (115) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Output [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] (116) HashAggregate [codegen id : 97] -Input [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Input [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#182, sum(returns#45)#183, sum(profit#46)#184] -Results [3]: [sum(sales#44)#182 AS sales#153, sum(returns#45)#183 AS returns#154, sum(profit#46)#184 AS profit#155] +Aggregate Attributes [3]: [sum(sales#44)#176, sum(returns#45)#177, sum(profit#46)#178] +Results [3]: [sum(sales#44)#176 AS sales#147, sum(returns#45)#177 AS returns#148, sum(profit#46)#178 AS profit#149] (117) HashAggregate [codegen id : 97] -Input [3]: [sales#153, returns#154, profit#155] +Input [3]: [sales#147, returns#148, profit#149] Keys: [] -Functions [3]: [partial_sum(sales#153), partial_sum(returns#154), partial_sum(profit#155)] -Aggregate Attributes [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] -Results [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] +Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] +Aggregate Attributes [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] +Results [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] (118) Exchange -Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#197] +Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#191] (119) HashAggregate [codegen id : 98] -Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] +Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] Keys: [] -Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] -Aggregate Attributes [3]: [sum(sales#153)#198, sum(returns#154)#199, sum(profit#155)#200] -Results [5]: [null AS channel#201, null AS id#202, sum(sales#153)#198 AS sales#203, sum(returns#154)#199 AS returns#204, sum(profit#155)#200 AS profit#205] +Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] +Aggregate Attributes [3]: [sum(sales#147)#192, sum(returns#148)#193, sum(profit#149)#194] +Results [5]: [null AS channel#195, null AS id#196, sum(sales#147)#192 AS sales#197, sum(returns#148)#193 AS returns#198, sum(profit#149)#194 AS profit#199] (120) Union (121) HashAggregate [codegen id : 99] -Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] -Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] (122) Exchange -Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] -Arguments: hashpartitioning(channel#42, id#43, sales#141, returns#142, profit#143, 5), ENSURE_REQUIREMENTS, [id=#206] +Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Arguments: hashpartitioning(channel#42, id#43, sales#135, returns#136, profit#137, 5), ENSURE_REQUIREMENTS, [id=#200] (123) HashAggregate [codegen id : 100] -Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] -Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] (124) TakeOrderedAndProject -Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] -Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#141, returns#142, profit#143] +Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#135, returns#136, profit#137] ===== Subqueries ===== @@ -710,6 +710,6 @@ Output [1]: [d_date_sk#22] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 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 9e5bd3aeaa1d1..e027ef7e53e8a 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 @@ -195,7 +195,7 @@ Input [2]: [d_date_sk#16, d_date#17] (16) Filter [codegen id : 5] Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 1998-08-04)) AND (d_date#17 <= 1998-09-03)) AND isnotnull(d_date_sk#16)) +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 10442)) AND (d_date#17 <= 10472)) AND isnotnull(d_date_sk#16)) (17) Project [codegen id : 5] Output [1]: [d_date_sk#16] @@ -380,224 +380,224 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#61] +Output [1]: [d_date_sk#16] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#61] +Right keys [1]: [d_date_sk#16] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#61] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#16] (60) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] (62) Filter [codegen id : 16] -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Condition : isnotnull(cp_catalog_page_sk#61) (63) BroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#62] +Right keys [1]: [cp_catalog_page_sk#61] Join condition: None (65) Project [codegen id : 19] -Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#62, cp_catalog_page_id#63] +Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] (66) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#65] +Output [1]: [i_item_sk#22] (67) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#65] +Right keys [1]: [i_item_sk#22] Join condition: None (68) Project [codegen id : 19] -Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] -Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, i_item_sk#65] +Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, i_item_sk#22] (69) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#66] +Output [1]: [p_promo_sk#25] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#66] +Right keys [1]: [p_promo_sk#25] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] -Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, p_promo_sk#66] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, p_promo_sk#25] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] -Keys [1]: [cp_catalog_page_id#63] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Keys [1]: [cp_catalog_page_id#62] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Results [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] (73) Exchange -Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] -Arguments: hashpartitioning(cp_catalog_page_id#63, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] -Keys [1]: [cp_catalog_page_id#63] +Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Keys [1]: [cp_catalog_page_id#62] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] -Results [5]: [catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#63) AS id#82, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#83, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#84, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#85] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] +Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_page_id#62) AS id#79, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#80, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#81, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#82] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) (78) Exchange -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] (84) Exchange -Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] +Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] -Right keys [2]: [wr_item_sk#94, wr_order_number#95] +Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] +Right keys [2]: [wr_item_sk#91, wr_order_number#92] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] -Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] +Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#100] +Output [1]: [d_date_sk#16] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#92] -Right keys [1]: [d_date_sk#100] +Left keys [1]: [ws_sold_date_sk#89] +Right keys [1]: [d_date_sk#16] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] -Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#100] +Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] +Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#16] (91) Scan parquet default.web_site -Output [2]: [web_site_sk#101, web_site_id#102] +Output [2]: [web_site_sk#97, web_site_id#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 26] -Input [2]: [web_site_sk#101, web_site_id#102] +Input [2]: [web_site_sk#97, web_site_id#98] (93) Filter [codegen id : 26] -Input [2]: [web_site_sk#101, web_site_id#102] -Condition : isnotnull(web_site_sk#101) +Input [2]: [web_site_sk#97, web_site_id#98] +Condition : isnotnull(web_site_sk#97) (94) BroadcastExchange -Input [2]: [web_site_sk#101, web_site_id#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] +Input [2]: [web_site_sk#97, web_site_id#98] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#87] -Right keys [1]: [web_site_sk#101] +Left keys [1]: [ws_web_site_sk#84] +Right keys [1]: [web_site_sk#97] Join condition: None (96) Project [codegen id : 29] -Output [7]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] -Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#101, web_site_id#102] +Output [7]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] (97) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#104] +Output [1]: [i_item_sk#22] (98) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#86] -Right keys [1]: [i_item_sk#104] +Left keys [1]: [ws_item_sk#83] +Right keys [1]: [i_item_sk#22] Join condition: None (99) Project [codegen id : 29] -Output [6]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] -Input [8]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, i_item_sk#104] +Output [6]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Input [8]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, i_item_sk#22] (100) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#105] +Output [1]: [p_promo_sk#25] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#88] -Right keys [1]: [p_promo_sk#105] +Left keys [1]: [ws_promo_sk#85] +Right keys [1]: [p_promo_sk#25] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] -Input [7]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, p_promo_sk#105] +Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Input [7]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, p_promo_sk#25] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] -Keys [1]: [web_site_id#102] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Results [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Keys [1]: [web_site_id#98] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] (104) Exchange -Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Arguments: hashpartitioning(web_site_id#102, 5), ENSURE_REQUIREMENTS, [id=#116] +Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Keys [1]: [web_site_id#102] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] -Results [5]: [web channel AS channel#120, concat(web_site, web_site_id#102) AS id#121, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#122, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#123, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#124] +Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Keys [1]: [web_site_id#98] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] +Results [5]: [web channel AS channel#114, concat(web_site, web_site_id#98) AS id#115, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#116, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#117, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#118] (106) Union @@ -605,99 +605,99 @@ Results [5]: [web channel AS channel#120, concat(web_site, web_site_id#102) AS i Input [5]: [channel#42, id#43, sales#44, returns#45, profit#46] Keys [2]: [channel#42, id#43] Functions [3]: [partial_sum(sales#44), partial_sum(returns#45), partial_sum(profit#46)] -Aggregate Attributes [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Results [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] +Results [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] (108) Exchange -Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] -Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#131] (109) HashAggregate [codegen id : 32] -Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#138, sum(returns#45)#139, sum(profit#46)#140] -Results [5]: [channel#42, id#43, cast(sum(sales#44)#138 as decimal(37,2)) AS sales#141, cast(sum(returns#45)#139 as decimal(38,2)) AS returns#142, cast(sum(profit#46)#140 as decimal(38,2)) AS profit#143] +Aggregate Attributes [3]: [sum(sales#44)#132, sum(returns#45)#133, sum(profit#46)#134] +Results [5]: [channel#42, id#43, cast(sum(sales#44)#132 as decimal(37,2)) AS sales#135, cast(sum(returns#45)#133 as decimal(38,2)) AS returns#136, cast(sum(profit#46)#134 as decimal(38,2)) AS profit#137] (110) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] +Output [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] (111) HashAggregate [codegen id : 64] -Input [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] +Input [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#150, sum(returns#45)#151, sum(profit#46)#152] -Results [4]: [channel#42, sum(sales#44)#150 AS sales#153, sum(returns#45)#151 AS returns#154, sum(profit#46)#152 AS profit#155] +Aggregate Attributes [3]: [sum(sales#44)#144, sum(returns#45)#145, sum(profit#46)#146] +Results [4]: [channel#42, sum(sales#44)#144 AS sales#147, sum(returns#45)#145 AS returns#148, sum(profit#46)#146 AS profit#149] (112) HashAggregate [codegen id : 64] -Input [4]: [channel#42, sales#153, returns#154, profit#155] +Input [4]: [channel#42, sales#147, returns#148, profit#149] Keys [1]: [channel#42] -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#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] +Aggregate Attributes [6]: [sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] +Results [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] (113) Exchange -Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] -Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#168] +Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#162] (114) HashAggregate [codegen id : 65] -Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] Keys [1]: [channel#42] -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#42, null AS id#172, sum(sales#153)#169 AS sales#173, sum(returns#154)#170 AS returns#174, sum(profit#155)#171 AS profit#175] +Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] +Aggregate Attributes [3]: [sum(sales#147)#163, sum(returns#148)#164, sum(profit#149)#165] +Results [5]: [channel#42, null AS id#166, sum(sales#147)#163 AS sales#167, sum(returns#148)#164 AS returns#168, sum(profit#149)#165 AS profit#169] (115) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Output [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] (116) HashAggregate [codegen id : 97] -Input [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Input [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#182, sum(returns#45)#183, sum(profit#46)#184] -Results [3]: [sum(sales#44)#182 AS sales#153, sum(returns#45)#183 AS returns#154, sum(profit#46)#184 AS profit#155] +Aggregate Attributes [3]: [sum(sales#44)#176, sum(returns#45)#177, sum(profit#46)#178] +Results [3]: [sum(sales#44)#176 AS sales#147, sum(returns#45)#177 AS returns#148, sum(profit#46)#178 AS profit#149] (117) HashAggregate [codegen id : 97] -Input [3]: [sales#153, returns#154, profit#155] +Input [3]: [sales#147, returns#148, profit#149] Keys: [] -Functions [3]: [partial_sum(sales#153), partial_sum(returns#154), partial_sum(profit#155)] -Aggregate Attributes [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] -Results [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] +Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] +Aggregate Attributes [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] +Results [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] (118) Exchange -Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#197] +Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#191] (119) HashAggregate [codegen id : 98] -Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] +Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] Keys: [] -Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] -Aggregate Attributes [3]: [sum(sales#153)#198, sum(returns#154)#199, sum(profit#155)#200] -Results [5]: [null AS channel#201, null AS id#202, sum(sales#153)#198 AS sales#203, sum(returns#154)#199 AS returns#204, sum(profit#155)#200 AS profit#205] +Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] +Aggregate Attributes [3]: [sum(sales#147)#192, sum(returns#148)#193, sum(profit#149)#194] +Results [5]: [null AS channel#195, null AS id#196, sum(sales#147)#192 AS sales#197, sum(returns#148)#193 AS returns#198, sum(profit#149)#194 AS profit#199] (120) Union (121) HashAggregate [codegen id : 99] -Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] -Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] (122) Exchange -Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] -Arguments: hashpartitioning(channel#42, id#43, sales#141, returns#142, profit#143, 5), ENSURE_REQUIREMENTS, [id=#206] +Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Arguments: hashpartitioning(channel#42, id#43, sales#135, returns#136, profit#137, 5), ENSURE_REQUIREMENTS, [id=#200] (123) HashAggregate [codegen id : 100] -Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] -Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] (124) TakeOrderedAndProject -Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] -Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#141, returns#142, profit#143] +Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#135, returns#136, profit#137] ===== Subqueries ===== @@ -710,6 +710,6 @@ Output [1]: [d_date_sk#16] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 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 64f6270e81183..3333d5d1aed52 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 @@ -131,60 +131,60 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#15] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#15,17,2) as decimal(27,2)) AS total_sum#16, i_category#10, i_class#9, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] (20) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#20, i_class#21, sum#22] +Output [3]: [i_category#10, i_class#9, sum#20] (21) HashAggregate [codegen id : 8] -Input [3]: [i_category#20, i_class#21, sum#22] -Keys [2]: [i_category#20, i_class#21] -Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#23))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#23))#24,17,2) AS total_sum#25, i_category#20] +Input [3]: [i_category#10, i_class#9, sum#20] +Keys [2]: [i_category#10, i_class#9] +Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#21] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#21,17,2) AS total_sum#22, i_category#10] (22) HashAggregate [codegen id : 8] -Input [2]: [total_sum#25, i_category#20] -Keys [1]: [i_category#20] -Functions [1]: [partial_sum(total_sum#25)] -Aggregate Attributes [2]: [sum#26, isEmpty#27] -Results [3]: [i_category#20, sum#28, isEmpty#29] +Input [2]: [total_sum#22, i_category#10] +Keys [1]: [i_category#10] +Functions [1]: [partial_sum(total_sum#22)] +Aggregate Attributes [2]: [sum#23, isEmpty#24] +Results [3]: [i_category#10, sum#25, isEmpty#26] (23) Exchange -Input [3]: [i_category#20, sum#28, isEmpty#29] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [i_category#10, sum#25, isEmpty#26] +Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, [id=#27] (24) HashAggregate [codegen id : 9] -Input [3]: [i_category#20, sum#28, isEmpty#29] -Keys [1]: [i_category#20] -Functions [1]: [sum(total_sum#25)] -Aggregate Attributes [1]: [sum(total_sum#25)#31] -Results [6]: [sum(total_sum#25)#31 AS total_sum#32, i_category#20, null AS i_class#33, 0 AS g_category#34, 1 AS g_class#35, 1 AS lochierarchy#36] +Input [3]: [i_category#10, sum#25, isEmpty#26] +Keys [1]: [i_category#10] +Functions [1]: [sum(total_sum#22)] +Aggregate Attributes [1]: [sum(total_sum#22)#28] +Results [6]: [sum(total_sum#22)#28 AS total_sum#29, i_category#10, null AS i_class#30, 0 AS g_category#31, 1 AS g_class#32, 1 AS lochierarchy#33] (25) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#37, i_class#38, sum#39] +Output [3]: [i_category#10, i_class#9, sum#34] (26) HashAggregate [codegen id : 13] -Input [3]: [i_category#37, i_class#38, sum#39] -Keys [2]: [i_category#37, i_class#38] -Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#40))#41] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#40))#41,17,2) AS total_sum#25] +Input [3]: [i_category#10, i_class#9, sum#34] +Keys [2]: [i_category#10, i_class#9] +Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#35] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#35,17,2) AS total_sum#22] (27) HashAggregate [codegen id : 13] -Input [1]: [total_sum#25] +Input [1]: [total_sum#22] Keys: [] -Functions [1]: [partial_sum(total_sum#25)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [2]: [sum#44, isEmpty#45] +Functions [1]: [partial_sum(total_sum#22)] +Aggregate Attributes [2]: [sum#36, isEmpty#37] +Results [2]: [sum#38, isEmpty#39] (28) Exchange -Input [2]: [sum#44, isEmpty#45] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#46] +Input [2]: [sum#38, isEmpty#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] (29) HashAggregate [codegen id : 14] -Input [2]: [sum#44, isEmpty#45] +Input [2]: [sum#38, isEmpty#39] Keys: [] -Functions [1]: [sum(total_sum#25)] -Aggregate Attributes [1]: [sum(total_sum#25)#47] -Results [6]: [sum(total_sum#25)#47 AS total_sum#48, null AS i_category#49, null AS i_class#50, 1 AS g_category#51, 1 AS g_class#52, 2 AS lochierarchy#53] +Functions [1]: [sum(total_sum#22)] +Aggregate Attributes [1]: [sum(total_sum#22)#41] +Results [6]: [sum(total_sum#22)#41 AS total_sum#42, null AS i_category#43, null AS i_class#44, 1 AS g_category#45, 1 AS g_class#46, 2 AS lochierarchy#47] (30) Union @@ -197,34 +197,34 @@ Results [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, (32) Exchange Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] -Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#54] +Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#48] (33) HashAggregate [codegen id : 16] Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Keys [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#55] +Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#49] (34) Exchange -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] -Arguments: hashpartitioning(lochierarchy#19, _w0#55, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] +Arguments: hashpartitioning(lochierarchy#19, _w0#49, 5), ENSURE_REQUIREMENTS, [id=#50] (35) Sort [codegen id : 17] -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] -Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] +Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#49 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 (36) Window -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] -Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#55, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#57], [lochierarchy#19, _w0#55], [total_sum#16 DESC NULLS LAST] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] +Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#49, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#51], [lochierarchy#19, _w0#49], [total_sum#16 DESC NULLS LAST] (37) Project [codegen id : 18] -Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] -Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55, rank_within_parent#57] +Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] +Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49, rank_within_parent#51] (38) TakeOrderedAndProject -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#57 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] +Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#51 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] ===== Subqueries ===== 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 64f6270e81183..3333d5d1aed52 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 @@ -131,60 +131,60 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#15] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#15,17,2) as decimal(27,2)) AS total_sum#16, i_category#10, i_class#9, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] (20) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#20, i_class#21, sum#22] +Output [3]: [i_category#10, i_class#9, sum#20] (21) HashAggregate [codegen id : 8] -Input [3]: [i_category#20, i_class#21, sum#22] -Keys [2]: [i_category#20, i_class#21] -Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#23))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#23))#24,17,2) AS total_sum#25, i_category#20] +Input [3]: [i_category#10, i_class#9, sum#20] +Keys [2]: [i_category#10, i_class#9] +Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#21] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#21,17,2) AS total_sum#22, i_category#10] (22) HashAggregate [codegen id : 8] -Input [2]: [total_sum#25, i_category#20] -Keys [1]: [i_category#20] -Functions [1]: [partial_sum(total_sum#25)] -Aggregate Attributes [2]: [sum#26, isEmpty#27] -Results [3]: [i_category#20, sum#28, isEmpty#29] +Input [2]: [total_sum#22, i_category#10] +Keys [1]: [i_category#10] +Functions [1]: [partial_sum(total_sum#22)] +Aggregate Attributes [2]: [sum#23, isEmpty#24] +Results [3]: [i_category#10, sum#25, isEmpty#26] (23) Exchange -Input [3]: [i_category#20, sum#28, isEmpty#29] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [i_category#10, sum#25, isEmpty#26] +Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, [id=#27] (24) HashAggregate [codegen id : 9] -Input [3]: [i_category#20, sum#28, isEmpty#29] -Keys [1]: [i_category#20] -Functions [1]: [sum(total_sum#25)] -Aggregate Attributes [1]: [sum(total_sum#25)#31] -Results [6]: [sum(total_sum#25)#31 AS total_sum#32, i_category#20, null AS i_class#33, 0 AS g_category#34, 1 AS g_class#35, 1 AS lochierarchy#36] +Input [3]: [i_category#10, sum#25, isEmpty#26] +Keys [1]: [i_category#10] +Functions [1]: [sum(total_sum#22)] +Aggregate Attributes [1]: [sum(total_sum#22)#28] +Results [6]: [sum(total_sum#22)#28 AS total_sum#29, i_category#10, null AS i_class#30, 0 AS g_category#31, 1 AS g_class#32, 1 AS lochierarchy#33] (25) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#37, i_class#38, sum#39] +Output [3]: [i_category#10, i_class#9, sum#34] (26) HashAggregate [codegen id : 13] -Input [3]: [i_category#37, i_class#38, sum#39] -Keys [2]: [i_category#37, i_class#38] -Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#40))#41] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#40))#41,17,2) AS total_sum#25] +Input [3]: [i_category#10, i_class#9, sum#34] +Keys [2]: [i_category#10, i_class#9] +Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#35] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#35,17,2) AS total_sum#22] (27) HashAggregate [codegen id : 13] -Input [1]: [total_sum#25] +Input [1]: [total_sum#22] Keys: [] -Functions [1]: [partial_sum(total_sum#25)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [2]: [sum#44, isEmpty#45] +Functions [1]: [partial_sum(total_sum#22)] +Aggregate Attributes [2]: [sum#36, isEmpty#37] +Results [2]: [sum#38, isEmpty#39] (28) Exchange -Input [2]: [sum#44, isEmpty#45] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#46] +Input [2]: [sum#38, isEmpty#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] (29) HashAggregate [codegen id : 14] -Input [2]: [sum#44, isEmpty#45] +Input [2]: [sum#38, isEmpty#39] Keys: [] -Functions [1]: [sum(total_sum#25)] -Aggregate Attributes [1]: [sum(total_sum#25)#47] -Results [6]: [sum(total_sum#25)#47 AS total_sum#48, null AS i_category#49, null AS i_class#50, 1 AS g_category#51, 1 AS g_class#52, 2 AS lochierarchy#53] +Functions [1]: [sum(total_sum#22)] +Aggregate Attributes [1]: [sum(total_sum#22)#41] +Results [6]: [sum(total_sum#22)#41 AS total_sum#42, null AS i_category#43, null AS i_class#44, 1 AS g_category#45, 1 AS g_class#46, 2 AS lochierarchy#47] (30) Union @@ -197,34 +197,34 @@ Results [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, (32) Exchange Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] -Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#54] +Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#48] (33) HashAggregate [codegen id : 16] Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Keys [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#55] +Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#49] (34) Exchange -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] -Arguments: hashpartitioning(lochierarchy#19, _w0#55, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] +Arguments: hashpartitioning(lochierarchy#19, _w0#49, 5), ENSURE_REQUIREMENTS, [id=#50] (35) Sort [codegen id : 17] -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] -Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] +Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#49 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 (36) Window -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] -Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#55, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#57], [lochierarchy#19, _w0#55], [total_sum#16 DESC NULLS LAST] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] +Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#49, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#51], [lochierarchy#19, _w0#49], [total_sum#16 DESC NULLS LAST] (37) Project [codegen id : 18] -Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] -Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55, rank_within_parent#57] +Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] +Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49, rank_within_parent#51] (38) TakeOrderedAndProject -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#57 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] +Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#51 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt index 89bb3e5d551a8..b036ff059dbb2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt @@ -56,7 +56,7 @@ Input [2]: [d_date_sk#5, d_date#6] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 1999-02-22)) AND (d_date#6 <= 1999-03-24)) AND isnotnull(d_date_sk#5)) +Condition : (((isnotnull(d_date#6) AND (d_date#6 >= 10644)) AND (d_date#6 <= 10674)) AND isnotnull(d_date_sk#5)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#5] @@ -87,7 +87,7 @@ Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 Output [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (14) ColumnarToRow [codegen id : 4] @@ -95,7 +95,7 @@ Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_cla (15) Filter [codegen id : 4] Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] -Condition : (i_category#14 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#9)) +Condition : (i_category#14 IN (Sports,Books,Home) AND isnotnull(i_item_sk#9)) (16) Exchange Input [6]: [i_item_sk#9, i_item_id#10, i_item_desc#11, i_current_price#12, i_class#13, i_category#14] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt index b7c5f4081a1f3..920a4c2a55a26 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt @@ -45,7 +45,7 @@ Condition : isnotnull(ss_item_sk#1) Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -53,7 +53,7 @@ Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class# (6) Filter [codegen id : 1] Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) +Condition : (i_category#10 IN (Sports,Books,Home) AND isnotnull(i_item_sk#5)) (7) BroadcastExchange Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] @@ -80,7 +80,7 @@ Input [2]: [d_date_sk#12, d_date#13] (12) Filter [codegen id : 2] Input [2]: [d_date_sk#12, d_date#13] -Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 1999-02-22)) AND (d_date#13 <= 1999-03-24)) AND isnotnull(d_date_sk#12)) +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) (13) Project [codegen id : 2] Output [1]: [d_date_sk#12] From 89cd341f364b4b7543458620555b40ffce0f4f88 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 23 Mar 2021 17:24:51 +0800 Subject: [PATCH 22/38] Revert "fix regression of reuse exchange" This reverts commit 3e33cf387afa956d000c1af5ac03ea1892569d67. --- .../analysis/DeduplicateRelations.scala | 44 +- .../approved-plans-v1_4/q32.sf100/explain.txt | 42 +- .../q32.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q32/explain.txt | 86 ++-- .../approved-plans-v1_4/q32/simplified.txt | 2 +- .../approved-plans-v1_4/q33.sf100/explain.txt | 271 +++++++----- .../q33.sf100/simplified.txt | 20 +- .../approved-plans-v1_4/q33/explain.txt | 271 +++++++----- .../approved-plans-v1_4/q33/simplified.txt | 20 +- .../approved-plans-v1_4/q41.sf100/explain.txt | 48 +-- .../q41.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q41/explain.txt | 48 +-- .../approved-plans-v1_4/q41/simplified.txt | 2 +- .../approved-plans-v1_4/q56.sf100/explain.txt | 273 +++++++----- .../q56.sf100/simplified.txt | 20 +- .../approved-plans-v1_4/q56/explain.txt | 273 +++++++----- .../approved-plans-v1_4/q56/simplified.txt | 20 +- .../approved-plans-v1_4/q6.sf100/explain.txt | 186 ++++---- .../q6.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q6/explain.txt | 100 ++--- .../approved-plans-v1_4/q6/simplified.txt | 2 +- .../approved-plans-v1_4/q60.sf100/explain.txt | 273 +++++++----- .../q60.sf100/simplified.txt | 20 +- .../approved-plans-v1_4/q60/explain.txt | 273 +++++++----- .../approved-plans-v1_4/q60/simplified.txt | 20 +- .../approved-plans-v1_4/q83.sf100/explain.txt | 402 ++++++++++-------- .../q83.sf100/simplified.txt | 25 +- .../approved-plans-v1_4/q83/explain.txt | 366 +++++++++------- .../approved-plans-v1_4/q83/simplified.txt | 25 +- .../approved-plans-v1_4/q92.sf100/explain.txt | 64 +-- .../q92.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q92/explain.txt | 100 ++--- .../approved-plans-v1_4/q92/simplified.txt | 2 +- .../approved-plans-v2_7/q6.sf100/explain.txt | 186 ++++---- .../q6.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q6/explain.txt | 100 ++--- .../approved-plans-v2_7/q6/simplified.txt | 2 +- 37 files changed, 2061 insertions(+), 1535 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index 306c63010d49c..d30d29bf513fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.analysis -import scala.collection.mutable.ArrayBuffer - import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, AttributeSet, NamedExpression, PlanExpression} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule @@ -79,39 +77,23 @@ object DeduplicateRelations extends Rule[LogicalPlan] { (m, Seq(m)) } - case plan: LogicalPlan => + case _ if plan.children.nonEmpty => + val newChildren = ArrayBuffer.empty[LogicalPlan] val relations = ArrayBuffer.empty[MultiInstanceRelation] - val newPlan = if (plan.children.nonEmpty) { - val newChildren = ArrayBuffer.empty[LogicalPlan] - for (c <- plan.children) { - val (renewed, collected) = renewDuplicatedRelations(existingRelations ++ relations, c) - newChildren += renewed - relations ++= collected - } - - if (plan.childrenResolved) { - val attrMap = AttributeMap( - plan - .children - .flatMap(_.output).zip(newChildren.flatMap(_.output)) - .filter { case (a1, a2) => a1.exprId != a2.exprId } - ) - plan.withNewChildren(newChildren).rewriteAttrs(attrMap) - } else { - plan.withNewChildren(newChildren) - } - } else { - plan + for (c <- plan.children) { + val (renewed, collected) = renewDuplicatedRelations(existingRelations ++ relations, c) + newChildren += renewed + relations ++= collected } - val planWithNewSubquery = newPlan.transformExpressions { - case subquery: PlanExpression[LogicalPlan @unchecked] => - val (renewed, collected) = renewDuplicatedRelations( - existingRelations ++ relations, subquery.plan) - relations ++= collected - subquery.withNewPlan(renewed) + if (plan.childrenResolved) { + val attrMap = AttributeMap(plan.children.flatMap(_.output).zip( + newChildren.flatMap(_.output)).filter { case (a1, a2) => a1.exprId != a2.exprId }) + val newPlan = plan.withNewChildren(newChildren).rewriteAttrs(attrMap) + (newPlan, relations) + } else { + (plan.withNewChildren(newChildren), relations) } - (planWithNewSubquery, relations) case _ => (plan, Nil) } diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt index 5c886e9f2295d..b70c36db4bc9d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt @@ -116,63 +116,63 @@ Input [3]: [cs_item_sk#4, sum#13, count#14] Keys [1]: [cs_item_sk#4] Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#5))] Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#5))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4 AS cs_item_sk#4#18] (19) Filter -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4#18] Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) (20) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#4] +Right keys [1]: [cs_item_sk#4#18] Join condition: None (21) Project [codegen id : 4] Output [2]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17] -Input [3]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4] +Input [3]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4#18] (22) BroadcastExchange Input [2]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (23) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] +Output [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#6), dynamicpruningexpression(cs_sold_date_sk#6 IN dynamicpruning#7)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] ReadSchema: struct (24) ColumnarToRow -Input [3]: [cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] +Input [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] (25) Filter -Input [3]: [cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] -Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_ext_discount_amt#20)) +Input [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] +Condition : (isnotnull(cs_item_sk#4) AND isnotnull(cs_ext_discount_amt#5)) (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#19] -Join condition: (cast(cs_ext_discount_amt#20 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) +Right keys [1]: [cs_item_sk#4] +Join condition: (cast(cs_ext_discount_amt#5 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) (27) Project [codegen id : 6] -Output [1]: [cs_sold_date_sk#21] -Input [5]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] +Output [1]: [cs_sold_date_sk#6] +Input [5]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] (28) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#8] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [cs_sold_date_sk#6] +Right keys [1]: [d_date_sk#8] Join condition: None (30) Project [codegen id : 6] -Output [1]: [1 AS excess discount amount #23] -Input [2]: [cs_sold_date_sk#21, d_date_sk#22] +Output [1]: [1 AS excess discount amount #20] +Input [2]: [cs_sold_date_sk#6, d_date_sk#8] (31) CollectLimit -Input [1]: [excess discount amount #23] +Input [1]: [excess discount amount #20] Arguments: 100 ===== Subqueries ===== @@ -184,6 +184,6 @@ ReusedExchange (32) (32) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#6 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt index 303bdf58604bf..d885ad3178181 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt @@ -18,7 +18,7 @@ CollectLimit InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] Filter [(1.3 * avg(cs_ext_discount_amt))] - HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] + HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,sum,count] InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index 91cdd05907a75..9537689459170 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -79,100 +79,100 @@ Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] (11) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] +Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] +Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] (13) Filter [codegen id : 3] -Input [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] -Condition : isnotnull(cs_item_sk#8) +Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] +Condition : isnotnull(cs_item_sk#1) (14) Scan parquet default.date_dim -Output [2]: [d_date_sk#11, d_date#12] +Output [2]: [d_date_sk#8, d_date#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#11, d_date#12] +Input [2]: [d_date_sk#8, d_date#9] (16) Filter [codegen id : 2] -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10983)) AND (d_date#12 <= 11073)) AND isnotnull(d_date_sk#11)) +Input [2]: [d_date_sk#8, d_date#9] +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) (17) Project [codegen id : 2] -Output [1]: [d_date_sk#11] -Input [2]: [d_date_sk#11, d_date#12] +Output [1]: [d_date_sk#8] +Input [2]: [d_date_sk#8, d_date#9] (18) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] +Input [1]: [d_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] (19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#10] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [cs_sold_date_sk#3] +Right keys [1]: [d_date_sk#8] Join condition: None (20) Project [codegen id : 3] -Output [2]: [cs_item_sk#8, cs_ext_discount_amt#9] -Input [4]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10, d_date_sk#11] +Output [2]: [cs_item_sk#1, cs_ext_discount_amt#2] +Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#8] (21) HashAggregate [codegen id : 3] -Input [2]: [cs_item_sk#8, cs_ext_discount_amt#9] -Keys [1]: [cs_item_sk#8] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#9))] -Aggregate Attributes [2]: [sum#14, count#15] -Results [3]: [cs_item_sk#8, sum#16, count#17] +Input [2]: [cs_item_sk#1, cs_ext_discount_amt#2] +Keys [1]: [cs_item_sk#1] +Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#2))] +Aggregate Attributes [2]: [sum#11, count#12] +Results [3]: [cs_item_sk#1, sum#13, count#14] (22) Exchange -Input [3]: [cs_item_sk#8, sum#16, count#17] -Arguments: hashpartitioning(cs_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [cs_item_sk#1, sum#13, count#14] +Arguments: hashpartitioning(cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#15] (23) HashAggregate [codegen id : 4] -Input [3]: [cs_item_sk#8, sum#16, count#17] -Keys [1]: [cs_item_sk#8] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#9))] -Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#9))#19] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#9))#19 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] +Input [3]: [cs_item_sk#1, sum#13, count#14] +Keys [1]: [cs_item_sk#1] +Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#2))] +Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#2))#16] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#2))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1 AS cs_item_sk#1#18] (24) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#20) +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] +Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) (25) BroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#21] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#19] (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] -Right keys [1]: [cs_item_sk#8] -Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#20) +Right keys [1]: [cs_item_sk#1#18] +Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) (27) Project [codegen id : 6] Output [1]: [cs_sold_date_sk#3] -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] +Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] (28) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#8] (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#8] Join condition: None (30) Project [codegen id : 6] -Output [1]: [1 AS excess discount amount #23] -Input [2]: [cs_sold_date_sk#3, d_date_sk#22] +Output [1]: [1 AS excess discount amount #20] +Input [2]: [cs_sold_date_sk#3, d_date_sk#8] (31) CollectLimit -Input [1]: [excess discount amount #23] +Input [1]: [excess discount amount #20] Arguments: 100 ===== Subqueries ===== @@ -182,8 +182,8 @@ ReusedExchange (32) (32) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index f3dd6d0954046..563281eef37bc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -24,7 +24,7 @@ CollectLimit BroadcastExchange #3 WholeStageCodegen (4) Filter [(1.3 * avg(cs_ext_discount_amt))] - HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] + HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,sum,count] InputAdapter Exchange [cs_item_sk] #4 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index eeda8611876d6..613e56b58c17b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- Union (63) +TakeOrderedAndProject (76) ++- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- Union (72) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (67) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) + :- * HashAggregate (56) + : +- Exchange (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,22 +50,31 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- ReusedExchange (42) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) + : +- BroadcastExchange (51) + : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : :- * Filter (44) + : : +- * ColumnarToRow (43) + : : +- Scan parquet default.item (42) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- * Filter (47) + : +- * ColumnarToRow (46) + : +- Scan parquet default.item (45) + +- * HashAggregate (71) + +- Exchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Filter (59) + : : : +- * ColumnarToRow (58) + : : : +- Scan parquet default.web_sales (57) + : : +- ReusedExchange (60) + : +- ReusedExchange (63) + +- ReusedExchange (66) (1) Scan parquet default.store_sales @@ -160,30 +169,30 @@ Input [2]: [i_item_sk#13, i_manufact_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_category#15, i_manufact_id#16] +Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#16] +Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#16] +Input [2]: [i_category#15, i_manufact_id#14] Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] -Output [1]: [i_manufact_id#16] -Input [2]: [i_category#15, i_manufact_id#16] +Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] +Input [2]: [i_category#15, i_manufact_id#14] (25) BroadcastExchange -Input [1]: [i_manufact_id#16] +Input [1]: [i_manufact_id#14#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_manufact_id#14] -Right keys [1]: [i_manufact_id#16] +Right keys [1]: [i_manufact_id#14#16] Join condition: None (27) BroadcastExchange @@ -256,140 +265,182 @@ Join condition: None Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) ReusedExchange [Reuses operator id: 27] +(42) Scan parquet default.item Output [2]: [i_item_sk#30, i_manufact_id#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct + +(43) ColumnarToRow [codegen id : 10] +Input [2]: [i_item_sk#30, i_manufact_id#31] + +(44) Filter [codegen id : 10] +Input [2]: [i_item_sk#30, i_manufact_id#31] +Condition : isnotnull(i_item_sk#30) + +(45) Scan parquet default.item +Output [2]: [i_category#15, i_manufact_id#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] +ReadSchema: struct + +(46) ColumnarToRow [codegen id : 9] +Input [2]: [i_category#15, i_manufact_id#14] + +(47) Filter [codegen id : 9] +Input [2]: [i_category#15, i_manufact_id#14] +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) + +(48) Project [codegen id : 9] +Output [1]: [i_manufact_id#14] +Input [2]: [i_category#15, i_manufact_id#14] + +(49) BroadcastExchange +Input [1]: [i_manufact_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] + +(50) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [i_manufact_id#31] +Right keys [1]: [i_manufact_id#14] +Join condition: None + +(51) BroadcastExchange +Input [2]: [i_item_sk#30, i_manufact_id#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] -(43) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] Right keys [1]: [i_item_sk#30] Join condition: None -(44) Project [codegen id : 11] +(53) Project [codegen id : 11] Output [2]: [cs_ext_sales_price#26, i_manufact_id#31] Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_manufact_id#31] -(45) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 11] Input [2]: [cs_ext_sales_price#26, i_manufact_id#31] Keys [1]: [i_manufact_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_manufact_id#31, sum#33] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_manufact_id#31, sum#35] -(46) Exchange -Input [2]: [i_manufact_id#31, sum#33] -Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] +(55) Exchange +Input [2]: [i_manufact_id#31, sum#35] +Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] -(47) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#31, sum#33] +(56) HashAggregate [codegen id : 12] +Input [2]: [i_manufact_id#31, sum#35] Keys [1]: [i_manufact_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] -Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] +Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] -(48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +(57) Scan parquet default.web_sales +Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +(58) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -(50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) +(59) Filter [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) -(51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#41] +(60) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#43] -(52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#41] +(61) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#42] +Right keys [1]: [d_date_sk#43] Join condition: None -(53) Project [codegen id : 17] -Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] -Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] +(62) Project [codegen id : 17] +Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] +Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#42] +(63) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#44] -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#42] +(64) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#40] +Right keys [1]: [ca_address_sk#44] Join condition: None -(56) Project [codegen id : 17] -Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] +(65) Project [codegen id : 17] +Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] -(57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#43, i_manufact_id#44] +(66) ReusedExchange [Reuses operator id: 51] +Output [2]: [i_item_sk#45, i_manufact_id#46] -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#43] +(67) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#45] Join condition: None -(59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#39, i_manufact_id#44] -Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_manufact_id#44] +(68) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#41, i_manufact_id#46] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_manufact_id#46] -(60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#39, i_manufact_id#44] -Keys [1]: [i_manufact_id#44] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum#45] -Results [2]: [i_manufact_id#44, sum#46] +(69) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#41, i_manufact_id#46] +Keys [1]: [i_manufact_id#46] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum#47] +Results [2]: [i_manufact_id#46, sum#48] -(61) Exchange -Input [2]: [i_manufact_id#44, sum#46] -Arguments: hashpartitioning(i_manufact_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] +(70) Exchange +Input [2]: [i_manufact_id#46, sum#48] +Arguments: hashpartitioning(i_manufact_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] -(62) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#44, sum#46] -Keys [1]: [i_manufact_id#44] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] -Results [2]: [i_manufact_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] +(71) HashAggregate [codegen id : 18] +Input [2]: [i_manufact_id#46, sum#48] +Keys [1]: [i_manufact_id#46] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] +Results [2]: [i_manufact_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] -(63) Union +(72) Union -(64) HashAggregate [codegen id : 19] +(73) HashAggregate [codegen id : 19] Input [2]: [i_manufact_id#14, total_sales#23] Keys [1]: [i_manufact_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [3]: [i_manufact_id#14, sum#52, isEmpty#53] +Aggregate Attributes [2]: [sum#52, isEmpty#53] +Results [3]: [i_manufact_id#14, sum#54, isEmpty#55] -(65) Exchange -Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] +(74) Exchange +Input [3]: [i_manufact_id#14, sum#54, isEmpty#55] +Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] -(66) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] +(75) HashAggregate [codegen id : 20] +Input [3]: [i_manufact_id#14, sum#54, isEmpty#55] Keys [1]: [i_manufact_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#55] -Results [2]: [i_manufact_id#14, sum(total_sales#23)#55 AS total_sales#56] +Aggregate Attributes [1]: [sum(total_sales#23)#57] +Results [2]: [i_manufact_id#14, sum(total_sales#23)#57 AS total_sales#58] -(67) TakeOrderedAndProject -Input [2]: [i_manufact_id#14, total_sales#56] -Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_manufact_id#14, total_sales#56] +(76) TakeOrderedAndProject +Input [2]: [i_manufact_id#14, total_sales#58] +Arguments: 100, [total_sales#58 ASC NULLS FIRST], [i_manufact_id#14, total_sales#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (68) +ReusedExchange (77) -(68) ReusedExchange [Reuses operator id: 8] +(77) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt index d1a115fa528d5..ad5473e9da920 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt @@ -79,11 +79,25 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + BroadcastExchange #8 + WholeStageCodegen (10) + BroadcastHashJoin [i_manufact_id,i_manufact_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_manufact_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_category,i_manufact_id] WholeStageCodegen (18) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_manufact_id] #8 + Exchange [i_manufact_id] #10 WholeStageCodegen (17) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_manufact_id] @@ -102,4 +116,4 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + ReusedExchange [i_item_sk,i_manufact_id] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index eeda8611876d6..613e56b58c17b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- Union (63) +TakeOrderedAndProject (76) ++- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- Union (72) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (67) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) + :- * HashAggregate (56) + : +- Exchange (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,22 +50,31 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- ReusedExchange (42) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) + : +- BroadcastExchange (51) + : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : :- * Filter (44) + : : +- * ColumnarToRow (43) + : : +- Scan parquet default.item (42) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- * Filter (47) + : +- * ColumnarToRow (46) + : +- Scan parquet default.item (45) + +- * HashAggregate (71) + +- Exchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Filter (59) + : : : +- * ColumnarToRow (58) + : : : +- Scan parquet default.web_sales (57) + : : +- ReusedExchange (60) + : +- ReusedExchange (63) + +- ReusedExchange (66) (1) Scan parquet default.store_sales @@ -160,30 +169,30 @@ Input [2]: [i_item_sk#13, i_manufact_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_category#15, i_manufact_id#16] +Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#16] +Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#16] +Input [2]: [i_category#15, i_manufact_id#14] Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] -Output [1]: [i_manufact_id#16] -Input [2]: [i_category#15, i_manufact_id#16] +Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] +Input [2]: [i_category#15, i_manufact_id#14] (25) BroadcastExchange -Input [1]: [i_manufact_id#16] +Input [1]: [i_manufact_id#14#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_manufact_id#14] -Right keys [1]: [i_manufact_id#16] +Right keys [1]: [i_manufact_id#14#16] Join condition: None (27) BroadcastExchange @@ -256,140 +265,182 @@ Join condition: None Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) ReusedExchange [Reuses operator id: 27] +(42) Scan parquet default.item Output [2]: [i_item_sk#30, i_manufact_id#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct + +(43) ColumnarToRow [codegen id : 10] +Input [2]: [i_item_sk#30, i_manufact_id#31] + +(44) Filter [codegen id : 10] +Input [2]: [i_item_sk#30, i_manufact_id#31] +Condition : isnotnull(i_item_sk#30) + +(45) Scan parquet default.item +Output [2]: [i_category#15, i_manufact_id#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] +ReadSchema: struct + +(46) ColumnarToRow [codegen id : 9] +Input [2]: [i_category#15, i_manufact_id#14] + +(47) Filter [codegen id : 9] +Input [2]: [i_category#15, i_manufact_id#14] +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) + +(48) Project [codegen id : 9] +Output [1]: [i_manufact_id#14] +Input [2]: [i_category#15, i_manufact_id#14] + +(49) BroadcastExchange +Input [1]: [i_manufact_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] + +(50) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [i_manufact_id#31] +Right keys [1]: [i_manufact_id#14] +Join condition: None + +(51) BroadcastExchange +Input [2]: [i_item_sk#30, i_manufact_id#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] -(43) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] Right keys [1]: [i_item_sk#30] Join condition: None -(44) Project [codegen id : 11] +(53) Project [codegen id : 11] Output [2]: [cs_ext_sales_price#26, i_manufact_id#31] Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_manufact_id#31] -(45) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 11] Input [2]: [cs_ext_sales_price#26, i_manufact_id#31] Keys [1]: [i_manufact_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_manufact_id#31, sum#33] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_manufact_id#31, sum#35] -(46) Exchange -Input [2]: [i_manufact_id#31, sum#33] -Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] +(55) Exchange +Input [2]: [i_manufact_id#31, sum#35] +Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] -(47) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#31, sum#33] +(56) HashAggregate [codegen id : 12] +Input [2]: [i_manufact_id#31, sum#35] Keys [1]: [i_manufact_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] -Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] +Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] -(48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +(57) Scan parquet default.web_sales +Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +(58) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -(50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) +(59) Filter [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) -(51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#41] +(60) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#43] -(52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#41] +(61) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#42] +Right keys [1]: [d_date_sk#43] Join condition: None -(53) Project [codegen id : 17] -Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] -Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] +(62) Project [codegen id : 17] +Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] +Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#42] +(63) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#44] -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#42] +(64) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#40] +Right keys [1]: [ca_address_sk#44] Join condition: None -(56) Project [codegen id : 17] -Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] +(65) Project [codegen id : 17] +Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] -(57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#43, i_manufact_id#44] +(66) ReusedExchange [Reuses operator id: 51] +Output [2]: [i_item_sk#45, i_manufact_id#46] -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#43] +(67) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#45] Join condition: None -(59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#39, i_manufact_id#44] -Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_manufact_id#44] +(68) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#41, i_manufact_id#46] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_manufact_id#46] -(60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#39, i_manufact_id#44] -Keys [1]: [i_manufact_id#44] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum#45] -Results [2]: [i_manufact_id#44, sum#46] +(69) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#41, i_manufact_id#46] +Keys [1]: [i_manufact_id#46] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum#47] +Results [2]: [i_manufact_id#46, sum#48] -(61) Exchange -Input [2]: [i_manufact_id#44, sum#46] -Arguments: hashpartitioning(i_manufact_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] +(70) Exchange +Input [2]: [i_manufact_id#46, sum#48] +Arguments: hashpartitioning(i_manufact_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] -(62) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#44, sum#46] -Keys [1]: [i_manufact_id#44] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] -Results [2]: [i_manufact_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] +(71) HashAggregate [codegen id : 18] +Input [2]: [i_manufact_id#46, sum#48] +Keys [1]: [i_manufact_id#46] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] +Results [2]: [i_manufact_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] -(63) Union +(72) Union -(64) HashAggregate [codegen id : 19] +(73) HashAggregate [codegen id : 19] Input [2]: [i_manufact_id#14, total_sales#23] Keys [1]: [i_manufact_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [3]: [i_manufact_id#14, sum#52, isEmpty#53] +Aggregate Attributes [2]: [sum#52, isEmpty#53] +Results [3]: [i_manufact_id#14, sum#54, isEmpty#55] -(65) Exchange -Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] +(74) Exchange +Input [3]: [i_manufact_id#14, sum#54, isEmpty#55] +Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] -(66) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] +(75) HashAggregate [codegen id : 20] +Input [3]: [i_manufact_id#14, sum#54, isEmpty#55] Keys [1]: [i_manufact_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#55] -Results [2]: [i_manufact_id#14, sum(total_sales#23)#55 AS total_sales#56] +Aggregate Attributes [1]: [sum(total_sales#23)#57] +Results [2]: [i_manufact_id#14, sum(total_sales#23)#57 AS total_sales#58] -(67) TakeOrderedAndProject -Input [2]: [i_manufact_id#14, total_sales#56] -Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_manufact_id#14, total_sales#56] +(76) TakeOrderedAndProject +Input [2]: [i_manufact_id#14, total_sales#58] +Arguments: 100, [total_sales#58 ASC NULLS FIRST], [i_manufact_id#14, total_sales#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (68) +ReusedExchange (77) -(68) ReusedExchange [Reuses operator id: 8] +(77) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index d1a115fa528d5..ad5473e9da920 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -79,11 +79,25 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + BroadcastExchange #8 + WholeStageCodegen (10) + BroadcastHashJoin [i_manufact_id,i_manufact_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_manufact_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_category,i_manufact_id] WholeStageCodegen (18) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_manufact_id] #8 + Exchange [i_manufact_id] #10 WholeStageCodegen (17) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_manufact_id] @@ -102,4 +116,4 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + ReusedExchange [i_item_sk,i_manufact_id] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt index 034c0fe9eb535..13d73e61e1443 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt @@ -40,61 +40,61 @@ Output [2]: [i_manufact#2, i_product_name#3] Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] (5) Scan parquet default.item -Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Output [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [Or(Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,powder),EqualTo(i_color,khaki)),Or(EqualTo(i_units,Ounce),EqualTo(i_units,Oz))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,brown),EqualTo(i_color,honeydew)),Or(EqualTo(i_units,Bunch),EqualTo(i_units,Ton))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,floral),EqualTo(i_color,deep)),Or(EqualTo(i_units,N/A),EqualTo(i_units,Dozen))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,light),EqualTo(i_color,cornflower)),Or(EqualTo(i_units,Box),EqualTo(i_units,Pound))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large)))))),Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,midnight),EqualTo(i_color,snow)),Or(EqualTo(i_units,Pallet),EqualTo(i_units,Gross))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,cyan),EqualTo(i_color,papaya)),Or(EqualTo(i_units,Cup),EqualTo(i_units,Dram))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,orange),EqualTo(i_color,frosted)),Or(EqualTo(i_units,Each),EqualTo(i_units,Tbl))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,forest),EqualTo(i_color,ghost)),Or(EqualTo(i_units,Lb),EqualTo(i_units,Bundle))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))))))), IsNotNull(i_manufact)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] (7) Filter [codegen id : 1] -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Condition : (((((i_category#4 = Women) AND (((((i_color#7 = powder) OR (i_color#7 = khaki)) AND ((i_units#8 = Ounce) OR (i_units#8 = Oz))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = brown) OR (i_color#7 = honeydew)) AND ((i_units#8 = Bunch) OR (i_units#8 = Ton))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = floral) OR (i_color#7 = deep)) AND ((i_units#8 = N/A) OR (i_units#8 = Dozen))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = light) OR (i_color#7 = cornflower)) AND ((i_units#8 = Box) OR (i_units#8 = Pound))) AND ((i_size#6 = medium) OR (i_size#6 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#7 = midnight) OR (i_color#7 = snow)) AND ((i_units#8 = Pallet) OR (i_units#8 = Gross))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = cyan) OR (i_color#7 = papaya)) AND ((i_units#8 = Cup) OR (i_units#8 = Dram))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = orange) OR (i_color#7 = frosted)) AND ((i_units#8 = Each) OR (i_units#8 = Tbl))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = forest) OR (i_color#7 = ghost)) AND ((i_units#8 = Lb) OR (i_units#8 = Bundle))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))))))) AND isnotnull(i_manufact#5)) +Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Condition : (((((i_category#4 = Women) AND (((((i_color#6 = powder) OR (i_color#6 = khaki)) AND ((i_units#7 = Ounce) OR (i_units#7 = Oz))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = brown) OR (i_color#6 = honeydew)) AND ((i_units#7 = Bunch) OR (i_units#7 = Ton))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = floral) OR (i_color#6 = deep)) AND ((i_units#7 = N/A) OR (i_units#7 = Dozen))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = light) OR (i_color#6 = cornflower)) AND ((i_units#7 = Box) OR (i_units#7 = Pound))) AND ((i_size#5 = medium) OR (i_size#5 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#6 = midnight) OR (i_color#6 = snow)) AND ((i_units#7 = Pallet) OR (i_units#7 = Gross))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = cyan) OR (i_color#6 = papaya)) AND ((i_units#7 = Cup) OR (i_units#7 = Dram))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = orange) OR (i_color#6 = frosted)) AND ((i_units#7 = Each) OR (i_units#7 = Tbl))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = forest) OR (i_color#6 = ghost)) AND ((i_units#7 = Lb) OR (i_units#7 = Bundle))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))))))) AND isnotnull(i_manufact#2)) (8) Project [codegen id : 1] -Output [1]: [i_manufact#5] -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Output [1]: [i_manufact#2] +Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] (9) HashAggregate [codegen id : 1] -Input [1]: [i_manufact#5] -Keys [1]: [i_manufact#5] +Input [1]: [i_manufact#2] +Keys [1]: [i_manufact#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#9] -Results [2]: [i_manufact#5, count#10] +Aggregate Attributes [1]: [count#8] +Results [2]: [i_manufact#2, count#9] (10) Exchange -Input [2]: [i_manufact#5, count#10] -Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [id=#11] +Input [2]: [i_manufact#2, count#9] +Arguments: hashpartitioning(i_manufact#2, 5), true, [id=#10] (11) HashAggregate [codegen id : 2] -Input [2]: [i_manufact#5, count#10] -Keys [1]: [i_manufact#5] +Input [2]: [i_manufact#2, count#9] +Keys [1]: [i_manufact#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#12] -Results [2]: [count(1)#12 AS item_cnt#13, i_manufact#5] +Aggregate Attributes [1]: [count(1)#11] +Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13] (12) Filter [codegen id : 2] -Input [2]: [item_cnt#13, i_manufact#5] -Condition : (item_cnt#13 > 0) +Input [2]: [item_cnt#12, i_manufact#2#13] +Condition : (item_cnt#12 > 0) (13) Project [codegen id : 2] -Output [1]: [i_manufact#5] -Input [2]: [item_cnt#13, i_manufact#5] +Output [1]: [i_manufact#2#13] +Input [2]: [item_cnt#12, i_manufact#2#13] (14) BroadcastExchange -Input [1]: [i_manufact#5] +Input [1]: [i_manufact#2#13] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14] (15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_manufact#2] -Right keys [1]: [i_manufact#5] +Right keys [1]: [i_manufact#2#13] Join condition: None (16) Project [codegen id : 3] Output [1]: [i_product_name#3] -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] +Input [3]: [i_manufact#2, i_product_name#3, i_manufact#2#13] (17) HashAggregate [codegen id : 3] Input [1]: [i_product_name#3] @@ -105,7 +105,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [id=#15] +Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt index d36800823bb3f..2d14d75ca9362 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (2) Project [i_manufact] Filter [item_cnt] - HashAggregate [i_manufact,count] [count(1),item_cnt,count] + HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,count] InputAdapter Exchange [i_manufact] #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index 034c0fe9eb535..13d73e61e1443 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -40,61 +40,61 @@ Output [2]: [i_manufact#2, i_product_name#3] Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] (5) Scan parquet default.item -Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Output [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [Or(Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,powder),EqualTo(i_color,khaki)),Or(EqualTo(i_units,Ounce),EqualTo(i_units,Oz))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,brown),EqualTo(i_color,honeydew)),Or(EqualTo(i_units,Bunch),EqualTo(i_units,Ton))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,floral),EqualTo(i_color,deep)),Or(EqualTo(i_units,N/A),EqualTo(i_units,Dozen))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,light),EqualTo(i_color,cornflower)),Or(EqualTo(i_units,Box),EqualTo(i_units,Pound))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large)))))),Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,midnight),EqualTo(i_color,snow)),Or(EqualTo(i_units,Pallet),EqualTo(i_units,Gross))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,cyan),EqualTo(i_color,papaya)),Or(EqualTo(i_units,Cup),EqualTo(i_units,Dram))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,orange),EqualTo(i_color,frosted)),Or(EqualTo(i_units,Each),EqualTo(i_units,Tbl))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,forest),EqualTo(i_color,ghost)),Or(EqualTo(i_units,Lb),EqualTo(i_units,Bundle))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))))))), IsNotNull(i_manufact)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] (7) Filter [codegen id : 1] -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Condition : (((((i_category#4 = Women) AND (((((i_color#7 = powder) OR (i_color#7 = khaki)) AND ((i_units#8 = Ounce) OR (i_units#8 = Oz))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = brown) OR (i_color#7 = honeydew)) AND ((i_units#8 = Bunch) OR (i_units#8 = Ton))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = floral) OR (i_color#7 = deep)) AND ((i_units#8 = N/A) OR (i_units#8 = Dozen))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = light) OR (i_color#7 = cornflower)) AND ((i_units#8 = Box) OR (i_units#8 = Pound))) AND ((i_size#6 = medium) OR (i_size#6 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#7 = midnight) OR (i_color#7 = snow)) AND ((i_units#8 = Pallet) OR (i_units#8 = Gross))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = cyan) OR (i_color#7 = papaya)) AND ((i_units#8 = Cup) OR (i_units#8 = Dram))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = orange) OR (i_color#7 = frosted)) AND ((i_units#8 = Each) OR (i_units#8 = Tbl))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = forest) OR (i_color#7 = ghost)) AND ((i_units#8 = Lb) OR (i_units#8 = Bundle))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))))))) AND isnotnull(i_manufact#5)) +Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Condition : (((((i_category#4 = Women) AND (((((i_color#6 = powder) OR (i_color#6 = khaki)) AND ((i_units#7 = Ounce) OR (i_units#7 = Oz))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = brown) OR (i_color#6 = honeydew)) AND ((i_units#7 = Bunch) OR (i_units#7 = Ton))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = floral) OR (i_color#6 = deep)) AND ((i_units#7 = N/A) OR (i_units#7 = Dozen))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = light) OR (i_color#6 = cornflower)) AND ((i_units#7 = Box) OR (i_units#7 = Pound))) AND ((i_size#5 = medium) OR (i_size#5 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#6 = midnight) OR (i_color#6 = snow)) AND ((i_units#7 = Pallet) OR (i_units#7 = Gross))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = cyan) OR (i_color#6 = papaya)) AND ((i_units#7 = Cup) OR (i_units#7 = Dram))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = orange) OR (i_color#6 = frosted)) AND ((i_units#7 = Each) OR (i_units#7 = Tbl))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = forest) OR (i_color#6 = ghost)) AND ((i_units#7 = Lb) OR (i_units#7 = Bundle))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))))))) AND isnotnull(i_manufact#2)) (8) Project [codegen id : 1] -Output [1]: [i_manufact#5] -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Output [1]: [i_manufact#2] +Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] (9) HashAggregate [codegen id : 1] -Input [1]: [i_manufact#5] -Keys [1]: [i_manufact#5] +Input [1]: [i_manufact#2] +Keys [1]: [i_manufact#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#9] -Results [2]: [i_manufact#5, count#10] +Aggregate Attributes [1]: [count#8] +Results [2]: [i_manufact#2, count#9] (10) Exchange -Input [2]: [i_manufact#5, count#10] -Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [id=#11] +Input [2]: [i_manufact#2, count#9] +Arguments: hashpartitioning(i_manufact#2, 5), true, [id=#10] (11) HashAggregate [codegen id : 2] -Input [2]: [i_manufact#5, count#10] -Keys [1]: [i_manufact#5] +Input [2]: [i_manufact#2, count#9] +Keys [1]: [i_manufact#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#12] -Results [2]: [count(1)#12 AS item_cnt#13, i_manufact#5] +Aggregate Attributes [1]: [count(1)#11] +Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13] (12) Filter [codegen id : 2] -Input [2]: [item_cnt#13, i_manufact#5] -Condition : (item_cnt#13 > 0) +Input [2]: [item_cnt#12, i_manufact#2#13] +Condition : (item_cnt#12 > 0) (13) Project [codegen id : 2] -Output [1]: [i_manufact#5] -Input [2]: [item_cnt#13, i_manufact#5] +Output [1]: [i_manufact#2#13] +Input [2]: [item_cnt#12, i_manufact#2#13] (14) BroadcastExchange -Input [1]: [i_manufact#5] +Input [1]: [i_manufact#2#13] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14] (15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_manufact#2] -Right keys [1]: [i_manufact#5] +Right keys [1]: [i_manufact#2#13] Join condition: None (16) Project [codegen id : 3] Output [1]: [i_product_name#3] -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] +Input [3]: [i_manufact#2, i_product_name#3, i_manufact#2#13] (17) HashAggregate [codegen id : 3] Input [1]: [i_product_name#3] @@ -105,7 +105,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [id=#15] +Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index d36800823bb3f..2d14d75ca9362 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (2) Project [i_manufact] Filter [item_cnt] - HashAggregate [i_manufact,count] [count(1),item_cnt,count] + HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,count] InputAdapter Exchange [i_manufact] #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt index 47eda1483a06c..3c6fe13107e04 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- Union (63) +TakeOrderedAndProject (76) ++- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- Union (72) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (67) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) + :- * HashAggregate (56) + : +- Exchange (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,22 +50,31 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- ReusedExchange (42) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) + : +- BroadcastExchange (51) + : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : :- * Filter (44) + : : +- * ColumnarToRow (43) + : : +- Scan parquet default.item (42) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- * Filter (47) + : +- * ColumnarToRow (46) + : +- Scan parquet default.item (45) + +- * HashAggregate (71) + +- Exchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Filter (59) + : : : +- * ColumnarToRow (58) + : : : +- Scan parquet default.web_sales (57) + : : +- ReusedExchange (60) + : +- ReusedExchange (63) + +- ReusedExchange (66) (1) Scan parquet default.store_sales @@ -160,30 +169,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#15, i_color#16] +Output [2]: [i_item_id#14, i_color#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#15, i_color#16] +Input [2]: [i_item_id#14, i_color#15] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#15, i_color#16] -Condition : i_color#16 IN (slate ,blanched ,burnished ) +Input [2]: [i_item_id#14, i_color#15] +Condition : i_color#15 IN (slate ,blanched ,burnished ) (24) Project [codegen id : 3] -Output [1]: [i_item_id#15] -Input [2]: [i_item_id#15, i_color#16] +Output [1]: [i_item_id#14 AS i_item_id#14#16] +Input [2]: [i_item_id#14, i_color#15] (25) BroadcastExchange -Input [1]: [i_item_id#15] +Input [1]: [i_item_id#14#16] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#15] +Right keys [1]: [i_item_id#14#16] Join condition: None (27) BroadcastExchange @@ -256,140 +265,182 @@ Join condition: None Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) ReusedExchange [Reuses operator id: 27] +(42) Scan parquet default.item Output [2]: [i_item_sk#30, i_item_id#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct + +(43) ColumnarToRow [codegen id : 10] +Input [2]: [i_item_sk#30, i_item_id#31] + +(44) Filter [codegen id : 10] +Input [2]: [i_item_sk#30, i_item_id#31] +Condition : isnotnull(i_item_sk#30) + +(45) Scan parquet default.item +Output [2]: [i_item_id#14, i_color#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] +ReadSchema: struct + +(46) ColumnarToRow [codegen id : 9] +Input [2]: [i_item_id#14, i_color#15] + +(47) Filter [codegen id : 9] +Input [2]: [i_item_id#14, i_color#15] +Condition : i_color#15 IN (slate ,blanched ,burnished ) + +(48) Project [codegen id : 9] +Output [1]: [i_item_id#14] +Input [2]: [i_item_id#14, i_color#15] + +(49) BroadcastExchange +Input [1]: [i_item_id#14] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] + +(50) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [i_item_id#31] +Right keys [1]: [i_item_id#14] +Join condition: None + +(51) BroadcastExchange +Input [2]: [i_item_sk#30, i_item_id#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] -(43) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] Right keys [1]: [i_item_sk#30] Join condition: None -(44) Project [codegen id : 11] +(53) Project [codegen id : 11] Output [2]: [cs_ext_sales_price#26, i_item_id#31] Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] -(45) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 11] Input [2]: [cs_ext_sales_price#26, i_item_id#31] Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#31, sum#33] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#31, sum#35] -(46) Exchange -Input [2]: [i_item_id#31, sum#33] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] +(55) Exchange +Input [2]: [i_item_id#31, sum#35] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] -(47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#31, sum#33] +(56) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#31, sum#35] Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] -Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] -(48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +(57) Scan parquet default.web_sales +Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +(58) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -(50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) +(59) Filter [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) -(51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#41] +(60) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#43] -(52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#41] +(61) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#42] +Right keys [1]: [d_date_sk#43] Join condition: None -(53) Project [codegen id : 17] -Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] -Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] +(62) Project [codegen id : 17] +Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] +Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#42] +(63) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#44] -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#42] +(64) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#40] +Right keys [1]: [ca_address_sk#44] Join condition: None -(56) Project [codegen id : 17] -Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] +(65) Project [codegen id : 17] +Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] -(57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#43, i_item_id#44] +(66) ReusedExchange [Reuses operator id: 51] +Output [2]: [i_item_sk#45, i_item_id#46] -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#43] +(67) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#45] Join condition: None -(59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#39, i_item_id#44] -Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] +(68) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#41, i_item_id#46] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_item_id#46] -(60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#39, i_item_id#44] -Keys [1]: [i_item_id#44] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum#45] -Results [2]: [i_item_id#44, sum#46] +(69) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#41, i_item_id#46] +Keys [1]: [i_item_id#46] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum#47] +Results [2]: [i_item_id#46, sum#48] -(61) Exchange -Input [2]: [i_item_id#44, sum#46] -Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] +(70) Exchange +Input [2]: [i_item_id#46, sum#48] +Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] -(62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#44, sum#46] -Keys [1]: [i_item_id#44] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] -Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] +(71) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#46, sum#48] +Keys [1]: [i_item_id#46] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] +Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] -(63) Union +(72) Union -(64) HashAggregate [codegen id : 19] +(73) HashAggregate [codegen id : 19] Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [3]: [i_item_id#14, sum#52, isEmpty#53] +Aggregate Attributes [2]: [sum#52, isEmpty#53] +Results [3]: [i_item_id#14, sum#54, isEmpty#55] -(65) Exchange -Input [3]: [i_item_id#14, sum#52, isEmpty#53] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] +(74) Exchange +Input [3]: [i_item_id#14, sum#54, isEmpty#55] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] -(66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#52, isEmpty#53] +(75) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#14, sum#54, isEmpty#55] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#55] -Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] +Aggregate Attributes [1]: [sum(total_sales#23)#57] +Results [2]: [i_item_id#14, sum(total_sales#23)#57 AS total_sales#58] -(67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#56] -Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] +(76) TakeOrderedAndProject +Input [2]: [i_item_id#14, total_sales#58] +Arguments: 100, [total_sales#58 ASC NULLS FIRST], [i_item_id#14, total_sales#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (68) +ReusedExchange (77) -(68) ReusedExchange [Reuses operator id: 8] +(77) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt index 2d21010c26b8d..34d320db2bb91 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt @@ -79,11 +79,25 @@ TakeOrderedAndProject [total_sales,i_item_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + BroadcastExchange #8 + WholeStageCodegen (10) + BroadcastHashJoin [i_item_id,i_item_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_item_id] + Filter [i_color] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_id,i_color] WholeStageCodegen (18) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #8 + Exchange [i_item_id] #10 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -102,4 +116,4 @@ TakeOrderedAndProject [total_sales,i_item_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [i_item_sk,i_item_id] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 47eda1483a06c..3c6fe13107e04 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- Union (63) +TakeOrderedAndProject (76) ++- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- Union (72) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (67) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) + :- * HashAggregate (56) + : +- Exchange (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,22 +50,31 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- ReusedExchange (42) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) + : +- BroadcastExchange (51) + : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : :- * Filter (44) + : : +- * ColumnarToRow (43) + : : +- Scan parquet default.item (42) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- * Filter (47) + : +- * ColumnarToRow (46) + : +- Scan parquet default.item (45) + +- * HashAggregate (71) + +- Exchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Filter (59) + : : : +- * ColumnarToRow (58) + : : : +- Scan parquet default.web_sales (57) + : : +- ReusedExchange (60) + : +- ReusedExchange (63) + +- ReusedExchange (66) (1) Scan parquet default.store_sales @@ -160,30 +169,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#15, i_color#16] +Output [2]: [i_item_id#14, i_color#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#15, i_color#16] +Input [2]: [i_item_id#14, i_color#15] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#15, i_color#16] -Condition : i_color#16 IN (slate ,blanched ,burnished ) +Input [2]: [i_item_id#14, i_color#15] +Condition : i_color#15 IN (slate ,blanched ,burnished ) (24) Project [codegen id : 3] -Output [1]: [i_item_id#15] -Input [2]: [i_item_id#15, i_color#16] +Output [1]: [i_item_id#14 AS i_item_id#14#16] +Input [2]: [i_item_id#14, i_color#15] (25) BroadcastExchange -Input [1]: [i_item_id#15] +Input [1]: [i_item_id#14#16] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#15] +Right keys [1]: [i_item_id#14#16] Join condition: None (27) BroadcastExchange @@ -256,140 +265,182 @@ Join condition: None Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) ReusedExchange [Reuses operator id: 27] +(42) Scan parquet default.item Output [2]: [i_item_sk#30, i_item_id#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct + +(43) ColumnarToRow [codegen id : 10] +Input [2]: [i_item_sk#30, i_item_id#31] + +(44) Filter [codegen id : 10] +Input [2]: [i_item_sk#30, i_item_id#31] +Condition : isnotnull(i_item_sk#30) + +(45) Scan parquet default.item +Output [2]: [i_item_id#14, i_color#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] +ReadSchema: struct + +(46) ColumnarToRow [codegen id : 9] +Input [2]: [i_item_id#14, i_color#15] + +(47) Filter [codegen id : 9] +Input [2]: [i_item_id#14, i_color#15] +Condition : i_color#15 IN (slate ,blanched ,burnished ) + +(48) Project [codegen id : 9] +Output [1]: [i_item_id#14] +Input [2]: [i_item_id#14, i_color#15] + +(49) BroadcastExchange +Input [1]: [i_item_id#14] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] + +(50) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [i_item_id#31] +Right keys [1]: [i_item_id#14] +Join condition: None + +(51) BroadcastExchange +Input [2]: [i_item_sk#30, i_item_id#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] -(43) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] Right keys [1]: [i_item_sk#30] Join condition: None -(44) Project [codegen id : 11] +(53) Project [codegen id : 11] Output [2]: [cs_ext_sales_price#26, i_item_id#31] Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] -(45) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 11] Input [2]: [cs_ext_sales_price#26, i_item_id#31] Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#31, sum#33] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#31, sum#35] -(46) Exchange -Input [2]: [i_item_id#31, sum#33] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] +(55) Exchange +Input [2]: [i_item_id#31, sum#35] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] -(47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#31, sum#33] +(56) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#31, sum#35] Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] -Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] -(48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +(57) Scan parquet default.web_sales +Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +(58) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -(50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) +(59) Filter [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) -(51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#41] +(60) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#43] -(52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#41] +(61) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#42] +Right keys [1]: [d_date_sk#43] Join condition: None -(53) Project [codegen id : 17] -Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] -Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] +(62) Project [codegen id : 17] +Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] +Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#42] +(63) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#44] -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#42] +(64) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#40] +Right keys [1]: [ca_address_sk#44] Join condition: None -(56) Project [codegen id : 17] -Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] +(65) Project [codegen id : 17] +Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] -(57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#43, i_item_id#44] +(66) ReusedExchange [Reuses operator id: 51] +Output [2]: [i_item_sk#45, i_item_id#46] -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#43] +(67) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#45] Join condition: None -(59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#39, i_item_id#44] -Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] +(68) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#41, i_item_id#46] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_item_id#46] -(60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#39, i_item_id#44] -Keys [1]: [i_item_id#44] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum#45] -Results [2]: [i_item_id#44, sum#46] +(69) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#41, i_item_id#46] +Keys [1]: [i_item_id#46] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum#47] +Results [2]: [i_item_id#46, sum#48] -(61) Exchange -Input [2]: [i_item_id#44, sum#46] -Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] +(70) Exchange +Input [2]: [i_item_id#46, sum#48] +Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] -(62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#44, sum#46] -Keys [1]: [i_item_id#44] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] -Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] +(71) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#46, sum#48] +Keys [1]: [i_item_id#46] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] +Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] -(63) Union +(72) Union -(64) HashAggregate [codegen id : 19] +(73) HashAggregate [codegen id : 19] Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [3]: [i_item_id#14, sum#52, isEmpty#53] +Aggregate Attributes [2]: [sum#52, isEmpty#53] +Results [3]: [i_item_id#14, sum#54, isEmpty#55] -(65) Exchange -Input [3]: [i_item_id#14, sum#52, isEmpty#53] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] +(74) Exchange +Input [3]: [i_item_id#14, sum#54, isEmpty#55] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] -(66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#52, isEmpty#53] +(75) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#14, sum#54, isEmpty#55] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#55] -Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] +Aggregate Attributes [1]: [sum(total_sales#23)#57] +Results [2]: [i_item_id#14, sum(total_sales#23)#57 AS total_sales#58] -(67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#56] -Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] +(76) TakeOrderedAndProject +Input [2]: [i_item_id#14, total_sales#58] +Arguments: 100, [total_sales#58 ASC NULLS FIRST], [i_item_id#14, total_sales#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (68) +ReusedExchange (77) -(68) ReusedExchange [Reuses operator id: 8] +(77) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 2d21010c26b8d..34d320db2bb91 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -79,11 +79,25 @@ TakeOrderedAndProject [total_sales,i_item_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + BroadcastExchange #8 + WholeStageCodegen (10) + BroadcastHashJoin [i_item_id,i_item_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_item_id] + Filter [i_color] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_id,i_color] WholeStageCodegen (18) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #8 + Exchange [i_item_id] #10 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -102,4 +116,4 @@ TakeOrderedAndProject [total_sales,i_item_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [i_item_sk,i_item_id] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt index 7e82b4d5df296..4c5b5f25108e0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt @@ -81,57 +81,57 @@ Input [3]: [i_item_sk#5, i_current_price#6, i_category#7] Condition : (isnotnull(i_current_price#6) AND isnotnull(i_item_sk#5)) (7) Scan parquet default.item -Output [2]: [i_current_price#8, i_category#9] +Output [2]: [i_current_price#6, i_category#7] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [i_current_price#8, i_category#9] +Input [2]: [i_current_price#6, i_category#7] (9) Filter [codegen id : 1] -Input [2]: [i_current_price#8, i_category#9] -Condition : isnotnull(i_category#9) +Input [2]: [i_current_price#6, i_category#7] +Condition : isnotnull(i_category#7) (10) HashAggregate [codegen id : 1] -Input [2]: [i_current_price#8, i_category#9] -Keys [1]: [i_category#9] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#8))] -Aggregate Attributes [2]: [sum#10, count#11] -Results [3]: [i_category#9, sum#12, count#13] +Input [2]: [i_current_price#6, i_category#7] +Keys [1]: [i_category#7] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#6))] +Aggregate Attributes [2]: [sum#8, count#9] +Results [3]: [i_category#7, sum#10, count#11] (11) Exchange -Input [3]: [i_category#9, sum#12, count#13] -Arguments: hashpartitioning(i_category#9, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [3]: [i_category#7, sum#10, count#11] +Arguments: hashpartitioning(i_category#7, 5), ENSURE_REQUIREMENTS, [id=#12] (12) HashAggregate [codegen id : 2] -Input [3]: [i_category#9, sum#12, count#13] -Keys [1]: [i_category#9] -Functions [1]: [avg(UnscaledValue(i_current_price#8))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#8))#15] -Results [2]: [cast((avg(UnscaledValue(i_current_price#8))#15 / 100.0) as decimal(11,6)) AS avg(i_current_price)#16, i_category#9] +Input [3]: [i_category#7, sum#10, count#11] +Keys [1]: [i_category#7] +Functions [1]: [avg(UnscaledValue(i_current_price#6))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#6))#13] +Results [2]: [cast((avg(UnscaledValue(i_current_price#6))#13 / 100.0) as decimal(11,6)) AS avg(i_current_price)#14, i_category#7 AS i_category#7#15] (13) BroadcastExchange -Input [2]: [avg(i_current_price)#16, i_category#9] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#17] +Input [2]: [avg(i_current_price)#14, i_category#7#15] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#16] (14) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_category#7] -Right keys [1]: [i_category#9] +Right keys [1]: [i_category#7#15] Join condition: None (15) Filter [codegen id : 3] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] -Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#16)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] +Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#14)), DecimalType(14,7), true)) (16) Project [codegen id : 3] Output [1]: [i_item_sk#5] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] (17) BroadcastExchange Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (18) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#1] @@ -143,143 +143,143 @@ Output [2]: [ss_customer_sk#2, ss_sold_date_sk#3] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, i_item_sk#5] (20) Scan parquet default.date_dim -Output [2]: [d_date_sk#19, d_month_seq#20] +Output [2]: [d_date_sk#18, d_month_seq#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#19, d_month_seq#20] +Input [2]: [d_date_sk#18, d_month_seq#19] (22) Filter [codegen id : 4] -Input [2]: [d_date_sk#19, d_month_seq#20] -Condition : ((isnotnull(d_month_seq#20) AND (d_month_seq#20 = Subquery scalar-subquery#21, [id=#22])) AND isnotnull(d_date_sk#19)) +Input [2]: [d_date_sk#18, d_month_seq#19] +Condition : ((isnotnull(d_month_seq#19) AND (d_month_seq#19 = Subquery scalar-subquery#20, [id=#21])) AND isnotnull(d_date_sk#18)) (23) Project [codegen id : 4] -Output [1]: [d_date_sk#19] -Input [2]: [d_date_sk#19, d_month_seq#20] +Output [1]: [d_date_sk#18] +Input [2]: [d_date_sk#18, d_month_seq#19] (24) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [d_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (25) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#19] +Right keys [1]: [d_date_sk#18] Join condition: None (26) Project [codegen id : 5] Output [1]: [ss_customer_sk#2] -Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#19] +Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#18] (27) Exchange Input [1]: [ss_customer_sk#2] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#23] (28) Sort [codegen id : 6] Input [1]: [ss_customer_sk#2] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#25, ca_state#26] +Output [2]: [ca_address_sk#24, ca_state#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#25, ca_state#26] +Input [2]: [ca_address_sk#24, ca_state#25] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#25, ca_state#26] -Condition : isnotnull(ca_address_sk#25) +Input [2]: [ca_address_sk#24, ca_state#25] +Condition : isnotnull(ca_address_sk#24) (32) Exchange -Input [2]: [ca_address_sk#25, ca_state#26] -Arguments: hashpartitioning(ca_address_sk#25, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] (33) Sort [codegen id : 8] -Input [2]: [ca_address_sk#25, ca_state#26] -Arguments: [ca_address_sk#25 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 (34) Scan parquet default.customer -Output [2]: [c_customer_sk#28, c_current_addr_sk#29] +Output [2]: [c_customer_sk#27, c_current_addr_sk#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 9] -Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Input [2]: [c_customer_sk#27, c_current_addr_sk#28] (36) Filter [codegen id : 9] -Input [2]: [c_customer_sk#28, c_current_addr_sk#29] -Condition : (isnotnull(c_current_addr_sk#29) AND isnotnull(c_customer_sk#28)) +Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Condition : (isnotnull(c_current_addr_sk#28) AND isnotnull(c_customer_sk#27)) (37) Exchange -Input [2]: [c_customer_sk#28, c_current_addr_sk#29] -Arguments: hashpartitioning(c_current_addr_sk#29, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Arguments: hashpartitioning(c_current_addr_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] (38) Sort [codegen id : 10] -Input [2]: [c_customer_sk#28, c_current_addr_sk#29] -Arguments: [c_current_addr_sk#29 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Arguments: [c_current_addr_sk#28 ASC NULLS FIRST], false, 0 (39) SortMergeJoin [codegen id : 11] -Left keys [1]: [ca_address_sk#25] -Right keys [1]: [c_current_addr_sk#29] +Left keys [1]: [ca_address_sk#24] +Right keys [1]: [c_current_addr_sk#28] Join condition: None (40) Project [codegen id : 11] -Output [2]: [ca_state#26, c_customer_sk#28] -Input [4]: [ca_address_sk#25, ca_state#26, c_customer_sk#28, c_current_addr_sk#29] +Output [2]: [ca_state#25, c_customer_sk#27] +Input [4]: [ca_address_sk#24, ca_state#25, c_customer_sk#27, c_current_addr_sk#28] (41) Exchange -Input [2]: [ca_state#26, c_customer_sk#28] -Arguments: hashpartitioning(c_customer_sk#28, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [2]: [ca_state#25, c_customer_sk#27] +Arguments: hashpartitioning(c_customer_sk#27, 5), ENSURE_REQUIREMENTS, [id=#30] (42) Sort [codegen id : 12] -Input [2]: [ca_state#26, c_customer_sk#28] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 +Input [2]: [ca_state#25, c_customer_sk#27] +Arguments: [c_customer_sk#27 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 13] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#28] +Right keys [1]: [c_customer_sk#27] Join condition: None (44) Project [codegen id : 13] -Output [1]: [ca_state#26] -Input [3]: [ss_customer_sk#2, ca_state#26, c_customer_sk#28] +Output [1]: [ca_state#25] +Input [3]: [ss_customer_sk#2, ca_state#25, c_customer_sk#27] (45) HashAggregate [codegen id : 13] -Input [1]: [ca_state#26] -Keys [1]: [ca_state#26] +Input [1]: [ca_state#25] +Keys [1]: [ca_state#25] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#32] -Results [2]: [ca_state#26, count#33] +Aggregate Attributes [1]: [count#31] +Results [2]: [ca_state#25, count#32] (46) Exchange -Input [2]: [ca_state#26, count#33] -Arguments: hashpartitioning(ca_state#26, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [2]: [ca_state#25, count#32] +Arguments: hashpartitioning(ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#33] (47) HashAggregate [codegen id : 14] -Input [2]: [ca_state#26, count#33] -Keys [1]: [ca_state#26] +Input [2]: [ca_state#25, count#32] +Keys [1]: [ca_state#25] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [3]: [ca_state#26 AS state#36, count(1)#35 AS cnt#37, count(1)#35 AS count(1)#38] +Aggregate Attributes [1]: [count(1)#34] +Results [3]: [ca_state#25 AS state#35, count(1)#34 AS cnt#36, count(1)#34 AS count(1)#37] (48) Filter [codegen id : 14] -Input [3]: [state#36, cnt#37, count(1)#38] -Condition : (count(1)#38 >= 10) +Input [3]: [state#35, cnt#36, count(1)#37] +Condition : (count(1)#37 >= 10) (49) Project [codegen id : 14] -Output [2]: [state#36, cnt#37] -Input [3]: [state#36, cnt#37, count(1)#38] +Output [2]: [state#35, cnt#36] +Input [3]: [state#35, cnt#36, count(1)#37] (50) TakeOrderedAndProject -Input [2]: [state#36, cnt#37] -Arguments: 100, [cnt#37 ASC NULLS FIRST], [state#36, cnt#37] +Input [2]: [state#35, cnt#36] +Arguments: 100, [cnt#36 ASC NULLS FIRST], [state#35, cnt#36] ===== Subqueries ===== @@ -288,9 +288,9 @@ ReusedExchange (51) (51) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#19] +Output [1]: [d_date_sk#18] -Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#21, [id=#22] +Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#20, [id=#21] * HashAggregate (58) +- Exchange (57) +- * HashAggregate (56) @@ -301,39 +301,39 @@ Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquer (52) Scan parquet default.date_dim -Output [3]: [d_month_seq#39, d_year#40, d_moy#41] +Output [3]: [d_month_seq#19, d_year#38, d_moy#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (53) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#39, d_year#40, d_moy#41] +Input [3]: [d_month_seq#19, d_year#38, d_moy#39] (54) Filter [codegen id : 1] -Input [3]: [d_month_seq#39, d_year#40, d_moy#41] -Condition : (((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2000)) AND (d_moy#41 = 1)) +Input [3]: [d_month_seq#19, d_year#38, d_moy#39] +Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) (55) Project [codegen id : 1] -Output [1]: [d_month_seq#39] -Input [3]: [d_month_seq#39, d_year#40, d_moy#41] +Output [1]: [d_month_seq#19] +Input [3]: [d_month_seq#19, d_year#38, d_moy#39] (56) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#39] -Keys [1]: [d_month_seq#39] +Input [1]: [d_month_seq#19] +Keys [1]: [d_month_seq#19] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#39] +Results [1]: [d_month_seq#19] (57) Exchange -Input [1]: [d_month_seq#39] -Arguments: hashpartitioning(d_month_seq#39, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [1]: [d_month_seq#19] +Arguments: hashpartitioning(d_month_seq#19, 5), ENSURE_REQUIREMENTS, [id=#40] (58) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#39] -Keys [1]: [d_month_seq#39] +Input [1]: [d_month_seq#19] +Keys [1]: [d_month_seq#19] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#39] +Results [1]: [d_month_seq#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt index 1cdb7a5df8e6f..a5097742ec146 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt @@ -38,7 +38,7 @@ TakeOrderedAndProject [cnt,state] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] InputAdapter Exchange [i_category] #6 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt index 0f98039fc0f7f..58fabaa8e44be 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt @@ -160,57 +160,57 @@ Input [3]: [i_item_sk#16, i_current_price#17, i_category#18] Condition : (isnotnull(i_current_price#17) AND isnotnull(i_item_sk#16)) (26) Scan parquet default.item -Output [2]: [i_current_price#19, i_category#20] +Output [2]: [i_current_price#17, i_category#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (27) ColumnarToRow [codegen id : 4] -Input [2]: [i_current_price#19, i_category#20] +Input [2]: [i_current_price#17, i_category#18] (28) Filter [codegen id : 4] -Input [2]: [i_current_price#19, i_category#20] -Condition : isnotnull(i_category#20) +Input [2]: [i_current_price#17, i_category#18] +Condition : isnotnull(i_category#18) (29) HashAggregate [codegen id : 4] -Input [2]: [i_current_price#19, i_category#20] -Keys [1]: [i_category#20] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#19))] -Aggregate Attributes [2]: [sum#21, count#22] -Results [3]: [i_category#20, sum#23, count#24] +Input [2]: [i_current_price#17, i_category#18] +Keys [1]: [i_category#18] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] +Aggregate Attributes [2]: [sum#19, count#20] +Results [3]: [i_category#18, sum#21, count#22] (30) Exchange -Input [3]: [i_category#20, sum#23, count#24] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [i_category#18, sum#21, count#22] +Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [id=#23] (31) HashAggregate [codegen id : 5] -Input [3]: [i_category#20, sum#23, count#24] -Keys [1]: [i_category#20] -Functions [1]: [avg(UnscaledValue(i_current_price#19))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#19))#26] -Results [2]: [cast((avg(UnscaledValue(i_current_price#19))#26 / 100.0) as decimal(11,6)) AS avg(i_current_price)#27, i_category#20] +Input [3]: [i_category#18, sum#21, count#22] +Keys [1]: [i_category#18] +Functions [1]: [avg(UnscaledValue(i_current_price#17))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#17))#24] +Results [2]: [cast((avg(UnscaledValue(i_current_price#17))#24 / 100.0) as decimal(11,6)) AS avg(i_current_price)#25, i_category#18 AS i_category#18#26] (32) BroadcastExchange -Input [2]: [avg(i_current_price)#27, i_category#20] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#28] +Input [2]: [avg(i_current_price)#25, i_category#18#26] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#27] (33) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_category#18] -Right keys [1]: [i_category#20] +Right keys [1]: [i_category#18#26] Join condition: None (34) Filter [codegen id : 6] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] -Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#27)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] +Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#25)), DecimalType(14,7), true)) (35) Project [codegen id : 6] Output [1]: [i_item_sk#16] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] (36) BroadcastExchange Input [1]: [i_item_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (37) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#6] @@ -225,31 +225,31 @@ Input [3]: [ca_state#2, ss_item_sk#6, i_item_sk#16] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#30] -Results [2]: [ca_state#2, count#31] +Aggregate Attributes [1]: [count#29] +Results [2]: [ca_state#2, count#30] (40) Exchange -Input [2]: [ca_state#2, count#31] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [2]: [ca_state#2, count#30] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#31] (41) HashAggregate [codegen id : 8] -Input [2]: [ca_state#2, count#31] +Input [2]: [ca_state#2, count#30] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#33] -Results [3]: [ca_state#2 AS state#34, count(1)#33 AS cnt#35, count(1)#33 AS count(1)#36] +Aggregate Attributes [1]: [count(1)#32] +Results [3]: [ca_state#2 AS state#33, count(1)#32 AS cnt#34, count(1)#32 AS count(1)#35] (42) Filter [codegen id : 8] -Input [3]: [state#34, cnt#35, count(1)#36] -Condition : (count(1)#36 >= 10) +Input [3]: [state#33, cnt#34, count(1)#35] +Condition : (count(1)#35 >= 10) (43) Project [codegen id : 8] -Output [2]: [state#34, cnt#35] -Input [3]: [state#34, cnt#35, count(1)#36] +Output [2]: [state#33, cnt#34] +Input [3]: [state#33, cnt#34, count(1)#35] (44) TakeOrderedAndProject -Input [2]: [state#34, cnt#35] -Arguments: 100, [cnt#35 ASC NULLS FIRST], [state#34, cnt#35] +Input [2]: [state#33, cnt#34] +Arguments: 100, [cnt#34 ASC NULLS FIRST], [state#33, cnt#34] ===== Subqueries ===== @@ -271,39 +271,39 @@ Subquery:2 Hosting operator id = 18 Hosting Expression = Subquery scalar-subquer (46) Scan parquet default.date_dim -Output [3]: [d_month_seq#37, d_year#38, d_moy#39] +Output [3]: [d_month_seq#12, d_year#36, d_moy#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (47) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#38, d_moy#39] +Input [3]: [d_month_seq#12, d_year#36, d_moy#37] (48) Filter [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#38, d_moy#39] -Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) +Input [3]: [d_month_seq#12, d_year#36, d_moy#37] +Condition : (((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2000)) AND (d_moy#37 = 1)) (49) Project [codegen id : 1] -Output [1]: [d_month_seq#37] -Input [3]: [d_month_seq#37, d_year#38, d_moy#39] +Output [1]: [d_month_seq#12] +Input [3]: [d_month_seq#12, d_year#36, d_moy#37] (50) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#37] -Keys [1]: [d_month_seq#37] +Input [1]: [d_month_seq#12] +Keys [1]: [d_month_seq#12] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#37] +Results [1]: [d_month_seq#12] (51) Exchange -Input [1]: [d_month_seq#37] -Arguments: hashpartitioning(d_month_seq#37, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [1]: [d_month_seq#12] +Arguments: hashpartitioning(d_month_seq#12, 5), ENSURE_REQUIREMENTS, [id=#38] (52) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#37] -Keys [1]: [d_month_seq#37] +Input [1]: [d_month_seq#12] +Keys [1]: [d_month_seq#12] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#37] +Results [1]: [d_month_seq#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index d7fad5948f64b..8b60c2f86a280 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -68,7 +68,7 @@ TakeOrderedAndProject [cnt,state] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] InputAdapter Exchange [i_category] #8 WholeStageCodegen (4) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt index 49cfd232239aa..8774d6d342fb0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- Union (63) +TakeOrderedAndProject (76) ++- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- Union (72) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (67) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) + :- * HashAggregate (56) + : +- Exchange (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,22 +50,31 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- ReusedExchange (42) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) + : +- BroadcastExchange (51) + : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : :- * Filter (44) + : : +- * ColumnarToRow (43) + : : +- Scan parquet default.item (42) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- * Filter (47) + : +- * ColumnarToRow (46) + : +- Scan parquet default.item (45) + +- * HashAggregate (71) + +- Exchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Filter (59) + : : : +- * ColumnarToRow (58) + : : : +- Scan parquet default.web_sales (57) + : : +- ReusedExchange (60) + : +- ReusedExchange (63) + +- ReusedExchange (66) (1) Scan parquet default.store_sales @@ -160,30 +169,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#15, i_category#16] +Output [2]: [i_item_id#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#15, i_category#16] +Input [2]: [i_item_id#14, i_category#15] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#15, i_category#16] -Condition : (isnotnull(i_category#16) AND (i_category#16 = Music )) +Input [2]: [i_item_id#14, i_category#15] +Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) (24) Project [codegen id : 3] -Output [1]: [i_item_id#15] -Input [2]: [i_item_id#15, i_category#16] +Output [1]: [i_item_id#14 AS i_item_id#14#16] +Input [2]: [i_item_id#14, i_category#15] (25) BroadcastExchange -Input [1]: [i_item_id#15] +Input [1]: [i_item_id#14#16] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#15] +Right keys [1]: [i_item_id#14#16] Join condition: None (27) BroadcastExchange @@ -256,140 +265,182 @@ Join condition: None Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) ReusedExchange [Reuses operator id: 27] +(42) Scan parquet default.item Output [2]: [i_item_sk#30, i_item_id#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct + +(43) ColumnarToRow [codegen id : 10] +Input [2]: [i_item_sk#30, i_item_id#31] + +(44) Filter [codegen id : 10] +Input [2]: [i_item_sk#30, i_item_id#31] +Condition : isnotnull(i_item_sk#30) + +(45) Scan parquet default.item +Output [2]: [i_item_id#14, i_category#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] +ReadSchema: struct + +(46) ColumnarToRow [codegen id : 9] +Input [2]: [i_item_id#14, i_category#15] + +(47) Filter [codegen id : 9] +Input [2]: [i_item_id#14, i_category#15] +Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) + +(48) Project [codegen id : 9] +Output [1]: [i_item_id#14] +Input [2]: [i_item_id#14, i_category#15] + +(49) BroadcastExchange +Input [1]: [i_item_id#14] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] + +(50) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [i_item_id#31] +Right keys [1]: [i_item_id#14] +Join condition: None + +(51) BroadcastExchange +Input [2]: [i_item_sk#30, i_item_id#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] -(43) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] Right keys [1]: [i_item_sk#30] Join condition: None -(44) Project [codegen id : 11] +(53) Project [codegen id : 11] Output [2]: [cs_ext_sales_price#26, i_item_id#31] Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] -(45) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 11] Input [2]: [cs_ext_sales_price#26, i_item_id#31] Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#31, sum#33] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#31, sum#35] -(46) Exchange -Input [2]: [i_item_id#31, sum#33] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] +(55) Exchange +Input [2]: [i_item_id#31, sum#35] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] -(47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#31, sum#33] +(56) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#31, sum#35] Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] -Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] -(48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +(57) Scan parquet default.web_sales +Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +(58) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -(50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) +(59) Filter [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) -(51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#41] +(60) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#43] -(52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#41] +(61) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#42] +Right keys [1]: [d_date_sk#43] Join condition: None -(53) Project [codegen id : 17] -Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] -Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] +(62) Project [codegen id : 17] +Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] +Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#42] +(63) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#44] -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#42] +(64) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#40] +Right keys [1]: [ca_address_sk#44] Join condition: None -(56) Project [codegen id : 17] -Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] +(65) Project [codegen id : 17] +Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] -(57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#43, i_item_id#44] +(66) ReusedExchange [Reuses operator id: 51] +Output [2]: [i_item_sk#45, i_item_id#46] -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#43] +(67) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#45] Join condition: None -(59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#39, i_item_id#44] -Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] +(68) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#41, i_item_id#46] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_item_id#46] -(60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#39, i_item_id#44] -Keys [1]: [i_item_id#44] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum#45] -Results [2]: [i_item_id#44, sum#46] +(69) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#41, i_item_id#46] +Keys [1]: [i_item_id#46] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum#47] +Results [2]: [i_item_id#46, sum#48] -(61) Exchange -Input [2]: [i_item_id#44, sum#46] -Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] +(70) Exchange +Input [2]: [i_item_id#46, sum#48] +Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] -(62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#44, sum#46] -Keys [1]: [i_item_id#44] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] -Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] +(71) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#46, sum#48] +Keys [1]: [i_item_id#46] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] +Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] -(63) Union +(72) Union -(64) HashAggregate [codegen id : 19] +(73) HashAggregate [codegen id : 19] Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [3]: [i_item_id#14, sum#52, isEmpty#53] +Aggregate Attributes [2]: [sum#52, isEmpty#53] +Results [3]: [i_item_id#14, sum#54, isEmpty#55] -(65) Exchange -Input [3]: [i_item_id#14, sum#52, isEmpty#53] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] +(74) Exchange +Input [3]: [i_item_id#14, sum#54, isEmpty#55] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] -(66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#52, isEmpty#53] +(75) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#14, sum#54, isEmpty#55] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#55] -Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] +Aggregate Attributes [1]: [sum(total_sales#23)#57] +Results [2]: [i_item_id#14, sum(total_sales#23)#57 AS total_sales#58] -(67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#56] -Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] +(76) TakeOrderedAndProject +Input [2]: [i_item_id#14, total_sales#58] +Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#58 ASC NULLS FIRST], [i_item_id#14, total_sales#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (68) +ReusedExchange (77) -(68) ReusedExchange [Reuses operator id: 8] +(77) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt index 46775417cbb70..948e022cd11c8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt @@ -79,11 +79,25 @@ TakeOrderedAndProject [i_item_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + BroadcastExchange #8 + WholeStageCodegen (10) + BroadcastHashJoin [i_item_id,i_item_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_item_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_id,i_category] WholeStageCodegen (18) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #8 + Exchange [i_item_id] #10 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -102,4 +116,4 @@ TakeOrderedAndProject [i_item_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [i_item_sk,i_item_id] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 49cfd232239aa..8774d6d342fb0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (67) -+- * HashAggregate (66) - +- Exchange (65) - +- * HashAggregate (64) - +- Union (63) +TakeOrderedAndProject (76) ++- * HashAggregate (75) + +- Exchange (74) + +- * HashAggregate (73) + +- Union (72) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (67) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (47) - : +- Exchange (46) - : +- * HashAggregate (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) + :- * HashAggregate (56) + : +- Exchange (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,22 +50,31 @@ TakeOrderedAndProject (67) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- ReusedExchange (42) - +- * HashAggregate (62) - +- Exchange (61) - +- * HashAggregate (60) - +- * Project (59) - +- * BroadcastHashJoin Inner BuildRight (58) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * Project (53) - : : +- * BroadcastHashJoin Inner BuildRight (52) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.web_sales (48) - : : +- ReusedExchange (51) - : +- ReusedExchange (54) - +- ReusedExchange (57) + : +- BroadcastExchange (51) + : +- * BroadcastHashJoin LeftSemi BuildRight (50) + : :- * Filter (44) + : : +- * ColumnarToRow (43) + : : +- Scan parquet default.item (42) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- * Filter (47) + : +- * ColumnarToRow (46) + : +- Scan parquet default.item (45) + +- * HashAggregate (71) + +- Exchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Filter (59) + : : : +- * ColumnarToRow (58) + : : : +- Scan parquet default.web_sales (57) + : : +- ReusedExchange (60) + : +- ReusedExchange (63) + +- ReusedExchange (66) (1) Scan parquet default.store_sales @@ -160,30 +169,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#15, i_category#16] +Output [2]: [i_item_id#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#15, i_category#16] +Input [2]: [i_item_id#14, i_category#15] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#15, i_category#16] -Condition : (isnotnull(i_category#16) AND (i_category#16 = Music )) +Input [2]: [i_item_id#14, i_category#15] +Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) (24) Project [codegen id : 3] -Output [1]: [i_item_id#15] -Input [2]: [i_item_id#15, i_category#16] +Output [1]: [i_item_id#14 AS i_item_id#14#16] +Input [2]: [i_item_id#14, i_category#15] (25) BroadcastExchange -Input [1]: [i_item_id#15] +Input [1]: [i_item_id#14#16] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#15] +Right keys [1]: [i_item_id#14#16] Join condition: None (27) BroadcastExchange @@ -256,140 +265,182 @@ Join condition: None Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] -(42) ReusedExchange [Reuses operator id: 27] +(42) Scan parquet default.item Output [2]: [i_item_sk#30, i_item_id#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct + +(43) ColumnarToRow [codegen id : 10] +Input [2]: [i_item_sk#30, i_item_id#31] + +(44) Filter [codegen id : 10] +Input [2]: [i_item_sk#30, i_item_id#31] +Condition : isnotnull(i_item_sk#30) + +(45) Scan parquet default.item +Output [2]: [i_item_id#14, i_category#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] +ReadSchema: struct + +(46) ColumnarToRow [codegen id : 9] +Input [2]: [i_item_id#14, i_category#15] + +(47) Filter [codegen id : 9] +Input [2]: [i_item_id#14, i_category#15] +Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) + +(48) Project [codegen id : 9] +Output [1]: [i_item_id#14] +Input [2]: [i_item_id#14, i_category#15] + +(49) BroadcastExchange +Input [1]: [i_item_id#14] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] + +(50) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [i_item_id#31] +Right keys [1]: [i_item_id#14] +Join condition: None + +(51) BroadcastExchange +Input [2]: [i_item_sk#30, i_item_id#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] -(43) BroadcastHashJoin [codegen id : 11] +(52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] Right keys [1]: [i_item_sk#30] Join condition: None -(44) Project [codegen id : 11] +(53) Project [codegen id : 11] Output [2]: [cs_ext_sales_price#26, i_item_id#31] Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] -(45) HashAggregate [codegen id : 11] +(54) HashAggregate [codegen id : 11] Input [2]: [cs_ext_sales_price#26, i_item_id#31] Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#31, sum#33] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#31, sum#35] -(46) Exchange -Input [2]: [i_item_id#31, sum#33] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] +(55) Exchange +Input [2]: [i_item_id#31, sum#35] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] -(47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#31, sum#33] +(56) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#31, sum#35] Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] -Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] -(48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +(57) Scan parquet default.web_sales +Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +(58) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -(50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) +(59) Filter [codegen id : 17] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) -(51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#41] +(60) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#43] -(52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#41] +(61) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#42] +Right keys [1]: [d_date_sk#43] Join condition: None -(53) Project [codegen id : 17] -Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] -Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] +(62) Project [codegen id : 17] +Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] +Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#42] +(63) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#44] -(55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#42] +(64) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#40] +Right keys [1]: [ca_address_sk#44] Join condition: None -(56) Project [codegen id : 17] -Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] +(65) Project [codegen id : 17] +Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] +Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] -(57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#43, i_item_id#44] +(66) ReusedExchange [Reuses operator id: 51] +Output [2]: [i_item_sk#45, i_item_id#46] -(58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#43] +(67) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#45] Join condition: None -(59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#39, i_item_id#44] -Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] +(68) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#41, i_item_id#46] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_item_id#46] -(60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#39, i_item_id#44] -Keys [1]: [i_item_id#44] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum#45] -Results [2]: [i_item_id#44, sum#46] +(69) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#41, i_item_id#46] +Keys [1]: [i_item_id#46] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum#47] +Results [2]: [i_item_id#46, sum#48] -(61) Exchange -Input [2]: [i_item_id#44, sum#46] -Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] +(70) Exchange +Input [2]: [i_item_id#46, sum#48] +Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] -(62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#44, sum#46] -Keys [1]: [i_item_id#44] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] -Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] +(71) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#46, sum#48] +Keys [1]: [i_item_id#46] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] +Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] -(63) Union +(72) Union -(64) HashAggregate [codegen id : 19] +(73) HashAggregate [codegen id : 19] Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [3]: [i_item_id#14, sum#52, isEmpty#53] +Aggregate Attributes [2]: [sum#52, isEmpty#53] +Results [3]: [i_item_id#14, sum#54, isEmpty#55] -(65) Exchange -Input [3]: [i_item_id#14, sum#52, isEmpty#53] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] +(74) Exchange +Input [3]: [i_item_id#14, sum#54, isEmpty#55] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] -(66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#52, isEmpty#53] +(75) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#14, sum#54, isEmpty#55] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#55] -Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] +Aggregate Attributes [1]: [sum(total_sales#23)#57] +Results [2]: [i_item_id#14, sum(total_sales#23)#57 AS total_sales#58] -(67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#56] -Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] +(76) TakeOrderedAndProject +Input [2]: [i_item_id#14, total_sales#58] +Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#58 ASC NULLS FIRST], [i_item_id#14, total_sales#58] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (68) +ReusedExchange (77) -(68) ReusedExchange [Reuses operator id: 8] +(77) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index 46775417cbb70..948e022cd11c8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -79,11 +79,25 @@ TakeOrderedAndProject [i_item_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + BroadcastExchange #8 + WholeStageCodegen (10) + BroadcastHashJoin [i_item_id,i_item_id] + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_item_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_id,i_category] WholeStageCodegen (18) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #8 + Exchange [i_item_id] #10 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -102,4 +116,4 @@ TakeOrderedAndProject [i_item_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #5 + ReusedExchange [i_item_sk,i_item_id] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt index a7c71a0668676..ef2ac135f64eb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (61) -+- * Project (60) - +- * BroadcastHashJoin Inner BuildRight (59) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) +TakeOrderedAndProject (72) ++- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) : :- * HashAggregate (30) : : +- Exchange (29) : : +- * HashAggregate (28) @@ -34,32 +34,43 @@ TakeOrderedAndProject (61) : : +- * Filter (24) : : +- * ColumnarToRow (23) : : +- Scan parquet default.item (22) - : +- BroadcastExchange (43) - : +- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (36) - : : +- * BroadcastHashJoin Inner BuildRight (35) + : +- BroadcastExchange (54) + : +- * HashAggregate (53) + : +- Exchange (52) + : +- * HashAggregate (51) + : +- * Project (50) + : +- * BroadcastHashJoin Inner BuildRight (49) + : :- * Project (47) + : : +- * BroadcastHashJoin Inner BuildRight (46) : : :- * Filter (33) : : : +- * ColumnarToRow (32) : : : +- Scan parquet default.catalog_returns (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- BroadcastExchange (58) - +- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- * Project (54) - +- * BroadcastHashJoin Inner BuildRight (53) - :- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet default.web_returns (46) - : +- ReusedExchange (49) - +- ReusedExchange (52) + : : +- BroadcastExchange (45) + : : +- * Project (44) + : : +- * BroadcastHashJoin LeftSemi BuildRight (43) + : : :- * Filter (36) + : : : +- * ColumnarToRow (35) + : : : +- Scan parquet default.date_dim (34) + : : +- BroadcastExchange (42) + : : +- * Project (41) + : : +- * BroadcastHashJoin LeftSemi BuildRight (40) + : : :- * ColumnarToRow (38) + : : : +- Scan parquet default.date_dim (37) + : : +- ReusedExchange (39) + : +- ReusedExchange (48) + +- BroadcastExchange (69) + +- * HashAggregate (68) + +- Exchange (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Filter (59) + : : +- * ColumnarToRow (58) + : : +- Scan parquet default.web_returns (57) + : +- ReusedExchange (60) + +- ReusedExchange (63) (1) Scan parquet default.store_returns @@ -92,51 +103,51 @@ Input [2]: [d_date_sk#4, d_date#5] Condition : isnotnull(d_date_sk#4) (7) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (8) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#6, d_week_seq#7] +Input [2]: [d_date#5, d_week_seq#6] (9) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (10) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#5, d_week_seq#6] (11) Filter [codegen id : 1] -Input [2]: [d_date#8, d_week_seq#9] -Condition : cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) +Input [2]: [d_date#5, d_week_seq#6] +Condition : cast(d_date#5 as string) IN (2000-06-30,2000-09-27,2000-11-17) (12) Project [codegen id : 1] -Output [1]: [d_week_seq#9] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_week_seq#6 AS d_week_seq#6#7] +Input [2]: [d_date#5, d_week_seq#6] (13) BroadcastExchange -Input [1]: [d_week_seq#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] +Input [1]: [d_week_seq#6#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] (14) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_week_seq#7] -Right keys [1]: [d_week_seq#9] +Left keys [1]: [d_week_seq#6] +Right keys [1]: [d_week_seq#6#7] Join condition: None (15) Project [codegen id : 2] -Output [1]: [d_date#6] -Input [2]: [d_date#6, d_week_seq#7] +Output [1]: [d_date#5 AS d_date#5#9] +Input [2]: [d_date#5, d_week_seq#6] (16) BroadcastExchange -Input [1]: [d_date#6] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#11] +Input [1]: [d_date#5#9] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] (17) BroadcastHashJoin [codegen id : 3] Left keys [1]: [d_date#5] -Right keys [1]: [d_date#6] +Right keys [1]: [d_date#5#9] Join condition: None (18) Project [codegen id : 3] @@ -145,7 +156,7 @@ Input [2]: [d_date_sk#4, d_date#5] (19) BroadcastExchange Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (20) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] @@ -157,191 +168,240 @@ Output [2]: [sr_item_sk#1, sr_return_quantity#2] Input [4]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, d_date_sk#4] (22) Scan parquet default.item -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#12, i_item_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (23) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#13, i_item_id#14] +Input [2]: [i_item_sk#12, i_item_id#13] (24) Filter [codegen id : 4] -Input [2]: [i_item_sk#13, i_item_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) +Input [2]: [i_item_sk#12, i_item_id#13] +Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) (25) BroadcastExchange -Input [2]: [i_item_sk#13, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] +Input [2]: [i_item_sk#12, i_item_id#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] (26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_item_sk#1] -Right keys [1]: [cast(i_item_sk#13 as bigint)] +Right keys [1]: [cast(i_item_sk#12 as bigint)] Join condition: None (27) Project [codegen id : 5] -Output [2]: [sr_return_quantity#2, i_item_id#14] -Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#13, i_item_id#14] +Output [2]: [sr_return_quantity#2, i_item_id#13] +Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#12, i_item_id#13] (28) HashAggregate [codegen id : 5] -Input [2]: [sr_return_quantity#2, i_item_id#14] -Keys [1]: [i_item_id#14] +Input [2]: [sr_return_quantity#2, i_item_id#13] +Keys [1]: [i_item_id#13] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#16] -Results [2]: [i_item_id#14, sum#17] +Aggregate Attributes [1]: [sum#15] +Results [2]: [i_item_id#13, sum#16] (29) Exchange -Input [2]: [i_item_id#14, sum#17] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [2]: [i_item_id#13, sum#16] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#17] (30) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#17] -Keys [1]: [i_item_id#14] +Input [2]: [i_item_id#13, sum#16] +Keys [1]: [i_item_id#13] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] -Results [2]: [i_item_id#14 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#18] +Results [2]: [i_item_id#13 AS item_id#19, sum(sr_return_quantity#2)#18 AS sr_item_qty#20] (31) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +Output [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] (33) Filter [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] -Condition : isnotnull(cr_item_sk#22) +Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Condition : isnotnull(cr_item_sk#21) -(34) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#25] +(34) Scan parquet default.date_dim +Output [2]: [d_date_sk#24, d_date#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct + +(35) ColumnarToRow [codegen id : 8] +Input [2]: [d_date_sk#24, d_date#25] + +(36) Filter [codegen id : 8] +Input [2]: [d_date_sk#24, d_date#25] +Condition : isnotnull(d_date_sk#24) + +(37) Scan parquet default.date_dim +Output [2]: [d_date#5, d_week_seq#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +ReadSchema: struct + +(38) ColumnarToRow [codegen id : 7] +Input [2]: [d_date#5, d_week_seq#6] + +(39) ReusedExchange [Reuses operator id: 13] +Output [1]: [d_week_seq#6#26] + +(40) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [d_week_seq#6] +Right keys [1]: [d_week_seq#6#26] +Join condition: None + +(41) Project [codegen id : 7] +Output [1]: [d_date#5] +Input [2]: [d_date#5, d_week_seq#6] + +(42) BroadcastExchange +Input [1]: [d_date#5] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#27] + +(43) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [d_date#25] +Right keys [1]: [d_date#5] +Join condition: None + +(44) Project [codegen id : 8] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_date#25] + +(45) BroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] -(35) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#24] -Right keys [1]: [d_date_sk#25] +(46) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_returned_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None -(36) Project [codegen id : 10] -Output [2]: [cr_item_sk#22, cr_return_quantity#23] -Input [4]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, d_date_sk#25] +(47) Project [codegen id : 10] +Output [2]: [cr_item_sk#21, cr_return_quantity#22] +Input [4]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, d_date_sk#24] -(37) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#26, i_item_id#27] +(48) ReusedExchange [Reuses operator id: 25] +Output [2]: [i_item_sk#29, i_item_id#30] -(38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#22] -Right keys [1]: [i_item_sk#26] +(49) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_item_sk#21] +Right keys [1]: [i_item_sk#29] Join condition: None -(39) Project [codegen id : 10] -Output [2]: [cr_return_quantity#23, i_item_id#27] -Input [4]: [cr_item_sk#22, cr_return_quantity#23, i_item_sk#26, i_item_id#27] - -(40) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#23, i_item_id#27] -Keys [1]: [i_item_id#27] -Functions [1]: [partial_sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#27, sum#29] - -(41) Exchange -Input [2]: [i_item_id#27, sum#29] -Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, [id=#30] - -(42) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#27, sum#29] -Keys [1]: [i_item_id#27] -Functions [1]: [sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum(cr_return_quantity#23)#31] -Results [2]: [i_item_id#27 AS item_id#32, sum(cr_return_quantity#23)#31 AS cr_item_qty#33] - -(43) BroadcastExchange -Input [2]: [item_id#32, cr_item_qty#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] - -(44) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#32] +(50) Project [codegen id : 10] +Output [2]: [cr_return_quantity#22, i_item_id#30] +Input [4]: [cr_item_sk#21, cr_return_quantity#22, i_item_sk#29, i_item_id#30] + +(51) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#22, i_item_id#30] +Keys [1]: [i_item_id#30] +Functions [1]: [partial_sum(cr_return_quantity#22)] +Aggregate Attributes [1]: [sum#31] +Results [2]: [i_item_id#30, sum#32] + +(52) Exchange +Input [2]: [i_item_id#30, sum#32] +Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [id=#33] + +(53) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#30, sum#32] +Keys [1]: [i_item_id#30] +Functions [1]: [sum(cr_return_quantity#22)] +Aggregate Attributes [1]: [sum(cr_return_quantity#22)#34] +Results [2]: [i_item_id#30 AS item_id#35, sum(cr_return_quantity#22)#34 AS cr_item_qty#36] + +(54) BroadcastExchange +Input [2]: [item_id#35, cr_item_qty#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#37] + +(55) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#35] Join condition: None -(45) Project [codegen id : 18] -Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#33] -Input [4]: [item_id#20, sr_item_qty#21, item_id#32, cr_item_qty#33] +(56) Project [codegen id : 18] +Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#36] +Input [4]: [item_id#19, sr_item_qty#20, item_id#35, cr_item_qty#36] -(46) Scan parquet default.web_returns -Output [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +(57) Scan parquet default.web_returns +Output [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#37), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(wr_returned_date_sk#40), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +(58) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] -(48) Filter [codegen id : 16] -Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] -Condition : isnotnull(wr_item_sk#35) +(59) Filter [codegen id : 16] +Input [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] +Condition : isnotnull(wr_item_sk#38) -(49) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#38] +(60) ReusedExchange [Reuses operator id: 45] +Output [1]: [d_date_sk#41] -(50) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#37] -Right keys [1]: [cast(d_date_sk#38 as bigint)] +(61) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#40] +Right keys [1]: [cast(d_date_sk#41 as bigint)] Join condition: None -(51) Project [codegen id : 16] -Output [2]: [wr_item_sk#35, wr_return_quantity#36] -Input [4]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37, d_date_sk#38] +(62) Project [codegen id : 16] +Output [2]: [wr_item_sk#38, wr_return_quantity#39] +Input [4]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40, d_date_sk#41] -(52) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#39, i_item_id#40] +(63) ReusedExchange [Reuses operator id: 25] +Output [2]: [i_item_sk#42, i_item_id#43] -(53) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#35] -Right keys [1]: [cast(i_item_sk#39 as bigint)] +(64) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#38] +Right keys [1]: [cast(i_item_sk#42 as bigint)] Join condition: None -(54) Project [codegen id : 16] -Output [2]: [wr_return_quantity#36, i_item_id#40] -Input [4]: [wr_item_sk#35, wr_return_quantity#36, i_item_sk#39, i_item_id#40] - -(55) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#36, i_item_id#40] -Keys [1]: [i_item_id#40] -Functions [1]: [partial_sum(wr_return_quantity#36)] -Aggregate Attributes [1]: [sum#41] -Results [2]: [i_item_id#40, sum#42] - -(56) Exchange -Input [2]: [i_item_id#40, sum#42] -Arguments: hashpartitioning(i_item_id#40, 5), ENSURE_REQUIREMENTS, [id=#43] - -(57) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#40, sum#42] -Keys [1]: [i_item_id#40] -Functions [1]: [sum(wr_return_quantity#36)] -Aggregate Attributes [1]: [sum(wr_return_quantity#36)#44] -Results [2]: [i_item_id#40 AS item_id#45, sum(wr_return_quantity#36)#44 AS wr_item_qty#46] - -(58) BroadcastExchange -Input [2]: [item_id#45, wr_item_qty#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#47] - -(59) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#45] +(65) Project [codegen id : 16] +Output [2]: [wr_return_quantity#39, i_item_id#43] +Input [4]: [wr_item_sk#38, wr_return_quantity#39, i_item_sk#42, i_item_id#43] + +(66) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#39, i_item_id#43] +Keys [1]: [i_item_id#43] +Functions [1]: [partial_sum(wr_return_quantity#39)] +Aggregate Attributes [1]: [sum#44] +Results [2]: [i_item_id#43, sum#45] + +(67) Exchange +Input [2]: [i_item_id#43, sum#45] +Arguments: hashpartitioning(i_item_id#43, 5), ENSURE_REQUIREMENTS, [id=#46] + +(68) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#43, sum#45] +Keys [1]: [i_item_id#43] +Functions [1]: [sum(wr_return_quantity#39)] +Aggregate Attributes [1]: [sum(wr_return_quantity#39)#47] +Results [2]: [i_item_id#43 AS item_id#48, sum(wr_return_quantity#39)#47 AS wr_item_qty#49] + +(69) BroadcastExchange +Input [2]: [item_id#48, wr_item_qty#49] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] + +(70) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#48] Join condition: None -(60) Project [codegen id : 18] -Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS sr_dev#48, cr_item_qty#33, (((cast(cr_item_qty#33 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS cr_dev#49, wr_item_qty#46, (((cast(wr_item_qty#46 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS wr_dev#50, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#51] -Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#33, item_id#45, wr_item_qty#46] +(71) Project [codegen id : 18] +Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS sr_dev#51, cr_item_qty#36, (((cast(cr_item_qty#36 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS cr_dev#52, wr_item_qty#49, (((cast(wr_item_qty#49 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS wr_dev#53, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#54] +Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#36, item_id#48, wr_item_qty#49] -(61) TakeOrderedAndProject -Input [8]: [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] -Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] +(72) TakeOrderedAndProject +Input [8]: [item_id#19, sr_item_qty#20, sr_dev#51, cr_item_qty#36, cr_dev#52, wr_item_qty#49, wr_dev#53, average#54] +Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#51, cr_item_qty#36, cr_dev#52, wr_item_qty#49, wr_dev#53, average#54] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt index b46131afd5ec7..819d571695a83 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt @@ -66,15 +66,32 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] InputAdapter - ReusedExchange [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (8) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + Filter [d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (7) + Project [d_date] + BroadcastHashJoin [d_week_seq,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] + InputAdapter + ReusedExchange [d_week_seq] #4 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 InputAdapter - BroadcastExchange #8 + BroadcastExchange #10 WholeStageCodegen (17) HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] InputAdapter - Exchange [i_item_id] #9 + Exchange [i_item_id] #11 WholeStageCodegen (16) HashAggregate [i_item_id,wr_return_quantity] [sum,sum] Project [wr_return_quantity,i_item_id] @@ -86,6 +103,6 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] InputAdapter - ReusedExchange [d_date_sk] #2 + ReusedExchange [d_date_sk] #8 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index fd73cee998f71..8b14a282c2671 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (61) -+- * Project (60) - +- * BroadcastHashJoin Inner BuildRight (59) - :- * Project (45) - : +- * BroadcastHashJoin Inner BuildRight (44) +TakeOrderedAndProject (72) ++- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) : :- * HashAggregate (30) : : +- Exchange (29) : : +- * HashAggregate (28) @@ -34,32 +34,43 @@ TakeOrderedAndProject (61) : : +- * Filter (17) : : +- * ColumnarToRow (16) : : +- Scan parquet default.date_dim (15) - : +- BroadcastExchange (43) - : +- * HashAggregate (42) - : +- Exchange (41) - : +- * HashAggregate (40) - : +- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) + : +- BroadcastExchange (54) + : +- * HashAggregate (53) + : +- Exchange (52) + : +- * HashAggregate (51) + : +- * Project (50) + : +- * BroadcastHashJoin Inner BuildRight (49) : :- * Project (36) : : +- * BroadcastHashJoin Inner BuildRight (35) : : :- * Filter (33) : : : +- * ColumnarToRow (32) : : : +- Scan parquet default.catalog_returns (31) : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- BroadcastExchange (58) - +- * HashAggregate (57) - +- Exchange (56) - +- * HashAggregate (55) - +- * Project (54) - +- * BroadcastHashJoin Inner BuildRight (53) - :- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Filter (48) - : : +- * ColumnarToRow (47) - : : +- Scan parquet default.web_returns (46) - : +- ReusedExchange (49) - +- ReusedExchange (52) + : +- BroadcastExchange (48) + : +- * Project (47) + : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : :- * Filter (39) + : : +- * ColumnarToRow (38) + : : +- Scan parquet default.date_dim (37) + : +- BroadcastExchange (45) + : +- * Project (44) + : +- * BroadcastHashJoin LeftSemi BuildRight (43) + : :- * ColumnarToRow (41) + : : +- Scan parquet default.date_dim (40) + : +- ReusedExchange (42) + +- BroadcastExchange (69) + +- * HashAggregate (68) + +- Exchange (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Filter (59) + : : +- * ColumnarToRow (58) + : : +- Scan parquet default.web_returns (57) + : +- ReusedExchange (60) + +- ReusedExchange (63) (1) Scan parquet default.store_returns @@ -119,51 +130,51 @@ Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) (13) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] -Input [2]: [d_date#9, d_week_seq#10] +Input [2]: [d_date#8, d_week_seq#9] (15) Scan parquet default.date_dim -Output [2]: [d_date#11, d_week_seq#12] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (16) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#11, d_week_seq#12] +Input [2]: [d_date#8, d_week_seq#9] (17) Filter [codegen id : 2] -Input [2]: [d_date#11, d_week_seq#12] -Condition : cast(d_date#11 as string) IN (2000-06-30,2000-09-27,2000-11-17) +Input [2]: [d_date#8, d_week_seq#9] +Condition : cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) (18) Project [codegen id : 2] -Output [1]: [d_week_seq#12] -Input [2]: [d_date#11, d_week_seq#12] +Output [1]: [d_week_seq#9 AS d_week_seq#9#10] +Input [2]: [d_date#8, d_week_seq#9] (19) BroadcastExchange -Input [1]: [d_week_seq#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] +Input [1]: [d_week_seq#9#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#12] +Left keys [1]: [d_week_seq#9] +Right keys [1]: [d_week_seq#9#10] Join condition: None (21) Project [codegen id : 3] -Output [1]: [d_date#9] -Input [2]: [d_date#9, d_week_seq#10] +Output [1]: [d_date#8 AS d_date#8#12] +Input [2]: [d_date#8, d_week_seq#9] (22) BroadcastExchange -Input [1]: [d_date#9] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#14] +Input [1]: [d_date#8#12] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] (23) BroadcastHashJoin [codegen id : 4] Left keys [1]: [d_date#8] -Right keys [1]: [d_date#9] +Right keys [1]: [d_date#8#12] Join condition: None (24) Project [codegen id : 4] @@ -172,7 +183,7 @@ Input [2]: [d_date_sk#7, d_date#8] (25) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] @@ -187,161 +198,210 @@ Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#5, d_date_sk# Input [2]: [sr_return_quantity#2, i_item_id#5] Keys [1]: [i_item_id#5] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#16] -Results [2]: [i_item_id#5, sum#17] +Aggregate Attributes [1]: [sum#15] +Results [2]: [i_item_id#5, sum#16] (29) Exchange -Input [2]: [i_item_id#5, sum#17] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [2]: [i_item_id#5, sum#16] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#17] (30) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#5, sum#17] +Input [2]: [i_item_id#5, sum#16] Keys [1]: [i_item_id#5] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] -Results [2]: [i_item_id#5 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#18] +Results [2]: [i_item_id#5 AS item_id#19, sum(sr_return_quantity#2)#18 AS sr_item_qty#20] (31) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +Output [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] (33) Filter [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] -Condition : isnotnull(cr_item_sk#22) +Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Condition : isnotnull(cr_item_sk#21) (34) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#25, i_item_id#26] +Output [2]: [i_item_sk#24, i_item_id#25] (35) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#22] -Right keys [1]: [i_item_sk#25] +Left keys [1]: [cr_item_sk#21] +Right keys [1]: [i_item_sk#24] Join condition: None (36) Project [codegen id : 10] -Output [3]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#26] -Input [5]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, i_item_sk#25, i_item_id#26] +Output [3]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#25] +Input [5]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, i_item_sk#24, i_item_id#25] -(37) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#27] +(37) Scan parquet default.date_dim +Output [2]: [d_date_sk#26, d_date#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct + +(38) ColumnarToRow [codegen id : 9] +Input [2]: [d_date_sk#26, d_date#27] + +(39) Filter [codegen id : 9] +Input [2]: [d_date_sk#26, d_date#27] +Condition : isnotnull(d_date_sk#26) + +(40) Scan parquet default.date_dim +Output [2]: [d_date#8, d_week_seq#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +ReadSchema: struct + +(41) ColumnarToRow [codegen id : 8] +Input [2]: [d_date#8, d_week_seq#9] + +(42) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_week_seq#9#28] + +(43) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [d_week_seq#9] +Right keys [1]: [d_week_seq#9#28] +Join condition: None + +(44) Project [codegen id : 8] +Output [1]: [d_date#8] +Input [2]: [d_date#8, d_week_seq#9] + +(45) BroadcastExchange +Input [1]: [d_date#8] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#29] + +(46) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [d_date#27] +Right keys [1]: [d_date#8] +Join condition: None + +(47) Project [codegen id : 9] +Output [1]: [d_date_sk#26] +Input [2]: [d_date_sk#26, d_date#27] + +(48) BroadcastExchange +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] -(38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#24] -Right keys [1]: [d_date_sk#27] +(49) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_returned_date_sk#23] +Right keys [1]: [d_date_sk#26] Join condition: None -(39) Project [codegen id : 10] -Output [2]: [cr_return_quantity#23, i_item_id#26] -Input [4]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#26, d_date_sk#27] - -(40) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#23, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#26, sum#29] - -(41) Exchange -Input [2]: [i_item_id#26, sum#29] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [id=#30] - -(42) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#26, sum#29] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum(cr_return_quantity#23)#31] -Results [2]: [i_item_id#26 AS item_id#32, sum(cr_return_quantity#23)#31 AS cr_item_qty#33] - -(43) BroadcastExchange -Input [2]: [item_id#32, cr_item_qty#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] - -(44) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#32] +(50) Project [codegen id : 10] +Output [2]: [cr_return_quantity#22, i_item_id#25] +Input [4]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#25, d_date_sk#26] + +(51) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#22, i_item_id#25] +Keys [1]: [i_item_id#25] +Functions [1]: [partial_sum(cr_return_quantity#22)] +Aggregate Attributes [1]: [sum#31] +Results [2]: [i_item_id#25, sum#32] + +(52) Exchange +Input [2]: [i_item_id#25, sum#32] +Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [id=#33] + +(53) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#25, sum#32] +Keys [1]: [i_item_id#25] +Functions [1]: [sum(cr_return_quantity#22)] +Aggregate Attributes [1]: [sum(cr_return_quantity#22)#34] +Results [2]: [i_item_id#25 AS item_id#35, sum(cr_return_quantity#22)#34 AS cr_item_qty#36] + +(54) BroadcastExchange +Input [2]: [item_id#35, cr_item_qty#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#37] + +(55) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#35] Join condition: None -(45) Project [codegen id : 18] -Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#33] -Input [4]: [item_id#20, sr_item_qty#21, item_id#32, cr_item_qty#33] +(56) Project [codegen id : 18] +Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#36] +Input [4]: [item_id#19, sr_item_qty#20, item_id#35, cr_item_qty#36] -(46) Scan parquet default.web_returns -Output [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +(57) Scan parquet default.web_returns +Output [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#37), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(wr_returned_date_sk#40), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(47) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +(58) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] -(48) Filter [codegen id : 16] -Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] -Condition : isnotnull(wr_item_sk#35) +(59) Filter [codegen id : 16] +Input [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] +Condition : isnotnull(wr_item_sk#38) -(49) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#38, i_item_id#39] +(60) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#41, i_item_id#42] -(50) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#35] -Right keys [1]: [cast(i_item_sk#38 as bigint)] +(61) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#38] +Right keys [1]: [cast(i_item_sk#41 as bigint)] Join condition: None -(51) Project [codegen id : 16] -Output [3]: [wr_return_quantity#36, wr_returned_date_sk#37, i_item_id#39] -Input [5]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37, i_item_sk#38, i_item_id#39] +(62) Project [codegen id : 16] +Output [3]: [wr_return_quantity#39, wr_returned_date_sk#40, i_item_id#42] +Input [5]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40, i_item_sk#41, i_item_id#42] -(52) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#40] +(63) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#43] -(53) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#37] -Right keys [1]: [cast(d_date_sk#40 as bigint)] +(64) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#40] +Right keys [1]: [cast(d_date_sk#43 as bigint)] Join condition: None -(54) Project [codegen id : 16] -Output [2]: [wr_return_quantity#36, i_item_id#39] -Input [4]: [wr_return_quantity#36, wr_returned_date_sk#37, i_item_id#39, d_date_sk#40] - -(55) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#36, i_item_id#39] -Keys [1]: [i_item_id#39] -Functions [1]: [partial_sum(wr_return_quantity#36)] -Aggregate Attributes [1]: [sum#41] -Results [2]: [i_item_id#39, sum#42] - -(56) Exchange -Input [2]: [i_item_id#39, sum#42] -Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, [id=#43] - -(57) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#39, sum#42] -Keys [1]: [i_item_id#39] -Functions [1]: [sum(wr_return_quantity#36)] -Aggregate Attributes [1]: [sum(wr_return_quantity#36)#44] -Results [2]: [i_item_id#39 AS item_id#45, sum(wr_return_quantity#36)#44 AS wr_item_qty#46] - -(58) BroadcastExchange -Input [2]: [item_id#45, wr_item_qty#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#47] - -(59) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#45] +(65) Project [codegen id : 16] +Output [2]: [wr_return_quantity#39, i_item_id#42] +Input [4]: [wr_return_quantity#39, wr_returned_date_sk#40, i_item_id#42, d_date_sk#43] + +(66) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#39, i_item_id#42] +Keys [1]: [i_item_id#42] +Functions [1]: [partial_sum(wr_return_quantity#39)] +Aggregate Attributes [1]: [sum#44] +Results [2]: [i_item_id#42, sum#45] + +(67) Exchange +Input [2]: [i_item_id#42, sum#45] +Arguments: hashpartitioning(i_item_id#42, 5), ENSURE_REQUIREMENTS, [id=#46] + +(68) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#42, sum#45] +Keys [1]: [i_item_id#42] +Functions [1]: [sum(wr_return_quantity#39)] +Aggregate Attributes [1]: [sum(wr_return_quantity#39)#47] +Results [2]: [i_item_id#42 AS item_id#48, sum(wr_return_quantity#39)#47 AS wr_item_qty#49] + +(69) BroadcastExchange +Input [2]: [item_id#48, wr_item_qty#49] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] + +(70) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#48] Join condition: None -(60) Project [codegen id : 18] -Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS sr_dev#48, cr_item_qty#33, (((cast(cr_item_qty#33 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS cr_dev#49, wr_item_qty#46, (((cast(wr_item_qty#46 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS wr_dev#50, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#51] -Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#33, item_id#45, wr_item_qty#46] +(71) Project [codegen id : 18] +Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS sr_dev#51, cr_item_qty#36, (((cast(cr_item_qty#36 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS cr_dev#52, wr_item_qty#49, (((cast(wr_item_qty#49 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS wr_dev#53, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#54] +Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#36, item_id#48, wr_item_qty#49] -(61) TakeOrderedAndProject -Input [8]: [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] -Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] +(72) TakeOrderedAndProject +Input [8]: [item_id#19, sr_item_qty#20, sr_dev#51, cr_item_qty#36, cr_dev#52, wr_item_qty#49, wr_dev#53, average#54] +Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#51, cr_item_qty#36, cr_dev#52, wr_item_qty#49, wr_dev#53, average#54] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index 7fc930f2885fd..9632348854668 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -68,13 +68,30 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #8 + WholeStageCodegen (9) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + Filter [d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (8) + Project [d_date] + BroadcastHashJoin [d_week_seq,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date,d_week_seq] + InputAdapter + ReusedExchange [d_week_seq] #5 InputAdapter - BroadcastExchange #8 + BroadcastExchange #10 WholeStageCodegen (17) HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] InputAdapter - Exchange [i_item_id] #9 + Exchange [i_item_id] #11 WholeStageCodegen (16) HashAggregate [i_item_id,wr_return_quantity] [sum,sum] Project [wr_return_quantity,i_item_id] @@ -88,4 +105,4 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - ReusedExchange [d_date_sk] #3 + ReusedExchange [d_date_sk] #8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt index 8ae9d897a7d79..8297114e1e9ab 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt @@ -119,82 +119,82 @@ Input [3]: [ws_item_sk#4, sum#13, count#14] Keys [1]: [ws_item_sk#4] Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#5))] Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#5))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4 AS ws_item_sk#4#18] (19) Filter -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4#18] Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) (20) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#1] -Right keys [1]: [ws_item_sk#4] +Right keys [1]: [ws_item_sk#4#18] Join condition: None (21) Project [codegen id : 4] Output [2]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17] -Input [3]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4] +Input [3]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4#18] (22) BroadcastExchange Input [2]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (23) Scan parquet default.web_sales -Output [3]: [ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] +Output [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_ext_discount_amt)] ReadSchema: struct (24) ColumnarToRow -Input [3]: [ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] +Input [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] (25) Filter -Input [3]: [ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] -Condition : (isnotnull(ws_item_sk#19) AND isnotnull(ws_ext_discount_amt#20)) +Input [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] +Condition : (isnotnull(ws_item_sk#4) AND isnotnull(ws_ext_discount_amt#5)) (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#1] -Right keys [1]: [ws_item_sk#19] -Join condition: (cast(ws_ext_discount_amt#20 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) +Right keys [1]: [ws_item_sk#4] +Join condition: (cast(ws_ext_discount_amt#5 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) (27) Project [codegen id : 6] -Output [2]: [ws_ext_discount_amt#20, ws_sold_date_sk#21] -Input [5]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] +Output [2]: [ws_ext_discount_amt#5, ws_sold_date_sk#6] +Input [5]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] (28) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#8] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#21] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [ws_sold_date_sk#6] +Right keys [1]: [d_date_sk#8] Join condition: None (30) Project [codegen id : 6] -Output [1]: [ws_ext_discount_amt#20] -Input [3]: [ws_ext_discount_amt#20, ws_sold_date_sk#21, d_date_sk#22] +Output [1]: [ws_ext_discount_amt#5] +Input [3]: [ws_ext_discount_amt#5, ws_sold_date_sk#6, d_date_sk#8] (31) HashAggregate [codegen id : 6] -Input [1]: [ws_ext_discount_amt#20] +Input [1]: [ws_ext_discount_amt#5] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#20))] -Aggregate Attributes [1]: [sum#23] -Results [1]: [sum#24] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#5))] +Aggregate Attributes [1]: [sum#20] +Results [1]: [sum#21] (32) Exchange -Input [1]: [sum#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#25] +Input [1]: [sum#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] (33) HashAggregate [codegen id : 7] -Input [1]: [sum#24] +Input [1]: [sum#21] Keys: [] -Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#20))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#20))#26] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#20))#26,17,2) AS Excess Discount Amount #27] +Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#5))#23] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#5))#23,17,2) AS Excess Discount Amount #24] (34) Sort [codegen id : 7] -Input [1]: [Excess Discount Amount #27] -Arguments: [Excess Discount Amount #27 ASC NULLS FIRST], true, 0 +Input [1]: [Excess Discount Amount #24] +Arguments: [Excess Discount Amount #24 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -205,6 +205,6 @@ ReusedExchange (35) (35) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt index e83a3e67e5c6f..cc3ffa0de4bfd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt @@ -23,7 +23,7 @@ WholeStageCodegen (7) InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] Filter [(1.3 * avg(ws_ext_discount_amt))] - HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] + HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,sum,count] InputAdapter Exchange [ws_item_sk] #4 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index bae7dd2f3a240..faf82026138f6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -82,119 +82,119 @@ Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] (11) Scan parquet default.web_sales -Output [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] +Output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] +Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] (13) Filter [codegen id : 3] -Input [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] -Condition : isnotnull(ws_item_sk#8) +Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] +Condition : isnotnull(ws_item_sk#1) (14) Scan parquet default.date_dim -Output [2]: [d_date_sk#11, d_date#12] +Output [2]: [d_date_sk#8, d_date#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#11, d_date#12] +Input [2]: [d_date_sk#8, d_date#9] (16) Filter [codegen id : 2] -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 10983)) AND (d_date#12 <= 11073)) AND isnotnull(d_date_sk#11)) +Input [2]: [d_date_sk#8, d_date#9] +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) (17) Project [codegen id : 2] -Output [1]: [d_date_sk#11] -Input [2]: [d_date_sk#11, d_date#12] +Output [1]: [d_date_sk#8] +Input [2]: [d_date_sk#8, d_date#9] (18) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] +Input [1]: [d_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] (19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#10] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#8] Join condition: None (20) Project [codegen id : 3] -Output [2]: [ws_item_sk#8, ws_ext_discount_amt#9] -Input [4]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10, d_date_sk#11] +Output [2]: [ws_item_sk#1, ws_ext_discount_amt#2] +Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#8] (21) HashAggregate [codegen id : 3] -Input [2]: [ws_item_sk#8, ws_ext_discount_amt#9] -Keys [1]: [ws_item_sk#8] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#9))] -Aggregate Attributes [2]: [sum#14, count#15] -Results [3]: [ws_item_sk#8, sum#16, count#17] +Input [2]: [ws_item_sk#1, ws_ext_discount_amt#2] +Keys [1]: [ws_item_sk#1] +Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#2))] +Aggregate Attributes [2]: [sum#11, count#12] +Results [3]: [ws_item_sk#1, sum#13, count#14] (22) Exchange -Input [3]: [ws_item_sk#8, sum#16, count#17] -Arguments: hashpartitioning(ws_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [ws_item_sk#1, sum#13, count#14] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#15] (23) HashAggregate [codegen id : 4] -Input [3]: [ws_item_sk#8, sum#16, count#17] -Keys [1]: [ws_item_sk#8] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#9))] -Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#9))#19] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#9))#19 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] +Input [3]: [ws_item_sk#1, sum#13, count#14] +Keys [1]: [ws_item_sk#1] +Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#2))] +Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#2))#16] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#2))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1 AS ws_item_sk#1#18] (24) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#20) +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] +Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) (25) BroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#21] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#19] (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] -Right keys [1]: [ws_item_sk#8] -Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#20) +Right keys [1]: [ws_item_sk#1#18] +Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) (27) Project [codegen id : 6] Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] +Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] (28) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#8] (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#8] Join condition: None (30) Project [codegen id : 6] Output [1]: [ws_ext_discount_amt#2] -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#22] +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#8] (31) HashAggregate [codegen id : 6] Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#23] -Results [1]: [sum#24] +Aggregate Attributes [1]: [sum#20] +Results [1]: [sum#21] (32) Exchange -Input [1]: [sum#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#25] +Input [1]: [sum#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] (33) HashAggregate [codegen id : 7] -Input [1]: [sum#24] +Input [1]: [sum#21] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#26] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#26,17,2) AS Excess Discount Amount #27] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#23] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#23,17,2) AS Excess Discount Amount #24] (34) Sort [codegen id : 7] -Input [1]: [Excess Discount Amount #27] -Arguments: [Excess Discount Amount #27 ASC NULLS FIRST], true, 0 +Input [1]: [Excess Discount Amount #24] +Arguments: [Excess Discount Amount #24 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -203,8 +203,8 @@ ReusedExchange (35) (35) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index 0df713c0e075e..3108c3bc82c7c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -29,7 +29,7 @@ WholeStageCodegen (7) BroadcastExchange #4 WholeStageCodegen (4) Filter [(1.3 * avg(ws_ext_discount_amt))] - HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] + HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,sum,count] InputAdapter Exchange [ws_item_sk] #5 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt index 3f2b62a7b8fa4..b2934b6bb8242 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt @@ -81,57 +81,57 @@ Input [3]: [i_item_sk#5, i_current_price#6, i_category#7] Condition : (isnotnull(i_current_price#6) AND isnotnull(i_item_sk#5)) (7) Scan parquet default.item -Output [2]: [i_current_price#8, i_category#9] +Output [2]: [i_current_price#6, i_category#7] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [i_current_price#8, i_category#9] +Input [2]: [i_current_price#6, i_category#7] (9) Filter [codegen id : 1] -Input [2]: [i_current_price#8, i_category#9] -Condition : isnotnull(i_category#9) +Input [2]: [i_current_price#6, i_category#7] +Condition : isnotnull(i_category#7) (10) HashAggregate [codegen id : 1] -Input [2]: [i_current_price#8, i_category#9] -Keys [1]: [i_category#9] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#8))] -Aggregate Attributes [2]: [sum#10, count#11] -Results [3]: [i_category#9, sum#12, count#13] +Input [2]: [i_current_price#6, i_category#7] +Keys [1]: [i_category#7] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#6))] +Aggregate Attributes [2]: [sum#8, count#9] +Results [3]: [i_category#7, sum#10, count#11] (11) Exchange -Input [3]: [i_category#9, sum#12, count#13] -Arguments: hashpartitioning(i_category#9, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [3]: [i_category#7, sum#10, count#11] +Arguments: hashpartitioning(i_category#7, 5), ENSURE_REQUIREMENTS, [id=#12] (12) HashAggregate [codegen id : 2] -Input [3]: [i_category#9, sum#12, count#13] -Keys [1]: [i_category#9] -Functions [1]: [avg(UnscaledValue(i_current_price#8))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#8))#15] -Results [2]: [cast((avg(UnscaledValue(i_current_price#8))#15 / 100.0) as decimal(11,6)) AS avg(i_current_price)#16, i_category#9] +Input [3]: [i_category#7, sum#10, count#11] +Keys [1]: [i_category#7] +Functions [1]: [avg(UnscaledValue(i_current_price#6))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#6))#13] +Results [2]: [cast((avg(UnscaledValue(i_current_price#6))#13 / 100.0) as decimal(11,6)) AS avg(i_current_price)#14, i_category#7 AS i_category#7#15] (13) BroadcastExchange -Input [2]: [avg(i_current_price)#16, i_category#9] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#17] +Input [2]: [avg(i_current_price)#14, i_category#7#15] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#16] (14) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_category#7] -Right keys [1]: [i_category#9] +Right keys [1]: [i_category#7#15] Join condition: None (15) Filter [codegen id : 3] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] -Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#16)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] +Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#14)), DecimalType(14,7), true)) (16) Project [codegen id : 3] Output [1]: [i_item_sk#5] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] (17) BroadcastExchange Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (18) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#1] @@ -143,143 +143,143 @@ Output [2]: [ss_customer_sk#2, ss_sold_date_sk#3] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, i_item_sk#5] (20) Scan parquet default.date_dim -Output [2]: [d_date_sk#19, d_month_seq#20] +Output [2]: [d_date_sk#18, d_month_seq#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#19, d_month_seq#20] +Input [2]: [d_date_sk#18, d_month_seq#19] (22) Filter [codegen id : 4] -Input [2]: [d_date_sk#19, d_month_seq#20] -Condition : ((isnotnull(d_month_seq#20) AND (d_month_seq#20 = Subquery scalar-subquery#21, [id=#22])) AND isnotnull(d_date_sk#19)) +Input [2]: [d_date_sk#18, d_month_seq#19] +Condition : ((isnotnull(d_month_seq#19) AND (d_month_seq#19 = Subquery scalar-subquery#20, [id=#21])) AND isnotnull(d_date_sk#18)) (23) Project [codegen id : 4] -Output [1]: [d_date_sk#19] -Input [2]: [d_date_sk#19, d_month_seq#20] +Output [1]: [d_date_sk#18] +Input [2]: [d_date_sk#18, d_month_seq#19] (24) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [d_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (25) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#19] +Right keys [1]: [d_date_sk#18] Join condition: None (26) Project [codegen id : 5] Output [1]: [ss_customer_sk#2] -Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#19] +Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#18] (27) Exchange Input [1]: [ss_customer_sk#2] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#23] (28) Sort [codegen id : 6] Input [1]: [ss_customer_sk#2] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#25, ca_state#26] +Output [2]: [ca_address_sk#24, ca_state#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#25, ca_state#26] +Input [2]: [ca_address_sk#24, ca_state#25] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#25, ca_state#26] -Condition : isnotnull(ca_address_sk#25) +Input [2]: [ca_address_sk#24, ca_state#25] +Condition : isnotnull(ca_address_sk#24) (32) Exchange -Input [2]: [ca_address_sk#25, ca_state#26] -Arguments: hashpartitioning(ca_address_sk#25, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] (33) Sort [codegen id : 8] -Input [2]: [ca_address_sk#25, ca_state#26] -Arguments: [ca_address_sk#25 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 (34) Scan parquet default.customer -Output [2]: [c_customer_sk#28, c_current_addr_sk#29] +Output [2]: [c_customer_sk#27, c_current_addr_sk#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 9] -Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Input [2]: [c_customer_sk#27, c_current_addr_sk#28] (36) Filter [codegen id : 9] -Input [2]: [c_customer_sk#28, c_current_addr_sk#29] -Condition : (isnotnull(c_current_addr_sk#29) AND isnotnull(c_customer_sk#28)) +Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Condition : (isnotnull(c_current_addr_sk#28) AND isnotnull(c_customer_sk#27)) (37) Exchange -Input [2]: [c_customer_sk#28, c_current_addr_sk#29] -Arguments: hashpartitioning(c_current_addr_sk#29, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Arguments: hashpartitioning(c_current_addr_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] (38) Sort [codegen id : 10] -Input [2]: [c_customer_sk#28, c_current_addr_sk#29] -Arguments: [c_current_addr_sk#29 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Arguments: [c_current_addr_sk#28 ASC NULLS FIRST], false, 0 (39) SortMergeJoin [codegen id : 11] -Left keys [1]: [ca_address_sk#25] -Right keys [1]: [c_current_addr_sk#29] +Left keys [1]: [ca_address_sk#24] +Right keys [1]: [c_current_addr_sk#28] Join condition: None (40) Project [codegen id : 11] -Output [2]: [ca_state#26, c_customer_sk#28] -Input [4]: [ca_address_sk#25, ca_state#26, c_customer_sk#28, c_current_addr_sk#29] +Output [2]: [ca_state#25, c_customer_sk#27] +Input [4]: [ca_address_sk#24, ca_state#25, c_customer_sk#27, c_current_addr_sk#28] (41) Exchange -Input [2]: [ca_state#26, c_customer_sk#28] -Arguments: hashpartitioning(c_customer_sk#28, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [2]: [ca_state#25, c_customer_sk#27] +Arguments: hashpartitioning(c_customer_sk#27, 5), ENSURE_REQUIREMENTS, [id=#30] (42) Sort [codegen id : 12] -Input [2]: [ca_state#26, c_customer_sk#28] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 +Input [2]: [ca_state#25, c_customer_sk#27] +Arguments: [c_customer_sk#27 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 13] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#28] +Right keys [1]: [c_customer_sk#27] Join condition: None (44) Project [codegen id : 13] -Output [1]: [ca_state#26] -Input [3]: [ss_customer_sk#2, ca_state#26, c_customer_sk#28] +Output [1]: [ca_state#25] +Input [3]: [ss_customer_sk#2, ca_state#25, c_customer_sk#27] (45) HashAggregate [codegen id : 13] -Input [1]: [ca_state#26] -Keys [1]: [ca_state#26] +Input [1]: [ca_state#25] +Keys [1]: [ca_state#25] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#32] -Results [2]: [ca_state#26, count#33] +Aggregate Attributes [1]: [count#31] +Results [2]: [ca_state#25, count#32] (46) Exchange -Input [2]: [ca_state#26, count#33] -Arguments: hashpartitioning(ca_state#26, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [2]: [ca_state#25, count#32] +Arguments: hashpartitioning(ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#33] (47) HashAggregate [codegen id : 14] -Input [2]: [ca_state#26, count#33] -Keys [1]: [ca_state#26] +Input [2]: [ca_state#25, count#32] +Keys [1]: [ca_state#25] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [4]: [ca_state#26 AS state#36, count(1)#35 AS cnt#37, count(1)#35 AS count(1)#38, ca_state#26] +Aggregate Attributes [1]: [count(1)#34] +Results [4]: [ca_state#25 AS state#35, count(1)#34 AS cnt#36, count(1)#34 AS count(1)#37, ca_state#25] (48) Filter [codegen id : 14] -Input [4]: [state#36, cnt#37, count(1)#38, ca_state#26] -Condition : (count(1)#38 >= 10) +Input [4]: [state#35, cnt#36, count(1)#37, ca_state#25] +Condition : (count(1)#37 >= 10) (49) Project [codegen id : 14] -Output [3]: [state#36, cnt#37, ca_state#26] -Input [4]: [state#36, cnt#37, count(1)#38, ca_state#26] +Output [3]: [state#35, cnt#36, ca_state#25] +Input [4]: [state#35, cnt#36, count(1)#37, ca_state#25] (50) TakeOrderedAndProject -Input [3]: [state#36, cnt#37, ca_state#26] -Arguments: 100, [cnt#37 ASC NULLS FIRST, ca_state#26 ASC NULLS FIRST], [state#36, cnt#37] +Input [3]: [state#35, cnt#36, ca_state#25] +Arguments: 100, [cnt#36 ASC NULLS FIRST, ca_state#25 ASC NULLS FIRST], [state#35, cnt#36] ===== Subqueries ===== @@ -288,9 +288,9 @@ ReusedExchange (51) (51) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#19] +Output [1]: [d_date_sk#18] -Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#21, [id=#22] +Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#20, [id=#21] * HashAggregate (58) +- Exchange (57) +- * HashAggregate (56) @@ -301,39 +301,39 @@ Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquer (52) Scan parquet default.date_dim -Output [3]: [d_month_seq#39, d_year#40, d_moy#41] +Output [3]: [d_month_seq#19, d_year#38, d_moy#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (53) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#39, d_year#40, d_moy#41] +Input [3]: [d_month_seq#19, d_year#38, d_moy#39] (54) Filter [codegen id : 1] -Input [3]: [d_month_seq#39, d_year#40, d_moy#41] -Condition : (((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2000)) AND (d_moy#41 = 1)) +Input [3]: [d_month_seq#19, d_year#38, d_moy#39] +Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) (55) Project [codegen id : 1] -Output [1]: [d_month_seq#39] -Input [3]: [d_month_seq#39, d_year#40, d_moy#41] +Output [1]: [d_month_seq#19] +Input [3]: [d_month_seq#19, d_year#38, d_moy#39] (56) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#39] -Keys [1]: [d_month_seq#39] +Input [1]: [d_month_seq#19] +Keys [1]: [d_month_seq#19] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#39] +Results [1]: [d_month_seq#19] (57) Exchange -Input [1]: [d_month_seq#39] -Arguments: hashpartitioning(d_month_seq#39, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [1]: [d_month_seq#19] +Arguments: hashpartitioning(d_month_seq#19, 5), ENSURE_REQUIREMENTS, [id=#40] (58) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#39] -Keys [1]: [d_month_seq#39] +Input [1]: [d_month_seq#19] +Keys [1]: [d_month_seq#19] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#39] +Results [1]: [d_month_seq#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt index f3badf6efe6b6..23b705e3b97e7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt @@ -38,7 +38,7 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] InputAdapter Exchange [i_category] #6 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt index b37db85388e0f..8598023b0276d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt @@ -160,57 +160,57 @@ Input [3]: [i_item_sk#16, i_current_price#17, i_category#18] Condition : (isnotnull(i_current_price#17) AND isnotnull(i_item_sk#16)) (26) Scan parquet default.item -Output [2]: [i_current_price#19, i_category#20] +Output [2]: [i_current_price#17, i_category#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (27) ColumnarToRow [codegen id : 4] -Input [2]: [i_current_price#19, i_category#20] +Input [2]: [i_current_price#17, i_category#18] (28) Filter [codegen id : 4] -Input [2]: [i_current_price#19, i_category#20] -Condition : isnotnull(i_category#20) +Input [2]: [i_current_price#17, i_category#18] +Condition : isnotnull(i_category#18) (29) HashAggregate [codegen id : 4] -Input [2]: [i_current_price#19, i_category#20] -Keys [1]: [i_category#20] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#19))] -Aggregate Attributes [2]: [sum#21, count#22] -Results [3]: [i_category#20, sum#23, count#24] +Input [2]: [i_current_price#17, i_category#18] +Keys [1]: [i_category#18] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] +Aggregate Attributes [2]: [sum#19, count#20] +Results [3]: [i_category#18, sum#21, count#22] (30) Exchange -Input [3]: [i_category#20, sum#23, count#24] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [i_category#18, sum#21, count#22] +Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [id=#23] (31) HashAggregate [codegen id : 5] -Input [3]: [i_category#20, sum#23, count#24] -Keys [1]: [i_category#20] -Functions [1]: [avg(UnscaledValue(i_current_price#19))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#19))#26] -Results [2]: [cast((avg(UnscaledValue(i_current_price#19))#26 / 100.0) as decimal(11,6)) AS avg(i_current_price)#27, i_category#20] +Input [3]: [i_category#18, sum#21, count#22] +Keys [1]: [i_category#18] +Functions [1]: [avg(UnscaledValue(i_current_price#17))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#17))#24] +Results [2]: [cast((avg(UnscaledValue(i_current_price#17))#24 / 100.0) as decimal(11,6)) AS avg(i_current_price)#25, i_category#18 AS i_category#18#26] (32) BroadcastExchange -Input [2]: [avg(i_current_price)#27, i_category#20] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#28] +Input [2]: [avg(i_current_price)#25, i_category#18#26] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#27] (33) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_category#18] -Right keys [1]: [i_category#20] +Right keys [1]: [i_category#18#26] Join condition: None (34) Filter [codegen id : 6] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] -Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#27)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] +Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#25)), DecimalType(14,7), true)) (35) Project [codegen id : 6] Output [1]: [i_item_sk#16] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] (36) BroadcastExchange Input [1]: [i_item_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (37) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#6] @@ -225,31 +225,31 @@ Input [3]: [ca_state#2, ss_item_sk#6, i_item_sk#16] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#30] -Results [2]: [ca_state#2, count#31] +Aggregate Attributes [1]: [count#29] +Results [2]: [ca_state#2, count#30] (40) Exchange -Input [2]: [ca_state#2, count#31] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [2]: [ca_state#2, count#30] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#31] (41) HashAggregate [codegen id : 8] -Input [2]: [ca_state#2, count#31] +Input [2]: [ca_state#2, count#30] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#33] -Results [4]: [ca_state#2 AS state#34, count(1)#33 AS cnt#35, count(1)#33 AS count(1)#36, ca_state#2] +Aggregate Attributes [1]: [count(1)#32] +Results [4]: [ca_state#2 AS state#33, count(1)#32 AS cnt#34, count(1)#32 AS count(1)#35, ca_state#2] (42) Filter [codegen id : 8] -Input [4]: [state#34, cnt#35, count(1)#36, ca_state#2] -Condition : (count(1)#36 >= 10) +Input [4]: [state#33, cnt#34, count(1)#35, ca_state#2] +Condition : (count(1)#35 >= 10) (43) Project [codegen id : 8] -Output [3]: [state#34, cnt#35, ca_state#2] -Input [4]: [state#34, cnt#35, count(1)#36, ca_state#2] +Output [3]: [state#33, cnt#34, ca_state#2] +Input [4]: [state#33, cnt#34, count(1)#35, ca_state#2] (44) TakeOrderedAndProject -Input [3]: [state#34, cnt#35, ca_state#2] -Arguments: 100, [cnt#35 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#34, cnt#35] +Input [3]: [state#33, cnt#34, ca_state#2] +Arguments: 100, [cnt#34 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#33, cnt#34] ===== Subqueries ===== @@ -271,39 +271,39 @@ Subquery:2 Hosting operator id = 18 Hosting Expression = Subquery scalar-subquer (46) Scan parquet default.date_dim -Output [3]: [d_month_seq#37, d_year#38, d_moy#39] +Output [3]: [d_month_seq#12, d_year#36, d_moy#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (47) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#38, d_moy#39] +Input [3]: [d_month_seq#12, d_year#36, d_moy#37] (48) Filter [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#38, d_moy#39] -Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) +Input [3]: [d_month_seq#12, d_year#36, d_moy#37] +Condition : (((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2000)) AND (d_moy#37 = 1)) (49) Project [codegen id : 1] -Output [1]: [d_month_seq#37] -Input [3]: [d_month_seq#37, d_year#38, d_moy#39] +Output [1]: [d_month_seq#12] +Input [3]: [d_month_seq#12, d_year#36, d_moy#37] (50) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#37] -Keys [1]: [d_month_seq#37] +Input [1]: [d_month_seq#12] +Keys [1]: [d_month_seq#12] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#37] +Results [1]: [d_month_seq#12] (51) Exchange -Input [1]: [d_month_seq#37] -Arguments: hashpartitioning(d_month_seq#37, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [1]: [d_month_seq#12] +Arguments: hashpartitioning(d_month_seq#12, 5), ENSURE_REQUIREMENTS, [id=#38] (52) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#37] -Keys [1]: [d_month_seq#37] +Input [1]: [d_month_seq#12] +Keys [1]: [d_month_seq#12] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#37] +Results [1]: [d_month_seq#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index 4ba09283e73cd..ceddf1b04fc78 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -68,7 +68,7 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] InputAdapter Exchange [i_category] #8 WholeStageCodegen (4) From 4dc1ea7267f245f3f5dd1ac8f769275fcea9aa54 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 23 Mar 2021 17:25:07 +0800 Subject: [PATCH 23/38] Revert "regen golden" This reverts commit 8293531c398abfead078cc8186956c9beb60ea86. --- .../q14a.sf100/explain.txt | 708 ++++----- .../q14a.sf100/simplified.txt | 158 +- .../approved-plans-v1_4/q14a/explain.txt | 634 ++++---- .../approved-plans-v1_4/q14a/simplified.txt | 80 +- .../q14b.sf100/explain.txt | 638 ++++---- .../q14b.sf100/simplified.txt | 146 +- .../approved-plans-v1_4/q14b/explain.txt | 584 ++++---- .../approved-plans-v1_4/q14b/simplified.txt | 80 +- .../approved-plans-v1_4/q33.sf100/explain.txt | 287 ++-- .../q33.sf100/simplified.txt | 20 +- .../approved-plans-v1_4/q33/explain.txt | 287 ++-- .../approved-plans-v1_4/q33/simplified.txt | 20 +- .../approved-plans-v1_4/q56.sf100/explain.txt | 283 ++-- .../q56.sf100/simplified.txt | 20 +- .../approved-plans-v1_4/q56/explain.txt | 283 ++-- .../approved-plans-v1_4/q56/simplified.txt | 20 +- .../approved-plans-v1_4/q60.sf100/explain.txt | 283 ++-- .../q60.sf100/simplified.txt | 20 +- .../approved-plans-v1_4/q60/explain.txt | 283 ++-- .../approved-plans-v1_4/q60/simplified.txt | 20 +- .../approved-plans-v1_4/q83.sf100/explain.txt | 294 ++-- .../q83.sf100/simplified.txt | 25 +- .../approved-plans-v1_4/q83/explain.txt | 284 ++-- .../approved-plans-v1_4/q83/simplified.txt | 25 +- .../approved-plans-v2_7/q14.sf100/explain.txt | 638 ++++---- .../q14.sf100/simplified.txt | 146 +- .../approved-plans-v2_7/q14/explain.txt | 584 ++++---- .../approved-plans-v2_7/q14/simplified.txt | 80 +- .../q14a.sf100/explain.txt | 1310 ++++++++--------- .../q14a.sf100/simplified.txt | 210 +-- .../approved-plans-v2_7/q14a/explain.txt | 1114 +++++++------- .../approved-plans-v2_7/q14a/simplified.txt | 80 +- 32 files changed, 4554 insertions(+), 5090 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt index 6033ad52be4e9..d4ab3f60494d9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt @@ -37,39 +37,39 @@ TakeOrderedAndProject (134) : : : : +- * HashAggregate (44) : : : : +- Exchange (43) : : : : +- * HashAggregate (42) - : : : : +- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * Project (18) - : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : :- * Filter (11) - : : : : : : +- * ColumnarToRow (10) - : : : : : : +- Scan parquet default.store_sales (9) - : : : : : +- BroadcastExchange (16) - : : : : : +- * Project (15) - : : : : : +- * Filter (14) - : : : : : +- * ColumnarToRow (13) - : : : : : +- Scan parquet default.date_dim (12) - : : : : +- BroadcastExchange (39) - : : : : +- SortMergeJoin LeftSemi (38) - : : : : :- * Sort (23) - : : : : : +- Exchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (37) - : : : : +- Exchange (36) - : : : : +- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (26) - : : : : : : +- * ColumnarToRow (25) - : : : : : : +- Scan parquet default.catalog_sales (24) - : : : : : +- ReusedExchange (27) - : : : : +- BroadcastExchange (33) - : : : : +- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet default.item (30) + : : : : +- SortMergeJoin LeftSemi (41) + : : : : :- * Sort (26) + : : : : : +- Exchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (40) + : : : : +- Exchange (39) + : : : : +- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.item (33) : : : +- * Sort (57) : : : +- Exchange (56) : : : +- * Project (55) @@ -165,10 +165,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 19] +(7) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 19] +(8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) @@ -180,10 +180,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 10] +(10) ColumnarToRow [codegen id : 5] Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -(11) Filter [codegen id : 10] +(11) Filter [codegen id : 5] Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -209,38 +209,51 @@ Input [2]: [d_date_sk#12, d_year#13] Input [1]: [d_date_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(17) BroadcastHashJoin [codegen id : 10] +(17) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#12] Join condition: None -(18) Project [codegen id : 10] +(18) Project [codegen id : 5] Output [1]: [ss_item_sk#1] Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] (19) Scan parquet default.item -Output [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Condition : (((isnotnull(i_item_sk#15) AND isnotnull(i_brand_id#16)) AND isnotnull(i_class_id#17)) AND isnotnull(i_category_id#18)) +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) -(22) Exchange -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: hashpartitioning(coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] +(22) BroadcastExchange +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] -(23) Sort [codegen id : 5] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: [coalesce(i_brand_id#16, 0) ASC NULLS FIRST, isnull(i_brand_id#16) ASC NULLS FIRST, coalesce(i_class_id#17, 0) ASC NULLS FIRST, isnull(i_class_id#17) ASC NULLS FIRST, coalesce(i_category_id#18, 0) ASC NULLS FIRST, isnull(i_category_id#18) ASC NULLS FIRST], false, 0 +(23) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#7] +Join condition: None + +(24) Project [codegen id : 5] +Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] +Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] + +(25) Exchange +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] -(24) Scan parquet default.catalog_sales +(26) Sort [codegen id : 6] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 + +(27) Scan parquet default.catalog_sales Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] @@ -248,228 +261,215 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 8] +(28) ColumnarToRow [codegen id : 9] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -(26) Filter [codegen id : 8] +(29) Filter [codegen id : 9] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] Condition : isnotnull(cs_item_sk#20) -(27) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#22] +(30) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#12] -(28) BroadcastHashJoin [codegen id : 8] +(31) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#12] Join condition: None -(29) Project [codegen id : 8] +(32) Project [codegen id : 9] Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#22] +Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] -(30) Scan parquet default.item -Output [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +(33) Scan parquet default.item +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 7] -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(32) Filter [codegen id : 7] -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -Condition : isnotnull(i_item_sk#23) +(35) Filter [codegen id : 8] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : isnotnull(i_item_sk#7) -(33) BroadcastExchange -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +(36) BroadcastExchange +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(34) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#23] +Right keys [1]: [i_item_sk#7] Join condition: None -(35) Project [codegen id : 8] -Output [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Input [5]: [cs_item_sk#20, i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] - -(36) Exchange -Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: hashpartitioning(coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26), 5), ENSURE_REQUIREMENTS, [id=#28] +(38) Project [codegen id : 9] +Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(37) Sort [codegen id : 9] -Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: [coalesce(i_brand_id#24, 0) ASC NULLS FIRST, isnull(i_brand_id#24) ASC NULLS FIRST, coalesce(i_class_id#25, 0) ASC NULLS FIRST, isnull(i_class_id#25) ASC NULLS FIRST, coalesce(i_category_id#26, 0) ASC NULLS FIRST, isnull(i_category_id#26) ASC NULLS FIRST], false, 0 +(39) Exchange +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] -(38) SortMergeJoin -Left keys [6]: [coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18)] -Right keys [6]: [coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26)] -Join condition: None +(40) Sort [codegen id : 10] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 -(39) BroadcastExchange -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] - -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] +(41) SortMergeJoin +Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] +Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] Join condition: None -(41) Project [codegen id : 10] -Output [3]: [i_brand_id#16 AS brand_id#30, i_class_id#17 AS class_id#31, i_category_id#18 AS category_id#32] -Input [5]: [ss_item_sk#1, i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] - -(42) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +(42) HashAggregate [codegen id : 11] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#16, class_id#17, category_id#18] (43) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] -(44) HashAggregate [codegen id : 11] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +(44) HashAggregate [codegen id : 12] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#16, class_id#17, category_id#18] (45) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32), 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] -(46) Sort [codegen id : 12] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: [coalesce(brand_id#30, 0) ASC NULLS FIRST, isnull(brand_id#30) ASC NULLS FIRST, coalesce(class_id#31, 0) ASC NULLS FIRST, isnull(class_id#31) ASC NULLS FIRST, coalesce(category_id#32, 0) ASC NULLS FIRST, isnull(category_id#32) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 13] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 15] -Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] +(48) ColumnarToRow [codegen id : 16] +Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -(49) Filter [codegen id : 15] -Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#35) +(49) Filter [codegen id : 16] +Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Condition : isnotnull(ws_item_sk#26) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#37] +Output [1]: [d_date_sk#12] -(51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#37] +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None -(52) Project [codegen id : 15] -Output [1]: [ws_item_sk#35] -Input [3]: [ws_item_sk#35, ws_sold_date_sk#36, d_date_sk#37] +(52) Project [codegen id : 16] +Output [1]: [ws_item_sk#26] +Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] -(53) ReusedExchange [Reuses operator id: 33] -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(53) ReusedExchange [Reuses operator id: 36] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(54) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [i_item_sk#38] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [i_item_sk#7] Join condition: None -(55) Project [codegen id : 15] -Output [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Input [5]: [ws_item_sk#35, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(55) Project [codegen id : 16] +Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (56) Exchange -Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: hashpartitioning(coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41), 5), ENSURE_REQUIREMENTS, [id=#42] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] -(57) Sort [codegen id : 16] -Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: [coalesce(i_brand_id#39, 0) ASC NULLS FIRST, isnull(i_brand_id#39) ASC NULLS FIRST, coalesce(i_class_id#40, 0) ASC NULLS FIRST, isnull(i_class_id#40) ASC NULLS FIRST, coalesce(i_category_id#41, 0) ASC NULLS FIRST, isnull(i_category_id#41) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 17] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] -Right keys [6]: [coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41)] +Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] +Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] Join condition: None -(59) HashAggregate [codegen id : 17] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +(59) HashAggregate [codegen id : 18] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#16, class_id#17, category_id#18] (60) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] -(61) HashAggregate [codegen id : 18] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +(61) HashAggregate [codegen id : 19] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#16, class_id#17, category_id#18] (62) BroadcastExchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#44] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] -(63) BroadcastHashJoin [codegen id : 19] +(63) BroadcastHashJoin [codegen id : 20] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#30, class_id#31, category_id#32] +Right keys [3]: [brand_id#16, class_id#17, category_id#18] Join condition: None -(64) Project [codegen id : 19] -Output [1]: [i_item_sk#7 AS ss_item_sk#45] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#30, class_id#31, category_id#32] +(64) Project [codegen id : 20] +Output [1]: [i_item_sk#7 AS ss_item_sk#31] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] (65) Exchange -Input [1]: [ss_item_sk#45] -Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [1]: [ss_item_sk#31] +Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] -(66) Sort [codegen id : 20] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 21] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#31] Join condition: None (68) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#47] +Output [3]: [d_date_sk#12, d_year#13, d_moy#33] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 21] -Input [3]: [d_date_sk#12, d_year#13, d_moy#47] +(69) ColumnarToRow [codegen id : 22] +Input [3]: [d_date_sk#12, d_year#13, d_moy#33] -(70) Filter [codegen id : 21] -Input [3]: [d_date_sk#12, d_year#13, d_moy#47] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#47)) AND (d_year#13 = 2001)) AND (d_moy#47 = 11)) AND isnotnull(d_date_sk#12)) +(70) Filter [codegen id : 22] +Input [3]: [d_date_sk#12, d_year#13, d_moy#33] +Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#33)) AND (d_year#13 = 2001)) AND (d_moy#33 = 11)) AND isnotnull(d_date_sk#12)) -(71) Project [codegen id : 21] +(71) Project [codegen id : 22] Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#47] +Input [3]: [d_date_sk#12, d_year#13, d_moy#33] (72) BroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] -(73) BroadcastHashJoin [codegen id : 42] +(73) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#12] Join condition: None -(74) Project [codegen id : 42] +(74) Project [codegen id : 44] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] @@ -480,273 +480,273 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 22] +(76) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(77) Filter [codegen id : 22] +(77) Filter [codegen id : 23] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : isnotnull(i_item_sk#7) (78) Exchange Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#49] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#35] -(79) Sort [codegen id : 23] +(79) Sort [codegen id : 24] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#31] -(81) Sort [codegen id : 41] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 43] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (82) SortMergeJoin Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#31] Join condition: None (83) BroadcastExchange Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -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=#36] -(84) BroadcastHashJoin [codegen id : 42] +(84) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#7] Join condition: None -(85) Project [codegen id : 42] +(85) Project [codegen id : 44] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(86) HashAggregate [codegen id : 42] +(86) HashAggregate [codegen id : 44] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#51, isEmpty#52, count#53] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] +Aggregate Attributes [3]: [sum#37, isEmpty#38, count#39] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#43] -(88) HashAggregate [codegen id : 43] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] +(88) HashAggregate [codegen id : 45] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44, count(1)#45] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sales#46, count(1)#45 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] -(89) Filter [codegen id : 43] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(89) Filter [codegen id : 45] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48 as decimal(32,6)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(90) Project [codegen id : 43] -Output [6]: [sales#60, number_sales#61, store AS channel#65, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] +(90) Project [codegen id : 45] +Output [6]: [sales#46, number_sales#47, store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] (91) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Output [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 44] -Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +(92) ColumnarToRow [codegen id : 46] +Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -(93) Filter [codegen id : 44] -Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +(93) Filter [codegen id : 46] +Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] Condition : isnotnull(cs_item_sk#20) (94) Exchange -Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] -Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#54] -(95) Sort [codegen id : 45] -Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +(95) Sort [codegen id : 47] +Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] Arguments: [cs_item_sk#20 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#31] -(97) Sort [codegen id : 63] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 66] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (98) SortMergeJoin Left keys [1]: [cs_item_sk#20] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#31] Join condition: None (99) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#69] +Output [1]: [d_date_sk#12] -(100) BroadcastHashJoin [codegen id : 85] +(100) BroadcastHashJoin [codegen id : 89] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#69] +Right keys [1]: [d_date_sk#12] Join condition: None -(101) Project [codegen id : 85] -Output [3]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67] -Input [5]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21, d_date_sk#69] +(101) Project [codegen id : 89] +Output [3]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53] +Input [5]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] (102) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(103) BroadcastHashJoin [codegen id : 85] +(103) BroadcastHashJoin [codegen id : 89] Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#70] +Right keys [1]: [i_item_sk#7] Join condition: None -(104) Project [codegen id : 85] -Output [5]: [cs_quantity#66, cs_list_price#67, i_brand_id#71, i_class_id#72, i_category_id#73] -Input [7]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73] +(104) Project [codegen id : 89] +Output [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [7]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(105) HashAggregate [codegen id : 85] -Input [5]: [cs_quantity#66, cs_list_price#67, i_brand_id#71, i_class_id#72, i_category_id#73] -Keys [3]: [i_brand_id#71, i_class_id#72, i_category_id#73] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#74, isEmpty#75, count#76] -Results [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] +(105) HashAggregate [codegen id : 89] +Input [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] (106) Exchange -Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] -Arguments: hashpartitioning(i_brand_id#71, i_class_id#72, i_category_id#73, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#61] -(107) HashAggregate [codegen id : 86] -Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] -Keys [3]: [i_brand_id#71, i_class_id#72, i_category_id#73] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81, count(1)#82] -Results [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sales#83, count(1)#82 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] +(107) HashAggregate [codegen id : 90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] -(108) Filter [codegen id : 86] -Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(108) Filter [codegen id : 90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(109) Project [codegen id : 86] -Output [6]: [sales#83, number_sales#84, catalog AS channel#86, i_brand_id#71, i_class_id#72, i_category_id#73] -Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] +(109) Project [codegen id : 90] +Output [6]: [sales#64, number_sales#65, catalog AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] (110) Scan parquet default.web_sales -Output [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Output [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 87] -Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +(111) ColumnarToRow [codegen id : 91] +Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -(112) Filter [codegen id : 87] -Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#35) +(112) Filter [codegen id : 91] +Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Condition : isnotnull(ws_item_sk#26) (113) Exchange -Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] -Arguments: hashpartitioning(ws_item_sk#35, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Arguments: hashpartitioning(ws_item_sk#26, 5), ENSURE_REQUIREMENTS, [id=#70] -(114) Sort [codegen id : 88] -Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] -Arguments: [ws_item_sk#35 ASC NULLS FIRST], false, 0 +(114) Sort [codegen id : 92] +Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Arguments: [ws_item_sk#26 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#31] -(116) Sort [codegen id : 106] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(116) Sort [codegen id : 111] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (117) SortMergeJoin -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [ss_item_sk#45] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [ss_item_sk#31] Join condition: None (118) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#90] +Output [1]: [d_date_sk#12] -(119) BroadcastHashJoin [codegen id : 128] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#90] +(119) BroadcastHashJoin [codegen id : 134] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None -(120) Project [codegen id : 128] -Output [3]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88] -Input [5]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36, d_date_sk#90] +(120) Project [codegen id : 134] +Output [3]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69] +Input [5]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] (121) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(122) BroadcastHashJoin [codegen id : 128] -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [i_item_sk#91] +(122) BroadcastHashJoin [codegen id : 134] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [i_item_sk#7] Join condition: None -(123) Project [codegen id : 128] -Output [5]: [ws_quantity#87, ws_list_price#88, i_brand_id#92, i_class_id#93, i_category_id#94] -Input [7]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94] +(123) Project [codegen id : 134] +Output [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [7]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(124) HashAggregate [codegen id : 128] -Input [5]: [ws_quantity#87, ws_list_price#88, i_brand_id#92, i_class_id#93, i_category_id#94] -Keys [3]: [i_brand_id#92, i_class_id#93, i_category_id#94] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#95, isEmpty#96, count#97] -Results [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] +(124) HashAggregate [codegen id : 134] +Input [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#71, isEmpty#72, count#73] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] (125) Exchange -Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] -Arguments: hashpartitioning(i_brand_id#92, i_class_id#93, i_category_id#94, 5), ENSURE_REQUIREMENTS, [id=#101] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#77] -(126) HashAggregate [codegen id : 129] -Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] -Keys [3]: [i_brand_id#92, i_class_id#93, i_category_id#94] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102, count(1)#103] -Results [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sales#104, count(1)#103 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] +(126) HashAggregate [codegen id : 135] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78, count(1)#79] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sales#80, count(1)#79 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] -(127) Filter [codegen id : 129] -Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(127) Filter [codegen id : 135] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(128) Project [codegen id : 129] -Output [6]: [sales#104, number_sales#105, web AS channel#107, i_brand_id#92, i_class_id#93, i_category_id#94] -Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] +(128) Project [codegen id : 135] +Output [6]: [sales#80, number_sales#81, web AS channel#83, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] (129) Union -(130) Expand [codegen id : 130] -Input [6]: [sales#60, number_sales#61, channel#65, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [List(sales#60, number_sales#61, channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, 0), List(sales#60, number_sales#61, channel#65, i_brand_id#8, i_class_id#9, null, 1), List(sales#60, number_sales#61, channel#65, i_brand_id#8, null, null, 3), List(sales#60, number_sales#61, channel#65, null, null, null, 7), List(sales#60, number_sales#61, null, null, null, null, 15)], [sales#60, number_sales#61, channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112] +(130) Expand [codegen id : 136] +Input [6]: [sales#46, number_sales#47, channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [List(sales#46, number_sales#47, channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 0), List(sales#46, number_sales#47, channel#51, i_brand_id#8, i_class_id#9, null, 1), List(sales#46, number_sales#47, channel#51, i_brand_id#8, null, null, 3), List(sales#46, number_sales#47, channel#51, null, null, null, 7), List(sales#46, number_sales#47, null, null, null, null, 15)], [sales#46, number_sales#47, channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] -(131) HashAggregate [codegen id : 130] -Input [7]: [sales#60, number_sales#61, channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112] -Keys [5]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112] -Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] -Aggregate Attributes [3]: [sum#113, isEmpty#114, sum#115] -Results [8]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112, sum#116, isEmpty#117, sum#118] +(131) HashAggregate [codegen id : 136] +Input [7]: [sales#46, number_sales#47, channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] +Keys [5]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] +Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] +Aggregate Attributes [3]: [sum#89, isEmpty#90, sum#91] +Results [8]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, sum#92, isEmpty#93, sum#94] (132) Exchange -Input [8]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112, sum#116, isEmpty#117, sum#118] -Arguments: hashpartitioning(channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112, 5), ENSURE_REQUIREMENTS, [id=#119] +Input [8]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, sum#92, isEmpty#93, sum#94] +Arguments: hashpartitioning(channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, 5), ENSURE_REQUIREMENTS, [id=#95] -(133) HashAggregate [codegen id : 131] -Input [8]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112, sum#116, isEmpty#117, sum#118] -Keys [5]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, spark_grouping_id#112] -Functions [2]: [sum(sales#60), sum(number_sales#61)] -Aggregate Attributes [2]: [sum(sales#60)#120, sum(number_sales#61)#121] -Results [6]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, sum(sales#60)#120 AS sum(sales)#122, sum(number_sales#61)#121 AS sum(number_sales)#123] +(133) HashAggregate [codegen id : 137] +Input [8]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, sum#92, isEmpty#93, sum#94] +Keys [5]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] +Functions [2]: [sum(sales#46), sum(number_sales#47)] +Aggregate Attributes [2]: [sum(sales#46)#96, sum(number_sales#47)#97] +Results [6]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, sum(sales#46)#96 AS sum(sales)#98, sum(number_sales#47)#97 AS sum(number_sales)#99] (134) TakeOrderedAndProject -Input [6]: [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, sum(sales)#122, sum(number_sales)#123] -Arguments: 100, [channel#108 ASC NULLS FIRST, i_brand_id#109 ASC NULLS FIRST, i_class_id#110 ASC NULLS FIRST, i_category_id#111 ASC NULLS FIRST], [channel#108, i_brand_id#109, i_class_id#110, i_category_id#111, sum(sales)#122, sum(number_sales)#123] +Input [6]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, sum(sales)#98, sum(number_sales)#99] +Arguments: 100, [channel#84 ASC NULLS FIRST, i_brand_id#85 ASC NULLS FIRST, i_class_id#86 ASC NULLS FIRST, i_category_id#87 ASC NULLS FIRST], [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, sum(sales)#98, sum(number_sales)#99] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#63, [id=#64] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#49, [id=#50] * HashAggregate (157) +- Exchange (156) +- * HashAggregate (155) @@ -776,7 +776,7 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#124)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#100)] ReadSchema: struct (136) ColumnarToRow [codegen id : 2] @@ -802,7 +802,7 @@ Input [2]: [d_date_sk#12, d_year#13] (141) BroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#125] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#101] (142) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#4] @@ -810,83 +810,83 @@ Right keys [1]: [d_date_sk#12] Join condition: None (143) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#126, ss_list_price#3 AS list_price#127] +Output [2]: [ss_quantity#2 AS quantity#102, ss_list_price#3 AS list_price#103] Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (144) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Output [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#124)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#100)] ReadSchema: struct (145) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Input [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] (146) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#128] +Output [1]: [d_date_sk#12] (147) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#128] +Right keys [1]: [d_date_sk#12] Join condition: None (148) Project [codegen id : 4] -Output [2]: [cs_quantity#66 AS quantity#129, cs_list_price#67 AS list_price#130] -Input [4]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21, d_date_sk#128] +Output [2]: [cs_quantity#52 AS quantity#104, cs_list_price#53 AS list_price#105] +Input [4]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] (149) Scan parquet default.web_sales -Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Output [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#124)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#100)] ReadSchema: struct (150) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Input [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] (151) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#131] +Output [1]: [d_date_sk#12] (152) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#131] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None (153) Project [codegen id : 6] -Output [2]: [ws_quantity#87 AS quantity#132, ws_list_price#88 AS list_price#133] -Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36, d_date_sk#131] +Output [2]: [ws_quantity#68 AS quantity#106, ws_list_price#69 AS list_price#107] +Input [4]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] (154) Union (155) HashAggregate [codegen id : 7] -Input [2]: [quantity#126, list_price#127] +Input [2]: [quantity#102, list_price#103] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#126 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#127 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#134, count#135] -Results [2]: [sum#136, count#137] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#108, count#109] +Results [2]: [sum#110, count#111] (156) Exchange -Input [2]: [sum#136, count#137] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#138] +Input [2]: [sum#110, count#111] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#112] (157) HashAggregate [codegen id : 8] -Input [2]: [sum#136, count#137] +Input [2]: [sum#110, count#111] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#126 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#127 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#126 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#127 as decimal(12,2)))), DecimalType(18,2), true))#139] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#126 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#127 as decimal(12,2)))), DecimalType(18,2), true))#139 AS average_sales#140] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))#113] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))#113 AS average_sales#114] -Subquery:2 Hosting operator id = 135 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#124 +Subquery:2 Hosting operator id = 135 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#100 ReusedExchange (158) (158) ReusedExchange [Reuses operator id: 141] Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 144 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#124 +Subquery:3 Hosting operator id = 144 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#100 -Subquery:4 Hosting operator id = 149 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#124 +Subquery:4 Hosting operator id = 149 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#100 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (159) @@ -902,16 +902,16 @@ ReusedExchange (160) (160) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 -Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt index 2134091c43b82..4ee1ff31e9beb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (131) + WholeStageCodegen (137) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (130) + WholeStageCodegen (136) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] InputAdapter Union - WholeStageCodegen (43) + WholeStageCodegen (45) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #3 @@ -57,7 +57,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (42) + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -76,11 +76,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - WholeStageCodegen (20) + WholeStageCodegen (21) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #5 - WholeStageCodegen (19) + WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -89,82 +89,84 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 - WholeStageCodegen (18) + WholeStageCodegen (19) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (17) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (12) + WholeStageCodegen (13) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (10) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #10 + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #11 - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (9) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #13 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (7) - Filter [i_item_sk] + Exchange [brand_id,class_id,category_id] #10 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #11 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (16) + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #11 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (8) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (17) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (15) + WholeStageCodegen (16) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -175,12 +177,12 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #10 + ReusedExchange [d_date_sk] #11 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 InputAdapter BroadcastExchange #4 - WholeStageCodegen (21) + WholeStageCodegen (22) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow @@ -189,27 +191,27 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter BroadcastExchange #16 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (23) + WholeStageCodegen (24) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #17 - WholeStageCodegen (22) + WholeStageCodegen (23) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (41) + WholeStageCodegen (43) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 - WholeStageCodegen (86) + WholeStageCodegen (90) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (85) + WholeStageCodegen (89) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -217,17 +219,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [cs_item_sk,ss_item_sk] - WholeStageCodegen (45) + WholeStageCodegen (47) Sort [cs_item_sk] InputAdapter Exchange [cs_item_sk] #22 - WholeStageCodegen (44) + WholeStageCodegen (46) Filter [cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (63) + WholeStageCodegen (66) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 @@ -235,14 +237,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [d_date_sk] #4 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (129) + WholeStageCodegen (135) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #23 - WholeStageCodegen (128) + WholeStageCodegen (134) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] @@ -250,17 +252,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ws_item_sk,ss_item_sk] - WholeStageCodegen (88) + WholeStageCodegen (92) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #24 - WholeStageCodegen (87) + WholeStageCodegen (91) Filter [ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (106) + WholeStageCodegen (111) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 34aa615cf910c..85a3a63413f92 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -31,36 +31,36 @@ TakeOrderedAndProject (115) : : : :- * HashAggregate (39) : : : : +- Exchange (38) : : : : +- * HashAggregate (37) - : : : : +- * Project (36) - : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : :- * Project (33) - : : : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet default.store_sales (7) - : : : : : +- BroadcastExchange (31) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Project (28) - : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet default.item (16) - : : : : : +- BroadcastExchange (26) - : : : : : +- * Project (25) - : : : : : +- * Filter (24) - : : : : : +- * ColumnarToRow (23) - : : : : : +- Scan parquet default.date_dim (22) - : : : : +- ReusedExchange (34) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : :- * Project (22) + : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : :- * Project (15) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (13) + : : : : : : +- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (20) + : : : : : +- * Project (19) + : : : : : +- * Filter (18) + : : : : : +- * ColumnarToRow (17) + : : : : : +- Scan parquet default.date_dim (16) + : : : : +- BroadcastExchange (35) + : : : : +- * Project (34) + : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * Project (31) + : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : :- * Filter (25) + : : : : : : +- * ColumnarToRow (24) + : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Filter (28) + : : : : : +- * ColumnarToRow (27) + : : : : : +- Scan parquet default.item (26) + : : : : +- ReusedExchange (32) : : : +- BroadcastExchange (49) : : : +- * Project (48) : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -161,226 +161,226 @@ Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) (10) Scan parquet default.item -Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +(11) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(12) Filter [codegen id : 4] -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14)) +(12) Filter [codegen id : 1] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(13) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +(13) BroadcastExchange +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] +(14) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join condition: None -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#15) +(15) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(16) Scan parquet default.item -Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +(16) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_year#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +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 + +(17) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#12, d_year#13] -(17) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +(18) Filter [codegen id : 2] +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) -(18) Filter [codegen id : 1] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Condition : isnotnull(i_item_sk#17) +(19) Project [codegen id : 2] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] -(19) BroadcastExchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +(20) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#15] -Right keys [1]: [i_item_sk#17] +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20] -Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +(22) Project [codegen id : 6] +Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] +Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] -(22) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_year#23] +(23) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] 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 - -(23) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#22, d_year#23] +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(24) Filter [codegen id : 2] -Input [2]: [d_date_sk#22, d_year#23] -Condition : (((isnotnull(d_year#23) AND (d_year#23 >= 1999)) AND (d_year#23 <= 2001)) AND isnotnull(d_date_sk#22)) +(24) ColumnarToRow [codegen id : 5] +Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -(25) Project [codegen id : 2] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_year#23] +(25) Filter [codegen id : 5] +Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +Condition : isnotnull(cs_item_sk#18) -(26) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +(26) Scan parquet default.item +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(27) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#22] -Join condition: None +(27) ColumnarToRow [codegen id : 3] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(28) Project [codegen id : 3] -Output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] -Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#22] +(28) Filter [codegen id : 3] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : isnotnull(i_item_sk#6) (29) BroadcastExchange -Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] -(30) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)] -Right keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +(30) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#6] Join condition: None -(31) BroadcastExchange -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +(31) Project [codegen id : 5] +Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#11] +(32) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] + +(33) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None -(33) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +(34) Project [codegen id : 5] +Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] -(34) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#27] +(35) BroadcastExchange +Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] -(35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +(36) BroadcastHashJoin [codegen id : 6] +Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] +Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None -(36) Project [codegen id : 6] -Output [3]: [i_brand_id#12 AS brand_id#28, i_class_id#13 AS class_id#29, i_category_id#14 AS category_id#30] -Input [5]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#27] - (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#15, class_id#16, category_id#17] (38) Exchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#15, class_id#16, category_id#17] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_item_sk#32) +Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Condition : isnotnull(ws_item_sk#23) -(43) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +(43) ReusedExchange [Reuses operator id: 29] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#34] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [i_item_sk#6] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37] -Input [6]: [ws_item_sk#32, ws_sold_date_sk#33, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(46) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#38] +(46) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] -Input [5]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] +Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (49) BroadcastExchange -Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#39] +Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] -Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] +Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] +Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#15, class_id#16, category_id#17] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#15, class_id#16, category_id#17] (53) BroadcastExchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#40] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#28, class_id#29, category_id#30] +Right keys [3]: [brand_id#15, class_id#16, category_id#17] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#41] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] +Output [1]: [i_item_sk#6 AS ss_item_sk#27] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] (56) BroadcastExchange -Input [1]: [ss_item_sk#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [ss_item_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#41] +Right keys [1]: [ss_item_sk#27] Join condition: None (58) Scan parquet default.item @@ -398,16 +398,16 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : isnotnull(i_item_sk#6) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#27] (62) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#41] +Right keys [1]: [ss_item_sk#27] Join condition: None (63) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] @@ -419,239 +419,239 @@ Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_ Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (66) Scan parquet default.date_dim -Output [3]: [d_date_sk#27, d_year#44, d_moy#45] +Output [3]: [d_date_sk#12, d_year#13, d_moy#30] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [3]: [d_date_sk#27, d_year#44, d_moy#45] +Input [3]: [d_date_sk#12, d_year#13, d_moy#30] (68) Filter [codegen id : 24] -Input [3]: [d_date_sk#27, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#27)) +Input [3]: [d_date_sk#12, d_year#13, d_moy#30] +Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#30)) AND (d_year#13 = 2001)) AND (d_moy#30 = 11)) AND isnotnull(d_date_sk#12)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#27] -Input [3]: [d_date_sk#27, d_year#44, d_moy#45] +Output [1]: [d_date_sk#12] +Input [3]: [d_date_sk#12, d_year#13, d_moy#30] (70) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None (72) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#27] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (73) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#47, isEmpty#48, count#49] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] +Aggregate Attributes [3]: [sum#32, isEmpty#33, count#34] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#38] (75) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] (76) Filter [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [sales#56, number_sales#57, store AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] +Output [6]: [sales#41, number_sales#42, store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] (78) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Output [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] (80) Filter [codegen id : 51] -Input [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#15) +Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Condition : isnotnull(cs_item_sk#18) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#27] (82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#15] -Right keys [1]: [ss_item_sk#41] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [ss_item_sk#27] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#15] -Right keys [1]: [i_item_sk#64] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#6] Join condition: None (85) Project [codegen id : 51] -Output [6]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [8]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +Output [6]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [8]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#68] +Output [1]: [d_date_sk#12] (87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#68] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None (88) Project [codegen id : 51] -Output [5]: [cs_quantity#62, cs_list_price#63, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [7]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_brand_id#65, i_class_id#66, i_category_id#67, d_date_sk#68] +Output [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#62, cs_list_price#63, i_brand_id#65, i_class_id#66, i_category_id#67] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] -Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] +Input [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#49, isEmpty#50, count#51] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] (90) Exchange -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Arguments: hashpartitioning(i_brand_id#65, i_class_id#66, i_category_id#67, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#55] (91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76, count(1)#77] -Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sales#78, count(1)#77 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56, count(1)#57] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sales#58, count(1)#57 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] (92) Filter [codegen id : 52] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [sales#78, number_sales#79, catalog AS channel#81, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] +Output [6]: [sales#58, number_sales#59, catalog AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] (94) Scan parquet default.web_sales -Output [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Output [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] (96) Filter [codegen id : 77] -Input [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] -Condition : isnotnull(ws_item_sk#32) +Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Condition : isnotnull(ws_item_sk#23) (97) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#27] (98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [ss_item_sk#41] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [ss_item_sk#27] Join condition: None (99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#84] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [i_item_sk#6] Join condition: None (101) Project [codegen id : 77] -Output [6]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_brand_id#85, i_class_id#86, i_category_id#87] -Input [8]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] +Output [6]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [8]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#88] +Output [1]: [d_date_sk#12] (103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#88] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (104) Project [codegen id : 77] -Output [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#85, i_class_id#86, i_category_id#87] -Input [7]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_brand_id#85, i_class_id#86, i_category_id#87, d_date_sk#88] +Output [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#85, i_class_id#86, i_category_id#87] -Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#89, isEmpty#90, count#91] -Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Input [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] (106) Exchange -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] -Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, 5), ENSURE_REQUIREMENTS, [id=#95] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#70] (107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] -Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96, count(1)#97] -Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sales#98, count(1)#97 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71, count(1)#72] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sales#73, count(1)#72 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] (108) Filter [codegen id : 78] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [sales#98, number_sales#99, web AS channel#101, i_brand_id#85, i_class_id#86, i_category_id#87] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] +Output [6]: [sales#73, number_sales#74, web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] (110) Union (111) Expand [codegen id : 79] -Input [6]: [sales#56, number_sales#57, channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: [List(sales#56, number_sales#57, channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#56, number_sales#57, channel#61, i_brand_id#7, i_class_id#8, null, 1), List(sales#56, number_sales#57, channel#61, i_brand_id#7, null, null, 3), List(sales#56, number_sales#57, channel#61, null, null, null, 7), List(sales#56, number_sales#57, null, null, null, null, 15)], [sales#56, number_sales#57, channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] +Input [6]: [sales#41, number_sales#42, channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: [List(sales#41, number_sales#42, channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#41, number_sales#42, channel#46, i_brand_id#7, i_class_id#8, null, 1), List(sales#41, number_sales#42, channel#46, i_brand_id#7, null, null, 3), List(sales#41, number_sales#42, channel#46, null, null, null, 7), List(sales#41, number_sales#42, null, null, null, null, 15)], [sales#41, number_sales#42, channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] (112) HashAggregate [codegen id : 79] -Input [7]: [sales#56, number_sales#57, channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] -Keys [5]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] -Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] -Aggregate Attributes [3]: [sum#107, isEmpty#108, sum#109] -Results [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] +Input [7]: [sales#41, number_sales#42, channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] +Keys [5]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] +Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] +Aggregate Attributes [3]: [sum#82, isEmpty#83, sum#84] +Results [8]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, sum#85, isEmpty#86, sum#87] (113) Exchange -Input [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] -Arguments: hashpartitioning(channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] +Input [8]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, sum#85, isEmpty#86, sum#87] +Arguments: hashpartitioning(channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, 5), ENSURE_REQUIREMENTS, [id=#88] (114) HashAggregate [codegen id : 80] -Input [8]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106, sum#110, isEmpty#111, sum#112] -Keys [5]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, spark_grouping_id#106] -Functions [2]: [sum(sales#56), sum(number_sales#57)] -Aggregate Attributes [2]: [sum(sales#56)#114, sum(number_sales#57)#115] -Results [6]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales#56)#114 AS sum(sales)#116, sum(number_sales#57)#115 AS sum(number_sales)#117] +Input [8]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, sum#85, isEmpty#86, sum#87] +Keys [5]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] +Functions [2]: [sum(sales#41), sum(number_sales#42)] +Aggregate Attributes [2]: [sum(sales#41)#89, sum(number_sales#42)#90] +Results [6]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, sum(sales#41)#89 AS sum(sales)#91, sum(number_sales#42)#90 AS sum(number_sales)#92] (115) TakeOrderedAndProject -Input [6]: [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales)#116, sum(number_sales)#117] -Arguments: 100, [channel#102 ASC NULLS FIRST, i_brand_id#103 ASC NULLS FIRST, i_class_id#104 ASC NULLS FIRST, i_category_id#105 ASC NULLS FIRST], [channel#102, i_brand_id#103, i_class_id#104, i_category_id#105, sum(sales)#116, sum(number_sales)#117] +Input [6]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, sum(sales)#91, sum(number_sales)#92] +Arguments: 100, [channel#77 ASC NULLS FIRST, i_brand_id#78 ASC NULLS FIRST, i_class_id#79 ASC NULLS FIRST, i_category_id#80 ASC NULLS FIRST], [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, sum(sales)#91, sum(number_sales)#92] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#59, [id=#60] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] * HashAggregate (138) +- Exchange (137) +- * HashAggregate (136) @@ -681,142 +681,142 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#118)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#93)] ReadSchema: struct (117) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (118) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_year#44] +Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (119) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#44] +Input [2]: [d_date_sk#12, d_year#13] (120) Filter [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#44] -Condition : (((isnotnull(d_year#44) AND (d_year#44 >= 1999)) AND (d_year#44 <= 2001)) AND isnotnull(d_date_sk#27)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (121) Project [codegen id : 1] -Output [1]: [d_date_sk#27] -Input [2]: [d_date_sk#27, d_year#44] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (122) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#119] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#94] (123) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None (124) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#120, ss_list_price#3 AS list_price#121] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#27] +Output [2]: [ss_quantity#2 AS quantity#95, ss_list_price#3 AS list_price#96] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (125) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Output [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#118)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#93)] ReadSchema: struct (126) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Input [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] (127) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#122] +Output [1]: [d_date_sk#12] (128) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#122] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None (129) Project [codegen id : 4] -Output [2]: [cs_quantity#62 AS quantity#123, cs_list_price#63 AS list_price#124] -Input [4]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, d_date_sk#122] +Output [2]: [cs_quantity#47 AS quantity#97, cs_list_price#48 AS list_price#98] +Input [4]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, d_date_sk#12] (130) Scan parquet default.web_sales -Output [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Output [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#118)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#93)] ReadSchema: struct (131) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Input [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] (132) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#125] +Output [1]: [d_date_sk#12] (133) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#125] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (134) Project [codegen id : 6] -Output [2]: [ws_quantity#82 AS quantity#126, ws_list_price#83 AS list_price#127] -Input [4]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, d_date_sk#125] +Output [2]: [ws_quantity#62 AS quantity#99, ws_list_price#63 AS list_price#100] +Input [4]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, d_date_sk#12] (135) Union (136) HashAggregate [codegen id : 7] -Input [2]: [quantity#120, list_price#121] +Input [2]: [quantity#95, list_price#96] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#120 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#121 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#128, count#129] -Results [2]: [sum#130, count#131] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#101, count#102] +Results [2]: [sum#103, count#104] (137) Exchange -Input [2]: [sum#130, count#131] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#132] +Input [2]: [sum#103, count#104] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] (138) HashAggregate [codegen id : 8] -Input [2]: [sum#130, count#131] +Input [2]: [sum#103, count#104] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#120 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#121 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#120 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#121 as decimal(12,2)))), DecimalType(18,2), true))#133] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#120 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#121 as decimal(12,2)))), DecimalType(18,2), true))#133 AS average_sales#134] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))#106] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))#106 AS average_sales#107] -Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#118 +Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#93 ReusedExchange (139) (139) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#118 +Subquery:3 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#93 -Subquery:4 Hosting operator id = 130 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#118 +Subquery:4 Hosting operator id = 130 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#93 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#12] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 ReusedExchange (141) -(141) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#27] +(141) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 -Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index 9ae1a0e30e90e..f9b76560e363c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -90,53 +90,53 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Exchange [brand_id,class_id,category_id] #6 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (1) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #9 + BroadcastExchange #10 WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (1) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Filter [i_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #7 + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #11 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt index 013c3339f6ec3..76f5ac48f96cc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt @@ -33,39 +33,39 @@ TakeOrderedAndProject (116) : : : : +- * HashAggregate (44) : : : : +- Exchange (43) : : : : +- * HashAggregate (42) - : : : : +- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * Project (18) - : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : :- * Filter (11) - : : : : : : +- * ColumnarToRow (10) - : : : : : : +- Scan parquet default.store_sales (9) - : : : : : +- BroadcastExchange (16) - : : : : : +- * Project (15) - : : : : : +- * Filter (14) - : : : : : +- * ColumnarToRow (13) - : : : : : +- Scan parquet default.date_dim (12) - : : : : +- BroadcastExchange (39) - : : : : +- SortMergeJoin LeftSemi (38) - : : : : :- * Sort (23) - : : : : : +- Exchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (37) - : : : : +- Exchange (36) - : : : : +- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (26) - : : : : : : +- * ColumnarToRow (25) - : : : : : : +- Scan parquet default.catalog_sales (24) - : : : : : +- ReusedExchange (27) - : : : : +- BroadcastExchange (33) - : : : : +- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet default.item (30) + : : : : +- SortMergeJoin LeftSemi (41) + : : : : :- * Sort (26) + : : : : : +- Exchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (40) + : : : : +- Exchange (39) + : : : : +- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.item (33) : : : +- * Sort (57) : : : +- Exchange (56) : : : +- * Project (55) @@ -147,10 +147,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 19] +(7) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 19] +(8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) @@ -162,10 +162,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 10] +(10) ColumnarToRow [codegen id : 5] Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -(11) Filter [codegen id : 10] +(11) Filter [codegen id : 5] Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -191,38 +191,51 @@ Input [2]: [d_date_sk#12, d_year#13] Input [1]: [d_date_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(17) BroadcastHashJoin [codegen id : 10] +(17) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#12] Join condition: None -(18) Project [codegen id : 10] +(18) Project [codegen id : 5] Output [1]: [ss_item_sk#1] Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] (19) Scan parquet default.item -Output [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Condition : (((isnotnull(i_item_sk#15) AND isnotnull(i_brand_id#16)) AND isnotnull(i_class_id#17)) AND isnotnull(i_category_id#18)) +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) -(22) Exchange -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: hashpartitioning(coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] +(22) BroadcastExchange +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] -(23) Sort [codegen id : 5] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: [coalesce(i_brand_id#16, 0) ASC NULLS FIRST, isnull(i_brand_id#16) ASC NULLS FIRST, coalesce(i_class_id#17, 0) ASC NULLS FIRST, isnull(i_class_id#17) ASC NULLS FIRST, coalesce(i_category_id#18, 0) ASC NULLS FIRST, isnull(i_category_id#18) ASC NULLS FIRST], false, 0 +(23) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#7] +Join condition: None + +(24) Project [codegen id : 5] +Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] +Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] + +(25) Exchange +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] -(24) Scan parquet default.catalog_sales +(26) Sort [codegen id : 6] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 + +(27) Scan parquet default.catalog_sales Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] @@ -230,228 +243,215 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 8] +(28) ColumnarToRow [codegen id : 9] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -(26) Filter [codegen id : 8] +(29) Filter [codegen id : 9] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] Condition : isnotnull(cs_item_sk#20) -(27) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#22] +(30) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#12] -(28) BroadcastHashJoin [codegen id : 8] +(31) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#12] Join condition: None -(29) Project [codegen id : 8] +(32) Project [codegen id : 9] Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#22] +Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] -(30) Scan parquet default.item -Output [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +(33) Scan parquet default.item +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 7] -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(32) Filter [codegen id : 7] -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -Condition : isnotnull(i_item_sk#23) +(35) Filter [codegen id : 8] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : isnotnull(i_item_sk#7) -(33) BroadcastExchange -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +(36) BroadcastExchange +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(34) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#23] +Right keys [1]: [i_item_sk#7] Join condition: None -(35) Project [codegen id : 8] -Output [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Input [5]: [cs_item_sk#20, i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +(38) Project [codegen id : 9] +Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(36) Exchange -Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: hashpartitioning(coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26), 5), ENSURE_REQUIREMENTS, [id=#28] +(39) Exchange +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] -(37) Sort [codegen id : 9] -Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: [coalesce(i_brand_id#24, 0) ASC NULLS FIRST, isnull(i_brand_id#24) ASC NULLS FIRST, coalesce(i_class_id#25, 0) ASC NULLS FIRST, isnull(i_class_id#25) ASC NULLS FIRST, coalesce(i_category_id#26, 0) ASC NULLS FIRST, isnull(i_category_id#26) ASC NULLS FIRST], false, 0 +(40) Sort [codegen id : 10] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 -(38) SortMergeJoin -Left keys [6]: [coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18)] -Right keys [6]: [coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26)] +(41) SortMergeJoin +Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] +Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] Join condition: None -(39) BroadcastExchange -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] - -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join condition: None - -(41) Project [codegen id : 10] -Output [3]: [i_brand_id#16 AS brand_id#30, i_class_id#17 AS class_id#31, i_category_id#18 AS category_id#32] -Input [5]: [ss_item_sk#1, i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] - -(42) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +(42) HashAggregate [codegen id : 11] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#16, class_id#17, category_id#18] (43) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] -(44) HashAggregate [codegen id : 11] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +(44) HashAggregate [codegen id : 12] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#16, class_id#17, category_id#18] (45) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32), 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] -(46) Sort [codegen id : 12] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: [coalesce(brand_id#30, 0) ASC NULLS FIRST, isnull(brand_id#30) ASC NULLS FIRST, coalesce(class_id#31, 0) ASC NULLS FIRST, isnull(class_id#31) ASC NULLS FIRST, coalesce(category_id#32, 0) ASC NULLS FIRST, isnull(category_id#32) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 13] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 15] -Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] +(48) ColumnarToRow [codegen id : 16] +Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -(49) Filter [codegen id : 15] -Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#35) +(49) Filter [codegen id : 16] +Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Condition : isnotnull(ws_item_sk#26) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#37] +Output [1]: [d_date_sk#12] -(51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#37] +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None -(52) Project [codegen id : 15] -Output [1]: [ws_item_sk#35] -Input [3]: [ws_item_sk#35, ws_sold_date_sk#36, d_date_sk#37] +(52) Project [codegen id : 16] +Output [1]: [ws_item_sk#26] +Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] -(53) ReusedExchange [Reuses operator id: 33] -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(53) ReusedExchange [Reuses operator id: 36] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(54) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [i_item_sk#38] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [i_item_sk#7] Join condition: None -(55) Project [codegen id : 15] -Output [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Input [5]: [ws_item_sk#35, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(55) Project [codegen id : 16] +Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (56) Exchange -Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: hashpartitioning(coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41), 5), ENSURE_REQUIREMENTS, [id=#42] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] -(57) Sort [codegen id : 16] -Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: [coalesce(i_brand_id#39, 0) ASC NULLS FIRST, isnull(i_brand_id#39) ASC NULLS FIRST, coalesce(i_class_id#40, 0) ASC NULLS FIRST, isnull(i_class_id#40) ASC NULLS FIRST, coalesce(i_category_id#41, 0) ASC NULLS FIRST, isnull(i_category_id#41) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 17] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] -Right keys [6]: [coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41)] +Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] +Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] Join condition: None -(59) HashAggregate [codegen id : 17] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +(59) HashAggregate [codegen id : 18] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#16, class_id#17, category_id#18] (60) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] -(61) HashAggregate [codegen id : 18] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +(61) HashAggregate [codegen id : 19] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#16, class_id#17, category_id#18] (62) BroadcastExchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#44] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] -(63) BroadcastHashJoin [codegen id : 19] +(63) BroadcastHashJoin [codegen id : 20] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#30, class_id#31, category_id#32] +Right keys [3]: [brand_id#16, class_id#17, category_id#18] Join condition: None -(64) Project [codegen id : 19] -Output [1]: [i_item_sk#7 AS ss_item_sk#45] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#30, class_id#31, category_id#32] +(64) Project [codegen id : 20] +Output [1]: [i_item_sk#7 AS ss_item_sk#31] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] (65) Exchange -Input [1]: [ss_item_sk#45] -Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [1]: [ss_item_sk#31] +Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] -(66) Sort [codegen id : 20] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 21] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#31] Join condition: None (68) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#47] +Output [2]: [d_date_sk#12, d_week_seq#33] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 21] -Input [2]: [d_date_sk#12, d_week_seq#47] +(69) ColumnarToRow [codegen id : 22] +Input [2]: [d_date_sk#12, d_week_seq#33] -(70) Filter [codegen id : 21] -Input [2]: [d_date_sk#12, d_week_seq#47] -Condition : ((isnotnull(d_week_seq#47) AND (d_week_seq#47 = Subquery scalar-subquery#48, [id=#49])) AND isnotnull(d_date_sk#12)) +(70) Filter [codegen id : 22] +Input [2]: [d_date_sk#12, d_week_seq#33] +Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#34, [id=#35])) AND isnotnull(d_date_sk#12)) -(71) Project [codegen id : 21] +(71) Project [codegen id : 22] Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#47] +Input [2]: [d_date_sk#12, d_week_seq#33] (72) BroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] -(73) BroadcastHashJoin [codegen id : 42] +(73) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#12] Join condition: None -(74) Project [codegen id : 42] +(74) Project [codegen id : 44] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] @@ -462,192 +462,192 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 22] +(76) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(77) Filter [codegen id : 22] +(77) Filter [codegen id : 23] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (78) Exchange Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#51] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#37] -(79) Sort [codegen id : 23] +(79) Sort [codegen id : 24] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#31] -(81) Sort [codegen id : 41] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 43] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (82) SortMergeJoin Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#31] Join condition: None (83) BroadcastExchange Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -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=#38] -(84) BroadcastHashJoin [codegen id : 42] +(84) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#7] Join condition: None -(85) Project [codegen id : 42] +(85) Project [codegen id : 44] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(86) HashAggregate [codegen id : 42] +(86) HashAggregate [codegen id : 44] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#53, isEmpty#54, count#55] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] +Aggregate Attributes [3]: [sum#39, isEmpty#40, count#41] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#45] -(88) HashAggregate [codegen id : 86] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] +(88) HashAggregate [codegen id : 90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60, count(1)#61] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sales#62, count(1)#61 AS number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46, count(1)#47] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sales#48, count(1)#47 AS number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] -(89) Filter [codegen id : 86] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 as decimal(32,6)) > cast(Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) +(89) Filter [codegen id : 90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(90) Project [codegen id : 86] -Output [6]: [store AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] +(90) Project [codegen id : 90] +Output [6]: [store AS channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] (91) Scan parquet default.store_sales -Output [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#71), dynamicpruningexpression(ss_sold_date_sk#71 IN dynamicpruning#72)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 43] -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +(92) ColumnarToRow [codegen id : 45] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(93) Filter [codegen id : 43] -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] -Condition : isnotnull(ss_item_sk#68) +(93) Filter [codegen id : 45] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (94) Exchange -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] -Arguments: hashpartitioning(ss_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#55] -(95) Sort [codegen id : 44] -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] -Arguments: [ss_item_sk#68 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 46] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#31] -(97) Sort [codegen id : 62] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 65] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ss_item_sk#68] -Right keys [1]: [ss_item_sk#45] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [ss_item_sk#31] Join condition: None (99) Scan parquet default.date_dim -Output [2]: [d_date_sk#74, d_week_seq#75] +Output [2]: [d_date_sk#12, d_week_seq#33] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 63] -Input [2]: [d_date_sk#74, d_week_seq#75] +(100) ColumnarToRow [codegen id : 66] +Input [2]: [d_date_sk#12, d_week_seq#33] -(101) Filter [codegen id : 63] -Input [2]: [d_date_sk#74, d_week_seq#75] -Condition : ((isnotnull(d_week_seq#75) AND (d_week_seq#75 = Subquery scalar-subquery#76, [id=#77])) AND isnotnull(d_date_sk#74)) +(101) Filter [codegen id : 66] +Input [2]: [d_date_sk#12, d_week_seq#33] +Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#56, [id=#57])) AND isnotnull(d_date_sk#12)) -(102) Project [codegen id : 63] -Output [1]: [d_date_sk#74] -Input [2]: [d_date_sk#74, d_week_seq#75] +(102) Project [codegen id : 66] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_week_seq#33] (103) BroadcastExchange -Input [1]: [d_date_sk#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] -(104) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_sold_date_sk#71] -Right keys [1]: [d_date_sk#74] +(104) BroadcastHashJoin [codegen id : 88] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None -(105) Project [codegen id : 84] -Output [3]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70] -Input [5]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71, d_date_sk#74] +(105) Project [codegen id : 88] +Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (106) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] +Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(107) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#68] -Right keys [1]: [i_item_sk#79] +(107) BroadcastHashJoin [codegen id : 88] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#59] Join condition: None -(108) Project [codegen id : 84] -Output [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#80, i_class_id#81, i_category_id#82] -Input [7]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] +(108) Project [codegen id : 88] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(109) HashAggregate [codegen id : 84] -Input [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#80, i_class_id#81, i_category_id#82] -Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] -Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] +(109) HashAggregate [codegen id : 88] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] +Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65] +Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] (110) Exchange -Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] -Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] +Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [id=#69] -(111) HashAggregate [codegen id : 85] -Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] -Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90, count(1)#91] -Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sales#92, count(1)#91 AS number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] +(111) HashAggregate [codegen id : 89] +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] +Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70, count(1)#71] +Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sales#72, count(1)#71 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] -(112) Filter [codegen id : 85] -Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) +(112) Filter [codegen id : 89] +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(113) Project [codegen id : 85] -Output [6]: [store AS channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] -Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] +(113) Project [codegen id : 89] +Output [6]: [store AS channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] (114) BroadcastExchange -Input [6]: [channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#96] +Input [6]: [channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#76] -(115) BroadcastHashJoin [codegen id : 86] +(115) BroadcastHashJoin [codegen id : 90] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] +Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Join condition: None (116) TakeOrderedAndProject -Input [12]: [channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] -Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Input [12]: [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#65, [id=#66] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#51, [id=#52] * HashAggregate (139) +- Exchange (138) +- * HashAggregate (137) @@ -677,7 +677,7 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#97)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#77)] ReadSchema: struct (118) ColumnarToRow [codegen id : 2] @@ -703,7 +703,7 @@ Input [2]: [d_date_sk#12, d_year#13] (123) BroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] (124) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#4] @@ -711,83 +711,83 @@ Right keys [1]: [d_date_sk#12] Join condition: None (125) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#99, ss_list_price#3 AS list_price#100] +Output [2]: [ss_quantity#2 AS quantity#79, ss_list_price#3 AS list_price#80] Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (126) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21] +Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#97)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#77)] ReadSchema: struct (127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21] +Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] (128) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#103] +Output [1]: [d_date_sk#12] (129) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#103] +Right keys [1]: [d_date_sk#12] Join condition: None (130) Project [codegen id : 4] -Output [2]: [cs_quantity#101 AS quantity#104, cs_list_price#102 AS list_price#105] -Input [4]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21, d_date_sk#103] +Output [2]: [cs_quantity#81 AS quantity#83, cs_list_price#82 AS list_price#84] +Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21, d_date_sk#12] (131) Scan parquet default.web_sales -Output [3]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36] +Output [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#97)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#77)] ReadSchema: struct (132) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36] +Input [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] (133) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#108] +Output [1]: [d_date_sk#12] (134) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#108] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None (135) Project [codegen id : 6] -Output [2]: [ws_quantity#106 AS quantity#109, ws_list_price#107 AS list_price#110] -Input [4]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36, d_date_sk#108] +Output [2]: [ws_quantity#85 AS quantity#87, ws_list_price#86 AS list_price#88] +Input [4]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27, d_date_sk#12] (136) Union (137) HashAggregate [codegen id : 7] -Input [2]: [quantity#99, list_price#100] +Input [2]: [quantity#79, list_price#80] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#111, count#112] -Results [2]: [sum#113, count#114] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#89, count#90] +Results [2]: [sum#91, count#92] (138) Exchange -Input [2]: [sum#113, count#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] +Input [2]: [sum#91, count#92] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] (139) HashAggregate [codegen id : 8] -Input [2]: [sum#113, count#114] +Input [2]: [sum#91, count#92] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#116] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#116 AS average_sales#117] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94 AS average_sales#95] -Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#97 +Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#77 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 123] Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#97 +Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#77 -Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#97 +Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#77 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (141) @@ -803,11 +803,11 @@ ReusedExchange (142) (142) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 -Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#34, [id=#35] * Project (146) +- * Filter (145) +- * ColumnarToRow (144) @@ -815,33 +815,33 @@ Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquer (143) Scan parquet default.date_dim -Output [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] (145) Filter [codegen id : 1] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#13 = 2000)) AND (d_moy#118 = 12)) AND (d_dom#119 = 11)) +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 2000)) AND (d_moy#96 = 12)) AND (d_dom#97 = 11)) (146) Project [codegen id : 1] -Output [1]: [d_week_seq#47] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Output [1]: [d_week_seq#33] +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] -Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#65, [id=#66] +Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] -Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#71 IN dynamicpruning#72 +Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#54 ReusedExchange (147) (147) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#74] +Output [1]: [d_date_sk#12] -Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#76, [id=#77] +Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#56, [id=#57] * Project (151) +- * Filter (150) +- * ColumnarToRow (149) @@ -849,21 +849,21 @@ Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subqu (148) Scan parquet default.date_dim -Output [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (149) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] (150) Filter [codegen id : 1] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#13 = 1999)) AND (d_moy#118 = 12)) AND (d_dom#119 = 11)) +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 1999)) AND (d_moy#96 = 12)) AND (d_dom#97 = 11)) (151) Project [codegen id : 1] -Output [1]: [d_week_seq#47] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Output [1]: [d_week_seq#33] +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt index 1b41abbc97b06..1d5907e319658 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (86) + WholeStageCodegen (90) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] @@ -49,7 +49,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (42) + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -68,11 +68,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #3 - WholeStageCodegen (20) + WholeStageCodegen (21) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 - WholeStageCodegen (19) + WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -81,82 +81,84 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (18) + WholeStageCodegen (19) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (17) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (12) + WholeStageCodegen (13) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (10) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #9 + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #10 - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (9) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (7) - Filter [i_item_sk] + Exchange [brand_id,class_id,category_id] #9 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #10 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (16) + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #10 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (8) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (17) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (15) + WholeStageCodegen (16) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -167,12 +169,12 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #9 + ReusedExchange [d_date_sk] #10 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter BroadcastExchange #3 - WholeStageCodegen (21) + WholeStageCodegen (22) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #3 @@ -188,29 +190,29 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #15 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (23) + WholeStageCodegen (24) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #16 - WholeStageCodegen (22) + WholeStageCodegen (23) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (41) + WholeStageCodegen (43) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #20 - WholeStageCodegen (85) + WholeStageCodegen (89) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (84) + WholeStageCodegen (88) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -218,24 +220,24 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] - WholeStageCodegen (44) + WholeStageCodegen (46) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #22 - WholeStageCodegen (43) + WholeStageCodegen (45) Filter [ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #6 ReusedExchange [d_date_sk] #23 - WholeStageCodegen (62) + WholeStageCodegen (65) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #23 - WholeStageCodegen (63) + WholeStageCodegen (66) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 83604fd5cdff8..01bcb83622844 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -27,36 +27,36 @@ TakeOrderedAndProject (100) : : : :- * HashAggregate (39) : : : : +- Exchange (38) : : : : +- * HashAggregate (37) - : : : : +- * Project (36) - : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : :- * Project (33) - : : : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet default.store_sales (7) - : : : : : +- BroadcastExchange (31) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Project (28) - : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet default.item (16) - : : : : : +- BroadcastExchange (26) - : : : : : +- * Project (25) - : : : : : +- * Filter (24) - : : : : : +- * ColumnarToRow (23) - : : : : : +- Scan parquet default.date_dim (22) - : : : : +- ReusedExchange (34) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : :- * Project (22) + : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : :- * Project (15) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (13) + : : : : : : +- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (20) + : : : : : +- * Project (19) + : : : : : +- * Filter (18) + : : : : : +- * ColumnarToRow (17) + : : : : : +- Scan parquet default.date_dim (16) + : : : : +- BroadcastExchange (35) + : : : : +- * Project (34) + : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * Project (31) + : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : :- * Filter (25) + : : : : : : +- * ColumnarToRow (24) + : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Filter (28) + : : : : : +- * ColumnarToRow (27) + : : : : : +- Scan parquet default.item (26) + : : : : +- ReusedExchange (32) : : : +- BroadcastExchange (49) : : : +- * Project (48) : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -146,226 +146,226 @@ Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) (10) Scan parquet default.item -Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +(11) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(12) Filter [codegen id : 4] -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14)) +(12) Filter [codegen id : 1] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(13) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +(13) BroadcastExchange +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] +(14) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join condition: None -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#15) +(15) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(16) Scan parquet default.item -Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +(16) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_year#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +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 -(17) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +(17) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#12, d_year#13] -(18) Filter [codegen id : 1] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Condition : isnotnull(i_item_sk#17) +(18) Filter [codegen id : 2] +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) -(19) BroadcastExchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +(19) Project [codegen id : 2] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#15] -Right keys [1]: [i_item_sk#17] +(20) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] + +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20] -Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +(22) Project [codegen id : 6] +Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] +Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] -(22) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_year#23] +(23) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] 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 - -(23) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#22, d_year#23] +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(24) Filter [codegen id : 2] -Input [2]: [d_date_sk#22, d_year#23] -Condition : (((isnotnull(d_year#23) AND (d_year#23 >= 1999)) AND (d_year#23 <= 2001)) AND isnotnull(d_date_sk#22)) +(24) ColumnarToRow [codegen id : 5] +Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -(25) Project [codegen id : 2] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_year#23] +(25) Filter [codegen id : 5] +Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +Condition : isnotnull(cs_item_sk#18) -(26) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +(26) Scan parquet default.item +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(27) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#22] -Join condition: None +(27) ColumnarToRow [codegen id : 3] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(28) Project [codegen id : 3] -Output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] -Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#22] +(28) Filter [codegen id : 3] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : isnotnull(i_item_sk#6) (29) BroadcastExchange -Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] -(30) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)] -Right keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +(30) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#6] Join condition: None -(31) BroadcastExchange -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +(31) Project [codegen id : 5] +Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#11] +(32) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] + +(33) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None -(33) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +(34) Project [codegen id : 5] +Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] -(34) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#27] +(35) BroadcastExchange +Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] -(35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +(36) BroadcastHashJoin [codegen id : 6] +Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] +Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None -(36) Project [codegen id : 6] -Output [3]: [i_brand_id#12 AS brand_id#28, i_class_id#13 AS class_id#29, i_category_id#14 AS category_id#30] -Input [5]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#27] - (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#15, class_id#16, category_id#17] (38) Exchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#15, class_id#16, category_id#17] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_item_sk#32) +Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Condition : isnotnull(ws_item_sk#23) -(43) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +(43) ReusedExchange [Reuses operator id: 29] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#34] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [i_item_sk#6] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37] -Input [6]: [ws_item_sk#32, ws_sold_date_sk#33, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(46) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#38] +(46) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] -Input [5]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] +Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (49) BroadcastExchange -Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#39] +Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] -Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] +Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] +Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#15, class_id#16, category_id#17] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#15, class_id#16, category_id#17] (53) BroadcastExchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#40] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#28, class_id#29, category_id#30] +Right keys [3]: [brand_id#15, class_id#16, category_id#17] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#41] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] +Output [1]: [i_item_sk#6 AS ss_item_sk#27] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] (56) BroadcastExchange -Input [1]: [ss_item_sk#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [ss_item_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#41] +Right keys [1]: [ss_item_sk#27] Join condition: None (58) Scan parquet default.item @@ -383,16 +383,16 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#27] (62) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#41] +Right keys [1]: [ss_item_sk#27] Join condition: None (63) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] @@ -404,170 +404,170 @@ Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_ Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (66) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_week_seq#44] +Output [2]: [d_date_sk#12, d_week_seq#30] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [2]: [d_date_sk#27, d_week_seq#44] +Input [2]: [d_date_sk#12, d_week_seq#30] (68) Filter [codegen id : 24] -Input [2]: [d_date_sk#27, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#27)) +Input [2]: [d_date_sk#12, d_week_seq#30] +Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#31, [id=#32])) AND isnotnull(d_date_sk#12)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#27] -Input [2]: [d_date_sk#27, d_week_seq#44] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_week_seq#30] (70) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None (72) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#27] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (73) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#48, isEmpty#49, count#50] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] +Aggregate Attributes [3]: [sum#34, isEmpty#35, count#36] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#40] (75) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55, count(1)#56] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sales#57, count(1)#56 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41, count(1)#42] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sales#43, count(1)#42 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] (76) Filter [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59 as decimal(32,6)) > cast(Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45 as decimal(32,6)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) (77) Project [codegen id : 52] -Output [6]: [store AS channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] +Output [6]: [store AS channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] (78) Scan parquet default.store_sales -Output [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] +Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#49)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (80) Filter [codegen id : 50] -Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] -Condition : isnotnull(ss_item_sk#63) +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#27] (82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#63] -Right keys [1]: [ss_item_sk#41] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [ss_item_sk#27] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] +Output [4]: [i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] (84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#63] -Right keys [1]: [i_item_sk#68] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#50] Join condition: None (85) Project [codegen id : 50] -Output [6]: [ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_brand_id#69, i_class_id#70, i_category_id#71] -Input [8]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#72, d_week_seq#73] +Output [2]: [d_date_sk#12, d_week_seq#30] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#72, d_week_seq#73] +Input [2]: [d_date_sk#12, d_week_seq#30] (88) Filter [codegen id : 49] -Input [2]: [d_date_sk#72, d_week_seq#73] -Condition : ((isnotnull(d_week_seq#73) AND (d_week_seq#73 = Subquery scalar-subquery#74, [id=#75])) AND isnotnull(d_date_sk#72)) +Input [2]: [d_date_sk#12, d_week_seq#30] +Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#54, [id=#55])) AND isnotnull(d_date_sk#12)) (89) Project [codegen id : 49] -Output [1]: [d_date_sk#72] -Input [2]: [d_date_sk#72, d_week_seq#73] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_week_seq#30] (90) BroadcastExchange -Input [1]: [d_date_sk#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] (91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#66] -Right keys [1]: [d_date_sk#72] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (92) Project [codegen id : 50] -Output [5]: [ss_quantity#64, ss_list_price#65, i_brand_id#69, i_class_id#70, i_category_id#71] -Input [7]: [ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_brand_id#69, i_class_id#70, i_category_id#71, d_date_sk#72] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53, d_date_sk#12] (93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#64, ss_list_price#65, i_brand_id#69, i_class_id#70, i_category_id#71] -Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#77, isEmpty#78, count#79] -Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] +Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#57, isEmpty#58, count#59] +Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] (94) Exchange -Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] -Arguments: hashpartitioning(i_brand_id#69, i_class_id#70, i_category_id#71, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] +Arguments: hashpartitioning(i_brand_id#51, i_class_id#52, i_category_id#53, 5), ENSURE_REQUIREMENTS, [id=#63] (95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] -Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84, count(1)#85] -Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sales#86, count(1)#85 AS number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] +Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] +Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] +Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] (96) Filter [codegen id : 51] -Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) +Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) (97) Project [codegen id : 51] -Output [6]: [store AS channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] -Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] +Output [6]: [store AS channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] (98) BroadcastExchange -Input [6]: [channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#90] +Input [6]: [channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] (99) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] +Right keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] Join condition: None (100) TakeOrderedAndProject -Input [12]: [channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] -Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] +Input [12]: [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#60, [id=#61] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#46, [id=#47] * HashAggregate (123) +- Exchange (122) +- * HashAggregate (121) @@ -597,137 +597,137 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#91)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#71)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (103) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_year#92] +Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#92] +Input [2]: [d_date_sk#12, d_year#13] (105) Filter [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#92] -Condition : (((isnotnull(d_year#92) AND (d_year#92 >= 1999)) AND (d_year#92 <= 2001)) AND isnotnull(d_date_sk#27)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (106) Project [codegen id : 1] -Output [1]: [d_date_sk#27] -Input [2]: [d_date_sk#27, d_year#92] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (107) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] (108) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None (109) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#94, ss_list_price#3 AS list_price#95] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#27] +Output [2]: [ss_quantity#2 AS quantity#73, ss_list_price#3 AS list_price#74] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (110) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16] +Output [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#91)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#71)] ReadSchema: struct (111) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16] +Input [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] (112) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#98] +Output [1]: [d_date_sk#12] (113) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#98] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None (114) Project [codegen id : 4] -Output [2]: [cs_quantity#96 AS quantity#99, cs_list_price#97 AS list_price#100] -Input [4]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16, d_date_sk#98] +Output [2]: [cs_quantity#75 AS quantity#77, cs_list_price#76 AS list_price#78] +Input [4]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19, d_date_sk#12] (115) Scan parquet default.web_sales -Output [3]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33] +Output [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#91)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#71)] ReadSchema: struct (116) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33] +Input [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] (117) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#103] +Output [1]: [d_date_sk#12] (118) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#103] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (119) Project [codegen id : 6] -Output [2]: [ws_quantity#101 AS quantity#104, ws_list_price#102 AS list_price#105] -Input [4]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33, d_date_sk#103] +Output [2]: [ws_quantity#79 AS quantity#81, ws_list_price#80 AS list_price#82] +Input [4]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24, d_date_sk#12] (120) Union (121) HashAggregate [codegen id : 7] -Input [2]: [quantity#94, list_price#95] +Input [2]: [quantity#73, list_price#74] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#106, count#107] -Results [2]: [sum#108, count#109] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#83, count#84] +Results [2]: [sum#85, count#86] (122) Exchange -Input [2]: [sum#108, count#109] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] +Input [2]: [sum#85, count#86] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#87] (123) HashAggregate [codegen id : 8] -Input [2]: [sum#108, count#109] +Input [2]: [sum#85, count#86] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))#111] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))#111 AS average_sales#112] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88 AS average_sales#89] -Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#91 +Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#71 ReusedExchange (124) (124) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#91 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#71 -Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#91 +Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#71 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#12] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 ReusedExchange (126) -(126) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#27] +(126) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 -Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#31, [id=#32] * Project (130) +- * Filter (129) +- * ColumnarToRow (128) @@ -735,33 +735,33 @@ Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer (127) Scan parquet default.date_dim -Output [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (128) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] (129) Filter [codegen id : 1] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] -Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#92 = 2000)) AND (d_moy#113 = 12)) AND (d_dom#114 = 11)) +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 2000)) AND (d_moy#90 = 12)) AND (d_dom#91 = 11)) (130) Project [codegen id : 1] -Output [1]: [d_week_seq#44] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Output [1]: [d_week_seq#30] +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] -Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#60, [id=#61] +Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] -Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67 +Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#49 ReusedExchange (131) (131) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#72] +Output [1]: [d_date_sk#12] -Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#74, [id=#75] +Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#54, [id=#55] * Project (135) +- * Filter (134) +- * ColumnarToRow (133) @@ -769,21 +769,21 @@ Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subque (132) Scan parquet default.date_dim -Output [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (133) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] (134) Filter [codegen id : 1] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] -Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#92 = 1999)) AND (d_moy#113 = 12)) AND (d_dom#114 = 11)) +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 1999)) AND (d_moy#90 = 12)) AND (d_dom#91 = 11)) (135) Project [codegen id : 1] -Output [1]: [d_week_seq#44] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Output [1]: [d_week_seq#30] +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index cb9b410ea8568..3507d76bcf9b9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -82,53 +82,53 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Exchange [brand_id,class_id,category_id] #5 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (1) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #8 + BroadcastExchange #9 WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Filter [i_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #6 + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index 613e56b58c17b..cba099c2bb5b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- Union (72) +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- Union (63) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (76) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) + :- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,31 +50,22 @@ TakeOrderedAndProject (76) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- BroadcastExchange (51) - : +- * BroadcastHashJoin LeftSemi BuildRight (50) - : :- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet default.item (42) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- * Filter (47) - : +- * ColumnarToRow (46) - : +- Scan parquet default.item (45) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Filter (59) - : : : +- * ColumnarToRow (58) - : : : +- Scan parquet default.web_sales (57) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- ReusedExchange (66) + : +- ReusedExchange (42) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.web_sales (48) + : : +- ReusedExchange (51) + : +- ReusedExchange (54) + +- ReusedExchange (57) (1) Scan parquet default.store_sales @@ -172,7 +163,7 @@ Condition : isnotnull(i_item_sk#13) Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -180,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics)) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] @@ -242,205 +233,163 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#28] +Output [1]: [d_date_sk#6] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#28] +Right keys [1]: [d_date_sk#6] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#29] +Output [1]: [ca_address_sk#10] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#29] +Right keys [1]: [ca_address_sk#10] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] - -(42) Scan parquet default.item -Output [2]: [i_item_sk#30, i_manufact_id#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 10] -Input [2]: [i_item_sk#30, i_manufact_id#31] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] -(44) Filter [codegen id : 10] -Input [2]: [i_item_sk#30, i_manufact_id#31] -Condition : isnotnull(i_item_sk#30) - -(45) Scan parquet default.item -Output [2]: [i_category#15, i_manufact_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 9] -Input [2]: [i_category#15, i_manufact_id#14] - -(47) Filter [codegen id : 9] -Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) - -(48) Project [codegen id : 9] -Output [1]: [i_manufact_id#14] -Input [2]: [i_category#15, i_manufact_id#14] - -(49) BroadcastExchange -Input [1]: [i_manufact_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [i_manufact_id#31] -Right keys [1]: [i_manufact_id#14] -Join condition: None - -(51) BroadcastExchange -Input [2]: [i_item_sk#30, i_manufact_id#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +(42) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#13, i_manufact_id#14] -(52) BroadcastHashJoin [codegen id : 11] +(43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#30] +Right keys [1]: [i_item_sk#13] Join condition: None -(53) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_manufact_id#31] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_manufact_id#31] +(44) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#26, i_manufact_id#14] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_manufact_id#14] -(54) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_manufact_id#31] -Keys [1]: [i_manufact_id#31] +(45) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#26, i_manufact_id#14] +Keys [1]: [i_manufact_id#14] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_manufact_id#31, sum#35] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_manufact_id#14, sum#29] -(55) Exchange -Input [2]: [i_manufact_id#31, sum#35] -Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] +(46) Exchange +Input [2]: [i_manufact_id#14, sum#29] +Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] -(56) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#31, sum#35] -Keys [1]: [i_manufact_id#31] +(47) HashAggregate [codegen id : 12] +Input [2]: [i_manufact_id#14, sum#29] +Keys [1]: [i_manufact_id#14] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] -Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] +Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] -(57) Scan parquet default.web_sales -Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(48) Scan parquet default.web_sales +Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(49) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -(59) Filter [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) +(50) Filter [codegen id : 17] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) -(60) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#43] +(51) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#6] -(61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#43] +(52) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#6] Join condition: None -(62) Project [codegen id : 17] -Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] -Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] +(53) Project [codegen id : 17] +Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] -(63) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#44] +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#10] -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#40] -Right keys [1]: [ca_address_sk#44] +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#34] +Right keys [1]: [ca_address_sk#10] Join condition: None -(65) Project [codegen id : 17] -Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] +(56) Project [codegen id : 17] +Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] -(66) ReusedExchange [Reuses operator id: 51] -Output [2]: [i_item_sk#45, i_manufact_id#46] +(57) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#13, i_manufact_id#14] -(67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [i_item_sk#45] +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#33] +Right keys [1]: [i_item_sk#13] Join condition: None -(68) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#41, i_manufact_id#46] -Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_manufact_id#46] +(59) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#35, i_manufact_id#14] +Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_manufact_id#14] -(69) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#41, i_manufact_id#46] -Keys [1]: [i_manufact_id#46] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum#47] -Results [2]: [i_manufact_id#46, sum#48] +(60) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#35, i_manufact_id#14] +Keys [1]: [i_manufact_id#14] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#37] +Results [2]: [i_manufact_id#14, sum#38] -(70) Exchange -Input [2]: [i_manufact_id#46, sum#48] -Arguments: hashpartitioning(i_manufact_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] +(61) Exchange +Input [2]: [i_manufact_id#14, sum#38] +Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] -(71) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#46, sum#48] -Keys [1]: [i_manufact_id#46] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] -Results [2]: [i_manufact_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] +(62) HashAggregate [codegen id : 18] +Input [2]: [i_manufact_id#14, sum#38] +Keys [1]: [i_manufact_id#14] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] +Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] -(72) Union +(63) Union -(73) HashAggregate [codegen id : 19] +(64) HashAggregate [codegen id : 19] Input [2]: [i_manufact_id#14, total_sales#23] Keys [1]: [i_manufact_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [3]: [i_manufact_id#14, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [i_manufact_id#14, sum#44, isEmpty#45] -(74) Exchange -Input [3]: [i_manufact_id#14, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] +(65) Exchange +Input [3]: [i_manufact_id#14, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] -(75) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#14, sum#54, isEmpty#55] +(66) HashAggregate [codegen id : 20] +Input [3]: [i_manufact_id#14, sum#44, isEmpty#45] Keys [1]: [i_manufact_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#57] -Results [2]: [i_manufact_id#14, sum(total_sales#23)#57 AS total_sales#58] +Aggregate Attributes [1]: [sum(total_sales#23)#47] +Results [2]: [i_manufact_id#14, sum(total_sales#23)#47 AS total_sales#48] -(76) TakeOrderedAndProject -Input [2]: [i_manufact_id#14, total_sales#58] -Arguments: 100, [total_sales#58 ASC NULLS FIRST], [i_manufact_id#14, total_sales#58] +(67) TakeOrderedAndProject +Input [2]: [i_manufact_id#14, total_sales#48] +Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_manufact_id#14, total_sales#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (77) +ReusedExchange (68) -(77) ReusedExchange [Reuses operator id: 8] +(68) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt index ad5473e9da920..d1a115fa528d5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt @@ -79,25 +79,11 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_manufact_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_category,i_manufact_id] + ReusedExchange [i_item_sk,i_manufact_id] #5 WholeStageCodegen (18) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_manufact_id] #10 + Exchange [i_manufact_id] #8 WholeStageCodegen (17) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_manufact_id] @@ -116,4 +102,4 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #8 + ReusedExchange [i_item_sk,i_manufact_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index 613e56b58c17b..cba099c2bb5b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- Union (72) +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- Union (63) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (76) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) + :- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,31 +50,22 @@ TakeOrderedAndProject (76) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- BroadcastExchange (51) - : +- * BroadcastHashJoin LeftSemi BuildRight (50) - : :- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet default.item (42) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- * Filter (47) - : +- * ColumnarToRow (46) - : +- Scan parquet default.item (45) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Filter (59) - : : : +- * ColumnarToRow (58) - : : : +- Scan parquet default.web_sales (57) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- ReusedExchange (66) + : +- ReusedExchange (42) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.web_sales (48) + : : +- ReusedExchange (51) + : +- ReusedExchange (54) + +- ReusedExchange (57) (1) Scan parquet default.store_sales @@ -172,7 +163,7 @@ Condition : isnotnull(i_item_sk#13) Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -180,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics)) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] @@ -242,205 +233,163 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#28] +Output [1]: [d_date_sk#6] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#28] +Right keys [1]: [d_date_sk#6] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#29] +Output [1]: [ca_address_sk#10] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#29] +Right keys [1]: [ca_address_sk#10] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] - -(42) Scan parquet default.item -Output [2]: [i_item_sk#30, i_manufact_id#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 10] -Input [2]: [i_item_sk#30, i_manufact_id#31] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] -(44) Filter [codegen id : 10] -Input [2]: [i_item_sk#30, i_manufact_id#31] -Condition : isnotnull(i_item_sk#30) - -(45) Scan parquet default.item -Output [2]: [i_category#15, i_manufact_id#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 9] -Input [2]: [i_category#15, i_manufact_id#14] - -(47) Filter [codegen id : 9] -Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) - -(48) Project [codegen id : 9] -Output [1]: [i_manufact_id#14] -Input [2]: [i_category#15, i_manufact_id#14] - -(49) BroadcastExchange -Input [1]: [i_manufact_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [i_manufact_id#31] -Right keys [1]: [i_manufact_id#14] -Join condition: None - -(51) BroadcastExchange -Input [2]: [i_item_sk#30, i_manufact_id#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +(42) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#13, i_manufact_id#14] -(52) BroadcastHashJoin [codegen id : 11] +(43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#30] +Right keys [1]: [i_item_sk#13] Join condition: None -(53) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_manufact_id#31] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_manufact_id#31] +(44) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#26, i_manufact_id#14] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_manufact_id#14] -(54) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_manufact_id#31] -Keys [1]: [i_manufact_id#31] +(45) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#26, i_manufact_id#14] +Keys [1]: [i_manufact_id#14] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_manufact_id#31, sum#35] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_manufact_id#14, sum#29] -(55) Exchange -Input [2]: [i_manufact_id#31, sum#35] -Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] +(46) Exchange +Input [2]: [i_manufact_id#14, sum#29] +Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] -(56) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#31, sum#35] -Keys [1]: [i_manufact_id#31] +(47) HashAggregate [codegen id : 12] +Input [2]: [i_manufact_id#14, sum#29] +Keys [1]: [i_manufact_id#14] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] -Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] +Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] -(57) Scan parquet default.web_sales -Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(48) Scan parquet default.web_sales +Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(49) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -(59) Filter [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) +(50) Filter [codegen id : 17] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) -(60) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#43] +(51) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#6] -(61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#43] +(52) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#6] Join condition: None -(62) Project [codegen id : 17] -Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] -Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] +(53) Project [codegen id : 17] +Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] -(63) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#44] +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#10] -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#40] -Right keys [1]: [ca_address_sk#44] +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#34] +Right keys [1]: [ca_address_sk#10] Join condition: None -(65) Project [codegen id : 17] -Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] +(56) Project [codegen id : 17] +Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] -(66) ReusedExchange [Reuses operator id: 51] -Output [2]: [i_item_sk#45, i_manufact_id#46] +(57) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#13, i_manufact_id#14] -(67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [i_item_sk#45] +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#33] +Right keys [1]: [i_item_sk#13] Join condition: None -(68) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#41, i_manufact_id#46] -Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_manufact_id#46] +(59) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#35, i_manufact_id#14] +Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_manufact_id#14] -(69) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#41, i_manufact_id#46] -Keys [1]: [i_manufact_id#46] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum#47] -Results [2]: [i_manufact_id#46, sum#48] +(60) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#35, i_manufact_id#14] +Keys [1]: [i_manufact_id#14] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#37] +Results [2]: [i_manufact_id#14, sum#38] -(70) Exchange -Input [2]: [i_manufact_id#46, sum#48] -Arguments: hashpartitioning(i_manufact_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] +(61) Exchange +Input [2]: [i_manufact_id#14, sum#38] +Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] -(71) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#46, sum#48] -Keys [1]: [i_manufact_id#46] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] -Results [2]: [i_manufact_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] +(62) HashAggregate [codegen id : 18] +Input [2]: [i_manufact_id#14, sum#38] +Keys [1]: [i_manufact_id#14] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] +Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] -(72) Union +(63) Union -(73) HashAggregate [codegen id : 19] +(64) HashAggregate [codegen id : 19] Input [2]: [i_manufact_id#14, total_sales#23] Keys [1]: [i_manufact_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [3]: [i_manufact_id#14, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [i_manufact_id#14, sum#44, isEmpty#45] -(74) Exchange -Input [3]: [i_manufact_id#14, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] +(65) Exchange +Input [3]: [i_manufact_id#14, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] -(75) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#14, sum#54, isEmpty#55] +(66) HashAggregate [codegen id : 20] +Input [3]: [i_manufact_id#14, sum#44, isEmpty#45] Keys [1]: [i_manufact_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#57] -Results [2]: [i_manufact_id#14, sum(total_sales#23)#57 AS total_sales#58] +Aggregate Attributes [1]: [sum(total_sales#23)#47] +Results [2]: [i_manufact_id#14, sum(total_sales#23)#47 AS total_sales#48] -(76) TakeOrderedAndProject -Input [2]: [i_manufact_id#14, total_sales#58] -Arguments: 100, [total_sales#58 ASC NULLS FIRST], [i_manufact_id#14, total_sales#58] +(67) TakeOrderedAndProject +Input [2]: [i_manufact_id#14, total_sales#48] +Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_manufact_id#14, total_sales#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (77) +ReusedExchange (68) -(77) ReusedExchange [Reuses operator id: 8] +(68) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt index ad5473e9da920..d1a115fa528d5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/simplified.txt @@ -79,25 +79,11 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_manufact_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_category,i_manufact_id] + ReusedExchange [i_item_sk,i_manufact_id] #5 WholeStageCodegen (18) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_manufact_id] #10 + Exchange [i_manufact_id] #8 WholeStageCodegen (17) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_manufact_id] @@ -116,4 +102,4 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #8 + ReusedExchange [i_item_sk,i_manufact_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt index 3c6fe13107e04..8e3bf5bf57bcf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- Union (72) +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- Union (63) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (76) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) + :- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,31 +50,22 @@ TakeOrderedAndProject (76) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- BroadcastExchange (51) - : +- * BroadcastHashJoin LeftSemi BuildRight (50) - : :- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet default.item (42) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- * Filter (47) - : +- * ColumnarToRow (46) - : +- Scan parquet default.item (45) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Filter (59) - : : : +- * ColumnarToRow (58) - : : : +- Scan parquet default.web_sales (57) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- ReusedExchange (66) + : +- ReusedExchange (42) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.web_sales (48) + : : +- ReusedExchange (51) + : +- ReusedExchange (54) + +- ReusedExchange (57) (1) Scan parquet default.store_sales @@ -242,205 +233,163 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#28] +Output [1]: [d_date_sk#6] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#28] +Right keys [1]: [d_date_sk#6] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#29] +Output [1]: [ca_address_sk#10] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#29] +Right keys [1]: [ca_address_sk#10] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] - -(42) Scan parquet default.item -Output [2]: [i_item_sk#30, i_item_id#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 10] -Input [2]: [i_item_sk#30, i_item_id#31] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] -(44) Filter [codegen id : 10] -Input [2]: [i_item_sk#30, i_item_id#31] -Condition : isnotnull(i_item_sk#30) - -(45) Scan parquet default.item -Output [2]: [i_item_id#14, i_color#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_id#14, i_color#15] - -(47) Filter [codegen id : 9] -Input [2]: [i_item_id#14, i_color#15] -Condition : i_color#15 IN (slate ,blanched ,burnished ) - -(48) Project [codegen id : 9] -Output [1]: [i_item_id#14] -Input [2]: [i_item_id#14, i_color#15] - -(49) BroadcastExchange -Input [1]: [i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [i_item_id#31] -Right keys [1]: [i_item_id#14] -Join condition: None - -(51) BroadcastExchange -Input [2]: [i_item_sk#30, i_item_id#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +(42) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#13, i_item_id#14] -(52) BroadcastHashJoin [codegen id : 11] +(43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#30] +Right keys [1]: [i_item_sk#13] Join condition: None -(53) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#31] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] +(44) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#26, i_item_id#14] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] -(54) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#31] -Keys [1]: [i_item_id#31] +(45) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#26, i_item_id#14] +Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#31, sum#35] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_item_id#14, sum#29] -(55) Exchange -Input [2]: [i_item_id#31, sum#35] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] +(46) Exchange +Input [2]: [i_item_id#14, sum#29] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] -(56) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#31, sum#35] -Keys [1]: [i_item_id#31] +(47) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#14, sum#29] +Keys [1]: [i_item_id#14] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] -Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] +Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] -(57) Scan parquet default.web_sales -Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(48) Scan parquet default.web_sales +Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(49) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -(59) Filter [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) +(50) Filter [codegen id : 17] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) -(60) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#43] +(51) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#6] -(61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#43] +(52) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#6] Join condition: None -(62) Project [codegen id : 17] -Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] -Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] +(53) Project [codegen id : 17] +Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] -(63) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#44] +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#10] -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#40] -Right keys [1]: [ca_address_sk#44] +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#34] +Right keys [1]: [ca_address_sk#10] Join condition: None -(65) Project [codegen id : 17] -Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] +(56) Project [codegen id : 17] +Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] -(66) ReusedExchange [Reuses operator id: 51] -Output [2]: [i_item_sk#45, i_item_id#46] +(57) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#13, i_item_id#14] -(67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [i_item_sk#45] +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#33] +Right keys [1]: [i_item_sk#13] Join condition: None -(68) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#41, i_item_id#46] -Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_item_id#46] +(59) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#35, i_item_id#14] +Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] -(69) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#41, i_item_id#46] -Keys [1]: [i_item_id#46] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum#47] -Results [2]: [i_item_id#46, sum#48] +(60) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#35, i_item_id#14] +Keys [1]: [i_item_id#14] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#37] +Results [2]: [i_item_id#14, sum#38] -(70) Exchange -Input [2]: [i_item_id#46, sum#48] -Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] +(61) Exchange +Input [2]: [i_item_id#14, sum#38] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] -(71) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#46, sum#48] -Keys [1]: [i_item_id#46] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] -Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] +(62) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#14, sum#38] +Keys [1]: [i_item_id#14] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] +Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] -(72) Union +(63) Union -(73) HashAggregate [codegen id : 19] +(64) HashAggregate [codegen id : 19] Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [3]: [i_item_id#14, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [i_item_id#14, sum#44, isEmpty#45] -(74) Exchange -Input [3]: [i_item_id#14, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] +(65) Exchange +Input [3]: [i_item_id#14, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] -(75) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#54, isEmpty#55] +(66) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#14, sum#44, isEmpty#45] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#57] -Results [2]: [i_item_id#14, sum(total_sales#23)#57 AS total_sales#58] +Aggregate Attributes [1]: [sum(total_sales#23)#47] +Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] -(76) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#58] -Arguments: 100, [total_sales#58 ASC NULLS FIRST], [i_item_id#14, total_sales#58] +(67) TakeOrderedAndProject +Input [2]: [i_item_id#14, total_sales#48] +Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (77) +ReusedExchange (68) -(77) ReusedExchange [Reuses operator id: 8] +(68) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt index 34d320db2bb91..2d21010c26b8d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/simplified.txt @@ -79,25 +79,11 @@ TakeOrderedAndProject [total_sales,i_item_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_item_id] - Filter [i_color] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_id,i_color] + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #10 + Exchange [i_item_id] #8 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -116,4 +102,4 @@ TakeOrderedAndProject [total_sales,i_item_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #8 + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 3c6fe13107e04..8e3bf5bf57bcf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- Union (72) +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- Union (63) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (76) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) + :- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,31 +50,22 @@ TakeOrderedAndProject (76) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- BroadcastExchange (51) - : +- * BroadcastHashJoin LeftSemi BuildRight (50) - : :- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet default.item (42) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- * Filter (47) - : +- * ColumnarToRow (46) - : +- Scan parquet default.item (45) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Filter (59) - : : : +- * ColumnarToRow (58) - : : : +- Scan parquet default.web_sales (57) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- ReusedExchange (66) + : +- ReusedExchange (42) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.web_sales (48) + : : +- ReusedExchange (51) + : +- ReusedExchange (54) + +- ReusedExchange (57) (1) Scan parquet default.store_sales @@ -242,205 +233,163 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#28] +Output [1]: [d_date_sk#6] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#28] +Right keys [1]: [d_date_sk#6] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#29] +Output [1]: [ca_address_sk#10] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#29] +Right keys [1]: [ca_address_sk#10] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] - -(42) Scan parquet default.item -Output [2]: [i_item_sk#30, i_item_id#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 10] -Input [2]: [i_item_sk#30, i_item_id#31] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] -(44) Filter [codegen id : 10] -Input [2]: [i_item_sk#30, i_item_id#31] -Condition : isnotnull(i_item_sk#30) - -(45) Scan parquet default.item -Output [2]: [i_item_id#14, i_color#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_id#14, i_color#15] - -(47) Filter [codegen id : 9] -Input [2]: [i_item_id#14, i_color#15] -Condition : i_color#15 IN (slate ,blanched ,burnished ) - -(48) Project [codegen id : 9] -Output [1]: [i_item_id#14] -Input [2]: [i_item_id#14, i_color#15] - -(49) BroadcastExchange -Input [1]: [i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [i_item_id#31] -Right keys [1]: [i_item_id#14] -Join condition: None - -(51) BroadcastExchange -Input [2]: [i_item_sk#30, i_item_id#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +(42) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#13, i_item_id#14] -(52) BroadcastHashJoin [codegen id : 11] +(43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#30] +Right keys [1]: [i_item_sk#13] Join condition: None -(53) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#31] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] +(44) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#26, i_item_id#14] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] -(54) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#31] -Keys [1]: [i_item_id#31] +(45) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#26, i_item_id#14] +Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#31, sum#35] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_item_id#14, sum#29] -(55) Exchange -Input [2]: [i_item_id#31, sum#35] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] +(46) Exchange +Input [2]: [i_item_id#14, sum#29] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] -(56) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#31, sum#35] -Keys [1]: [i_item_id#31] +(47) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#14, sum#29] +Keys [1]: [i_item_id#14] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] -Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] +Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] -(57) Scan parquet default.web_sales -Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(48) Scan parquet default.web_sales +Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(49) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -(59) Filter [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) +(50) Filter [codegen id : 17] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) -(60) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#43] +(51) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#6] -(61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#43] +(52) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#6] Join condition: None -(62) Project [codegen id : 17] -Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] -Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] +(53) Project [codegen id : 17] +Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] -(63) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#44] +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#10] -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#40] -Right keys [1]: [ca_address_sk#44] +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#34] +Right keys [1]: [ca_address_sk#10] Join condition: None -(65) Project [codegen id : 17] -Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] +(56) Project [codegen id : 17] +Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] -(66) ReusedExchange [Reuses operator id: 51] -Output [2]: [i_item_sk#45, i_item_id#46] +(57) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#13, i_item_id#14] -(67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [i_item_sk#45] +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#33] +Right keys [1]: [i_item_sk#13] Join condition: None -(68) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#41, i_item_id#46] -Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_item_id#46] +(59) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#35, i_item_id#14] +Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] -(69) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#41, i_item_id#46] -Keys [1]: [i_item_id#46] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum#47] -Results [2]: [i_item_id#46, sum#48] +(60) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#35, i_item_id#14] +Keys [1]: [i_item_id#14] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#37] +Results [2]: [i_item_id#14, sum#38] -(70) Exchange -Input [2]: [i_item_id#46, sum#48] -Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] +(61) Exchange +Input [2]: [i_item_id#14, sum#38] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] -(71) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#46, sum#48] -Keys [1]: [i_item_id#46] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] -Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] +(62) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#14, sum#38] +Keys [1]: [i_item_id#14] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] +Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] -(72) Union +(63) Union -(73) HashAggregate [codegen id : 19] +(64) HashAggregate [codegen id : 19] Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [3]: [i_item_id#14, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [i_item_id#14, sum#44, isEmpty#45] -(74) Exchange -Input [3]: [i_item_id#14, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] +(65) Exchange +Input [3]: [i_item_id#14, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] -(75) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#54, isEmpty#55] +(66) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#14, sum#44, isEmpty#45] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#57] -Results [2]: [i_item_id#14, sum(total_sales#23)#57 AS total_sales#58] +Aggregate Attributes [1]: [sum(total_sales#23)#47] +Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] -(76) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#58] -Arguments: 100, [total_sales#58 ASC NULLS FIRST], [i_item_id#14, total_sales#58] +(67) TakeOrderedAndProject +Input [2]: [i_item_id#14, total_sales#48] +Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (77) +ReusedExchange (68) -(77) ReusedExchange [Reuses operator id: 8] +(68) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt index 34d320db2bb91..2d21010c26b8d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/simplified.txt @@ -79,25 +79,11 @@ TakeOrderedAndProject [total_sales,i_item_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_item_id] - Filter [i_color] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_id,i_color] + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #10 + Exchange [i_item_id] #8 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -116,4 +102,4 @@ TakeOrderedAndProject [total_sales,i_item_id] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #8 + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt index 8774d6d342fb0..10b8d0d9e7f05 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- Union (72) +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- Union (63) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (76) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) + :- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,31 +50,22 @@ TakeOrderedAndProject (76) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- BroadcastExchange (51) - : +- * BroadcastHashJoin LeftSemi BuildRight (50) - : :- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet default.item (42) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- * Filter (47) - : +- * ColumnarToRow (46) - : +- Scan parquet default.item (45) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Filter (59) - : : : +- * ColumnarToRow (58) - : : : +- Scan parquet default.web_sales (57) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- ReusedExchange (66) + : +- ReusedExchange (42) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.web_sales (48) + : : +- ReusedExchange (51) + : +- ReusedExchange (54) + +- ReusedExchange (57) (1) Scan parquet default.store_sales @@ -242,205 +233,163 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#28] +Output [1]: [d_date_sk#6] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#28] +Right keys [1]: [d_date_sk#6] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#29] +Output [1]: [ca_address_sk#10] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#29] +Right keys [1]: [ca_address_sk#10] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] - -(42) Scan parquet default.item -Output [2]: [i_item_sk#30, i_item_id#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 10] -Input [2]: [i_item_sk#30, i_item_id#31] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] -(44) Filter [codegen id : 10] -Input [2]: [i_item_sk#30, i_item_id#31] -Condition : isnotnull(i_item_sk#30) - -(45) Scan parquet default.item -Output [2]: [i_item_id#14, i_category#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_id#14, i_category#15] - -(47) Filter [codegen id : 9] -Input [2]: [i_item_id#14, i_category#15] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) - -(48) Project [codegen id : 9] -Output [1]: [i_item_id#14] -Input [2]: [i_item_id#14, i_category#15] - -(49) BroadcastExchange -Input [1]: [i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [i_item_id#31] -Right keys [1]: [i_item_id#14] -Join condition: None - -(51) BroadcastExchange -Input [2]: [i_item_sk#30, i_item_id#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +(42) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#13, i_item_id#14] -(52) BroadcastHashJoin [codegen id : 11] +(43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#30] +Right keys [1]: [i_item_sk#13] Join condition: None -(53) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#31] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] +(44) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#26, i_item_id#14] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] -(54) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#31] -Keys [1]: [i_item_id#31] +(45) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#26, i_item_id#14] +Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#31, sum#35] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_item_id#14, sum#29] -(55) Exchange -Input [2]: [i_item_id#31, sum#35] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] +(46) Exchange +Input [2]: [i_item_id#14, sum#29] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] -(56) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#31, sum#35] -Keys [1]: [i_item_id#31] +(47) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#14, sum#29] +Keys [1]: [i_item_id#14] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] -Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] +Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] -(57) Scan parquet default.web_sales -Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(48) Scan parquet default.web_sales +Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(49) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -(59) Filter [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) +(50) Filter [codegen id : 17] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) -(60) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#43] +(51) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#6] -(61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#43] +(52) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#6] Join condition: None -(62) Project [codegen id : 17] -Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] -Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] +(53) Project [codegen id : 17] +Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] -(63) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#44] +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#10] -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#40] -Right keys [1]: [ca_address_sk#44] +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#34] +Right keys [1]: [ca_address_sk#10] Join condition: None -(65) Project [codegen id : 17] -Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] +(56) Project [codegen id : 17] +Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] -(66) ReusedExchange [Reuses operator id: 51] -Output [2]: [i_item_sk#45, i_item_id#46] +(57) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#13, i_item_id#14] -(67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [i_item_sk#45] +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#33] +Right keys [1]: [i_item_sk#13] Join condition: None -(68) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#41, i_item_id#46] -Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_item_id#46] +(59) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#35, i_item_id#14] +Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] -(69) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#41, i_item_id#46] -Keys [1]: [i_item_id#46] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum#47] -Results [2]: [i_item_id#46, sum#48] +(60) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#35, i_item_id#14] +Keys [1]: [i_item_id#14] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#37] +Results [2]: [i_item_id#14, sum#38] -(70) Exchange -Input [2]: [i_item_id#46, sum#48] -Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] +(61) Exchange +Input [2]: [i_item_id#14, sum#38] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] -(71) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#46, sum#48] -Keys [1]: [i_item_id#46] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] -Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] +(62) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#14, sum#38] +Keys [1]: [i_item_id#14] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] +Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] -(72) Union +(63) Union -(73) HashAggregate [codegen id : 19] +(64) HashAggregate [codegen id : 19] Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [3]: [i_item_id#14, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [i_item_id#14, sum#44, isEmpty#45] -(74) Exchange -Input [3]: [i_item_id#14, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] +(65) Exchange +Input [3]: [i_item_id#14, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] -(75) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#54, isEmpty#55] +(66) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#14, sum#44, isEmpty#45] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#57] -Results [2]: [i_item_id#14, sum(total_sales#23)#57 AS total_sales#58] +Aggregate Attributes [1]: [sum(total_sales#23)#47] +Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] -(76) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#58] -Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#58 ASC NULLS FIRST], [i_item_id#14, total_sales#58] +(67) TakeOrderedAndProject +Input [2]: [i_item_id#14, total_sales#48] +Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (77) +ReusedExchange (68) -(77) ReusedExchange [Reuses operator id: 8] +(68) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt index 948e022cd11c8..46775417cbb70 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/simplified.txt @@ -79,25 +79,11 @@ TakeOrderedAndProject [i_item_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_item_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_id,i_category] + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #10 + Exchange [i_item_id] #8 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -116,4 +102,4 @@ TakeOrderedAndProject [i_item_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #8 + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 8774d6d342fb0..10b8d0d9e7f05 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * HashAggregate (75) - +- Exchange (74) - +- * HashAggregate (73) - +- Union (72) +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- Union (63) :- * HashAggregate (32) : +- Exchange (31) : +- * HashAggregate (30) @@ -36,11 +36,11 @@ TakeOrderedAndProject (76) : +- * Filter (23) : +- * ColumnarToRow (22) : +- Scan parquet default.item (21) - :- * HashAggregate (56) - : +- Exchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) + :- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) : :- * Project (41) : : +- * BroadcastHashJoin Inner BuildRight (40) : : :- * Project (38) @@ -50,31 +50,22 @@ TakeOrderedAndProject (76) : : : : +- Scan parquet default.catalog_sales (33) : : : +- ReusedExchange (36) : : +- ReusedExchange (39) - : +- BroadcastExchange (51) - : +- * BroadcastHashJoin LeftSemi BuildRight (50) - : :- * Filter (44) - : : +- * ColumnarToRow (43) - : : +- Scan parquet default.item (42) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- * Filter (47) - : +- * ColumnarToRow (46) - : +- Scan parquet default.item (45) - +- * HashAggregate (71) - +- Exchange (70) - +- * HashAggregate (69) - +- * Project (68) - +- * BroadcastHashJoin Inner BuildRight (67) - :- * Project (65) - : +- * BroadcastHashJoin Inner BuildRight (64) - : :- * Project (62) - : : +- * BroadcastHashJoin Inner BuildRight (61) - : : :- * Filter (59) - : : : +- * ColumnarToRow (58) - : : : +- Scan parquet default.web_sales (57) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- ReusedExchange (66) + : +- ReusedExchange (42) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * Project (56) + : +- * BroadcastHashJoin Inner BuildRight (55) + : :- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.web_sales (48) + : : +- ReusedExchange (51) + : +- ReusedExchange (54) + +- ReusedExchange (57) (1) Scan parquet default.store_sales @@ -242,205 +233,163 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#28] +Output [1]: [d_date_sk#6] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#28] +Right keys [1]: [d_date_sk#6] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#29] +Output [1]: [ca_address_sk#10] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#29] +Right keys [1]: [ca_address_sk#10] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] - -(42) Scan parquet default.item -Output [2]: [i_item_sk#30, i_item_id#31] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct - -(43) ColumnarToRow [codegen id : 10] -Input [2]: [i_item_sk#30, i_item_id#31] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] -(44) Filter [codegen id : 10] -Input [2]: [i_item_sk#30, i_item_id#31] -Condition : isnotnull(i_item_sk#30) - -(45) Scan parquet default.item -Output [2]: [i_item_id#14, i_category#15] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_id#14, i_category#15] - -(47) Filter [codegen id : 9] -Input [2]: [i_item_id#14, i_category#15] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) - -(48) Project [codegen id : 9] -Output [1]: [i_item_id#14] -Input [2]: [i_item_id#14, i_category#15] - -(49) BroadcastExchange -Input [1]: [i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] - -(50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [i_item_id#31] -Right keys [1]: [i_item_id#14] -Join condition: None - -(51) BroadcastExchange -Input [2]: [i_item_sk#30, i_item_id#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +(42) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#13, i_item_id#14] -(52) BroadcastHashJoin [codegen id : 11] +(43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#30] +Right keys [1]: [i_item_sk#13] Join condition: None -(53) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#31] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] +(44) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#26, i_item_id#14] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] -(54) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#31] -Keys [1]: [i_item_id#31] +(45) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#26, i_item_id#14] +Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#31, sum#35] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_item_id#14, sum#29] -(55) Exchange -Input [2]: [i_item_id#31, sum#35] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#36] +(46) Exchange +Input [2]: [i_item_id#14, sum#29] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] -(56) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#31, sum#35] -Keys [1]: [i_item_id#31] +(47) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#14, sum#29] +Keys [1]: [i_item_id#14] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#37] -Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#37,17,2) AS total_sales#38] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] +Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] -(57) Scan parquet default.web_sales -Output [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(48) Scan parquet default.web_sales +Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] +(49) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -(59) Filter [codegen id : 17] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42] -Condition : (isnotnull(ws_bill_addr_sk#40) AND isnotnull(ws_item_sk#39)) +(50) Filter [codegen id : 17] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) -(60) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#43] +(51) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#6] -(61) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#43] +(52) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#6] Join condition: None -(62) Project [codegen id : 17] -Output [3]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41] -Input [5]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ws_sold_date_sk#42, d_date_sk#43] +(53) Project [codegen id : 17] +Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] -(63) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#44] +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#10] -(64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#40] -Right keys [1]: [ca_address_sk#44] +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#34] +Right keys [1]: [ca_address_sk#10] Join condition: None -(65) Project [codegen id : 17] -Output [2]: [ws_item_sk#39, ws_ext_sales_price#41] -Input [4]: [ws_item_sk#39, ws_bill_addr_sk#40, ws_ext_sales_price#41, ca_address_sk#44] +(56) Project [codegen id : 17] +Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] -(66) ReusedExchange [Reuses operator id: 51] -Output [2]: [i_item_sk#45, i_item_id#46] +(57) ReusedExchange [Reuses operator id: 27] +Output [2]: [i_item_sk#13, i_item_id#14] -(67) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#39] -Right keys [1]: [i_item_sk#45] +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#33] +Right keys [1]: [i_item_sk#13] Join condition: None -(68) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#41, i_item_id#46] -Input [4]: [ws_item_sk#39, ws_ext_sales_price#41, i_item_sk#45, i_item_id#46] +(59) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#35, i_item_id#14] +Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] -(69) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#41, i_item_id#46] -Keys [1]: [i_item_id#46] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum#47] -Results [2]: [i_item_id#46, sum#48] +(60) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#35, i_item_id#14] +Keys [1]: [i_item_id#14] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#37] +Results [2]: [i_item_id#14, sum#38] -(70) Exchange -Input [2]: [i_item_id#46, sum#48] -Arguments: hashpartitioning(i_item_id#46, 5), ENSURE_REQUIREMENTS, [id=#49] +(61) Exchange +Input [2]: [i_item_id#14, sum#38] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] -(71) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#46, sum#48] -Keys [1]: [i_item_id#46] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#41))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#41))#50] -Results [2]: [i_item_id#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#41))#50,17,2) AS total_sales#51] +(62) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#14, sum#38] +Keys [1]: [i_item_id#14] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] +Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] -(72) Union +(63) Union -(73) HashAggregate [codegen id : 19] +(64) HashAggregate [codegen id : 19] Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [3]: [i_item_id#14, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [i_item_id#14, sum#44, isEmpty#45] -(74) Exchange -Input [3]: [i_item_id#14, sum#54, isEmpty#55] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#56] +(65) Exchange +Input [3]: [i_item_id#14, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] -(75) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#54, isEmpty#55] +(66) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#14, sum#44, isEmpty#45] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#57] -Results [2]: [i_item_id#14, sum(total_sales#23)#57 AS total_sales#58] +Aggregate Attributes [1]: [sum(total_sales#23)#47] +Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] -(76) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#58] -Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#58 ASC NULLS FIRST], [i_item_id#14, total_sales#58] +(67) TakeOrderedAndProject +Input [2]: [i_item_id#14, total_sales#48] +Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -ReusedExchange (77) +ReusedExchange (68) -(77) ReusedExchange [Reuses operator id: 8] +(68) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt index 948e022cd11c8..46775417cbb70 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/simplified.txt @@ -79,25 +79,11 @@ TakeOrderedAndProject [i_item_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (10) - BroadcastHashJoin [i_item_id,i_item_id] - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_item_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_id,i_category] + ReusedExchange [i_item_sk,i_item_id] #5 WholeStageCodegen (18) HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter - Exchange [i_item_id] #10 + Exchange [i_item_id] #8 WholeStageCodegen (17) HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id] @@ -116,4 +102,4 @@ TakeOrderedAndProject [i_item_id,total_sales] InputAdapter ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [i_item_sk,i_item_id] #8 + ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt index ef2ac135f64eb..d006b61d20c33 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (72) -+- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) +TakeOrderedAndProject (61) ++- * Project (60) + +- * BroadcastHashJoin Inner BuildRight (59) + :- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) : :- * HashAggregate (30) : : +- Exchange (29) : : +- * HashAggregate (28) @@ -34,43 +34,32 @@ TakeOrderedAndProject (72) : : +- * Filter (24) : : +- * ColumnarToRow (23) : : +- Scan parquet default.item (22) - : +- BroadcastExchange (54) - : +- * HashAggregate (53) - : +- Exchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) - : :- * Project (47) - : : +- * BroadcastHashJoin Inner BuildRight (46) + : +- BroadcastExchange (43) + : +- * HashAggregate (42) + : +- Exchange (41) + : +- * HashAggregate (40) + : +- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (36) + : : +- * BroadcastHashJoin Inner BuildRight (35) : : :- * Filter (33) : : : +- * ColumnarToRow (32) : : : +- Scan parquet default.catalog_returns (31) - : : +- BroadcastExchange (45) - : : +- * Project (44) - : : +- * BroadcastHashJoin LeftSemi BuildRight (43) - : : :- * Filter (36) - : : : +- * ColumnarToRow (35) - : : : +- Scan parquet default.date_dim (34) - : : +- BroadcastExchange (42) - : : +- * Project (41) - : : +- * BroadcastHashJoin LeftSemi BuildRight (40) - : : :- * ColumnarToRow (38) - : : : +- Scan parquet default.date_dim (37) - : : +- ReusedExchange (39) - : +- ReusedExchange (48) - +- BroadcastExchange (69) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * Project (65) - +- * BroadcastHashJoin Inner BuildRight (64) - :- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Filter (59) - : : +- * ColumnarToRow (58) - : : +- Scan parquet default.web_returns (57) - : +- ReusedExchange (60) - +- ReusedExchange (63) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) + +- BroadcastExchange (58) + +- * HashAggregate (57) + +- Exchange (56) + +- * HashAggregate (55) + +- * Project (54) + +- * BroadcastHashJoin Inner BuildRight (53) + :- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Filter (48) + : : +- * ColumnarToRow (47) + : : +- Scan parquet default.web_returns (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet default.store_returns @@ -227,181 +216,132 @@ Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] Condition : isnotnull(cr_item_sk#21) -(34) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_date#25] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(35) ColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#24, d_date#25] - -(36) Filter [codegen id : 8] -Input [2]: [d_date_sk#24, d_date#25] -Condition : isnotnull(d_date_sk#24) - -(37) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 7] -Input [2]: [d_date#5, d_week_seq#6] - -(39) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_week_seq#6#26] - -(40) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [d_week_seq#6] -Right keys [1]: [d_week_seq#6#26] -Join condition: None - -(41) Project [codegen id : 7] -Output [1]: [d_date#5] -Input [2]: [d_date#5, d_week_seq#6] - -(42) BroadcastExchange -Input [1]: [d_date#5] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#27] - -(43) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [d_date#25] -Right keys [1]: [d_date#5] -Join condition: None - -(44) Project [codegen id : 8] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_date#25] - -(45) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +(34) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#4] -(46) BroadcastHashJoin [codegen id : 10] +(35) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cr_returned_date_sk#23] -Right keys [1]: [d_date_sk#24] +Right keys [1]: [d_date_sk#4] Join condition: None -(47) Project [codegen id : 10] +(36) Project [codegen id : 10] Output [2]: [cr_item_sk#21, cr_return_quantity#22] -Input [4]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, d_date_sk#24] +Input [4]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, d_date_sk#4] -(48) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#29, i_item_id#30] +(37) ReusedExchange [Reuses operator id: 25] +Output [2]: [i_item_sk#12, i_item_id#13] -(49) BroadcastHashJoin [codegen id : 10] +(38) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cr_item_sk#21] -Right keys [1]: [i_item_sk#29] +Right keys [1]: [i_item_sk#12] Join condition: None -(50) Project [codegen id : 10] -Output [2]: [cr_return_quantity#22, i_item_id#30] -Input [4]: [cr_item_sk#21, cr_return_quantity#22, i_item_sk#29, i_item_id#30] +(39) Project [codegen id : 10] +Output [2]: [cr_return_quantity#22, i_item_id#13] +Input [4]: [cr_item_sk#21, cr_return_quantity#22, i_item_sk#12, i_item_id#13] -(51) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#22, i_item_id#30] -Keys [1]: [i_item_id#30] +(40) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#22, i_item_id#13] +Keys [1]: [i_item_id#13] Functions [1]: [partial_sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum#31] -Results [2]: [i_item_id#30, sum#32] +Aggregate Attributes [1]: [sum#24] +Results [2]: [i_item_id#13, sum#25] -(52) Exchange -Input [2]: [i_item_id#30, sum#32] -Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [id=#33] +(41) Exchange +Input [2]: [i_item_id#13, sum#25] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#26] -(53) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#30, sum#32] -Keys [1]: [i_item_id#30] +(42) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#13, sum#25] +Keys [1]: [i_item_id#13] Functions [1]: [sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum(cr_return_quantity#22)#34] -Results [2]: [i_item_id#30 AS item_id#35, sum(cr_return_quantity#22)#34 AS cr_item_qty#36] +Aggregate Attributes [1]: [sum(cr_return_quantity#22)#27] +Results [2]: [i_item_id#13 AS item_id#28, sum(cr_return_quantity#22)#27 AS cr_item_qty#29] -(54) BroadcastExchange -Input [2]: [item_id#35, cr_item_qty#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#37] +(43) BroadcastExchange +Input [2]: [item_id#28, cr_item_qty#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] -(55) BroadcastHashJoin [codegen id : 18] +(44) BroadcastHashJoin [codegen id : 18] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#35] +Right keys [1]: [item_id#28] Join condition: None -(56) Project [codegen id : 18] -Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#36] -Input [4]: [item_id#19, sr_item_qty#20, item_id#35, cr_item_qty#36] +(45) Project [codegen id : 18] +Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] +Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] -(57) Scan parquet default.web_returns -Output [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] +(46) Scan parquet default.web_returns +Output [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#40), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(wr_returned_date_sk#33), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] +(47) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] -(59) Filter [codegen id : 16] -Input [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] -Condition : isnotnull(wr_item_sk#38) +(48) Filter [codegen id : 16] +Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +Condition : isnotnull(wr_item_sk#31) -(60) ReusedExchange [Reuses operator id: 45] -Output [1]: [d_date_sk#41] +(49) ReusedExchange [Reuses operator id: 19] +Output [1]: [d_date_sk#4] -(61) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#40] -Right keys [1]: [cast(d_date_sk#41 as bigint)] +(50) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#33] +Right keys [1]: [cast(d_date_sk#4 as bigint)] Join condition: None -(62) Project [codegen id : 16] -Output [2]: [wr_item_sk#38, wr_return_quantity#39] -Input [4]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40, d_date_sk#41] +(51) Project [codegen id : 16] +Output [2]: [wr_item_sk#31, wr_return_quantity#32] +Input [4]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33, d_date_sk#4] -(63) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#42, i_item_id#43] +(52) ReusedExchange [Reuses operator id: 25] +Output [2]: [i_item_sk#12, i_item_id#13] -(64) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#38] -Right keys [1]: [cast(i_item_sk#42 as bigint)] +(53) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#31] +Right keys [1]: [cast(i_item_sk#12 as bigint)] Join condition: None -(65) Project [codegen id : 16] -Output [2]: [wr_return_quantity#39, i_item_id#43] -Input [4]: [wr_item_sk#38, wr_return_quantity#39, i_item_sk#42, i_item_id#43] - -(66) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#39, i_item_id#43] -Keys [1]: [i_item_id#43] -Functions [1]: [partial_sum(wr_return_quantity#39)] -Aggregate Attributes [1]: [sum#44] -Results [2]: [i_item_id#43, sum#45] - -(67) Exchange -Input [2]: [i_item_id#43, sum#45] -Arguments: hashpartitioning(i_item_id#43, 5), ENSURE_REQUIREMENTS, [id=#46] - -(68) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#43, sum#45] -Keys [1]: [i_item_id#43] -Functions [1]: [sum(wr_return_quantity#39)] -Aggregate Attributes [1]: [sum(wr_return_quantity#39)#47] -Results [2]: [i_item_id#43 AS item_id#48, sum(wr_return_quantity#39)#47 AS wr_item_qty#49] - -(69) BroadcastExchange -Input [2]: [item_id#48, wr_item_qty#49] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] - -(70) BroadcastHashJoin [codegen id : 18] +(54) Project [codegen id : 16] +Output [2]: [wr_return_quantity#32, i_item_id#13] +Input [4]: [wr_item_sk#31, wr_return_quantity#32, i_item_sk#12, i_item_id#13] + +(55) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#32, i_item_id#13] +Keys [1]: [i_item_id#13] +Functions [1]: [partial_sum(wr_return_quantity#32)] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#13, sum#35] + +(56) Exchange +Input [2]: [i_item_id#13, sum#35] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#36] + +(57) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#13, sum#35] +Keys [1]: [i_item_id#13] +Functions [1]: [sum(wr_return_quantity#32)] +Aggregate Attributes [1]: [sum(wr_return_quantity#32)#37] +Results [2]: [i_item_id#13 AS item_id#38, sum(wr_return_quantity#32)#37 AS wr_item_qty#39] + +(58) BroadcastExchange +Input [2]: [item_id#38, wr_item_qty#39] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] + +(59) BroadcastHashJoin [codegen id : 18] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#48] +Right keys [1]: [item_id#38] Join condition: None -(71) Project [codegen id : 18] -Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS sr_dev#51, cr_item_qty#36, (((cast(cr_item_qty#36 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS cr_dev#52, wr_item_qty#49, (((cast(wr_item_qty#49 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS wr_dev#53, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#54] -Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#36, item_id#48, wr_item_qty#49] +(60) Project [codegen id : 18] +Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] +Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] -(72) TakeOrderedAndProject -Input [8]: [item_id#19, sr_item_qty#20, sr_dev#51, cr_item_qty#36, cr_dev#52, wr_item_qty#49, wr_dev#53, average#54] -Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#51, cr_item_qty#36, cr_dev#52, wr_item_qty#49, wr_dev#53, average#54] +(61) TakeOrderedAndProject +Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt index 819d571695a83..b46131afd5ec7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/simplified.txt @@ -66,32 +66,15 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter Scan parquet default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] InputAdapter - BroadcastExchange #8 - WholeStageCodegen (8) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (7) - Project [d_date] - BroadcastHashJoin [d_week_seq,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - InputAdapter - ReusedExchange [d_week_seq] #4 + ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 InputAdapter - BroadcastExchange #10 + BroadcastExchange #8 WholeStageCodegen (17) HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] InputAdapter - Exchange [i_item_id] #11 + Exchange [i_item_id] #9 WholeStageCodegen (16) HashAggregate [i_item_id,wr_return_quantity] [sum,sum] Project [wr_return_quantity,i_item_id] @@ -103,6 +86,6 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter Scan parquet default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] InputAdapter - ReusedExchange [d_date_sk] #8 + ReusedExchange [d_date_sk] #2 InputAdapter ReusedExchange [i_item_sk,i_item_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index 8b14a282c2671..c926ab7e3ccb6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (72) -+- * Project (71) - +- * BroadcastHashJoin Inner BuildRight (70) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) +TakeOrderedAndProject (61) ++- * Project (60) + +- * BroadcastHashJoin Inner BuildRight (59) + :- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) : :- * HashAggregate (30) : : +- Exchange (29) : : +- * HashAggregate (28) @@ -34,43 +34,32 @@ TakeOrderedAndProject (72) : : +- * Filter (17) : : +- * ColumnarToRow (16) : : +- Scan parquet default.date_dim (15) - : +- BroadcastExchange (54) - : +- * HashAggregate (53) - : +- Exchange (52) - : +- * HashAggregate (51) - : +- * Project (50) - : +- * BroadcastHashJoin Inner BuildRight (49) + : +- BroadcastExchange (43) + : +- * HashAggregate (42) + : +- Exchange (41) + : +- * HashAggregate (40) + : +- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) : :- * Project (36) : : +- * BroadcastHashJoin Inner BuildRight (35) : : :- * Filter (33) : : : +- * ColumnarToRow (32) : : : +- Scan parquet default.catalog_returns (31) : : +- ReusedExchange (34) - : +- BroadcastExchange (48) - : +- * Project (47) - : +- * BroadcastHashJoin LeftSemi BuildRight (46) - : :- * Filter (39) - : : +- * ColumnarToRow (38) - : : +- Scan parquet default.date_dim (37) - : +- BroadcastExchange (45) - : +- * Project (44) - : +- * BroadcastHashJoin LeftSemi BuildRight (43) - : :- * ColumnarToRow (41) - : : +- Scan parquet default.date_dim (40) - : +- ReusedExchange (42) - +- BroadcastExchange (69) - +- * HashAggregate (68) - +- Exchange (67) - +- * HashAggregate (66) - +- * Project (65) - +- * BroadcastHashJoin Inner BuildRight (64) - :- * Project (62) - : +- * BroadcastHashJoin Inner BuildRight (61) - : :- * Filter (59) - : : +- * ColumnarToRow (58) - : : +- Scan parquet default.web_returns (57) - : +- ReusedExchange (60) - +- ReusedExchange (63) + : +- ReusedExchange (37) + +- BroadcastExchange (58) + +- * HashAggregate (57) + +- Exchange (56) + +- * HashAggregate (55) + +- * Project (54) + +- * BroadcastHashJoin Inner BuildRight (53) + :- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Filter (48) + : : +- * ColumnarToRow (47) + : : +- Scan parquet default.web_returns (46) + : +- ReusedExchange (49) + +- ReusedExchange (52) (1) Scan parquet default.store_returns @@ -228,180 +217,131 @@ Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] Condition : isnotnull(cr_item_sk#21) (34) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#24, i_item_id#25] +Output [2]: [i_item_sk#4, i_item_id#5] (35) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cr_item_sk#21] -Right keys [1]: [i_item_sk#24] +Right keys [1]: [i_item_sk#4] Join condition: None (36) Project [codegen id : 10] -Output [3]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#25] -Input [5]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, i_item_sk#24, i_item_id#25] - -(37) Scan parquet default.date_dim -Output [2]: [d_date_sk#26, d_date#27] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] -ReadSchema: struct - -(38) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#26, d_date#27] - -(39) Filter [codegen id : 9] -Input [2]: [d_date_sk#26, d_date#27] -Condition : isnotnull(d_date_sk#26) - -(40) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -ReadSchema: struct - -(41) ColumnarToRow [codegen id : 8] -Input [2]: [d_date#8, d_week_seq#9] +Output [3]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#5] +Input [5]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, i_item_sk#4, i_item_id#5] -(42) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_week_seq#9#28] - -(43) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [d_week_seq#9] -Right keys [1]: [d_week_seq#9#28] -Join condition: None - -(44) Project [codegen id : 8] -Output [1]: [d_date#8] -Input [2]: [d_date#8, d_week_seq#9] - -(45) BroadcastExchange -Input [1]: [d_date#8] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#29] - -(46) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_date#27] -Right keys [1]: [d_date#8] -Join condition: None - -(47) Project [codegen id : 9] -Output [1]: [d_date_sk#26] -Input [2]: [d_date_sk#26, d_date#27] - -(48) BroadcastExchange -Input [1]: [d_date_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] +(37) ReusedExchange [Reuses operator id: 25] +Output [1]: [d_date_sk#7] -(49) BroadcastHashJoin [codegen id : 10] +(38) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cr_returned_date_sk#23] -Right keys [1]: [d_date_sk#26] +Right keys [1]: [d_date_sk#7] Join condition: None -(50) Project [codegen id : 10] -Output [2]: [cr_return_quantity#22, i_item_id#25] -Input [4]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#25, d_date_sk#26] +(39) Project [codegen id : 10] +Output [2]: [cr_return_quantity#22, i_item_id#5] +Input [4]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#5, d_date_sk#7] -(51) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#22, i_item_id#25] -Keys [1]: [i_item_id#25] +(40) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#22, i_item_id#5] +Keys [1]: [i_item_id#5] Functions [1]: [partial_sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum#31] -Results [2]: [i_item_id#25, sum#32] +Aggregate Attributes [1]: [sum#24] +Results [2]: [i_item_id#5, sum#25] -(52) Exchange -Input [2]: [i_item_id#25, sum#32] -Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [id=#33] +(41) Exchange +Input [2]: [i_item_id#5, sum#25] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#26] -(53) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#25, sum#32] -Keys [1]: [i_item_id#25] +(42) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#5, sum#25] +Keys [1]: [i_item_id#5] Functions [1]: [sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum(cr_return_quantity#22)#34] -Results [2]: [i_item_id#25 AS item_id#35, sum(cr_return_quantity#22)#34 AS cr_item_qty#36] +Aggregate Attributes [1]: [sum(cr_return_quantity#22)#27] +Results [2]: [i_item_id#5 AS item_id#28, sum(cr_return_quantity#22)#27 AS cr_item_qty#29] -(54) BroadcastExchange -Input [2]: [item_id#35, cr_item_qty#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#37] +(43) BroadcastExchange +Input [2]: [item_id#28, cr_item_qty#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] -(55) BroadcastHashJoin [codegen id : 18] +(44) BroadcastHashJoin [codegen id : 18] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#35] +Right keys [1]: [item_id#28] Join condition: None -(56) Project [codegen id : 18] -Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#36] -Input [4]: [item_id#19, sr_item_qty#20, item_id#35, cr_item_qty#36] +(45) Project [codegen id : 18] +Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] +Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] -(57) Scan parquet default.web_returns -Output [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] +(46) Scan parquet default.web_returns +Output [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#40), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(wr_returned_date_sk#33), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct -(58) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] +(47) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] -(59) Filter [codegen id : 16] -Input [3]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40] -Condition : isnotnull(wr_item_sk#38) +(48) Filter [codegen id : 16] +Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +Condition : isnotnull(wr_item_sk#31) -(60) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#41, i_item_id#42] +(49) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#4, i_item_id#5] -(61) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#38] -Right keys [1]: [cast(i_item_sk#41 as bigint)] +(50) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#31] +Right keys [1]: [cast(i_item_sk#4 as bigint)] Join condition: None -(62) Project [codegen id : 16] -Output [3]: [wr_return_quantity#39, wr_returned_date_sk#40, i_item_id#42] -Input [5]: [wr_item_sk#38, wr_return_quantity#39, wr_returned_date_sk#40, i_item_sk#41, i_item_id#42] +(51) Project [codegen id : 16] +Output [3]: [wr_return_quantity#32, wr_returned_date_sk#33, i_item_id#5] +Input [5]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33, i_item_sk#4, i_item_id#5] -(63) ReusedExchange [Reuses operator id: 48] -Output [1]: [d_date_sk#43] +(52) ReusedExchange [Reuses operator id: 25] +Output [1]: [d_date_sk#7] -(64) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#40] -Right keys [1]: [cast(d_date_sk#43 as bigint)] +(53) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#33] +Right keys [1]: [cast(d_date_sk#7 as bigint)] Join condition: None -(65) Project [codegen id : 16] -Output [2]: [wr_return_quantity#39, i_item_id#42] -Input [4]: [wr_return_quantity#39, wr_returned_date_sk#40, i_item_id#42, d_date_sk#43] - -(66) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#39, i_item_id#42] -Keys [1]: [i_item_id#42] -Functions [1]: [partial_sum(wr_return_quantity#39)] -Aggregate Attributes [1]: [sum#44] -Results [2]: [i_item_id#42, sum#45] - -(67) Exchange -Input [2]: [i_item_id#42, sum#45] -Arguments: hashpartitioning(i_item_id#42, 5), ENSURE_REQUIREMENTS, [id=#46] - -(68) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#42, sum#45] -Keys [1]: [i_item_id#42] -Functions [1]: [sum(wr_return_quantity#39)] -Aggregate Attributes [1]: [sum(wr_return_quantity#39)#47] -Results [2]: [i_item_id#42 AS item_id#48, sum(wr_return_quantity#39)#47 AS wr_item_qty#49] - -(69) BroadcastExchange -Input [2]: [item_id#48, wr_item_qty#49] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] - -(70) BroadcastHashJoin [codegen id : 18] +(54) Project [codegen id : 16] +Output [2]: [wr_return_quantity#32, i_item_id#5] +Input [4]: [wr_return_quantity#32, wr_returned_date_sk#33, i_item_id#5, d_date_sk#7] + +(55) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#32, i_item_id#5] +Keys [1]: [i_item_id#5] +Functions [1]: [partial_sum(wr_return_quantity#32)] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#5, sum#35] + +(56) Exchange +Input [2]: [i_item_id#5, sum#35] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#36] + +(57) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#5, sum#35] +Keys [1]: [i_item_id#5] +Functions [1]: [sum(wr_return_quantity#32)] +Aggregate Attributes [1]: [sum(wr_return_quantity#32)#37] +Results [2]: [i_item_id#5 AS item_id#38, sum(wr_return_quantity#32)#37 AS wr_item_qty#39] + +(58) BroadcastExchange +Input [2]: [item_id#38, wr_item_qty#39] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] + +(59) BroadcastHashJoin [codegen id : 18] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#48] +Right keys [1]: [item_id#38] Join condition: None -(71) Project [codegen id : 18] -Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS sr_dev#51, cr_item_qty#36, (((cast(cr_item_qty#36 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS cr_dev#52, wr_item_qty#49, (((cast(wr_item_qty#49 as double) / cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as double)) / 3.0) * 100.0) AS wr_dev#53, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#36) + wr_item_qty#49) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#54] -Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#36, item_id#48, wr_item_qty#49] +(60) Project [codegen id : 18] +Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] +Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] -(72) TakeOrderedAndProject -Input [8]: [item_id#19, sr_item_qty#20, sr_dev#51, cr_item_qty#36, cr_dev#52, wr_item_qty#49, wr_dev#53, average#54] -Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#51, cr_item_qty#36, cr_dev#52, wr_item_qty#49, wr_dev#53, average#54] +(61) TakeOrderedAndProject +Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt index 9632348854668..7fc930f2885fd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/simplified.txt @@ -68,30 +68,13 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (9) - Project [d_date_sk] - BroadcastHashJoin [d_date,d_date] - Filter [d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (8) - Project [d_date] - BroadcastHashJoin [d_week_seq,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date,d_week_seq] - InputAdapter - ReusedExchange [d_week_seq] #5 + ReusedExchange [d_date_sk] #3 InputAdapter - BroadcastExchange #10 + BroadcastExchange #8 WholeStageCodegen (17) HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] InputAdapter - Exchange [i_item_id] #11 + Exchange [i_item_id] #9 WholeStageCodegen (16) HashAggregate [i_item_id,wr_return_quantity] [sum,sum] Project [wr_return_quantity,i_item_id] @@ -105,4 +88,4 @@ TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty InputAdapter ReusedExchange [i_item_sk,i_item_id] #2 InputAdapter - ReusedExchange [d_date_sk] #8 + ReusedExchange [d_date_sk] #3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt index b19d4f3aeae0f..312aba94bd9c6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt @@ -33,39 +33,39 @@ TakeOrderedAndProject (116) : : : : +- * HashAggregate (44) : : : : +- Exchange (43) : : : : +- * HashAggregate (42) - : : : : +- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * Project (18) - : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : :- * Filter (11) - : : : : : : +- * ColumnarToRow (10) - : : : : : : +- Scan parquet default.store_sales (9) - : : : : : +- BroadcastExchange (16) - : : : : : +- * Project (15) - : : : : : +- * Filter (14) - : : : : : +- * ColumnarToRow (13) - : : : : : +- Scan parquet default.date_dim (12) - : : : : +- BroadcastExchange (39) - : : : : +- SortMergeJoin LeftSemi (38) - : : : : :- * Sort (23) - : : : : : +- Exchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (37) - : : : : +- Exchange (36) - : : : : +- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (26) - : : : : : : +- * ColumnarToRow (25) - : : : : : : +- Scan parquet default.catalog_sales (24) - : : : : : +- ReusedExchange (27) - : : : : +- BroadcastExchange (33) - : : : : +- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet default.item (30) + : : : : +- SortMergeJoin LeftSemi (41) + : : : : :- * Sort (26) + : : : : : +- Exchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (40) + : : : : +- Exchange (39) + : : : : +- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.item (33) : : : +- * Sort (57) : : : +- Exchange (56) : : : +- * Project (55) @@ -147,10 +147,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 19] +(7) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 19] +(8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) @@ -162,10 +162,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 10] +(10) ColumnarToRow [codegen id : 5] Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -(11) Filter [codegen id : 10] +(11) Filter [codegen id : 5] Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -191,38 +191,51 @@ Input [2]: [d_date_sk#12, d_year#13] Input [1]: [d_date_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(17) BroadcastHashJoin [codegen id : 10] +(17) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#12] Join condition: None -(18) Project [codegen id : 10] +(18) Project [codegen id : 5] Output [1]: [ss_item_sk#1] Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] (19) Scan parquet default.item -Output [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Condition : (((isnotnull(i_item_sk#15) AND isnotnull(i_brand_id#16)) AND isnotnull(i_class_id#17)) AND isnotnull(i_category_id#18)) +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) -(22) Exchange -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: hashpartitioning(coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] +(22) BroadcastExchange +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] -(23) Sort [codegen id : 5] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: [coalesce(i_brand_id#16, 0) ASC NULLS FIRST, isnull(i_brand_id#16) ASC NULLS FIRST, coalesce(i_class_id#17, 0) ASC NULLS FIRST, isnull(i_class_id#17) ASC NULLS FIRST, coalesce(i_category_id#18, 0) ASC NULLS FIRST, isnull(i_category_id#18) ASC NULLS FIRST], false, 0 +(23) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#7] +Join condition: None + +(24) Project [codegen id : 5] +Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] +Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] + +(25) Exchange +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] -(24) Scan parquet default.catalog_sales +(26) Sort [codegen id : 6] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 + +(27) Scan parquet default.catalog_sales Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] @@ -230,228 +243,215 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 8] +(28) ColumnarToRow [codegen id : 9] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -(26) Filter [codegen id : 8] +(29) Filter [codegen id : 9] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] Condition : isnotnull(cs_item_sk#20) -(27) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#22] +(30) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#12] -(28) BroadcastHashJoin [codegen id : 8] +(31) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#12] Join condition: None -(29) Project [codegen id : 8] +(32) Project [codegen id : 9] Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#22] +Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] -(30) Scan parquet default.item -Output [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +(33) Scan parquet default.item +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 7] -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(32) Filter [codegen id : 7] -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -Condition : isnotnull(i_item_sk#23) +(35) Filter [codegen id : 8] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : isnotnull(i_item_sk#7) -(33) BroadcastExchange -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +(36) BroadcastExchange +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(34) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#23] +Right keys [1]: [i_item_sk#7] Join condition: None -(35) Project [codegen id : 8] -Output [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Input [5]: [cs_item_sk#20, i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +(38) Project [codegen id : 9] +Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(36) Exchange -Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: hashpartitioning(coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26), 5), ENSURE_REQUIREMENTS, [id=#28] +(39) Exchange +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] -(37) Sort [codegen id : 9] -Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: [coalesce(i_brand_id#24, 0) ASC NULLS FIRST, isnull(i_brand_id#24) ASC NULLS FIRST, coalesce(i_class_id#25, 0) ASC NULLS FIRST, isnull(i_class_id#25) ASC NULLS FIRST, coalesce(i_category_id#26, 0) ASC NULLS FIRST, isnull(i_category_id#26) ASC NULLS FIRST], false, 0 +(40) Sort [codegen id : 10] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 -(38) SortMergeJoin -Left keys [6]: [coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18)] -Right keys [6]: [coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26)] +(41) SortMergeJoin +Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] +Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] Join condition: None -(39) BroadcastExchange -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] - -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] -Join condition: None - -(41) Project [codegen id : 10] -Output [3]: [i_brand_id#16 AS brand_id#30, i_class_id#17 AS class_id#31, i_category_id#18 AS category_id#32] -Input [5]: [ss_item_sk#1, i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] - -(42) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +(42) HashAggregate [codegen id : 11] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#16, class_id#17, category_id#18] (43) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] -(44) HashAggregate [codegen id : 11] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +(44) HashAggregate [codegen id : 12] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#16, class_id#17, category_id#18] (45) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32), 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] -(46) Sort [codegen id : 12] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: [coalesce(brand_id#30, 0) ASC NULLS FIRST, isnull(brand_id#30) ASC NULLS FIRST, coalesce(class_id#31, 0) ASC NULLS FIRST, isnull(class_id#31) ASC NULLS FIRST, coalesce(category_id#32, 0) ASC NULLS FIRST, isnull(category_id#32) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 13] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 15] -Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] +(48) ColumnarToRow [codegen id : 16] +Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -(49) Filter [codegen id : 15] -Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#35) +(49) Filter [codegen id : 16] +Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Condition : isnotnull(ws_item_sk#26) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#37] +Output [1]: [d_date_sk#12] -(51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#37] +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None -(52) Project [codegen id : 15] -Output [1]: [ws_item_sk#35] -Input [3]: [ws_item_sk#35, ws_sold_date_sk#36, d_date_sk#37] +(52) Project [codegen id : 16] +Output [1]: [ws_item_sk#26] +Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] -(53) ReusedExchange [Reuses operator id: 33] -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(53) ReusedExchange [Reuses operator id: 36] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(54) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [i_item_sk#38] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [i_item_sk#7] Join condition: None -(55) Project [codegen id : 15] -Output [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Input [5]: [ws_item_sk#35, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(55) Project [codegen id : 16] +Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (56) Exchange -Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: hashpartitioning(coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41), 5), ENSURE_REQUIREMENTS, [id=#42] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] -(57) Sort [codegen id : 16] -Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: [coalesce(i_brand_id#39, 0) ASC NULLS FIRST, isnull(i_brand_id#39) ASC NULLS FIRST, coalesce(i_class_id#40, 0) ASC NULLS FIRST, isnull(i_class_id#40) ASC NULLS FIRST, coalesce(i_category_id#41, 0) ASC NULLS FIRST, isnull(i_category_id#41) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 17] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] -Right keys [6]: [coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41)] +Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] +Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] Join condition: None -(59) HashAggregate [codegen id : 17] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +(59) HashAggregate [codegen id : 18] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#16, class_id#17, category_id#18] (60) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] -(61) HashAggregate [codegen id : 18] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +(61) HashAggregate [codegen id : 19] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#16, class_id#17, category_id#18] (62) BroadcastExchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#44] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] -(63) BroadcastHashJoin [codegen id : 19] +(63) BroadcastHashJoin [codegen id : 20] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#30, class_id#31, category_id#32] +Right keys [3]: [brand_id#16, class_id#17, category_id#18] Join condition: None -(64) Project [codegen id : 19] -Output [1]: [i_item_sk#7 AS ss_item_sk#45] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#30, class_id#31, category_id#32] +(64) Project [codegen id : 20] +Output [1]: [i_item_sk#7 AS ss_item_sk#31] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] (65) Exchange -Input [1]: [ss_item_sk#45] -Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [1]: [ss_item_sk#31] +Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] -(66) Sort [codegen id : 20] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 21] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#31] Join condition: None (68) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#47] +Output [2]: [d_date_sk#12, d_week_seq#33] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 21] -Input [2]: [d_date_sk#12, d_week_seq#47] +(69) ColumnarToRow [codegen id : 22] +Input [2]: [d_date_sk#12, d_week_seq#33] -(70) Filter [codegen id : 21] -Input [2]: [d_date_sk#12, d_week_seq#47] -Condition : ((isnotnull(d_week_seq#47) AND (d_week_seq#47 = Subquery scalar-subquery#48, [id=#49])) AND isnotnull(d_date_sk#12)) +(70) Filter [codegen id : 22] +Input [2]: [d_date_sk#12, d_week_seq#33] +Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#34, [id=#35])) AND isnotnull(d_date_sk#12)) -(71) Project [codegen id : 21] +(71) Project [codegen id : 22] Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#47] +Input [2]: [d_date_sk#12, d_week_seq#33] (72) BroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] -(73) BroadcastHashJoin [codegen id : 42] +(73) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#12] Join condition: None -(74) Project [codegen id : 42] +(74) Project [codegen id : 44] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] @@ -462,192 +462,192 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 22] +(76) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(77) Filter [codegen id : 22] +(77) Filter [codegen id : 23] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (78) Exchange Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#51] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#37] -(79) Sort [codegen id : 23] +(79) Sort [codegen id : 24] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#31] -(81) Sort [codegen id : 41] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 43] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (82) SortMergeJoin Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#31] Join condition: None (83) BroadcastExchange Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -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=#38] -(84) BroadcastHashJoin [codegen id : 42] +(84) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#7] Join condition: None -(85) Project [codegen id : 42] +(85) Project [codegen id : 44] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(86) HashAggregate [codegen id : 42] +(86) HashAggregate [codegen id : 44] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#53, isEmpty#54, count#55] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] +Aggregate Attributes [3]: [sum#39, isEmpty#40, count#41] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#45] -(88) HashAggregate [codegen id : 86] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#56, isEmpty#57, count#58] +(88) HashAggregate [codegen id : 90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60, count(1)#61] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sales#62, count(1)#61 AS number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#60 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46, count(1)#47] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sales#48, count(1)#47 AS number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] -(89) Filter [codegen id : 86] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 as decimal(32,6)) > cast(Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) +(89) Filter [codegen id : 90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(90) Project [codegen id : 86] -Output [6]: [store AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64] +(90) Project [codegen id : 90] +Output [6]: [store AS channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] (91) Scan parquet default.store_sales -Output [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#71), dynamicpruningexpression(ss_sold_date_sk#71 IN dynamicpruning#72)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 43] -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] +(92) ColumnarToRow [codegen id : 45] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(93) Filter [codegen id : 43] -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] -Condition : isnotnull(ss_item_sk#68) +(93) Filter [codegen id : 45] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (94) Exchange -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] -Arguments: hashpartitioning(ss_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#55] -(95) Sort [codegen id : 44] -Input [4]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71] -Arguments: [ss_item_sk#68 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 46] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#31] -(97) Sort [codegen id : 62] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 65] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ss_item_sk#68] -Right keys [1]: [ss_item_sk#45] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [ss_item_sk#31] Join condition: None (99) Scan parquet default.date_dim -Output [2]: [d_date_sk#74, d_week_seq#75] +Output [2]: [d_date_sk#12, d_week_seq#33] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 63] -Input [2]: [d_date_sk#74, d_week_seq#75] +(100) ColumnarToRow [codegen id : 66] +Input [2]: [d_date_sk#12, d_week_seq#33] -(101) Filter [codegen id : 63] -Input [2]: [d_date_sk#74, d_week_seq#75] -Condition : ((isnotnull(d_week_seq#75) AND (d_week_seq#75 = Subquery scalar-subquery#76, [id=#77])) AND isnotnull(d_date_sk#74)) +(101) Filter [codegen id : 66] +Input [2]: [d_date_sk#12, d_week_seq#33] +Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#56, [id=#57])) AND isnotnull(d_date_sk#12)) -(102) Project [codegen id : 63] -Output [1]: [d_date_sk#74] -Input [2]: [d_date_sk#74, d_week_seq#75] +(102) Project [codegen id : 66] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_week_seq#33] (103) BroadcastExchange -Input [1]: [d_date_sk#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] -(104) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_sold_date_sk#71] -Right keys [1]: [d_date_sk#74] +(104) BroadcastHashJoin [codegen id : 88] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None -(105) Project [codegen id : 84] -Output [3]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70] -Input [5]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, ss_sold_date_sk#71, d_date_sk#74] +(105) Project [codegen id : 88] +Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (106) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] +Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(107) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#68] -Right keys [1]: [i_item_sk#79] +(107) BroadcastHashJoin [codegen id : 88] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#59] Join condition: None -(108) Project [codegen id : 84] -Output [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#80, i_class_id#81, i_category_id#82] -Input [7]: [ss_item_sk#68, ss_quantity#69, ss_list_price#70, i_item_sk#79, i_brand_id#80, i_class_id#81, i_category_id#82] +(108) Project [codegen id : 88] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(109) HashAggregate [codegen id : 84] -Input [5]: [ss_quantity#69, ss_list_price#70, i_brand_id#80, i_class_id#81, i_category_id#82] -Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85] -Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] +(109) HashAggregate [codegen id : 88] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] +Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65] +Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] (110) Exchange -Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] -Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] +Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [id=#69] -(111) HashAggregate [codegen id : 85] -Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88] -Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90, count(1)#91] -Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sales#92, count(1)#91 AS number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#90 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] +(111) HashAggregate [codegen id : 89] +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] +Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70, count(1)#71] +Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sales#72, count(1)#71 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] -(112) Filter [codegen id : 85] -Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#65, [id=#66] as decimal(32,6)))) +(112) Filter [codegen id : 89] +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(113) Project [codegen id : 85] -Output [6]: [store AS channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] -Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#69 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#70 as decimal(12,2)))), DecimalType(18,2), true))#94] +(113) Project [codegen id : 89] +Output [6]: [store AS channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] (114) BroadcastExchange -Input [6]: [channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#96] +Input [6]: [channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#76] -(115) BroadcastHashJoin [codegen id : 86] +(115) BroadcastHashJoin [codegen id : 90] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82] +Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Join condition: None (116) TakeOrderedAndProject -Input [12]: [channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] -Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#62, number_sales#63, channel#95, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93] +Input [12]: [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#65, [id=#66] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#51, [id=#52] * HashAggregate (139) +- Exchange (138) +- * HashAggregate (137) @@ -677,7 +677,7 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#97)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#77)] ReadSchema: struct (118) ColumnarToRow [codegen id : 2] @@ -703,7 +703,7 @@ Input [2]: [d_date_sk#12, d_year#13] (123) BroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] (124) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#4] @@ -711,83 +711,83 @@ Right keys [1]: [d_date_sk#12] Join condition: None (125) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#99, ss_list_price#3 AS list_price#100] +Output [2]: [ss_quantity#2 AS quantity#79, ss_list_price#3 AS list_price#80] Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (126) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21] +Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#97)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#77)] ReadSchema: struct (127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21] +Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] (128) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#103] +Output [1]: [d_date_sk#12] (129) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#103] +Right keys [1]: [d_date_sk#12] Join condition: None (130) Project [codegen id : 4] -Output [2]: [cs_quantity#101 AS quantity#104, cs_list_price#102 AS list_price#105] -Input [4]: [cs_quantity#101, cs_list_price#102, cs_sold_date_sk#21, d_date_sk#103] +Output [2]: [cs_quantity#81 AS quantity#83, cs_list_price#82 AS list_price#84] +Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21, d_date_sk#12] (131) Scan parquet default.web_sales -Output [3]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36] +Output [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#97)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#77)] ReadSchema: struct (132) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36] +Input [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] (133) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#108] +Output [1]: [d_date_sk#12] (134) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#108] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None (135) Project [codegen id : 6] -Output [2]: [ws_quantity#106 AS quantity#109, ws_list_price#107 AS list_price#110] -Input [4]: [ws_quantity#106, ws_list_price#107, ws_sold_date_sk#36, d_date_sk#108] +Output [2]: [ws_quantity#85 AS quantity#87, ws_list_price#86 AS list_price#88] +Input [4]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27, d_date_sk#12] (136) Union (137) HashAggregate [codegen id : 7] -Input [2]: [quantity#99, list_price#100] +Input [2]: [quantity#79, list_price#80] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#111, count#112] -Results [2]: [sum#113, count#114] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#89, count#90] +Results [2]: [sum#91, count#92] (138) Exchange -Input [2]: [sum#113, count#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] +Input [2]: [sum#91, count#92] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] (139) HashAggregate [codegen id : 8] -Input [2]: [sum#113, count#114] +Input [2]: [sum#91, count#92] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#116] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#99 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#100 as decimal(12,2)))), DecimalType(18,2), true))#116 AS average_sales#117] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94 AS average_sales#95] -Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#97 +Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#77 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 123] Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#97 +Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#77 -Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#97 +Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#77 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (141) @@ -803,11 +803,11 @@ ReusedExchange (142) (142) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 -Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#34, [id=#35] * Project (146) +- * Filter (145) +- * ColumnarToRow (144) @@ -815,33 +815,33 @@ Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquer (143) Scan parquet default.date_dim -Output [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] (145) Filter [codegen id : 1] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#13 = 1999)) AND (d_moy#118 = 12)) AND (d_dom#119 = 16)) +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 1999)) AND (d_moy#96 = 12)) AND (d_dom#97 = 16)) (146) Project [codegen id : 1] -Output [1]: [d_week_seq#47] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Output [1]: [d_week_seq#33] +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] -Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#65, [id=#66] +Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] -Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#71 IN dynamicpruning#72 +Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#54 ReusedExchange (147) (147) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#74] +Output [1]: [d_date_sk#12] -Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#76, [id=#77] +Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#56, [id=#57] * Project (151) +- * Filter (150) +- * ColumnarToRow (149) @@ -849,21 +849,21 @@ Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subqu (148) Scan parquet default.date_dim -Output [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (149) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] (150) Filter [codegen id : 1] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#13 = 1998)) AND (d_moy#118 = 12)) AND (d_dom#119 = 16)) +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 1998)) AND (d_moy#96 = 12)) AND (d_dom#97 = 16)) (151) Project [codegen id : 1] -Output [1]: [d_week_seq#47] -Input [4]: [d_week_seq#47, d_year#13, d_moy#118, d_dom#119] +Output [1]: [d_week_seq#33] +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt index 1b41abbc97b06..1d5907e319658 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (86) + WholeStageCodegen (90) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] @@ -49,7 +49,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (42) + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -68,11 +68,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #3 - WholeStageCodegen (20) + WholeStageCodegen (21) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 - WholeStageCodegen (19) + WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -81,82 +81,84 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (18) + WholeStageCodegen (19) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (17) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (12) + WholeStageCodegen (13) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (10) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #9 + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #10 - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (9) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (7) - Filter [i_item_sk] + Exchange [brand_id,class_id,category_id] #9 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #10 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (16) + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #10 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (8) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (17) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (15) + WholeStageCodegen (16) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -167,12 +169,12 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #9 + ReusedExchange [d_date_sk] #10 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter BroadcastExchange #3 - WholeStageCodegen (21) + WholeStageCodegen (22) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #3 @@ -188,29 +190,29 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #15 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (23) + WholeStageCodegen (24) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #16 - WholeStageCodegen (22) + WholeStageCodegen (23) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (41) + WholeStageCodegen (43) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #20 - WholeStageCodegen (85) + WholeStageCodegen (89) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (84) + WholeStageCodegen (88) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -218,24 +220,24 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] - WholeStageCodegen (44) + WholeStageCodegen (46) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #22 - WholeStageCodegen (43) + WholeStageCodegen (45) Filter [ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #6 ReusedExchange [d_date_sk] #23 - WholeStageCodegen (62) + WholeStageCodegen (65) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #23 - WholeStageCodegen (63) + WholeStageCodegen (66) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 4c62de561c263..258224e574cfb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -27,36 +27,36 @@ TakeOrderedAndProject (100) : : : :- * HashAggregate (39) : : : : +- Exchange (38) : : : : +- * HashAggregate (37) - : : : : +- * Project (36) - : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : :- * Project (33) - : : : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet default.store_sales (7) - : : : : : +- BroadcastExchange (31) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Project (28) - : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet default.item (16) - : : : : : +- BroadcastExchange (26) - : : : : : +- * Project (25) - : : : : : +- * Filter (24) - : : : : : +- * ColumnarToRow (23) - : : : : : +- Scan parquet default.date_dim (22) - : : : : +- ReusedExchange (34) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : :- * Project (22) + : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : :- * Project (15) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (13) + : : : : : : +- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (20) + : : : : : +- * Project (19) + : : : : : +- * Filter (18) + : : : : : +- * ColumnarToRow (17) + : : : : : +- Scan parquet default.date_dim (16) + : : : : +- BroadcastExchange (35) + : : : : +- * Project (34) + : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * Project (31) + : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : :- * Filter (25) + : : : : : : +- * ColumnarToRow (24) + : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Filter (28) + : : : : : +- * ColumnarToRow (27) + : : : : : +- Scan parquet default.item (26) + : : : : +- ReusedExchange (32) : : : +- BroadcastExchange (49) : : : +- * Project (48) : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -146,226 +146,226 @@ Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) (10) Scan parquet default.item -Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +(11) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(12) Filter [codegen id : 4] -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14)) +(12) Filter [codegen id : 1] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(13) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +(13) BroadcastExchange +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] +(14) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join condition: None -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#15) +(15) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(16) Scan parquet default.item -Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +(16) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_year#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +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 -(17) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +(17) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#12, d_year#13] -(18) Filter [codegen id : 1] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Condition : isnotnull(i_item_sk#17) +(18) Filter [codegen id : 2] +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) -(19) BroadcastExchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +(19) Project [codegen id : 2] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#15] -Right keys [1]: [i_item_sk#17] +(20) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] + +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20] -Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +(22) Project [codegen id : 6] +Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] +Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] -(22) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_year#23] +(23) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] 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 - -(23) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#22, d_year#23] +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(24) Filter [codegen id : 2] -Input [2]: [d_date_sk#22, d_year#23] -Condition : (((isnotnull(d_year#23) AND (d_year#23 >= 1998)) AND (d_year#23 <= 2000)) AND isnotnull(d_date_sk#22)) +(24) ColumnarToRow [codegen id : 5] +Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -(25) Project [codegen id : 2] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_year#23] +(25) Filter [codegen id : 5] +Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +Condition : isnotnull(cs_item_sk#18) -(26) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +(26) Scan parquet default.item +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(27) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#22] -Join condition: None +(27) ColumnarToRow [codegen id : 3] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(28) Project [codegen id : 3] -Output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] -Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#22] +(28) Filter [codegen id : 3] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : isnotnull(i_item_sk#6) (29) BroadcastExchange -Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] -(30) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)] -Right keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +(30) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#6] Join condition: None -(31) BroadcastExchange -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +(31) Project [codegen id : 5] +Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#11] +(32) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] + +(33) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None -(33) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +(34) Project [codegen id : 5] +Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] -(34) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#27] +(35) BroadcastExchange +Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] -(35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +(36) BroadcastHashJoin [codegen id : 6] +Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] +Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None -(36) Project [codegen id : 6] -Output [3]: [i_brand_id#12 AS brand_id#28, i_class_id#13 AS class_id#29, i_category_id#14 AS category_id#30] -Input [5]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#27] - (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#15, class_id#16, category_id#17] (38) Exchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#15, class_id#16, category_id#17] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_item_sk#32) +Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Condition : isnotnull(ws_item_sk#23) -(43) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +(43) ReusedExchange [Reuses operator id: 29] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#34] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [i_item_sk#6] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37] -Input [6]: [ws_item_sk#32, ws_sold_date_sk#33, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(46) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#38] +(46) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] -Input [5]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] +Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (49) BroadcastExchange -Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#39] +Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] -Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] +Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] +Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#15, class_id#16, category_id#17] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#15, class_id#16, category_id#17] (53) BroadcastExchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#40] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#28, class_id#29, category_id#30] +Right keys [3]: [brand_id#15, class_id#16, category_id#17] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#41] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] +Output [1]: [i_item_sk#6 AS ss_item_sk#27] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] (56) BroadcastExchange -Input [1]: [ss_item_sk#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [ss_item_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#41] +Right keys [1]: [ss_item_sk#27] Join condition: None (58) Scan parquet default.item @@ -383,16 +383,16 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#27] (62) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#41] +Right keys [1]: [ss_item_sk#27] Join condition: None (63) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] @@ -404,170 +404,170 @@ Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_ Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (66) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_week_seq#44] +Output [2]: [d_date_sk#12, d_week_seq#30] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [2]: [d_date_sk#27, d_week_seq#44] +Input [2]: [d_date_sk#12, d_week_seq#30] (68) Filter [codegen id : 24] -Input [2]: [d_date_sk#27, d_week_seq#44] -Condition : ((isnotnull(d_week_seq#44) AND (d_week_seq#44 = Subquery scalar-subquery#45, [id=#46])) AND isnotnull(d_date_sk#27)) +Input [2]: [d_date_sk#12, d_week_seq#30] +Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#31, [id=#32])) AND isnotnull(d_date_sk#12)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#27] -Input [2]: [d_date_sk#27, d_week_seq#44] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_week_seq#30] (70) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None (72) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#27] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (73) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#48, isEmpty#49, count#50] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] +Aggregate Attributes [3]: [sum#34, isEmpty#35, count#36] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#40] (75) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#51, isEmpty#52, count#53] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55, count(1)#56] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sales#57, count(1)#56 AS number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#55 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41, count(1)#42] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sales#43, count(1)#42 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] (76) Filter [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59 as decimal(32,6)) > cast(Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45 as decimal(32,6)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) (77) Project [codegen id : 52] -Output [6]: [store AS channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#59] +Output [6]: [store AS channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] (78) Scan parquet default.store_sales -Output [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] +Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#49)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (80) Filter [codegen id : 50] -Input [4]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66] -Condition : isnotnull(ss_item_sk#63) +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#27] (82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#63] -Right keys [1]: [ss_item_sk#41] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [ss_item_sk#27] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] +Output [4]: [i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] (84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#63] -Right keys [1]: [i_item_sk#68] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#50] Join condition: None (85) Project [codegen id : 50] -Output [6]: [ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_brand_id#69, i_class_id#70, i_category_id#71] -Input [8]: [ss_item_sk#63, ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_item_sk#68, i_brand_id#69, i_class_id#70, i_category_id#71] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#72, d_week_seq#73] +Output [2]: [d_date_sk#12, d_week_seq#30] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#72, d_week_seq#73] +Input [2]: [d_date_sk#12, d_week_seq#30] (88) Filter [codegen id : 49] -Input [2]: [d_date_sk#72, d_week_seq#73] -Condition : ((isnotnull(d_week_seq#73) AND (d_week_seq#73 = Subquery scalar-subquery#74, [id=#75])) AND isnotnull(d_date_sk#72)) +Input [2]: [d_date_sk#12, d_week_seq#30] +Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#54, [id=#55])) AND isnotnull(d_date_sk#12)) (89) Project [codegen id : 49] -Output [1]: [d_date_sk#72] -Input [2]: [d_date_sk#72, d_week_seq#73] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_week_seq#30] (90) BroadcastExchange -Input [1]: [d_date_sk#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] (91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#66] -Right keys [1]: [d_date_sk#72] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (92) Project [codegen id : 50] -Output [5]: [ss_quantity#64, ss_list_price#65, i_brand_id#69, i_class_id#70, i_category_id#71] -Input [7]: [ss_quantity#64, ss_list_price#65, ss_sold_date_sk#66, i_brand_id#69, i_class_id#70, i_category_id#71, d_date_sk#72] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53, d_date_sk#12] (93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#64, ss_list_price#65, i_brand_id#69, i_class_id#70, i_category_id#71] -Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#77, isEmpty#78, count#79] -Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] +Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#57, isEmpty#58, count#59] +Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] (94) Exchange -Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] -Arguments: hashpartitioning(i_brand_id#69, i_class_id#70, i_category_id#71, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] +Arguments: hashpartitioning(i_brand_id#51, i_class_id#52, i_category_id#53, 5), ENSURE_REQUIREMENTS, [id=#63] (95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum#80, isEmpty#81, count#82] -Keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84, count(1)#85] -Results [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sales#86, count(1)#85 AS number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#84 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] +Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] +Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] +Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] (96) Filter [codegen id : 51] -Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#60, [id=#61] as decimal(32,6)))) +Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) (97) Project [codegen id : 51] -Output [6]: [store AS channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] -Input [6]: [i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#64 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#65 as decimal(12,2)))), DecimalType(18,2), true))#88] +Output [6]: [store AS channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] (98) BroadcastExchange -Input [6]: [channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#90] +Input [6]: [channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] (99) BroadcastHashJoin [codegen id : 52] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [i_brand_id#69, i_class_id#70, i_category_id#71] +Right keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] Join condition: None (100) TakeOrderedAndProject -Input [12]: [channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] -Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#62, i_brand_id#7, i_class_id#8, i_category_id#9, sales#57, number_sales#58, channel#89, i_brand_id#69, i_class_id#70, i_category_id#71, sales#86, number_sales#87] +Input [12]: [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#60, [id=#61] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#46, [id=#47] * HashAggregate (123) +- Exchange (122) +- * HashAggregate (121) @@ -597,137 +597,137 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#91)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#71)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (103) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_year#92] +Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#92] +Input [2]: [d_date_sk#12, d_year#13] (105) Filter [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#92] -Condition : (((isnotnull(d_year#92) AND (d_year#92 >= 1998)) AND (d_year#92 <= 2000)) AND isnotnull(d_date_sk#27)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) (106) Project [codegen id : 1] -Output [1]: [d_date_sk#27] -Input [2]: [d_date_sk#27, d_year#92] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (107) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] (108) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None (109) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#94, ss_list_price#3 AS list_price#95] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#27] +Output [2]: [ss_quantity#2 AS quantity#73, ss_list_price#3 AS list_price#74] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (110) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16] +Output [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#91)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#71)] ReadSchema: struct (111) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16] +Input [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] (112) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#98] +Output [1]: [d_date_sk#12] (113) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#98] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None (114) Project [codegen id : 4] -Output [2]: [cs_quantity#96 AS quantity#99, cs_list_price#97 AS list_price#100] -Input [4]: [cs_quantity#96, cs_list_price#97, cs_sold_date_sk#16, d_date_sk#98] +Output [2]: [cs_quantity#75 AS quantity#77, cs_list_price#76 AS list_price#78] +Input [4]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19, d_date_sk#12] (115) Scan parquet default.web_sales -Output [3]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33] +Output [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#91)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#71)] ReadSchema: struct (116) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33] +Input [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] (117) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#103] +Output [1]: [d_date_sk#12] (118) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#103] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (119) Project [codegen id : 6] -Output [2]: [ws_quantity#101 AS quantity#104, ws_list_price#102 AS list_price#105] -Input [4]: [ws_quantity#101, ws_list_price#102, ws_sold_date_sk#33, d_date_sk#103] +Output [2]: [ws_quantity#79 AS quantity#81, ws_list_price#80 AS list_price#82] +Input [4]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24, d_date_sk#12] (120) Union (121) HashAggregate [codegen id : 7] -Input [2]: [quantity#94, list_price#95] +Input [2]: [quantity#73, list_price#74] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#106, count#107] -Results [2]: [sum#108, count#109] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#83, count#84] +Results [2]: [sum#85, count#86] (122) Exchange -Input [2]: [sum#108, count#109] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] +Input [2]: [sum#85, count#86] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#87] (123) HashAggregate [codegen id : 8] -Input [2]: [sum#108, count#109] +Input [2]: [sum#85, count#86] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))#111] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#94 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#95 as decimal(12,2)))), DecimalType(18,2), true))#111 AS average_sales#112] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88 AS average_sales#89] -Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#91 +Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#71 ReusedExchange (124) (124) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#91 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#71 -Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#91 +Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#71 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#12] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 ReusedExchange (126) -(126) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#27] +(126) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 -Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#45, [id=#46] +Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#31, [id=#32] * Project (130) +- * Filter (129) +- * ColumnarToRow (128) @@ -735,33 +735,33 @@ Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer (127) Scan parquet default.date_dim -Output [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (128) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] (129) Filter [codegen id : 1] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] -Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#92 = 1999)) AND (d_moy#113 = 12)) AND (d_dom#114 = 16)) +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 1999)) AND (d_moy#90 = 12)) AND (d_dom#91 = 16)) (130) Project [codegen id : 1] -Output [1]: [d_week_seq#44] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Output [1]: [d_week_seq#30] +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] -Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#60, [id=#61] +Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] -Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67 +Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#49 ReusedExchange (131) (131) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#72] +Output [1]: [d_date_sk#12] -Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#74, [id=#75] +Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#54, [id=#55] * Project (135) +- * Filter (134) +- * ColumnarToRow (133) @@ -769,21 +769,21 @@ Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subque (132) Scan parquet default.date_dim -Output [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (133) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] (134) Filter [codegen id : 1] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] -Condition : (((((isnotnull(d_year#92) AND isnotnull(d_moy#113)) AND isnotnull(d_dom#114)) AND (d_year#92 = 1998)) AND (d_moy#113 = 12)) AND (d_dom#114 = 16)) +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 1998)) AND (d_moy#90 = 12)) AND (d_dom#91 = 16)) (135) Project [codegen id : 1] -Output [1]: [d_week_seq#44] -Input [4]: [d_week_seq#44, d_year#92, d_moy#113, d_dom#114] +Output [1]: [d_week_seq#30] +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index cb9b410ea8568..3507d76bcf9b9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -82,53 +82,53 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Exchange [brand_id,class_id,category_id] #5 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (1) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #8 + BroadcastExchange #9 WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Filter [i_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #6 + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 WholeStageCodegen (9) 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 26141bdb911ca..9b47400d3b780 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 @@ -40,39 +40,39 @@ TakeOrderedAndProject (213) : : : : : +- * HashAggregate (44) : : : : : +- Exchange (43) : : : : : +- * HashAggregate (42) - : : : : : +- * Project (41) - : : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (39) - : : : : : +- SortMergeJoin LeftSemi (38) - : : : : : :- * Sort (23) - : : : : : : +- Exchange (22) - : : : : : : +- * Filter (21) - : : : : : : +- * ColumnarToRow (20) - : : : : : : +- Scan parquet default.item (19) - : : : : : +- * Sort (37) - : : : : : +- Exchange (36) - : : : : : +- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (29) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : : :- * Filter (26) - : : : : : : : +- * ColumnarToRow (25) - : : : : : : : +- Scan parquet default.catalog_sales (24) - : : : : : : +- ReusedExchange (27) - : : : : : +- BroadcastExchange (33) - : : : : : +- * Filter (32) - : : : : : +- * ColumnarToRow (31) - : : : : : +- Scan parquet default.item (30) + : : : : : +- SortMergeJoin LeftSemi (41) + : : : : : :- * Sort (26) + : : : : : : +- Exchange (25) + : : : : : : +- * Project (24) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : : :- * Project (18) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : : :- * Filter (11) + : : : : : : : : +- * ColumnarToRow (10) + : : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : : +- BroadcastExchange (16) + : : : : : : : +- * Project (15) + : : : : : : : +- * Filter (14) + : : : : : : : +- * ColumnarToRow (13) + : : : : : : : +- Scan parquet default.date_dim (12) + : : : : : : +- BroadcastExchange (22) + : : : : : : +- * Filter (21) + : : : : : : +- * ColumnarToRow (20) + : : : : : : +- Scan parquet default.item (19) + : : : : : +- * Sort (40) + : : : : : +- Exchange (39) + : : : : : +- * Project (38) + : : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : : :- * Project (32) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : : :- * Filter (29) + : : : : : : : +- * ColumnarToRow (28) + : : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : : +- ReusedExchange (30) + : : : : : +- BroadcastExchange (36) + : : : : : +- * Filter (35) + : : : : : +- * ColumnarToRow (34) + : : : : : +- Scan parquet default.item (33) : : : : +- * Sort (57) : : : : +- Exchange (56) : : : : +- * Project (55) @@ -244,10 +244,10 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 19] +(7) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 19] +(8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) @@ -259,10 +259,10 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 10] +(10) ColumnarToRow [codegen id : 5] Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -(11) Filter [codegen id : 10] +(11) Filter [codegen id : 5] Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) @@ -288,38 +288,51 @@ Input [2]: [d_date_sk#12, d_year#13] Input [1]: [d_date_sk#12] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(17) BroadcastHashJoin [codegen id : 10] +(17) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#12] Join condition: None -(18) Project [codegen id : 10] +(18) Project [codegen id : 5] Output [1]: [ss_item_sk#1] Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] (19) Scan parquet default.item -Output [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Condition : (((isnotnull(i_item_sk#15) AND isnotnull(i_brand_id#16)) AND isnotnull(i_class_id#17)) AND isnotnull(i_category_id#18)) +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) + +(22) BroadcastExchange +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] + +(23) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#7] +Join condition: None -(22) Exchange -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: hashpartitioning(coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] +(24) Project [codegen id : 5] +Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] +Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(23) Sort [codegen id : 5] -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: [coalesce(i_brand_id#16, 0) ASC NULLS FIRST, isnull(i_brand_id#16) ASC NULLS FIRST, coalesce(i_class_id#17, 0) ASC NULLS FIRST, isnull(i_class_id#17) ASC NULLS FIRST, coalesce(i_category_id#18, 0) ASC NULLS FIRST, isnull(i_category_id#18) ASC NULLS FIRST], false, 0 +(25) Exchange +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] -(24) Scan parquet default.catalog_sales +(26) Sort [codegen id : 6] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 + +(27) Scan parquet default.catalog_sales Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] @@ -327,228 +340,215 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 8] +(28) ColumnarToRow [codegen id : 9] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -(26) Filter [codegen id : 8] +(29) Filter [codegen id : 9] Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] Condition : isnotnull(cs_item_sk#20) -(27) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#22] +(30) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#12] -(28) BroadcastHashJoin [codegen id : 8] +(31) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#12] Join condition: None -(29) Project [codegen id : 8] +(32) Project [codegen id : 9] Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#22] +Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] -(30) Scan parquet default.item -Output [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +(33) Scan parquet default.item +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 7] -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(32) Filter [codegen id : 7] -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -Condition : isnotnull(i_item_sk#23) +(35) Filter [codegen id : 8] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : isnotnull(i_item_sk#7) -(33) BroadcastExchange -Input [4]: [i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +(36) BroadcastExchange +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(34) BroadcastHashJoin [codegen id : 8] +(37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#23] +Right keys [1]: [i_item_sk#7] Join condition: None -(35) Project [codegen id : 8] -Output [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Input [5]: [cs_item_sk#20, i_item_sk#23, i_brand_id#24, i_class_id#25, i_category_id#26] - -(36) Exchange -Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: hashpartitioning(coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26), 5), ENSURE_REQUIREMENTS, [id=#28] +(38) Project [codegen id : 9] +Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(37) Sort [codegen id : 9] -Input [3]: [i_brand_id#24, i_class_id#25, i_category_id#26] -Arguments: [coalesce(i_brand_id#24, 0) ASC NULLS FIRST, isnull(i_brand_id#24) ASC NULLS FIRST, coalesce(i_class_id#25, 0) ASC NULLS FIRST, isnull(i_class_id#25) ASC NULLS FIRST, coalesce(i_category_id#26, 0) ASC NULLS FIRST, isnull(i_category_id#26) ASC NULLS FIRST], false, 0 - -(38) SortMergeJoin -Left keys [6]: [coalesce(i_brand_id#16, 0), isnull(i_brand_id#16), coalesce(i_class_id#17, 0), isnull(i_class_id#17), coalesce(i_category_id#18, 0), isnull(i_category_id#18)] -Right keys [6]: [coalesce(i_brand_id#24, 0), isnull(i_brand_id#24), coalesce(i_class_id#25, 0), isnull(i_class_id#25), coalesce(i_category_id#26, 0), isnull(i_category_id#26)] -Join condition: None +(39) Exchange +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] -(39) BroadcastExchange -Input [4]: [i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +(40) Sort [codegen id : 10] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#15] +(41) SortMergeJoin +Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] +Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] Join condition: None -(41) Project [codegen id : 10] -Output [3]: [i_brand_id#16 AS brand_id#30, i_class_id#17 AS class_id#31, i_category_id#18 AS category_id#32] -Input [5]: [ss_item_sk#1, i_item_sk#15, i_brand_id#16, i_class_id#17, i_category_id#18] - -(42) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +(42) HashAggregate [codegen id : 11] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#16, class_id#17, category_id#18] (43) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] -(44) HashAggregate [codegen id : 11] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +(44) HashAggregate [codegen id : 12] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#16, class_id#17, category_id#18] (45) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32), 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] -(46) Sort [codegen id : 12] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: [coalesce(brand_id#30, 0) ASC NULLS FIRST, isnull(brand_id#30) ASC NULLS FIRST, coalesce(class_id#31, 0) ASC NULLS FIRST, isnull(class_id#31) ASC NULLS FIRST, coalesce(category_id#32, 0) ASC NULLS FIRST, isnull(category_id#32) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 13] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#35, ws_sold_date_sk#36] +Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 15] -Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] +(48) ColumnarToRow [codegen id : 16] +Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -(49) Filter [codegen id : 15] -Input [2]: [ws_item_sk#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#35) +(49) Filter [codegen id : 16] +Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Condition : isnotnull(ws_item_sk#26) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#37] +Output [1]: [d_date_sk#12] -(51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#37] +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None -(52) Project [codegen id : 15] -Output [1]: [ws_item_sk#35] -Input [3]: [ws_item_sk#35, ws_sold_date_sk#36, d_date_sk#37] +(52) Project [codegen id : 16] +Output [1]: [ws_item_sk#26] +Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] -(53) ReusedExchange [Reuses operator id: 33] -Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(53) ReusedExchange [Reuses operator id: 36] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(54) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [i_item_sk#38] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [i_item_sk#7] Join condition: None -(55) Project [codegen id : 15] -Output [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Input [5]: [ws_item_sk#35, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +(55) Project [codegen id : 16] +Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (56) Exchange -Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: hashpartitioning(coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41), 5), ENSURE_REQUIREMENTS, [id=#42] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] -(57) Sort [codegen id : 16] -Input [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] -Arguments: [coalesce(i_brand_id#39, 0) ASC NULLS FIRST, isnull(i_brand_id#39) ASC NULLS FIRST, coalesce(i_class_id#40, 0) ASC NULLS FIRST, isnull(i_class_id#40) ASC NULLS FIRST, coalesce(i_category_id#41, 0) ASC NULLS FIRST, isnull(i_category_id#41) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 17] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] -Right keys [6]: [coalesce(i_brand_id#39, 0), isnull(i_brand_id#39), coalesce(i_class_id#40, 0), isnull(i_class_id#40), coalesce(i_category_id#41, 0), isnull(i_category_id#41)] +Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] +Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] Join condition: None -(59) HashAggregate [codegen id : 17] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +(59) HashAggregate [codegen id : 18] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#16, class_id#17, category_id#18] (60) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] -(61) HashAggregate [codegen id : 18] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +(61) HashAggregate [codegen id : 19] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#16, class_id#17, category_id#18] (62) BroadcastExchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#44] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] -(63) BroadcastHashJoin [codegen id : 19] +(63) BroadcastHashJoin [codegen id : 20] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#30, class_id#31, category_id#32] +Right keys [3]: [brand_id#16, class_id#17, category_id#18] Join condition: None -(64) Project [codegen id : 19] -Output [1]: [i_item_sk#7 AS ss_item_sk#45] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#30, class_id#31, category_id#32] +(64) Project [codegen id : 20] +Output [1]: [i_item_sk#7 AS ss_item_sk#31] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] (65) Exchange -Input [1]: [ss_item_sk#45] -Arguments: hashpartitioning(ss_item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [1]: [ss_item_sk#31] +Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] -(66) Sort [codegen id : 20] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 21] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#31] Join condition: None (68) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#47] +Output [3]: [d_date_sk#12, d_year#13, d_moy#33] 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 -(69) ColumnarToRow [codegen id : 21] -Input [3]: [d_date_sk#12, d_year#13, d_moy#47] +(69) ColumnarToRow [codegen id : 22] +Input [3]: [d_date_sk#12, d_year#13, d_moy#33] -(70) Filter [codegen id : 21] -Input [3]: [d_date_sk#12, d_year#13, d_moy#47] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#47)) AND (d_year#13 = 2000)) AND (d_moy#47 = 11)) AND isnotnull(d_date_sk#12)) +(70) Filter [codegen id : 22] +Input [3]: [d_date_sk#12, d_year#13, d_moy#33] +Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#33)) AND (d_year#13 = 2000)) AND (d_moy#33 = 11)) AND isnotnull(d_date_sk#12)) -(71) Project [codegen id : 21] +(71) Project [codegen id : 22] Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#47] +Input [3]: [d_date_sk#12, d_year#13, d_moy#33] (72) BroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] -(73) BroadcastHashJoin [codegen id : 42] +(73) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_sold_date_sk#4] Right keys [1]: [d_date_sk#12] Join condition: None -(74) Project [codegen id : 42] +(74) Project [codegen id : 44] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] @@ -559,657 +559,657 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 22] +(76) ColumnarToRow [codegen id : 23] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(77) Filter [codegen id : 22] +(77) Filter [codegen id : 23] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : isnotnull(i_item_sk#7) (78) Exchange Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#49] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#35] -(79) Sort [codegen id : 23] +(79) Sort [codegen id : 24] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#31] -(81) Sort [codegen id : 41] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 43] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (82) SortMergeJoin Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#31] Join condition: None (83) BroadcastExchange Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -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=#36] -(84) BroadcastHashJoin [codegen id : 42] +(84) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_item_sk#1] Right keys [1]: [i_item_sk#7] Join condition: None -(85) Project [codegen id : 42] +(85) Project [codegen id : 44] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(86) HashAggregate [codegen id : 42] +(86) HashAggregate [codegen id : 44] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#51, isEmpty#52, count#53] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] +Aggregate Attributes [3]: [sum#37, isEmpty#38, count#39] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#43] -(88) HashAggregate [codegen id : 43] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#54, isEmpty#55, count#56] +(88) HashAggregate [codegen id : 45] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58, count(1)#59] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sales#60, count(1)#59 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44, count(1)#45] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sales#46, count(1)#45 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] -(89) Filter [codegen id : 43] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 as decimal(32,6)) > cast(Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(89) Filter [codegen id : 45] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48 as decimal(32,6)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(90) Project [codegen id : 43] -Output [6]: [store AS channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62] +(90) Project [codegen id : 45] +Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] (91) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Output [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 44] -Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +(92) ColumnarToRow [codegen id : 46] +Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -(93) Filter [codegen id : 44] -Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +(93) Filter [codegen id : 46] +Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] Condition : isnotnull(cs_item_sk#20) (94) Exchange -Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] -Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#54] -(95) Sort [codegen id : 45] -Input [4]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +(95) Sort [codegen id : 47] +Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] Arguments: [cs_item_sk#20 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#31] -(97) Sort [codegen id : 63] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 66] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (98) SortMergeJoin Left keys [1]: [cs_item_sk#20] -Right keys [1]: [ss_item_sk#45] +Right keys [1]: [ss_item_sk#31] Join condition: None (99) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#69] +Output [1]: [d_date_sk#12] -(100) BroadcastHashJoin [codegen id : 85] +(100) BroadcastHashJoin [codegen id : 89] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#69] +Right keys [1]: [d_date_sk#12] Join condition: None -(101) Project [codegen id : 85] -Output [3]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67] -Input [5]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21, d_date_sk#69] +(101) Project [codegen id : 89] +Output [3]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53] +Input [5]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] (102) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(103) BroadcastHashJoin [codegen id : 85] +(103) BroadcastHashJoin [codegen id : 89] Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#70] +Right keys [1]: [i_item_sk#7] Join condition: None -(104) Project [codegen id : 85] -Output [5]: [cs_quantity#66, cs_list_price#67, i_brand_id#71, i_class_id#72, i_category_id#73] -Input [7]: [cs_item_sk#20, cs_quantity#66, cs_list_price#67, i_item_sk#70, i_brand_id#71, i_class_id#72, i_category_id#73] +(104) Project [codegen id : 89] +Output [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [7]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(105) HashAggregate [codegen id : 85] -Input [5]: [cs_quantity#66, cs_list_price#67, i_brand_id#71, i_class_id#72, i_category_id#73] -Keys [3]: [i_brand_id#71, i_class_id#72, i_category_id#73] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#74, isEmpty#75, count#76] -Results [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] +(105) HashAggregate [codegen id : 89] +Input [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] (106) Exchange -Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] -Arguments: hashpartitioning(i_brand_id#71, i_class_id#72, i_category_id#73, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#61] -(107) HashAggregate [codegen id : 86] -Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum#77, isEmpty#78, count#79] -Keys [3]: [i_brand_id#71, i_class_id#72, i_category_id#73] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81, count(1)#82] -Results [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sales#83, count(1)#82 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#81 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] +(107) HashAggregate [codegen id : 90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] -(108) Filter [codegen id : 86] -Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(108) Filter [codegen id : 90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(109) Project [codegen id : 86] -Output [6]: [catalog AS channel#86, i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84] -Input [6]: [i_brand_id#71, i_class_id#72, i_category_id#73, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#66 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#67 as decimal(12,2)))), DecimalType(18,2), true))#85] +(109) Project [codegen id : 90] +Output [6]: [catalog AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] (110) Scan parquet default.web_sales -Output [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Output [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 87] -Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +(111) ColumnarToRow [codegen id : 91] +Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -(112) Filter [codegen id : 87] -Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#35) +(112) Filter [codegen id : 91] +Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Condition : isnotnull(ws_item_sk#26) (113) Exchange -Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] -Arguments: hashpartitioning(ws_item_sk#35, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Arguments: hashpartitioning(ws_item_sk#26, 5), ENSURE_REQUIREMENTS, [id=#70] -(114) Sort [codegen id : 88] -Input [4]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] -Arguments: [ws_item_sk#35 ASC NULLS FIRST], false, 0 +(114) Sort [codegen id : 92] +Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Arguments: [ws_item_sk#26 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#45] +Output [1]: [ss_item_sk#31] -(116) Sort [codegen id : 106] -Input [1]: [ss_item_sk#45] -Arguments: [ss_item_sk#45 ASC NULLS FIRST], false, 0 +(116) Sort [codegen id : 111] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (117) SortMergeJoin -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [ss_item_sk#45] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [ss_item_sk#31] Join condition: None (118) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#90] +Output [1]: [d_date_sk#12] -(119) BroadcastHashJoin [codegen id : 128] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#90] +(119) BroadcastHashJoin [codegen id : 134] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None -(120) Project [codegen id : 128] -Output [3]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88] -Input [5]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36, d_date_sk#90] +(120) Project [codegen id : 134] +Output [3]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69] +Input [5]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] (121) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(122) BroadcastHashJoin [codegen id : 128] -Left keys [1]: [ws_item_sk#35] -Right keys [1]: [i_item_sk#91] +(122) BroadcastHashJoin [codegen id : 134] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [i_item_sk#7] Join condition: None -(123) Project [codegen id : 128] -Output [5]: [ws_quantity#87, ws_list_price#88, i_brand_id#92, i_class_id#93, i_category_id#94] -Input [7]: [ws_item_sk#35, ws_quantity#87, ws_list_price#88, i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94] +(123) Project [codegen id : 134] +Output [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [7]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(124) HashAggregate [codegen id : 128] -Input [5]: [ws_quantity#87, ws_list_price#88, i_brand_id#92, i_class_id#93, i_category_id#94] -Keys [3]: [i_brand_id#92, i_class_id#93, i_category_id#94] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#95, isEmpty#96, count#97] -Results [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] +(124) HashAggregate [codegen id : 134] +Input [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#71, isEmpty#72, count#73] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] (125) Exchange -Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] -Arguments: hashpartitioning(i_brand_id#92, i_class_id#93, i_category_id#94, 5), ENSURE_REQUIREMENTS, [id=#101] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#77] -(126) HashAggregate [codegen id : 129] -Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum#98, isEmpty#99, count#100] -Keys [3]: [i_brand_id#92, i_class_id#93, i_category_id#94] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102, count(1)#103] -Results [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sales#104, count(1)#103 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#102 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] +(126) HashAggregate [codegen id : 135] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78, count(1)#79] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sales#80, count(1)#79 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] -(127) Filter [codegen id : 129] -Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(127) Filter [codegen id : 135] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(128) Project [codegen id : 129] -Output [6]: [web AS channel#107, i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105] -Input [6]: [i_brand_id#92, i_class_id#93, i_category_id#94, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#87 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#88 as decimal(12,2)))), DecimalType(18,2), true))#106] +(128) Project [codegen id : 135] +Output [6]: [web AS channel#83, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] (129) Union -(130) HashAggregate [codegen id : 130] -Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sales#60, number_sales#61] -Keys [4]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] -Aggregate Attributes [3]: [sum#108, isEmpty#109, sum#110] -Results [7]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum#111, isEmpty#112, sum#113] +(130) HashAggregate [codegen id : 136] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] +Aggregate Attributes [3]: [sum#84, isEmpty#85, sum#86] +Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#87, isEmpty#88, sum#89] (131) Exchange -Input [7]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum#111, isEmpty#112, sum#113] -Arguments: hashpartitioning(channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#114] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#87, isEmpty#88, sum#89] +Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#90] -(132) HashAggregate [codegen id : 131] -Input [7]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum#111, isEmpty#112, sum#113] -Keys [4]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#60), sum(number_sales#61)] -Aggregate Attributes [2]: [sum(sales#60)#115, sum(number_sales#61)#116] -Results [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum(sales#60)#115 AS sum_sales#117, sum(number_sales#61)#116 AS number_sales#118] +(132) HashAggregate [codegen id : 137] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#87, isEmpty#88, sum#89] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(sales#46), sum(number_sales#47)] +Aggregate Attributes [2]: [sum(sales#46)#91, sum(number_sales#47)#92] +Results [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum(sales#46)#91 AS sum_sales#93, sum(number_sales#47)#92 AS number_sales#94] (133) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sum#122, isEmpty#123, count#124] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#95, isEmpty#96, count#97] -(134) HashAggregate [codegen id : 174] -Input [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sum#122, isEmpty#123, count#124] -Keys [3]: [i_brand_id#119, i_class_id#120, i_category_id#121] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#127, count(1)#128] -Results [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#127 AS sales#60, count(1)#128 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#127 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129] +(134) HashAggregate [codegen id : 182] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#95, isEmpty#96, count#97] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#98, count(1)#99] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#98 AS sales#46, count(1)#99 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#98 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100] -(135) Filter [codegen id : 174] -Input [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(135) Filter [codegen id : 182] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(136) Project [codegen id : 174] -Output [6]: [store AS channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sales#60, number_sales#61] -Input [6]: [i_brand_id#119, i_class_id#120, i_category_id#121, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#125 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#126 as decimal(12,2)))), DecimalType(18,2), true))#129] +(136) Project [codegen id : 182] +Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100] (137) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sum#133, isEmpty#134, count#135] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#101, isEmpty#102, count#103] -(138) HashAggregate [codegen id : 217] -Input [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sum#133, isEmpty#134, count#135] -Keys [3]: [i_brand_id#130, i_class_id#131, i_category_id#132] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#138, count(1)#139] -Results [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sales#83, count(1)#139 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140] +(138) HashAggregate [codegen id : 227] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#101, isEmpty#102, count#103] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#104, count(1)#105] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sales#64, count(1)#105 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106] -(139) Filter [codegen id : 217] -Input [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(139) Filter [codegen id : 227] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(140) Project [codegen id : 217] -Output [6]: [catalog AS channel#86, i_brand_id#130, i_class_id#131, i_category_id#132, sales#83, number_sales#84] -Input [6]: [i_brand_id#130, i_class_id#131, i_category_id#132, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#140] +(140) Project [codegen id : 227] +Output [6]: [catalog AS channel#107, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106] (141) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sum#144, isEmpty#145, count#146] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#108, isEmpty#109, count#110] -(142) HashAggregate [codegen id : 260] -Input [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sum#144, isEmpty#145, count#146] -Keys [3]: [i_brand_id#141, i_class_id#142, i_category_id#143] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#149, count(1)#150] -Results [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#149 AS sales#104, count(1)#150 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151] +(142) HashAggregate [codegen id : 272] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#108, isEmpty#109, count#110] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#111, count(1)#112] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#111 AS sales#80, count(1)#112 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#111 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113] -(143) Filter [codegen id : 260] -Input [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(143) Filter [codegen id : 272] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(144) Project [codegen id : 260] -Output [6]: [web AS channel#107, i_brand_id#141, i_class_id#142, i_category_id#143, sales#104, number_sales#105] -Input [6]: [i_brand_id#141, i_class_id#142, i_category_id#143, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#147 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#148 as decimal(12,2)))), DecimalType(18,2), true))#151] +(144) Project [codegen id : 272] +Output [6]: [web AS channel#114, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113] (145) Union -(146) HashAggregate [codegen id : 261] -Input [6]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sales#60, number_sales#61] -Keys [4]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121] -Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] -Aggregate Attributes [3]: [sum#152, isEmpty#153, sum#154] -Results [7]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sum#155, isEmpty#156, sum#157] +(146) HashAggregate [codegen id : 273] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] +Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] +Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#118, isEmpty#119, sum#120] (147) Exchange -Input [7]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sum#155, isEmpty#156, sum#157] -Arguments: hashpartitioning(channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, 5), ENSURE_REQUIREMENTS, [id=#158] - -(148) HashAggregate [codegen id : 262] -Input [7]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121, sum#155, isEmpty#156, sum#157] -Keys [4]: [channel#65, i_brand_id#119, i_class_id#120, i_category_id#121] -Functions [2]: [sum(sales#60), sum(number_sales#61)] -Aggregate Attributes [2]: [sum(sales#60)#159, sum(number_sales#61)#160] -Results [5]: [channel#65, i_brand_id#119, i_class_id#120, sum(sales#60)#159 AS sum_sales#117, sum(number_sales#61)#160 AS number_sales#118] - -(149) HashAggregate [codegen id : 262] -Input [5]: [channel#65, i_brand_id#119, i_class_id#120, sum_sales#117, number_sales#118] -Keys [3]: [channel#65, i_brand_id#119, i_class_id#120] -Functions [2]: [partial_sum(sum_sales#117), partial_sum(number_sales#118)] -Aggregate Attributes [3]: [sum#161, isEmpty#162, sum#163] -Results [6]: [channel#65, i_brand_id#119, i_class_id#120, sum#164, isEmpty#165, sum#166] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#118, isEmpty#119, sum#120] +Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#121] + +(148) HashAggregate [codegen id : 274] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#118, isEmpty#119, sum#120] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(sales#46), sum(number_sales#47)] +Aggregate Attributes [2]: [sum(sales#46)#122, sum(number_sales#47)#123] +Results [5]: [channel#51, i_brand_id#8, i_class_id#9, sum(sales#46)#122 AS sum_sales#93, sum(number_sales#47)#123 AS number_sales#94] + +(149) HashAggregate [codegen id : 274] +Input [5]: [channel#51, i_brand_id#8, i_class_id#9, sum_sales#93, number_sales#94] +Keys [3]: [channel#51, i_brand_id#8, i_class_id#9] +Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] +Aggregate Attributes [3]: [sum#124, isEmpty#125, sum#126] +Results [6]: [channel#51, i_brand_id#8, i_class_id#9, sum#127, isEmpty#128, sum#129] (150) Exchange -Input [6]: [channel#65, i_brand_id#119, i_class_id#120, sum#164, isEmpty#165, sum#166] -Arguments: hashpartitioning(channel#65, i_brand_id#119, i_class_id#120, 5), ENSURE_REQUIREMENTS, [id=#167] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, sum#127, isEmpty#128, sum#129] +Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, 5), ENSURE_REQUIREMENTS, [id=#130] -(151) HashAggregate [codegen id : 263] -Input [6]: [channel#65, i_brand_id#119, i_class_id#120, sum#164, isEmpty#165, sum#166] -Keys [3]: [channel#65, i_brand_id#119, i_class_id#120] -Functions [2]: [sum(sum_sales#117), sum(number_sales#118)] -Aggregate Attributes [2]: [sum(sum_sales#117)#168, sum(number_sales#118)#169] -Results [6]: [channel#65, i_brand_id#119, i_class_id#120, null AS i_category_id#170, sum(sum_sales#117)#168 AS sum(sum_sales)#171, sum(number_sales#118)#169 AS sum(number_sales)#172] +(151) HashAggregate [codegen id : 275] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, sum#127, isEmpty#128, sum#129] +Keys [3]: [channel#51, i_brand_id#8, i_class_id#9] +Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] +Aggregate Attributes [2]: [sum(sum_sales#93)#131, sum(number_sales#94)#132] +Results [6]: [channel#51, i_brand_id#8, i_class_id#9, null AS i_category_id#133, sum(sum_sales#93)#131 AS sum(sum_sales)#134, sum(number_sales#94)#132 AS sum(number_sales)#135] (152) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sum#176, isEmpty#177, count#178] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#136, isEmpty#137, count#138] -(153) HashAggregate [codegen id : 306] -Input [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sum#176, isEmpty#177, count#178] -Keys [3]: [i_brand_id#173, i_class_id#174, i_category_id#175] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#181, count(1)#182] -Results [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sales#60, count(1)#182 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183] +(153) HashAggregate [codegen id : 320] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#136, isEmpty#137, count#138] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#139, count(1)#140] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sales#46, count(1)#140 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141] -(154) Filter [codegen id : 306] -Input [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(154) Filter [codegen id : 320] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(155) Project [codegen id : 306] -Output [6]: [store AS channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sales#60, number_sales#61] -Input [6]: [i_brand_id#173, i_class_id#174, i_category_id#175, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#179 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#180 as decimal(12,2)))), DecimalType(18,2), true))#183] +(155) Project [codegen id : 320] +Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141] (156) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sum#187, isEmpty#188, count#189] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#142, isEmpty#143, count#144] -(157) HashAggregate [codegen id : 349] -Input [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sum#187, isEmpty#188, count#189] -Keys [3]: [i_brand_id#184, i_class_id#185, i_category_id#186] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#192, count(1)#193] -Results [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sales#83, count(1)#193 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#192 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194] +(157) HashAggregate [codegen id : 365] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#142, isEmpty#143, count#144] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#145, count(1)#146] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sales#64, count(1)#146 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147] -(158) Filter [codegen id : 349] -Input [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(158) Filter [codegen id : 365] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(159) Project [codegen id : 349] -Output [6]: [catalog AS channel#86, i_brand_id#184, i_class_id#185, i_category_id#186, sales#83, number_sales#84] -Input [6]: [i_brand_id#184, i_class_id#185, i_category_id#186, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#190 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#191 as decimal(12,2)))), DecimalType(18,2), true))#194] +(159) Project [codegen id : 365] +Output [6]: [catalog AS channel#148, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147] (160) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sum#198, isEmpty#199, count#200] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#149, isEmpty#150, count#151] -(161) HashAggregate [codegen id : 392] -Input [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sum#198, isEmpty#199, count#200] -Keys [3]: [i_brand_id#195, i_class_id#196, i_category_id#197] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#203, count(1)#204] -Results [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#203 AS sales#104, count(1)#204 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#203 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205] +(161) HashAggregate [codegen id : 410] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#149, isEmpty#150, count#151] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#152, count(1)#153] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#152 AS sales#80, count(1)#153 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#152 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154] -(162) Filter [codegen id : 392] -Input [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(162) Filter [codegen id : 410] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(163) Project [codegen id : 392] -Output [6]: [web AS channel#107, i_brand_id#195, i_class_id#196, i_category_id#197, sales#104, number_sales#105] -Input [6]: [i_brand_id#195, i_class_id#196, i_category_id#197, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#201 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#202 as decimal(12,2)))), DecimalType(18,2), true))#205] +(163) Project [codegen id : 410] +Output [6]: [web AS channel#155, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154] (164) Union -(165) HashAggregate [codegen id : 393] -Input [6]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sales#60, number_sales#61] -Keys [4]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175] -Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] -Aggregate Attributes [3]: [sum#206, isEmpty#207, sum#208] -Results [7]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sum#209, isEmpty#210, sum#211] +(165) HashAggregate [codegen id : 411] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] +Aggregate Attributes [3]: [sum#156, isEmpty#157, sum#158] +Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#159, isEmpty#160, sum#161] (166) Exchange -Input [7]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sum#209, isEmpty#210, sum#211] -Arguments: hashpartitioning(channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, 5), ENSURE_REQUIREMENTS, [id=#212] - -(167) HashAggregate [codegen id : 394] -Input [7]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175, sum#209, isEmpty#210, sum#211] -Keys [4]: [channel#65, i_brand_id#173, i_class_id#174, i_category_id#175] -Functions [2]: [sum(sales#60), sum(number_sales#61)] -Aggregate Attributes [2]: [sum(sales#60)#213, sum(number_sales#61)#214] -Results [4]: [channel#65, i_brand_id#173, sum(sales#60)#213 AS sum_sales#117, sum(number_sales#61)#214 AS number_sales#118] - -(168) HashAggregate [codegen id : 394] -Input [4]: [channel#65, i_brand_id#173, sum_sales#117, number_sales#118] -Keys [2]: [channel#65, i_brand_id#173] -Functions [2]: [partial_sum(sum_sales#117), partial_sum(number_sales#118)] -Aggregate Attributes [3]: [sum#215, isEmpty#216, sum#217] -Results [5]: [channel#65, i_brand_id#173, sum#218, isEmpty#219, sum#220] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#159, isEmpty#160, sum#161] +Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#162] + +(167) HashAggregate [codegen id : 412] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#159, isEmpty#160, sum#161] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(sales#46), sum(number_sales#47)] +Aggregate Attributes [2]: [sum(sales#46)#163, sum(number_sales#47)#164] +Results [4]: [channel#51, i_brand_id#8, sum(sales#46)#163 AS sum_sales#93, sum(number_sales#47)#164 AS number_sales#94] + +(168) HashAggregate [codegen id : 412] +Input [4]: [channel#51, i_brand_id#8, sum_sales#93, number_sales#94] +Keys [2]: [channel#51, i_brand_id#8] +Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] +Aggregate Attributes [3]: [sum#165, isEmpty#166, sum#167] +Results [5]: [channel#51, i_brand_id#8, sum#168, isEmpty#169, sum#170] (169) Exchange -Input [5]: [channel#65, i_brand_id#173, sum#218, isEmpty#219, sum#220] -Arguments: hashpartitioning(channel#65, i_brand_id#173, 5), ENSURE_REQUIREMENTS, [id=#221] +Input [5]: [channel#51, i_brand_id#8, sum#168, isEmpty#169, sum#170] +Arguments: hashpartitioning(channel#51, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [id=#171] -(170) HashAggregate [codegen id : 395] -Input [5]: [channel#65, i_brand_id#173, sum#218, isEmpty#219, sum#220] -Keys [2]: [channel#65, i_brand_id#173] -Functions [2]: [sum(sum_sales#117), sum(number_sales#118)] -Aggregate Attributes [2]: [sum(sum_sales#117)#222, sum(number_sales#118)#223] -Results [6]: [channel#65, i_brand_id#173, null AS i_class_id#224, null AS i_category_id#225, sum(sum_sales#117)#222 AS sum(sum_sales)#226, sum(number_sales#118)#223 AS sum(number_sales)#227] +(170) HashAggregate [codegen id : 413] +Input [5]: [channel#51, i_brand_id#8, sum#168, isEmpty#169, sum#170] +Keys [2]: [channel#51, i_brand_id#8] +Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] +Aggregate Attributes [2]: [sum(sum_sales#93)#172, sum(number_sales#94)#173] +Results [6]: [channel#51, i_brand_id#8, null AS i_class_id#174, null AS i_category_id#175, sum(sum_sales#93)#172 AS sum(sum_sales)#176, sum(number_sales#94)#173 AS sum(number_sales)#177] (171) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sum#231, isEmpty#232, count#233] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#178, isEmpty#179, count#180] -(172) HashAggregate [codegen id : 438] -Input [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sum#231, isEmpty#232, count#233] -Keys [3]: [i_brand_id#228, i_class_id#229, i_category_id#230] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#236, count(1)#237] -Results [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#236 AS sales#60, count(1)#237 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#236 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238] +(172) HashAggregate [codegen id : 458] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#178, isEmpty#179, count#180] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#181, count(1)#182] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sales#46, count(1)#182 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183] -(173) Filter [codegen id : 438] -Input [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(173) Filter [codegen id : 458] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(174) Project [codegen id : 438] -Output [6]: [store AS channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sales#60, number_sales#61] -Input [6]: [i_brand_id#228, i_class_id#229, i_category_id#230, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#234 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#235 as decimal(12,2)))), DecimalType(18,2), true))#238] +(174) Project [codegen id : 458] +Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183] (175) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sum#242, isEmpty#243, count#244] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#184, isEmpty#185, count#186] -(176) HashAggregate [codegen id : 481] -Input [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sum#242, isEmpty#243, count#244] -Keys [3]: [i_brand_id#239, i_class_id#240, i_category_id#241] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#247, count(1)#248] -Results [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#247 AS sales#83, count(1)#248 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#247 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249] +(176) HashAggregate [codegen id : 503] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#184, isEmpty#185, count#186] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#187, count(1)#188] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sales#64, count(1)#188 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189] -(177) Filter [codegen id : 481] -Input [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(177) Filter [codegen id : 503] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(178) Project [codegen id : 481] -Output [6]: [catalog AS channel#86, i_brand_id#239, i_class_id#240, i_category_id#241, sales#83, number_sales#84] -Input [6]: [i_brand_id#239, i_class_id#240, i_category_id#241, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#245 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#246 as decimal(12,2)))), DecimalType(18,2), true))#249] +(178) Project [codegen id : 503] +Output [6]: [catalog AS channel#190, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189] (179) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sum#253, isEmpty#254, count#255] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#191, isEmpty#192, count#193] -(180) HashAggregate [codegen id : 524] -Input [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sum#253, isEmpty#254, count#255] -Keys [3]: [i_brand_id#250, i_class_id#251, i_category_id#252] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#258, count(1)#259] -Results [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#258 AS sales#104, count(1)#259 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#258 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260] +(180) HashAggregate [codegen id : 548] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#191, isEmpty#192, count#193] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#194, count(1)#195] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#194 AS sales#80, count(1)#195 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#194 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196] -(181) Filter [codegen id : 524] -Input [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(181) Filter [codegen id : 548] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(182) Project [codegen id : 524] -Output [6]: [web AS channel#107, i_brand_id#250, i_class_id#251, i_category_id#252, sales#104, number_sales#105] -Input [6]: [i_brand_id#250, i_class_id#251, i_category_id#252, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#256 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#257 as decimal(12,2)))), DecimalType(18,2), true))#260] +(182) Project [codegen id : 548] +Output [6]: [web AS channel#197, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196] (183) Union -(184) HashAggregate [codegen id : 525] -Input [6]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sales#60, number_sales#61] -Keys [4]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230] -Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] -Aggregate Attributes [3]: [sum#261, isEmpty#262, sum#263] -Results [7]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sum#264, isEmpty#265, sum#266] +(184) HashAggregate [codegen id : 549] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] +Aggregate Attributes [3]: [sum#198, isEmpty#199, sum#200] +Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#201, isEmpty#202, sum#203] (185) Exchange -Input [7]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sum#264, isEmpty#265, sum#266] -Arguments: hashpartitioning(channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, 5), ENSURE_REQUIREMENTS, [id=#267] - -(186) HashAggregate [codegen id : 526] -Input [7]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230, sum#264, isEmpty#265, sum#266] -Keys [4]: [channel#65, i_brand_id#228, i_class_id#229, i_category_id#230] -Functions [2]: [sum(sales#60), sum(number_sales#61)] -Aggregate Attributes [2]: [sum(sales#60)#268, sum(number_sales#61)#269] -Results [3]: [channel#65, sum(sales#60)#268 AS sum_sales#117, sum(number_sales#61)#269 AS number_sales#118] - -(187) HashAggregate [codegen id : 526] -Input [3]: [channel#65, sum_sales#117, number_sales#118] -Keys [1]: [channel#65] -Functions [2]: [partial_sum(sum_sales#117), partial_sum(number_sales#118)] -Aggregate Attributes [3]: [sum#270, isEmpty#271, sum#272] -Results [4]: [channel#65, sum#273, isEmpty#274, sum#275] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#201, isEmpty#202, sum#203] +Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#204] + +(186) HashAggregate [codegen id : 550] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#201, isEmpty#202, sum#203] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(sales#46), sum(number_sales#47)] +Aggregate Attributes [2]: [sum(sales#46)#205, sum(number_sales#47)#206] +Results [3]: [channel#51, sum(sales#46)#205 AS sum_sales#93, sum(number_sales#47)#206 AS number_sales#94] + +(187) HashAggregate [codegen id : 550] +Input [3]: [channel#51, sum_sales#93, number_sales#94] +Keys [1]: [channel#51] +Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] +Aggregate Attributes [3]: [sum#207, isEmpty#208, sum#209] +Results [4]: [channel#51, sum#210, isEmpty#211, sum#212] (188) Exchange -Input [4]: [channel#65, sum#273, isEmpty#274, sum#275] -Arguments: hashpartitioning(channel#65, 5), ENSURE_REQUIREMENTS, [id=#276] +Input [4]: [channel#51, sum#210, isEmpty#211, sum#212] +Arguments: hashpartitioning(channel#51, 5), ENSURE_REQUIREMENTS, [id=#213] -(189) HashAggregate [codegen id : 527] -Input [4]: [channel#65, sum#273, isEmpty#274, sum#275] -Keys [1]: [channel#65] -Functions [2]: [sum(sum_sales#117), sum(number_sales#118)] -Aggregate Attributes [2]: [sum(sum_sales#117)#277, sum(number_sales#118)#278] -Results [6]: [channel#65, null AS i_brand_id#279, null AS i_class_id#280, null AS i_category_id#281, sum(sum_sales#117)#277 AS sum(sum_sales)#282, sum(number_sales#118)#278 AS sum(number_sales)#283] +(189) HashAggregate [codegen id : 551] +Input [4]: [channel#51, sum#210, isEmpty#211, sum#212] +Keys [1]: [channel#51] +Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] +Aggregate Attributes [2]: [sum(sum_sales#93)#214, sum(number_sales#94)#215] +Results [6]: [channel#51, null AS i_brand_id#216, null AS i_class_id#217, null AS i_category_id#218, sum(sum_sales#93)#214 AS sum(sum_sales)#219, sum(number_sales#94)#215 AS sum(number_sales)#220] (190) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sum#287, isEmpty#288, count#289] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#221, isEmpty#222, count#223] -(191) HashAggregate [codegen id : 570] -Input [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sum#287, isEmpty#288, count#289] -Keys [3]: [i_brand_id#284, i_class_id#285, i_category_id#286] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#292, count(1)#293] -Results [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#292 AS sales#60, count(1)#293 AS number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#292 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294] +(191) HashAggregate [codegen id : 596] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#221, isEmpty#222, count#223] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#224, count(1)#225] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sales#46, count(1)#225 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226] -(192) Filter [codegen id : 570] -Input [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(192) Filter [codegen id : 596] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(193) Project [codegen id : 570] -Output [6]: [store AS channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sales#60, number_sales#61] -Input [6]: [i_brand_id#284, i_class_id#285, i_category_id#286, sales#60, number_sales#61, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#290 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#291 as decimal(12,2)))), DecimalType(18,2), true))#294] +(193) Project [codegen id : 596] +Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226] (194) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sum#298, isEmpty#299, count#300] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#227, isEmpty#228, count#229] -(195) HashAggregate [codegen id : 613] -Input [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sum#298, isEmpty#299, count#300] -Keys [3]: [i_brand_id#295, i_class_id#296, i_category_id#297] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#303, count(1)#304] -Results [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#303 AS sales#83, count(1)#304 AS number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#303 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305] +(195) HashAggregate [codegen id : 641] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#227, isEmpty#228, count#229] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#230, count(1)#231] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sales#64, count(1)#231 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232] -(196) Filter [codegen id : 613] -Input [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(196) Filter [codegen id : 641] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(197) Project [codegen id : 613] -Output [6]: [catalog AS channel#86, i_brand_id#295, i_class_id#296, i_category_id#297, sales#83, number_sales#84] -Input [6]: [i_brand_id#295, i_class_id#296, i_category_id#297, sales#83, number_sales#84, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#301 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#302 as decimal(12,2)))), DecimalType(18,2), true))#305] +(197) Project [codegen id : 641] +Output [6]: [catalog AS channel#233, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232] (198) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sum#309, isEmpty#310, count#311] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#234, isEmpty#235, count#236] -(199) HashAggregate [codegen id : 656] -Input [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sum#309, isEmpty#310, count#311] -Keys [3]: [i_brand_id#306, i_class_id#307, i_category_id#308] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#314, count(1)#315] -Results [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#314 AS sales#104, count(1)#315 AS number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#314 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316] +(199) HashAggregate [codegen id : 686] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#234, isEmpty#235, count#236] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#237, count(1)#238] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#237 AS sales#80, count(1)#238 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#237 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239] -(200) Filter [codegen id : 656] -Input [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#63, [id=#64] as decimal(32,6)))) +(200) Filter [codegen id : 686] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(201) Project [codegen id : 656] -Output [6]: [web AS channel#107, i_brand_id#306, i_class_id#307, i_category_id#308, sales#104, number_sales#105] -Input [6]: [i_brand_id#306, i_class_id#307, i_category_id#308, sales#104, number_sales#105, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#312 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#313 as decimal(12,2)))), DecimalType(18,2), true))#316] +(201) Project [codegen id : 686] +Output [6]: [web AS channel#240, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239] (202) Union -(203) HashAggregate [codegen id : 657] -Input [6]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sales#60, number_sales#61] -Keys [4]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286] -Functions [2]: [partial_sum(sales#60), partial_sum(number_sales#61)] -Aggregate Attributes [3]: [sum#317, isEmpty#318, sum#319] -Results [7]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sum#320, isEmpty#321, sum#322] +(203) HashAggregate [codegen id : 687] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] +Aggregate Attributes [3]: [sum#241, isEmpty#242, sum#243] +Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#244, isEmpty#245, sum#246] (204) Exchange -Input [7]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sum#320, isEmpty#321, sum#322] -Arguments: hashpartitioning(channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, 5), ENSURE_REQUIREMENTS, [id=#323] - -(205) HashAggregate [codegen id : 658] -Input [7]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286, sum#320, isEmpty#321, sum#322] -Keys [4]: [channel#65, i_brand_id#284, i_class_id#285, i_category_id#286] -Functions [2]: [sum(sales#60), sum(number_sales#61)] -Aggregate Attributes [2]: [sum(sales#60)#324, sum(number_sales#61)#325] -Results [2]: [sum(sales#60)#324 AS sum_sales#117, sum(number_sales#61)#325 AS number_sales#118] - -(206) HashAggregate [codegen id : 658] -Input [2]: [sum_sales#117, number_sales#118] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#244, isEmpty#245, sum#246] +Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#247] + +(205) HashAggregate [codegen id : 688] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#244, isEmpty#245, sum#246] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(sales#46), sum(number_sales#47)] +Aggregate Attributes [2]: [sum(sales#46)#248, sum(number_sales#47)#249] +Results [2]: [sum(sales#46)#248 AS sum_sales#93, sum(number_sales#47)#249 AS number_sales#94] + +(206) HashAggregate [codegen id : 688] +Input [2]: [sum_sales#93, number_sales#94] Keys: [] -Functions [2]: [partial_sum(sum_sales#117), partial_sum(number_sales#118)] -Aggregate Attributes [3]: [sum#326, isEmpty#327, sum#328] -Results [3]: [sum#329, isEmpty#330, sum#331] +Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] +Aggregate Attributes [3]: [sum#250, isEmpty#251, sum#252] +Results [3]: [sum#253, isEmpty#254, sum#255] (207) Exchange -Input [3]: [sum#329, isEmpty#330, sum#331] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#332] +Input [3]: [sum#253, isEmpty#254, sum#255] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#256] -(208) HashAggregate [codegen id : 659] -Input [3]: [sum#329, isEmpty#330, sum#331] +(208) HashAggregate [codegen id : 689] +Input [3]: [sum#253, isEmpty#254, sum#255] Keys: [] -Functions [2]: [sum(sum_sales#117), sum(number_sales#118)] -Aggregate Attributes [2]: [sum(sum_sales#117)#333, sum(number_sales#118)#334] -Results [6]: [null AS channel#335, null AS i_brand_id#336, null AS i_class_id#337, null AS i_category_id#338, sum(sum_sales#117)#333 AS sum(sum_sales)#339, sum(number_sales#118)#334 AS sum(number_sales)#340] +Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] +Aggregate Attributes [2]: [sum(sum_sales#93)#257, sum(number_sales#94)#258] +Results [6]: [null AS channel#259, null AS i_brand_id#260, null AS i_class_id#261, null AS i_category_id#262, sum(sum_sales#93)#257 AS sum(sum_sales)#263, sum(number_sales#94)#258 AS sum(number_sales)#264] (209) Union -(210) HashAggregate [codegen id : 660] -Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] -Keys [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +(210) HashAggregate [codegen id : 690] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Keys [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +Results [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] (211) Exchange -Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] -Arguments: hashpartitioning(channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118, 5), ENSURE_REQUIREMENTS, [id=#341] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94, 5), ENSURE_REQUIREMENTS, [id=#265] -(212) HashAggregate [codegen id : 661] -Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] -Keys [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +(212) HashAggregate [codegen id : 691] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Keys [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +Results [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] (213) TakeOrderedAndProject -Input [6]: [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] -Arguments: 100, [channel#65 ASC NULLS FIRST, i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#65, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#117, number_sales#118] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Arguments: 100, [channel#51 ASC NULLS FIRST, i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#63, [id=#64] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#49, [id=#50] * HashAggregate (240) +- Exchange (239) +- * HashAggregate (238) @@ -1243,7 +1243,7 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#342)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#266)] ReadSchema: struct (215) ColumnarToRow [codegen id : 2] @@ -1269,7 +1269,7 @@ Input [2]: [d_date_sk#12, d_year#13] (220) BroadcastExchange Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#343] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#267] (221) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#4] @@ -1277,107 +1277,107 @@ Right keys [1]: [d_date_sk#12] Join condition: None (222) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#344, ss_list_price#3 AS list_price#345] +Output [2]: [ss_quantity#2 AS quantity#268, ss_list_price#3 AS list_price#269] Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (223) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Output [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#346)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#270)] ReadSchema: struct (224) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21] +Input [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] (225) Scan parquet default.date_dim -Output [2]: [d_date_sk#347, d_year#348] +Output [2]: [d_date_sk#12, d_year#13] 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 (226) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#347, d_year#348] +Input [2]: [d_date_sk#12, d_year#13] (227) Filter [codegen id : 3] -Input [2]: [d_date_sk#347, d_year#348] -Condition : (((isnotnull(d_year#348) AND (d_year#348 >= 1998)) AND (d_year#348 <= 2000)) AND isnotnull(d_date_sk#347)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) (228) Project [codegen id : 3] -Output [1]: [d_date_sk#347] -Input [2]: [d_date_sk#347, d_year#348] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (229) BroadcastExchange -Input [1]: [d_date_sk#347] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#349] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#271] (230) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#347] +Right keys [1]: [d_date_sk#12] Join condition: None (231) Project [codegen id : 4] -Output [2]: [cs_quantity#66 AS quantity#350, cs_list_price#67 AS list_price#351] -Input [4]: [cs_quantity#66, cs_list_price#67, cs_sold_date_sk#21, d_date_sk#347] +Output [2]: [cs_quantity#52 AS quantity#272, cs_list_price#53 AS list_price#273] +Input [4]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] (232) Scan parquet default.web_sales -Output [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Output [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#346)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#270)] ReadSchema: struct (233) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36] +Input [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] (234) ReusedExchange [Reuses operator id: 229] -Output [1]: [d_date_sk#352] +Output [1]: [d_date_sk#12] (235) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#352] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None (236) Project [codegen id : 6] -Output [2]: [ws_quantity#87 AS quantity#353, ws_list_price#88 AS list_price#354] -Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#36, d_date_sk#352] +Output [2]: [ws_quantity#68 AS quantity#274, ws_list_price#69 AS list_price#275] +Input [4]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] (237) Union (238) HashAggregate [codegen id : 7] -Input [2]: [quantity#344, list_price#345] +Input [2]: [quantity#268, list_price#269] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#344 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#345 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#355, count#356] -Results [2]: [sum#357, count#358] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#276, count#277] +Results [2]: [sum#278, count#279] (239) Exchange -Input [2]: [sum#357, count#358] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#359] +Input [2]: [sum#278, count#279] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#280] (240) HashAggregate [codegen id : 8] -Input [2]: [sum#357, count#358] +Input [2]: [sum#278, count#279] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#344 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#345 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#344 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#345 as decimal(12,2)))), DecimalType(18,2), true))#360] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#344 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#345 as decimal(12,2)))), DecimalType(18,2), true))#360 AS average_sales#361] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#281] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#281 AS average_sales#282] -Subquery:2 Hosting operator id = 214 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#342 +Subquery:2 Hosting operator id = 214 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#266 ReusedExchange (241) (241) ReusedExchange [Reuses operator id: 220] Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 223 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#346 +Subquery:3 Hosting operator id = 223 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#270 ReusedExchange (242) (242) ReusedExchange [Reuses operator id: 229] -Output [1]: [d_date_sk#347] +Output [1]: [d_date_sk#12] -Subquery:4 Hosting operator id = 232 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#346 +Subquery:4 Hosting operator id = 232 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#270 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (243) @@ -1393,40 +1393,40 @@ ReusedExchange (244) (244) ReusedExchange [Reuses operator id: 16] Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 -Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#5 -Subquery:13 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:13 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:14 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:14 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:15 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:15 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:16 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:16 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:17 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:17 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:18 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:18 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:19 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:19 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:20 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:20 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:21 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:21 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:22 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:22 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:23 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:23 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:24 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#63, [id=#64] +Subquery:24 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt index cb186b0ddcd3b..d27eb5a32b387 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt @@ -1,21 +1,21 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - WholeStageCodegen (661) + WholeStageCodegen (691) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (660) + WholeStageCodegen (690) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union - WholeStageCodegen (131) + WholeStageCodegen (137) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (130) + WholeStageCodegen (136) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (43) + WholeStageCodegen (45) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (42) + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -90,11 +90,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - WholeStageCodegen (20) + WholeStageCodegen (21) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #6 - WholeStageCodegen (19) + WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -103,82 +103,84 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #7 - WholeStageCodegen (18) + WholeStageCodegen (19) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (17) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (12) + WholeStageCodegen (13) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #10 - WholeStageCodegen (10) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #11 + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #12 - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #13 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (9) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (7) - Filter [i_item_sk] + Exchange [brand_id,class_id,category_id] #11 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #12 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (16) + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #12 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (8) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (17) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (15) + WholeStageCodegen (16) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -189,12 +191,12 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #11 + ReusedExchange [d_date_sk] #12 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 InputAdapter BroadcastExchange #5 - WholeStageCodegen (21) + WholeStageCodegen (22) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow @@ -203,27 +205,27 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter BroadcastExchange #17 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (23) + WholeStageCodegen (24) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #18 - WholeStageCodegen (22) + WholeStageCodegen (23) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (41) + WholeStageCodegen (43) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 - WholeStageCodegen (86) + WholeStageCodegen (90) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #23 - WholeStageCodegen (85) + WholeStageCodegen (89) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -231,17 +233,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [cs_item_sk,ss_item_sk] - WholeStageCodegen (45) + WholeStageCodegen (47) Sort [cs_item_sk] InputAdapter Exchange [cs_item_sk] #24 - WholeStageCodegen (44) + WholeStageCodegen (46) Filter [cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (63) + WholeStageCodegen (66) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 @@ -249,14 +251,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedExchange [d_date_sk] #5 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (129) + WholeStageCodegen (135) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #25 - WholeStageCodegen (128) + WholeStageCodegen (134) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] @@ -264,17 +266,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ws_item_sk,ss_item_sk] - WholeStageCodegen (88) + WholeStageCodegen (92) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #26 - WholeStageCodegen (87) + WholeStageCodegen (91) Filter [ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (106) + WholeStageCodegen (111) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 @@ -282,136 +284,136 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedExchange [d_date_sk] #5 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (263) + WholeStageCodegen (275) HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id] #27 - WholeStageCodegen (262) + WholeStageCodegen (274) HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #28 - WholeStageCodegen (261) + WholeStageCodegen (273) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (174) + WholeStageCodegen (182) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (217) + WholeStageCodegen (227) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (260) + WholeStageCodegen (272) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (395) + WholeStageCodegen (413) HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id] #32 - WholeStageCodegen (394) + WholeStageCodegen (412) HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #33 - WholeStageCodegen (393) + WholeStageCodegen (411) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (306) + WholeStageCodegen (320) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (349) + WholeStageCodegen (365) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (392) + WholeStageCodegen (410) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (527) + WholeStageCodegen (551) HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel] #34 - WholeStageCodegen (526) + WholeStageCodegen (550) HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #35 - WholeStageCodegen (525) + WholeStageCodegen (549) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (438) + WholeStageCodegen (458) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (481) + WholeStageCodegen (503) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (524) + WholeStageCodegen (548) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (659) + WholeStageCodegen (689) HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange #36 - WholeStageCodegen (658) + WholeStageCodegen (688) HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #37 - WholeStageCodegen (657) + WholeStageCodegen (687) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (570) + WholeStageCodegen (596) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (613) + WholeStageCodegen (641) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (656) + WholeStageCodegen (686) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 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 4bd1e57d994ab..04fe14cab2c9b 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 @@ -34,36 +34,36 @@ TakeOrderedAndProject (194) : : : : :- * HashAggregate (39) : : : : : +- Exchange (38) : : : : : +- * HashAggregate (37) - : : : : : +- * Project (36) - : : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : : :- * Project (33) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (31) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : : : : : : :- * Filter (12) - : : : : : : : +- * ColumnarToRow (11) - : : : : : : : +- Scan parquet default.item (10) - : : : : : : +- BroadcastExchange (29) - : : : : : : +- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Filter (15) - : : : : : : : : +- * ColumnarToRow (14) - : : : : : : : : +- Scan parquet default.catalog_sales (13) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * Filter (18) - : : : : : : : +- * ColumnarToRow (17) - : : : : : : : +- Scan parquet default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * Project (25) - : : : : : : +- * Filter (24) - : : : : : : +- * ColumnarToRow (23) - : : : : : : +- Scan parquet default.date_dim (22) - : : : : : +- ReusedExchange (34) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : : :- * Project (22) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : : :- * Project (15) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : : :- * Filter (9) + : : : : : : : : +- * ColumnarToRow (8) + : : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : : +- BroadcastExchange (13) + : : : : : : : +- * Filter (12) + : : : : : : : +- * ColumnarToRow (11) + : : : : : : : +- Scan parquet default.item (10) + : : : : : : +- BroadcastExchange (20) + : : : : : : +- * Project (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet default.date_dim (16) + : : : : : +- BroadcastExchange (35) + : : : : : +- * Project (34) + : : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : : :- * Project (31) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : : :- * Filter (25) + : : : : : : : +- * ColumnarToRow (24) + : : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : : +- BroadcastExchange (29) + : : : : : : +- * Filter (28) + : : : : : : +- * ColumnarToRow (27) + : : : : : : +- Scan parquet default.item (26) + : : : : : +- ReusedExchange (32) : : : : +- BroadcastExchange (49) : : : : +- * Project (48) : : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -240,226 +240,226 @@ Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] Condition : isnotnull(ss_item_sk#1) (10) Scan parquet default.item -Output [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +(11) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(12) Filter [codegen id : 4] -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -Condition : (((isnotnull(i_item_sk#11) AND isnotnull(i_brand_id#12)) AND isnotnull(i_class_id#13)) AND isnotnull(i_category_id#14)) +(12) Filter [codegen id : 1] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(13) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#15, cs_sold_date_sk#16] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +(13) BroadcastExchange +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] +(14) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join condition: None -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#15, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#15) +(15) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(16) Scan parquet default.item -Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +(16) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_year#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +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 + +(17) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#12, d_year#13] -(17) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +(18) Filter [codegen id : 2] +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) -(18) Filter [codegen id : 1] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Condition : isnotnull(i_item_sk#17) +(19) Project [codegen id : 2] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] -(19) BroadcastExchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +(20) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#15] -Right keys [1]: [i_item_sk#17] +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20] -Input [6]: [cs_item_sk#15, cs_sold_date_sk#16, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +(22) Project [codegen id : 6] +Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] +Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] -(22) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_year#23] +(23) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] 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 - -(23) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#22, d_year#23] +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(24) Filter [codegen id : 2] -Input [2]: [d_date_sk#22, d_year#23] -Condition : (((isnotnull(d_year#23) AND (d_year#23 >= 1999)) AND (d_year#23 <= 2001)) AND isnotnull(d_date_sk#22)) +(24) ColumnarToRow [codegen id : 5] +Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -(25) Project [codegen id : 2] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_year#23] +(25) Filter [codegen id : 5] +Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +Condition : isnotnull(cs_item_sk#18) -(26) BroadcastExchange -Input [1]: [d_date_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +(26) Scan parquet default.item +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(27) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#22] -Join condition: None +(27) ColumnarToRow [codegen id : 3] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(28) Project [codegen id : 3] -Output [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] -Input [5]: [cs_sold_date_sk#16, i_brand_id#18, i_class_id#19, i_category_id#20, d_date_sk#22] +(28) Filter [codegen id : 3] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : isnotnull(i_item_sk#6) (29) BroadcastExchange -Input [3]: [i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] -(30) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#12, 0), isnull(i_brand_id#12), coalesce(i_class_id#13, 0), isnull(i_class_id#13), coalesce(i_category_id#14, 0), isnull(i_category_id#14)] -Right keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +(30) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#6] Join condition: None -(31) BroadcastExchange -Input [4]: [i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +(31) Project [codegen id : 5] +Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#11] +(32) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] + +(33) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None -(33) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#11, i_brand_id#12, i_class_id#13, i_category_id#14] +(34) Project [codegen id : 5] +Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] -(34) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#27] +(35) BroadcastExchange +Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] -(35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +(36) BroadcastHashJoin [codegen id : 6] +Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] +Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None -(36) Project [codegen id : 6] -Output [3]: [i_brand_id#12 AS brand_id#28, i_class_id#13 AS class_id#29, i_category_id#14 AS category_id#30] -Input [5]: [ss_sold_date_sk#4, i_brand_id#12, i_class_id#13, i_category_id#14, d_date_sk#27] - (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#15, class_id#16, category_id#17] (38) Exchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: hashpartitioning(brand_id#28, class_id#29, category_id#30, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#15, class_id#16, category_id#17] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] +Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_item_sk#32) +Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Condition : isnotnull(ws_item_sk#23) -(43) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +(43) ReusedExchange [Reuses operator id: 29] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#34] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [i_item_sk#6] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37] -Input [6]: [ws_item_sk#32, ws_sold_date_sk#33, i_item_sk#34, i_brand_id#35, i_class_id#36, i_category_id#37] +Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(46) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#38] +(46) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#38] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] -Input [5]: [ws_sold_date_sk#33, i_brand_id#35, i_class_id#36, i_category_id#37, d_date_sk#38] +Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (49) BroadcastExchange -Input [3]: [i_brand_id#35, i_class_id#36, i_category_id#37] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#39] +Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#28, 0), isnull(brand_id#28), coalesce(class_id#29, 0), isnull(class_id#29), coalesce(category_id#30, 0), isnull(category_id#30)] -Right keys [6]: [coalesce(i_brand_id#35, 0), isnull(i_brand_id#35), coalesce(i_class_id#36, 0), isnull(i_class_id#36), coalesce(i_category_id#37, 0), isnull(i_category_id#37)] +Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] +Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#15, class_id#16, category_id#17] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#28, class_id#29, category_id#30] -Keys [3]: [brand_id#28, class_id#29, category_id#30] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#28, class_id#29, category_id#30] +Results [3]: [brand_id#15, class_id#16, category_id#17] (53) BroadcastExchange -Input [3]: [brand_id#28, class_id#29, category_id#30] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#40] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#28, class_id#29, category_id#30] +Right keys [3]: [brand_id#15, class_id#16, category_id#17] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#41] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#28, class_id#29, category_id#30] +Output [1]: [i_item_sk#6 AS ss_item_sk#27] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] (56) BroadcastExchange -Input [1]: [ss_item_sk#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [ss_item_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#41] +Right keys [1]: [ss_item_sk#27] Join condition: None (58) Scan parquet default.item @@ -477,16 +477,16 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : isnotnull(i_item_sk#6) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#27] (62) BroadcastHashJoin [codegen id : 23] Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#41] +Right keys [1]: [ss_item_sk#27] Join condition: None (63) BroadcastExchange Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] @@ -498,623 +498,623 @@ Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_ Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (66) Scan parquet default.date_dim -Output [3]: [d_date_sk#27, d_year#44, d_moy#45] +Output [3]: [d_date_sk#12, d_year#13, d_moy#30] 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 (67) ColumnarToRow [codegen id : 24] -Input [3]: [d_date_sk#27, d_year#44, d_moy#45] +Input [3]: [d_date_sk#12, d_year#13, d_moy#30] (68) Filter [codegen id : 24] -Input [3]: [d_date_sk#27, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 11)) AND isnotnull(d_date_sk#27)) +Input [3]: [d_date_sk#12, d_year#13, d_moy#30] +Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#30)) AND (d_year#13 = 2000)) AND (d_moy#30 = 11)) AND isnotnull(d_date_sk#12)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#27] -Input [3]: [d_date_sk#27, d_year#44, d_moy#45] +Output [1]: [d_date_sk#12] +Input [3]: [d_date_sk#12, d_year#13, d_moy#30] (70) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None (72) Project [codegen id : 25] Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#27] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (73) HashAggregate [codegen id : 25] Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#47, isEmpty#48, count#49] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] +Aggregate Attributes [3]: [sum#32, isEmpty#33, count#34] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#38] (75) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#50, isEmpty#51, count#52] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54, count(1)#55] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sales#56, count(1)#55 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#54 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] (76) Filter [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58 as decimal(32,6)) > cast(Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [store AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#58] +Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] (78) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Output [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] (80) Filter [codegen id : 51] -Input [4]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] -Condition : isnotnull(cs_item_sk#15) +Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Condition : isnotnull(cs_item_sk#18) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#27] (82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#15] -Right keys [1]: [ss_item_sk#41] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [ss_item_sk#27] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#15] -Right keys [1]: [i_item_sk#64] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#6] Join condition: None (85) Project [codegen id : 51] -Output [6]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [8]: [cs_item_sk#15, cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_item_sk#64, i_brand_id#65, i_class_id#66, i_category_id#67] +Output [6]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [8]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#68] +Output [1]: [d_date_sk#12] (87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#68] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None (88) Project [codegen id : 51] -Output [5]: [cs_quantity#62, cs_list_price#63, i_brand_id#65, i_class_id#66, i_category_id#67] -Input [7]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, i_brand_id#65, i_class_id#66, i_category_id#67, d_date_sk#68] +Output [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#62, cs_list_price#63, i_brand_id#65, i_class_id#66, i_category_id#67] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#69, isEmpty#70, count#71] -Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] +Input [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#49, isEmpty#50, count#51] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] (90) Exchange -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Arguments: hashpartitioning(i_brand_id#65, i_class_id#66, i_category_id#67, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#55] (91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum#72, isEmpty#73, count#74] -Keys [3]: [i_brand_id#65, i_class_id#66, i_category_id#67] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76, count(1)#77] -Results [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sales#78, count(1)#77 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#76 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56, count(1)#57] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sales#58, count(1)#57 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] (92) Filter [codegen id : 52] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [catalog AS channel#81, i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79] -Input [6]: [i_brand_id#65, i_class_id#66, i_category_id#67, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#80] +Output [6]: [catalog AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] (94) Scan parquet default.web_sales -Output [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Output [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] (96) Filter [codegen id : 77] -Input [4]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] -Condition : isnotnull(ws_item_sk#32) +Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Condition : isnotnull(ws_item_sk#23) (97) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#41] +Output [1]: [ss_item_sk#27] (98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [ss_item_sk#41] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [ss_item_sk#27] Join condition: None (99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#32] -Right keys [1]: [i_item_sk#84] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [i_item_sk#6] Join condition: None (101) Project [codegen id : 77] -Output [6]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_brand_id#85, i_class_id#86, i_category_id#87] -Input [8]: [ws_item_sk#32, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] +Output [6]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [8]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#88] +Output [1]: [d_date_sk#12] (103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#88] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (104) Project [codegen id : 77] -Output [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#85, i_class_id#86, i_category_id#87] -Input [7]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, i_brand_id#85, i_class_id#86, i_category_id#87, d_date_sk#88] +Output [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#85, i_class_id#86, i_category_id#87] -Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#89, isEmpty#90, count#91] -Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Input [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] (106) Exchange -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] -Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, 5), ENSURE_REQUIREMENTS, [id=#95] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#70] (107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] -Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96, count(1)#97] -Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sales#98, count(1)#97 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#96 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71, count(1)#72] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sales#73, count(1)#72 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] (108) Filter [codegen id : 78] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [web AS channel#101, i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99] -Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#82 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#83 as decimal(12,2)))), DecimalType(18,2), true))#100] +Output [6]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] (110) Union (111) HashAggregate [codegen id : 79] -Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sales#56, number_sales#57] -Keys [4]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] -Aggregate Attributes [3]: [sum#102, isEmpty#103, sum#104] -Results [7]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum#105, isEmpty#106, sum#107] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] +Aggregate Attributes [3]: [sum#77, isEmpty#78, sum#79] +Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#80, isEmpty#81, sum#82] (112) Exchange -Input [7]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum#105, isEmpty#106, sum#107] -Arguments: hashpartitioning(channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#108] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#80, isEmpty#81, sum#82] +Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#83] (113) HashAggregate [codegen id : 80] -Input [7]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum#105, isEmpty#106, sum#107] -Keys [4]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#56), sum(number_sales#57)] -Aggregate Attributes [2]: [sum(sales#56)#109, sum(number_sales#57)#110] -Results [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#56)#109 AS sum_sales#111, sum(number_sales#57)#110 AS number_sales#112] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#80, isEmpty#81, sum#82] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#41), sum(number_sales#42)] +Aggregate Attributes [2]: [sum(sales#41)#84, sum(number_sales#42)#85] +Results [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#41)#84 AS sum_sales#86, sum(number_sales#42)#85 AS number_sales#87] (114) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sum#116, isEmpty#117, count#118] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#88, isEmpty#89, count#90] (115) HashAggregate [codegen id : 106] -Input [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sum#116, isEmpty#117, count#118] -Keys [3]: [i_brand_id#113, i_class_id#114, i_category_id#115] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#121, count(1)#122] -Results [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#121 AS sales#56, count(1)#122 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#121 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#88, isEmpty#89, count#90] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#41, count(1)#92 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93] (116) Filter [codegen id : 106] -Input [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (117) Project [codegen id : 106] -Output [6]: [store AS channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sales#56, number_sales#57] -Input [6]: [i_brand_id#113, i_class_id#114, i_category_id#115, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#119 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#120 as decimal(12,2)))), DecimalType(18,2), true))#123] +Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93] (118) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#94, isEmpty#95, count#96] (119) HashAggregate [codegen id : 132] -Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] -Keys [3]: [i_brand_id#124, i_class_id#125, i_category_id#126] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132, count(1)#133] -Results [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sales#78, count(1)#133 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#94, isEmpty#95, count#96] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#58, count(1)#98 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99] (120) Filter [codegen id : 132] -Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (121) Project [codegen id : 132] -Output [6]: [catalog AS channel#81, i_brand_id#124, i_class_id#125, i_category_id#126, sales#78, number_sales#79] -Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] +Output [6]: [catalog AS channel#100, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99] (122) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#101, isEmpty#102, count#103] (123) HashAggregate [codegen id : 158] -Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] -Keys [3]: [i_brand_id#135, i_class_id#136, i_category_id#137] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143, count(1)#144] -Results [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sales#98, count(1)#144 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#101, isEmpty#102, count#103] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#104, count(1)#105] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sales#73, count(1)#105 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106] (124) Filter [codegen id : 158] -Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (125) Project [codegen id : 158] -Output [6]: [web AS channel#101, i_brand_id#135, i_class_id#136, i_category_id#137, sales#98, number_sales#99] -Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] +Output [6]: [web AS channel#107, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106] (126) Union (127) HashAggregate [codegen id : 159] -Input [6]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sales#56, number_sales#57] -Keys [4]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115] -Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] -Aggregate Attributes [3]: [sum#146, isEmpty#147, sum#148] -Results [7]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sum#149, isEmpty#150, sum#151] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] +Aggregate Attributes [3]: [sum#108, isEmpty#109, sum#110] +Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#111, isEmpty#112, sum#113] (128) Exchange -Input [7]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sum#149, isEmpty#150, sum#151] -Arguments: hashpartitioning(channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, 5), ENSURE_REQUIREMENTS, [id=#152] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#111, isEmpty#112, sum#113] +Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#114] (129) HashAggregate [codegen id : 160] -Input [7]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115, sum#149, isEmpty#150, sum#151] -Keys [4]: [channel#61, i_brand_id#113, i_class_id#114, i_category_id#115] -Functions [2]: [sum(sales#56), sum(number_sales#57)] -Aggregate Attributes [2]: [sum(sales#56)#153, sum(number_sales#57)#154] -Results [5]: [channel#61, i_brand_id#113, i_class_id#114, sum(sales#56)#153 AS sum_sales#111, sum(number_sales#57)#154 AS number_sales#112] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#111, isEmpty#112, sum#113] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#41), sum(number_sales#42)] +Aggregate Attributes [2]: [sum(sales#41)#115, sum(number_sales#42)#116] +Results [5]: [channel#46, i_brand_id#7, i_class_id#8, sum(sales#41)#115 AS sum_sales#86, sum(number_sales#42)#116 AS number_sales#87] (130) HashAggregate [codegen id : 160] -Input [5]: [channel#61, i_brand_id#113, i_class_id#114, sum_sales#111, number_sales#112] -Keys [3]: [channel#61, i_brand_id#113, i_class_id#114] -Functions [2]: [partial_sum(sum_sales#111), partial_sum(number_sales#112)] -Aggregate Attributes [3]: [sum#155, isEmpty#156, sum#157] -Results [6]: [channel#61, i_brand_id#113, i_class_id#114, sum#158, isEmpty#159, sum#160] +Input [5]: [channel#46, i_brand_id#7, i_class_id#8, sum_sales#86, number_sales#87] +Keys [3]: [channel#46, i_brand_id#7, i_class_id#8] +Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] +Aggregate Attributes [3]: [sum#117, isEmpty#118, sum#119] +Results [6]: [channel#46, i_brand_id#7, i_class_id#8, sum#120, isEmpty#121, sum#122] (131) Exchange -Input [6]: [channel#61, i_brand_id#113, i_class_id#114, sum#158, isEmpty#159, sum#160] -Arguments: hashpartitioning(channel#61, i_brand_id#113, i_class_id#114, 5), ENSURE_REQUIREMENTS, [id=#161] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, sum#120, isEmpty#121, sum#122] +Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, 5), ENSURE_REQUIREMENTS, [id=#123] (132) HashAggregate [codegen id : 161] -Input [6]: [channel#61, i_brand_id#113, i_class_id#114, sum#158, isEmpty#159, sum#160] -Keys [3]: [channel#61, i_brand_id#113, i_class_id#114] -Functions [2]: [sum(sum_sales#111), sum(number_sales#112)] -Aggregate Attributes [2]: [sum(sum_sales#111)#162, sum(number_sales#112)#163] -Results [6]: [channel#61, i_brand_id#113, i_class_id#114, null AS i_category_id#164, sum(sum_sales#111)#162 AS sum(sum_sales)#165, sum(number_sales#112)#163 AS sum(number_sales)#166] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, sum#120, isEmpty#121, sum#122] +Keys [3]: [channel#46, i_brand_id#7, i_class_id#8] +Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] +Aggregate Attributes [2]: [sum(sum_sales#86)#124, sum(number_sales#87)#125] +Results [6]: [channel#46, i_brand_id#7, i_class_id#8, null AS i_category_id#126, sum(sum_sales#86)#124 AS sum(sum_sales)#127, sum(number_sales#87)#125 AS sum(number_sales)#128] (133) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sum#170, isEmpty#171, count#172] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#129, isEmpty#130, count#131] (134) HashAggregate [codegen id : 187] -Input [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sum#170, isEmpty#171, count#172] -Keys [3]: [i_brand_id#167, i_class_id#168, i_category_id#169] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#175, count(1)#176] -Results [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#175 AS sales#56, count(1)#176 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#175 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#129, isEmpty#130, count#131] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#132, count(1)#133] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sales#41, count(1)#133 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134] (135) Filter [codegen id : 187] -Input [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (136) Project [codegen id : 187] -Output [6]: [store AS channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sales#56, number_sales#57] -Input [6]: [i_brand_id#167, i_class_id#168, i_category_id#169, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#173 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#174 as decimal(12,2)))), DecimalType(18,2), true))#177] +Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134] (137) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#135, isEmpty#136, count#137] (138) HashAggregate [codegen id : 213] -Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] -Keys [3]: [i_brand_id#178, i_class_id#179, i_category_id#180] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186, count(1)#187] -Results [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sales#78, count(1)#187 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#135, isEmpty#136, count#137] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#138, count(1)#139] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sales#58, count(1)#139 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140] (139) Filter [codegen id : 213] -Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (140) Project [codegen id : 213] -Output [6]: [catalog AS channel#81, i_brand_id#178, i_class_id#179, i_category_id#180, sales#78, number_sales#79] -Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] +Output [6]: [catalog AS channel#141, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140] (141) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#142, isEmpty#143, count#144] (142) HashAggregate [codegen id : 239] -Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] -Keys [3]: [i_brand_id#189, i_class_id#190, i_category_id#191] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197, count(1)#198] -Results [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sales#98, count(1)#198 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#142, isEmpty#143, count#144] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#145, count(1)#146] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sales#73, count(1)#146 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147] (143) Filter [codegen id : 239] -Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (144) Project [codegen id : 239] -Output [6]: [web AS channel#101, i_brand_id#189, i_class_id#190, i_category_id#191, sales#98, number_sales#99] -Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] +Output [6]: [web AS channel#148, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147] (145) Union (146) HashAggregate [codegen id : 240] -Input [6]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sales#56, number_sales#57] -Keys [4]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169] -Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] -Aggregate Attributes [3]: [sum#200, isEmpty#201, sum#202] -Results [7]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sum#203, isEmpty#204, sum#205] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] +Aggregate Attributes [3]: [sum#149, isEmpty#150, sum#151] +Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] (147) Exchange -Input [7]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sum#203, isEmpty#204, sum#205] -Arguments: hashpartitioning(channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, 5), ENSURE_REQUIREMENTS, [id=#206] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] +Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#155] (148) HashAggregate [codegen id : 241] -Input [7]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169, sum#203, isEmpty#204, sum#205] -Keys [4]: [channel#61, i_brand_id#167, i_class_id#168, i_category_id#169] -Functions [2]: [sum(sales#56), sum(number_sales#57)] -Aggregate Attributes [2]: [sum(sales#56)#207, sum(number_sales#57)#208] -Results [4]: [channel#61, i_brand_id#167, sum(sales#56)#207 AS sum_sales#111, sum(number_sales#57)#208 AS number_sales#112] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#41), sum(number_sales#42)] +Aggregate Attributes [2]: [sum(sales#41)#156, sum(number_sales#42)#157] +Results [4]: [channel#46, i_brand_id#7, sum(sales#41)#156 AS sum_sales#86, sum(number_sales#42)#157 AS number_sales#87] (149) HashAggregate [codegen id : 241] -Input [4]: [channel#61, i_brand_id#167, sum_sales#111, number_sales#112] -Keys [2]: [channel#61, i_brand_id#167] -Functions [2]: [partial_sum(sum_sales#111), partial_sum(number_sales#112)] -Aggregate Attributes [3]: [sum#209, isEmpty#210, sum#211] -Results [5]: [channel#61, i_brand_id#167, sum#212, isEmpty#213, sum#214] +Input [4]: [channel#46, i_brand_id#7, sum_sales#86, number_sales#87] +Keys [2]: [channel#46, i_brand_id#7] +Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] +Aggregate Attributes [3]: [sum#158, isEmpty#159, sum#160] +Results [5]: [channel#46, i_brand_id#7, sum#161, isEmpty#162, sum#163] (150) Exchange -Input [5]: [channel#61, i_brand_id#167, sum#212, isEmpty#213, sum#214] -Arguments: hashpartitioning(channel#61, i_brand_id#167, 5), ENSURE_REQUIREMENTS, [id=#215] +Input [5]: [channel#46, i_brand_id#7, sum#161, isEmpty#162, sum#163] +Arguments: hashpartitioning(channel#46, i_brand_id#7, 5), ENSURE_REQUIREMENTS, [id=#164] (151) HashAggregate [codegen id : 242] -Input [5]: [channel#61, i_brand_id#167, sum#212, isEmpty#213, sum#214] -Keys [2]: [channel#61, i_brand_id#167] -Functions [2]: [sum(sum_sales#111), sum(number_sales#112)] -Aggregate Attributes [2]: [sum(sum_sales#111)#216, sum(number_sales#112)#217] -Results [6]: [channel#61, i_brand_id#167, null AS i_class_id#218, null AS i_category_id#219, sum(sum_sales#111)#216 AS sum(sum_sales)#220, sum(number_sales#112)#217 AS sum(number_sales)#221] +Input [5]: [channel#46, i_brand_id#7, sum#161, isEmpty#162, sum#163] +Keys [2]: [channel#46, i_brand_id#7] +Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] +Aggregate Attributes [2]: [sum(sum_sales#86)#165, sum(number_sales#87)#166] +Results [6]: [channel#46, i_brand_id#7, null AS i_class_id#167, null AS i_category_id#168, sum(sum_sales#86)#165 AS sum(sum_sales)#169, sum(number_sales#87)#166 AS sum(number_sales)#170] (152) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sum#225, isEmpty#226, count#227] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#171, isEmpty#172, count#173] (153) HashAggregate [codegen id : 268] -Input [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sum#225, isEmpty#226, count#227] -Keys [3]: [i_brand_id#222, i_class_id#223, i_category_id#224] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#230, count(1)#231] -Results [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sales#56, count(1)#231 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#171, isEmpty#172, count#173] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#174, count(1)#175] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sales#41, count(1)#175 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176] (154) Filter [codegen id : 268] -Input [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (155) Project [codegen id : 268] -Output [6]: [store AS channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sales#56, number_sales#57] -Input [6]: [i_brand_id#222, i_class_id#223, i_category_id#224, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#228 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#229 as decimal(12,2)))), DecimalType(18,2), true))#232] +Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176] (156) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#177, isEmpty#178, count#179] (157) HashAggregate [codegen id : 294] -Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] -Keys [3]: [i_brand_id#233, i_class_id#234, i_category_id#235] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241, count(1)#242] -Results [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sales#78, count(1)#242 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#177, isEmpty#178, count#179] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#180, count(1)#181] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sales#58, count(1)#181 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182] (158) Filter [codegen id : 294] -Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (159) Project [codegen id : 294] -Output [6]: [catalog AS channel#81, i_brand_id#233, i_class_id#234, i_category_id#235, sales#78, number_sales#79] -Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] +Output [6]: [catalog AS channel#183, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182] (160) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#184, isEmpty#185, count#186] (161) HashAggregate [codegen id : 320] -Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] -Keys [3]: [i_brand_id#244, i_class_id#245, i_category_id#246] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252, count(1)#253] -Results [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sales#98, count(1)#253 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#184, isEmpty#185, count#186] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#187, count(1)#188] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sales#73, count(1)#188 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189] (162) Filter [codegen id : 320] -Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (163) Project [codegen id : 320] -Output [6]: [web AS channel#101, i_brand_id#244, i_class_id#245, i_category_id#246, sales#98, number_sales#99] -Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] +Output [6]: [web AS channel#190, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189] (164) Union (165) HashAggregate [codegen id : 321] -Input [6]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sales#56, number_sales#57] -Keys [4]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224] -Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] -Aggregate Attributes [3]: [sum#255, isEmpty#256, sum#257] -Results [7]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sum#258, isEmpty#259, sum#260] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] +Aggregate Attributes [3]: [sum#191, isEmpty#192, sum#193] +Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#194, isEmpty#195, sum#196] (166) Exchange -Input [7]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sum#258, isEmpty#259, sum#260] -Arguments: hashpartitioning(channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, 5), ENSURE_REQUIREMENTS, [id=#261] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#194, isEmpty#195, sum#196] +Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#197] (167) HashAggregate [codegen id : 322] -Input [7]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224, sum#258, isEmpty#259, sum#260] -Keys [4]: [channel#61, i_brand_id#222, i_class_id#223, i_category_id#224] -Functions [2]: [sum(sales#56), sum(number_sales#57)] -Aggregate Attributes [2]: [sum(sales#56)#262, sum(number_sales#57)#263] -Results [3]: [channel#61, sum(sales#56)#262 AS sum_sales#111, sum(number_sales#57)#263 AS number_sales#112] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#194, isEmpty#195, sum#196] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#41), sum(number_sales#42)] +Aggregate Attributes [2]: [sum(sales#41)#198, sum(number_sales#42)#199] +Results [3]: [channel#46, sum(sales#41)#198 AS sum_sales#86, sum(number_sales#42)#199 AS number_sales#87] (168) HashAggregate [codegen id : 322] -Input [3]: [channel#61, sum_sales#111, number_sales#112] -Keys [1]: [channel#61] -Functions [2]: [partial_sum(sum_sales#111), partial_sum(number_sales#112)] -Aggregate Attributes [3]: [sum#264, isEmpty#265, sum#266] -Results [4]: [channel#61, sum#267, isEmpty#268, sum#269] +Input [3]: [channel#46, sum_sales#86, number_sales#87] +Keys [1]: [channel#46] +Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] +Aggregate Attributes [3]: [sum#200, isEmpty#201, sum#202] +Results [4]: [channel#46, sum#203, isEmpty#204, sum#205] (169) Exchange -Input [4]: [channel#61, sum#267, isEmpty#268, sum#269] -Arguments: hashpartitioning(channel#61, 5), ENSURE_REQUIREMENTS, [id=#270] +Input [4]: [channel#46, sum#203, isEmpty#204, sum#205] +Arguments: hashpartitioning(channel#46, 5), ENSURE_REQUIREMENTS, [id=#206] (170) HashAggregate [codegen id : 323] -Input [4]: [channel#61, sum#267, isEmpty#268, sum#269] -Keys [1]: [channel#61] -Functions [2]: [sum(sum_sales#111), sum(number_sales#112)] -Aggregate Attributes [2]: [sum(sum_sales#111)#271, sum(number_sales#112)#272] -Results [6]: [channel#61, null AS i_brand_id#273, null AS i_class_id#274, null AS i_category_id#275, sum(sum_sales#111)#271 AS sum(sum_sales)#276, sum(number_sales#112)#272 AS sum(number_sales)#277] +Input [4]: [channel#46, sum#203, isEmpty#204, sum#205] +Keys [1]: [channel#46] +Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] +Aggregate Attributes [2]: [sum(sum_sales#86)#207, sum(number_sales#87)#208] +Results [6]: [channel#46, null AS i_brand_id#209, null AS i_class_id#210, null AS i_category_id#211, sum(sum_sales#86)#207 AS sum(sum_sales)#212, sum(number_sales#87)#208 AS sum(number_sales)#213] (171) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sum#281, isEmpty#282, count#283] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#214, isEmpty#215, count#216] (172) HashAggregate [codegen id : 349] -Input [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sum#281, isEmpty#282, count#283] -Keys [3]: [i_brand_id#278, i_class_id#279, i_category_id#280] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#286, count(1)#287] -Results [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#286 AS sales#56, count(1)#287 AS number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#286 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#214, isEmpty#215, count#216] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#217, count(1)#218] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#217 AS sales#41, count(1)#218 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#217 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219] (173) Filter [codegen id : 349] -Input [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (174) Project [codegen id : 349] -Output [6]: [store AS channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sales#56, number_sales#57] -Input [6]: [i_brand_id#278, i_class_id#279, i_category_id#280, sales#56, number_sales#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#284 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#285 as decimal(12,2)))), DecimalType(18,2), true))#288] +Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219] (175) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] (176) HashAggregate [codegen id : 375] -Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] -Keys [3]: [i_brand_id#289, i_class_id#290, i_category_id#291] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297, count(1)#298] -Results [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sales#78, count(1)#298 AS number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#223, count(1)#224] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sales#58, count(1)#224 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225] (177) Filter [codegen id : 375] -Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (178) Project [codegen id : 375] -Output [6]: [catalog AS channel#81, i_brand_id#289, i_class_id#290, i_category_id#291, sales#78, number_sales#79] -Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#78, number_sales#79, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] +Output [6]: [catalog AS channel#226, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225] (179) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#227, isEmpty#228, count#229] (180) HashAggregate [codegen id : 401] -Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] -Keys [3]: [i_brand_id#300, i_class_id#301, i_category_id#302] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308, count(1)#309] -Results [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sales#98, count(1)#309 AS number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#227, isEmpty#228, count#229] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#230, count(1)#231] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sales#73, count(1)#231 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232] (181) Filter [codegen id : 401] -Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#59, [id=#60] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (182) Project [codegen id : 401] -Output [6]: [web AS channel#101, i_brand_id#300, i_class_id#301, i_category_id#302, sales#98, number_sales#99] -Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#98, number_sales#99, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] +Output [6]: [web AS channel#233, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232] (183) Union (184) HashAggregate [codegen id : 402] -Input [6]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sales#56, number_sales#57] -Keys [4]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280] -Functions [2]: [partial_sum(sales#56), partial_sum(number_sales#57)] -Aggregate Attributes [3]: [sum#311, isEmpty#312, sum#313] -Results [7]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sum#314, isEmpty#315, sum#316] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] +Aggregate Attributes [3]: [sum#234, isEmpty#235, sum#236] +Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#237, isEmpty#238, sum#239] (185) Exchange -Input [7]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sum#314, isEmpty#315, sum#316] -Arguments: hashpartitioning(channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, 5), ENSURE_REQUIREMENTS, [id=#317] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#237, isEmpty#238, sum#239] +Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#240] (186) HashAggregate [codegen id : 403] -Input [7]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280, sum#314, isEmpty#315, sum#316] -Keys [4]: [channel#61, i_brand_id#278, i_class_id#279, i_category_id#280] -Functions [2]: [sum(sales#56), sum(number_sales#57)] -Aggregate Attributes [2]: [sum(sales#56)#318, sum(number_sales#57)#319] -Results [2]: [sum(sales#56)#318 AS sum_sales#111, sum(number_sales#57)#319 AS number_sales#112] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#237, isEmpty#238, sum#239] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#41), sum(number_sales#42)] +Aggregate Attributes [2]: [sum(sales#41)#241, sum(number_sales#42)#242] +Results [2]: [sum(sales#41)#241 AS sum_sales#86, sum(number_sales#42)#242 AS number_sales#87] (187) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#111, number_sales#112] +Input [2]: [sum_sales#86, number_sales#87] Keys: [] -Functions [2]: [partial_sum(sum_sales#111), partial_sum(number_sales#112)] -Aggregate Attributes [3]: [sum#320, isEmpty#321, sum#322] -Results [3]: [sum#323, isEmpty#324, sum#325] +Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] +Aggregate Attributes [3]: [sum#243, isEmpty#244, sum#245] +Results [3]: [sum#246, isEmpty#247, sum#248] (188) Exchange -Input [3]: [sum#323, isEmpty#324, sum#325] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#326] +Input [3]: [sum#246, isEmpty#247, sum#248] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#249] (189) HashAggregate [codegen id : 404] -Input [3]: [sum#323, isEmpty#324, sum#325] +Input [3]: [sum#246, isEmpty#247, sum#248] Keys: [] -Functions [2]: [sum(sum_sales#111), sum(number_sales#112)] -Aggregate Attributes [2]: [sum(sum_sales#111)#327, sum(number_sales#112)#328] -Results [6]: [null AS channel#329, null AS i_brand_id#330, null AS i_class_id#331, null AS i_category_id#332, sum(sum_sales#111)#327 AS sum(sum_sales)#333, sum(number_sales#112)#328 AS sum(number_sales)#334] +Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] +Aggregate Attributes [2]: [sum(sum_sales#86)#250, sum(number_sales#87)#251] +Results [6]: [null AS channel#252, null AS i_brand_id#253, null AS i_class_id#254, null AS i_category_id#255, sum(sum_sales#86)#250 AS sum(sum_sales)#256, sum(number_sales#87)#251 AS sum(number_sales)#257] (190) Union (191) HashAggregate [codegen id : 405] -Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] -Keys [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Keys [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Results [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] (192) Exchange -Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] -Arguments: hashpartitioning(channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112, 5), ENSURE_REQUIREMENTS, [id=#335] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87, 5), ENSURE_REQUIREMENTS, [id=#258] (193) HashAggregate [codegen id : 406] -Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] -Keys [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Keys [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Results [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] (194) TakeOrderedAndProject -Input [6]: [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] -Arguments: 100, [channel#61 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#111, number_sales#112] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Arguments: 100, [channel#46 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#59, [id=#60] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] * HashAggregate (221) +- Exchange (220) +- * HashAggregate (219) @@ -1148,190 +1148,190 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#336)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#259)] ReadSchema: struct (196) ColumnarToRow [codegen id : 2] Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (197) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_year#44] +Output [2]: [d_date_sk#12, d_year#13] 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 (198) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#44] +Input [2]: [d_date_sk#12, d_year#13] (199) Filter [codegen id : 1] -Input [2]: [d_date_sk#27, d_year#44] -Condition : (((isnotnull(d_year#44) AND (d_year#44 >= 1999)) AND (d_year#44 <= 2001)) AND isnotnull(d_date_sk#27)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (200) Project [codegen id : 1] -Output [1]: [d_date_sk#27] -Input [2]: [d_date_sk#27, d_year#44] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (201) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#337] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#260] (202) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None (203) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#338, ss_list_price#3 AS list_price#339] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#27] +Output [2]: [ss_quantity#2 AS quantity#261, ss_list_price#3 AS list_price#262] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (204) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Output [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#340)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#263)] ReadSchema: struct (205) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16] +Input [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] (206) Scan parquet default.date_dim -Output [2]: [d_date_sk#341, d_year#342] +Output [2]: [d_date_sk#12, d_year#13] 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 (207) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#341, d_year#342] +Input [2]: [d_date_sk#12, d_year#13] (208) Filter [codegen id : 3] -Input [2]: [d_date_sk#341, d_year#342] -Condition : (((isnotnull(d_year#342) AND (d_year#342 >= 1998)) AND (d_year#342 <= 2000)) AND isnotnull(d_date_sk#341)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) (209) Project [codegen id : 3] -Output [1]: [d_date_sk#341] -Input [2]: [d_date_sk#341, d_year#342] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (210) BroadcastExchange -Input [1]: [d_date_sk#341] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#343] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#264] (211) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#341] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None (212) Project [codegen id : 4] -Output [2]: [cs_quantity#62 AS quantity#344, cs_list_price#63 AS list_price#345] -Input [4]: [cs_quantity#62, cs_list_price#63, cs_sold_date_sk#16, d_date_sk#341] +Output [2]: [cs_quantity#47 AS quantity#265, cs_list_price#48 AS list_price#266] +Input [4]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, d_date_sk#12] (213) Scan parquet default.web_sales -Output [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Output [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#340)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#263)] ReadSchema: struct (214) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33] +Input [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] (215) ReusedExchange [Reuses operator id: 210] -Output [1]: [d_date_sk#346] +Output [1]: [d_date_sk#12] (216) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#346] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (217) Project [codegen id : 6] -Output [2]: [ws_quantity#82 AS quantity#347, ws_list_price#83 AS list_price#348] -Input [4]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#33, d_date_sk#346] +Output [2]: [ws_quantity#62 AS quantity#267, ws_list_price#63 AS list_price#268] +Input [4]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, d_date_sk#12] (218) Union (219) HashAggregate [codegen id : 7] -Input [2]: [quantity#338, list_price#339] +Input [2]: [quantity#261, list_price#262] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#338 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#339 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#349, count#350] -Results [2]: [sum#351, count#352] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#269, count#270] +Results [2]: [sum#271, count#272] (220) Exchange -Input [2]: [sum#351, count#352] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#353] +Input [2]: [sum#271, count#272] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#273] (221) HashAggregate [codegen id : 8] -Input [2]: [sum#351, count#352] +Input [2]: [sum#271, count#272] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#338 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#339 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#338 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#339 as decimal(12,2)))), DecimalType(18,2), true))#354] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#338 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#339 as decimal(12,2)))), DecimalType(18,2), true))#354 AS average_sales#355] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#274] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#274 AS average_sales#275] -Subquery:2 Hosting operator id = 195 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#336 +Subquery:2 Hosting operator id = 195 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#259 ReusedExchange (222) (222) ReusedExchange [Reuses operator id: 201] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 204 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#340 +Subquery:3 Hosting operator id = 204 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#263 ReusedExchange (223) (223) ReusedExchange [Reuses operator id: 210] -Output [1]: [d_date_sk#341] +Output [1]: [d_date_sk#12] -Subquery:4 Hosting operator id = 213 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#340 +Subquery:4 Hosting operator id = 213 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#263 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (224) (224) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#12] Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 ReusedExchange (225) -(225) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#27] +(225) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 -Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#5 -Subquery:13 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:13 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:14 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:14 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:15 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:15 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:16 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:16 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:17 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:17 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:18 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:18 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:19 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:19 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:20 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:20 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:21 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:21 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:22 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:22 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:23 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:23 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:24 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#59, [id=#60] +Subquery:24 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index e351f9e687027..1634c1e247a12 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -104,53 +104,53 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Exchange [brand_id,class_id,category_id] #7 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #10 + BroadcastExchange #11 WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Filter [i_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #8 + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter BroadcastExchange #12 WholeStageCodegen (9) From 2f03409a856d4048a5d62ba1c08508cd8c8aa081 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 23 Mar 2021 17:32:09 +0800 Subject: [PATCH 24/38] add back subquery fix --- .../analysis/DeduplicateRelations.scala | 44 +++++++++++++------ 1 file changed, 31 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index d30d29bf513fe..306c63010d49c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.analysis +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, AttributeSet, NamedExpression, PlanExpression} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule @@ -77,23 +79,39 @@ object DeduplicateRelations extends Rule[LogicalPlan] { (m, Seq(m)) } - case _ if plan.children.nonEmpty => - val newChildren = ArrayBuffer.empty[LogicalPlan] + case plan: LogicalPlan => val relations = ArrayBuffer.empty[MultiInstanceRelation] - for (c <- plan.children) { - val (renewed, collected) = renewDuplicatedRelations(existingRelations ++ relations, c) - newChildren += renewed - relations ++= collected - } + val newPlan = if (plan.children.nonEmpty) { + val newChildren = ArrayBuffer.empty[LogicalPlan] + for (c <- plan.children) { + val (renewed, collected) = renewDuplicatedRelations(existingRelations ++ relations, c) + newChildren += renewed + relations ++= collected + } - if (plan.childrenResolved) { - val attrMap = AttributeMap(plan.children.flatMap(_.output).zip( - newChildren.flatMap(_.output)).filter { case (a1, a2) => a1.exprId != a2.exprId }) - val newPlan = plan.withNewChildren(newChildren).rewriteAttrs(attrMap) - (newPlan, relations) + if (plan.childrenResolved) { + val attrMap = AttributeMap( + plan + .children + .flatMap(_.output).zip(newChildren.flatMap(_.output)) + .filter { case (a1, a2) => a1.exprId != a2.exprId } + ) + plan.withNewChildren(newChildren).rewriteAttrs(attrMap) + } else { + plan.withNewChildren(newChildren) + } } else { - (plan.withNewChildren(newChildren), relations) + plan + } + + val planWithNewSubquery = newPlan.transformExpressions { + case subquery: PlanExpression[LogicalPlan @unchecked] => + val (renewed, collected) = renewDuplicatedRelations( + existingRelations ++ relations, subquery.plan) + relations ++= collected + subquery.withNewPlan(renewed) } + (planWithNewSubquery, relations) case _ => (plan, Nil) } From f1160f2cac0249a1b4bb44ee634a0ad5bb243108 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 23 Mar 2021 17:37:22 +0800 Subject: [PATCH 25/38] gen golden files --- .../q14a.sf100/explain.txt | 844 +++++----- .../q14a.sf100/simplified.txt | 158 +- .../approved-plans-v1_4/q14a/explain.txt | 678 ++++---- .../approved-plans-v1_4/q14a/simplified.txt | 80 +- .../q14b.sf100/explain.txt | 762 ++++----- .../q14b.sf100/simplified.txt | 146 +- .../approved-plans-v1_4/q14b/explain.txt | 630 +++---- .../approved-plans-v1_4/q14b/simplified.txt | 80 +- .../approved-plans-v1_4/q32.sf100/explain.txt | 44 +- .../q32.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q32/explain.txt | 86 +- .../approved-plans-v1_4/q32/simplified.txt | 2 +- .../approved-plans-v1_4/q41.sf100/explain.txt | 48 +- .../q41.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q41/explain.txt | 48 +- .../approved-plans-v1_4/q41/simplified.txt | 2 +- .../approved-plans-v1_4/q6.sf100/explain.txt | 186 +-- .../q6.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q6/explain.txt | 100 +- .../approved-plans-v1_4/q6/simplified.txt | 2 +- .../approved-plans-v1_4/q92.sf100/explain.txt | 66 +- .../q92.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q92/explain.txt | 100 +- .../approved-plans-v1_4/q92/simplified.txt | 2 +- .../approved-plans-v2_7/q14.sf100/explain.txt | 762 ++++----- .../q14.sf100/simplified.txt | 146 +- .../approved-plans-v2_7/q14/explain.txt | 630 +++---- .../approved-plans-v2_7/q14/simplified.txt | 80 +- .../q14a.sf100/explain.txt | 1446 ++++++++--------- .../q14a.sf100/simplified.txt | 210 ++- .../approved-plans-v2_7/q14a/explain.txt | 1158 ++++++------- .../approved-plans-v2_7/q14a/simplified.txt | 80 +- .../approved-plans-v2_7/q6.sf100/explain.txt | 186 +-- .../q6.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q6/explain.txt | 100 +- .../approved-plans-v2_7/q6/simplified.txt | 2 +- 36 files changed, 4433 insertions(+), 4441 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt index d4ab3f60494d9..e4ec487623d2c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt @@ -37,39 +37,39 @@ TakeOrderedAndProject (134) : : : : +- * HashAggregate (44) : : : : +- Exchange (43) : : : : +- * HashAggregate (42) - : : : : +- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) + : : : : +- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Project (18) + : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : :- * Filter (11) + : : : : : : +- * ColumnarToRow (10) + : : : : : : +- Scan parquet default.store_sales (9) + : : : : : +- BroadcastExchange (16) + : : : : : +- * Project (15) + : : : : : +- * Filter (14) + : : : : : +- * ColumnarToRow (13) + : : : : : +- Scan parquet default.date_dim (12) + : : : : +- BroadcastExchange (39) + : : : : +- SortMergeJoin LeftSemi (38) + : : : : :- * Sort (23) + : : : : : +- Exchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (37) + : : : : +- Exchange (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- Scan parquet default.catalog_sales (24) + : : : : : +- ReusedExchange (27) + : : : : +- BroadcastExchange (33) + : : : : +- * Filter (32) + : : : : +- * ColumnarToRow (31) + : : : : +- Scan parquet default.item (30) : : : +- * Sort (57) : : : +- Exchange (56) : : : +- * Project (55) @@ -165,588 +165,588 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 20] +(7) ColumnarToRow [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 20] +(8) Filter [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 5] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +(10) ColumnarToRow [codegen id : 10] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -(11) Filter [codegen id : 5] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +(11) Filter [codegen id : 10] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_item_sk#11) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#14, d_year#15] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#14, d_year#15] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1999)) AND (d_year#15 <= 2001)) AND isnotnull(d_date_sk#14)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#14] +Input [2]: [d_date_sk#14, d_year#15] (16) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(17) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(17) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#14] Join condition: None -(18) Project [codegen id : 5] -Output [1]: [ss_item_sk#1] -Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] +(18) Project [codegen id : 10] +Output [1]: [ss_item_sk#11] +Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] (19) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) -(22) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] +(22) Exchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join condition: None +(23) Sort [codegen id : 5] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 -(24) Project [codegen id : 5] -Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] -Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] - -(25) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] - -(26) Sort [codegen id : 6] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 - -(27) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] +(24) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +(25) ColumnarToRow [codegen id : 8] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] -(29) Filter [codegen id : 9] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +(26) Filter [codegen id : 8] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#22) -(30) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +(27) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#24] -(31) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +(28) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None -(32) Project [codegen id : 9] -Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] +(29) Project [codegen id : 8] +Output [1]: [cs_item_sk#22] +Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] -(33) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(30) Scan parquet default.item +Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(31) ColumnarToRow [codegen id : 7] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -(35) Filter [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +(32) Filter [codegen id : 7] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Condition : isnotnull(i_item_sk#25) -(36) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +(33) BroadcastExchange +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_item_sk#22] +Right keys [1]: [i_item_sk#25] Join condition: None -(38) Project [codegen id : 9] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(35) Project [codegen id : 8] +Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -(39) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] +(36) Exchange +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] -(40) Sort [codegen id : 10] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(37) Sort [codegen id : 9] +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 -(41) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +(38) SortMergeJoin +Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] Join condition: None -(42) HashAggregate [codegen id : 11] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(39) BroadcastExchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] + +(40) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_item_sk#11] +Right keys [1]: [i_item_sk#17] +Join condition: None + +(41) Project [codegen id : 10] +Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] +Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] + +(42) HashAggregate [codegen id : 10] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (43) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(44) HashAggregate [codegen id : 12] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(44) HashAggregate [codegen id : 11] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (45) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] -(46) Sort [codegen id : 13] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 12] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +(48) ColumnarToRow [codegen id : 15] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -(49) Filter [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(49) Filter [codegen id : 15] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_item_sk#37) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#39] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(51) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#39] Join condition: None -(52) Project [codegen id : 16] -Output [1]: [ws_item_sk#26] -Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] +(52) Project [codegen id : 15] +Output [1]: [ws_item_sk#37] +Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] -(53) ReusedExchange [Reuses operator id: 36] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(53) ReusedExchange [Reuses operator id: 33] +Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(54) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#40] Join condition: None -(55) Project [codegen id : 16] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(55) Project [codegen id : 15] +Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] (56) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] -(57) Sort [codegen id : 17] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 16] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] +Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] Join condition: None -(59) HashAggregate [codegen id : 18] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(59) HashAggregate [codegen id : 17] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (60) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] -(61) HashAggregate [codegen id : 19] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(61) HashAggregate [codegen id : 18] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (62) BroadcastExchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] -(63) BroadcastHashJoin [codegen id : 20] +(63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#16, class_id#17, category_id#18] +Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join condition: None -(64) Project [codegen id : 20] -Output [1]: [i_item_sk#7 AS ss_item_sk#31] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] +(64) Project [codegen id : 19] +Output [1]: [i_item_sk#7 AS ss_item_sk#47] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] (65) Exchange -Input [1]: [ss_item_sk#31] -Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [1]: [ss_item_sk#47] +Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] -(66) Sort [codegen id : 21] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 20] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#47] Join condition: None (68) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#33] +Output [3]: [d_date_sk#49, d_year#50, d_moy#51] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 22] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] +(69) ColumnarToRow [codegen id : 21] +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] -(70) Filter [codegen id : 22] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#33)) AND (d_year#13 = 2001)) AND (d_moy#33 = 11)) AND isnotnull(d_date_sk#12)) +(70) Filter [codegen id : 21] +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] +Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2001)) AND (d_moy#51 = 11)) AND isnotnull(d_date_sk#49)) -(71) Project [codegen id : 22] -Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] +(71) Project [codegen id : 21] +Output [1]: [d_date_sk#49] +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] (72) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] +Input [1]: [d_date_sk#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] -(73) BroadcastHashJoin [codegen id : 44] +(73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#49] Join condition: None -(74) Project [codegen id : 44] +(74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] (75) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(76) ColumnarToRow [codegen id : 22] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(77) Filter [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +(77) Filter [codegen id : 22] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Condition : isnotnull(i_item_sk#53) (78) Exchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: hashpartitioning(i_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#57] -(79) Sort [codegen id : 24] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 +(79) Sort [codegen id : 23] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: [i_item_sk#53 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(81) Sort [codegen id : 43] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 41] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [i_item_sk#53] +Right keys [1]: [ss_item_sk#47] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] -(84) BroadcastHashJoin [codegen id : 44] +(84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#53] Join condition: None -(85) Project [codegen id : 44] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(85) Project [codegen id : 42] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(86) HashAggregate [codegen id : 44] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +(86) HashAggregate [codegen id : 42] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] +Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#37, isEmpty#38, count#39] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] +Aggregate Attributes [3]: [sum#59, isEmpty#60, count#61] +Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] +Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#65] -(88) HashAggregate [codegen id : 45] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +(88) HashAggregate [codegen id : 43] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] +Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44, count(1)#45] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sales#46, count(1)#45 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66, count(1)#67] +Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] -(89) Filter [codegen id : 45] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48 as decimal(32,6)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(89) Filter [codegen id : 43] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(32,6)) > cast(Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(90) Project [codegen id : 45] -Output [6]: [sales#46, number_sales#47, store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] +(90) Project [codegen id : 43] +Output [6]: [sales#68, number_sales#69, store AS channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] (91) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Output [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#77), dynamicpruningexpression(cs_sold_date_sk#77 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 46] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +(92) ColumnarToRow [codegen id : 44] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -(93) Filter [codegen id : 46] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +(93) Filter [codegen id : 44] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Condition : isnotnull(cs_item_sk#74) (94) Exchange -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Arguments: hashpartitioning(cs_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#78] -(95) Sort [codegen id : 47] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -Arguments: [cs_item_sk#20 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 45] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Arguments: [cs_item_sk#74 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(97) Sort [codegen id : 66] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 63] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [cs_item_sk#74] +Right keys [1]: [ss_item_sk#47] Join condition: None (99) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#79] -(100) BroadcastHashJoin [codegen id : 89] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +(100) BroadcastHashJoin [codegen id : 85] +Left keys [1]: [cs_sold_date_sk#77] +Right keys [1]: [d_date_sk#79] Join condition: None -(101) Project [codegen id : 89] -Output [3]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53] -Input [5]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] +(101) Project [codegen id : 85] +Output [3]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76] +Input [5]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77, d_date_sk#79] (102) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -(103) BroadcastHashJoin [codegen id : 89] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +(103) BroadcastHashJoin [codegen id : 85] +Left keys [1]: [cs_item_sk#74] +Right keys [1]: [i_item_sk#80] Join condition: None -(104) Project [codegen id : 89] -Output [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(104) Project [codegen id : 85] +Output [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] +Input [7]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -(105) HashAggregate [codegen id : 89] -Input [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] +(105) HashAggregate [codegen id : 85] +Input [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] +Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] +Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] (106) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, [id=#90] -(107) HashAggregate [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] +(107) HashAggregate [codegen id : 86] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] +Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] -(108) Filter [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(108) Filter [codegen id : 86] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(109) Project [codegen id : 90] -Output [6]: [sales#64, number_sales#65, catalog AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] +(109) Project [codegen id : 86] +Output [6]: [sales#93, number_sales#94, catalog AS channel#96, i_brand_id#81, i_class_id#82, i_category_id#83] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] (110) Scan parquet default.web_sales -Output [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Output [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#100), dynamicpruningexpression(ws_sold_date_sk#100 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 91] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +(111) ColumnarToRow [codegen id : 87] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -(112) Filter [codegen id : 91] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(112) Filter [codegen id : 87] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Condition : isnotnull(ws_item_sk#97) (113) Exchange -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_item_sk#26, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Arguments: hashpartitioning(ws_item_sk#97, 5), ENSURE_REQUIREMENTS, [id=#101] -(114) Sort [codegen id : 92] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Arguments: [ws_item_sk#26 ASC NULLS FIRST], false, 0 +(114) Sort [codegen id : 88] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Arguments: [ws_item_sk#97 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(116) Sort [codegen id : 111] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(116) Sort [codegen id : 106] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (117) SortMergeJoin -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [ws_item_sk#97] +Right keys [1]: [ss_item_sk#47] Join condition: None (118) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#102] -(119) BroadcastHashJoin [codegen id : 134] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(119) BroadcastHashJoin [codegen id : 128] +Left keys [1]: [ws_sold_date_sk#100] +Right keys [1]: [d_date_sk#102] Join condition: None -(120) Project [codegen id : 134] -Output [3]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69] -Input [5]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] +(120) Project [codegen id : 128] +Output [3]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99] +Input [5]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100, d_date_sk#102] (121) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] -(122) BroadcastHashJoin [codegen id : 134] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(122) BroadcastHashJoin [codegen id : 128] +Left keys [1]: [ws_item_sk#97] +Right keys [1]: [i_item_sk#103] Join condition: None -(123) Project [codegen id : 134] -Output [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(123) Project [codegen id : 128] +Output [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] +Input [7]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] -(124) HashAggregate [codegen id : 134] -Input [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#71, isEmpty#72, count#73] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] +(124) HashAggregate [codegen id : 128] +Input [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] +Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#107, isEmpty#108, count#109] +Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] (125) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +Arguments: hashpartitioning(i_brand_id#104, i_class_id#105, i_category_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] -(126) HashAggregate [codegen id : 135] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78, count(1)#79] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sales#80, count(1)#79 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +(126) HashAggregate [codegen id : 129] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114, count(1)#115] +Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sales#116, count(1)#115 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] -(127) Filter [codegen id : 135] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(127) Filter [codegen id : 129] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(128) Project [codegen id : 135] -Output [6]: [sales#80, number_sales#81, web AS channel#83, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +(128) Project [codegen id : 129] +Output [6]: [sales#116, number_sales#117, web AS channel#119, i_brand_id#104, i_class_id#105, i_category_id#106] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] (129) Union -(130) Expand [codegen id : 136] -Input [6]: [sales#46, number_sales#47, channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [List(sales#46, number_sales#47, channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 0), List(sales#46, number_sales#47, channel#51, i_brand_id#8, i_class_id#9, null, 1), List(sales#46, number_sales#47, channel#51, i_brand_id#8, null, null, 3), List(sales#46, number_sales#47, channel#51, null, null, null, 7), List(sales#46, number_sales#47, null, null, null, null, 15)], [sales#46, number_sales#47, channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] +(130) Expand [codegen id : 130] +Input [6]: [sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: [List(sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, 0), List(sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, null, 1), List(sales#68, number_sales#69, channel#73, i_brand_id#54, null, null, 3), List(sales#68, number_sales#69, channel#73, null, null, null, 7), List(sales#68, number_sales#69, null, null, null, null, 15)], [sales#68, number_sales#69, channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] -(131) HashAggregate [codegen id : 136] -Input [7]: [sales#46, number_sales#47, channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] -Keys [5]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#89, isEmpty#90, sum#91] -Results [8]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, sum#92, isEmpty#93, sum#94] +(131) HashAggregate [codegen id : 130] +Input [7]: [sales#68, number_sales#69, channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] +Keys [5]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#125, isEmpty#126, sum#127] +Results [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] (132) Exchange -Input [8]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, sum#92, isEmpty#93, sum#94] -Arguments: hashpartitioning(channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, 5), ENSURE_REQUIREMENTS, [id=#95] +Input [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] +Arguments: hashpartitioning(channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, 5), ENSURE_REQUIREMENTS, [id=#131] -(133) HashAggregate [codegen id : 137] -Input [8]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, sum#92, isEmpty#93, sum#94] -Keys [5]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#96, sum(number_sales#47)#97] -Results [6]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, sum(sales#46)#96 AS sum(sales)#98, sum(number_sales#47)#97 AS sum(number_sales)#99] +(133) HashAggregate [codegen id : 131] +Input [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] +Keys [5]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#132, sum(number_sales#69)#133] +Results [6]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales#68)#132 AS sum(sales)#134, sum(number_sales#69)#133 AS sum(number_sales)#135] (134) TakeOrderedAndProject -Input [6]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, sum(sales)#98, sum(number_sales)#99] -Arguments: 100, [channel#84 ASC NULLS FIRST, i_brand_id#85 ASC NULLS FIRST, i_class_id#86 ASC NULLS FIRST, i_category_id#87 ASC NULLS FIRST], [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, sum(sales)#98, sum(number_sales)#99] +Input [6]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales)#134, sum(number_sales)#135] +Arguments: 100, [channel#120 ASC NULLS FIRST, i_brand_id#121 ASC NULLS FIRST, i_class_id#122 ASC NULLS FIRST, i_category_id#123 ASC NULLS FIRST], [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales)#134, sum(number_sales)#135] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#49, [id=#50] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#71, [id=#72] * HashAggregate (157) +- Exchange (156) +- * HashAggregate (155) @@ -773,145 +773,145 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (135) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#100)] +PartitionFilters: [isnotnull(ss_sold_date_sk#138), dynamicpruningexpression(ss_sold_date_sk#138 IN dynamicpruning#139)] ReadSchema: struct (136) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138] (137) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#140, d_year#141] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (138) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#140, d_year#141] (139) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#140, d_year#141] +Condition : (((isnotnull(d_year#141) AND (d_year#141 >= 1999)) AND (d_year#141 <= 2001)) AND isnotnull(d_date_sk#140)) (140) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#140] +Input [2]: [d_date_sk#140, d_year#141] (141) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#101] +Input [1]: [d_date_sk#140] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#142] (142) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#138] +Right keys [1]: [d_date_sk#140] Join condition: None (143) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#102, ss_list_price#3 AS list_price#103] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#136 AS quantity#143, ss_list_price#137 AS list_price#144] +Input [4]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138, d_date_sk#140] (144) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Output [3]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#100)] +PartitionFilters: [isnotnull(cs_sold_date_sk#147), dynamicpruningexpression(cs_sold_date_sk#147 IN dynamicpruning#139)] ReadSchema: struct (145) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Input [3]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147] (146) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#148] (147) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#147] +Right keys [1]: [d_date_sk#148] Join condition: None (148) Project [codegen id : 4] -Output [2]: [cs_quantity#52 AS quantity#104, cs_list_price#53 AS list_price#105] -Input [4]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] +Output [2]: [cs_quantity#145 AS quantity#149, cs_list_price#146 AS list_price#150] +Input [4]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147, d_date_sk#148] (149) Scan parquet default.web_sales -Output [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Output [3]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#100)] +PartitionFilters: [isnotnull(ws_sold_date_sk#153), dynamicpruningexpression(ws_sold_date_sk#153 IN dynamicpruning#139)] ReadSchema: struct (150) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Input [3]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153] (151) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#154] (152) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#153] +Right keys [1]: [d_date_sk#154] Join condition: None (153) Project [codegen id : 6] -Output [2]: [ws_quantity#68 AS quantity#106, ws_list_price#69 AS list_price#107] -Input [4]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] +Output [2]: [ws_quantity#151 AS quantity#155, ws_list_price#152 AS list_price#156] +Input [4]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153, d_date_sk#154] (154) Union (155) HashAggregate [codegen id : 7] -Input [2]: [quantity#102, list_price#103] +Input [2]: [quantity#143, list_price#144] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#108, count#109] -Results [2]: [sum#110, count#111] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#157, count#158] +Results [2]: [sum#159, count#160] (156) Exchange -Input [2]: [sum#110, count#111] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#112] +Input [2]: [sum#159, count#160] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#161] (157) HashAggregate [codegen id : 8] -Input [2]: [sum#110, count#111] +Input [2]: [sum#159, count#160] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))#113] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))#113 AS average_sales#114] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))#162] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))#162 AS average_sales#163] -Subquery:2 Hosting operator id = 135 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#100 +Subquery:2 Hosting operator id = 135 Hosting Expression = ss_sold_date_sk#138 IN dynamicpruning#139 ReusedExchange (158) (158) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#140] -Subquery:3 Hosting operator id = 144 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#100 +Subquery:3 Hosting operator id = 144 Hosting Expression = cs_sold_date_sk#147 IN dynamicpruning#139 -Subquery:4 Hosting operator id = 149 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#100 +Subquery:4 Hosting operator id = 149 Hosting Expression = ws_sold_date_sk#153 IN dynamicpruning#139 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (159) (159) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#49] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 ReusedExchange (160) (160) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#14] -Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 -Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#77 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt index 4ee1ff31e9beb..2134091c43b82 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (137) + WholeStageCodegen (131) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (136) + WholeStageCodegen (130) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] InputAdapter Union - WholeStageCodegen (45) + WholeStageCodegen (43) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #3 @@ -57,7 +57,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (44) + WholeStageCodegen (42) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -76,11 +76,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - WholeStageCodegen (21) + WholeStageCodegen (20) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #5 - WholeStageCodegen (20) + WholeStageCodegen (19) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -89,84 +89,82 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 - WholeStageCodegen (19) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (18) + WholeStageCodegen (17) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (13) + WholeStageCodegen (12) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (11) + WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #10 InputAdapter - Exchange [brand_id,class_id,category_id] #10 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] + BroadcastExchange #10 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #11 + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #10 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (7) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #13 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (17) + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (16) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (16) + WholeStageCodegen (15) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -177,12 +175,12 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #11 + ReusedExchange [d_date_sk] #10 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 InputAdapter BroadcastExchange #4 - WholeStageCodegen (22) + WholeStageCodegen (21) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow @@ -191,27 +189,27 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter BroadcastExchange #16 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #17 - WholeStageCodegen (23) + WholeStageCodegen (22) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 - WholeStageCodegen (90) + WholeStageCodegen (86) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (89) + WholeStageCodegen (85) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -219,17 +217,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [cs_item_sk,ss_item_sk] - WholeStageCodegen (47) + WholeStageCodegen (45) Sort [cs_item_sk] InputAdapter Exchange [cs_item_sk] #22 - WholeStageCodegen (46) + WholeStageCodegen (44) Filter [cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (66) + WholeStageCodegen (63) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 @@ -237,14 +235,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [d_date_sk] #4 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (135) + WholeStageCodegen (129) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #23 - WholeStageCodegen (134) + WholeStageCodegen (128) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] @@ -252,17 +250,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ws_item_sk,ss_item_sk] - WholeStageCodegen (92) + WholeStageCodegen (88) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #24 - WholeStageCodegen (91) + WholeStageCodegen (87) Filter [ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (111) + WholeStageCodegen (106) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 85a3a63413f92..6f61fc8e96ae1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -31,36 +31,36 @@ TakeOrderedAndProject (115) : : : :- * HashAggregate (39) : : : : +- Exchange (38) : : : : +- * HashAggregate (37) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) + : : : : +- * Project (36) + : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : :- * Project (33) + : : : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet default.store_sales (7) + : : : : : +- BroadcastExchange (31) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Project (28) + : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet default.item (16) + : : : : : +- BroadcastExchange (26) + : : : : : +- * Project (25) + : : : : : +- * Filter (24) + : : : : : +- * ColumnarToRow (23) + : : : : : +- Scan parquet default.date_dim (22) + : : : : +- ReusedExchange (34) : : : +- BroadcastExchange (49) : : : +- * Project (48) : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -146,512 +146,512 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_item_sk#10) (10) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(11) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(12) Filter [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +(12) Filter [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(13) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +(13) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join condition: None +(14) ColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(15) Filter [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Condition : isnotnull(cs_item_sk#17) -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +(16) Scan parquet default.item +Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] 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 - -(17) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(18) Filter [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +(17) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(19) Project [codegen id : 2] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +(18) Filter [codegen id : 1] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Condition : isnotnull(i_item_sk#19) -(20) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +(19) BroadcastExchange +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#19] Join condition: None -(22) Project [codegen id : 6] -Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] -Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(23) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(22) Scan parquet default.date_dim +Output [2]: [d_date_sk#24, d_year#25] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +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 -(24) ColumnarToRow [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(23) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#24, d_year#25] -(25) Filter [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +(24) Filter [codegen id : 2] +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) -(26) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +(25) Project [codegen id : 2] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_year#25] -(27) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(26) BroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(28) Filter [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +(27) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] +Join condition: None + +(28) Project [codegen id : 3] +Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] (29) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] -(30) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +(30) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join condition: None -(31) Project [codegen id : 5] -Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(32) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(31) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +(32) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] Join condition: None -(34) Project [codegen id : 5] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(33) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(35) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] +(34) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#29] -(36) BroadcastHashJoin [codegen id : 6] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#29] Join condition: None +(36) Project [codegen id : 6] +Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] + (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (38) Exchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_item_sk#34) -(43) ReusedExchange [Reuses operator id: 29] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(43) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#36] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] +Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -(46) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(46) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#40] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#40] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] (49) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (53) BroadcastExchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#15, class_id#16, category_id#17] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#27] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] +Output [1]: [i_item_sk#6 AS ss_item_sk#43] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] (56) BroadcastExchange -Input [1]: [ss_item_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [ss_item_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#43] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Condition : isnotnull(i_item_sk#45) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [i_item_sk#45] +Right keys [1]: [ss_item_sk#43] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#45] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (66) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#30] +Output [3]: [d_date_sk#50, d_year#51, d_moy#52] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] (68) Filter [codegen id : 24] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#30)) AND (d_year#13 = 2001)) AND (d_moy#30 = 11)) AND isnotnull(d_date_sk#12)) +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] +Condition : ((((isnotnull(d_year#51) AND isnotnull(d_moy#52)) AND (d_year#51 = 2001)) AND (d_moy#52 = 11)) AND isnotnull(d_date_sk#50)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] +Output [1]: [d_date_sk#50] +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] (70) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [1]: [d_date_sk#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#50] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#32, isEmpty#33, count#34] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] +Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] +Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#60] (75) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] (76) Filter [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [sales#41, number_sales#42, store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] +Output [6]: [sales#63, number_sales#64, store AS channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] (78) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Output [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] (80) Filter [codegen id : 51] -Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +Condition : isnotnull(cs_item_sk#69) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [cs_item_sk#69] +Right keys [1]: [ss_item_sk#43] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] (84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [cs_item_sk#69] +Right keys [1]: [i_item_sk#73] Join condition: None (85) Project [codegen id : 51] -Output [6]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [8]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] (86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#77] (87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#72] +Right keys [1]: [d_date_sk#77] Join condition: None (88) Project [codegen id : 51] -Output [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [7]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76, d_date_sk#77] (89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#49, isEmpty#50, count#51] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] +Input [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] +Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#78, isEmpty#79, count#80] +Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] (90) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Arguments: hashpartitioning(i_brand_id#74, i_class_id#75, i_category_id#76, 5), ENSURE_REQUIREMENTS, [id=#84] (91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56, count(1)#57] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sales#58, count(1)#57 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85, count(1)#86] +Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sales#87, count(1)#86 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] (92) Filter [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [sales#58, number_sales#59, catalog AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] +Output [6]: [sales#87, number_sales#88, catalog AS channel#90, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] (94) Scan parquet default.web_sales -Output [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Output [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] (96) Filter [codegen id : 77] -Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Condition : isnotnull(ws_item_sk#91) (97) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [ws_item_sk#91] +Right keys [1]: [ss_item_sk#43] Join condition: None (99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] (100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#91] +Right keys [1]: [i_item_sk#95] Join condition: None (101) Project [codegen id : 77] -Output [6]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [8]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] (102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#99] (103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#94] +Right keys [1]: [d_date_sk#99] Join condition: None (104) Project [codegen id : 77] -Output [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [7]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98, d_date_sk#99] (105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] +Input [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] +Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#100, isEmpty#101, count#102] +Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] (106) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Arguments: hashpartitioning(i_brand_id#96, i_class_id#97, i_category_id#98, 5), ENSURE_REQUIREMENTS, [id=#106] (107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71, count(1)#72] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sales#73, count(1)#72 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] +Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#109, count(1)#108 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] (108) Filter [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [sales#73, number_sales#74, web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] +Output [6]: [sales#109, number_sales#110, web AS channel#112, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] (110) Union (111) Expand [codegen id : 79] -Input [6]: [sales#41, number_sales#42, channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: [List(sales#41, number_sales#42, channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#41, number_sales#42, channel#46, i_brand_id#7, i_class_id#8, null, 1), List(sales#41, number_sales#42, channel#46, i_brand_id#7, null, null, 3), List(sales#41, number_sales#42, channel#46, null, null, null, 7), List(sales#41, number_sales#42, null, null, null, null, 15)], [sales#41, number_sales#42, channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] +Input [6]: [sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] +Arguments: [List(sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, 0), List(sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, null, 1), List(sales#63, number_sales#64, channel#68, i_brand_id#46, null, null, 3), List(sales#63, number_sales#64, channel#68, null, null, null, 7), List(sales#63, number_sales#64, null, null, null, null, 15)], [sales#63, number_sales#64, channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] (112) HashAggregate [codegen id : 79] -Input [7]: [sales#41, number_sales#42, channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] -Keys [5]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#82, isEmpty#83, sum#84] -Results [8]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, sum#85, isEmpty#86, sum#87] +Input [7]: [sales#63, number_sales#64, channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] +Keys [5]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#118, isEmpty#119, sum#120] +Results [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] (113) Exchange -Input [8]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, sum#85, isEmpty#86, sum#87] -Arguments: hashpartitioning(channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] +Arguments: hashpartitioning(channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, 5), ENSURE_REQUIREMENTS, [id=#124] (114) HashAggregate [codegen id : 80] -Input [8]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, sum#85, isEmpty#86, sum#87] -Keys [5]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#89, sum(number_sales#42)#90] -Results [6]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, sum(sales#41)#89 AS sum(sales)#91, sum(number_sales#42)#90 AS sum(number_sales)#92] +Input [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] +Keys [5]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#125, sum(number_sales#64)#126] +Results [6]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales#63)#125 AS sum(sales)#127, sum(number_sales#64)#126 AS sum(number_sales)#128] (115) TakeOrderedAndProject -Input [6]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, sum(sales)#91, sum(number_sales)#92] -Arguments: 100, [channel#77 ASC NULLS FIRST, i_brand_id#78 ASC NULLS FIRST, i_class_id#79 ASC NULLS FIRST, i_category_id#80 ASC NULLS FIRST], [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, sum(sales)#91, sum(number_sales)#92] +Input [6]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales)#127, sum(number_sales)#128] +Arguments: 100, [channel#113 ASC NULLS FIRST, i_brand_id#114 ASC NULLS FIRST, i_class_id#115 ASC NULLS FIRST, i_category_id#116 ASC NULLS FIRST], [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales)#127, sum(number_sales)#128] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#66, [id=#67] * HashAggregate (138) +- Exchange (137) +- * HashAggregate (136) @@ -678,145 +678,145 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (116) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#93)] +PartitionFilters: [isnotnull(ss_sold_date_sk#131), dynamicpruningexpression(ss_sold_date_sk#131 IN dynamicpruning#132)] ReadSchema: struct (117) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131] (118) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#133, d_year#134] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (119) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#133, d_year#134] (120) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#133, d_year#134] +Condition : (((isnotnull(d_year#134) AND (d_year#134 >= 1999)) AND (d_year#134 <= 2001)) AND isnotnull(d_date_sk#133)) (121) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#133] +Input [2]: [d_date_sk#133, d_year#134] (122) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#94] +Input [1]: [d_date_sk#133] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#135] (123) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#131] +Right keys [1]: [d_date_sk#133] Join condition: None (124) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#95, ss_list_price#3 AS list_price#96] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#129 AS quantity#136, ss_list_price#130 AS list_price#137] +Input [4]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131, d_date_sk#133] (125) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Output [3]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#93)] +PartitionFilters: [isnotnull(cs_sold_date_sk#140), dynamicpruningexpression(cs_sold_date_sk#140 IN dynamicpruning#132)] ReadSchema: struct (126) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Input [3]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140] (127) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#141] (128) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#140] +Right keys [1]: [d_date_sk#141] Join condition: None (129) Project [codegen id : 4] -Output [2]: [cs_quantity#47 AS quantity#97, cs_list_price#48 AS list_price#98] -Input [4]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, d_date_sk#12] +Output [2]: [cs_quantity#138 AS quantity#142, cs_list_price#139 AS list_price#143] +Input [4]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140, d_date_sk#141] (130) Scan parquet default.web_sales -Output [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Output [3]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#93)] +PartitionFilters: [isnotnull(ws_sold_date_sk#146), dynamicpruningexpression(ws_sold_date_sk#146 IN dynamicpruning#132)] ReadSchema: struct (131) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Input [3]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146] (132) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#147] (133) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#146] +Right keys [1]: [d_date_sk#147] Join condition: None (134) Project [codegen id : 6] -Output [2]: [ws_quantity#62 AS quantity#99, ws_list_price#63 AS list_price#100] -Input [4]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, d_date_sk#12] +Output [2]: [ws_quantity#144 AS quantity#148, ws_list_price#145 AS list_price#149] +Input [4]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146, d_date_sk#147] (135) Union (136) HashAggregate [codegen id : 7] -Input [2]: [quantity#95, list_price#96] +Input [2]: [quantity#136, list_price#137] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#101, count#102] -Results [2]: [sum#103, count#104] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#150, count#151] +Results [2]: [sum#152, count#153] (137) Exchange -Input [2]: [sum#103, count#104] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] +Input [2]: [sum#152, count#153] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#154] (138) HashAggregate [codegen id : 8] -Input [2]: [sum#103, count#104] +Input [2]: [sum#152, count#153] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))#106] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))#106 AS average_sales#107] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#155] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#155 AS average_sales#156] -Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#93 +Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#131 IN dynamicpruning#132 ReusedExchange (139) (139) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#133] -Subquery:3 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#93 +Subquery:3 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#140 IN dynamicpruning#132 -Subquery:4 Hosting operator id = 130 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#93 +Subquery:4 Hosting operator id = 130 Hosting Expression = ws_sold_date_sk#146 IN dynamicpruning#132 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#50] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 ReusedExchange (141) -(141) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(141) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#29] -Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index f9b76560e363c..9ae1a0e30e90e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -90,53 +90,53 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Exchange [brand_id,class_id,category_id] #6 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #10 + BroadcastExchange #9 WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (1) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #7 + BroadcastExchange #7 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #11 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt index 76f5ac48f96cc..fab231dc80a6c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt @@ -33,39 +33,39 @@ TakeOrderedAndProject (116) : : : : +- * HashAggregate (44) : : : : +- Exchange (43) : : : : +- * HashAggregate (42) - : : : : +- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) + : : : : +- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Project (18) + : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : :- * Filter (11) + : : : : : : +- * ColumnarToRow (10) + : : : : : : +- Scan parquet default.store_sales (9) + : : : : : +- BroadcastExchange (16) + : : : : : +- * Project (15) + : : : : : +- * Filter (14) + : : : : : +- * ColumnarToRow (13) + : : : : : +- Scan parquet default.date_dim (12) + : : : : +- BroadcastExchange (39) + : : : : +- SortMergeJoin LeftSemi (38) + : : : : :- * Sort (23) + : : : : : +- Exchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (37) + : : : : +- Exchange (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- Scan parquet default.catalog_sales (24) + : : : : : +- ReusedExchange (27) + : : : : +- BroadcastExchange (33) + : : : : +- * Filter (32) + : : : : +- * ColumnarToRow (31) + : : : : +- Scan parquet default.item (30) : : : +- * Sort (57) : : : +- Exchange (56) : : : +- * Project (55) @@ -147,507 +147,507 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 20] +(7) ColumnarToRow [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 20] +(8) Filter [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 5] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +(10) ColumnarToRow [codegen id : 10] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -(11) Filter [codegen id : 5] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +(11) Filter [codegen id : 10] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_item_sk#11) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#14, d_year#15] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#14, d_year#15] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1999)) AND (d_year#15 <= 2001)) AND isnotnull(d_date_sk#14)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#14] +Input [2]: [d_date_sk#14, d_year#15] (16) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(17) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(17) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#14] Join condition: None -(18) Project [codegen id : 5] -Output [1]: [ss_item_sk#1] -Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] +(18) Project [codegen id : 10] +Output [1]: [ss_item_sk#11] +Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] (19) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) - -(22) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] - -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join condition: None - -(24) Project [codegen id : 5] -Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] -Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) -(25) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] +(22) Exchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] -(26) Sort [codegen id : 6] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 +(23) Sort [codegen id : 5] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 -(27) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] +(24) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +(25) ColumnarToRow [codegen id : 8] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] -(29) Filter [codegen id : 9] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +(26) Filter [codegen id : 8] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#22) -(30) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +(27) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#24] -(31) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +(28) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None -(32) Project [codegen id : 9] -Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] +(29) Project [codegen id : 8] +Output [1]: [cs_item_sk#22] +Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] -(33) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(30) Scan parquet default.item +Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(31) ColumnarToRow [codegen id : 7] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -(35) Filter [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +(32) Filter [codegen id : 7] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Condition : isnotnull(i_item_sk#25) -(36) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +(33) BroadcastExchange +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_item_sk#22] +Right keys [1]: [i_item_sk#25] Join condition: None -(38) Project [codegen id : 9] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(35) Project [codegen id : 8] +Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] + +(36) Exchange +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] -(39) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] +(37) Sort [codegen id : 9] +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 -(40) Sort [codegen id : 10] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(38) SortMergeJoin +Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] +Join condition: None + +(39) BroadcastExchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] -(41) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +(40) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_item_sk#11] +Right keys [1]: [i_item_sk#17] Join condition: None -(42) HashAggregate [codegen id : 11] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(41) Project [codegen id : 10] +Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] +Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] + +(42) HashAggregate [codegen id : 10] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (43) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(44) HashAggregate [codegen id : 12] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(44) HashAggregate [codegen id : 11] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (45) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] -(46) Sort [codegen id : 13] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 12] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +(48) ColumnarToRow [codegen id : 15] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -(49) Filter [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(49) Filter [codegen id : 15] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_item_sk#37) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#39] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(51) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#39] Join condition: None -(52) Project [codegen id : 16] -Output [1]: [ws_item_sk#26] -Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] +(52) Project [codegen id : 15] +Output [1]: [ws_item_sk#37] +Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] -(53) ReusedExchange [Reuses operator id: 36] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(53) ReusedExchange [Reuses operator id: 33] +Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(54) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#40] Join condition: None -(55) Project [codegen id : 16] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(55) Project [codegen id : 15] +Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] (56) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] -(57) Sort [codegen id : 17] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 16] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] +Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] Join condition: None -(59) HashAggregate [codegen id : 18] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(59) HashAggregate [codegen id : 17] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (60) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] -(61) HashAggregate [codegen id : 19] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(61) HashAggregate [codegen id : 18] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (62) BroadcastExchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] -(63) BroadcastHashJoin [codegen id : 20] +(63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#16, class_id#17, category_id#18] +Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join condition: None -(64) Project [codegen id : 20] -Output [1]: [i_item_sk#7 AS ss_item_sk#31] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] +(64) Project [codegen id : 19] +Output [1]: [i_item_sk#7 AS ss_item_sk#47] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] (65) Exchange -Input [1]: [ss_item_sk#31] -Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [1]: [ss_item_sk#47] +Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] -(66) Sort [codegen id : 21] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 20] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#47] Join condition: None (68) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#33] +Output [2]: [d_date_sk#49, d_week_seq#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 22] -Input [2]: [d_date_sk#12, d_week_seq#33] +(69) ColumnarToRow [codegen id : 21] +Input [2]: [d_date_sk#49, d_week_seq#50] -(70) Filter [codegen id : 22] -Input [2]: [d_date_sk#12, d_week_seq#33] -Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#34, [id=#35])) AND isnotnull(d_date_sk#12)) +(70) Filter [codegen id : 21] +Input [2]: [d_date_sk#49, d_week_seq#50] +Condition : ((isnotnull(d_week_seq#50) AND (d_week_seq#50 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#49)) -(71) Project [codegen id : 22] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#33] +(71) Project [codegen id : 21] +Output [1]: [d_date_sk#49] +Input [2]: [d_date_sk#49, d_week_seq#50] (72) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [1]: [d_date_sk#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] -(73) BroadcastHashJoin [codegen id : 44] +(73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#49] Join condition: None -(74) Project [codegen id : 44] +(74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] (75) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(76) ColumnarToRow [codegen id : 22] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -(77) Filter [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +(77) Filter [codegen id : 22] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Condition : (((isnotnull(i_item_sk#54) AND isnotnull(i_brand_id#55)) AND isnotnull(i_class_id#56)) AND isnotnull(i_category_id#57)) (78) Exchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: hashpartitioning(i_item_sk#54, 5), ENSURE_REQUIREMENTS, [id=#58] -(79) Sort [codegen id : 24] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 +(79) Sort [codegen id : 23] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: [i_item_sk#54 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(81) Sort [codegen id : 43] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 41] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [i_item_sk#54] +Right keys [1]: [ss_item_sk#47] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#59] -(84) BroadcastHashJoin [codegen id : 44] +(84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#54] Join condition: None -(85) Project [codegen id : 44] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(85) Project [codegen id : 42] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -(86) HashAggregate [codegen id : 44] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +(86) HashAggregate [codegen id : 42] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] +Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#39, isEmpty#40, count#41] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] +Aggregate Attributes [3]: [sum#60, isEmpty#61, count#62] +Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] +Arguments: hashpartitioning(i_brand_id#55, i_class_id#56, i_category_id#57, 5), ENSURE_REQUIREMENTS, [id=#66] -(88) HashAggregate [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +(88) HashAggregate [codegen id : 86] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] +Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46, count(1)#47] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sales#48, count(1)#47 AS number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67, count(1)#68] +Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#69, count(1)#68 AS number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] -(89) Filter [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) +(89) Filter [codegen id : 86] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(32,6)) > cast(Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) -(90) Project [codegen id : 90] -Output [6]: [store AS channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] +(90) Project [codegen id : 86] +Output [6]: [store AS channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] (91) Scan parquet default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#54)] +PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 45] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +(92) ColumnarToRow [codegen id : 43] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -(93) Filter [codegen id : 45] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +(93) Filter [codegen id : 43] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Condition : isnotnull(ss_item_sk#75) (94) Exchange -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Arguments: hashpartitioning(ss_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#80] -(95) Sort [codegen id : 46] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 44] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Arguments: [ss_item_sk#75 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(97) Sort [codegen id : 65] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 62] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [ss_item_sk#75] +Right keys [1]: [ss_item_sk#47] Join condition: None (99) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#33] +Output [2]: [d_date_sk#81, d_week_seq#82] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 66] -Input [2]: [d_date_sk#12, d_week_seq#33] +(100) ColumnarToRow [codegen id : 63] +Input [2]: [d_date_sk#81, d_week_seq#82] -(101) Filter [codegen id : 66] -Input [2]: [d_date_sk#12, d_week_seq#33] -Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#56, [id=#57])) AND isnotnull(d_date_sk#12)) +(101) Filter [codegen id : 63] +Input [2]: [d_date_sk#81, d_week_seq#82] +Condition : ((isnotnull(d_week_seq#82) AND (d_week_seq#82 = Subquery scalar-subquery#83, [id=#84])) AND isnotnull(d_date_sk#81)) -(102) Project [codegen id : 66] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#33] +(102) Project [codegen id : 63] +Output [1]: [d_date_sk#81] +Input [2]: [d_date_sk#81, d_week_seq#82] (103) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] +Input [1]: [d_date_sk#81] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#85] -(104) BroadcastHashJoin [codegen id : 88] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(104) BroadcastHashJoin [codegen id : 84] +Left keys [1]: [ss_sold_date_sk#78] +Right keys [1]: [d_date_sk#81] Join condition: None -(105) Project [codegen id : 88] -Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +(105) Project [codegen id : 84] +Output [3]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77] +Input [5]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#81] (106) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +Output [4]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] -(107) BroadcastHashJoin [codegen id : 88] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#59] +(107) BroadcastHashJoin [codegen id : 84] +Left keys [1]: [ss_item_sk#75] +Right keys [1]: [i_item_sk#86] Join condition: None -(108) Project [codegen id : 88] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +(108) Project [codegen id : 84] +Output [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] +Input [7]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] -(109) HashAggregate [codegen id : 88] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] +(109) HashAggregate [codegen id : 84] +Input [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] +Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] +Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] (110) Exchange -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] -Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] +Arguments: hashpartitioning(i_brand_id#87, i_class_id#88, i_category_id#89, 5), ENSURE_REQUIREMENTS, [id=#96] -(111) HashAggregate [codegen id : 89] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70, count(1)#71] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sales#72, count(1)#71 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] +(111) HashAggregate [codegen id : 85] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] +Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] +Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#99, count(1)#98 AS number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] -(112) Filter [codegen id : 89] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) +(112) Filter [codegen id : 85] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) -(113) Project [codegen id : 89] -Output [6]: [store AS channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] +(113) Project [codegen id : 85] +Output [6]: [store AS channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] (114) BroadcastExchange -Input [6]: [channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#76] +Input [6]: [channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#103] -(115) BroadcastHashJoin [codegen id : 90] -Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +(115) BroadcastHashJoin [codegen id : 86] +Left keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] +Right keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] Join condition: None (116) TakeOrderedAndProject -Input [12]: [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Input [12]: [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Arguments: 100, [i_brand_id#55 ASC NULLS FIRST, i_class_id#56 ASC NULLS FIRST, i_category_id#57 ASC NULLS FIRST], [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#72, [id=#73] * HashAggregate (139) +- Exchange (138) +- * HashAggregate (137) @@ -674,140 +674,140 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (117) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ss_sold_date_sk#106), dynamicpruningexpression(ss_sold_date_sk#106 IN dynamicpruning#107)] ReadSchema: struct (118) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] (119) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#108, d_year#109] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (120) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#108, d_year#109] (121) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#108, d_year#109] +Condition : (((isnotnull(d_year#109) AND (d_year#109 >= 1999)) AND (d_year#109 <= 2001)) AND isnotnull(d_date_sk#108)) (122) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#108] +Input [2]: [d_date_sk#108, d_year#109] (123) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] +Input [1]: [d_date_sk#108] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#110] (124) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#106] +Right keys [1]: [d_date_sk#108] Join condition: None (125) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#79, ss_list_price#3 AS list_price#80] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#104 AS quantity#111, ss_list_price#105 AS list_price#112] +Input [4]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106, d_date_sk#108] (126) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] +Output [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(cs_sold_date_sk#115), dynamicpruningexpression(cs_sold_date_sk#115 IN dynamicpruning#107)] ReadSchema: struct (127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] +Input [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] (128) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#116] (129) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#115] +Right keys [1]: [d_date_sk#116] Join condition: None (130) Project [codegen id : 4] -Output [2]: [cs_quantity#81 AS quantity#83, cs_list_price#82 AS list_price#84] -Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21, d_date_sk#12] +Output [2]: [cs_quantity#113 AS quantity#117, cs_list_price#114 AS list_price#118] +Input [4]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115, d_date_sk#116] (131) Scan parquet default.web_sales -Output [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] +Output [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#107)] ReadSchema: struct (132) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] +Input [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] (133) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#122] (134) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#121] +Right keys [1]: [d_date_sk#122] Join condition: None (135) Project [codegen id : 6] -Output [2]: [ws_quantity#85 AS quantity#87, ws_list_price#86 AS list_price#88] -Input [4]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27, d_date_sk#12] +Output [2]: [ws_quantity#119 AS quantity#123, ws_list_price#120 AS list_price#124] +Input [4]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121, d_date_sk#122] (136) Union (137) HashAggregate [codegen id : 7] -Input [2]: [quantity#79, list_price#80] +Input [2]: [quantity#111, list_price#112] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#89, count#90] -Results [2]: [sum#91, count#92] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#125, count#126] +Results [2]: [sum#127, count#128] (138) Exchange -Input [2]: [sum#91, count#92] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] +Input [2]: [sum#127, count#128] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] (139) HashAggregate [codegen id : 8] -Input [2]: [sum#91, count#92] +Input [2]: [sum#127, count#128] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94 AS average_sales#95] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130 AS average_sales#131] -Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#77 +Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#106 IN dynamicpruning#107 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#108] -Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#77 +Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#115 IN dynamicpruning#107 -Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#77 +Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#107 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (141) (141) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#49] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 ReusedExchange (142) (142) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#14] -Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 -Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#34, [id=#35] +Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#51, [id=#52] * Project (146) +- * Filter (145) +- * ColumnarToRow (144) @@ -815,33 +815,33 @@ Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquer (143) Scan parquet default.date_dim -Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] (145) Filter [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 2000)) AND (d_moy#96 = 12)) AND (d_dom#97 = 11)) +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Condition : (((((isnotnull(d_year#133) AND isnotnull(d_moy#134)) AND isnotnull(d_dom#135)) AND (d_year#133 = 2000)) AND (d_moy#134 = 12)) AND (d_dom#135 = 11)) (146) Project [codegen id : 1] -Output [1]: [d_week_seq#33] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [1]: [d_week_seq#132] +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] -Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] +Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] -Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#54 +Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#79 ReusedExchange (147) (147) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#81] -Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#56, [id=#57] +Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#83, [id=#84] * Project (151) +- * Filter (150) +- * ColumnarToRow (149) @@ -849,21 +849,21 @@ Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subqu (148) Scan parquet default.date_dim -Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (149) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] (150) Filter [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 1999)) AND (d_moy#96 = 12)) AND (d_dom#97 = 11)) +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Condition : (((((isnotnull(d_year#137) AND isnotnull(d_moy#138)) AND isnotnull(d_dom#139)) AND (d_year#137 = 1999)) AND (d_moy#138 = 12)) AND (d_dom#139 = 11)) (151) Project [codegen id : 1] -Output [1]: [d_week_seq#33] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [1]: [d_week_seq#136] +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt index 1d5907e319658..1b41abbc97b06 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (90) + WholeStageCodegen (86) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] @@ -49,7 +49,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (44) + WholeStageCodegen (42) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -68,11 +68,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #3 - WholeStageCodegen (21) + WholeStageCodegen (20) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 - WholeStageCodegen (20) + WholeStageCodegen (19) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -81,84 +81,82 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (19) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (18) + WholeStageCodegen (17) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (13) + WholeStageCodegen (12) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (11) + WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #9 InputAdapter - Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] + BroadcastExchange #9 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #10 + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #9 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (7) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (17) + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (16) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (16) + WholeStageCodegen (15) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -169,12 +167,12 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #10 + ReusedExchange [d_date_sk] #9 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter BroadcastExchange #3 - WholeStageCodegen (22) + WholeStageCodegen (21) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #3 @@ -190,29 +188,29 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #15 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #16 - WholeStageCodegen (23) + WholeStageCodegen (22) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #20 - WholeStageCodegen (89) + WholeStageCodegen (85) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (88) + WholeStageCodegen (84) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -220,24 +218,24 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] - WholeStageCodegen (46) + WholeStageCodegen (44) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #22 - WholeStageCodegen (45) + WholeStageCodegen (43) Filter [ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #6 ReusedExchange [d_date_sk] #23 - WholeStageCodegen (65) + WholeStageCodegen (62) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #23 - WholeStageCodegen (66) + WholeStageCodegen (63) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 01bcb83622844..ae653b43971db 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -27,36 +27,36 @@ TakeOrderedAndProject (100) : : : :- * HashAggregate (39) : : : : +- Exchange (38) : : : : +- * HashAggregate (37) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) + : : : : +- * Project (36) + : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : :- * Project (33) + : : : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet default.store_sales (7) + : : : : : +- BroadcastExchange (31) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Project (28) + : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet default.item (16) + : : : : : +- BroadcastExchange (26) + : : : : : +- * Project (25) + : : : : : +- * Filter (24) + : : : : : +- * ColumnarToRow (23) + : : : : : +- Scan parquet default.date_dim (22) + : : : : +- ReusedExchange (34) : : : +- BroadcastExchange (49) : : : +- * Project (48) : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -131,443 +131,443 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_item_sk#10) (10) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(11) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(12) Filter [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +(12) Filter [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(13) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +(13) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join condition: None +(14) ColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(15) Filter [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Condition : isnotnull(cs_item_sk#17) -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +(16) Scan parquet default.item +Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] 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 - -(17) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(18) Filter [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +(17) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(19) Project [codegen id : 2] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +(18) Filter [codegen id : 1] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Condition : isnotnull(i_item_sk#19) -(20) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +(19) BroadcastExchange +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#19] Join condition: None -(22) Project [codegen id : 6] -Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] -Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(23) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(22) Scan parquet default.date_dim +Output [2]: [d_date_sk#24, d_year#25] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +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 -(24) ColumnarToRow [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(23) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#24, d_year#25] -(25) Filter [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +(24) Filter [codegen id : 2] +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) -(26) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +(25) Project [codegen id : 2] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_year#25] -(27) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(26) BroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(28) Filter [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +(27) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] +Join condition: None + +(28) Project [codegen id : 3] +Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] (29) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] -(30) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +(30) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join condition: None -(31) Project [codegen id : 5] -Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(32) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(31) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +(32) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] Join condition: None -(34) Project [codegen id : 5] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(33) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(35) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] +(34) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#29] -(36) BroadcastHashJoin [codegen id : 6] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#29] Join condition: None +(36) Project [codegen id : 6] +Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] + (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (38) Exchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_item_sk#34) -(43) ReusedExchange [Reuses operator id: 29] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(43) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#36] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] +Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -(46) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(46) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#40] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#40] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] (49) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (53) BroadcastExchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#15, class_id#16, category_id#17] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#27] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] +Output [1]: [i_item_sk#6 AS ss_item_sk#43] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] (56) BroadcastExchange -Input [1]: [ss_item_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [ss_item_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#43] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Condition : (((isnotnull(i_item_sk#45) AND isnotnull(i_brand_id#46)) AND isnotnull(i_class_id#47)) AND isnotnull(i_category_id#48)) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [i_item_sk#45] +Right keys [1]: [ss_item_sk#43] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#45] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (66) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#30] +Output [2]: [d_date_sk#50, d_week_seq#51] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [2]: [d_date_sk#12, d_week_seq#30] +Input [2]: [d_date_sk#50, d_week_seq#51] (68) Filter [codegen id : 24] -Input [2]: [d_date_sk#12, d_week_seq#30] -Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#31, [id=#32])) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#50, d_week_seq#51] +Condition : ((isnotnull(d_week_seq#51) AND (d_week_seq#51 = Subquery scalar-subquery#52, [id=#53])) AND isnotnull(d_date_sk#50)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#30] +Output [1]: [d_date_sk#50] +Input [2]: [d_date_sk#50, d_week_seq#51] (70) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] +Input [1]: [d_date_sk#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#50] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#34, isEmpty#35, count#36] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] +Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] +Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#61] (75) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41, count(1)#42] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sales#43, count(1)#42 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] (76) Filter [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45 as decimal(32,6)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) (77) Project [codegen id : 52] -Output [6]: [store AS channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] +Output [6]: [store AS channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] (78) Scan parquet default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#49)] +PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] (80) Filter [codegen id : 50] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +Condition : isnotnull(ss_item_sk#70) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [ss_item_sk#70] +Right keys [1]: [ss_item_sk#43] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] +Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] (84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#50] +Left keys [1]: [ss_item_sk#70] +Right keys [1]: [i_item_sk#75] Join condition: None (85) Project [codegen id : 50] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] +Output [6]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] +Input [8]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#30] +Output [2]: [d_date_sk#79, d_week_seq#80] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#12, d_week_seq#30] +Input [2]: [d_date_sk#79, d_week_seq#80] (88) Filter [codegen id : 49] -Input [2]: [d_date_sk#12, d_week_seq#30] -Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#54, [id=#55])) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#79, d_week_seq#80] +Condition : ((isnotnull(d_week_seq#80) AND (d_week_seq#80 = Subquery scalar-subquery#81, [id=#82])) AND isnotnull(d_date_sk#79)) (89) Project [codegen id : 49] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#30] +Output [1]: [d_date_sk#79] +Input [2]: [d_date_sk#79, d_week_seq#80] (90) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +Input [1]: [d_date_sk#79] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#83] (91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#73] +Right keys [1]: [d_date_sk#79] Join condition: None (92) Project [codegen id : 50] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53, d_date_sk#12] +Output [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] +Input [7]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78, d_date_sk#79] (93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] -Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#57, isEmpty#58, count#59] -Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] +Input [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] +Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] +Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] (94) Exchange -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] -Arguments: hashpartitioning(i_brand_id#51, i_class_id#52, i_category_id#53, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] +Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, [id=#90] (95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] -Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] -Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] +Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] +Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] (96) Filter [codegen id : 51] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) (97) Project [codegen id : 51] -Output [6]: [store AS channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] +Output [6]: [store AS channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] (98) BroadcastExchange -Input [6]: [channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] +Input [6]: [channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#97] (99) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] +Left keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Right keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] Join condition: None (100) TakeOrderedAndProject -Input [12]: [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Input [12]: [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Arguments: 100, [i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#46, [id=#47] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#67, [id=#68] * HashAggregate (123) +- Exchange (122) +- * HashAggregate (121) @@ -594,140 +594,140 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(ss_sold_date_sk#100), dynamicpruningexpression(ss_sold_date_sk#100 IN dynamicpruning#101)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] (103) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#102, d_year#103] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#102, d_year#103] (105) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#102, d_year#103] +Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 <= 2001)) AND isnotnull(d_date_sk#102)) (106) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#102] +Input [2]: [d_date_sk#102, d_year#103] (107) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] +Input [1]: [d_date_sk#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#104] (108) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#100] +Right keys [1]: [d_date_sk#102] Join condition: None (109) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#73, ss_list_price#3 AS list_price#74] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#98 AS quantity#105, ss_list_price#99 AS list_price#106] +Input [4]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100, d_date_sk#102] (110) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] +Output [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(cs_sold_date_sk#109), dynamicpruningexpression(cs_sold_date_sk#109 IN dynamicpruning#101)] ReadSchema: struct (111) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] +Input [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] (112) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#110] (113) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#109] +Right keys [1]: [d_date_sk#110] Join condition: None (114) Project [codegen id : 4] -Output [2]: [cs_quantity#75 AS quantity#77, cs_list_price#76 AS list_price#78] -Input [4]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19, d_date_sk#12] +Output [2]: [cs_quantity#107 AS quantity#111, cs_list_price#108 AS list_price#112] +Input [4]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109, d_date_sk#110] (115) Scan parquet default.web_sales -Output [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] +Output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(ws_sold_date_sk#115), dynamicpruningexpression(ws_sold_date_sk#115 IN dynamicpruning#101)] ReadSchema: struct (116) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] +Input [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] (117) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#116] (118) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#115] +Right keys [1]: [d_date_sk#116] Join condition: None (119) Project [codegen id : 6] -Output [2]: [ws_quantity#79 AS quantity#81, ws_list_price#80 AS list_price#82] -Input [4]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24, d_date_sk#12] +Output [2]: [ws_quantity#113 AS quantity#117, ws_list_price#114 AS list_price#118] +Input [4]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115, d_date_sk#116] (120) Union (121) HashAggregate [codegen id : 7] -Input [2]: [quantity#73, list_price#74] +Input [2]: [quantity#105, list_price#106] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#83, count#84] -Results [2]: [sum#85, count#86] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#119, count#120] +Results [2]: [sum#121, count#122] (122) Exchange -Input [2]: [sum#85, count#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#87] +Input [2]: [sum#121, count#122] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#123] (123) HashAggregate [codegen id : 8] -Input [2]: [sum#85, count#86] +Input [2]: [sum#121, count#122] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88 AS average_sales#89] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124 AS average_sales#125] -Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#71 +Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#100 IN dynamicpruning#101 ReusedExchange (124) (124) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#102] -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#71 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#109 IN dynamicpruning#101 -Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#71 +Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#115 IN dynamicpruning#101 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#50] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 ReusedExchange (126) -(126) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(126) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#29] -Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#31, [id=#32] +Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] * Project (130) +- * Filter (129) +- * ColumnarToRow (128) @@ -735,33 +735,33 @@ Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer (127) Scan parquet default.date_dim -Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (128) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] (129) Filter [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 2000)) AND (d_moy#90 = 12)) AND (d_dom#91 = 11)) +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Condition : (((((isnotnull(d_year#127) AND isnotnull(d_moy#128)) AND isnotnull(d_dom#129)) AND (d_year#127 = 2000)) AND (d_moy#128 = 12)) AND (d_dom#129 = 11)) (130) Project [codegen id : 1] -Output [1]: [d_week_seq#30] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [1]: [d_week_seq#126] +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] -Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] -Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#49 +Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 ReusedExchange (131) (131) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#79] -Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#81, [id=#82] * Project (135) +- * Filter (134) +- * ColumnarToRow (133) @@ -769,21 +769,21 @@ Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subque (132) Scan parquet default.date_dim -Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (133) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] (134) Filter [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 1999)) AND (d_moy#90 = 12)) AND (d_dom#91 = 11)) +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Condition : (((((isnotnull(d_year#131) AND isnotnull(d_moy#132)) AND isnotnull(d_dom#133)) AND (d_year#131 = 1999)) AND (d_moy#132 = 12)) AND (d_dom#133 = 11)) (135) Project [codegen id : 1] -Output [1]: [d_week_seq#30] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [1]: [d_week_seq#130] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index 3507d76bcf9b9..cb9b410ea8568 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -82,53 +82,53 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Exchange [brand_id,class_id,category_id] #5 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #9 + BroadcastExchange #8 WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 + BroadcastExchange #6 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt index b70c36db4bc9d..5d171e5f595b9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt @@ -81,7 +81,7 @@ Input [2]: [d_date_sk#8, d_date#9] (11) Filter [codegen id : 2] Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) (12) Project [codegen id : 2] Output [1]: [d_date_sk#8] @@ -116,63 +116,63 @@ Input [3]: [cs_item_sk#4, sum#13, count#14] Keys [1]: [cs_item_sk#4] Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#5))] Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#5))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4 AS cs_item_sk#4#18] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4] (19) Filter -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4#18] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4] Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) (20) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#4#18] +Right keys [1]: [cs_item_sk#4] Join condition: None (21) Project [codegen id : 4] Output [2]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17] -Input [3]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4#18] +Input [3]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4] (22) BroadcastExchange Input [2]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (23) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] +Output [3]: [cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#6), dynamicpruningexpression(cs_sold_date_sk#6 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#7)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] ReadSchema: struct (24) ColumnarToRow -Input [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] +Input [3]: [cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] (25) Filter -Input [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] -Condition : (isnotnull(cs_item_sk#4) AND isnotnull(cs_ext_discount_amt#5)) +Input [3]: [cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] +Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_ext_discount_amt#20)) (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#4] -Join condition: (cast(cs_ext_discount_amt#5 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) +Right keys [1]: [cs_item_sk#19] +Join condition: (cast(cs_ext_discount_amt#20 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) (27) Project [codegen id : 6] -Output [1]: [cs_sold_date_sk#6] -Input [5]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] +Output [1]: [cs_sold_date_sk#21] +Input [5]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] (28) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 6] -Output [1]: [1 AS excess discount amount #20] -Input [2]: [cs_sold_date_sk#6, d_date_sk#8] +Output [1]: [1 AS excess discount amount #23] +Input [2]: [cs_sold_date_sk#21, d_date_sk#22] (31) CollectLimit -Input [1]: [excess discount amount #20] +Input [1]: [excess discount amount #23] Arguments: 100 ===== Subqueries ===== @@ -184,6 +184,6 @@ ReusedExchange (32) (32) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#6 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt index d885ad3178181..303bdf58604bf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt @@ -18,7 +18,7 @@ CollectLimit InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] Filter [(1.3 * avg(cs_ext_discount_amt))] - HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,sum,count] + HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index 9537689459170..ad918310a918a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -79,100 +79,100 @@ Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] (11) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] +Output [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#4)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] +Input [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] (13) Filter [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) +Input [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] +Condition : isnotnull(cs_item_sk#8) (14) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#9] +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#8, d_date#9] +Input [2]: [d_date_sk#11, d_date#12] (16) Filter [codegen id : 2] -Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) (17) Project [codegen id : 2] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#9] +Output [1]: [d_date_sk#11] +Input [2]: [d_date_sk#11, d_date#12] (18) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] (19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#10] +Right keys [1]: [d_date_sk#11] Join condition: None (20) Project [codegen id : 3] -Output [2]: [cs_item_sk#1, cs_ext_discount_amt#2] -Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#8] +Output [2]: [cs_item_sk#8, cs_ext_discount_amt#9] +Input [4]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10, d_date_sk#11] (21) HashAggregate [codegen id : 3] -Input [2]: [cs_item_sk#1, cs_ext_discount_amt#2] -Keys [1]: [cs_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [2]: [sum#11, count#12] -Results [3]: [cs_item_sk#1, sum#13, count#14] +Input [2]: [cs_item_sk#8, cs_ext_discount_amt#9] +Keys [1]: [cs_item_sk#8] +Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#9))] +Aggregate Attributes [2]: [sum#14, count#15] +Results [3]: [cs_item_sk#8, sum#16, count#17] (22) Exchange -Input [3]: [cs_item_sk#1, sum#13, count#14] -Arguments: hashpartitioning(cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [3]: [cs_item_sk#8, sum#16, count#17] +Arguments: hashpartitioning(cs_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#18] (23) HashAggregate [codegen id : 4] -Input [3]: [cs_item_sk#1, sum#13, count#14] -Keys [1]: [cs_item_sk#1] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#2))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#2))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1 AS cs_item_sk#1#18] +Input [3]: [cs_item_sk#8, sum#16, count#17] +Keys [1]: [cs_item_sk#8] +Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#9))] +Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#9))#19] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#9))#19 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] (24) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] +Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#20) (25) BroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#19] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#21] (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] -Right keys [1]: [cs_item_sk#1#18] -Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) +Right keys [1]: [cs_item_sk#8] +Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#20) (27) Project [codegen id : 6] Output [1]: [cs_sold_date_sk#3] -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] +Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] (28) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 6] -Output [1]: [1 AS excess discount amount #20] -Input [2]: [cs_sold_date_sk#3, d_date_sk#8] +Output [1]: [1 AS excess discount amount #23] +Input [2]: [cs_sold_date_sk#3, d_date_sk#22] (31) CollectLimit -Input [1]: [excess discount amount #20] +Input [1]: [excess discount amount #23] Arguments: 100 ===== Subqueries ===== @@ -182,8 +182,8 @@ ReusedExchange (32) (32) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] -Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index 563281eef37bc..f3dd6d0954046 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -24,7 +24,7 @@ CollectLimit BroadcastExchange #3 WholeStageCodegen (4) Filter [(1.3 * avg(cs_ext_discount_amt))] - HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,sum,count] + HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] InputAdapter Exchange [cs_item_sk] #4 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt index 13d73e61e1443..034c0fe9eb535 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt @@ -40,61 +40,61 @@ Output [2]: [i_manufact#2, i_product_name#3] Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] (5) Scan parquet default.item -Output [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [Or(Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,powder),EqualTo(i_color,khaki)),Or(EqualTo(i_units,Ounce),EqualTo(i_units,Oz))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,brown),EqualTo(i_color,honeydew)),Or(EqualTo(i_units,Bunch),EqualTo(i_units,Ton))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,floral),EqualTo(i_color,deep)),Or(EqualTo(i_units,N/A),EqualTo(i_units,Dozen))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,light),EqualTo(i_color,cornflower)),Or(EqualTo(i_units,Box),EqualTo(i_units,Pound))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large)))))),Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,midnight),EqualTo(i_color,snow)),Or(EqualTo(i_units,Pallet),EqualTo(i_units,Gross))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,cyan),EqualTo(i_color,papaya)),Or(EqualTo(i_units,Cup),EqualTo(i_units,Dram))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,orange),EqualTo(i_color,frosted)),Or(EqualTo(i_units,Each),EqualTo(i_units,Tbl))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,forest),EqualTo(i_color,ghost)),Or(EqualTo(i_units,Lb),EqualTo(i_units,Bundle))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))))))), IsNotNull(i_manufact)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] (7) Filter [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Condition : (((((i_category#4 = Women) AND (((((i_color#6 = powder) OR (i_color#6 = khaki)) AND ((i_units#7 = Ounce) OR (i_units#7 = Oz))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = brown) OR (i_color#6 = honeydew)) AND ((i_units#7 = Bunch) OR (i_units#7 = Ton))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = floral) OR (i_color#6 = deep)) AND ((i_units#7 = N/A) OR (i_units#7 = Dozen))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = light) OR (i_color#6 = cornflower)) AND ((i_units#7 = Box) OR (i_units#7 = Pound))) AND ((i_size#5 = medium) OR (i_size#5 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#6 = midnight) OR (i_color#6 = snow)) AND ((i_units#7 = Pallet) OR (i_units#7 = Gross))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = cyan) OR (i_color#6 = papaya)) AND ((i_units#7 = Cup) OR (i_units#7 = Dram))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = orange) OR (i_color#6 = frosted)) AND ((i_units#7 = Each) OR (i_units#7 = Tbl))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = forest) OR (i_color#6 = ghost)) AND ((i_units#7 = Lb) OR (i_units#7 = Bundle))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))))))) AND isnotnull(i_manufact#2)) +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Condition : (((((i_category#4 = Women) AND (((((i_color#7 = powder) OR (i_color#7 = khaki)) AND ((i_units#8 = Ounce) OR (i_units#8 = Oz))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = brown) OR (i_color#7 = honeydew)) AND ((i_units#8 = Bunch) OR (i_units#8 = Ton))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = floral) OR (i_color#7 = deep)) AND ((i_units#8 = N/A) OR (i_units#8 = Dozen))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = light) OR (i_color#7 = cornflower)) AND ((i_units#8 = Box) OR (i_units#8 = Pound))) AND ((i_size#6 = medium) OR (i_size#6 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#7 = midnight) OR (i_color#7 = snow)) AND ((i_units#8 = Pallet) OR (i_units#8 = Gross))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = cyan) OR (i_color#7 = papaya)) AND ((i_units#8 = Cup) OR (i_units#8 = Dram))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = orange) OR (i_color#7 = frosted)) AND ((i_units#8 = Each) OR (i_units#8 = Tbl))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = forest) OR (i_color#7 = ghost)) AND ((i_units#8 = Lb) OR (i_units#8 = Bundle))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))))))) AND isnotnull(i_manufact#5)) (8) Project [codegen id : 1] -Output [1]: [i_manufact#2] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Output [1]: [i_manufact#5] +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] (9) HashAggregate [codegen id : 1] -Input [1]: [i_manufact#2] -Keys [1]: [i_manufact#2] +Input [1]: [i_manufact#5] +Keys [1]: [i_manufact#5] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#8] -Results [2]: [i_manufact#2, count#9] +Aggregate Attributes [1]: [count#9] +Results [2]: [i_manufact#5, count#10] (10) Exchange -Input [2]: [i_manufact#2, count#9] -Arguments: hashpartitioning(i_manufact#2, 5), true, [id=#10] +Input [2]: [i_manufact#5, count#10] +Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [id=#11] (11) HashAggregate [codegen id : 2] -Input [2]: [i_manufact#2, count#9] -Keys [1]: [i_manufact#2] +Input [2]: [i_manufact#5, count#10] +Keys [1]: [i_manufact#5] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#11] -Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13] +Aggregate Attributes [1]: [count(1)#12] +Results [2]: [count(1)#12 AS item_cnt#13, i_manufact#5] (12) Filter [codegen id : 2] -Input [2]: [item_cnt#12, i_manufact#2#13] -Condition : (item_cnt#12 > 0) +Input [2]: [item_cnt#13, i_manufact#5] +Condition : (item_cnt#13 > 0) (13) Project [codegen id : 2] -Output [1]: [i_manufact#2#13] -Input [2]: [item_cnt#12, i_manufact#2#13] +Output [1]: [i_manufact#5] +Input [2]: [item_cnt#13, i_manufact#5] (14) BroadcastExchange -Input [1]: [i_manufact#2#13] +Input [1]: [i_manufact#5] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14] (15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_manufact#2] -Right keys [1]: [i_manufact#2#13] +Right keys [1]: [i_manufact#5] Join condition: None (16) Project [codegen id : 3] Output [1]: [i_product_name#3] -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#2#13] +Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] (17) HashAggregate [codegen id : 3] Input [1]: [i_product_name#3] @@ -105,7 +105,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] +Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt index 2d14d75ca9362..d36800823bb3f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (2) Project [i_manufact] Filter [item_cnt] - HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,count] + HashAggregate [i_manufact,count] [count(1),item_cnt,count] InputAdapter Exchange [i_manufact] #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index 13d73e61e1443..034c0fe9eb535 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -40,61 +40,61 @@ Output [2]: [i_manufact#2, i_product_name#3] Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] (5) Scan parquet default.item -Output [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [Or(Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,powder),EqualTo(i_color,khaki)),Or(EqualTo(i_units,Ounce),EqualTo(i_units,Oz))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,brown),EqualTo(i_color,honeydew)),Or(EqualTo(i_units,Bunch),EqualTo(i_units,Ton))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,floral),EqualTo(i_color,deep)),Or(EqualTo(i_units,N/A),EqualTo(i_units,Dozen))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,light),EqualTo(i_color,cornflower)),Or(EqualTo(i_units,Box),EqualTo(i_units,Pound))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large)))))),Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,midnight),EqualTo(i_color,snow)),Or(EqualTo(i_units,Pallet),EqualTo(i_units,Gross))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,cyan),EqualTo(i_color,papaya)),Or(EqualTo(i_units,Cup),EqualTo(i_units,Dram))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,orange),EqualTo(i_color,frosted)),Or(EqualTo(i_units,Each),EqualTo(i_units,Tbl))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,forest),EqualTo(i_color,ghost)),Or(EqualTo(i_units,Lb),EqualTo(i_units,Bundle))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))))))), IsNotNull(i_manufact)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] (7) Filter [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Condition : (((((i_category#4 = Women) AND (((((i_color#6 = powder) OR (i_color#6 = khaki)) AND ((i_units#7 = Ounce) OR (i_units#7 = Oz))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = brown) OR (i_color#6 = honeydew)) AND ((i_units#7 = Bunch) OR (i_units#7 = Ton))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = floral) OR (i_color#6 = deep)) AND ((i_units#7 = N/A) OR (i_units#7 = Dozen))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = light) OR (i_color#6 = cornflower)) AND ((i_units#7 = Box) OR (i_units#7 = Pound))) AND ((i_size#5 = medium) OR (i_size#5 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#6 = midnight) OR (i_color#6 = snow)) AND ((i_units#7 = Pallet) OR (i_units#7 = Gross))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = cyan) OR (i_color#6 = papaya)) AND ((i_units#7 = Cup) OR (i_units#7 = Dram))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = orange) OR (i_color#6 = frosted)) AND ((i_units#7 = Each) OR (i_units#7 = Tbl))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = forest) OR (i_color#6 = ghost)) AND ((i_units#7 = Lb) OR (i_units#7 = Bundle))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))))))) AND isnotnull(i_manufact#2)) +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Condition : (((((i_category#4 = Women) AND (((((i_color#7 = powder) OR (i_color#7 = khaki)) AND ((i_units#8 = Ounce) OR (i_units#8 = Oz))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = brown) OR (i_color#7 = honeydew)) AND ((i_units#8 = Bunch) OR (i_units#8 = Ton))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = floral) OR (i_color#7 = deep)) AND ((i_units#8 = N/A) OR (i_units#8 = Dozen))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = light) OR (i_color#7 = cornflower)) AND ((i_units#8 = Box) OR (i_units#8 = Pound))) AND ((i_size#6 = medium) OR (i_size#6 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#7 = midnight) OR (i_color#7 = snow)) AND ((i_units#8 = Pallet) OR (i_units#8 = Gross))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = cyan) OR (i_color#7 = papaya)) AND ((i_units#8 = Cup) OR (i_units#8 = Dram))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = orange) OR (i_color#7 = frosted)) AND ((i_units#8 = Each) OR (i_units#8 = Tbl))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = forest) OR (i_color#7 = ghost)) AND ((i_units#8 = Lb) OR (i_units#8 = Bundle))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))))))) AND isnotnull(i_manufact#5)) (8) Project [codegen id : 1] -Output [1]: [i_manufact#2] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Output [1]: [i_manufact#5] +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] (9) HashAggregate [codegen id : 1] -Input [1]: [i_manufact#2] -Keys [1]: [i_manufact#2] +Input [1]: [i_manufact#5] +Keys [1]: [i_manufact#5] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#8] -Results [2]: [i_manufact#2, count#9] +Aggregate Attributes [1]: [count#9] +Results [2]: [i_manufact#5, count#10] (10) Exchange -Input [2]: [i_manufact#2, count#9] -Arguments: hashpartitioning(i_manufact#2, 5), true, [id=#10] +Input [2]: [i_manufact#5, count#10] +Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [id=#11] (11) HashAggregate [codegen id : 2] -Input [2]: [i_manufact#2, count#9] -Keys [1]: [i_manufact#2] +Input [2]: [i_manufact#5, count#10] +Keys [1]: [i_manufact#5] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#11] -Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13] +Aggregate Attributes [1]: [count(1)#12] +Results [2]: [count(1)#12 AS item_cnt#13, i_manufact#5] (12) Filter [codegen id : 2] -Input [2]: [item_cnt#12, i_manufact#2#13] -Condition : (item_cnt#12 > 0) +Input [2]: [item_cnt#13, i_manufact#5] +Condition : (item_cnt#13 > 0) (13) Project [codegen id : 2] -Output [1]: [i_manufact#2#13] -Input [2]: [item_cnt#12, i_manufact#2#13] +Output [1]: [i_manufact#5] +Input [2]: [item_cnt#13, i_manufact#5] (14) BroadcastExchange -Input [1]: [i_manufact#2#13] +Input [1]: [i_manufact#5] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14] (15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_manufact#2] -Right keys [1]: [i_manufact#2#13] +Right keys [1]: [i_manufact#5] Join condition: None (16) Project [codegen id : 3] Output [1]: [i_product_name#3] -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#2#13] +Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] (17) HashAggregate [codegen id : 3] Input [1]: [i_product_name#3] @@ -105,7 +105,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] +Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index 2d14d75ca9362..d36800823bb3f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (2) Project [i_manufact] Filter [item_cnt] - HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,count] + HashAggregate [i_manufact,count] [count(1),item_cnt,count] InputAdapter Exchange [i_manufact] #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt index 4c5b5f25108e0..7e82b4d5df296 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt @@ -81,57 +81,57 @@ Input [3]: [i_item_sk#5, i_current_price#6, i_category#7] Condition : (isnotnull(i_current_price#6) AND isnotnull(i_item_sk#5)) (7) Scan parquet default.item -Output [2]: [i_current_price#6, i_category#7] +Output [2]: [i_current_price#8, i_category#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] +Input [2]: [i_current_price#8, i_category#9] (9) Filter [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] -Condition : isnotnull(i_category#7) +Input [2]: [i_current_price#8, i_category#9] +Condition : isnotnull(i_category#9) (10) HashAggregate [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] -Keys [1]: [i_category#7] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#6))] -Aggregate Attributes [2]: [sum#8, count#9] -Results [3]: [i_category#7, sum#10, count#11] +Input [2]: [i_current_price#8, i_category#9] +Keys [1]: [i_category#9] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#8))] +Aggregate Attributes [2]: [sum#10, count#11] +Results [3]: [i_category#9, sum#12, count#13] (11) Exchange -Input [3]: [i_category#7, sum#10, count#11] -Arguments: hashpartitioning(i_category#7, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [3]: [i_category#9, sum#12, count#13] +Arguments: hashpartitioning(i_category#9, 5), ENSURE_REQUIREMENTS, [id=#14] (12) HashAggregate [codegen id : 2] -Input [3]: [i_category#7, sum#10, count#11] -Keys [1]: [i_category#7] -Functions [1]: [avg(UnscaledValue(i_current_price#6))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#6))#13] -Results [2]: [cast((avg(UnscaledValue(i_current_price#6))#13 / 100.0) as decimal(11,6)) AS avg(i_current_price)#14, i_category#7 AS i_category#7#15] +Input [3]: [i_category#9, sum#12, count#13] +Keys [1]: [i_category#9] +Functions [1]: [avg(UnscaledValue(i_current_price#8))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#8))#15] +Results [2]: [cast((avg(UnscaledValue(i_current_price#8))#15 / 100.0) as decimal(11,6)) AS avg(i_current_price)#16, i_category#9] (13) BroadcastExchange -Input [2]: [avg(i_current_price)#14, i_category#7#15] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#16] +Input [2]: [avg(i_current_price)#16, i_category#9] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#17] (14) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_category#7] -Right keys [1]: [i_category#7#15] +Right keys [1]: [i_category#9] Join condition: None (15) Filter [codegen id : 3] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] -Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#14)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] +Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#16)), DecimalType(14,7), true)) (16) Project [codegen id : 3] Output [1]: [i_item_sk#5] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] (17) BroadcastExchange Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (18) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#1] @@ -143,143 +143,143 @@ Output [2]: [ss_customer_sk#2, ss_sold_date_sk#3] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, i_item_sk#5] (20) Scan parquet default.date_dim -Output [2]: [d_date_sk#18, d_month_seq#19] +Output [2]: [d_date_sk#19, d_month_seq#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#18, d_month_seq#19] +Input [2]: [d_date_sk#19, d_month_seq#20] (22) Filter [codegen id : 4] -Input [2]: [d_date_sk#18, d_month_seq#19] -Condition : ((isnotnull(d_month_seq#19) AND (d_month_seq#19 = Subquery scalar-subquery#20, [id=#21])) AND isnotnull(d_date_sk#18)) +Input [2]: [d_date_sk#19, d_month_seq#20] +Condition : ((isnotnull(d_month_seq#20) AND (d_month_seq#20 = Subquery scalar-subquery#21, [id=#22])) AND isnotnull(d_date_sk#19)) (23) Project [codegen id : 4] -Output [1]: [d_date_sk#18] -Input [2]: [d_date_sk#18, d_month_seq#19] +Output [1]: [d_date_sk#19] +Input [2]: [d_date_sk#19, d_month_seq#20] (24) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (25) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (26) Project [codegen id : 5] Output [1]: [ss_customer_sk#2] -Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#18] +Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#19] (27) Exchange Input [1]: [ss_customer_sk#2] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#23] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] (28) Sort [codegen id : 6] Input [1]: [ss_customer_sk#2] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#25, ca_state#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#24, ca_state#25] +Input [2]: [ca_address_sk#25, ca_state#26] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : isnotnull(ca_address_sk#24) +Input [2]: [ca_address_sk#25, ca_state#26] +Condition : isnotnull(ca_address_sk#25) (32) Exchange -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [ca_address_sk#25, ca_state#26] +Arguments: hashpartitioning(ca_address_sk#25, 5), ENSURE_REQUIREMENTS, [id=#27] (33) Sort [codegen id : 8] -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#25, ca_state#26] +Arguments: [ca_address_sk#25 ASC NULLS FIRST], false, 0 (34) Scan parquet default.customer -Output [2]: [c_customer_sk#27, c_current_addr_sk#28] +Output [2]: [c_customer_sk#28, c_current_addr_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 9] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] (36) Filter [codegen id : 9] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Condition : (isnotnull(c_current_addr_sk#28) AND isnotnull(c_customer_sk#27)) +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Condition : (isnotnull(c_current_addr_sk#29) AND isnotnull(c_customer_sk#28)) (37) Exchange -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Arguments: hashpartitioning(c_current_addr_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Arguments: hashpartitioning(c_current_addr_sk#29, 5), ENSURE_REQUIREMENTS, [id=#30] (38) Sort [codegen id : 10] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Arguments: [c_current_addr_sk#28 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Arguments: [c_current_addr_sk#29 ASC NULLS FIRST], false, 0 (39) SortMergeJoin [codegen id : 11] -Left keys [1]: [ca_address_sk#24] -Right keys [1]: [c_current_addr_sk#28] +Left keys [1]: [ca_address_sk#25] +Right keys [1]: [c_current_addr_sk#29] Join condition: None (40) Project [codegen id : 11] -Output [2]: [ca_state#25, c_customer_sk#27] -Input [4]: [ca_address_sk#24, ca_state#25, c_customer_sk#27, c_current_addr_sk#28] +Output [2]: [ca_state#26, c_customer_sk#28] +Input [4]: [ca_address_sk#25, ca_state#26, c_customer_sk#28, c_current_addr_sk#29] (41) Exchange -Input [2]: [ca_state#25, c_customer_sk#27] -Arguments: hashpartitioning(c_customer_sk#27, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [ca_state#26, c_customer_sk#28] +Arguments: hashpartitioning(c_customer_sk#28, 5), ENSURE_REQUIREMENTS, [id=#31] (42) Sort [codegen id : 12] -Input [2]: [ca_state#25, c_customer_sk#27] -Arguments: [c_customer_sk#27 ASC NULLS FIRST], false, 0 +Input [2]: [ca_state#26, c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 13] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#27] +Right keys [1]: [c_customer_sk#28] Join condition: None (44) Project [codegen id : 13] -Output [1]: [ca_state#25] -Input [3]: [ss_customer_sk#2, ca_state#25, c_customer_sk#27] +Output [1]: [ca_state#26] +Input [3]: [ss_customer_sk#2, ca_state#26, c_customer_sk#28] (45) HashAggregate [codegen id : 13] -Input [1]: [ca_state#25] -Keys [1]: [ca_state#25] +Input [1]: [ca_state#26] +Keys [1]: [ca_state#26] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#31] -Results [2]: [ca_state#25, count#32] +Aggregate Attributes [1]: [count#32] +Results [2]: [ca_state#26, count#33] (46) Exchange -Input [2]: [ca_state#25, count#32] -Arguments: hashpartitioning(ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [2]: [ca_state#26, count#33] +Arguments: hashpartitioning(ca_state#26, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 14] -Input [2]: [ca_state#25, count#32] -Keys [1]: [ca_state#25] +Input [2]: [ca_state#26, count#33] +Keys [1]: [ca_state#26] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#34] -Results [3]: [ca_state#25 AS state#35, count(1)#34 AS cnt#36, count(1)#34 AS count(1)#37] +Aggregate Attributes [1]: [count(1)#35] +Results [3]: [ca_state#26 AS state#36, count(1)#35 AS cnt#37, count(1)#35 AS count(1)#38] (48) Filter [codegen id : 14] -Input [3]: [state#35, cnt#36, count(1)#37] -Condition : (count(1)#37 >= 10) +Input [3]: [state#36, cnt#37, count(1)#38] +Condition : (count(1)#38 >= 10) (49) Project [codegen id : 14] -Output [2]: [state#35, cnt#36] -Input [3]: [state#35, cnt#36, count(1)#37] +Output [2]: [state#36, cnt#37] +Input [3]: [state#36, cnt#37, count(1)#38] (50) TakeOrderedAndProject -Input [2]: [state#35, cnt#36] -Arguments: 100, [cnt#36 ASC NULLS FIRST], [state#35, cnt#36] +Input [2]: [state#36, cnt#37] +Arguments: 100, [cnt#37 ASC NULLS FIRST], [state#36, cnt#37] ===== Subqueries ===== @@ -288,9 +288,9 @@ ReusedExchange (51) (51) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#19] -Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#20, [id=#21] +Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#21, [id=#22] * HashAggregate (58) +- Exchange (57) +- * HashAggregate (56) @@ -301,39 +301,39 @@ Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquer (52) Scan parquet default.date_dim -Output [3]: [d_month_seq#19, d_year#38, d_moy#39] +Output [3]: [d_month_seq#39, d_year#40, d_moy#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (53) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] (54) Filter [codegen id : 1] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] -Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] +Condition : (((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2000)) AND (d_moy#41 = 1)) (55) Project [codegen id : 1] -Output [1]: [d_month_seq#19] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] +Output [1]: [d_month_seq#39] +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] (56) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#19] -Keys [1]: [d_month_seq#19] +Input [1]: [d_month_seq#39] +Keys [1]: [d_month_seq#39] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#19] +Results [1]: [d_month_seq#39] (57) Exchange -Input [1]: [d_month_seq#19] -Arguments: hashpartitioning(d_month_seq#19, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [1]: [d_month_seq#39] +Arguments: hashpartitioning(d_month_seq#39, 5), ENSURE_REQUIREMENTS, [id=#42] (58) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#19] -Keys [1]: [d_month_seq#19] +Input [1]: [d_month_seq#39] +Keys [1]: [d_month_seq#39] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#19] +Results [1]: [d_month_seq#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt index a5097742ec146..1cdb7a5df8e6f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt @@ -38,7 +38,7 @@ TakeOrderedAndProject [cnt,state] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] InputAdapter Exchange [i_category] #6 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt index 58fabaa8e44be..0f98039fc0f7f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt @@ -160,57 +160,57 @@ Input [3]: [i_item_sk#16, i_current_price#17, i_category#18] Condition : (isnotnull(i_current_price#17) AND isnotnull(i_item_sk#16)) (26) Scan parquet default.item -Output [2]: [i_current_price#17, i_category#18] +Output [2]: [i_current_price#19, i_category#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (27) ColumnarToRow [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] +Input [2]: [i_current_price#19, i_category#20] (28) Filter [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] -Condition : isnotnull(i_category#18) +Input [2]: [i_current_price#19, i_category#20] +Condition : isnotnull(i_category#20) (29) HashAggregate [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] -Keys [1]: [i_category#18] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] -Aggregate Attributes [2]: [sum#19, count#20] -Results [3]: [i_category#18, sum#21, count#22] +Input [2]: [i_current_price#19, i_category#20] +Keys [1]: [i_category#20] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#19))] +Aggregate Attributes [2]: [sum#21, count#22] +Results [3]: [i_category#20, sum#23, count#24] (30) Exchange -Input [3]: [i_category#18, sum#21, count#22] -Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [i_category#20, sum#23, count#24] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#25] (31) HashAggregate [codegen id : 5] -Input [3]: [i_category#18, sum#21, count#22] -Keys [1]: [i_category#18] -Functions [1]: [avg(UnscaledValue(i_current_price#17))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#17))#24] -Results [2]: [cast((avg(UnscaledValue(i_current_price#17))#24 / 100.0) as decimal(11,6)) AS avg(i_current_price)#25, i_category#18 AS i_category#18#26] +Input [3]: [i_category#20, sum#23, count#24] +Keys [1]: [i_category#20] +Functions [1]: [avg(UnscaledValue(i_current_price#19))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#19))#26] +Results [2]: [cast((avg(UnscaledValue(i_current_price#19))#26 / 100.0) as decimal(11,6)) AS avg(i_current_price)#27, i_category#20] (32) BroadcastExchange -Input [2]: [avg(i_current_price)#25, i_category#18#26] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#27] +Input [2]: [avg(i_current_price)#27, i_category#20] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#28] (33) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_category#18] -Right keys [1]: [i_category#18#26] +Right keys [1]: [i_category#20] Join condition: None (34) Filter [codegen id : 6] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] -Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#25)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] +Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#27)), DecimalType(14,7), true)) (35) Project [codegen id : 6] Output [1]: [i_item_sk#16] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] (36) BroadcastExchange Input [1]: [i_item_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (37) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#6] @@ -225,31 +225,31 @@ Input [3]: [ca_state#2, ss_item_sk#6, i_item_sk#16] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [2]: [ca_state#2, count#30] +Aggregate Attributes [1]: [count#30] +Results [2]: [ca_state#2, count#31] (40) Exchange -Input [2]: [ca_state#2, count#30] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [2]: [ca_state#2, count#31] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#32] (41) HashAggregate [codegen id : 8] -Input [2]: [ca_state#2, count#30] +Input [2]: [ca_state#2, count#31] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [3]: [ca_state#2 AS state#33, count(1)#32 AS cnt#34, count(1)#32 AS count(1)#35] +Aggregate Attributes [1]: [count(1)#33] +Results [3]: [ca_state#2 AS state#34, count(1)#33 AS cnt#35, count(1)#33 AS count(1)#36] (42) Filter [codegen id : 8] -Input [3]: [state#33, cnt#34, count(1)#35] -Condition : (count(1)#35 >= 10) +Input [3]: [state#34, cnt#35, count(1)#36] +Condition : (count(1)#36 >= 10) (43) Project [codegen id : 8] -Output [2]: [state#33, cnt#34] -Input [3]: [state#33, cnt#34, count(1)#35] +Output [2]: [state#34, cnt#35] +Input [3]: [state#34, cnt#35, count(1)#36] (44) TakeOrderedAndProject -Input [2]: [state#33, cnt#34] -Arguments: 100, [cnt#34 ASC NULLS FIRST], [state#33, cnt#34] +Input [2]: [state#34, cnt#35] +Arguments: 100, [cnt#35 ASC NULLS FIRST], [state#34, cnt#35] ===== Subqueries ===== @@ -271,39 +271,39 @@ Subquery:2 Hosting operator id = 18 Hosting Expression = Subquery scalar-subquer (46) Scan parquet default.date_dim -Output [3]: [d_month_seq#12, d_year#36, d_moy#37] +Output [3]: [d_month_seq#37, d_year#38, d_moy#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (47) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] (48) Filter [codegen id : 1] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] -Condition : (((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2000)) AND (d_moy#37 = 1)) +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] +Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) (49) Project [codegen id : 1] -Output [1]: [d_month_seq#12] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] +Output [1]: [d_month_seq#37] +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] (50) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#12] -Keys [1]: [d_month_seq#12] +Input [1]: [d_month_seq#37] +Keys [1]: [d_month_seq#37] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#12] +Results [1]: [d_month_seq#37] (51) Exchange -Input [1]: [d_month_seq#12] -Arguments: hashpartitioning(d_month_seq#12, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [1]: [d_month_seq#37] +Arguments: hashpartitioning(d_month_seq#37, 5), ENSURE_REQUIREMENTS, [id=#40] (52) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#12] -Keys [1]: [d_month_seq#12] +Input [1]: [d_month_seq#37] +Keys [1]: [d_month_seq#37] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#12] +Results [1]: [d_month_seq#37] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index 8b60c2f86a280..d7fad5948f64b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -68,7 +68,7 @@ TakeOrderedAndProject [cnt,state] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] InputAdapter Exchange [i_category] #8 WholeStageCodegen (4) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt index 8297114e1e9ab..94e76ced527a0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt @@ -84,7 +84,7 @@ Input [2]: [d_date_sk#8, d_date#9] (11) Filter [codegen id : 2] Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) (12) Project [codegen id : 2] Output [1]: [d_date_sk#8] @@ -119,82 +119,82 @@ Input [3]: [ws_item_sk#4, sum#13, count#14] Keys [1]: [ws_item_sk#4] Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#5))] Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#5))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4 AS ws_item_sk#4#18] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4] (19) Filter -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4#18] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4] Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) (20) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#1] -Right keys [1]: [ws_item_sk#4#18] +Right keys [1]: [ws_item_sk#4] Join condition: None (21) Project [codegen id : 4] Output [2]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17] -Input [3]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4#18] +Input [3]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4] (22) BroadcastExchange Input [2]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (23) Scan parquet default.web_sales -Output [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] +Output [3]: [ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_ext_discount_amt)] ReadSchema: struct (24) ColumnarToRow -Input [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] +Input [3]: [ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] (25) Filter -Input [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] -Condition : (isnotnull(ws_item_sk#4) AND isnotnull(ws_ext_discount_amt#5)) +Input [3]: [ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] +Condition : (isnotnull(ws_item_sk#19) AND isnotnull(ws_ext_discount_amt#20)) (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#1] -Right keys [1]: [ws_item_sk#4] -Join condition: (cast(ws_ext_discount_amt#5 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) +Right keys [1]: [ws_item_sk#19] +Join condition: (cast(ws_ext_discount_amt#20 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) (27) Project [codegen id : 6] -Output [2]: [ws_ext_discount_amt#5, ws_sold_date_sk#6] -Input [5]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] +Output [2]: [ws_ext_discount_amt#20, ws_sold_date_sk#21] +Input [5]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] (28) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ws_sold_date_sk#21] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 6] -Output [1]: [ws_ext_discount_amt#5] -Input [3]: [ws_ext_discount_amt#5, ws_sold_date_sk#6, d_date_sk#8] +Output [1]: [ws_ext_discount_amt#20] +Input [3]: [ws_ext_discount_amt#20, ws_sold_date_sk#21, d_date_sk#22] (31) HashAggregate [codegen id : 6] -Input [1]: [ws_ext_discount_amt#5] +Input [1]: [ws_ext_discount_amt#20] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#5))] -Aggregate Attributes [1]: [sum#20] -Results [1]: [sum#21] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#20))] +Aggregate Attributes [1]: [sum#23] +Results [1]: [sum#24] (32) Exchange -Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [sum#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#25] (33) HashAggregate [codegen id : 7] -Input [1]: [sum#21] +Input [1]: [sum#24] Keys: [] -Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#5))#23] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#5))#23,17,2) AS Excess Discount Amount #24] +Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#20))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#20))#26] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#20))#26,17,2) AS Excess Discount Amount #27] (34) Sort [codegen id : 7] -Input [1]: [Excess Discount Amount #24] -Arguments: [Excess Discount Amount #24 ASC NULLS FIRST], true, 0 +Input [1]: [Excess Discount Amount #27] +Arguments: [Excess Discount Amount #27 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -205,6 +205,6 @@ ReusedExchange (35) (35) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt index cc3ffa0de4bfd..e83a3e67e5c6f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt @@ -23,7 +23,7 @@ WholeStageCodegen (7) InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] Filter [(1.3 * avg(ws_ext_discount_amt))] - HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,sum,count] + HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] InputAdapter Exchange [ws_item_sk] #4 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index faf82026138f6..46953fe184f28 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -82,119 +82,119 @@ Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] (11) Scan parquet default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] +Output [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] +Input [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] (13) Filter [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) +Input [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] +Condition : isnotnull(ws_item_sk#8) (14) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#9] +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#8, d_date#9] +Input [2]: [d_date_sk#11, d_date#12] (16) Filter [codegen id : 2] -Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) (17) Project [codegen id : 2] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#9] +Output [1]: [d_date_sk#11] +Input [2]: [d_date_sk#11, d_date#12] (18) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] (19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ws_sold_date_sk#10] +Right keys [1]: [d_date_sk#11] Join condition: None (20) Project [codegen id : 3] -Output [2]: [ws_item_sk#1, ws_ext_discount_amt#2] -Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#8] +Output [2]: [ws_item_sk#8, ws_ext_discount_amt#9] +Input [4]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10, d_date_sk#11] (21) HashAggregate [codegen id : 3] -Input [2]: [ws_item_sk#1, ws_ext_discount_amt#2] -Keys [1]: [ws_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [2]: [sum#11, count#12] -Results [3]: [ws_item_sk#1, sum#13, count#14] +Input [2]: [ws_item_sk#8, ws_ext_discount_amt#9] +Keys [1]: [ws_item_sk#8] +Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#9))] +Aggregate Attributes [2]: [sum#14, count#15] +Results [3]: [ws_item_sk#8, sum#16, count#17] (22) Exchange -Input [3]: [ws_item_sk#1, sum#13, count#14] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [3]: [ws_item_sk#8, sum#16, count#17] +Arguments: hashpartitioning(ws_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#18] (23) HashAggregate [codegen id : 4] -Input [3]: [ws_item_sk#1, sum#13, count#14] -Keys [1]: [ws_item_sk#1] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#2))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#2))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1 AS ws_item_sk#1#18] +Input [3]: [ws_item_sk#8, sum#16, count#17] +Keys [1]: [ws_item_sk#8] +Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#9))] +Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#9))#19] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#9))#19 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] (24) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] +Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#20) (25) BroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#19] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#21] (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] -Right keys [1]: [ws_item_sk#1#18] -Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) +Right keys [1]: [ws_item_sk#8] +Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#20) (27) Project [codegen id : 6] Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] +Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] (28) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 6] Output [1]: [ws_ext_discount_amt#2] -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#8] +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#22] (31) HashAggregate [codegen id : 6] Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#20] -Results [1]: [sum#21] +Aggregate Attributes [1]: [sum#23] +Results [1]: [sum#24] (32) Exchange -Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [sum#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#25] (33) HashAggregate [codegen id : 7] -Input [1]: [sum#21] +Input [1]: [sum#24] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#23] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#23,17,2) AS Excess Discount Amount #24] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#26] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#26,17,2) AS Excess Discount Amount #27] (34) Sort [codegen id : 7] -Input [1]: [Excess Discount Amount #24] -Arguments: [Excess Discount Amount #24 ASC NULLS FIRST], true, 0 +Input [1]: [Excess Discount Amount #27] +Arguments: [Excess Discount Amount #27 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -203,8 +203,8 @@ ReusedExchange (35) (35) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index 3108c3bc82c7c..0df713c0e075e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -29,7 +29,7 @@ WholeStageCodegen (7) BroadcastExchange #4 WholeStageCodegen (4) Filter [(1.3 * avg(ws_ext_discount_amt))] - HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,sum,count] + HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] InputAdapter Exchange [ws_item_sk] #5 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt index 312aba94bd9c6..25baf22f2b4b5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt @@ -33,39 +33,39 @@ TakeOrderedAndProject (116) : : : : +- * HashAggregate (44) : : : : +- Exchange (43) : : : : +- * HashAggregate (42) - : : : : +- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) + : : : : +- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Project (18) + : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : :- * Filter (11) + : : : : : : +- * ColumnarToRow (10) + : : : : : : +- Scan parquet default.store_sales (9) + : : : : : +- BroadcastExchange (16) + : : : : : +- * Project (15) + : : : : : +- * Filter (14) + : : : : : +- * ColumnarToRow (13) + : : : : : +- Scan parquet default.date_dim (12) + : : : : +- BroadcastExchange (39) + : : : : +- SortMergeJoin LeftSemi (38) + : : : : :- * Sort (23) + : : : : : +- Exchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (37) + : : : : +- Exchange (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- Scan parquet default.catalog_sales (24) + : : : : : +- ReusedExchange (27) + : : : : +- BroadcastExchange (33) + : : : : +- * Filter (32) + : : : : +- * ColumnarToRow (31) + : : : : +- Scan parquet default.item (30) : : : +- * Sort (57) : : : +- Exchange (56) : : : +- * Project (55) @@ -147,507 +147,507 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 20] +(7) ColumnarToRow [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 20] +(8) Filter [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 5] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +(10) ColumnarToRow [codegen id : 10] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -(11) Filter [codegen id : 5] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +(11) Filter [codegen id : 10] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_item_sk#11) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#14, d_year#15] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#14, d_year#15] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1998)) AND (d_year#15 <= 2000)) AND isnotnull(d_date_sk#14)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#14] +Input [2]: [d_date_sk#14, d_year#15] (16) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(17) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(17) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#14] Join condition: None -(18) Project [codegen id : 5] -Output [1]: [ss_item_sk#1] -Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] +(18) Project [codegen id : 10] +Output [1]: [ss_item_sk#11] +Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] (19) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) - -(22) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] - -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join condition: None - -(24) Project [codegen id : 5] -Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] -Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) -(25) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] +(22) Exchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] -(26) Sort [codegen id : 6] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 +(23) Sort [codegen id : 5] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 -(27) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] +(24) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +(25) ColumnarToRow [codegen id : 8] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] -(29) Filter [codegen id : 9] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +(26) Filter [codegen id : 8] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#22) -(30) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +(27) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#24] -(31) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +(28) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None -(32) Project [codegen id : 9] -Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] +(29) Project [codegen id : 8] +Output [1]: [cs_item_sk#22] +Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] -(33) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(30) Scan parquet default.item +Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(31) ColumnarToRow [codegen id : 7] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -(35) Filter [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +(32) Filter [codegen id : 7] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Condition : isnotnull(i_item_sk#25) -(36) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +(33) BroadcastExchange +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_item_sk#22] +Right keys [1]: [i_item_sk#25] Join condition: None -(38) Project [codegen id : 9] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(35) Project [codegen id : 8] +Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] + +(36) Exchange +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] -(39) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] +(37) Sort [codegen id : 9] +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 -(40) Sort [codegen id : 10] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(38) SortMergeJoin +Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] +Join condition: None + +(39) BroadcastExchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] -(41) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +(40) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_item_sk#11] +Right keys [1]: [i_item_sk#17] Join condition: None -(42) HashAggregate [codegen id : 11] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(41) Project [codegen id : 10] +Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] +Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] + +(42) HashAggregate [codegen id : 10] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (43) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(44) HashAggregate [codegen id : 12] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(44) HashAggregate [codegen id : 11] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (45) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] -(46) Sort [codegen id : 13] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 12] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +(48) ColumnarToRow [codegen id : 15] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -(49) Filter [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(49) Filter [codegen id : 15] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_item_sk#37) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#39] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(51) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#39] Join condition: None -(52) Project [codegen id : 16] -Output [1]: [ws_item_sk#26] -Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] +(52) Project [codegen id : 15] +Output [1]: [ws_item_sk#37] +Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] -(53) ReusedExchange [Reuses operator id: 36] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(53) ReusedExchange [Reuses operator id: 33] +Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(54) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#40] Join condition: None -(55) Project [codegen id : 16] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(55) Project [codegen id : 15] +Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] (56) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] -(57) Sort [codegen id : 17] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 16] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] +Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] Join condition: None -(59) HashAggregate [codegen id : 18] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(59) HashAggregate [codegen id : 17] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (60) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] -(61) HashAggregate [codegen id : 19] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(61) HashAggregate [codegen id : 18] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (62) BroadcastExchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] -(63) BroadcastHashJoin [codegen id : 20] +(63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#16, class_id#17, category_id#18] +Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join condition: None -(64) Project [codegen id : 20] -Output [1]: [i_item_sk#7 AS ss_item_sk#31] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] +(64) Project [codegen id : 19] +Output [1]: [i_item_sk#7 AS ss_item_sk#47] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] (65) Exchange -Input [1]: [ss_item_sk#31] -Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [1]: [ss_item_sk#47] +Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] -(66) Sort [codegen id : 21] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 20] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#47] Join condition: None (68) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#33] +Output [2]: [d_date_sk#49, d_week_seq#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 22] -Input [2]: [d_date_sk#12, d_week_seq#33] +(69) ColumnarToRow [codegen id : 21] +Input [2]: [d_date_sk#49, d_week_seq#50] -(70) Filter [codegen id : 22] -Input [2]: [d_date_sk#12, d_week_seq#33] -Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#34, [id=#35])) AND isnotnull(d_date_sk#12)) +(70) Filter [codegen id : 21] +Input [2]: [d_date_sk#49, d_week_seq#50] +Condition : ((isnotnull(d_week_seq#50) AND (d_week_seq#50 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#49)) -(71) Project [codegen id : 22] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#33] +(71) Project [codegen id : 21] +Output [1]: [d_date_sk#49] +Input [2]: [d_date_sk#49, d_week_seq#50] (72) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [1]: [d_date_sk#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] -(73) BroadcastHashJoin [codegen id : 44] +(73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#49] Join condition: None -(74) Project [codegen id : 44] +(74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] (75) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(76) ColumnarToRow [codegen id : 22] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -(77) Filter [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +(77) Filter [codegen id : 22] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Condition : (((isnotnull(i_item_sk#54) AND isnotnull(i_brand_id#55)) AND isnotnull(i_class_id#56)) AND isnotnull(i_category_id#57)) (78) Exchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: hashpartitioning(i_item_sk#54, 5), ENSURE_REQUIREMENTS, [id=#58] -(79) Sort [codegen id : 24] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 +(79) Sort [codegen id : 23] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: [i_item_sk#54 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(81) Sort [codegen id : 43] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 41] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [i_item_sk#54] +Right keys [1]: [ss_item_sk#47] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#59] -(84) BroadcastHashJoin [codegen id : 44] +(84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#54] Join condition: None -(85) Project [codegen id : 44] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(85) Project [codegen id : 42] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -(86) HashAggregate [codegen id : 44] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +(86) HashAggregate [codegen id : 42] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] +Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#39, isEmpty#40, count#41] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] +Aggregate Attributes [3]: [sum#60, isEmpty#61, count#62] +Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] +Arguments: hashpartitioning(i_brand_id#55, i_class_id#56, i_category_id#57, 5), ENSURE_REQUIREMENTS, [id=#66] -(88) HashAggregate [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +(88) HashAggregate [codegen id : 86] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] +Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46, count(1)#47] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sales#48, count(1)#47 AS number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67, count(1)#68] +Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#69, count(1)#68 AS number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] -(89) Filter [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) +(89) Filter [codegen id : 86] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(32,6)) > cast(Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) -(90) Project [codegen id : 90] -Output [6]: [store AS channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] +(90) Project [codegen id : 86] +Output [6]: [store AS channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] (91) Scan parquet default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#54)] +PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 45] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +(92) ColumnarToRow [codegen id : 43] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -(93) Filter [codegen id : 45] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +(93) Filter [codegen id : 43] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Condition : isnotnull(ss_item_sk#75) (94) Exchange -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Arguments: hashpartitioning(ss_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#80] -(95) Sort [codegen id : 46] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 44] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Arguments: [ss_item_sk#75 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(97) Sort [codegen id : 65] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 62] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [ss_item_sk#75] +Right keys [1]: [ss_item_sk#47] Join condition: None (99) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#33] +Output [2]: [d_date_sk#81, d_week_seq#82] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 66] -Input [2]: [d_date_sk#12, d_week_seq#33] +(100) ColumnarToRow [codegen id : 63] +Input [2]: [d_date_sk#81, d_week_seq#82] -(101) Filter [codegen id : 66] -Input [2]: [d_date_sk#12, d_week_seq#33] -Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#56, [id=#57])) AND isnotnull(d_date_sk#12)) +(101) Filter [codegen id : 63] +Input [2]: [d_date_sk#81, d_week_seq#82] +Condition : ((isnotnull(d_week_seq#82) AND (d_week_seq#82 = Subquery scalar-subquery#83, [id=#84])) AND isnotnull(d_date_sk#81)) -(102) Project [codegen id : 66] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#33] +(102) Project [codegen id : 63] +Output [1]: [d_date_sk#81] +Input [2]: [d_date_sk#81, d_week_seq#82] (103) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] +Input [1]: [d_date_sk#81] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#85] -(104) BroadcastHashJoin [codegen id : 88] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(104) BroadcastHashJoin [codegen id : 84] +Left keys [1]: [ss_sold_date_sk#78] +Right keys [1]: [d_date_sk#81] Join condition: None -(105) Project [codegen id : 88] -Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +(105) Project [codegen id : 84] +Output [3]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77] +Input [5]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#81] (106) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +Output [4]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] -(107) BroadcastHashJoin [codegen id : 88] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#59] +(107) BroadcastHashJoin [codegen id : 84] +Left keys [1]: [ss_item_sk#75] +Right keys [1]: [i_item_sk#86] Join condition: None -(108) Project [codegen id : 88] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +(108) Project [codegen id : 84] +Output [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] +Input [7]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] -(109) HashAggregate [codegen id : 88] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] +(109) HashAggregate [codegen id : 84] +Input [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] +Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] +Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] (110) Exchange -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] -Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] +Arguments: hashpartitioning(i_brand_id#87, i_class_id#88, i_category_id#89, 5), ENSURE_REQUIREMENTS, [id=#96] -(111) HashAggregate [codegen id : 89] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70, count(1)#71] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sales#72, count(1)#71 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] +(111) HashAggregate [codegen id : 85] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] +Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] +Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#99, count(1)#98 AS number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] -(112) Filter [codegen id : 89] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) +(112) Filter [codegen id : 85] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) -(113) Project [codegen id : 89] -Output [6]: [store AS channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] +(113) Project [codegen id : 85] +Output [6]: [store AS channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] (114) BroadcastExchange -Input [6]: [channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#76] +Input [6]: [channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#103] -(115) BroadcastHashJoin [codegen id : 90] -Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +(115) BroadcastHashJoin [codegen id : 86] +Left keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] +Right keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] Join condition: None (116) TakeOrderedAndProject -Input [12]: [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Input [12]: [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Arguments: 100, [i_brand_id#55 ASC NULLS FIRST, i_class_id#56 ASC NULLS FIRST, i_category_id#57 ASC NULLS FIRST], [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#72, [id=#73] * HashAggregate (139) +- Exchange (138) +- * HashAggregate (137) @@ -674,140 +674,140 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (117) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ss_sold_date_sk#106), dynamicpruningexpression(ss_sold_date_sk#106 IN dynamicpruning#107)] ReadSchema: struct (118) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] (119) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#108, d_year#109] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (120) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#108, d_year#109] (121) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#108, d_year#109] +Condition : (((isnotnull(d_year#109) AND (d_year#109 >= 1998)) AND (d_year#109 <= 2000)) AND isnotnull(d_date_sk#108)) (122) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#108] +Input [2]: [d_date_sk#108, d_year#109] (123) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] +Input [1]: [d_date_sk#108] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#110] (124) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#106] +Right keys [1]: [d_date_sk#108] Join condition: None (125) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#79, ss_list_price#3 AS list_price#80] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#104 AS quantity#111, ss_list_price#105 AS list_price#112] +Input [4]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106, d_date_sk#108] (126) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] +Output [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(cs_sold_date_sk#115), dynamicpruningexpression(cs_sold_date_sk#115 IN dynamicpruning#107)] ReadSchema: struct (127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] +Input [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] (128) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#116] (129) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#115] +Right keys [1]: [d_date_sk#116] Join condition: None (130) Project [codegen id : 4] -Output [2]: [cs_quantity#81 AS quantity#83, cs_list_price#82 AS list_price#84] -Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21, d_date_sk#12] +Output [2]: [cs_quantity#113 AS quantity#117, cs_list_price#114 AS list_price#118] +Input [4]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115, d_date_sk#116] (131) Scan parquet default.web_sales -Output [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] +Output [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#107)] ReadSchema: struct (132) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] +Input [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] (133) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#122] (134) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#121] +Right keys [1]: [d_date_sk#122] Join condition: None (135) Project [codegen id : 6] -Output [2]: [ws_quantity#85 AS quantity#87, ws_list_price#86 AS list_price#88] -Input [4]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27, d_date_sk#12] +Output [2]: [ws_quantity#119 AS quantity#123, ws_list_price#120 AS list_price#124] +Input [4]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121, d_date_sk#122] (136) Union (137) HashAggregate [codegen id : 7] -Input [2]: [quantity#79, list_price#80] +Input [2]: [quantity#111, list_price#112] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#89, count#90] -Results [2]: [sum#91, count#92] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#125, count#126] +Results [2]: [sum#127, count#128] (138) Exchange -Input [2]: [sum#91, count#92] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] +Input [2]: [sum#127, count#128] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] (139) HashAggregate [codegen id : 8] -Input [2]: [sum#91, count#92] +Input [2]: [sum#127, count#128] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94 AS average_sales#95] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130 AS average_sales#131] -Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#77 +Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#106 IN dynamicpruning#107 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#108] -Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#77 +Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#115 IN dynamicpruning#107 -Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#77 +Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#107 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (141) (141) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#49] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 ReusedExchange (142) (142) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#14] -Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 -Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#34, [id=#35] +Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#51, [id=#52] * Project (146) +- * Filter (145) +- * ColumnarToRow (144) @@ -815,33 +815,33 @@ Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquer (143) Scan parquet default.date_dim -Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] (145) Filter [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 1999)) AND (d_moy#96 = 12)) AND (d_dom#97 = 16)) +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Condition : (((((isnotnull(d_year#133) AND isnotnull(d_moy#134)) AND isnotnull(d_dom#135)) AND (d_year#133 = 1999)) AND (d_moy#134 = 12)) AND (d_dom#135 = 16)) (146) Project [codegen id : 1] -Output [1]: [d_week_seq#33] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [1]: [d_week_seq#132] +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] -Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] +Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] -Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#54 +Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#79 ReusedExchange (147) (147) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#81] -Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#56, [id=#57] +Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#83, [id=#84] * Project (151) +- * Filter (150) +- * ColumnarToRow (149) @@ -849,21 +849,21 @@ Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subqu (148) Scan parquet default.date_dim -Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (149) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] (150) Filter [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 1998)) AND (d_moy#96 = 12)) AND (d_dom#97 = 16)) +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Condition : (((((isnotnull(d_year#137) AND isnotnull(d_moy#138)) AND isnotnull(d_dom#139)) AND (d_year#137 = 1998)) AND (d_moy#138 = 12)) AND (d_dom#139 = 16)) (151) Project [codegen id : 1] -Output [1]: [d_week_seq#33] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [1]: [d_week_seq#136] +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt index 1d5907e319658..1b41abbc97b06 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (90) + WholeStageCodegen (86) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] @@ -49,7 +49,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (44) + WholeStageCodegen (42) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -68,11 +68,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #3 - WholeStageCodegen (21) + WholeStageCodegen (20) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 - WholeStageCodegen (20) + WholeStageCodegen (19) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -81,84 +81,82 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (19) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (18) + WholeStageCodegen (17) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (13) + WholeStageCodegen (12) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (11) + WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #9 InputAdapter - Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] + BroadcastExchange #9 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #10 + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #9 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (7) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (17) + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (16) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (16) + WholeStageCodegen (15) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -169,12 +167,12 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #10 + ReusedExchange [d_date_sk] #9 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter BroadcastExchange #3 - WholeStageCodegen (22) + WholeStageCodegen (21) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #3 @@ -190,29 +188,29 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #15 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #16 - WholeStageCodegen (23) + WholeStageCodegen (22) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #20 - WholeStageCodegen (89) + WholeStageCodegen (85) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (88) + WholeStageCodegen (84) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -220,24 +218,24 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] - WholeStageCodegen (46) + WholeStageCodegen (44) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #22 - WholeStageCodegen (45) + WholeStageCodegen (43) Filter [ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #6 ReusedExchange [d_date_sk] #23 - WholeStageCodegen (65) + WholeStageCodegen (62) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #23 - WholeStageCodegen (66) + WholeStageCodegen (63) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 258224e574cfb..bdafb17d69d45 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -27,36 +27,36 @@ TakeOrderedAndProject (100) : : : :- * HashAggregate (39) : : : : +- Exchange (38) : : : : +- * HashAggregate (37) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) + : : : : +- * Project (36) + : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : :- * Project (33) + : : : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet default.store_sales (7) + : : : : : +- BroadcastExchange (31) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Project (28) + : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet default.item (16) + : : : : : +- BroadcastExchange (26) + : : : : : +- * Project (25) + : : : : : +- * Filter (24) + : : : : : +- * ColumnarToRow (23) + : : : : : +- Scan parquet default.date_dim (22) + : : : : +- ReusedExchange (34) : : : +- BroadcastExchange (49) : : : +- * Project (48) : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -131,443 +131,443 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_item_sk#10) (10) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(11) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(12) Filter [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +(12) Filter [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(13) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +(13) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join condition: None +(14) ColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(15) Filter [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Condition : isnotnull(cs_item_sk#17) -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +(16) Scan parquet default.item +Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] 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 - -(17) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(18) Filter [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +(17) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(19) Project [codegen id : 2] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +(18) Filter [codegen id : 1] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Condition : isnotnull(i_item_sk#19) -(20) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +(19) BroadcastExchange +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#19] Join condition: None -(22) Project [codegen id : 6] -Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] -Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(23) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(22) Scan parquet default.date_dim +Output [2]: [d_date_sk#24, d_year#25] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +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 -(24) ColumnarToRow [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(23) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#24, d_year#25] -(25) Filter [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +(24) Filter [codegen id : 2] +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) -(26) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +(25) Project [codegen id : 2] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_year#25] -(27) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(26) BroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(28) Filter [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +(27) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] +Join condition: None + +(28) Project [codegen id : 3] +Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] (29) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] -(30) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +(30) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join condition: None -(31) Project [codegen id : 5] -Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(32) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(31) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +(32) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] Join condition: None -(34) Project [codegen id : 5] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(33) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(35) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] +(34) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#29] -(36) BroadcastHashJoin [codegen id : 6] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#29] Join condition: None +(36) Project [codegen id : 6] +Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] + (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (38) Exchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_item_sk#34) -(43) ReusedExchange [Reuses operator id: 29] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(43) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#36] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] +Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -(46) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(46) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#40] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#40] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] (49) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (53) BroadcastExchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#15, class_id#16, category_id#17] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#27] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] +Output [1]: [i_item_sk#6 AS ss_item_sk#43] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] (56) BroadcastExchange -Input [1]: [ss_item_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [ss_item_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#43] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Condition : (((isnotnull(i_item_sk#45) AND isnotnull(i_brand_id#46)) AND isnotnull(i_class_id#47)) AND isnotnull(i_category_id#48)) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [i_item_sk#45] +Right keys [1]: [ss_item_sk#43] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#45] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (66) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#30] +Output [2]: [d_date_sk#50, d_week_seq#51] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [2]: [d_date_sk#12, d_week_seq#30] +Input [2]: [d_date_sk#50, d_week_seq#51] (68) Filter [codegen id : 24] -Input [2]: [d_date_sk#12, d_week_seq#30] -Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#31, [id=#32])) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#50, d_week_seq#51] +Condition : ((isnotnull(d_week_seq#51) AND (d_week_seq#51 = Subquery scalar-subquery#52, [id=#53])) AND isnotnull(d_date_sk#50)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#30] +Output [1]: [d_date_sk#50] +Input [2]: [d_date_sk#50, d_week_seq#51] (70) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] +Input [1]: [d_date_sk#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#50] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#34, isEmpty#35, count#36] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] +Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] +Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#61] (75) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41, count(1)#42] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sales#43, count(1)#42 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] (76) Filter [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45 as decimal(32,6)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) (77) Project [codegen id : 52] -Output [6]: [store AS channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] +Output [6]: [store AS channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] (78) Scan parquet default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#49)] +PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] (80) Filter [codegen id : 50] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +Condition : isnotnull(ss_item_sk#70) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [ss_item_sk#70] +Right keys [1]: [ss_item_sk#43] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] +Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] (84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#50] +Left keys [1]: [ss_item_sk#70] +Right keys [1]: [i_item_sk#75] Join condition: None (85) Project [codegen id : 50] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] +Output [6]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] +Input [8]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#30] +Output [2]: [d_date_sk#79, d_week_seq#80] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#12, d_week_seq#30] +Input [2]: [d_date_sk#79, d_week_seq#80] (88) Filter [codegen id : 49] -Input [2]: [d_date_sk#12, d_week_seq#30] -Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#54, [id=#55])) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#79, d_week_seq#80] +Condition : ((isnotnull(d_week_seq#80) AND (d_week_seq#80 = Subquery scalar-subquery#81, [id=#82])) AND isnotnull(d_date_sk#79)) (89) Project [codegen id : 49] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#30] +Output [1]: [d_date_sk#79] +Input [2]: [d_date_sk#79, d_week_seq#80] (90) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +Input [1]: [d_date_sk#79] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#83] (91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#73] +Right keys [1]: [d_date_sk#79] Join condition: None (92) Project [codegen id : 50] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53, d_date_sk#12] +Output [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] +Input [7]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78, d_date_sk#79] (93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] -Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#57, isEmpty#58, count#59] -Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] +Input [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] +Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] +Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] (94) Exchange -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] -Arguments: hashpartitioning(i_brand_id#51, i_class_id#52, i_category_id#53, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] +Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, [id=#90] (95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] -Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] -Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] +Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] +Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] (96) Filter [codegen id : 51] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) (97) Project [codegen id : 51] -Output [6]: [store AS channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] +Output [6]: [store AS channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] (98) BroadcastExchange -Input [6]: [channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] +Input [6]: [channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#97] (99) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] +Left keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Right keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] Join condition: None (100) TakeOrderedAndProject -Input [12]: [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Input [12]: [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Arguments: 100, [i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#46, [id=#47] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#67, [id=#68] * HashAggregate (123) +- Exchange (122) +- * HashAggregate (121) @@ -594,140 +594,140 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(ss_sold_date_sk#100), dynamicpruningexpression(ss_sold_date_sk#100 IN dynamicpruning#101)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] (103) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#102, d_year#103] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#102, d_year#103] (105) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#102, d_year#103] +Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 <= 2000)) AND isnotnull(d_date_sk#102)) (106) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#102] +Input [2]: [d_date_sk#102, d_year#103] (107) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] +Input [1]: [d_date_sk#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#104] (108) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#100] +Right keys [1]: [d_date_sk#102] Join condition: None (109) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#73, ss_list_price#3 AS list_price#74] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#98 AS quantity#105, ss_list_price#99 AS list_price#106] +Input [4]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100, d_date_sk#102] (110) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] +Output [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(cs_sold_date_sk#109), dynamicpruningexpression(cs_sold_date_sk#109 IN dynamicpruning#101)] ReadSchema: struct (111) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] +Input [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] (112) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#110] (113) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#109] +Right keys [1]: [d_date_sk#110] Join condition: None (114) Project [codegen id : 4] -Output [2]: [cs_quantity#75 AS quantity#77, cs_list_price#76 AS list_price#78] -Input [4]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19, d_date_sk#12] +Output [2]: [cs_quantity#107 AS quantity#111, cs_list_price#108 AS list_price#112] +Input [4]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109, d_date_sk#110] (115) Scan parquet default.web_sales -Output [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] +Output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(ws_sold_date_sk#115), dynamicpruningexpression(ws_sold_date_sk#115 IN dynamicpruning#101)] ReadSchema: struct (116) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] +Input [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] (117) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#116] (118) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#115] +Right keys [1]: [d_date_sk#116] Join condition: None (119) Project [codegen id : 6] -Output [2]: [ws_quantity#79 AS quantity#81, ws_list_price#80 AS list_price#82] -Input [4]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24, d_date_sk#12] +Output [2]: [ws_quantity#113 AS quantity#117, ws_list_price#114 AS list_price#118] +Input [4]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115, d_date_sk#116] (120) Union (121) HashAggregate [codegen id : 7] -Input [2]: [quantity#73, list_price#74] +Input [2]: [quantity#105, list_price#106] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#83, count#84] -Results [2]: [sum#85, count#86] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#119, count#120] +Results [2]: [sum#121, count#122] (122) Exchange -Input [2]: [sum#85, count#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#87] +Input [2]: [sum#121, count#122] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#123] (123) HashAggregate [codegen id : 8] -Input [2]: [sum#85, count#86] +Input [2]: [sum#121, count#122] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88 AS average_sales#89] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124 AS average_sales#125] -Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#71 +Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#100 IN dynamicpruning#101 ReusedExchange (124) (124) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#102] -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#71 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#109 IN dynamicpruning#101 -Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#71 +Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#115 IN dynamicpruning#101 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#50] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 ReusedExchange (126) -(126) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(126) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#29] -Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#31, [id=#32] +Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] * Project (130) +- * Filter (129) +- * ColumnarToRow (128) @@ -735,33 +735,33 @@ Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer (127) Scan parquet default.date_dim -Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (128) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] (129) Filter [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 1999)) AND (d_moy#90 = 12)) AND (d_dom#91 = 16)) +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Condition : (((((isnotnull(d_year#127) AND isnotnull(d_moy#128)) AND isnotnull(d_dom#129)) AND (d_year#127 = 1999)) AND (d_moy#128 = 12)) AND (d_dom#129 = 16)) (130) Project [codegen id : 1] -Output [1]: [d_week_seq#30] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [1]: [d_week_seq#126] +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] -Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] -Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#49 +Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 ReusedExchange (131) (131) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#79] -Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#81, [id=#82] * Project (135) +- * Filter (134) +- * ColumnarToRow (133) @@ -769,21 +769,21 @@ Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subque (132) Scan parquet default.date_dim -Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (133) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] (134) Filter [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 1998)) AND (d_moy#90 = 12)) AND (d_dom#91 = 16)) +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Condition : (((((isnotnull(d_year#131) AND isnotnull(d_moy#132)) AND isnotnull(d_dom#133)) AND (d_year#131 = 1998)) AND (d_moy#132 = 12)) AND (d_dom#133 = 16)) (135) Project [codegen id : 1] -Output [1]: [d_week_seq#30] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [1]: [d_week_seq#130] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index 3507d76bcf9b9..cb9b410ea8568 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -82,53 +82,53 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Exchange [brand_id,class_id,category_id] #5 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #9 + BroadcastExchange #8 WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 + BroadcastExchange #6 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 WholeStageCodegen (9) 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 9b47400d3b780..43ebe6e0b4d9b 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 @@ -40,39 +40,39 @@ TakeOrderedAndProject (213) : : : : : +- * HashAggregate (44) : : : : : +- Exchange (43) : : : : : +- * HashAggregate (42) - : : : : : +- SortMergeJoin LeftSemi (41) - : : : : : :- * Sort (26) - : : : : : : +- Exchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (18) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : : :- * Filter (11) - : : : : : : : : +- * ColumnarToRow (10) - : : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : : +- BroadcastExchange (16) - : : : : : : : +- * Project (15) - : : : : : : : +- * Filter (14) - : : : : : : : +- * ColumnarToRow (13) - : : : : : : : +- Scan parquet default.date_dim (12) - : : : : : : +- BroadcastExchange (22) - : : : : : : +- * Filter (21) - : : : : : : +- * ColumnarToRow (20) - : : : : : : +- Scan parquet default.item (19) - : : : : : +- * Sort (40) - : : : : : +- Exchange (39) - : : : : : +- * Project (38) - : : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : : :- * Project (32) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : : :- * Filter (29) - : : : : : : : +- * ColumnarToRow (28) - : : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : : +- ReusedExchange (30) - : : : : : +- BroadcastExchange (36) - : : : : : +- * Filter (35) - : : : : : +- * ColumnarToRow (34) - : : : : : +- Scan parquet default.item (33) + : : : : : +- * Project (41) + : : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (39) + : : : : : +- SortMergeJoin LeftSemi (38) + : : : : : :- * Sort (23) + : : : : : : +- Exchange (22) + : : : : : : +- * Filter (21) + : : : : : : +- * ColumnarToRow (20) + : : : : : : +- Scan parquet default.item (19) + : : : : : +- * Sort (37) + : : : : : +- Exchange (36) + : : : : : +- * Project (35) + : : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : : :- * Project (29) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : : :- * Filter (26) + : : : : : : : +- * ColumnarToRow (25) + : : : : : : : +- Scan parquet default.catalog_sales (24) + : : : : : : +- ReusedExchange (27) + : : : : : +- BroadcastExchange (33) + : : : : : +- * Filter (32) + : : : : : +- * ColumnarToRow (31) + : : : : : +- Scan parquet default.item (30) : : : : +- * Sort (57) : : : : +- Exchange (56) : : : : +- * Project (55) @@ -244,972 +244,972 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 20] +(7) ColumnarToRow [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 20] +(8) Filter [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 5] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +(10) ColumnarToRow [codegen id : 10] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -(11) Filter [codegen id : 5] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +(11) Filter [codegen id : 10] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_item_sk#11) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#14, d_year#15] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#14, d_year#15] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1999)) AND (d_year#15 <= 2001)) AND isnotnull(d_date_sk#14)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#14] +Input [2]: [d_date_sk#14, d_year#15] (16) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(17) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(17) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#14] Join condition: None -(18) Project [codegen id : 5] -Output [1]: [ss_item_sk#1] -Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] +(18) Project [codegen id : 10] +Output [1]: [ss_item_sk#11] +Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] (19) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) -(22) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] - -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join condition: None +(22) Exchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] -(24) Project [codegen id : 5] -Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] -Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(23) Sort [codegen id : 5] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 -(25) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] - -(26) Sort [codegen id : 6] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 - -(27) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] +(24) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +(25) ColumnarToRow [codegen id : 8] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] -(29) Filter [codegen id : 9] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +(26) Filter [codegen id : 8] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#22) -(30) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +(27) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#24] -(31) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +(28) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None -(32) Project [codegen id : 9] -Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] +(29) Project [codegen id : 8] +Output [1]: [cs_item_sk#22] +Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] -(33) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(30) Scan parquet default.item +Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(31) ColumnarToRow [codegen id : 7] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -(35) Filter [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +(32) Filter [codegen id : 7] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Condition : isnotnull(i_item_sk#25) -(36) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +(33) BroadcastExchange +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_item_sk#22] +Right keys [1]: [i_item_sk#25] Join condition: None -(38) Project [codegen id : 9] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(35) Project [codegen id : 8] +Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -(39) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] +(36) Exchange +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] -(40) Sort [codegen id : 10] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(37) Sort [codegen id : 9] +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 -(41) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +(38) SortMergeJoin +Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] Join condition: None -(42) HashAggregate [codegen id : 11] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(39) BroadcastExchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] + +(40) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_item_sk#11] +Right keys [1]: [i_item_sk#17] +Join condition: None + +(41) Project [codegen id : 10] +Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] +Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] + +(42) HashAggregate [codegen id : 10] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (43) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(44) HashAggregate [codegen id : 12] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(44) HashAggregate [codegen id : 11] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (45) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] -(46) Sort [codegen id : 13] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 12] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +(48) ColumnarToRow [codegen id : 15] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -(49) Filter [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(49) Filter [codegen id : 15] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_item_sk#37) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#39] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(51) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#39] Join condition: None -(52) Project [codegen id : 16] -Output [1]: [ws_item_sk#26] -Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] +(52) Project [codegen id : 15] +Output [1]: [ws_item_sk#37] +Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] -(53) ReusedExchange [Reuses operator id: 36] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(53) ReusedExchange [Reuses operator id: 33] +Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(54) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#40] Join condition: None -(55) Project [codegen id : 16] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(55) Project [codegen id : 15] +Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] (56) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] -(57) Sort [codegen id : 17] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 16] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] +Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] Join condition: None -(59) HashAggregate [codegen id : 18] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(59) HashAggregate [codegen id : 17] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (60) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] -(61) HashAggregate [codegen id : 19] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(61) HashAggregate [codegen id : 18] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (62) BroadcastExchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] -(63) BroadcastHashJoin [codegen id : 20] +(63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#16, class_id#17, category_id#18] +Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join condition: None -(64) Project [codegen id : 20] -Output [1]: [i_item_sk#7 AS ss_item_sk#31] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] +(64) Project [codegen id : 19] +Output [1]: [i_item_sk#7 AS ss_item_sk#47] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] (65) Exchange -Input [1]: [ss_item_sk#31] -Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [1]: [ss_item_sk#47] +Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] -(66) Sort [codegen id : 21] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 20] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#47] Join condition: None (68) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#33] +Output [3]: [d_date_sk#49, d_year#50, d_moy#51] 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 -(69) ColumnarToRow [codegen id : 22] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] +(69) ColumnarToRow [codegen id : 21] +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] -(70) Filter [codegen id : 22] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#33)) AND (d_year#13 = 2000)) AND (d_moy#33 = 11)) AND isnotnull(d_date_sk#12)) +(70) Filter [codegen id : 21] +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] +Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2000)) AND (d_moy#51 = 11)) AND isnotnull(d_date_sk#49)) -(71) Project [codegen id : 22] -Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] +(71) Project [codegen id : 21] +Output [1]: [d_date_sk#49] +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] (72) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] +Input [1]: [d_date_sk#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] -(73) BroadcastHashJoin [codegen id : 44] +(73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#49] Join condition: None -(74) Project [codegen id : 44] +(74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] (75) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(76) ColumnarToRow [codegen id : 22] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(77) Filter [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +(77) Filter [codegen id : 22] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Condition : isnotnull(i_item_sk#53) (78) Exchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: hashpartitioning(i_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#57] -(79) Sort [codegen id : 24] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 +(79) Sort [codegen id : 23] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: [i_item_sk#53 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(81) Sort [codegen id : 43] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 41] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [i_item_sk#53] +Right keys [1]: [ss_item_sk#47] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] -(84) BroadcastHashJoin [codegen id : 44] +(84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#53] Join condition: None -(85) Project [codegen id : 44] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(85) Project [codegen id : 42] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(86) HashAggregate [codegen id : 44] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +(86) HashAggregate [codegen id : 42] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] +Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#37, isEmpty#38, count#39] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] +Aggregate Attributes [3]: [sum#59, isEmpty#60, count#61] +Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] +Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#65] -(88) HashAggregate [codegen id : 45] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +(88) HashAggregate [codegen id : 43] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] +Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44, count(1)#45] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sales#46, count(1)#45 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66, count(1)#67] +Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] -(89) Filter [codegen id : 45] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48 as decimal(32,6)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(89) Filter [codegen id : 43] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(32,6)) > cast(Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(90) Project [codegen id : 45] -Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] +(90) Project [codegen id : 43] +Output [6]: [store AS channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] (91) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Output [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#77), dynamicpruningexpression(cs_sold_date_sk#77 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 46] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +(92) ColumnarToRow [codegen id : 44] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -(93) Filter [codegen id : 46] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +(93) Filter [codegen id : 44] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Condition : isnotnull(cs_item_sk#74) (94) Exchange -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Arguments: hashpartitioning(cs_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#78] -(95) Sort [codegen id : 47] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -Arguments: [cs_item_sk#20 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 45] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Arguments: [cs_item_sk#74 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(97) Sort [codegen id : 66] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 63] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [cs_item_sk#74] +Right keys [1]: [ss_item_sk#47] Join condition: None (99) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#79] -(100) BroadcastHashJoin [codegen id : 89] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +(100) BroadcastHashJoin [codegen id : 85] +Left keys [1]: [cs_sold_date_sk#77] +Right keys [1]: [d_date_sk#79] Join condition: None -(101) Project [codegen id : 89] -Output [3]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53] -Input [5]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] +(101) Project [codegen id : 85] +Output [3]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76] +Input [5]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77, d_date_sk#79] (102) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -(103) BroadcastHashJoin [codegen id : 89] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +(103) BroadcastHashJoin [codegen id : 85] +Left keys [1]: [cs_item_sk#74] +Right keys [1]: [i_item_sk#80] Join condition: None -(104) Project [codegen id : 89] -Output [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(104) Project [codegen id : 85] +Output [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] +Input [7]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -(105) HashAggregate [codegen id : 89] -Input [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] +(105) HashAggregate [codegen id : 85] +Input [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] +Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] +Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] (106) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, [id=#90] -(107) HashAggregate [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] +(107) HashAggregate [codegen id : 86] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] +Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] -(108) Filter [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(108) Filter [codegen id : 86] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(109) Project [codegen id : 90] -Output [6]: [catalog AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] +(109) Project [codegen id : 86] +Output [6]: [catalog AS channel#96, i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] (110) Scan parquet default.web_sales -Output [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Output [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#100), dynamicpruningexpression(ws_sold_date_sk#100 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 91] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +(111) ColumnarToRow [codegen id : 87] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -(112) Filter [codegen id : 91] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(112) Filter [codegen id : 87] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Condition : isnotnull(ws_item_sk#97) (113) Exchange -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_item_sk#26, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Arguments: hashpartitioning(ws_item_sk#97, 5), ENSURE_REQUIREMENTS, [id=#101] -(114) Sort [codegen id : 92] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Arguments: [ws_item_sk#26 ASC NULLS FIRST], false, 0 +(114) Sort [codegen id : 88] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Arguments: [ws_item_sk#97 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(116) Sort [codegen id : 111] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(116) Sort [codegen id : 106] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (117) SortMergeJoin -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [ws_item_sk#97] +Right keys [1]: [ss_item_sk#47] Join condition: None (118) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#102] -(119) BroadcastHashJoin [codegen id : 134] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(119) BroadcastHashJoin [codegen id : 128] +Left keys [1]: [ws_sold_date_sk#100] +Right keys [1]: [d_date_sk#102] Join condition: None -(120) Project [codegen id : 134] -Output [3]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69] -Input [5]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] +(120) Project [codegen id : 128] +Output [3]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99] +Input [5]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100, d_date_sk#102] (121) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] -(122) BroadcastHashJoin [codegen id : 134] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(122) BroadcastHashJoin [codegen id : 128] +Left keys [1]: [ws_item_sk#97] +Right keys [1]: [i_item_sk#103] Join condition: None -(123) Project [codegen id : 134] -Output [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(123) Project [codegen id : 128] +Output [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] +Input [7]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] -(124) HashAggregate [codegen id : 134] -Input [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#71, isEmpty#72, count#73] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] +(124) HashAggregate [codegen id : 128] +Input [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] +Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#107, isEmpty#108, count#109] +Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] (125) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +Arguments: hashpartitioning(i_brand_id#104, i_class_id#105, i_category_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] -(126) HashAggregate [codegen id : 135] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78, count(1)#79] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sales#80, count(1)#79 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +(126) HashAggregate [codegen id : 129] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114, count(1)#115] +Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sales#116, count(1)#115 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] -(127) Filter [codegen id : 135] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(127) Filter [codegen id : 129] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(128) Project [codegen id : 135] -Output [6]: [web AS channel#83, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +(128) Project [codegen id : 129] +Output [6]: [web AS channel#119, i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] (129) Union -(130) HashAggregate [codegen id : 136] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#84, isEmpty#85, sum#86] -Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#87, isEmpty#88, sum#89] +(130) HashAggregate [codegen id : 130] +Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69] +Keys [4]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#120, isEmpty#121, sum#122] +Results [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] (131) Exchange -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#87, isEmpty#88, sum#89] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] +Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#126] -(132) HashAggregate [codegen id : 137] -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#87, isEmpty#88, sum#89] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#91, sum(number_sales#47)#92] -Results [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum(sales#46)#91 AS sum_sales#93, sum(number_sales#47)#92 AS number_sales#94] +(132) HashAggregate [codegen id : 131] +Input [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] +Keys [4]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#127, sum(number_sales#69)#128] +Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum(sales#68)#127 AS sum_sales#129, sum(number_sales#69)#128 AS number_sales#130] (133) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#95, isEmpty#96, count#97] +Output [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, count#136] -(134) HashAggregate [codegen id : 182] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#95, isEmpty#96, count#97] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#98, count(1)#99] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#98 AS sales#46, count(1)#99 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#98 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100] +(134) HashAggregate [codegen id : 174] +Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, count#136] +Keys [3]: [i_brand_id#131, i_class_id#132, i_category_id#133] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139, count(1)#140] +Results [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sales#68, count(1)#140 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] -(135) Filter [codegen id : 182] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(135) Filter [codegen id : 174] +Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(136) Project [codegen id : 182] -Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100] +(136) Project [codegen id : 174] +Output [6]: [store AS channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69] +Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] (137) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#101, isEmpty#102, count#103] +Output [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum#145, isEmpty#146, count#147] -(138) HashAggregate [codegen id : 227] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#101, isEmpty#102, count#103] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#104, count(1)#105] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sales#64, count(1)#105 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106] +(138) HashAggregate [codegen id : 217] +Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum#145, isEmpty#146, count#147] +Keys [3]: [i_brand_id#142, i_class_id#143, i_category_id#144] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150, count(1)#151] +Results [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sales#93, count(1)#151 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] -(139) Filter [codegen id : 227] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(139) Filter [codegen id : 217] +Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(140) Project [codegen id : 227] -Output [6]: [catalog AS channel#107, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106] +(140) Project [codegen id : 217] +Output [6]: [catalog AS channel#96, i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94] +Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] (141) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#108, isEmpty#109, count#110] +Output [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum#156, isEmpty#157, count#158] -(142) HashAggregate [codegen id : 272] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#108, isEmpty#109, count#110] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#111, count(1)#112] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#111 AS sales#80, count(1)#112 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#111 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113] +(142) HashAggregate [codegen id : 260] +Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum#156, isEmpty#157, count#158] +Keys [3]: [i_brand_id#153, i_class_id#154, i_category_id#155] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161, count(1)#162] +Results [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161 AS sales#116, count(1)#162 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] -(143) Filter [codegen id : 272] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(143) Filter [codegen id : 260] +Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(144) Project [codegen id : 272] -Output [6]: [web AS channel#114, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113] +(144) Project [codegen id : 260] +Output [6]: [web AS channel#119, i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117] +Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] (145) Union -(146) HashAggregate [codegen id : 273] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] -Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#118, isEmpty#119, sum#120] +(146) HashAggregate [codegen id : 261] +Input [6]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69] +Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#164, isEmpty#165, sum#166] +Results [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] (147) Exchange -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#118, isEmpty#119, sum#120] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#121] - -(148) HashAggregate [codegen id : 274] -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#118, isEmpty#119, sum#120] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#122, sum(number_sales#47)#123] -Results [5]: [channel#51, i_brand_id#8, i_class_id#9, sum(sales#46)#122 AS sum_sales#93, sum(number_sales#47)#123 AS number_sales#94] - -(149) HashAggregate [codegen id : 274] -Input [5]: [channel#51, i_brand_id#8, i_class_id#9, sum_sales#93, number_sales#94] -Keys [3]: [channel#51, i_brand_id#8, i_class_id#9] -Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] -Aggregate Attributes [3]: [sum#124, isEmpty#125, sum#126] -Results [6]: [channel#51, i_brand_id#8, i_class_id#9, sum#127, isEmpty#128, sum#129] +Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] +Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, 5), ENSURE_REQUIREMENTS, [id=#170] + +(148) HashAggregate [codegen id : 262] +Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] +Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#171, sum(number_sales#69)#172] +Results [5]: [channel#73, i_brand_id#131, i_class_id#132, sum(sales#68)#171 AS sum_sales#129, sum(number_sales#69)#172 AS number_sales#130] + +(149) HashAggregate [codegen id : 262] +Input [5]: [channel#73, i_brand_id#131, i_class_id#132, sum_sales#129, number_sales#130] +Keys [3]: [channel#73, i_brand_id#131, i_class_id#132] +Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] +Aggregate Attributes [3]: [sum#173, isEmpty#174, sum#175] +Results [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] (150) Exchange -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, sum#127, isEmpty#128, sum#129] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, 5), ENSURE_REQUIREMENTS, [id=#130] +Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] +Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, 5), ENSURE_REQUIREMENTS, [id=#179] -(151) HashAggregate [codegen id : 275] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, sum#127, isEmpty#128, sum#129] -Keys [3]: [channel#51, i_brand_id#8, i_class_id#9] -Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] -Aggregate Attributes [2]: [sum(sum_sales#93)#131, sum(number_sales#94)#132] -Results [6]: [channel#51, i_brand_id#8, i_class_id#9, null AS i_category_id#133, sum(sum_sales#93)#131 AS sum(sum_sales)#134, sum(number_sales#94)#132 AS sum(number_sales)#135] +(151) HashAggregate [codegen id : 263] +Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] +Keys [3]: [channel#73, i_brand_id#131, i_class_id#132] +Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] +Aggregate Attributes [2]: [sum(sum_sales#129)#180, sum(number_sales#130)#181] +Results [6]: [channel#73, i_brand_id#131, i_class_id#132, null AS i_category_id#182, sum(sum_sales#129)#180 AS sum(sum_sales)#183, sum(number_sales#130)#181 AS sum(number_sales)#184] (152) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#136, isEmpty#137, count#138] +Output [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum#188, isEmpty#189, count#190] -(153) HashAggregate [codegen id : 320] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#136, isEmpty#137, count#138] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#139, count(1)#140] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sales#46, count(1)#140 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141] +(153) HashAggregate [codegen id : 306] +Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum#188, isEmpty#189, count#190] +Keys [3]: [i_brand_id#185, i_class_id#186, i_category_id#187] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193, count(1)#194] +Results [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193 AS sales#68, count(1)#194 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] -(154) Filter [codegen id : 320] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(154) Filter [codegen id : 306] +Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(155) Project [codegen id : 320] -Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141] +(155) Project [codegen id : 306] +Output [6]: [store AS channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69] +Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] (156) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#142, isEmpty#143, count#144] +Output [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum#199, isEmpty#200, count#201] -(157) HashAggregate [codegen id : 365] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#142, isEmpty#143, count#144] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#145, count(1)#146] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sales#64, count(1)#146 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147] +(157) HashAggregate [codegen id : 349] +Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum#199, isEmpty#200, count#201] +Keys [3]: [i_brand_id#196, i_class_id#197, i_category_id#198] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204, count(1)#205] +Results [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204 AS sales#93, count(1)#205 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] -(158) Filter [codegen id : 365] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(158) Filter [codegen id : 349] +Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(159) Project [codegen id : 365] -Output [6]: [catalog AS channel#148, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147] +(159) Project [codegen id : 349] +Output [6]: [catalog AS channel#96, i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94] +Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] (160) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#149, isEmpty#150, count#151] +Output [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum#210, isEmpty#211, count#212] -(161) HashAggregate [codegen id : 410] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#149, isEmpty#150, count#151] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#152, count(1)#153] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#152 AS sales#80, count(1)#153 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#152 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154] +(161) HashAggregate [codegen id : 392] +Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum#210, isEmpty#211, count#212] +Keys [3]: [i_brand_id#207, i_class_id#208, i_category_id#209] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215, count(1)#216] +Results [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sales#116, count(1)#216 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] -(162) Filter [codegen id : 410] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(162) Filter [codegen id : 392] +Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(163) Project [codegen id : 410] -Output [6]: [web AS channel#155, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154] +(163) Project [codegen id : 392] +Output [6]: [web AS channel#119, i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117] +Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] (164) Union -(165) HashAggregate [codegen id : 411] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#156, isEmpty#157, sum#158] -Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#159, isEmpty#160, sum#161] +(165) HashAggregate [codegen id : 393] +Input [6]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69] +Keys [4]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#218, isEmpty#219, sum#220] +Results [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] (166) Exchange -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#159, isEmpty#160, sum#161] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#162] - -(167) HashAggregate [codegen id : 412] -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#159, isEmpty#160, sum#161] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#163, sum(number_sales#47)#164] -Results [4]: [channel#51, i_brand_id#8, sum(sales#46)#163 AS sum_sales#93, sum(number_sales#47)#164 AS number_sales#94] - -(168) HashAggregate [codegen id : 412] -Input [4]: [channel#51, i_brand_id#8, sum_sales#93, number_sales#94] -Keys [2]: [channel#51, i_brand_id#8] -Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] -Aggregate Attributes [3]: [sum#165, isEmpty#166, sum#167] -Results [5]: [channel#51, i_brand_id#8, sum#168, isEmpty#169, sum#170] +Input [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] +Arguments: hashpartitioning(channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, 5), ENSURE_REQUIREMENTS, [id=#224] + +(167) HashAggregate [codegen id : 394] +Input [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] +Keys [4]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#225, sum(number_sales#69)#226] +Results [4]: [channel#73, i_brand_id#185, sum(sales#68)#225 AS sum_sales#129, sum(number_sales#69)#226 AS number_sales#130] + +(168) HashAggregate [codegen id : 394] +Input [4]: [channel#73, i_brand_id#185, sum_sales#129, number_sales#130] +Keys [2]: [channel#73, i_brand_id#185] +Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] +Aggregate Attributes [3]: [sum#227, isEmpty#228, sum#229] +Results [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] (169) Exchange -Input [5]: [channel#51, i_brand_id#8, sum#168, isEmpty#169, sum#170] -Arguments: hashpartitioning(channel#51, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [id=#171] +Input [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] +Arguments: hashpartitioning(channel#73, i_brand_id#185, 5), ENSURE_REQUIREMENTS, [id=#233] -(170) HashAggregate [codegen id : 413] -Input [5]: [channel#51, i_brand_id#8, sum#168, isEmpty#169, sum#170] -Keys [2]: [channel#51, i_brand_id#8] -Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] -Aggregate Attributes [2]: [sum(sum_sales#93)#172, sum(number_sales#94)#173] -Results [6]: [channel#51, i_brand_id#8, null AS i_class_id#174, null AS i_category_id#175, sum(sum_sales#93)#172 AS sum(sum_sales)#176, sum(number_sales#94)#173 AS sum(number_sales)#177] +(170) HashAggregate [codegen id : 395] +Input [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] +Keys [2]: [channel#73, i_brand_id#185] +Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] +Aggregate Attributes [2]: [sum(sum_sales#129)#234, sum(number_sales#130)#235] +Results [6]: [channel#73, i_brand_id#185, null AS i_class_id#236, null AS i_category_id#237, sum(sum_sales#129)#234 AS sum(sum_sales)#238, sum(number_sales#130)#235 AS sum(number_sales)#239] (171) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#178, isEmpty#179, count#180] +Output [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum#243, isEmpty#244, count#245] -(172) HashAggregate [codegen id : 458] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#178, isEmpty#179, count#180] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#181, count(1)#182] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sales#46, count(1)#182 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183] +(172) HashAggregate [codegen id : 438] +Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum#243, isEmpty#244, count#245] +Keys [3]: [i_brand_id#240, i_class_id#241, i_category_id#242] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248, count(1)#249] +Results [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248 AS sales#68, count(1)#249 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] -(173) Filter [codegen id : 458] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(173) Filter [codegen id : 438] +Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(174) Project [codegen id : 458] -Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183] +(174) Project [codegen id : 438] +Output [6]: [store AS channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69] +Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] (175) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#184, isEmpty#185, count#186] +Output [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum#254, isEmpty#255, count#256] -(176) HashAggregate [codegen id : 503] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#184, isEmpty#185, count#186] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#187, count(1)#188] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sales#64, count(1)#188 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189] +(176) HashAggregate [codegen id : 481] +Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum#254, isEmpty#255, count#256] +Keys [3]: [i_brand_id#251, i_class_id#252, i_category_id#253] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259, count(1)#260] +Results [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259 AS sales#93, count(1)#260 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] -(177) Filter [codegen id : 503] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(177) Filter [codegen id : 481] +Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(178) Project [codegen id : 503] -Output [6]: [catalog AS channel#190, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189] +(178) Project [codegen id : 481] +Output [6]: [catalog AS channel#96, i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94] +Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] (179) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#191, isEmpty#192, count#193] +Output [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum#265, isEmpty#266, count#267] -(180) HashAggregate [codegen id : 548] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#191, isEmpty#192, count#193] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#194, count(1)#195] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#194 AS sales#80, count(1)#195 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#194 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196] +(180) HashAggregate [codegen id : 524] +Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum#265, isEmpty#266, count#267] +Keys [3]: [i_brand_id#262, i_class_id#263, i_category_id#264] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270, count(1)#271] +Results [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270 AS sales#116, count(1)#271 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] -(181) Filter [codegen id : 548] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(181) Filter [codegen id : 524] +Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(182) Project [codegen id : 548] -Output [6]: [web AS channel#197, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196] +(182) Project [codegen id : 524] +Output [6]: [web AS channel#119, i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117] +Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] (183) Union -(184) HashAggregate [codegen id : 549] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#198, isEmpty#199, sum#200] -Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#201, isEmpty#202, sum#203] +(184) HashAggregate [codegen id : 525] +Input [6]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69] +Keys [4]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#273, isEmpty#274, sum#275] +Results [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] (185) Exchange -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#201, isEmpty#202, sum#203] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#204] - -(186) HashAggregate [codegen id : 550] -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#201, isEmpty#202, sum#203] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#205, sum(number_sales#47)#206] -Results [3]: [channel#51, sum(sales#46)#205 AS sum_sales#93, sum(number_sales#47)#206 AS number_sales#94] - -(187) HashAggregate [codegen id : 550] -Input [3]: [channel#51, sum_sales#93, number_sales#94] -Keys [1]: [channel#51] -Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] -Aggregate Attributes [3]: [sum#207, isEmpty#208, sum#209] -Results [4]: [channel#51, sum#210, isEmpty#211, sum#212] +Input [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] +Arguments: hashpartitioning(channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, 5), ENSURE_REQUIREMENTS, [id=#279] + +(186) HashAggregate [codegen id : 526] +Input [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] +Keys [4]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#280, sum(number_sales#69)#281] +Results [3]: [channel#73, sum(sales#68)#280 AS sum_sales#129, sum(number_sales#69)#281 AS number_sales#130] + +(187) HashAggregate [codegen id : 526] +Input [3]: [channel#73, sum_sales#129, number_sales#130] +Keys [1]: [channel#73] +Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] +Aggregate Attributes [3]: [sum#282, isEmpty#283, sum#284] +Results [4]: [channel#73, sum#285, isEmpty#286, sum#287] (188) Exchange -Input [4]: [channel#51, sum#210, isEmpty#211, sum#212] -Arguments: hashpartitioning(channel#51, 5), ENSURE_REQUIREMENTS, [id=#213] +Input [4]: [channel#73, sum#285, isEmpty#286, sum#287] +Arguments: hashpartitioning(channel#73, 5), ENSURE_REQUIREMENTS, [id=#288] -(189) HashAggregate [codegen id : 551] -Input [4]: [channel#51, sum#210, isEmpty#211, sum#212] -Keys [1]: [channel#51] -Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] -Aggregate Attributes [2]: [sum(sum_sales#93)#214, sum(number_sales#94)#215] -Results [6]: [channel#51, null AS i_brand_id#216, null AS i_class_id#217, null AS i_category_id#218, sum(sum_sales#93)#214 AS sum(sum_sales)#219, sum(number_sales#94)#215 AS sum(number_sales)#220] +(189) HashAggregate [codegen id : 527] +Input [4]: [channel#73, sum#285, isEmpty#286, sum#287] +Keys [1]: [channel#73] +Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] +Aggregate Attributes [2]: [sum(sum_sales#129)#289, sum(number_sales#130)#290] +Results [6]: [channel#73, null AS i_brand_id#291, null AS i_class_id#292, null AS i_category_id#293, sum(sum_sales#129)#289 AS sum(sum_sales)#294, sum(number_sales#130)#290 AS sum(number_sales)#295] (190) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#221, isEmpty#222, count#223] +Output [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum#299, isEmpty#300, count#301] -(191) HashAggregate [codegen id : 596] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#221, isEmpty#222, count#223] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#224, count(1)#225] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sales#46, count(1)#225 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226] +(191) HashAggregate [codegen id : 570] +Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum#299, isEmpty#300, count#301] +Keys [3]: [i_brand_id#296, i_class_id#297, i_category_id#298] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304, count(1)#305] +Results [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304 AS sales#68, count(1)#305 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] -(192) Filter [codegen id : 596] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(192) Filter [codegen id : 570] +Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(193) Project [codegen id : 596] -Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226] +(193) Project [codegen id : 570] +Output [6]: [store AS channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69] +Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] (194) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#227, isEmpty#228, count#229] +Output [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum#310, isEmpty#311, count#312] -(195) HashAggregate [codegen id : 641] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#227, isEmpty#228, count#229] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#230, count(1)#231] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sales#64, count(1)#231 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232] +(195) HashAggregate [codegen id : 613] +Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum#310, isEmpty#311, count#312] +Keys [3]: [i_brand_id#307, i_class_id#308, i_category_id#309] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315, count(1)#316] +Results [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315 AS sales#93, count(1)#316 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] -(196) Filter [codegen id : 641] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(196) Filter [codegen id : 613] +Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(197) Project [codegen id : 641] -Output [6]: [catalog AS channel#233, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232] +(197) Project [codegen id : 613] +Output [6]: [catalog AS channel#96, i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94] +Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] (198) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#234, isEmpty#235, count#236] +Output [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum#321, isEmpty#322, count#323] -(199) HashAggregate [codegen id : 686] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#234, isEmpty#235, count#236] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#237, count(1)#238] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#237 AS sales#80, count(1)#238 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#237 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239] +(199) HashAggregate [codegen id : 656] +Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum#321, isEmpty#322, count#323] +Keys [3]: [i_brand_id#318, i_class_id#319, i_category_id#320] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326, count(1)#327] +Results [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326 AS sales#116, count(1)#327 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] -(200) Filter [codegen id : 686] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(200) Filter [codegen id : 656] +Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(201) Project [codegen id : 686] -Output [6]: [web AS channel#240, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239] +(201) Project [codegen id : 656] +Output [6]: [web AS channel#119, i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117] +Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] (202) Union -(203) HashAggregate [codegen id : 687] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#241, isEmpty#242, sum#243] -Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#244, isEmpty#245, sum#246] +(203) HashAggregate [codegen id : 657] +Input [6]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69] +Keys [4]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#329, isEmpty#330, sum#331] +Results [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] (204) Exchange -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#244, isEmpty#245, sum#246] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#247] - -(205) HashAggregate [codegen id : 688] -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#244, isEmpty#245, sum#246] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#248, sum(number_sales#47)#249] -Results [2]: [sum(sales#46)#248 AS sum_sales#93, sum(number_sales#47)#249 AS number_sales#94] - -(206) HashAggregate [codegen id : 688] -Input [2]: [sum_sales#93, number_sales#94] +Input [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] +Arguments: hashpartitioning(channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, 5), ENSURE_REQUIREMENTS, [id=#335] + +(205) HashAggregate [codegen id : 658] +Input [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] +Keys [4]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#336, sum(number_sales#69)#337] +Results [2]: [sum(sales#68)#336 AS sum_sales#129, sum(number_sales#69)#337 AS number_sales#130] + +(206) HashAggregate [codegen id : 658] +Input [2]: [sum_sales#129, number_sales#130] Keys: [] -Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] -Aggregate Attributes [3]: [sum#250, isEmpty#251, sum#252] -Results [3]: [sum#253, isEmpty#254, sum#255] +Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] +Aggregate Attributes [3]: [sum#338, isEmpty#339, sum#340] +Results [3]: [sum#341, isEmpty#342, sum#343] (207) Exchange -Input [3]: [sum#253, isEmpty#254, sum#255] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#256] +Input [3]: [sum#341, isEmpty#342, sum#343] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#344] -(208) HashAggregate [codegen id : 689] -Input [3]: [sum#253, isEmpty#254, sum#255] +(208) HashAggregate [codegen id : 659] +Input [3]: [sum#341, isEmpty#342, sum#343] Keys: [] -Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] -Aggregate Attributes [2]: [sum(sum_sales#93)#257, sum(number_sales#94)#258] -Results [6]: [null AS channel#259, null AS i_brand_id#260, null AS i_class_id#261, null AS i_category_id#262, sum(sum_sales#93)#257 AS sum(sum_sales)#263, sum(number_sales#94)#258 AS sum(number_sales)#264] +Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] +Aggregate Attributes [2]: [sum(sum_sales#129)#345, sum(number_sales#130)#346] +Results [6]: [null AS channel#347, null AS i_brand_id#348, null AS i_class_id#349, null AS i_category_id#350, sum(sum_sales#129)#345 AS sum(sum_sales)#351, sum(number_sales#130)#346 AS sum(number_sales)#352] (209) Union -(210) HashAggregate [codegen id : 690] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] -Keys [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +(210) HashAggregate [codegen id : 660] +Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] (211) Exchange -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94, 5), ENSURE_REQUIREMENTS, [id=#265] +Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130, 5), ENSURE_REQUIREMENTS, [id=#353] -(212) HashAggregate [codegen id : 691] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] -Keys [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +(212) HashAggregate [codegen id : 661] +Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] (213) TakeOrderedAndProject -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] -Arguments: 100, [channel#51 ASC NULLS FIRST, i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Arguments: 100, [channel#73 ASC NULLS FIRST, i_brand_id#54 ASC NULLS FIRST, i_class_id#55 ASC NULLS FIRST, i_category_id#56 ASC NULLS FIRST], [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#49, [id=#50] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#71, [id=#72] * HashAggregate (240) +- Exchange (239) +- * HashAggregate (238) @@ -1240,193 +1240,193 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (214) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#266)] +PartitionFilters: [isnotnull(ss_sold_date_sk#356), dynamicpruningexpression(ss_sold_date_sk#356 IN dynamicpruning#357)] ReadSchema: struct (215) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356] (216) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#358, d_year#359] 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 (217) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#358, d_year#359] (218) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#358, d_year#359] +Condition : (((isnotnull(d_year#359) AND (d_year#359 >= 1999)) AND (d_year#359 <= 2001)) AND isnotnull(d_date_sk#358)) (219) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#358] +Input [2]: [d_date_sk#358, d_year#359] (220) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#267] +Input [1]: [d_date_sk#358] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#360] (221) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#356] +Right keys [1]: [d_date_sk#358] Join condition: None (222) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#268, ss_list_price#3 AS list_price#269] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#354 AS quantity#361, ss_list_price#355 AS list_price#362] +Input [4]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356, d_date_sk#358] (223) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Output [3]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#270)] +PartitionFilters: [isnotnull(cs_sold_date_sk#365), dynamicpruningexpression(cs_sold_date_sk#365 IN dynamicpruning#366)] ReadSchema: struct (224) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Input [3]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365] (225) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#367, d_year#368] 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 (226) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#367, d_year#368] (227) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#367, d_year#368] +Condition : (((isnotnull(d_year#368) AND (d_year#368 >= 1998)) AND (d_year#368 <= 2000)) AND isnotnull(d_date_sk#367)) (228) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#367] +Input [2]: [d_date_sk#367, d_year#368] (229) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#271] +Input [1]: [d_date_sk#367] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#369] (230) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#365] +Right keys [1]: [d_date_sk#367] Join condition: None (231) Project [codegen id : 4] -Output [2]: [cs_quantity#52 AS quantity#272, cs_list_price#53 AS list_price#273] -Input [4]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] +Output [2]: [cs_quantity#363 AS quantity#370, cs_list_price#364 AS list_price#371] +Input [4]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365, d_date_sk#367] (232) Scan parquet default.web_sales -Output [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Output [3]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#270)] +PartitionFilters: [isnotnull(ws_sold_date_sk#374), dynamicpruningexpression(ws_sold_date_sk#374 IN dynamicpruning#366)] ReadSchema: struct (233) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Input [3]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374] (234) ReusedExchange [Reuses operator id: 229] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#375] (235) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#374] +Right keys [1]: [d_date_sk#375] Join condition: None (236) Project [codegen id : 6] -Output [2]: [ws_quantity#68 AS quantity#274, ws_list_price#69 AS list_price#275] -Input [4]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] +Output [2]: [ws_quantity#372 AS quantity#376, ws_list_price#373 AS list_price#377] +Input [4]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374, d_date_sk#375] (237) Union (238) HashAggregate [codegen id : 7] -Input [2]: [quantity#268, list_price#269] +Input [2]: [quantity#361, list_price#362] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#276, count#277] -Results [2]: [sum#278, count#279] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#378, count#379] +Results [2]: [sum#380, count#381] (239) Exchange -Input [2]: [sum#278, count#279] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#280] +Input [2]: [sum#380, count#381] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#382] (240) HashAggregate [codegen id : 8] -Input [2]: [sum#278, count#279] +Input [2]: [sum#380, count#381] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#281] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#281 AS average_sales#282] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))#383] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))#383 AS average_sales#384] -Subquery:2 Hosting operator id = 214 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#266 +Subquery:2 Hosting operator id = 214 Hosting Expression = ss_sold_date_sk#356 IN dynamicpruning#357 ReusedExchange (241) (241) ReusedExchange [Reuses operator id: 220] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#358] -Subquery:3 Hosting operator id = 223 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#270 +Subquery:3 Hosting operator id = 223 Hosting Expression = cs_sold_date_sk#365 IN dynamicpruning#366 ReusedExchange (242) (242) ReusedExchange [Reuses operator id: 229] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#367] -Subquery:4 Hosting operator id = 232 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#270 +Subquery:4 Hosting operator id = 232 Hosting Expression = ws_sold_date_sk#374 IN dynamicpruning#366 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (243) (243) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#49] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 ReusedExchange (244) (244) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#14] -Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 -Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#77 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#5 -Subquery:13 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:13 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:14 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:14 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:15 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:15 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:16 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:16 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:17 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:17 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:18 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:18 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:19 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:19 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:20 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:20 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:21 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:21 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:22 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:22 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:23 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:23 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:24 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:24 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt index d27eb5a32b387..cb186b0ddcd3b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt @@ -1,21 +1,21 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - WholeStageCodegen (691) + WholeStageCodegen (661) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (690) + WholeStageCodegen (660) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union - WholeStageCodegen (137) + WholeStageCodegen (131) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (136) + WholeStageCodegen (130) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (45) + WholeStageCodegen (43) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (44) + WholeStageCodegen (42) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -90,11 +90,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) + WholeStageCodegen (20) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #6 - WholeStageCodegen (20) + WholeStageCodegen (19) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -103,84 +103,82 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #7 - WholeStageCodegen (19) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (18) + WholeStageCodegen (17) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (13) + WholeStageCodegen (12) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #10 - WholeStageCodegen (11) + WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #11 InputAdapter - Exchange [brand_id,class_id,category_id] #11 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #12 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] + BroadcastExchange #11 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #12 + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #11 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (7) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #12 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (17) + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (16) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (16) + WholeStageCodegen (15) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -191,12 +189,12 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #12 + ReusedExchange [d_date_sk] #11 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 InputAdapter BroadcastExchange #5 - WholeStageCodegen (22) + WholeStageCodegen (21) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow @@ -205,27 +203,27 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter BroadcastExchange #17 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #18 - WholeStageCodegen (23) + WholeStageCodegen (22) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 - WholeStageCodegen (90) + WholeStageCodegen (86) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #23 - WholeStageCodegen (89) + WholeStageCodegen (85) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -233,17 +231,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [cs_item_sk,ss_item_sk] - WholeStageCodegen (47) + WholeStageCodegen (45) Sort [cs_item_sk] InputAdapter Exchange [cs_item_sk] #24 - WholeStageCodegen (46) + WholeStageCodegen (44) Filter [cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (66) + WholeStageCodegen (63) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 @@ -251,14 +249,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedExchange [d_date_sk] #5 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (135) + WholeStageCodegen (129) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #25 - WholeStageCodegen (134) + WholeStageCodegen (128) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] @@ -266,17 +264,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ws_item_sk,ss_item_sk] - WholeStageCodegen (92) + WholeStageCodegen (88) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #26 - WholeStageCodegen (91) + WholeStageCodegen (87) Filter [ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (111) + WholeStageCodegen (106) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 @@ -284,136 +282,136 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedExchange [d_date_sk] #5 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (275) + WholeStageCodegen (263) HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id] #27 - WholeStageCodegen (274) + WholeStageCodegen (262) HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #28 - WholeStageCodegen (273) + WholeStageCodegen (261) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (182) + WholeStageCodegen (174) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (227) + WholeStageCodegen (217) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (272) + WholeStageCodegen (260) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (413) + WholeStageCodegen (395) HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id] #32 - WholeStageCodegen (412) + WholeStageCodegen (394) HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #33 - WholeStageCodegen (411) + WholeStageCodegen (393) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (320) + WholeStageCodegen (306) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (365) + WholeStageCodegen (349) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (410) + WholeStageCodegen (392) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (551) + WholeStageCodegen (527) HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel] #34 - WholeStageCodegen (550) + WholeStageCodegen (526) HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #35 - WholeStageCodegen (549) + WholeStageCodegen (525) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (458) + WholeStageCodegen (438) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (503) + WholeStageCodegen (481) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (548) + WholeStageCodegen (524) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (689) + WholeStageCodegen (659) HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange #36 - WholeStageCodegen (688) + WholeStageCodegen (658) HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #37 - WholeStageCodegen (687) + WholeStageCodegen (657) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (596) + WholeStageCodegen (570) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (641) + WholeStageCodegen (613) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (686) + WholeStageCodegen (656) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 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 04fe14cab2c9b..755973f79e493 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 @@ -34,36 +34,36 @@ TakeOrderedAndProject (194) : : : : :- * HashAggregate (39) : : : : : +- Exchange (38) : : : : : +- * HashAggregate (37) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : : :- * Project (22) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : : :- * Project (15) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : :- * Filter (9) - : : : : : : : : +- * ColumnarToRow (8) - : : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : : +- BroadcastExchange (13) - : : : : : : : +- * Filter (12) - : : : : : : : +- * ColumnarToRow (11) - : : : : : : : +- Scan parquet default.item (10) - : : : : : : +- BroadcastExchange (20) - : : : : : : +- * Project (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet default.date_dim (16) - : : : : : +- BroadcastExchange (35) - : : : : : +- * Project (34) - : : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : : :- * Project (31) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : : :- * Filter (25) - : : : : : : : +- * ColumnarToRow (24) - : : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : : +- BroadcastExchange (29) - : : : : : : +- * Filter (28) - : : : : : : +- * ColumnarToRow (27) - : : : : : : +- Scan parquet default.item (26) - : : : : : +- ReusedExchange (32) + : : : : : +- * Project (36) + : : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : : :- * Project (33) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (31) + : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : : : : : :- * Filter (12) + : : : : : : : +- * ColumnarToRow (11) + : : : : : : : +- Scan parquet default.item (10) + : : : : : : +- BroadcastExchange (29) + : : : : : : +- * Project (28) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * Filter (15) + : : : : : : : : +- * ColumnarToRow (14) + : : : : : : : : +- Scan parquet default.catalog_sales (13) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * Filter (18) + : : : : : : : +- * ColumnarToRow (17) + : : : : : : : +- Scan parquet default.item (16) + : : : : : : +- BroadcastExchange (26) + : : : : : : +- * Project (25) + : : : : : : +- * Filter (24) + : : : : : : +- * ColumnarToRow (23) + : : : : : : +- Scan parquet default.date_dim (22) + : : : : : +- ReusedExchange (34) : : : : +- BroadcastExchange (49) : : : : +- * Project (48) : : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -225,896 +225,896 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_item_sk#10) (10) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(11) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(12) Filter [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +(12) Filter [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(13) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +(13) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join condition: None +(14) ColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(15) Filter [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Condition : isnotnull(cs_item_sk#17) -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +(16) Scan parquet default.item +Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] 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 - -(17) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(18) Filter [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +(17) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(19) Project [codegen id : 2] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +(18) Filter [codegen id : 1] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Condition : isnotnull(i_item_sk#19) -(20) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +(19) BroadcastExchange +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#19] Join condition: None -(22) Project [codegen id : 6] -Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] -Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(23) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(22) Scan parquet default.date_dim +Output [2]: [d_date_sk#24, d_year#25] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +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 -(24) ColumnarToRow [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(23) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#24, d_year#25] -(25) Filter [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +(24) Filter [codegen id : 2] +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) -(26) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +(25) Project [codegen id : 2] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_year#25] -(27) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(26) BroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(28) Filter [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +(27) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] +Join condition: None + +(28) Project [codegen id : 3] +Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] (29) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] -(30) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +(30) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join condition: None -(31) Project [codegen id : 5] -Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(32) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(31) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +(32) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] Join condition: None -(34) Project [codegen id : 5] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(33) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(35) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] +(34) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#29] -(36) BroadcastHashJoin [codegen id : 6] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#29] Join condition: None +(36) Project [codegen id : 6] +Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] + (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (38) Exchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_item_sk#34) -(43) ReusedExchange [Reuses operator id: 29] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(43) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#36] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] +Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -(46) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(46) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#40] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#40] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] (49) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (53) BroadcastExchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#15, class_id#16, category_id#17] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#27] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] +Output [1]: [i_item_sk#6 AS ss_item_sk#43] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] (56) BroadcastExchange -Input [1]: [ss_item_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [ss_item_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#43] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Condition : isnotnull(i_item_sk#45) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [i_item_sk#45] +Right keys [1]: [ss_item_sk#43] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#45] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (66) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#30] +Output [3]: [d_date_sk#50, d_year#51, d_moy#52] 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 (67) ColumnarToRow [codegen id : 24] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] (68) Filter [codegen id : 24] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#30)) AND (d_year#13 = 2000)) AND (d_moy#30 = 11)) AND isnotnull(d_date_sk#12)) +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] +Condition : ((((isnotnull(d_year#51) AND isnotnull(d_moy#52)) AND (d_year#51 = 2000)) AND (d_moy#52 = 11)) AND isnotnull(d_date_sk#50)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] +Output [1]: [d_date_sk#50] +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] (70) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [1]: [d_date_sk#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#50] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#32, isEmpty#33, count#34] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] +Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] +Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#60] (75) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] (76) Filter [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] +Output [6]: [store AS channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] (78) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Output [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] (80) Filter [codegen id : 51] -Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +Condition : isnotnull(cs_item_sk#69) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [cs_item_sk#69] +Right keys [1]: [ss_item_sk#43] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] (84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [cs_item_sk#69] +Right keys [1]: [i_item_sk#73] Join condition: None (85) Project [codegen id : 51] -Output [6]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [8]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] (86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#77] (87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#72] +Right keys [1]: [d_date_sk#77] Join condition: None (88) Project [codegen id : 51] -Output [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [7]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76, d_date_sk#77] (89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#49, isEmpty#50, count#51] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] +Input [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] +Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#78, isEmpty#79, count#80] +Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] (90) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Arguments: hashpartitioning(i_brand_id#74, i_class_id#75, i_category_id#76, 5), ENSURE_REQUIREMENTS, [id=#84] (91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56, count(1)#57] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sales#58, count(1)#57 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85, count(1)#86] +Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sales#87, count(1)#86 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] (92) Filter [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [catalog AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] +Output [6]: [catalog AS channel#90, i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] (94) Scan parquet default.web_sales -Output [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Output [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] (96) Filter [codegen id : 77] -Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Condition : isnotnull(ws_item_sk#91) (97) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [ws_item_sk#91] +Right keys [1]: [ss_item_sk#43] Join condition: None (99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] (100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#91] +Right keys [1]: [i_item_sk#95] Join condition: None (101) Project [codegen id : 77] -Output [6]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [8]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] (102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#99] (103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#94] +Right keys [1]: [d_date_sk#99] Join condition: None (104) Project [codegen id : 77] -Output [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [7]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98, d_date_sk#99] (105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] +Input [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] +Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#100, isEmpty#101, count#102] +Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] (106) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Arguments: hashpartitioning(i_brand_id#96, i_class_id#97, i_category_id#98, 5), ENSURE_REQUIREMENTS, [id=#106] (107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71, count(1)#72] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sales#73, count(1)#72 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] +Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#109, count(1)#108 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] (108) Filter [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] +Output [6]: [web AS channel#112, i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] (110) Union (111) HashAggregate [codegen id : 79] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#77, isEmpty#78, sum#79] -Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#80, isEmpty#81, sum#82] +Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64] +Keys [4]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#113, isEmpty#114, sum#115] +Results [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] (112) Exchange -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#80, isEmpty#81, sum#82] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] +Arguments: hashpartitioning(channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#119] (113) HashAggregate [codegen id : 80] -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#80, isEmpty#81, sum#82] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#84, sum(number_sales#42)#85] -Results [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#41)#84 AS sum_sales#86, sum(number_sales#42)#85 AS number_sales#87] +Input [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] +Keys [4]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#120, sum(number_sales#64)#121] +Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum(sales#63)#120 AS sum_sales#122, sum(number_sales#64)#121 AS number_sales#123] (114) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#88, isEmpty#89, count#90] +Output [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] (115) HashAggregate [codegen id : 106] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#88, isEmpty#89, count#90] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#41, count(1)#92 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93] +Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] +Keys [3]: [i_brand_id#124, i_class_id#125, i_category_id#126] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132, count(1)#133] +Results [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sales#63, count(1)#133 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] (116) Filter [codegen id : 106] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (117) Project [codegen id : 106] -Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93] +Output [6]: [store AS channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64] +Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] (118) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#94, isEmpty#95, count#96] +Output [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] (119) HashAggregate [codegen id : 132] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#94, isEmpty#95, count#96] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#58, count(1)#98 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99] +Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] +Keys [3]: [i_brand_id#135, i_class_id#136, i_category_id#137] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143, count(1)#144] +Results [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sales#87, count(1)#144 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] (120) Filter [codegen id : 132] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (121) Project [codegen id : 132] -Output [6]: [catalog AS channel#100, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99] +Output [6]: [catalog AS channel#90, i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88] +Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] (122) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#101, isEmpty#102, count#103] +Output [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum#149, isEmpty#150, count#151] (123) HashAggregate [codegen id : 158] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#101, isEmpty#102, count#103] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#104, count(1)#105] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sales#73, count(1)#105 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106] +Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum#149, isEmpty#150, count#151] +Keys [3]: [i_brand_id#146, i_class_id#147, i_category_id#148] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154, count(1)#155] +Results [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154 AS sales#109, count(1)#155 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] (124) Filter [codegen id : 158] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (125) Project [codegen id : 158] -Output [6]: [web AS channel#107, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106] +Output [6]: [web AS channel#112, i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110] +Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] (126) Union (127) HashAggregate [codegen id : 159] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#108, isEmpty#109, sum#110] -Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#111, isEmpty#112, sum#113] +Input [6]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64] +Keys [4]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#157, isEmpty#158, sum#159] +Results [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] (128) Exchange -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#111, isEmpty#112, sum#113] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#114] +Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] +Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, 5), ENSURE_REQUIREMENTS, [id=#163] (129) HashAggregate [codegen id : 160] -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#111, isEmpty#112, sum#113] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#115, sum(number_sales#42)#116] -Results [5]: [channel#46, i_brand_id#7, i_class_id#8, sum(sales#41)#115 AS sum_sales#86, sum(number_sales#42)#116 AS number_sales#87] +Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] +Keys [4]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#164, sum(number_sales#64)#165] +Results [5]: [channel#68, i_brand_id#124, i_class_id#125, sum(sales#63)#164 AS sum_sales#122, sum(number_sales#64)#165 AS number_sales#123] (130) HashAggregate [codegen id : 160] -Input [5]: [channel#46, i_brand_id#7, i_class_id#8, sum_sales#86, number_sales#87] -Keys [3]: [channel#46, i_brand_id#7, i_class_id#8] -Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] -Aggregate Attributes [3]: [sum#117, isEmpty#118, sum#119] -Results [6]: [channel#46, i_brand_id#7, i_class_id#8, sum#120, isEmpty#121, sum#122] +Input [5]: [channel#68, i_brand_id#124, i_class_id#125, sum_sales#122, number_sales#123] +Keys [3]: [channel#68, i_brand_id#124, i_class_id#125] +Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] +Aggregate Attributes [3]: [sum#166, isEmpty#167, sum#168] +Results [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] (131) Exchange -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, sum#120, isEmpty#121, sum#122] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, 5), ENSURE_REQUIREMENTS, [id=#123] +Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] +Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, 5), ENSURE_REQUIREMENTS, [id=#172] (132) HashAggregate [codegen id : 161] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, sum#120, isEmpty#121, sum#122] -Keys [3]: [channel#46, i_brand_id#7, i_class_id#8] -Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] -Aggregate Attributes [2]: [sum(sum_sales#86)#124, sum(number_sales#87)#125] -Results [6]: [channel#46, i_brand_id#7, i_class_id#8, null AS i_category_id#126, sum(sum_sales#86)#124 AS sum(sum_sales)#127, sum(number_sales#87)#125 AS sum(number_sales)#128] +Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] +Keys [3]: [channel#68, i_brand_id#124, i_class_id#125] +Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] +Aggregate Attributes [2]: [sum(sum_sales#122)#173, sum(number_sales#123)#174] +Results [6]: [channel#68, i_brand_id#124, i_class_id#125, null AS i_category_id#175, sum(sum_sales#122)#173 AS sum(sum_sales)#176, sum(number_sales#123)#174 AS sum(number_sales)#177] (133) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#129, isEmpty#130, count#131] +Output [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] (134) HashAggregate [codegen id : 187] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#129, isEmpty#130, count#131] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#132, count(1)#133] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sales#41, count(1)#133 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134] +Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] +Keys [3]: [i_brand_id#178, i_class_id#179, i_category_id#180] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186, count(1)#187] +Results [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sales#63, count(1)#187 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] (135) Filter [codegen id : 187] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (136) Project [codegen id : 187] -Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134] +Output [6]: [store AS channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64] +Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] (137) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#135, isEmpty#136, count#137] +Output [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] (138) HashAggregate [codegen id : 213] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#135, isEmpty#136, count#137] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#138, count(1)#139] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sales#58, count(1)#139 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140] +Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] +Keys [3]: [i_brand_id#189, i_class_id#190, i_category_id#191] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197, count(1)#198] +Results [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sales#87, count(1)#198 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] (139) Filter [codegen id : 213] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (140) Project [codegen id : 213] -Output [6]: [catalog AS channel#141, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140] +Output [6]: [catalog AS channel#90, i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88] +Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] (141) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#142, isEmpty#143, count#144] +Output [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum#203, isEmpty#204, count#205] (142) HashAggregate [codegen id : 239] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#142, isEmpty#143, count#144] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#145, count(1)#146] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sales#73, count(1)#146 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147] +Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum#203, isEmpty#204, count#205] +Keys [3]: [i_brand_id#200, i_class_id#201, i_category_id#202] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208, count(1)#209] +Results [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208 AS sales#109, count(1)#209 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] (143) Filter [codegen id : 239] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (144) Project [codegen id : 239] -Output [6]: [web AS channel#148, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147] +Output [6]: [web AS channel#112, i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110] +Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] (145) Union (146) HashAggregate [codegen id : 240] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#149, isEmpty#150, sum#151] -Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] +Input [6]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64] +Keys [4]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#211, isEmpty#212, sum#213] +Results [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] (147) Exchange -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#155] +Input [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] +Arguments: hashpartitioning(channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, 5), ENSURE_REQUIREMENTS, [id=#217] (148) HashAggregate [codegen id : 241] -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#156, sum(number_sales#42)#157] -Results [4]: [channel#46, i_brand_id#7, sum(sales#41)#156 AS sum_sales#86, sum(number_sales#42)#157 AS number_sales#87] +Input [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] +Keys [4]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#218, sum(number_sales#64)#219] +Results [4]: [channel#68, i_brand_id#178, sum(sales#63)#218 AS sum_sales#122, sum(number_sales#64)#219 AS number_sales#123] (149) HashAggregate [codegen id : 241] -Input [4]: [channel#46, i_brand_id#7, sum_sales#86, number_sales#87] -Keys [2]: [channel#46, i_brand_id#7] -Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] -Aggregate Attributes [3]: [sum#158, isEmpty#159, sum#160] -Results [5]: [channel#46, i_brand_id#7, sum#161, isEmpty#162, sum#163] +Input [4]: [channel#68, i_brand_id#178, sum_sales#122, number_sales#123] +Keys [2]: [channel#68, i_brand_id#178] +Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] +Aggregate Attributes [3]: [sum#220, isEmpty#221, sum#222] +Results [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] (150) Exchange -Input [5]: [channel#46, i_brand_id#7, sum#161, isEmpty#162, sum#163] -Arguments: hashpartitioning(channel#46, i_brand_id#7, 5), ENSURE_REQUIREMENTS, [id=#164] +Input [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] +Arguments: hashpartitioning(channel#68, i_brand_id#178, 5), ENSURE_REQUIREMENTS, [id=#226] (151) HashAggregate [codegen id : 242] -Input [5]: [channel#46, i_brand_id#7, sum#161, isEmpty#162, sum#163] -Keys [2]: [channel#46, i_brand_id#7] -Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] -Aggregate Attributes [2]: [sum(sum_sales#86)#165, sum(number_sales#87)#166] -Results [6]: [channel#46, i_brand_id#7, null AS i_class_id#167, null AS i_category_id#168, sum(sum_sales#86)#165 AS sum(sum_sales)#169, sum(number_sales#87)#166 AS sum(number_sales)#170] +Input [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] +Keys [2]: [channel#68, i_brand_id#178] +Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] +Aggregate Attributes [2]: [sum(sum_sales#122)#227, sum(number_sales#123)#228] +Results [6]: [channel#68, i_brand_id#178, null AS i_class_id#229, null AS i_category_id#230, sum(sum_sales#122)#227 AS sum(sum_sales)#231, sum(number_sales#123)#228 AS sum(number_sales)#232] (152) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#171, isEmpty#172, count#173] +Output [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] (153) HashAggregate [codegen id : 268] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#171, isEmpty#172, count#173] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#174, count(1)#175] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sales#41, count(1)#175 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176] +Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] +Keys [3]: [i_brand_id#233, i_class_id#234, i_category_id#235] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241, count(1)#242] +Results [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sales#63, count(1)#242 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] (154) Filter [codegen id : 268] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (155) Project [codegen id : 268] -Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176] +Output [6]: [store AS channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64] +Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] (156) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#177, isEmpty#178, count#179] +Output [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] (157) HashAggregate [codegen id : 294] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#177, isEmpty#178, count#179] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#180, count(1)#181] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sales#58, count(1)#181 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182] +Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] +Keys [3]: [i_brand_id#244, i_class_id#245, i_category_id#246] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252, count(1)#253] +Results [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sales#87, count(1)#253 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] (158) Filter [codegen id : 294] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (159) Project [codegen id : 294] -Output [6]: [catalog AS channel#183, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182] +Output [6]: [catalog AS channel#90, i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88] +Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] (160) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#184, isEmpty#185, count#186] +Output [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#260] (161) HashAggregate [codegen id : 320] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#184, isEmpty#185, count#186] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#187, count(1)#188] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sales#73, count(1)#188 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189] +Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#260] +Keys [3]: [i_brand_id#255, i_class_id#256, i_category_id#257] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263, count(1)#264] +Results [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263 AS sales#109, count(1)#264 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] (162) Filter [codegen id : 320] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (163) Project [codegen id : 320] -Output [6]: [web AS channel#190, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189] +Output [6]: [web AS channel#112, i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110] +Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] (164) Union (165) HashAggregate [codegen id : 321] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#191, isEmpty#192, sum#193] -Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#194, isEmpty#195, sum#196] +Input [6]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64] +Keys [4]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#266, isEmpty#267, sum#268] +Results [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] (166) Exchange -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#194, isEmpty#195, sum#196] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#197] +Input [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] +Arguments: hashpartitioning(channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, 5), ENSURE_REQUIREMENTS, [id=#272] (167) HashAggregate [codegen id : 322] -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#194, isEmpty#195, sum#196] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#198, sum(number_sales#42)#199] -Results [3]: [channel#46, sum(sales#41)#198 AS sum_sales#86, sum(number_sales#42)#199 AS number_sales#87] +Input [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] +Keys [4]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#273, sum(number_sales#64)#274] +Results [3]: [channel#68, sum(sales#63)#273 AS sum_sales#122, sum(number_sales#64)#274 AS number_sales#123] (168) HashAggregate [codegen id : 322] -Input [3]: [channel#46, sum_sales#86, number_sales#87] -Keys [1]: [channel#46] -Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] -Aggregate Attributes [3]: [sum#200, isEmpty#201, sum#202] -Results [4]: [channel#46, sum#203, isEmpty#204, sum#205] +Input [3]: [channel#68, sum_sales#122, number_sales#123] +Keys [1]: [channel#68] +Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] +Aggregate Attributes [3]: [sum#275, isEmpty#276, sum#277] +Results [4]: [channel#68, sum#278, isEmpty#279, sum#280] (169) Exchange -Input [4]: [channel#46, sum#203, isEmpty#204, sum#205] -Arguments: hashpartitioning(channel#46, 5), ENSURE_REQUIREMENTS, [id=#206] +Input [4]: [channel#68, sum#278, isEmpty#279, sum#280] +Arguments: hashpartitioning(channel#68, 5), ENSURE_REQUIREMENTS, [id=#281] (170) HashAggregate [codegen id : 323] -Input [4]: [channel#46, sum#203, isEmpty#204, sum#205] -Keys [1]: [channel#46] -Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] -Aggregate Attributes [2]: [sum(sum_sales#86)#207, sum(number_sales#87)#208] -Results [6]: [channel#46, null AS i_brand_id#209, null AS i_class_id#210, null AS i_category_id#211, sum(sum_sales#86)#207 AS sum(sum_sales)#212, sum(number_sales#87)#208 AS sum(number_sales)#213] +Input [4]: [channel#68, sum#278, isEmpty#279, sum#280] +Keys [1]: [channel#68] +Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] +Aggregate Attributes [2]: [sum(sum_sales#122)#282, sum(number_sales#123)#283] +Results [6]: [channel#68, null AS i_brand_id#284, null AS i_class_id#285, null AS i_category_id#286, sum(sum_sales#122)#282 AS sum(sum_sales)#287, sum(number_sales#123)#283 AS sum(number_sales)#288] (171) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#214, isEmpty#215, count#216] +Output [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] (172) HashAggregate [codegen id : 349] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#214, isEmpty#215, count#216] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#217, count(1)#218] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#217 AS sales#41, count(1)#218 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#217 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219] +Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] +Keys [3]: [i_brand_id#289, i_class_id#290, i_category_id#291] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297, count(1)#298] +Results [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sales#63, count(1)#298 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] (173) Filter [codegen id : 349] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (174) Project [codegen id : 349] -Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219] +Output [6]: [store AS channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64] +Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] (175) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] +Output [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] (176) HashAggregate [codegen id : 375] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#223, count(1)#224] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sales#58, count(1)#224 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225] +Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] +Keys [3]: [i_brand_id#300, i_class_id#301, i_category_id#302] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308, count(1)#309] +Results [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sales#87, count(1)#309 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] (177) Filter [codegen id : 375] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (178) Project [codegen id : 375] -Output [6]: [catalog AS channel#226, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225] +Output [6]: [catalog AS channel#90, i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88] +Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] (179) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#227, isEmpty#228, count#229] +Output [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum#314, isEmpty#315, count#316] (180) HashAggregate [codegen id : 401] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#227, isEmpty#228, count#229] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#230, count(1)#231] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sales#73, count(1)#231 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232] +Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum#314, isEmpty#315, count#316] +Keys [3]: [i_brand_id#311, i_class_id#312, i_category_id#313] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319, count(1)#320] +Results [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319 AS sales#109, count(1)#320 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] (181) Filter [codegen id : 401] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (182) Project [codegen id : 401] -Output [6]: [web AS channel#233, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232] +Output [6]: [web AS channel#112, i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110] +Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] (183) Union (184) HashAggregate [codegen id : 402] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#234, isEmpty#235, sum#236] -Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#237, isEmpty#238, sum#239] +Input [6]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64] +Keys [4]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#322, isEmpty#323, sum#324] +Results [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] (185) Exchange -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#237, isEmpty#238, sum#239] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#240] +Input [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] +Arguments: hashpartitioning(channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, 5), ENSURE_REQUIREMENTS, [id=#328] (186) HashAggregate [codegen id : 403] -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#237, isEmpty#238, sum#239] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#241, sum(number_sales#42)#242] -Results [2]: [sum(sales#41)#241 AS sum_sales#86, sum(number_sales#42)#242 AS number_sales#87] +Input [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] +Keys [4]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#329, sum(number_sales#64)#330] +Results [2]: [sum(sales#63)#329 AS sum_sales#122, sum(number_sales#64)#330 AS number_sales#123] (187) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#86, number_sales#87] +Input [2]: [sum_sales#122, number_sales#123] Keys: [] -Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] -Aggregate Attributes [3]: [sum#243, isEmpty#244, sum#245] -Results [3]: [sum#246, isEmpty#247, sum#248] +Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] +Aggregate Attributes [3]: [sum#331, isEmpty#332, sum#333] +Results [3]: [sum#334, isEmpty#335, sum#336] (188) Exchange -Input [3]: [sum#246, isEmpty#247, sum#248] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#249] +Input [3]: [sum#334, isEmpty#335, sum#336] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#337] (189) HashAggregate [codegen id : 404] -Input [3]: [sum#246, isEmpty#247, sum#248] +Input [3]: [sum#334, isEmpty#335, sum#336] Keys: [] -Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] -Aggregate Attributes [2]: [sum(sum_sales#86)#250, sum(number_sales#87)#251] -Results [6]: [null AS channel#252, null AS i_brand_id#253, null AS i_class_id#254, null AS i_category_id#255, sum(sum_sales#86)#250 AS sum(sum_sales)#256, sum(number_sales#87)#251 AS sum(number_sales)#257] +Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] +Aggregate Attributes [2]: [sum(sum_sales#122)#338, sum(number_sales#123)#339] +Results [6]: [null AS channel#340, null AS i_brand_id#341, null AS i_class_id#342, null AS i_category_id#343, sum(sum_sales#122)#338 AS sum(sum_sales)#344, sum(number_sales#123)#339 AS sum(number_sales)#345] (190) Union (191) HashAggregate [codegen id : 405] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] -Keys [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Keys [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] (192) Exchange -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87, 5), ENSURE_REQUIREMENTS, [id=#258] +Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Arguments: hashpartitioning(channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123, 5), ENSURE_REQUIREMENTS, [id=#346] (193) HashAggregate [codegen id : 406] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] -Keys [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Keys [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] (194) TakeOrderedAndProject -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] -Arguments: 100, [channel#46 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Arguments: 100, [channel#68 ASC NULLS FIRST, i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#66, [id=#67] * HashAggregate (221) +- Exchange (220) +- * HashAggregate (219) @@ -1145,193 +1145,193 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (195) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#259)] +PartitionFilters: [isnotnull(ss_sold_date_sk#349), dynamicpruningexpression(ss_sold_date_sk#349 IN dynamicpruning#350)] ReadSchema: struct (196) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349] (197) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#351, d_year#352] 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 (198) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#351, d_year#352] (199) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#351, d_year#352] +Condition : (((isnotnull(d_year#352) AND (d_year#352 >= 1999)) AND (d_year#352 <= 2001)) AND isnotnull(d_date_sk#351)) (200) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#351] +Input [2]: [d_date_sk#351, d_year#352] (201) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#260] +Input [1]: [d_date_sk#351] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#353] (202) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#349] +Right keys [1]: [d_date_sk#351] Join condition: None (203) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#261, ss_list_price#3 AS list_price#262] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#347 AS quantity#354, ss_list_price#348 AS list_price#355] +Input [4]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349, d_date_sk#351] (204) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Output [3]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#263)] +PartitionFilters: [isnotnull(cs_sold_date_sk#358), dynamicpruningexpression(cs_sold_date_sk#358 IN dynamicpruning#359)] ReadSchema: struct (205) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Input [3]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358] (206) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#360, d_year#361] 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 (207) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#360, d_year#361] (208) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#360, d_year#361] +Condition : (((isnotnull(d_year#361) AND (d_year#361 >= 1998)) AND (d_year#361 <= 2000)) AND isnotnull(d_date_sk#360)) (209) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#360] +Input [2]: [d_date_sk#360, d_year#361] (210) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#264] +Input [1]: [d_date_sk#360] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#362] (211) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#358] +Right keys [1]: [d_date_sk#360] Join condition: None (212) Project [codegen id : 4] -Output [2]: [cs_quantity#47 AS quantity#265, cs_list_price#48 AS list_price#266] -Input [4]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, d_date_sk#12] +Output [2]: [cs_quantity#356 AS quantity#363, cs_list_price#357 AS list_price#364] +Input [4]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358, d_date_sk#360] (213) Scan parquet default.web_sales -Output [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Output [3]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#263)] +PartitionFilters: [isnotnull(ws_sold_date_sk#367), dynamicpruningexpression(ws_sold_date_sk#367 IN dynamicpruning#359)] ReadSchema: struct (214) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Input [3]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367] (215) ReusedExchange [Reuses operator id: 210] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#368] (216) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#367] +Right keys [1]: [d_date_sk#368] Join condition: None (217) Project [codegen id : 6] -Output [2]: [ws_quantity#62 AS quantity#267, ws_list_price#63 AS list_price#268] -Input [4]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, d_date_sk#12] +Output [2]: [ws_quantity#365 AS quantity#369, ws_list_price#366 AS list_price#370] +Input [4]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367, d_date_sk#368] (218) Union (219) HashAggregate [codegen id : 7] -Input [2]: [quantity#261, list_price#262] +Input [2]: [quantity#354, list_price#355] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#269, count#270] -Results [2]: [sum#271, count#272] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#371, count#372] +Results [2]: [sum#373, count#374] (220) Exchange -Input [2]: [sum#271, count#272] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#273] +Input [2]: [sum#373, count#374] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#375] (221) HashAggregate [codegen id : 8] -Input [2]: [sum#271, count#272] +Input [2]: [sum#373, count#374] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#274] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#274 AS average_sales#275] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))#376] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))#376 AS average_sales#377] -Subquery:2 Hosting operator id = 195 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#259 +Subquery:2 Hosting operator id = 195 Hosting Expression = ss_sold_date_sk#349 IN dynamicpruning#350 ReusedExchange (222) (222) ReusedExchange [Reuses operator id: 201] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#351] -Subquery:3 Hosting operator id = 204 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#263 +Subquery:3 Hosting operator id = 204 Hosting Expression = cs_sold_date_sk#358 IN dynamicpruning#359 ReusedExchange (223) (223) ReusedExchange [Reuses operator id: 210] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#360] -Subquery:4 Hosting operator id = 213 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#263 +Subquery:4 Hosting operator id = 213 Hosting Expression = ws_sold_date_sk#367 IN dynamicpruning#359 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (224) (224) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#50] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 ReusedExchange (225) -(225) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(225) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#29] -Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#5 -Subquery:13 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:13 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:14 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:14 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:15 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:15 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:16 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:16 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:17 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:17 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:18 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:18 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:19 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:19 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:20 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:20 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:21 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:21 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:22 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:22 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:23 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:23 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:24 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:24 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index 1634c1e247a12..e351f9e687027 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -104,53 +104,53 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Exchange [brand_id,class_id,category_id] #7 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #11 + BroadcastExchange #10 WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (1) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #8 + BroadcastExchange #8 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter BroadcastExchange #12 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt index b2934b6bb8242..3f2b62a7b8fa4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt @@ -81,57 +81,57 @@ Input [3]: [i_item_sk#5, i_current_price#6, i_category#7] Condition : (isnotnull(i_current_price#6) AND isnotnull(i_item_sk#5)) (7) Scan parquet default.item -Output [2]: [i_current_price#6, i_category#7] +Output [2]: [i_current_price#8, i_category#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] +Input [2]: [i_current_price#8, i_category#9] (9) Filter [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] -Condition : isnotnull(i_category#7) +Input [2]: [i_current_price#8, i_category#9] +Condition : isnotnull(i_category#9) (10) HashAggregate [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] -Keys [1]: [i_category#7] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#6))] -Aggregate Attributes [2]: [sum#8, count#9] -Results [3]: [i_category#7, sum#10, count#11] +Input [2]: [i_current_price#8, i_category#9] +Keys [1]: [i_category#9] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#8))] +Aggregate Attributes [2]: [sum#10, count#11] +Results [3]: [i_category#9, sum#12, count#13] (11) Exchange -Input [3]: [i_category#7, sum#10, count#11] -Arguments: hashpartitioning(i_category#7, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [3]: [i_category#9, sum#12, count#13] +Arguments: hashpartitioning(i_category#9, 5), ENSURE_REQUIREMENTS, [id=#14] (12) HashAggregate [codegen id : 2] -Input [3]: [i_category#7, sum#10, count#11] -Keys [1]: [i_category#7] -Functions [1]: [avg(UnscaledValue(i_current_price#6))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#6))#13] -Results [2]: [cast((avg(UnscaledValue(i_current_price#6))#13 / 100.0) as decimal(11,6)) AS avg(i_current_price)#14, i_category#7 AS i_category#7#15] +Input [3]: [i_category#9, sum#12, count#13] +Keys [1]: [i_category#9] +Functions [1]: [avg(UnscaledValue(i_current_price#8))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#8))#15] +Results [2]: [cast((avg(UnscaledValue(i_current_price#8))#15 / 100.0) as decimal(11,6)) AS avg(i_current_price)#16, i_category#9] (13) BroadcastExchange -Input [2]: [avg(i_current_price)#14, i_category#7#15] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#16] +Input [2]: [avg(i_current_price)#16, i_category#9] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#17] (14) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_category#7] -Right keys [1]: [i_category#7#15] +Right keys [1]: [i_category#9] Join condition: None (15) Filter [codegen id : 3] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] -Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#14)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] +Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#16)), DecimalType(14,7), true)) (16) Project [codegen id : 3] Output [1]: [i_item_sk#5] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] (17) BroadcastExchange Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (18) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#1] @@ -143,143 +143,143 @@ Output [2]: [ss_customer_sk#2, ss_sold_date_sk#3] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, i_item_sk#5] (20) Scan parquet default.date_dim -Output [2]: [d_date_sk#18, d_month_seq#19] +Output [2]: [d_date_sk#19, d_month_seq#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#18, d_month_seq#19] +Input [2]: [d_date_sk#19, d_month_seq#20] (22) Filter [codegen id : 4] -Input [2]: [d_date_sk#18, d_month_seq#19] -Condition : ((isnotnull(d_month_seq#19) AND (d_month_seq#19 = Subquery scalar-subquery#20, [id=#21])) AND isnotnull(d_date_sk#18)) +Input [2]: [d_date_sk#19, d_month_seq#20] +Condition : ((isnotnull(d_month_seq#20) AND (d_month_seq#20 = Subquery scalar-subquery#21, [id=#22])) AND isnotnull(d_date_sk#19)) (23) Project [codegen id : 4] -Output [1]: [d_date_sk#18] -Input [2]: [d_date_sk#18, d_month_seq#19] +Output [1]: [d_date_sk#19] +Input [2]: [d_date_sk#19, d_month_seq#20] (24) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (25) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (26) Project [codegen id : 5] Output [1]: [ss_customer_sk#2] -Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#18] +Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#19] (27) Exchange Input [1]: [ss_customer_sk#2] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#23] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] (28) Sort [codegen id : 6] Input [1]: [ss_customer_sk#2] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#25, ca_state#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#24, ca_state#25] +Input [2]: [ca_address_sk#25, ca_state#26] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : isnotnull(ca_address_sk#24) +Input [2]: [ca_address_sk#25, ca_state#26] +Condition : isnotnull(ca_address_sk#25) (32) Exchange -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [ca_address_sk#25, ca_state#26] +Arguments: hashpartitioning(ca_address_sk#25, 5), ENSURE_REQUIREMENTS, [id=#27] (33) Sort [codegen id : 8] -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#25, ca_state#26] +Arguments: [ca_address_sk#25 ASC NULLS FIRST], false, 0 (34) Scan parquet default.customer -Output [2]: [c_customer_sk#27, c_current_addr_sk#28] +Output [2]: [c_customer_sk#28, c_current_addr_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 9] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] (36) Filter [codegen id : 9] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Condition : (isnotnull(c_current_addr_sk#28) AND isnotnull(c_customer_sk#27)) +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Condition : (isnotnull(c_current_addr_sk#29) AND isnotnull(c_customer_sk#28)) (37) Exchange -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Arguments: hashpartitioning(c_current_addr_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Arguments: hashpartitioning(c_current_addr_sk#29, 5), ENSURE_REQUIREMENTS, [id=#30] (38) Sort [codegen id : 10] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Arguments: [c_current_addr_sk#28 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Arguments: [c_current_addr_sk#29 ASC NULLS FIRST], false, 0 (39) SortMergeJoin [codegen id : 11] -Left keys [1]: [ca_address_sk#24] -Right keys [1]: [c_current_addr_sk#28] +Left keys [1]: [ca_address_sk#25] +Right keys [1]: [c_current_addr_sk#29] Join condition: None (40) Project [codegen id : 11] -Output [2]: [ca_state#25, c_customer_sk#27] -Input [4]: [ca_address_sk#24, ca_state#25, c_customer_sk#27, c_current_addr_sk#28] +Output [2]: [ca_state#26, c_customer_sk#28] +Input [4]: [ca_address_sk#25, ca_state#26, c_customer_sk#28, c_current_addr_sk#29] (41) Exchange -Input [2]: [ca_state#25, c_customer_sk#27] -Arguments: hashpartitioning(c_customer_sk#27, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [ca_state#26, c_customer_sk#28] +Arguments: hashpartitioning(c_customer_sk#28, 5), ENSURE_REQUIREMENTS, [id=#31] (42) Sort [codegen id : 12] -Input [2]: [ca_state#25, c_customer_sk#27] -Arguments: [c_customer_sk#27 ASC NULLS FIRST], false, 0 +Input [2]: [ca_state#26, c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 13] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#27] +Right keys [1]: [c_customer_sk#28] Join condition: None (44) Project [codegen id : 13] -Output [1]: [ca_state#25] -Input [3]: [ss_customer_sk#2, ca_state#25, c_customer_sk#27] +Output [1]: [ca_state#26] +Input [3]: [ss_customer_sk#2, ca_state#26, c_customer_sk#28] (45) HashAggregate [codegen id : 13] -Input [1]: [ca_state#25] -Keys [1]: [ca_state#25] +Input [1]: [ca_state#26] +Keys [1]: [ca_state#26] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#31] -Results [2]: [ca_state#25, count#32] +Aggregate Attributes [1]: [count#32] +Results [2]: [ca_state#26, count#33] (46) Exchange -Input [2]: [ca_state#25, count#32] -Arguments: hashpartitioning(ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [2]: [ca_state#26, count#33] +Arguments: hashpartitioning(ca_state#26, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 14] -Input [2]: [ca_state#25, count#32] -Keys [1]: [ca_state#25] +Input [2]: [ca_state#26, count#33] +Keys [1]: [ca_state#26] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#34] -Results [4]: [ca_state#25 AS state#35, count(1)#34 AS cnt#36, count(1)#34 AS count(1)#37, ca_state#25] +Aggregate Attributes [1]: [count(1)#35] +Results [4]: [ca_state#26 AS state#36, count(1)#35 AS cnt#37, count(1)#35 AS count(1)#38, ca_state#26] (48) Filter [codegen id : 14] -Input [4]: [state#35, cnt#36, count(1)#37, ca_state#25] -Condition : (count(1)#37 >= 10) +Input [4]: [state#36, cnt#37, count(1)#38, ca_state#26] +Condition : (count(1)#38 >= 10) (49) Project [codegen id : 14] -Output [3]: [state#35, cnt#36, ca_state#25] -Input [4]: [state#35, cnt#36, count(1)#37, ca_state#25] +Output [3]: [state#36, cnt#37, ca_state#26] +Input [4]: [state#36, cnt#37, count(1)#38, ca_state#26] (50) TakeOrderedAndProject -Input [3]: [state#35, cnt#36, ca_state#25] -Arguments: 100, [cnt#36 ASC NULLS FIRST, ca_state#25 ASC NULLS FIRST], [state#35, cnt#36] +Input [3]: [state#36, cnt#37, ca_state#26] +Arguments: 100, [cnt#37 ASC NULLS FIRST, ca_state#26 ASC NULLS FIRST], [state#36, cnt#37] ===== Subqueries ===== @@ -288,9 +288,9 @@ ReusedExchange (51) (51) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#19] -Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#20, [id=#21] +Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#21, [id=#22] * HashAggregate (58) +- Exchange (57) +- * HashAggregate (56) @@ -301,39 +301,39 @@ Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquer (52) Scan parquet default.date_dim -Output [3]: [d_month_seq#19, d_year#38, d_moy#39] +Output [3]: [d_month_seq#39, d_year#40, d_moy#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (53) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] (54) Filter [codegen id : 1] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] -Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] +Condition : (((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2000)) AND (d_moy#41 = 1)) (55) Project [codegen id : 1] -Output [1]: [d_month_seq#19] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] +Output [1]: [d_month_seq#39] +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] (56) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#19] -Keys [1]: [d_month_seq#19] +Input [1]: [d_month_seq#39] +Keys [1]: [d_month_seq#39] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#19] +Results [1]: [d_month_seq#39] (57) Exchange -Input [1]: [d_month_seq#19] -Arguments: hashpartitioning(d_month_seq#19, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [1]: [d_month_seq#39] +Arguments: hashpartitioning(d_month_seq#39, 5), ENSURE_REQUIREMENTS, [id=#42] (58) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#19] -Keys [1]: [d_month_seq#19] +Input [1]: [d_month_seq#39] +Keys [1]: [d_month_seq#39] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#19] +Results [1]: [d_month_seq#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt index 23b705e3b97e7..f3badf6efe6b6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt @@ -38,7 +38,7 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] InputAdapter Exchange [i_category] #6 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt index 8598023b0276d..b37db85388e0f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt @@ -160,57 +160,57 @@ Input [3]: [i_item_sk#16, i_current_price#17, i_category#18] Condition : (isnotnull(i_current_price#17) AND isnotnull(i_item_sk#16)) (26) Scan parquet default.item -Output [2]: [i_current_price#17, i_category#18] +Output [2]: [i_current_price#19, i_category#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (27) ColumnarToRow [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] +Input [2]: [i_current_price#19, i_category#20] (28) Filter [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] -Condition : isnotnull(i_category#18) +Input [2]: [i_current_price#19, i_category#20] +Condition : isnotnull(i_category#20) (29) HashAggregate [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] -Keys [1]: [i_category#18] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] -Aggregate Attributes [2]: [sum#19, count#20] -Results [3]: [i_category#18, sum#21, count#22] +Input [2]: [i_current_price#19, i_category#20] +Keys [1]: [i_category#20] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#19))] +Aggregate Attributes [2]: [sum#21, count#22] +Results [3]: [i_category#20, sum#23, count#24] (30) Exchange -Input [3]: [i_category#18, sum#21, count#22] -Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [i_category#20, sum#23, count#24] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#25] (31) HashAggregate [codegen id : 5] -Input [3]: [i_category#18, sum#21, count#22] -Keys [1]: [i_category#18] -Functions [1]: [avg(UnscaledValue(i_current_price#17))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#17))#24] -Results [2]: [cast((avg(UnscaledValue(i_current_price#17))#24 / 100.0) as decimal(11,6)) AS avg(i_current_price)#25, i_category#18 AS i_category#18#26] +Input [3]: [i_category#20, sum#23, count#24] +Keys [1]: [i_category#20] +Functions [1]: [avg(UnscaledValue(i_current_price#19))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#19))#26] +Results [2]: [cast((avg(UnscaledValue(i_current_price#19))#26 / 100.0) as decimal(11,6)) AS avg(i_current_price)#27, i_category#20] (32) BroadcastExchange -Input [2]: [avg(i_current_price)#25, i_category#18#26] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#27] +Input [2]: [avg(i_current_price)#27, i_category#20] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#28] (33) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_category#18] -Right keys [1]: [i_category#18#26] +Right keys [1]: [i_category#20] Join condition: None (34) Filter [codegen id : 6] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] -Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#25)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] +Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#27)), DecimalType(14,7), true)) (35) Project [codegen id : 6] Output [1]: [i_item_sk#16] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] (36) BroadcastExchange Input [1]: [i_item_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (37) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#6] @@ -225,31 +225,31 @@ Input [3]: [ca_state#2, ss_item_sk#6, i_item_sk#16] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [2]: [ca_state#2, count#30] +Aggregate Attributes [1]: [count#30] +Results [2]: [ca_state#2, count#31] (40) Exchange -Input [2]: [ca_state#2, count#30] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [2]: [ca_state#2, count#31] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#32] (41) HashAggregate [codegen id : 8] -Input [2]: [ca_state#2, count#30] +Input [2]: [ca_state#2, count#31] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [4]: [ca_state#2 AS state#33, count(1)#32 AS cnt#34, count(1)#32 AS count(1)#35, ca_state#2] +Aggregate Attributes [1]: [count(1)#33] +Results [4]: [ca_state#2 AS state#34, count(1)#33 AS cnt#35, count(1)#33 AS count(1)#36, ca_state#2] (42) Filter [codegen id : 8] -Input [4]: [state#33, cnt#34, count(1)#35, ca_state#2] -Condition : (count(1)#35 >= 10) +Input [4]: [state#34, cnt#35, count(1)#36, ca_state#2] +Condition : (count(1)#36 >= 10) (43) Project [codegen id : 8] -Output [3]: [state#33, cnt#34, ca_state#2] -Input [4]: [state#33, cnt#34, count(1)#35, ca_state#2] +Output [3]: [state#34, cnt#35, ca_state#2] +Input [4]: [state#34, cnt#35, count(1)#36, ca_state#2] (44) TakeOrderedAndProject -Input [3]: [state#33, cnt#34, ca_state#2] -Arguments: 100, [cnt#34 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#33, cnt#34] +Input [3]: [state#34, cnt#35, ca_state#2] +Arguments: 100, [cnt#35 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#34, cnt#35] ===== Subqueries ===== @@ -271,39 +271,39 @@ Subquery:2 Hosting operator id = 18 Hosting Expression = Subquery scalar-subquer (46) Scan parquet default.date_dim -Output [3]: [d_month_seq#12, d_year#36, d_moy#37] +Output [3]: [d_month_seq#37, d_year#38, d_moy#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (47) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] (48) Filter [codegen id : 1] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] -Condition : (((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2000)) AND (d_moy#37 = 1)) +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] +Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) (49) Project [codegen id : 1] -Output [1]: [d_month_seq#12] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] +Output [1]: [d_month_seq#37] +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] (50) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#12] -Keys [1]: [d_month_seq#12] +Input [1]: [d_month_seq#37] +Keys [1]: [d_month_seq#37] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#12] +Results [1]: [d_month_seq#37] (51) Exchange -Input [1]: [d_month_seq#12] -Arguments: hashpartitioning(d_month_seq#12, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [1]: [d_month_seq#37] +Arguments: hashpartitioning(d_month_seq#37, 5), ENSURE_REQUIREMENTS, [id=#40] (52) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#12] -Keys [1]: [d_month_seq#12] +Input [1]: [d_month_seq#37] +Keys [1]: [d_month_seq#37] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#12] +Results [1]: [d_month_seq#37] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index ceddf1b04fc78..4ba09283e73cd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -68,7 +68,7 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] InputAdapter Exchange [i_category] #8 WholeStageCodegen (4) From 6fcf259ed51eb28d21350f4522f8590e809ff9a7 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Wed, 24 Mar 2021 16:18:32 +0800 Subject: [PATCH 26/38] add comment --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 3 +++ 1 file changed, 3 insertions(+) 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 49abe27286d88..12adfd5cda60d 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 @@ -1434,6 +1434,9 @@ class Analyzer(override val catalogManager: CatalogManager) private def hasConflictingAttrs(p: LogicalPlan): Boolean = { p.children.length > 1 && { + // Note that duplicated attributes are allowed within a single node, + // e.g., df.select($"a", $"a"), so we should only check conflicting + // attributes between nodes. p.children.tail.foldLeft(p.children.head.outputSet) { case (conflictAttrs, child) => conflictAttrs.intersect(child.outputSet) }.nonEmpty From 9211170820fa3f9ddfd7588f43fdb41537fae6a4 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 25 Mar 2021 09:37:23 +0800 Subject: [PATCH 27/38] gen golden --- .../q10.sf100/explain.txt | 102 ++-- .../approved-plans-modified/q10/explain.txt | 100 ++-- .../q27.sf100/explain.txt | 162 +++--- .../approved-plans-modified/q27/explain.txt | 162 +++--- .../q46.sf100/explain.txt | 18 +- .../approved-plans-modified/q46/explain.txt | 14 +- .../q59.sf100/explain.txt | 102 ++-- .../approved-plans-modified/q59/explain.txt | 102 ++-- .../q65.sf100/explain.txt | 86 +-- .../approved-plans-modified/q65/explain.txt | 48 +- .../q68.sf100/explain.txt | 20 +- .../approved-plans-modified/q68/explain.txt | 14 +- .../approved-plans-v1_4/q1.sf100/explain.txt | 122 ++-- .../approved-plans-v1_4/q1/explain.txt | 116 ++-- .../approved-plans-v1_4/q10.sf100/explain.txt | 104 ++-- .../approved-plans-v1_4/q10/explain.txt | 94 ++-- .../approved-plans-v1_4/q11.sf100/explain.txt | 272 ++++----- .../approved-plans-v1_4/q11/explain.txt | 254 ++++----- .../approved-plans-v1_4/q16.sf100/explain.txt | 128 ++--- .../approved-plans-v1_4/q16/explain.txt | 128 ++--- .../approved-plans-v1_4/q17.sf100/explain.txt | 8 +- .../approved-plans-v1_4/q17/explain.txt | 8 +- .../approved-plans-v1_4/q2.sf100/explain.txt | 52 +- .../approved-plans-v1_4/q2/explain.txt | 52 +- .../q23a.sf100/explain.txt | 362 ++++++------ .../approved-plans-v1_4/q23a/explain.txt | 274 ++++----- .../q23b.sf100/explain.txt | 528 +++++++++--------- .../approved-plans-v1_4/q23b/explain.txt | 404 +++++++------- .../q24a.sf100/explain.txt | 206 +++---- .../approved-plans-v1_4/q24a/explain.txt | 170 +++--- .../q24b.sf100/explain.txt | 206 +++---- .../approved-plans-v1_4/q24b/explain.txt | 170 +++--- .../approved-plans-v1_4/q28.sf100/explain.txt | 300 +++++----- .../approved-plans-v1_4/q28/explain.txt | 300 +++++----- .../approved-plans-v1_4/q30.sf100/explain.txt | 142 ++--- .../approved-plans-v1_4/q30/explain.txt | 130 ++--- .../approved-plans-v1_4/q31.sf100/explain.txt | 430 +++++++------- .../approved-plans-v1_4/q31/explain.txt | 362 ++++++------ .../approved-plans-v1_4/q32/explain.txt | 5 - .../approved-plans-v1_4/q33.sf100/explain.txt | 138 ++--- .../approved-plans-v1_4/q33/explain.txt | 138 ++--- .../approved-plans-v1_4/q35.sf100/explain.txt | 116 ++-- .../approved-plans-v1_4/q35/explain.txt | 96 ++-- .../q39a.sf100/explain.txt | 108 ++-- .../approved-plans-v1_4/q39a/explain.txt | 104 ++-- .../q39b.sf100/explain.txt | 108 ++-- .../approved-plans-v1_4/q39b/explain.txt | 104 ++-- .../approved-plans-v1_4/q4.sf100/explain.txt | 442 +++++++-------- .../approved-plans-v1_4/q4/explain.txt | 412 +++++++------- .../approved-plans-v1_4/q41.sf100/explain.txt | 5 - .../approved-plans-v1_4/q41/explain.txt | 5 - .../approved-plans-v1_4/q44.sf100/explain.txt | 118 ++-- .../approved-plans-v1_4/q44/explain.txt | 118 ++-- .../approved-plans-v1_4/q45.sf100/explain.txt | 36 +- .../approved-plans-v1_4/q45/explain.txt | 36 +- .../approved-plans-v1_4/q46.sf100/explain.txt | 44 +- .../approved-plans-v1_4/q46/explain.txt | 14 +- .../approved-plans-v1_4/q47.sf100/explain.txt | 66 +-- .../approved-plans-v1_4/q47/explain.txt | 58 +- .../approved-plans-v1_4/q49.sf100/explain.txt | 198 +++---- .../approved-plans-v1_4/q49/explain.txt | 152 ++--- .../approved-plans-v1_4/q5.sf100/explain.txt | 162 +++--- .../approved-plans-v1_4/q5/explain.txt | 174 +++--- .../approved-plans-v1_4/q54.sf100/explain.txt | 118 ++-- .../approved-plans-v1_4/q54/explain.txt | 120 ++-- .../approved-plans-v1_4/q56.sf100/explain.txt | 140 ++--- .../approved-plans-v1_4/q56/explain.txt | 140 ++--- .../approved-plans-v1_4/q57.sf100/explain.txt | 66 +-- .../approved-plans-v1_4/q57/explain.txt | 58 +- .../approved-plans-v1_4/q58.sf100/explain.txt | 226 ++++---- .../approved-plans-v1_4/q58/explain.txt | 228 ++++---- .../approved-plans-v1_4/q59.sf100/explain.txt | 70 +-- .../approved-plans-v1_4/q59/explain.txt | 70 +-- .../approved-plans-v1_4/q60.sf100/explain.txt | 140 ++--- .../approved-plans-v1_4/q60/explain.txt | 140 ++--- .../approved-plans-v1_4/q61.sf100/explain.txt | 84 +-- .../approved-plans-v1_4/q61/explain.txt | 94 ++-- .../approved-plans-v1_4/q64.sf100/explain.txt | 316 +++++------ .../approved-plans-v1_4/q64/explain.txt | 284 +++++----- .../approved-plans-v1_4/q65.sf100/explain.txt | 98 ++-- .../approved-plans-v1_4/q65/explain.txt | 48 +- .../approved-plans-v1_4/q66.sf100/explain.txt | 70 +-- .../approved-plans-v1_4/q66/explain.txt | 74 +-- .../approved-plans-v1_4/q68.sf100/explain.txt | 44 +- .../approved-plans-v1_4/q68/explain.txt | 14 +- .../approved-plans-v1_4/q69.sf100/explain.txt | 96 ++-- .../approved-plans-v1_4/q69/explain.txt | 94 ++-- .../approved-plans-v1_4/q70.sf100/explain.txt | 136 ++--- .../approved-plans-v1_4/q70/explain.txt | 136 ++--- .../approved-plans-v1_4/q71.sf100/explain.txt | 80 +-- .../approved-plans-v1_4/q71/explain.txt | 80 +-- .../approved-plans-v1_4/q74.sf100/explain.txt | 268 ++++----- .../approved-plans-v1_4/q74/explain.txt | 250 ++++----- .../approved-plans-v1_4/q75.sf100/explain.txt | 368 ++++++------ .../approved-plans-v1_4/q75/explain.txt | 368 ++++++------ .../approved-plans-v1_4/q76.sf100/explain.txt | 80 +-- .../approved-plans-v1_4/q76/explain.txt | 64 +-- .../approved-plans-v1_4/q77.sf100/explain.txt | 284 +++++----- .../approved-plans-v1_4/q77/explain.txt | 300 +++++----- .../approved-plans-v1_4/q78.sf100/explain.txt | 146 ++--- .../approved-plans-v1_4/q78/explain.txt | 146 ++--- .../approved-plans-v1_4/q8.sf100/explain.txt | 120 ++-- .../approved-plans-v1_4/q8/explain.txt | 106 ++-- .../approved-plans-v1_4/q80.sf100/explain.txt | 204 +++---- .../approved-plans-v1_4/q80/explain.txt | 208 +++---- .../approved-plans-v1_4/q81.sf100/explain.txt | 144 ++--- .../approved-plans-v1_4/q81/explain.txt | 126 ++--- .../approved-plans-v1_4/q83.sf100/explain.txt | 210 +++---- .../approved-plans-v1_4/q83/explain.txt | 186 +++--- .../approved-plans-v1_4/q88.sf100/explain.txt | 504 ++++++++--------- .../approved-plans-v1_4/q88/explain.txt | 504 ++++++++--------- .../approved-plans-v1_4/q9.sf100/explain.txt | 416 +++++++------- .../approved-plans-v1_4/q9/explain.txt | 416 +++++++------- .../approved-plans-v1_4/q90.sf100/explain.txt | 80 +-- .../approved-plans-v1_4/q90/explain.txt | 80 +-- .../approved-plans-v1_4/q92/explain.txt | 5 - .../approved-plans-v1_4/q94.sf100/explain.txt | 134 ++--- .../approved-plans-v1_4/q94/explain.txt | 134 ++--- .../approved-plans-v1_4/q95.sf100/explain.txt | 194 +++---- .../approved-plans-v1_4/q95/explain.txt | 194 +++---- .../approved-plans-v1_4/q97.sf100/explain.txt | 40 +- .../approved-plans-v1_4/q97/explain.txt | 40 +- .../q10a.sf100/explain.txt | 96 ++-- .../approved-plans-v2_7/q10a/explain.txt | 94 ++-- .../approved-plans-v2_7/q11.sf100/explain.txt | 268 ++++----- .../approved-plans-v2_7/q11/explain.txt | 250 ++++----- .../q18a.sf100/explain.txt | 402 ++++++------- .../approved-plans-v2_7/q18a/explain.txt | 410 +++++++------- .../q22a.sf100/explain.txt | 122 ++-- .../approved-plans-v2_7/q22a/explain.txt | 122 ++-- .../approved-plans-v2_7/q24.sf100/explain.txt | 182 +++--- .../approved-plans-v2_7/q24/explain.txt | 170 +++--- .../q27a.sf100/explain.txt | 162 +++--- .../approved-plans-v2_7/q27a/explain.txt | 162 +++--- .../approved-plans-v2_7/q35.sf100/explain.txt | 116 ++-- .../approved-plans-v2_7/q35/explain.txt | 96 ++-- .../q35a.sf100/explain.txt | 116 ++-- .../approved-plans-v2_7/q35a/explain.txt | 96 ++-- .../q36a.sf100/explain.txt | 92 +-- .../approved-plans-v2_7/q36a/explain.txt | 92 +-- .../approved-plans-v2_7/q47.sf100/explain.txt | 66 +-- .../approved-plans-v2_7/q47/explain.txt | 58 +- .../approved-plans-v2_7/q49.sf100/explain.txt | 198 +++---- .../approved-plans-v2_7/q49/explain.txt | 152 ++--- .../q51a.sf100/explain.txt | 244 ++++---- .../approved-plans-v2_7/q51a/explain.txt | 242 ++++---- .../approved-plans-v2_7/q57.sf100/explain.txt | 66 +-- .../approved-plans-v2_7/q57/explain.txt | 58 +- .../approved-plans-v2_7/q5a.sf100/explain.txt | 230 ++++---- .../approved-plans-v2_7/q5a/explain.txt | 242 ++++---- .../approved-plans-v2_7/q64.sf100/explain.txt | 316 +++++------ .../approved-plans-v2_7/q64/explain.txt | 284 +++++----- .../q67a.sf100/explain.txt | 296 +++++----- .../approved-plans-v2_7/q67a/explain.txt | 296 +++++----- .../q70a.sf100/explain.txt | 212 +++---- .../approved-plans-v2_7/q70a/explain.txt | 212 +++---- .../approved-plans-v2_7/q74.sf100/explain.txt | 268 ++++----- .../approved-plans-v2_7/q74/explain.txt | 250 ++++----- .../approved-plans-v2_7/q75.sf100/explain.txt | 368 ++++++------ .../approved-plans-v2_7/q75/explain.txt | 368 ++++++------ .../q77a.sf100/explain.txt | 364 ++++++------ .../approved-plans-v2_7/q77a/explain.txt | 380 ++++++------- .../approved-plans-v2_7/q78.sf100/explain.txt | 146 ++--- .../approved-plans-v2_7/q78/explain.txt | 146 ++--- .../q80a.sf100/explain.txt | 268 ++++----- .../approved-plans-v2_7/q80a/explain.txt | 272 ++++----- .../q86a.sf100/explain.txt | 92 +-- .../approved-plans-v2_7/q86a/explain.txt | 92 +-- 168 files changed, 14003 insertions(+), 14023 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt index 8a4b341ac925b..1fdcba1ed4de4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt @@ -137,22 +137,22 @@ Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] Condition : isnotnull(cs_ship_customer_sk#13) (19) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#15] (20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (21) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13 AS customer_sk#15] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#8] +Output [1]: [cs_ship_customer_sk#13 AS customer_sk#16] +Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] (22) Union (23) Exchange Input [1]: [customer_sk#12] -Arguments: hashpartitioning(customer_sk#12, 5), ENSURE_REQUIREMENTS, [id=#16] +Arguments: hashpartitioning(customer_sk#12, 5), ENSURE_REQUIREMENTS, [id=#17] (24) Sort [codegen id : 7] Input [1]: [customer_sk#12] @@ -164,43 +164,43 @@ Right keys [1]: [customer_sk#12] Join condition: None (26) Scan parquet default.store_sales -Output [2]: [ss_customer_sk#17, ss_sold_date_sk#18] +Output [2]: [ss_customer_sk#18, ss_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#19), dynamicpruningexpression(ss_sold_date_sk#19 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 9] -Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] +Input [2]: [ss_customer_sk#18, ss_sold_date_sk#19] (28) Filter [codegen id : 9] -Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#17) +Input [2]: [ss_customer_sk#18, ss_sold_date_sk#19] +Condition : isnotnull(ss_customer_sk#18) (29) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#20] (30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#18] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ss_sold_date_sk#19] +Right keys [1]: [d_date_sk#20] Join condition: None (31) Project [codegen id : 9] -Output [1]: [ss_customer_sk#17 AS customer_sk#19] -Input [3]: [ss_customer_sk#17, ss_sold_date_sk#18, d_date_sk#8] +Output [1]: [ss_customer_sk#18 AS customer_sk#21] +Input [3]: [ss_customer_sk#18, ss_sold_date_sk#19, d_date_sk#20] (32) Exchange -Input [1]: [customer_sk#19] -Arguments: hashpartitioning(customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [customer_sk#21] +Arguments: hashpartitioning(customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#22] (33) Sort [codegen id : 10] -Input [1]: [customer_sk#19] -Arguments: [customer_sk#19 ASC NULLS FIRST], false, 0 +Input [1]: [customer_sk#21] +Arguments: [customer_sk#21 ASC NULLS FIRST], false, 0 (34) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#19] +Right keys [1]: [customer_sk#21] Join condition: None (35) Project [codegen id : 12] @@ -208,84 +208,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (36) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_county#22] +Output [2]: [ca_address_sk#23, ca_county#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 11] -Input [2]: [ca_address_sk#21, ca_county#22] +Input [2]: [ca_address_sk#23, ca_county#24] (38) Filter [codegen id : 11] -Input [2]: [ca_address_sk#21, ca_county#22] -Condition : (ca_county#22 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#21)) +Input [2]: [ca_address_sk#23, ca_county#24] +Condition : (ca_county#24 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#23)) (39) Project [codegen id : 11] -Output [1]: [ca_address_sk#21] -Input [2]: [ca_address_sk#21, ca_county#22] +Output [1]: [ca_address_sk#23] +Input [2]: [ca_address_sk#23, ca_county#24] (40) BroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [ca_address_sk#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#23] Join condition: None (42) Project [codegen id : 12] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#23] (43) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (44) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (45) ColumnarToRow -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] (46) Filter -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Condition : isnotnull(cd_demo_sk#25) +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Condition : isnotnull(cd_demo_sk#27) (47) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#27] Join condition: None (48) Project [codegen id : 13] -Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] (49) HashAggregate [codegen id : 13] -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#34] -Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#35] +Aggregate Attributes [1]: [count#36] +Results [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] (50) Exchange -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#35] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] +Arguments: hashpartitioning(cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, 5), ENSURE_REQUIREMENTS, [id=#38] (51) HashAggregate [codegen id : 14] -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#35] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] +Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#37] -Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#37 AS cnt1#38, cd_purchase_estimate#29, count(1)#37 AS cnt2#39, cd_credit_rating#30, count(1)#37 AS cnt3#40, cd_dep_count#31, count(1)#37 AS cnt4#41, cd_dep_employed_count#32, count(1)#37 AS cnt5#42, cd_dep_college_count#33, count(1)#37 AS cnt6#43] +Aggregate Attributes [1]: [count(1)#39] +Results [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, count(1)#39 AS cnt1#40, cd_purchase_estimate#31, count(1)#39 AS cnt2#41, cd_credit_rating#32, count(1)#39 AS cnt3#42, cd_dep_count#33, count(1)#39 AS cnt4#43, cd_dep_employed_count#34, count(1)#39 AS cnt5#44, cd_dep_college_count#35, count(1)#39 AS cnt6#45] (52) TakeOrderedAndProject -Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#38, cd_purchase_estimate#29, cnt2#39, cd_credit_rating#30, cnt3#40, cd_dep_count#31, cnt4#41, cd_dep_employed_count#32, cnt5#42, cd_dep_college_count#33, cnt6#43] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#38, cd_purchase_estimate#29, cnt2#39, cd_credit_rating#30, cnt3#40, cd_dep_count#31, cnt4#41, cd_dep_employed_count#32, cnt5#42, cd_dep_college_count#33, cnt6#43] +Input [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#40, cd_purchase_estimate#31, cnt2#41, cd_credit_rating#32, cnt3#42, cd_dep_count#33, cnt4#43, cd_dep_employed_count#34, cnt5#44, cd_dep_college_count#35, cnt6#45] +Arguments: 100, [cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_education_status#30 ASC NULLS FIRST, cd_purchase_estimate#31 ASC NULLS FIRST, cd_credit_rating#32 ASC NULLS FIRST, cd_dep_count#33 ASC NULLS FIRST, cd_dep_employed_count#34 ASC NULLS FIRST, cd_dep_college_count#35 ASC NULLS FIRST], [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#40, cd_purchase_estimate#31, cnt2#41, cd_credit_rating#32, cnt3#42, cd_dep_count#33, cnt4#43, cd_dep_employed_count#34, cnt5#44, cd_dep_college_count#35, cnt6#45] ===== Subqueries ===== @@ -298,6 +298,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#19 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt index 6f64579d09c78..ff55b1c511456 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt @@ -125,22 +125,22 @@ Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] Condition : isnotnull(cs_ship_customer_sk#12) (17) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] -Output [1]: [cs_ship_customer_sk#12 AS customer_sk#14] -Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#7] +Output [1]: [cs_ship_customer_sk#12 AS customer_sk#15] +Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] (20) Union (21) BroadcastExchange Input [1]: [customer_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (22) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] @@ -148,39 +148,39 @@ Right keys [1]: [customer_sk#11] Join condition: None (23) Scan parquet default.store_sales -Output [2]: [ss_customer_sk#16, ss_sold_date_sk#17] +Output [2]: [ss_customer_sk#17, ss_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#17), dynamicpruningexpression(ss_sold_date_sk#17 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [2]: [ss_customer_sk#16, ss_sold_date_sk#17] +Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] (25) Filter [codegen id : 6] -Input [2]: [ss_customer_sk#16, ss_sold_date_sk#17] -Condition : isnotnull(ss_customer_sk#16) +Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] +Condition : isnotnull(ss_customer_sk#17) (26) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#19] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#17] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [ss_sold_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (28) Project [codegen id : 6] -Output [1]: [ss_customer_sk#16 AS customer_sk#18] -Input [3]: [ss_customer_sk#16, ss_sold_date_sk#17, d_date_sk#7] +Output [1]: [ss_customer_sk#17 AS customer_sk#20] +Input [3]: [ss_customer_sk#17, ss_sold_date_sk#18, d_date_sk#19] (29) BroadcastExchange -Input [1]: [customer_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [customer_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (30) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#18] +Right keys [1]: [customer_sk#20] Join condition: None (31) Project [codegen id : 9] @@ -188,84 +188,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (32) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] +Output [2]: [ca_address_sk#22, ca_county#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (33) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_county#21] +Input [2]: [ca_address_sk#22, ca_county#23] (34) Filter [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) +Input [2]: [ca_address_sk#22, ca_county#23] +Condition : (ca_county#23 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#22)) (35) Project [codegen id : 7] -Output [1]: [ca_address_sk#20] -Input [2]: [ca_address_sk#20, ca_county#21] +Output [1]: [ca_address_sk#22] +Input [2]: [ca_address_sk#22, ca_county#23] (36) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [ca_address_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#22] Join condition: None (38) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#22] (39) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (41) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Condition : isnotnull(cd_demo_sk#23) +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Condition : isnotnull(cd_demo_sk#25) (42) BroadcastExchange -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (43) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#25] Join condition: None (44) Project [codegen id : 9] -Output [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (45) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] +Aggregate Attributes [1]: [count#35] +Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] (46) Exchange -Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] -Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] (47) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] -Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#36] -Results [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#36 AS cnt1#37, cd_purchase_estimate#27, count(1)#36 AS cnt2#38, cd_credit_rating#28, count(1)#36 AS cnt3#39, cd_dep_count#29, count(1)#36 AS cnt4#40, cd_dep_employed_count#30, count(1)#36 AS cnt5#41, cd_dep_college_count#31, count(1)#36 AS cnt6#42] +Aggregate Attributes [1]: [count(1)#38] +Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] (48) TakeOrderedAndProject -Input [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] -Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] +Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] +Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] ===== Subqueries ===== @@ -278,6 +278,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 14 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#17 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#6 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 baed833e42a47..8f7cbc7d0ffbc 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -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)] +PartitionFilters: [isnotnull(ss_sold_date_sk#61), (ss_sold_date_sk#61 >= 2451545), (ss_sold_date_sk#61 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -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] +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] (35) Filter [codegen id : 11] -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)) +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#62] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#61] +Right keys [1]: [d_date_sk#62] Join condition: None (38) Project [codegen id : 11] -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] +Output [7]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#62] (39) Scan parquet default.store -Output [2]: [s_store_sk#18, s_state#19] +Output [2]: [s_store_sk#63, s_state#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_state, [TN,AL,SD]), IsNotNull(s_store_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 8] -Input [2]: [s_store_sk#18, s_state#19] +Input [2]: [s_store_sk#63, s_state#64] (41) Filter [codegen id : 8] -Input [2]: [s_store_sk#18, s_state#19] -Condition : (s_state#19 IN (TN,AL,SD) AND isnotnull(s_store_sk#18)) +Input [2]: [s_store_sk#63, s_state#64] +Condition : (s_state#64 IN (TN,AL,SD) AND isnotnull(s_store_sk#63)) (42) Project [codegen id : 8] -Output [1]: [s_store_sk#18] -Input [2]: [s_store_sk#18, s_state#19] +Output [1]: [s_store_sk#63] +Input [2]: [s_store_sk#63, s_state#64] (43) BroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [s_store_sk#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] (44) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#56] +Right keys [1]: [s_store_sk#63] Join condition: None (45) Project [codegen id : 11] -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#18] +Output [6]: [ss_item_sk#54, ss_cdemo_sk#55, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, s_store_sk#63] (46) ReusedExchange [Reuses operator id: 15] -Output [1]: [cd_demo_sk#13] +Output [1]: [cd_demo_sk#66] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#13] +Left keys [1]: [ss_cdemo_sk#55] +Right keys [1]: [cd_demo_sk#66] Join condition: None (48) Project [codegen id : 11] -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#13] +Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [7]: [ss_item_sk#54, ss_cdemo_sk#55, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, cd_demo_sk#66] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#21, i_item_id#22] +Output [2]: [i_item_sk#67, i_item_id#68] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#67] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] +Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] +Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#22] +Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#68] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] +Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] (53) Exchange -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [1]: [i_item_id#22] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Keys [1]: [i_item_id#68] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] -Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] +Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] +Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] (55) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -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)] +PartitionFilters: [isnotnull(ss_sold_date_sk#103), (ss_sold_date_sk#103 >= 2451545), (ss_sold_date_sk#103 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 17] -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] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] (57) Filter [codegen id : 17] -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)) +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#104] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#103] +Right keys [1]: [d_date_sk#104] Join condition: None (60) Project [codegen id : 17] -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] +Output [7]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#104] (61) ReusedExchange [Reuses operator id: 43] -Output [1]: [s_store_sk#18] +Output [1]: [s_store_sk#105] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#98] +Right keys [1]: [s_store_sk#105] Join condition: None (63) Project [codegen id : 17] -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#18] +Output [6]: [ss_item_sk#96, ss_cdemo_sk#97, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#105] (64) ReusedExchange [Reuses operator id: 15] -Output [1]: [cd_demo_sk#13] +Output [1]: [cd_demo_sk#106] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#13] +Left keys [1]: [ss_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#106] Join condition: None (66) Project [codegen id : 17] -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#13] +Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [7]: [ss_item_sk#96, ss_cdemo_sk#97, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, cd_demo_sk#106] (67) Scan parquet default.item -Output [1]: [i_item_sk#21] +Output [1]: [i_item_sk#107] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#21] +Input [1]: [i_item_sk#107] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#21] -Condition : isnotnull(i_item_sk#21) +Input [1]: [i_item_sk#107] +Condition : isnotnull(i_item_sk#107) (70) BroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] +Input [1]: [i_item_sk#107] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#96] +Right keys [1]: [i_item_sk#107] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] +Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] +Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] -Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] +Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] (74) Exchange -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] -Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] +Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] +Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 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 735f763ae5d20..bfb9e8384817d 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -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)] +PartitionFilters: [isnotnull(ss_sold_date_sk#61), (ss_sold_date_sk#61 >= 2451545), (ss_sold_date_sk#61 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -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] +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] (35) Filter [codegen id : 11] -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)) +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#62] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#55] +Right keys [1]: [cd_demo_sk#62] Join condition: None (38) Project [codegen id : 11] -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] +Output [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, cd_demo_sk#62] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#63] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#61] +Right keys [1]: [d_date_sk#63] Join condition: None (41) Project [codegen id : 11] -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] +Output [6]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [8]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#63] (42) Scan parquet default.store -Output [2]: [s_store_sk#18, s_state#19] +Output [2]: [s_store_sk#64, s_state#65] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_state, [TN,AL,SD]), IsNotNull(s_store_sk)] ReadSchema: struct (43) ColumnarToRow [codegen id : 9] -Input [2]: [s_store_sk#18, s_state#19] +Input [2]: [s_store_sk#64, s_state#65] (44) Filter [codegen id : 9] -Input [2]: [s_store_sk#18, s_state#19] -Condition : (s_state#19 IN (TN,AL,SD) AND isnotnull(s_store_sk#18)) +Input [2]: [s_store_sk#64, s_state#65] +Condition : (s_state#65 IN (TN,AL,SD) AND isnotnull(s_store_sk#64)) (45) Project [codegen id : 9] -Output [1]: [s_store_sk#18] -Input [2]: [s_store_sk#18, s_state#19] +Output [1]: [s_store_sk#64] +Input [2]: [s_store_sk#64, s_state#65] (46) BroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [s_store_sk#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#66] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#56] +Right keys [1]: [s_store_sk#64] Join condition: None (48) Project [codegen id : 11] -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#18] +Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, s_store_sk#64] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#21, i_item_id#22] +Output [2]: [i_item_sk#67, i_item_id#68] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#67] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] +Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] +Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#22] +Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#68] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] +Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] (53) Exchange -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [1]: [i_item_id#22] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Keys [1]: [i_item_id#68] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] -Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] +Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] +Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] (55) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -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)] +PartitionFilters: [isnotnull(ss_sold_date_sk#103), (ss_sold_date_sk#103 >= 2451545), (ss_sold_date_sk#103 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 17] -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] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] (57) Filter [codegen id : 17] -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)) +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#104] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#104] Join condition: None (60) Project [codegen id : 17] -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] +Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] (61) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#105] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#103] +Right keys [1]: [d_date_sk#105] Join condition: None (63) Project [codegen id : 17] -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] +Output [6]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#105] (64) ReusedExchange [Reuses operator id: 46] -Output [1]: [s_store_sk#18] +Output [1]: [s_store_sk#106] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#98] +Right keys [1]: [s_store_sk#106] Join condition: None (66) Project [codegen id : 17] -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#18] +Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#106] (67) Scan parquet default.item -Output [1]: [i_item_sk#21] +Output [1]: [i_item_sk#107] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#21] +Input [1]: [i_item_sk#107] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#21] -Condition : isnotnull(i_item_sk#21) +Input [1]: [i_item_sk#107] +Condition : isnotnull(i_item_sk#107) (70) BroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] +Input [1]: [i_item_sk#107] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#96] +Right keys [1]: [i_item_sk#107] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] +Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] +Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] -Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] +Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] (74) Exchange -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] -Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] +Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] +Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#15] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt index 4eb212c336669..e62003f82faec 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt @@ -261,24 +261,24 @@ Input [7]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_add Arguments: [c_current_addr_sk#36 ASC NULLS FIRST], false, 0 (47) ReusedExchange [Reuses operator id: 30] -Output [2]: [ca_address_sk#22, ca_city#23] +Output [2]: [ca_address_sk#41, ca_city#42] (48) Sort [codegen id : 15] -Input [2]: [ca_address_sk#22, ca_city#23] -Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#41, ca_city#42] +Arguments: [ca_address_sk#41 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#36] -Right keys [1]: [ca_address_sk#22] -Join condition: NOT (ca_city#23 = bought_city#31) +Right keys [1]: [ca_address_sk#41] +Join condition: NOT (ca_city#42 = bought_city#31) (50) Project [codegen id : 16] -Output [7]: [c_last_name#38, c_first_name#37, ca_city#23, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#36, c_first_name#37, c_last_name#38, ca_address_sk#22, ca_city#23] +Output [7]: [c_last_name#38, c_first_name#37, ca_city#42, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#36, c_first_name#37, c_last_name#38, ca_address_sk#41, ca_city#42] (51) TakeOrderedAndProject -Input [7]: [c_last_name#38, c_first_name#37, ca_city#23, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Arguments: 100, [c_last_name#38 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, ca_city#23 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#38, c_first_name#37, ca_city#23, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [7]: [c_last_name#38, c_first_name#37, ca_city#42, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Arguments: 100, [c_last_name#38 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, ca_city#42 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#38, c_first_name#37, ca_city#42, bought_city#31, ss_ticket_number#5, amt#32, profit#33] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt index 951558ca1b130..a00e2eeac49b7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt @@ -225,20 +225,20 @@ Output [7]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_ad Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#31, amt#32, profit#33, c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#21, ca_city#22] +Output [2]: [ca_address_sk#39, ca_city#40] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#21] -Join condition: NOT (ca_city#22 = bought_city#31) +Right keys [1]: [ca_address_sk#39] +Join condition: NOT (ca_city#40 = bought_city#31) (42) Project [codegen id : 8] -Output [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#21, ca_city#22] +Output [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#39, ca_city#40] (43) TakeOrderedAndProject -Input [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#22 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#40 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] ===== Subqueries ===== 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 4cdcd9685acbe..d20f7cf4a2857 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#1, ss_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3)] +PartitionFilters: [isnotnull(ss_sold_date_sk#56)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 6] -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] (28) Filter [codegen id : 6] -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] +Condition : isnotnull(ss_store_sk#54) (29) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] +Output [3]: [d_date_sk#57, d_week_seq#58, d_day_name#59] (30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#4] +Left keys [1]: [ss_sold_date_sk#56] +Right keys [1]: [d_date_sk#57] Join condition: None (31) Project [codegen id : 6] -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] +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] (32) HashAggregate [codegen id : 6] -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] +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] (33) Exchange -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] +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] (34) HashAggregate [codegen id : 9] -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))#67, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72] -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))#67,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72,17,2) AS sat_sales#36] +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] (35) Scan parquet default.store -Output [2]: [s_store_sk#37, s_store_id#38] +Output [2]: [s_store_sk#79, s_store_id#80] 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#37, s_store_id#38] +Input [2]: [s_store_sk#79, s_store_id#80] (37) Filter [codegen id : 7] -Input [2]: [s_store_sk#37, s_store_id#38] -Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) +Input [2]: [s_store_sk#79, s_store_id#80] +Condition : (isnotnull(s_store_sk#79) AND isnotnull(s_store_id#80)) (38) BroadcastExchange -Input [2]: [s_store_sk#37, s_store_id#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [s_store_sk#79, s_store_id#80] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] (39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#37] +Left keys [1]: [ss_store_sk#54] +Right keys [1]: [s_store_sk#79] Join condition: None (40) Project [codegen id : 9] -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#38] -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#37, s_store_id#38] +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] (41) Scan parquet default.date_dim -Output [2]: [d_month_seq#74, d_week_seq#75] +Output [2]: [d_month_seq#82, d_week_seq#83] 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#74, d_week_seq#75] +Input [2]: [d_month_seq#82, d_week_seq#83] (43) Filter [codegen id : 8] -Input [2]: [d_month_seq#74, d_week_seq#75] -Condition : (((isnotnull(d_month_seq#74) AND (d_month_seq#74 >= 1197)) AND (d_month_seq#74 <= 1208)) AND isnotnull(d_week_seq#75)) +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)) (44) Project [codegen id : 8] -Output [1]: [d_week_seq#75] -Input [2]: [d_month_seq#74, d_week_seq#75] +Output [1]: [d_week_seq#83] +Input [2]: [d_month_seq#82, d_week_seq#83] (45) BroadcastExchange -Input [1]: [d_week_seq#75] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] +Input [1]: [d_week_seq#83] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#84] (46) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#75] +Left keys [1]: [d_week_seq#58] +Right keys [1]: [d_week_seq#83] Join condition: None (47) Project [codegen id : 9] -Output [8]: [d_week_seq#5 AS d_week_seq2#77, s_store_id#38 AS s_store_id2#78, sun_sales#30 AS sun_sales2#79, mon_sales#31 AS mon_sales2#80, wed_sales#33 AS wed_sales2#81, thu_sales#34 AS thu_sales2#82, fri_sales#35 AS fri_sales2#83, sat_sales#36 AS sat_sales2#84] -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#38, d_week_seq#75] +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] (48) BroadcastExchange -Input [8]: [d_week_seq2#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#85] +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] (49) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#78, (d_week_seq2#77 - 52)] +Right keys [2]: [s_store_id2#86, (d_week_seq2#85 - 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#79)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#86, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#80)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#87, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#88, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#81)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#89, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#82)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#90, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#83)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#91, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#84)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#92] -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#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] +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] (51) TakeOrderedAndProject -Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] -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)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] +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] 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 4cdcd9685acbe..d20f7cf4a2857 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#1, ss_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3)] +PartitionFilters: [isnotnull(ss_sold_date_sk#56)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 6] -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] (28) Filter [codegen id : 6] -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] +Condition : isnotnull(ss_store_sk#54) (29) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] +Output [3]: [d_date_sk#57, d_week_seq#58, d_day_name#59] (30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#4] +Left keys [1]: [ss_sold_date_sk#56] +Right keys [1]: [d_date_sk#57] Join condition: None (31) Project [codegen id : 6] -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] +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] (32) HashAggregate [codegen id : 6] -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] +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] (33) Exchange -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] +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] (34) HashAggregate [codegen id : 9] -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))#67, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72] -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))#67,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72,17,2) AS sat_sales#36] +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] (35) Scan parquet default.store -Output [2]: [s_store_sk#37, s_store_id#38] +Output [2]: [s_store_sk#79, s_store_id#80] 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#37, s_store_id#38] +Input [2]: [s_store_sk#79, s_store_id#80] (37) Filter [codegen id : 7] -Input [2]: [s_store_sk#37, s_store_id#38] -Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) +Input [2]: [s_store_sk#79, s_store_id#80] +Condition : (isnotnull(s_store_sk#79) AND isnotnull(s_store_id#80)) (38) BroadcastExchange -Input [2]: [s_store_sk#37, s_store_id#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [s_store_sk#79, s_store_id#80] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] (39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#37] +Left keys [1]: [ss_store_sk#54] +Right keys [1]: [s_store_sk#79] Join condition: None (40) Project [codegen id : 9] -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#38] -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#37, s_store_id#38] +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] (41) Scan parquet default.date_dim -Output [2]: [d_month_seq#74, d_week_seq#75] +Output [2]: [d_month_seq#82, d_week_seq#83] 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#74, d_week_seq#75] +Input [2]: [d_month_seq#82, d_week_seq#83] (43) Filter [codegen id : 8] -Input [2]: [d_month_seq#74, d_week_seq#75] -Condition : (((isnotnull(d_month_seq#74) AND (d_month_seq#74 >= 1197)) AND (d_month_seq#74 <= 1208)) AND isnotnull(d_week_seq#75)) +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)) (44) Project [codegen id : 8] -Output [1]: [d_week_seq#75] -Input [2]: [d_month_seq#74, d_week_seq#75] +Output [1]: [d_week_seq#83] +Input [2]: [d_month_seq#82, d_week_seq#83] (45) BroadcastExchange -Input [1]: [d_week_seq#75] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] +Input [1]: [d_week_seq#83] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#84] (46) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#75] +Left keys [1]: [d_week_seq#58] +Right keys [1]: [d_week_seq#83] Join condition: None (47) Project [codegen id : 9] -Output [8]: [d_week_seq#5 AS d_week_seq2#77, s_store_id#38 AS s_store_id2#78, sun_sales#30 AS sun_sales2#79, mon_sales#31 AS mon_sales2#80, wed_sales#33 AS wed_sales2#81, thu_sales#34 AS thu_sales2#82, fri_sales#35 AS fri_sales2#83, sat_sales#36 AS sat_sales2#84] -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#38, d_week_seq#75] +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] (48) BroadcastExchange -Input [8]: [d_week_seq2#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#85] +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] (49) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#78, (d_week_seq2#77 - 52)] +Right keys [2]: [s_store_id2#86, (d_week_seq2#85 - 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#79)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#86, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#80)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#87, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#88, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#81)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#89, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#82)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#90, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#83)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#91, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#84)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#92] -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#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] +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] (51) TakeOrderedAndProject -Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] -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)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] +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] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt index 741a644c4fdbc..240f579025038 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt @@ -127,123 +127,123 @@ Input [4]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17 Condition : isnotnull(ss_store_sk#15) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#18] (19) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#17] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#18] Join condition: None (20) Project [codegen id : 4] Output [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] -Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#6] +Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#18] (21) HashAggregate [codegen id : 4] Input [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] +Aggregate Attributes [1]: [sum#19] +Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] (22) Exchange -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] -Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] +Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#21] (23) HashAggregate [codegen id : 5] -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#21] -Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#21,17,2) AS revenue#22] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#22] +Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#22,17,2) AS revenue#23] (24) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#15, revenue#22] +Input [2]: [ss_store_sk#15, revenue#23] Keys [1]: [ss_store_sk#15] -Functions [1]: [partial_avg(revenue#22)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ss_store_sk#15, sum#25, count#26] +Functions [1]: [partial_avg(revenue#23)] +Aggregate Attributes [2]: [sum#24, count#25] +Results [3]: [ss_store_sk#15, sum#26, count#27] (25) Exchange -Input [3]: [ss_store_sk#15, sum#25, count#26] -Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ss_store_sk#15, sum#26, count#27] +Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#28] (26) HashAggregate [codegen id : 6] -Input [3]: [ss_store_sk#15, sum#25, count#26] +Input [3]: [ss_store_sk#15, sum#26, count#27] Keys [1]: [ss_store_sk#15] -Functions [1]: [avg(revenue#22)] -Aggregate Attributes [1]: [avg(revenue#22)#28] -Results [2]: [ss_store_sk#15, avg(revenue#22)#28 AS ave#29] +Functions [1]: [avg(revenue#23)] +Aggregate Attributes [1]: [avg(revenue#23)#29] +Results [2]: [ss_store_sk#15, avg(revenue#23)#29 AS ave#30] (27) BroadcastExchange -Input [2]: [ss_store_sk#15, ave#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] +Input [2]: [ss_store_sk#15, ave#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (28) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_store_sk#2] Right keys [1]: [ss_store_sk#15] -Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#29)), DecimalType(23,7), true)) +Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#30)), DecimalType(23,7), true)) (29) Project [codegen id : 7] Output [3]: [ss_store_sk#2, ss_item_sk#1, revenue#13] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#29] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#30] (30) BroadcastExchange Input [3]: [ss_store_sk#2, ss_item_sk#1, revenue#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] (31) Scan parquet default.store -Output [2]: [s_store_sk#32, s_store_name#33] +Output [2]: [s_store_sk#33, s_store_name#34] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (32) ColumnarToRow -Input [2]: [s_store_sk#32, s_store_name#33] +Input [2]: [s_store_sk#33, s_store_name#34] (33) Filter -Input [2]: [s_store_sk#32, s_store_name#33] -Condition : isnotnull(s_store_sk#32) +Input [2]: [s_store_sk#33, s_store_name#34] +Condition : isnotnull(s_store_sk#33) (34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#32] +Right keys [1]: [s_store_sk#33] Join condition: None (35) Project [codegen id : 8] -Output [3]: [ss_item_sk#1, revenue#13, s_store_name#33] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#32, s_store_name#33] +Output [3]: [ss_item_sk#1, revenue#13, s_store_name#34] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#33, s_store_name#34] (36) BroadcastExchange -Input [3]: [ss_item_sk#1, revenue#13, s_store_name#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] +Input [3]: [ss_item_sk#1, revenue#13, s_store_name#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] (37) Scan parquet default.item -Output [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Output [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (38) ColumnarToRow -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] (39) Filter -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Condition : isnotnull(i_item_sk#35) +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Condition : isnotnull(i_item_sk#36) (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#35] +Right keys [1]: [i_item_sk#36] Join condition: None (41) Project [codegen id : 9] -Output [6]: [s_store_name#33, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Input [8]: [ss_item_sk#1, revenue#13, s_store_name#33, i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Output [6]: [s_store_name#34, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Input [8]: [ss_item_sk#1, revenue#13, s_store_name#34, i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] (42) TakeOrderedAndProject -Input [6]: [s_store_name#33, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Arguments: 100, [s_store_name#33 ASC NULLS FIRST, i_item_desc#36 ASC NULLS FIRST], [s_store_name#33, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [6]: [s_store_name#34, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Arguments: 100, [s_store_name#34 ASC NULLS FIRST, i_item_desc#37 ASC NULLS FIRST], [s_store_name#34, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt index 0851190e54fe0..43ef320ed235f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt @@ -181,65 +181,65 @@ Input [4]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26 Condition : isnotnull(ss_store_sk#24) (30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#27] (31) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#27] Join condition: None (32) Project [codegen id : 6] Output [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] -Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#8] +Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#27] (33) HashAggregate [codegen id : 6] Input [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#27] -Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] +Aggregate Attributes [1]: [sum#28] +Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] (34) Exchange -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] -Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] +Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#30] (35) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#30] -Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#30,17,2) AS revenue#31] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] +Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS revenue#32] (36) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#24, revenue#31] +Input [2]: [ss_store_sk#24, revenue#32] Keys [1]: [ss_store_sk#24] -Functions [1]: [partial_avg(revenue#31)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [3]: [ss_store_sk#24, sum#34, count#35] +Functions [1]: [partial_avg(revenue#32)] +Aggregate Attributes [2]: [sum#33, count#34] +Results [3]: [ss_store_sk#24, sum#35, count#36] (37) Exchange -Input [3]: [ss_store_sk#24, sum#34, count#35] -Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ss_store_sk#24, sum#35, count#36] +Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#37] (38) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#24, sum#34, count#35] +Input [3]: [ss_store_sk#24, sum#35, count#36] Keys [1]: [ss_store_sk#24] -Functions [1]: [avg(revenue#31)] -Aggregate Attributes [1]: [avg(revenue#31)#37] -Results [2]: [ss_store_sk#24, avg(revenue#31)#37 AS ave#38] +Functions [1]: [avg(revenue#32)] +Aggregate Attributes [1]: [avg(revenue#32)#38] +Results [2]: [ss_store_sk#24, avg(revenue#32)#38 AS ave#39] (39) BroadcastExchange -Input [2]: [ss_store_sk#24, ave#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Input [2]: [ss_store_sk#24, ave#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#24] -Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#38)), DecimalType(23,7), true)) +Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#39)), DecimalType(23,7), true)) (41) Project [codegen id : 9] Output [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#38] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#39] (42) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt index 10ac72a6b44de..0b96b76a0ffc4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt @@ -232,31 +232,31 @@ Input [8]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37 Arguments: HashedRelationBroadcastMode(List(cast(input[5, int, true] as bigint)),false), [id=#44] (41) Scan parquet default.customer_address -Output [2]: [ca_address_sk#23, ca_city#24] +Output [2]: [ca_address_sk#45, ca_city#46] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] ReadSchema: struct (42) ColumnarToRow -Input [2]: [ca_address_sk#23, ca_city#24] +Input [2]: [ca_address_sk#45, ca_city#46] (43) Filter -Input [2]: [ca_address_sk#23, ca_city#24] -Condition : (isnotnull(ca_address_sk#23) AND isnotnull(ca_city#24)) +Input [2]: [ca_address_sk#45, ca_city#46] +Condition : (isnotnull(ca_address_sk#45) AND isnotnull(ca_city#46)) (44) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#41] -Right keys [1]: [ca_address_sk#23] -Join condition: NOT (ca_city#24 = bought_city#35) +Right keys [1]: [ca_address_sk#45] +Join condition: NOT (ca_city#46 = bought_city#35) (45) Project [codegen id : 8] -Output [8]: [c_last_name#43, c_first_name#42, ca_city#24, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#41, c_first_name#42, c_last_name#43, ca_address_sk#23, ca_city#24] +Output [8]: [c_last_name#43, c_first_name#42, ca_city#46, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#41, c_first_name#42, c_last_name#43, ca_address_sk#45, ca_city#46] (46) TakeOrderedAndProject -Input [8]: [c_last_name#43, c_first_name#42, ca_city#24, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Arguments: 100, [c_last_name#43 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#43, c_first_name#42, ca_city#24, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [8]: [c_last_name#43, c_first_name#42, ca_city#46, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Arguments: 100, [c_last_name#43 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#43, c_first_name#42, ca_city#46, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt index 81faa98299f82..f50329db7e05e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt @@ -225,20 +225,20 @@ Output [8]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#3 Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_customer_sk#39, c_current_addr_sk#40, c_first_name#41, c_last_name#42] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#22, ca_city#23] +Output [2]: [ca_address_sk#44, ca_city#45] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#40] -Right keys [1]: [ca_address_sk#22] -Join condition: NOT (ca_city#23 = bought_city#35) +Right keys [1]: [ca_address_sk#44] +Join condition: NOT (ca_city#45 = bought_city#35) (42) Project [codegen id : 8] -Output [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#22, ca_city#23] +Output [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#44, ca_city#45] (43) TakeOrderedAndProject -Input [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] ===== Subqueries ===== 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 9c68e55238aee..a950e7b9e8639 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 @@ -117,158 +117,158 @@ Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) (15) Scan parquet default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Output [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#5)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct (16) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] (17) Filter [codegen id : 4] -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) +Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] +Condition : isnotnull(sr_store_sk#17) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#20] (19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#4] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [sr_returned_date_sk#19] +Right keys [1]: [cast(d_date_sk#20 as bigint)] Join condition: None (20) Project [codegen id : 4] -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] +Output [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] +Input [5]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19, d_date_sk#20] (21) HashAggregate [codegen id : 4] -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#16] -Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] +Keys [2]: [sr_customer_sk#16, sr_store_sk#17] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#18))] +Aggregate Attributes [1]: [sum#21] +Results [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] (22) Exchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] +Arguments: hashpartitioning(sr_customer_sk#16, sr_store_sk#17, 5), ENSURE_REQUIREMENTS, [id=#23] (23) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] -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))#19] -Results [2]: [sr_store_sk#2 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#19,17,2) AS ctr_total_return#15] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] +Keys [2]: [sr_customer_sk#16, sr_store_sk#17] +Functions [1]: [sum(UnscaledValue(sr_return_amt#18))] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#18))#24] +Results [2]: [sr_store_sk#17 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#18))#24,17,2) AS ctr_total_return#15] (24) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#14, ctr_total_return#15] Keys [1]: [ctr_store_sk#14] Functions [1]: [partial_avg(ctr_total_return#15)] -Aggregate Attributes [2]: [sum#20, count#21] -Results [3]: [ctr_store_sk#14, sum#22, count#23] +Aggregate Attributes [2]: [sum#25, count#26] +Results [3]: [ctr_store_sk#14, sum#27, count#28] (25) Exchange -Input [3]: [ctr_store_sk#14, sum#22, count#23] -Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [ctr_store_sk#14, sum#27, count#28] +Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#29] (26) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#14, sum#22, count#23] +Input [3]: [ctr_store_sk#14, sum#27, count#28] Keys [1]: [ctr_store_sk#14] Functions [1]: [avg(ctr_total_return#15)] -Aggregate Attributes [1]: [avg(ctr_total_return#15)#25] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#25) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14 AS ctr_store_sk#14#27] +Aggregate Attributes [1]: [avg(ctr_total_return#15)#30] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14 AS ctr_store_sk#14#32] (27) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#26) +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) (28) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#28] +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] +Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#33] (29) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [ctr_store_sk#14#27] -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#26) +Right keys [1]: [ctr_store_sk#14#32] +Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) (30) Project [codegen id : 8] Output [2]: [ctr_customer_sk#13, ctr_store_sk#14] -Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] +Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] (31) Scan parquet default.store -Output [2]: [s_store_sk#29, s_state#30] +Output [2]: [s_store_sk#34, s_state#35] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#29, s_state#30] +Input [2]: [s_store_sk#34, s_state#35] (33) Filter [codegen id : 7] -Input [2]: [s_store_sk#29, s_state#30] -Condition : ((isnotnull(s_state#30) AND (s_state#30 = TN)) AND isnotnull(s_store_sk#29)) +Input [2]: [s_store_sk#34, s_state#35] +Condition : ((isnotnull(s_state#35) AND (s_state#35 = TN)) AND isnotnull(s_store_sk#34)) (34) Project [codegen id : 7] -Output [1]: [s_store_sk#29] -Input [2]: [s_store_sk#29, s_state#30] +Output [1]: [s_store_sk#34] +Input [2]: [s_store_sk#34, s_state#35] (35) BroadcastExchange -Input [1]: [s_store_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [1]: [s_store_sk#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (36) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [cast(s_store_sk#29 as bigint)] +Right keys [1]: [cast(s_store_sk#34 as bigint)] Join condition: None (37) Project [codegen id : 8] Output [1]: [ctr_customer_sk#13] -Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#29] +Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#34] (38) Exchange Input [1]: [ctr_customer_sk#13] -Arguments: hashpartitioning(ctr_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#32] +Arguments: hashpartitioning(ctr_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#37] (39) Sort [codegen id : 9] Input [1]: [ctr_customer_sk#13] Arguments: [ctr_customer_sk#13 ASC NULLS FIRST], false, 0 (40) Scan parquet default.customer -Output [2]: [c_customer_sk#33, c_customer_id#34] +Output [2]: [c_customer_sk#38, c_customer_id#39] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [2]: [c_customer_sk#33, c_customer_id#34] +Input [2]: [c_customer_sk#38, c_customer_id#39] (42) Filter [codegen id : 10] -Input [2]: [c_customer_sk#33, c_customer_id#34] -Condition : isnotnull(c_customer_sk#33) +Input [2]: [c_customer_sk#38, c_customer_id#39] +Condition : isnotnull(c_customer_sk#38) (43) Exchange -Input [2]: [c_customer_sk#33, c_customer_id#34] -Arguments: hashpartitioning(cast(c_customer_sk#33 as bigint), 5), ENSURE_REQUIREMENTS, [id=#35] +Input [2]: [c_customer_sk#38, c_customer_id#39] +Arguments: hashpartitioning(cast(c_customer_sk#38 as bigint), 5), ENSURE_REQUIREMENTS, [id=#40] (44) Sort [codegen id : 11] -Input [2]: [c_customer_sk#33, c_customer_id#34] -Arguments: [cast(c_customer_sk#33 as bigint) ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#38, c_customer_id#39] +Arguments: [cast(c_customer_sk#38 as bigint) ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 12] Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [cast(c_customer_sk#33 as bigint)] +Right keys [1]: [cast(c_customer_sk#38 as bigint)] Join condition: None (46) Project [codegen id : 12] -Output [1]: [c_customer_id#34] -Input [3]: [ctr_customer_sk#13, c_customer_sk#33, c_customer_id#34] +Output [1]: [c_customer_id#39] +Input [3]: [ctr_customer_sk#13, c_customer_sk#38, c_customer_id#39] (47) TakeOrderedAndProject -Input [1]: [c_customer_id#34] -Arguments: 100, [c_customer_id#34 ASC NULLS FIRST], [c_customer_id#34] +Input [1]: [c_customer_id#39] +Arguments: 100, [c_customer_id#39 ASC NULLS FIRST], [c_customer_id#39] ===== Subqueries ===== @@ -279,6 +279,6 @@ ReusedExchange (48) (48) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 15 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 15 Hosting Expression = sr_returned_date_sk#19 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 4487f04c8293d..015228686d39b 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 @@ -114,146 +114,146 @@ Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) (15) Scan parquet default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Output [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#5)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct (16) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] (17) Filter [codegen id : 4] -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) +Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] +Condition : isnotnull(sr_store_sk#17) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#20] (19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#4] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [sr_returned_date_sk#19] +Right keys [1]: [cast(d_date_sk#20 as bigint)] Join condition: None (20) Project [codegen id : 4] -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] +Output [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] +Input [5]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19, d_date_sk#20] (21) HashAggregate [codegen id : 4] -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#16] -Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] +Keys [2]: [sr_customer_sk#16, sr_store_sk#17] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#18))] +Aggregate Attributes [1]: [sum#21] +Results [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] (22) Exchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] +Arguments: hashpartitioning(sr_customer_sk#16, sr_store_sk#17, 5), ENSURE_REQUIREMENTS, [id=#23] (23) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] -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))#19] -Results [2]: [sr_store_sk#2 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#19,17,2) AS ctr_total_return#15] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] +Keys [2]: [sr_customer_sk#16, sr_store_sk#17] +Functions [1]: [sum(UnscaledValue(sr_return_amt#18))] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#18))#24] +Results [2]: [sr_store_sk#17 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#18))#24,17,2) AS ctr_total_return#15] (24) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#14, ctr_total_return#15] Keys [1]: [ctr_store_sk#14] Functions [1]: [partial_avg(ctr_total_return#15)] -Aggregate Attributes [2]: [sum#20, count#21] -Results [3]: [ctr_store_sk#14, sum#22, count#23] +Aggregate Attributes [2]: [sum#25, count#26] +Results [3]: [ctr_store_sk#14, sum#27, count#28] (25) Exchange -Input [3]: [ctr_store_sk#14, sum#22, count#23] -Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [ctr_store_sk#14, sum#27, count#28] +Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#29] (26) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#14, sum#22, count#23] +Input [3]: [ctr_store_sk#14, sum#27, count#28] Keys [1]: [ctr_store_sk#14] Functions [1]: [avg(ctr_total_return#15)] -Aggregate Attributes [1]: [avg(ctr_total_return#15)#25] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#25) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14 AS ctr_store_sk#14#27] +Aggregate Attributes [1]: [avg(ctr_total_return#15)#30] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14 AS ctr_store_sk#14#32] (27) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#26) +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) (28) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#28] +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] +Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#33] (29) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [ctr_store_sk#14#27] -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#26) +Right keys [1]: [ctr_store_sk#14#32] +Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) (30) Project [codegen id : 9] Output [2]: [ctr_customer_sk#13, ctr_store_sk#14] -Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] +Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] (31) Scan parquet default.store -Output [2]: [s_store_sk#29, s_state#30] +Output [2]: [s_store_sk#34, s_state#35] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#29, s_state#30] +Input [2]: [s_store_sk#34, s_state#35] (33) Filter [codegen id : 7] -Input [2]: [s_store_sk#29, s_state#30] -Condition : ((isnotnull(s_state#30) AND (s_state#30 = TN)) AND isnotnull(s_store_sk#29)) +Input [2]: [s_store_sk#34, s_state#35] +Condition : ((isnotnull(s_state#35) AND (s_state#35 = TN)) AND isnotnull(s_store_sk#34)) (34) Project [codegen id : 7] -Output [1]: [s_store_sk#29] -Input [2]: [s_store_sk#29, s_state#30] +Output [1]: [s_store_sk#34] +Input [2]: [s_store_sk#34, s_state#35] (35) BroadcastExchange -Input [1]: [s_store_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [1]: [s_store_sk#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [cast(s_store_sk#29 as bigint)] +Right keys [1]: [cast(s_store_sk#34 as bigint)] Join condition: None (37) Project [codegen id : 9] Output [1]: [ctr_customer_sk#13] -Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#29] +Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#34] (38) Scan parquet default.customer -Output [2]: [c_customer_sk#32, c_customer_id#33] +Output [2]: [c_customer_sk#37, c_customer_id#38] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 8] -Input [2]: [c_customer_sk#32, c_customer_id#33] +Input [2]: [c_customer_sk#37, c_customer_id#38] (40) Filter [codegen id : 8] -Input [2]: [c_customer_sk#32, c_customer_id#33] -Condition : isnotnull(c_customer_sk#32) +Input [2]: [c_customer_sk#37, c_customer_id#38] +Condition : isnotnull(c_customer_sk#37) (41) BroadcastExchange -Input [2]: [c_customer_sk#32, c_customer_id#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [2]: [c_customer_sk#37, c_customer_id#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#39] (42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [cast(c_customer_sk#32 as bigint)] +Right keys [1]: [cast(c_customer_sk#37 as bigint)] Join condition: None (43) Project [codegen id : 9] -Output [1]: [c_customer_id#33] -Input [3]: [ctr_customer_sk#13, c_customer_sk#32, c_customer_id#33] +Output [1]: [c_customer_id#38] +Input [3]: [ctr_customer_sk#13, c_customer_sk#37, c_customer_id#38] (44) TakeOrderedAndProject -Input [1]: [c_customer_id#33] -Arguments: 100, [c_customer_id#33 ASC NULLS FIRST], [c_customer_id#33] +Input [1]: [c_customer_id#38] +Arguments: 100, [c_customer_id#38 ASC NULLS FIRST], [c_customer_id#38] ===== Subqueries ===== @@ -264,6 +264,6 @@ ReusedExchange (45) (45) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 15 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 15 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt index c25973f527384..2dd0a13e74a74 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt @@ -143,20 +143,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#17] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#10] +Right keys [1]: [d_date_sk#17] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] -Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] +Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] (23) Exchange Input [1]: [ws_bill_customer_sk#15] -Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] @@ -168,38 +168,38 @@ Right keys [1]: [ws_bill_customer_sk#15] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Output [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#21] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [cs_sold_date_sk#20] +Right keys [1]: [d_date_sk#21] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#18] -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] +Output [1]: [cs_ship_customer_sk#19] +Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] (31) Exchange -Input [1]: [cs_ship_customer_sk#18] -Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [cs_ship_customer_sk#19] +Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#18] -Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#19] +Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#19] Join condition: None (34) Filter [codegen id : 13] @@ -211,96 +211,96 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (36) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_county#22] +Output [2]: [ca_address_sk#23, ca_county#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 12] -Input [2]: [ca_address_sk#21, ca_county#22] +Input [2]: [ca_address_sk#23, ca_county#24] (38) Filter [codegen id : 12] -Input [2]: [ca_address_sk#21, ca_county#22] -Condition : (ca_county#22 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#21)) +Input [2]: [ca_address_sk#23, ca_county#24] +Condition : (ca_county#24 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#23)) (39) Project [codegen id : 12] -Output [1]: [ca_address_sk#21] -Input [2]: [ca_address_sk#21, ca_county#22] +Output [1]: [ca_address_sk#23] +Input [2]: [ca_address_sk#23, ca_county#24] (40) BroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [ca_address_sk#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (41) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#23] Join condition: None (42) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#21] +Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#23] (43) Exchange Input [1]: [c_current_cdemo_sk#4] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#26] (44) Sort [codegen id : 14] Input [1]: [c_current_cdemo_sk#4] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 (45) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (46) ColumnarToRow [codegen id : 15] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] (47) Filter [codegen id : 15] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Condition : isnotnull(cd_demo_sk#25) +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Condition : isnotnull(cd_demo_sk#27) (48) Exchange -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: hashpartitioning(cd_demo_sk#25, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Arguments: hashpartitioning(cd_demo_sk#27, 5), ENSURE_REQUIREMENTS, [id=#36] (49) Sort [codegen id : 16] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: [cd_demo_sk#25 ASC NULLS FIRST], false, 0 +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Arguments: [cd_demo_sk#27 ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 17] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#27] Join condition: None (51) Project [codegen id : 17] -Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] (52) HashAggregate [codegen id : 17] -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#35] -Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Aggregate Attributes [1]: [count#37] +Results [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] (53) Exchange -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] +Arguments: hashpartitioning(cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, 5), ENSURE_REQUIREMENTS, [id=#39] (54) HashAggregate [codegen id : 18] -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] +Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] +Aggregate Attributes [1]: [count(1)#40] +Results [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, count(1)#40 AS cnt1#41, cd_purchase_estimate#31, count(1)#40 AS cnt2#42, cd_credit_rating#32, count(1)#40 AS cnt3#43, cd_dep_count#33, count(1)#40 AS cnt4#44, cd_dep_employed_count#34, count(1)#40 AS cnt5#45, cd_dep_college_count#35, count(1)#40 AS cnt6#46] (55) TakeOrderedAndProject -Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] +Input [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#41, cd_purchase_estimate#31, cnt2#42, cd_credit_rating#32, cnt3#43, cd_dep_count#33, cnt4#44, cd_dep_employed_count#34, cnt5#45, cd_dep_college_count#35, cnt6#46] +Arguments: 100, [cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_education_status#30 ASC NULLS FIRST, cd_purchase_estimate#31 ASC NULLS FIRST, cd_credit_rating#32 ASC NULLS FIRST, cd_dep_count#33 ASC NULLS FIRST, cd_dep_employed_count#34 ASC NULLS FIRST, cd_dep_college_count#35 ASC NULLS FIRST], [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#41, cd_purchase_estimate#31, cnt2#42, cd_credit_rating#32, cnt3#43, cd_dep_count#33, cnt4#44, cd_dep_employed_count#34, cnt5#45, cd_dep_college_count#35, cnt6#46] ===== Subqueries ===== @@ -313,6 +313,6 @@ Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index 07bcd309e9ce6..370551a2de652 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -123,20 +123,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#16] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] -Right keys [1]: [d_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] +Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#16] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] @@ -144,34 +144,34 @@ Right keys [1]: [ws_bill_customer_sk#14] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#8)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#20] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#9] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#20] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#17] -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] +Output [1]: [cs_ship_customer_sk#18] +Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#20] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [cs_ship_customer_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#17] +Right keys [1]: [cs_ship_customer_sk#18] Join condition: None (29) Filter [codegen id : 9] @@ -183,84 +183,84 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] +Output [2]: [ca_address_sk#22, ca_county#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_county#21] +Input [2]: [ca_address_sk#22, ca_county#23] (33) Filter [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#20)) +Input [2]: [ca_address_sk#22, ca_county#23] +Condition : (ca_county#23 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#22)) (34) Project [codegen id : 7] -Output [1]: [ca_address_sk#20] -Input [2]: [ca_address_sk#20, ca_county#21] +Output [1]: [ca_address_sk#22] +Input [2]: [ca_address_sk#22, ca_county#23] (35) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [ca_address_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#22] Join condition: None (37) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] +Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22] (38) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (40) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Condition : isnotnull(cd_demo_sk#23) +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Condition : isnotnull(cd_demo_sk#25) (41) BroadcastExchange -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#25] Join condition: None (43) Project [codegen id : 9] -Output [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (44) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] +Aggregate Attributes [1]: [count#35] +Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] (45) Exchange -Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] -Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] (46) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] -Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#36] -Results [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#36 AS cnt1#37, cd_purchase_estimate#27, count(1)#36 AS cnt2#38, cd_credit_rating#28, count(1)#36 AS cnt3#39, cd_dep_count#29, count(1)#36 AS cnt4#40, cd_dep_employed_count#30, count(1)#36 AS cnt5#41, cd_dep_college_count#31, count(1)#36 AS cnt6#42] +Aggregate Attributes [1]: [count(1)#38] +Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] (47) TakeOrderedAndProject -Input [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] -Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] +Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] +Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] ===== Subqueries ===== @@ -273,6 +273,6 @@ Output [1]: [d_date_sk#9] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 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 da2143dc20a90..92e8e38634ea6 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 @@ -200,289 +200,289 @@ Input [2]: [customer_id#23, year_total#24] Arguments: [customer_id#23 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] (27) Filter [codegen id : 10] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_customer_sk#1) +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] +Condition : isnotnull(ss_customer_sk#26) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#31, d_year#32] 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 (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#6, d_year#7] +Input [2]: [d_date_sk#31, d_year#32] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#31, d_year#32] +Condition : ((isnotnull(d_year#32) AND (d_year#32 = 2002)) AND isnotnull(d_date_sk#31)) (31) BroadcastExchange -Input [2]: [d_date_sk#6, d_year#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [2]: [d_date_sk#31, d_year#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ss_sold_date_sk#29] +Right keys [1]: [d_date_sk#31] Join condition: None (33) Project [codegen id : 10] -Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4, d_date_sk#6, d_year#7] +Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Input [6]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29, d_date_sk#31, d_year#32] (34) Exchange -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#34] (35) Sort [codegen id : 11] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] (37) Sort [codegen id : 13] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Arguments: [c_customer_sk#35 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#35] Join condition: None (39) Project [codegen id : 14] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Input [12]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Input [12]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32, c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] (40) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#29] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] +Input [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#43] +Results [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] (41) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] +Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, 5), ENSURE_REQUIREMENTS, [id=#45] (42) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32] -Results [3]: [c_customer_id#11 AS customer_id#33, c_preferred_cust_flag#14 AS customer_preferred_cust_flag#34, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32,18,2) AS year_total#35] +Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] +Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46] +Results [3]: [c_customer_id#36 AS customer_id#47, c_preferred_cust_flag#39 AS customer_preferred_cust_flag#48, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46,18,2) AS year_total#49] (43) Exchange -Input [3]: [customer_id#33, customer_preferred_cust_flag#34, year_total#35] -Arguments: hashpartitioning(customer_id#33, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] +Arguments: hashpartitioning(customer_id#47, 5), ENSURE_REQUIREMENTS, [id=#50] (44) Sort [codegen id : 16] -Input [3]: [customer_id#33, customer_preferred_cust_flag#34, year_total#35] -Arguments: [customer_id#33 ASC NULLS FIRST], false, 0 +Input [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] +Arguments: [customer_id#47 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#33] +Right keys [1]: [customer_id#47] Join condition: None (46) Project [codegen id : 17] -Output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35] -Input [5]: [customer_id#23, year_total#24, customer_id#33, customer_preferred_cust_flag#34, year_total#35] +Output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49] +Input [5]: [customer_id#23, year_total#24, customer_id#47, customer_preferred_cust_flag#48, year_total#49] (47) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 19] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] (49) Filter [codegen id : 19] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_customer_sk#37) +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) (50) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#55, d_year#56] (51) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#54] +Right keys [1]: [d_date_sk#55] Join condition: None (52) Project [codegen id : 19] -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Input [6]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#6, d_year#7] +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] (53) Exchange -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Arguments: hashpartitioning(ws_bill_customer_sk#37, 5), ENSURE_REQUIREMENTS, [id=#41] +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] (54) Sort [codegen id : 20] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Arguments: [ws_bill_customer_sk#37 ASC NULLS FIRST], false, 0 +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 (55) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +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] (56) Sort [codegen id : 22] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +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 (57) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#37] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ws_bill_customer_sk#51] +Right keys [1]: [c_customer_sk#58] Join condition: None (58) Project [codegen id : 23] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Input [12]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +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] (59) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#42] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#43] +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] (60) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#43] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#44] +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] (61) HashAggregate [codegen id : 24] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#43] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45] -Results [2]: [c_customer_id#11 AS customer_id#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45,18,2) AS year_total#47] +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] (62) Filter [codegen id : 24] -Input [2]: [customer_id#46, year_total#47] -Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) +Input [2]: [customer_id#70, year_total#71] +Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.00)) (63) Project [codegen id : 24] -Output [2]: [customer_id#46 AS customer_id#48, year_total#47 AS year_total#49] -Input [2]: [customer_id#46, year_total#47] +Output [2]: [customer_id#70 AS customer_id#72, year_total#71 AS year_total#73] +Input [2]: [customer_id#70, year_total#71] (64) Exchange -Input [2]: [customer_id#48, year_total#49] -Arguments: hashpartitioning(customer_id#48, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [customer_id#72, year_total#73] +Arguments: hashpartitioning(customer_id#72, 5), ENSURE_REQUIREMENTS, [id=#74] (65) Sort [codegen id : 25] -Input [2]: [customer_id#48, year_total#49] -Arguments: [customer_id#48 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#72, year_total#73] +Arguments: [customer_id#72 ASC NULLS FIRST], false, 0 (66) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#48] +Right keys [1]: [customer_id#72] Join condition: None (67) Project [codegen id : 26] -Output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35, year_total#49] -Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35, customer_id#48, year_total#49] +Output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73] +Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, customer_id#72, year_total#73] (68) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ws_sold_date_sk#78), dynamicpruningexpression(ws_sold_date_sk#78 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] (70) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_customer_sk#37) +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] +Condition : isnotnull(ws_bill_customer_sk#75) (71) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#79, d_year#80] (72) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#78] +Right keys [1]: [d_date_sk#79] Join condition: None (73) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Input [6]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#6, d_year#7] +Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Input [6]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78, d_date_sk#79, d_year#80] (74) Exchange -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Arguments: hashpartitioning(ws_bill_customer_sk#37, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Arguments: hashpartitioning(ws_bill_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] (75) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Arguments: [ws_bill_customer_sk#37 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Arguments: [ws_bill_customer_sk#75 ASC NULLS FIRST], false, 0 (76) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] (77) Sort [codegen id : 31] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] +Arguments: [c_customer_sk#82 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#37] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ws_bill_customer_sk#75] +Right keys [1]: [c_customer_sk#82] Join condition: None (79) Project [codegen id : 32] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Input [12]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Input [12]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80, c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] (80) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#52] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#53] +Input [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#90] +Results [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] (81) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#53] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] +Arguments: hashpartitioning(c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, 5), ENSURE_REQUIREMENTS, [id=#92] (82) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#53] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55] -Results [2]: [c_customer_id#11 AS customer_id#56, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55,18,2) AS year_total#57] +Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] +Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93] +Results [2]: [c_customer_id#83 AS customer_id#94, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93,18,2) AS year_total#95] (83) Exchange -Input [2]: [customer_id#56, year_total#57] -Arguments: hashpartitioning(customer_id#56, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [customer_id#94, year_total#95] +Arguments: hashpartitioning(customer_id#94, 5), ENSURE_REQUIREMENTS, [id=#96] (84) Sort [codegen id : 34] -Input [2]: [customer_id#56, year_total#57] -Arguments: [customer_id#56 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#94, year_total#95] +Arguments: [customer_id#94 ASC NULLS FIRST], false, 0 (85) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#56] -Join condition: (CASE WHEN (year_total#49 > 0.00) THEN CheckOverflow((promote_precision(year_total#57) / promote_precision(year_total#49)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#35) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE null END) +Right keys [1]: [customer_id#94] +Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#95) / promote_precision(year_total#73)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#49) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE null END) (86) Project [codegen id : 35] -Output [1]: [customer_preferred_cust_flag#34] -Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35, year_total#49, customer_id#56, year_total#57] +Output [1]: [customer_preferred_cust_flag#48] +Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73, customer_id#94, year_total#95] (87) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#34] -Arguments: 100, [customer_preferred_cust_flag#34 ASC NULLS FIRST], [customer_preferred_cust_flag#34] +Input [1]: [customer_preferred_cust_flag#48] +Arguments: 100, [customer_preferred_cust_flag#48 ASC NULLS FIRST], [customer_preferred_cust_flag#48] ===== Subqueries ===== @@ -493,15 +493,15 @@ ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#6, d_year#7] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#26 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#30 ReusedExchange (89) (89) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#31, d_year#32] -Subquery:3 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#26 +Subquery:4 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#78 IN dynamicpruning#30 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 5a5249a3aa81b..4081f69732168 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 @@ -172,286 +172,286 @@ Input [2]: [customer_id#22, year_total#23] Condition : (isnotnull(year_total#23) AND (year_total#23 > 0.00)) (20) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] (22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_customer_id#25)) (23) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Output [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ss_sold_date_sk#35), dynamicpruningexpression(ss_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] (25) Filter [codegen id : 4] -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_customer_sk#9) +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Condition : isnotnull(ss_customer_sk#32) (26) BroadcastExchange -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#9] +Left keys [1]: [c_customer_sk#24] +Right keys [1]: [ss_customer_sk#32] Join condition: None (28) Project [codegen id : 6] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Input [12]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#38, d_year#39] 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 (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#15, d_year#16] +Input [2]: [d_date_sk#38, d_year#39] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#15, d_year#16] -Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2002)) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#38, d_year#39] +Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) (32) BroadcastExchange -Input [2]: [d_date_sk#15, d_year#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +Input [2]: [d_date_sk#38, d_year#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#35] +Right keys [1]: [d_date_sk#38] Join condition: None (34) Project [codegen id : 6] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#15, d_year#16] +Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] +Input [12]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#38, d_year#39] (35) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#27] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] +Input [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] +Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#41] +Results [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] (36) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] +Arguments: hashpartitioning(c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, 5), ENSURE_REQUIREMENTS, [id=#43] (37) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30] -Results [3]: [c_customer_id#2 AS customer_id#31, c_preferred_cust_flag#5 AS customer_preferred_cust_flag#32, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30,18,2) AS year_total#33] +Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] +Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44] +Results [3]: [c_customer_id#25 AS customer_id#45, c_preferred_cust_flag#28 AS customer_preferred_cust_flag#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44,18,2) AS year_total#47] (38) BroadcastExchange -Input [3]: [customer_id#31, customer_preferred_cust_flag#32, year_total#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] +Input [3]: [customer_id#45, customer_preferred_cust_flag#46, year_total#47] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#31] +Right keys [1]: [customer_id#45] Join condition: None (40) Project [codegen id : 16] -Output [4]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33] -Input [5]: [customer_id#22, year_total#23, customer_id#31, customer_preferred_cust_flag#32, year_total#33] +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] (41) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +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] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (42) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +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] (43) Filter [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +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)) (44) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Output [4]: [ws_bill_customer_sk#57, ws_ext_discount_amt#58, ws_ext_list_price#59, ws_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (45) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Input [4]: [ws_bill_customer_sk#57, ws_ext_discount_amt#58, ws_ext_list_price#59, ws_sold_date_sk#60] (46) Filter [codegen id : 8] -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_bill_customer_sk#35) +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) (47) BroadcastExchange -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#39] +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] (48) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#35] +Left keys [1]: [c_customer_sk#49] +Right keys [1]: [ws_bill_customer_sk#57] Join condition: None (49) Project [codegen id : 10] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +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] (50) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#62, d_year#63] (51) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ws_sold_date_sk#60] +Right keys [1]: [d_date_sk#62] Join condition: None (52) Project [codegen id : 10] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38, d_date_sk#15, d_year#16] +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] (53) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#40] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#41] +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] (54) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#41] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#42] +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] (55) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#41] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#43] -Results [2]: [c_customer_id#2 AS customer_id#44, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#43,18,2) AS year_total#45] +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] (56) Filter [codegen id : 11] -Input [2]: [customer_id#44, year_total#45] -Condition : (isnotnull(year_total#45) AND (year_total#45 > 0.00)) +Input [2]: [customer_id#68, year_total#69] +Condition : (isnotnull(year_total#69) AND (year_total#69 > 0.00)) (57) Project [codegen id : 11] -Output [2]: [customer_id#44 AS customer_id#46, year_total#45 AS year_total#47] -Input [2]: [customer_id#44, year_total#45] +Output [2]: [customer_id#68 AS customer_id#70, year_total#69 AS year_total#71] +Input [2]: [customer_id#68, year_total#69] (58) BroadcastExchange -Input [2]: [customer_id#46, year_total#47] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] +Input [2]: [customer_id#70, year_total#71] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#72] (59) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#46] +Right keys [1]: [customer_id#70] Join condition: None (60) Project [codegen id : 16] -Output [5]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33, year_total#47] -Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33, customer_id#46, year_total#47] +Output [5]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, year_total#71] +Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, customer_id#70, year_total#71] (61) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (62) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] (63) Filter [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_customer_id#74)) (64) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Output [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ws_sold_date_sk#84), dynamicpruningexpression(ws_sold_date_sk#84 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] (66) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_bill_customer_sk#35) +Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Condition : isnotnull(ws_bill_customer_sk#81) (67) BroadcastExchange -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#85] (68) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#35] +Left keys [1]: [c_customer_sk#73] +Right keys [1]: [ws_bill_customer_sk#81] Join condition: None (69) Project [codegen id : 14] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Output [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Input [12]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] (70) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#86, d_year#87] (71) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ws_sold_date_sk#84] +Right keys [1]: [d_date_sk#86] Join condition: None (72) Project [codegen id : 14] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38, d_date_sk#15, d_year#16] +Output [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, d_year#87] +Input [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84, d_date_sk#86, d_year#87] (73) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#50] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#51] +Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, d_year#87] +Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#88] +Results [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] (74) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#51] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] +Arguments: hashpartitioning(c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, 5), ENSURE_REQUIREMENTS, [id=#90] (75) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#51] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#53] -Results [2]: [c_customer_id#2 AS customer_id#54, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#53,18,2) AS year_total#55] +Input [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] +Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))#91] +Results [2]: [c_customer_id#74 AS customer_id#92, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))#91,18,2) AS year_total#93] (76) BroadcastExchange -Input [2]: [customer_id#54, year_total#55] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] +Input [2]: [customer_id#92, year_total#93] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#94] (77) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#54] -Join condition: (CASE WHEN (year_total#47 > 0.00) THEN CheckOverflow((promote_precision(year_total#55) / promote_precision(year_total#47)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#33) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE null END) +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 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) (78) Project [codegen id : 16] -Output [1]: [customer_preferred_cust_flag#32] -Input [7]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33, year_total#47, customer_id#54, year_total#55] +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#71, customer_id#92, year_total#93] (79) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#32] -Arguments: 100, [customer_preferred_cust_flag#32 ASC NULLS FIRST], [customer_preferred_cust_flag#32] +Input [1]: [customer_preferred_cust_flag#46] +Arguments: 100, [customer_preferred_cust_flag#46 ASC NULLS FIRST], [customer_preferred_cust_flag#46] ===== Subqueries ===== @@ -462,15 +462,15 @@ ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#15, d_year#16] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#24 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 ReusedExchange (81) (81) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#38, d_year#39] -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#24 +Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#84 IN dynamicpruning#36 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index 9c56438870f64..7604ea1e786d8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -74,187 +74,187 @@ Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet default.catalog_sales -Output [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Output [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Input [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] (9) Project [codegen id : 3] -Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#10, cs_order_number#5 AS cs_order_number#5#11] -Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Output [2]: [cs_warehouse_sk#10, cs_order_number#11] +Input [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] (10) Exchange -Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] -Arguments: hashpartitioning(cs_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [2]: [cs_warehouse_sk#10, cs_order_number#11] +Arguments: hashpartitioning(cs_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 4] -Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] -Arguments: [cs_order_number#5#11 ASC NULLS FIRST], false, 0 +Input [2]: [cs_warehouse_sk#10, cs_order_number#11] +Arguments: [cs_order_number#11 ASC NULLS FIRST], false, 0 (12) SortMergeJoin Left keys [1]: [cs_order_number#5] -Right keys [1]: [cs_order_number#5#11] -Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#10) +Right keys [1]: [cs_order_number#11] +Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#10) (13) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] (14) Scan parquet default.catalog_returns -Output [2]: [cr_order_number#13, cr_returned_date_sk#14] +Output [2]: [cr_order_number#14, cr_returned_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] ReadSchema: struct (15) ColumnarToRow [codegen id : 6] -Input [2]: [cr_order_number#13, cr_returned_date_sk#14] +Input [2]: [cr_order_number#14, cr_returned_date_sk#15] (16) Project [codegen id : 6] -Output [1]: [cr_order_number#13] -Input [2]: [cr_order_number#13, cr_returned_date_sk#14] +Output [1]: [cr_order_number#14] +Input [2]: [cr_order_number#14, cr_returned_date_sk#15] (17) Exchange -Input [1]: [cr_order_number#13] -Arguments: hashpartitioning(cr_order_number#13, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [1]: [cr_order_number#14] +Arguments: hashpartitioning(cr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] (18) Sort [codegen id : 7] -Input [1]: [cr_order_number#13] -Arguments: [cr_order_number#13 ASC NULLS FIRST], false, 0 +Input [1]: [cr_order_number#14] +Arguments: [cr_order_number#14 ASC NULLS FIRST], false, 0 (19) SortMergeJoin Left keys [1]: [cs_order_number#5] -Right keys [1]: [cr_order_number#13] +Right keys [1]: [cr_order_number#14] Join condition: None (20) Scan parquet default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] +Output [2]: [ca_address_sk#17, ca_state#18] 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 (21) ColumnarToRow [codegen id : 8] -Input [2]: [ca_address_sk#16, ca_state#17] +Input [2]: [ca_address_sk#17, ca_state#18] (22) Filter [codegen id : 8] -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_address_sk#16)) +Input [2]: [ca_address_sk#17, ca_state#18] +Condition : ((isnotnull(ca_state#18) AND (ca_state#18 = GA)) AND isnotnull(ca_address_sk#17)) (23) Project [codegen id : 8] -Output [1]: [ca_address_sk#16] -Input [2]: [ca_address_sk#16, ca_state#17] +Output [1]: [ca_address_sk#17] +Input [2]: [ca_address_sk#17, ca_state#18] (24) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Input [1]: [ca_address_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#16] +Right keys [1]: [ca_address_sk#17] Join condition: None (26) Project [codegen id : 11] Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] +Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#17] (27) Scan parquet default.call_center -Output [2]: [cc_call_center_sk#19, cc_county#20] +Output [2]: [cc_call_center_sk#20, cc_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 9] -Input [2]: [cc_call_center_sk#19, cc_county#20] +Input [2]: [cc_call_center_sk#20, cc_county#21] (29) Filter [codegen id : 9] -Input [2]: [cc_call_center_sk#19, cc_county#20] -Condition : ((isnotnull(cc_county#20) AND (cc_county#20 = Williamson County)) AND isnotnull(cc_call_center_sk#19)) +Input [2]: [cc_call_center_sk#20, cc_county#21] +Condition : ((isnotnull(cc_county#21) AND (cc_county#21 = Williamson County)) AND isnotnull(cc_call_center_sk#20)) (30) Project [codegen id : 9] -Output [1]: [cc_call_center_sk#19] -Input [2]: [cc_call_center_sk#19, cc_county#20] +Output [1]: [cc_call_center_sk#20] +Input [2]: [cc_call_center_sk#20, cc_county#21] (31) BroadcastExchange -Input [1]: [cc_call_center_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [cc_call_center_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] -Right keys [1]: [cc_call_center_sk#19] +Right keys [1]: [cc_call_center_sk#20] Join condition: None (33) Project [codegen id : 11] Output [4]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [6]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#19] +Input [6]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#20] (34) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_date#23] +Output [2]: [d_date_sk#23, d_date#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 10] -Input [2]: [d_date_sk#22, d_date#23] +Input [2]: [d_date_sk#23, d_date#24] (36) Filter [codegen id : 10] -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2002-02-01)) AND (d_date#23 <= 2002-04-02)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#23, d_date#24] +Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 2002-02-01)) AND (d_date#24 <= 2002-04-02)) AND isnotnull(d_date_sk#23)) (37) Project [codegen id : 10] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_date#23] +Output [1]: [d_date_sk#23] +Input [2]: [d_date_sk#23, d_date#24] (38) 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#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#23] Join condition: None (40) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [5]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#22] +Input [5]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#23] (41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] -Results [3]: [cs_order_number#5, sum#27, sum#28] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] +Results [3]: [cs_order_number#5, sum#28, sum#29] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_order_number#5, sum#27, sum#28] +Input [3]: [cs_order_number#5, sum#28, sum#29] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] -Results [3]: [cs_order_number#5, sum#27, sum#28] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] +Results [3]: [cs_order_number#5, sum#28, sum#29] (43) HashAggregate [codegen id : 11] -Input [3]: [cs_order_number#5, sum#27, sum#28] +Input [3]: [cs_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] -Results [3]: [sum#27, sum#28, count#30] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] +Results [3]: [sum#28, sum#29, count#31] (44) Exchange -Input [3]: [sum#27, sum#28, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [sum#28, sum#29, count#31] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] (45) HashAggregate [codegen id : 12] -Input [3]: [sum#27, sum#28, count#30] +Input [3]: [sum#28, sum#29, count#31] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] -Results [3]: [count(cs_order_number#5)#29 AS order count #32, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#25,17,2) AS total shipping cost #33, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#26,17,2) AS total net profit #34] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] +Results [3]: [count(cs_order_number#5)#30 AS order count #33, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#26,17,2) AS total shipping cost #34, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#27,17,2) AS total net profit #35] (46) Sort [codegen id : 12] -Input [3]: [order count #32, total shipping cost #33, total net profit #34] -Arguments: [order count #32 ASC NULLS FIRST], true, 0 +Input [3]: [order count #33, total shipping cost #34, total net profit #35] +Arguments: [order count #33 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index dde6a9f564859..683d83235cce5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -74,187 +74,187 @@ Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet default.catalog_sales -Output [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Output [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Input [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] (9) Project [codegen id : 3] -Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#10, cs_order_number#5 AS cs_order_number#5#11] -Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Output [2]: [cs_warehouse_sk#10, cs_order_number#11] +Input [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] (10) Exchange -Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] -Arguments: hashpartitioning(cs_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [2]: [cs_warehouse_sk#10, cs_order_number#11] +Arguments: hashpartitioning(cs_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 4] -Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] -Arguments: [cs_order_number#5#11 ASC NULLS FIRST], false, 0 +Input [2]: [cs_warehouse_sk#10, cs_order_number#11] +Arguments: [cs_order_number#11 ASC NULLS FIRST], false, 0 (12) SortMergeJoin Left keys [1]: [cs_order_number#5] -Right keys [1]: [cs_order_number#5#11] -Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#10) +Right keys [1]: [cs_order_number#11] +Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#10) (13) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] (14) Scan parquet default.catalog_returns -Output [2]: [cr_order_number#13, cr_returned_date_sk#14] +Output [2]: [cr_order_number#14, cr_returned_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] ReadSchema: struct (15) ColumnarToRow [codegen id : 6] -Input [2]: [cr_order_number#13, cr_returned_date_sk#14] +Input [2]: [cr_order_number#14, cr_returned_date_sk#15] (16) Project [codegen id : 6] -Output [1]: [cr_order_number#13] -Input [2]: [cr_order_number#13, cr_returned_date_sk#14] +Output [1]: [cr_order_number#14] +Input [2]: [cr_order_number#14, cr_returned_date_sk#15] (17) Exchange -Input [1]: [cr_order_number#13] -Arguments: hashpartitioning(cr_order_number#13, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [1]: [cr_order_number#14] +Arguments: hashpartitioning(cr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] (18) Sort [codegen id : 7] -Input [1]: [cr_order_number#13] -Arguments: [cr_order_number#13 ASC NULLS FIRST], false, 0 +Input [1]: [cr_order_number#14] +Arguments: [cr_order_number#14 ASC NULLS FIRST], false, 0 (19) SortMergeJoin Left keys [1]: [cs_order_number#5] -Right keys [1]: [cr_order_number#13] +Right keys [1]: [cr_order_number#14] Join condition: None (20) Scan parquet default.date_dim -Output [2]: [d_date_sk#16, d_date#17] +Output [2]: [d_date_sk#17, d_date#18] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#16, d_date#17] +Input [2]: [d_date_sk#17, d_date#18] (22) Filter [codegen id : 8] -Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2002-02-01)) AND (d_date#17 <= 2002-04-02)) AND isnotnull(d_date_sk#16)) +Input [2]: [d_date_sk#17, d_date#18] +Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 2002-02-01)) AND (d_date#18 <= 2002-04-02)) AND isnotnull(d_date_sk#17)) (23) Project [codegen id : 8] -Output [1]: [d_date_sk#16] -Input [2]: [d_date_sk#16, d_date#17] +Output [1]: [d_date_sk#17] +Input [2]: [d_date_sk#17, d_date#18] (24) BroadcastExchange -Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Input [1]: [d_date_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#17] Join condition: None (26) Project [codegen id : 11] Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#16] +Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#17] (27) Scan parquet default.customer_address -Output [2]: [ca_address_sk#19, ca_state#20] +Output [2]: [ca_address_sk#20, ca_state#21] 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 (28) ColumnarToRow [codegen id : 9] -Input [2]: [ca_address_sk#19, ca_state#20] +Input [2]: [ca_address_sk#20, ca_state#21] (29) Filter [codegen id : 9] -Input [2]: [ca_address_sk#19, ca_state#20] -Condition : ((isnotnull(ca_state#20) AND (ca_state#20 = GA)) AND isnotnull(ca_address_sk#19)) +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : ((isnotnull(ca_state#21) AND (ca_state#21 = GA)) AND isnotnull(ca_address_sk#20)) (30) Project [codegen id : 9] -Output [1]: [ca_address_sk#19] -Input [2]: [ca_address_sk#19, ca_state#20] +Output [1]: [ca_address_sk#20] +Input [2]: [ca_address_sk#20, ca_state#21] (31) BroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [ca_address_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#19] +Right keys [1]: [ca_address_sk#20] Join condition: None (33) Project [codegen id : 11] Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#19] +Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#20] (34) Scan parquet default.call_center -Output [2]: [cc_call_center_sk#22, cc_county#23] +Output [2]: [cc_call_center_sk#23, cc_county#24] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 10] -Input [2]: [cc_call_center_sk#22, cc_county#23] +Input [2]: [cc_call_center_sk#23, cc_county#24] (36) Filter [codegen id : 10] -Input [2]: [cc_call_center_sk#22, cc_county#23] -Condition : ((isnotnull(cc_county#23) AND (cc_county#23 = Williamson County)) AND isnotnull(cc_call_center_sk#22)) +Input [2]: [cc_call_center_sk#23, cc_county#24] +Condition : ((isnotnull(cc_county#24) AND (cc_county#24 = Williamson County)) AND isnotnull(cc_call_center_sk#23)) (37) Project [codegen id : 10] -Output [1]: [cc_call_center_sk#22] -Input [2]: [cc_call_center_sk#22, cc_county#23] +Output [1]: [cc_call_center_sk#23] +Input [2]: [cc_call_center_sk#23, cc_county#24] (38) BroadcastExchange -Input [1]: [cc_call_center_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +Input [1]: [cc_call_center_sk#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] -Right keys [1]: [cc_call_center_sk#22] +Right keys [1]: [cc_call_center_sk#23] Join condition: None (40) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#22] +Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#23] (41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] -Results [3]: [cs_order_number#5, sum#27, sum#28] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] +Results [3]: [cs_order_number#5, sum#28, sum#29] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_order_number#5, sum#27, sum#28] +Input [3]: [cs_order_number#5, sum#28, sum#29] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] -Results [3]: [cs_order_number#5, sum#27, sum#28] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] +Results [3]: [cs_order_number#5, sum#28, sum#29] (43) HashAggregate [codegen id : 11] -Input [3]: [cs_order_number#5, sum#27, sum#28] +Input [3]: [cs_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] -Results [3]: [sum#27, sum#28, count#30] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] +Results [3]: [sum#28, sum#29, count#31] (44) Exchange -Input [3]: [sum#27, sum#28, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [sum#28, sum#29, count#31] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] (45) HashAggregate [codegen id : 12] -Input [3]: [sum#27, sum#28, count#30] +Input [3]: [sum#28, sum#29, count#31] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] -Results [3]: [count(cs_order_number#5)#29 AS order count #32, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#25,17,2) AS total shipping cost #33, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#26,17,2) AS total net profit #34] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] +Results [3]: [count(cs_order_number#5)#30 AS order count #33, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#26,17,2) AS total shipping cost #34, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#27,17,2) AS total net profit #35] (46) Sort [codegen id : 12] -Input [3]: [order count #32, total shipping cost #33, total net profit #34] -Arguments: [order count #32 ASC NULLS FIRST], true, 0 +Input [3]: [order count #33, total shipping cost #34, total net profit #35] +Arguments: [order count #33 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index 8918c2a36e2ec..97467fe1dc1db 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -77,7 +77,7 @@ Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isno Output [2]: [d_date_sk#8, d_quarter_name#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -85,7 +85,7 @@ Input [2]: [d_date_sk#8, d_quarter_name#9] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#8, d_quarter_name#9] -Condition : ((isnotnull(d_quarter_name#9) AND (rpad(d_quarter_name#9, 6, ) = 2001Q1)) AND isnotnull(d_date_sk#8)) +Condition : ((isnotnull(d_quarter_name#9) AND (d_quarter_name#9 = 2001Q1)) AND isnotnull(d_date_sk#8)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#8] @@ -197,7 +197,7 @@ Condition : ((isnotnull(sr_customer_sk#21) AND isnotnull(sr_item_sk#20)) AND isn Output [2]: [d_date_sk#26, d_quarter_name#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 9] @@ -205,7 +205,7 @@ Input [2]: [d_date_sk#26, d_quarter_name#27] (33) Filter [codegen id : 9] Input [2]: [d_date_sk#26, d_quarter_name#27] -Condition : (rpad(d_quarter_name#27, 6, ) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#26)) +Condition : (d_quarter_name#27 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#26)) (34) Project [codegen id : 9] Output [1]: [d_date_sk#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index dafa24943be82..32c8de9a1d16e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -124,7 +124,7 @@ Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_i Output [2]: [d_date_sk#21, d_quarter_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 3] @@ -132,7 +132,7 @@ Input [2]: [d_date_sk#21, d_quarter_name#22] (18) Filter [codegen id : 3] Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : ((isnotnull(d_quarter_name#22) AND (rpad(d_quarter_name#22, 6, ) = 2001Q1)) AND isnotnull(d_date_sk#21)) +Condition : ((isnotnull(d_quarter_name#22) AND (d_quarter_name#22 = 2001Q1)) AND isnotnull(d_date_sk#21)) (19) Project [codegen id : 3] Output [1]: [d_date_sk#21] @@ -155,7 +155,7 @@ Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_re Output [2]: [d_date_sk#24, d_quarter_name#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] @@ -163,7 +163,7 @@ Input [2]: [d_date_sk#24, d_quarter_name#25] (25) Filter [codegen id : 4] Input [2]: [d_date_sk#24, d_quarter_name#25] -Condition : (rpad(d_quarter_name#25, 6, ) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) +Condition : (d_quarter_name#25 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) (26) Project [codegen id : 4] Output [1]: [d_date_sk#24] 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 f540e067bbc92..c31bb7470648f 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#10, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] +Output [8]: [d_week_seq#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] (25) HashAggregate [codegen id : 11] -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))#60, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#60,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66,17,2) AS sat_sales#41] +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] (26) Scan parquet default.date_dim -Output [2]: [d_week_seq#67, d_year#68] +Output [2]: [d_week_seq#69, d_year#70] 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#67, d_year#68] +Input [2]: [d_week_seq#69, d_year#70] (28) Filter [codegen id : 10] -Input [2]: [d_week_seq#67, d_year#68] -Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) +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)) (29) Project [codegen id : 10] -Output [1]: [d_week_seq#67] -Input [2]: [d_week_seq#67, d_year#68] +Output [1]: [d_week_seq#69] +Input [2]: [d_week_seq#69, d_year#70] (30) BroadcastExchange -Input [1]: [d_week_seq#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] +Input [1]: [d_week_seq#69] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] (31) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#67] +Left keys [1]: [d_week_seq#53] +Right keys [1]: [d_week_seq#69] Join condition: None (32) Project [codegen id : 11] -Output [8]: [d_week_seq#10 AS d_week_seq2#70, sun_sales#35 AS sun_sales2#71, mon_sales#36 AS mon_sales2#72, tue_sales#37 AS tue_sales2#73, wed_sales#38 AS wed_sales2#74, thu_sales#39 AS thu_sales2#75, fri_sales#40 AS fri_sales2#76, sat_sales#41 AS sat_sales2#77] -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#67] +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] (33) BroadcastExchange -Input [8]: [d_week_seq2#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#78] +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] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq1#45] -Right keys [1]: [(d_week_seq2#70 - 53)] +Right keys [1]: [(d_week_seq2#72 - 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#71)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#79, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#72)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#80, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#73)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#81, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#74)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#82, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#75)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#83, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#76)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#84, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#77)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#85] -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#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] +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] (36) Exchange -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] -Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#86] +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] (37) Sort [codegen id : 13] -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] +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: [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 f540e067bbc92..c31bb7470648f 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#10, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] +Output [8]: [d_week_seq#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] (25) HashAggregate [codegen id : 11] -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))#60, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#60,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66,17,2) AS sat_sales#41] +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] (26) Scan parquet default.date_dim -Output [2]: [d_week_seq#67, d_year#68] +Output [2]: [d_week_seq#69, d_year#70] 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#67, d_year#68] +Input [2]: [d_week_seq#69, d_year#70] (28) Filter [codegen id : 10] -Input [2]: [d_week_seq#67, d_year#68] -Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) +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)) (29) Project [codegen id : 10] -Output [1]: [d_week_seq#67] -Input [2]: [d_week_seq#67, d_year#68] +Output [1]: [d_week_seq#69] +Input [2]: [d_week_seq#69, d_year#70] (30) BroadcastExchange -Input [1]: [d_week_seq#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] +Input [1]: [d_week_seq#69] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] (31) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#67] +Left keys [1]: [d_week_seq#53] +Right keys [1]: [d_week_seq#69] Join condition: None (32) Project [codegen id : 11] -Output [8]: [d_week_seq#10 AS d_week_seq2#70, sun_sales#35 AS sun_sales2#71, mon_sales#36 AS mon_sales2#72, tue_sales#37 AS tue_sales2#73, wed_sales#38 AS wed_sales2#74, thu_sales#39 AS thu_sales2#75, fri_sales#40 AS fri_sales2#76, sat_sales#41 AS sat_sales2#77] -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#67] +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] (33) BroadcastExchange -Input [8]: [d_week_seq2#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#78] +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] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq1#45] -Right keys [1]: [(d_week_seq2#70 - 53)] +Right keys [1]: [(d_week_seq2#72 - 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#71)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#79, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#72)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#80, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#73)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#81, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#74)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#82, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#75)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#83, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#76)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#84, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#77)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#85] -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#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] +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] (36) Exchange -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] -Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#86] +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] (37) Sort [codegen id : 13] -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] +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: [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 aa96edd4c7fba..f42a7615324c2 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 @@ -246,91 +246,91 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (33) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (35) Filter [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Condition : isnotnull(ss_customer_sk#26) (36) Project [codegen id : 11] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (37) Exchange Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#29] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#30] (38) Sort [codegen id : 12] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (39) Scan parquet default.customer -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 13] -Input [1]: [c_customer_sk#30] +Input [1]: [c_customer_sk#31] (41) Filter [codegen id : 13] -Input [1]: [c_customer_sk#30] -Condition : isnotnull(c_customer_sk#30) +Input [1]: [c_customer_sk#31] +Condition : isnotnull(c_customer_sk#31) (42) Exchange -Input [1]: [c_customer_sk#30] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [1]: [c_customer_sk#31] +Arguments: hashpartitioning(c_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] (43) Sort [codegen id : 14] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (44) SortMergeJoin [codegen id : 15] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#31] Join condition: None (45) Project [codegen id : 15] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#31] (46) HashAggregate [codegen id : 15] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Keys [1]: [c_customer_sk#31] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Aggregate Attributes [2]: [sum#33, isEmpty#34] +Results [3]: [c_customer_sk#31, sum#35, isEmpty#36] (47) HashAggregate [codegen id : 15] -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Keys [1]: [c_customer_sk#30] +Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] +Keys [1]: [c_customer_sk#31] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (48) Filter [codegen id : 15] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (49) Project [codegen id : 15] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#31] +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (50) Sort [codegen id : 15] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (51) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#31] Join condition: None (52) Project [codegen id : 17] @@ -338,209 +338,209 @@ Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (53) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_year#13, d_moy#40] +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 16] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (55) Filter [codegen id : 16] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#40)) AND (d_year#13 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#11)) +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) (56) Project [codegen id : 16] -Output [1]: [d_date_sk#11] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] +Output [1]: [d_date_sk#43] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (57) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] (58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#11] +Right keys [1]: [d_date_sk#43] Join condition: None (59) Project [codegen id : 17] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#42] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#11] +Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#47] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] (60) Scan parquet default.web_sales -Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#6)] ReadSchema: struct (61) ColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (62) Exchange -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: hashpartitioning(ws_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: hashpartitioning(ws_item_sk#48, 5), ENSURE_REQUIREMENTS, [id=#53] (63) Sort [codegen id : 19] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: [ws_item_sk#43 ASC NULLS FIRST], false, 0 +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: [ws_item_sk#48 ASC NULLS FIRST], false, 0 (64) ReusedExchange [Reuses operator id: unknown] -Output [2]: [ss_item_sk#8, d_date#12] +Output [2]: [ss_item_sk#54, d_date#55] (65) Sort [codegen id : 22] -Input [2]: [ss_item_sk#8, d_date#12] -Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 +Input [2]: [ss_item_sk#54, d_date#55] +Arguments: [ss_item_sk#54 ASC NULLS FIRST], false, 0 (66) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#16, i_item_desc#17] +Output [2]: [i_item_sk#56, i_item_desc#57] (67) Sort [codegen id : 24] -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16 ASC NULLS FIRST], false, 0 +Input [2]: [i_item_sk#56, i_item_desc#57] +Arguments: [i_item_sk#56 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#8] -Right keys [1]: [i_item_sk#16] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#56] Join condition: None (69) Project [codegen id : 25] -Output [3]: [d_date#12, i_item_sk#16, i_item_desc#17] -Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#16, i_item_desc#17] +Output [3]: [d_date#55, i_item_sk#56, i_item_desc#57] +Input [4]: [ss_item_sk#54, d_date#55, i_item_sk#56, i_item_desc#57] (70) HashAggregate [codegen id : 25] -Input [3]: [d_date#12, i_item_sk#16, i_item_desc#17] -Keys [3]: [substr(i_item_desc#17, 1, 30) AS substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12] +Input [3]: [d_date#55, i_item_sk#56, i_item_desc#57] +Keys [3]: [substr(i_item_desc#57, 1, 30) AS substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#50] -Results [4]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12, count#51] +Aggregate Attributes [1]: [count#59] +Results [4]: [substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55, count#60] (71) HashAggregate [codegen id : 25] -Input [4]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12, count#51] -Keys [3]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12] +Input [4]: [substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55, count#60] +Keys [3]: [substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#52] -Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#52 AS count(1)#53] +Aggregate Attributes [1]: [count(1)#61] +Results [2]: [i_item_sk#56 AS item_sk#23, count(1)#61 AS count(1)#62] (72) Filter [codegen id : 25] -Input [2]: [item_sk#23, count(1)#53] -Condition : (count(1)#53 > 4) +Input [2]: [item_sk#23, count(1)#62] +Condition : (count(1)#62 > 4) (73) Project [codegen id : 25] Output [1]: [item_sk#23] -Input [2]: [item_sk#23, count(1)#53] +Input [2]: [item_sk#23, count(1)#62] (74) Sort [codegen id : 25] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 (75) SortMergeJoin -Left keys [1]: [ws_item_sk#43] +Left keys [1]: [ws_item_sk#48] Right keys [1]: [item_sk#23] Join condition: None (76) Project [codegen id : 26] -Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (77) Exchange -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: hashpartitioning(ws_bill_customer_sk#49, 5), ENSURE_REQUIREMENTS, [id=#63] (78) Sort [codegen id : 27] -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: [ws_bill_customer_sk#49 ASC NULLS FIRST], false, 0 (79) ReusedExchange [Reuses operator id: 37] -Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Output [3]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66] (80) Sort [codegen id : 29] -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66] +Arguments: [ss_customer_sk#64 ASC NULLS FIRST], false, 0 (81) ReusedExchange [Reuses operator id: 42] -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#67] (82) Sort [codegen id : 31] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#67] +Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 (83) SortMergeJoin [codegen id : 32] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ss_customer_sk#64] +Right keys [1]: [c_customer_sk#67] Join condition: None (84) Project [codegen id : 32] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#65, ss_sales_price#66, c_customer_sk#67] +Input [4]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66, c_customer_sk#67] (85) HashAggregate [codegen id : 32] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#55, isEmpty#56] -Results [3]: [c_customer_sk#30, sum#57, isEmpty#58] +Input [3]: [ss_quantity#65, ss_sales_price#66, c_customer_sk#67] +Keys [1]: [c_customer_sk#67] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#68, isEmpty#69] +Results [3]: [c_customer_sk#67, sum#70, isEmpty#71] (86) HashAggregate [codegen id : 32] -Input [3]: [c_customer_sk#30, sum#57, isEmpty#58] -Keys [1]: [c_customer_sk#30] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#59] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#59 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] +Input [3]: [c_customer_sk#67, sum#70, isEmpty#71] +Keys [1]: [c_customer_sk#67] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))#72] +Results [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))#72 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] (87) Filter [codegen id : 32] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (88) Project [codegen id : 32] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] +Output [1]: [c_customer_sk#67] +Input [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] (89) Sort [codegen id : 32] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#67] +Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#44] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ws_bill_customer_sk#49] +Right keys [1]: [c_customer_sk#67] Join condition: None (91) Project [codegen id : 34] -Output [3]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [3]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (92) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#74] (93) BroadcastHashJoin [codegen id : 34] -Left keys [1]: [ws_sold_date_sk#47] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ws_sold_date_sk#52] +Right keys [1]: [d_date_sk#74] Join condition: None (94) Project [codegen id : 34] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2), true) AS sales#61] -Input [4]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#11] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#51 as decimal(12,2)))), DecimalType(18,2), true) AS sales#75] +Input [4]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52, d_date_sk#74] (95) Union (96) HashAggregate [codegen id : 35] -Input [1]: [sales#42] +Input [1]: [sales#47] Keys: [] -Functions [1]: [partial_sum(sales#42)] -Aggregate Attributes [2]: [sum#62, isEmpty#63] -Results [2]: [sum#64, isEmpty#65] +Functions [1]: [partial_sum(sales#47)] +Aggregate Attributes [2]: [sum#76, isEmpty#77] +Results [2]: [sum#78, isEmpty#79] (97) Exchange -Input [2]: [sum#64, isEmpty#65] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] +Input [2]: [sum#78, isEmpty#79] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#80] (98) HashAggregate [codegen id : 36] -Input [2]: [sum#64, isEmpty#65] +Input [2]: [sum#78, isEmpty#79] Keys: [] -Functions [1]: [sum(sales#42)] -Aggregate Attributes [1]: [sum(sales#42)#67] -Results [1]: [sum(sales#42)#67 AS sum(sales)#68] +Functions [1]: [sum(sales#47)] +Aggregate Attributes [1]: [sum(sales#47)#81] +Results [1]: [sum(sales#47)#81 AS sum(sales)#82] ===== Subqueries ===== @@ -549,7 +549,7 @@ ReusedExchange (99) (99) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#43] Subquery:2 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 ReusedExchange (100) @@ -558,7 +558,7 @@ ReusedExchange (100) (100) ReusedExchange [Reuses operator id: 12] Output [2]: [d_date_sk#11, d_date#12] -Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (124) +- Exchange (123) +- * HashAggregate (122) @@ -586,131 +586,131 @@ Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Output [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#69)] +PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#87)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] (103) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] -Condition : isnotnull(ss_customer_sk#26) +Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] +Condition : isnotnull(ss_customer_sk#83) (104) Scan parquet default.date_dim -Output [2]: [d_date_sk#11, d_year#13] +Output [2]: [d_date_sk#88, d_year#89] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (105) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#11, d_year#13] +Input [2]: [d_date_sk#88, d_year#89] (106) Filter [codegen id : 1] -Input [2]: [d_date_sk#11, d_year#13] -Condition : (d_year#13 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) +Input [2]: [d_date_sk#88, d_year#89] +Condition : (d_year#89 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#88)) (107) Project [codegen id : 1] -Output [1]: [d_date_sk#11] -Input [2]: [d_date_sk#11, d_year#13] +Output [1]: [d_date_sk#88] +Input [2]: [d_date_sk#88, d_year#89] (108) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#70] +Input [1]: [d_date_sk#88] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#90] (109) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ss_sold_date_sk#86] +Right keys [1]: [d_date_sk#88] Join condition: None (110) Project [codegen id : 2] -Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9, d_date_sk#11] +Output [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] +Input [5]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86, d_date_sk#88] (111) Exchange -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] +Arguments: hashpartitioning(ss_customer_sk#83, 5), ENSURE_REQUIREMENTS, [id=#91] (112) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] +Arguments: [ss_customer_sk#83 ASC NULLS FIRST], false, 0 (113) Scan parquet default.customer -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#92] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (114) ColumnarToRow [codegen id : 4] -Input [1]: [c_customer_sk#30] +Input [1]: [c_customer_sk#92] (115) Filter [codegen id : 4] -Input [1]: [c_customer_sk#30] -Condition : isnotnull(c_customer_sk#30) +Input [1]: [c_customer_sk#92] +Condition : isnotnull(c_customer_sk#92) (116) Exchange -Input [1]: [c_customer_sk#30] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [1]: [c_customer_sk#92] +Arguments: hashpartitioning(c_customer_sk#92, 5), ENSURE_REQUIREMENTS, [id=#93] (117) Sort [codegen id : 5] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#92] +Arguments: [c_customer_sk#92 ASC NULLS FIRST], false, 0 (118) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ss_customer_sk#83] +Right keys [1]: [c_customer_sk#92] Join condition: None (119) Project [codegen id : 6] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#92] +Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, c_customer_sk#92] (120) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#73, isEmpty#74] -Results [3]: [c_customer_sk#30, sum#75, isEmpty#76] +Input [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#92] +Keys [1]: [c_customer_sk#92] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#94, isEmpty#95] +Results [3]: [c_customer_sk#92, sum#96, isEmpty#97] (121) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#30, sum#75, isEmpty#76] -Keys [1]: [c_customer_sk#30] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#77] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#77 AS csales#78] +Input [3]: [c_customer_sk#92, sum#96, isEmpty#97] +Keys [1]: [c_customer_sk#92] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98 AS csales#99] (122) HashAggregate [codegen id : 6] -Input [1]: [csales#78] +Input [1]: [csales#99] Keys: [] -Functions [1]: [partial_max(csales#78)] -Aggregate Attributes [1]: [max#79] -Results [1]: [max#80] +Functions [1]: [partial_max(csales#99)] +Aggregate Attributes [1]: [max#100] +Results [1]: [max#101] (123) Exchange -Input [1]: [max#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +Input [1]: [max#101] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#102] (124) HashAggregate [codegen id : 7] -Input [1]: [max#80] +Input [1]: [max#101] Keys: [] -Functions [1]: [max(csales#78)] -Aggregate Attributes [1]: [max(csales#78)#82] -Results [1]: [max(csales#78)#82 AS tpcds_cmax#83] +Functions [1]: [max(csales#99)] +Aggregate Attributes [1]: [max(csales#99)#103] +Results [1]: [max(csales#99)#103 AS tpcds_cmax#104] -Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#69 +Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#87 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 108] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#88] -Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:6 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] 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 692671c9f287d..8c2aed03ce0cc 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 @@ -206,83 +206,83 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (29) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (31) Filter [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Condition : isnotnull(ss_customer_sk#26) (32) Project [codegen id : 8] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (33) Scan parquet default.customer -Output [1]: [c_customer_sk#29] +Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 7] -Input [1]: [c_customer_sk#29] +Input [1]: [c_customer_sk#30] (35) Filter [codegen id : 7] -Input [1]: [c_customer_sk#29] -Condition : isnotnull(c_customer_sk#29) +Input [1]: [c_customer_sk#30] +Condition : isnotnull(c_customer_sk#30) (36) BroadcastExchange -Input [1]: [c_customer_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [1]: [c_customer_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (37) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#30] Join condition: None (38) Project [codegen id : 8] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (39) HashAggregate [codegen id : 8] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#31, isEmpty#32] -Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] (40) Exchange -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#36] (41) HashAggregate [codegen id : 9] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (42) Filter [codegen id : 9] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (43) Project [codegen id : 9] -Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (44) Sort [codegen id : 9] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (45) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#30] Join condition: None (46) Project [codegen id : 11] @@ -290,128 +290,128 @@ Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (47) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_year#12, d_moy#40] +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 10] -Input [3]: [d_date_sk#10, d_year#12, d_moy#40] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (49) Filter [codegen id : 10] -Input [3]: [d_date_sk#10, d_year#12, d_moy#40] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#40)) AND (d_year#12 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#10)) +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) (50) Project [codegen id : 10] -Output [1]: [d_date_sk#10] -Input [3]: [d_date_sk#10, d_year#12, d_moy#40] +Output [1]: [d_date_sk#43] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (51) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] (52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#10] +Right keys [1]: [d_date_sk#43] Join condition: None (53) Project [codegen id : 11] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#42] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#10] +Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#47] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] (54) Scan parquet default.web_sales -Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#6)] ReadSchema: struct (55) ColumnarToRow [codegen id : 16] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (56) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] (57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#43] +Left keys [1]: [ws_item_sk#48] Right keys [1]: [item_sk#22] Join condition: None (58) Project [codegen id : 16] -Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (59) Exchange -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: hashpartitioning(ws_bill_customer_sk#49, 5), ENSURE_REQUIREMENTS, [id=#53] (60) Sort [codegen id : 17] -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: [ws_bill_customer_sk#49 ASC NULLS FIRST], false, 0 (61) ReusedExchange [Reuses operator id: 40] -Output [3]: [c_customer_sk#29, sum#49, isEmpty#50] +Output [3]: [c_customer_sk#54, sum#55, isEmpty#56] (62) HashAggregate [codegen id : 20] -Input [3]: [c_customer_sk#29, sum#49, isEmpty#50] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#51] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#51 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] +Input [3]: [c_customer_sk#54, sum#55, isEmpty#56] +Keys [1]: [c_customer_sk#54] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))#59] +Results [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))#59 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] (63) Filter [codegen id : 20] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (64) Project [codegen id : 20] -Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] +Output [1]: [c_customer_sk#54] +Input [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] (65) Sort [codegen id : 20] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#54] +Arguments: [c_customer_sk#54 ASC NULLS FIRST], false, 0 (66) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#44] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ws_bill_customer_sk#49] +Right keys [1]: [c_customer_sk#54] Join condition: None (67) Project [codegen id : 22] -Output [3]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [3]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (68) ReusedExchange [Reuses operator id: 51] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#61] (69) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#47] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#52] +Right keys [1]: [d_date_sk#61] Join condition: None (70) Project [codegen id : 22] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2), true) AS sales#53] -Input [4]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#10] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#51 as decimal(12,2)))), DecimalType(18,2), true) AS sales#62] +Input [4]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52, d_date_sk#61] (71) Union (72) HashAggregate [codegen id : 23] -Input [1]: [sales#42] +Input [1]: [sales#47] Keys: [] -Functions [1]: [partial_sum(sales#42)] -Aggregate Attributes [2]: [sum#54, isEmpty#55] -Results [2]: [sum#56, isEmpty#57] +Functions [1]: [partial_sum(sales#47)] +Aggregate Attributes [2]: [sum#63, isEmpty#64] +Results [2]: [sum#65, isEmpty#66] (73) Exchange -Input [2]: [sum#56, isEmpty#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [sum#65, isEmpty#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] (74) HashAggregate [codegen id : 24] -Input [2]: [sum#56, isEmpty#57] +Input [2]: [sum#65, isEmpty#66] Keys: [] -Functions [1]: [sum(sales#42)] -Aggregate Attributes [1]: [sum(sales#42)#59] -Results [1]: [sum(sales#42)#59 AS sum(sales)#60] +Functions [1]: [sum(sales#47)] +Aggregate Attributes [1]: [sum(sales#47)#68] +Results [1]: [sum(sales#47)#68 AS sum(sales)#69] ===== Subqueries ===== @@ -420,7 +420,7 @@ ReusedExchange (75) (75) ReusedExchange [Reuses operator id: 51] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#43] Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 ReusedExchange (76) @@ -429,7 +429,7 @@ ReusedExchange (76) (76) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#10, d_date#11] -Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (98) +- Exchange (97) +- * HashAggregate (96) @@ -455,123 +455,123 @@ Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquer (77) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Output [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (78) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] (79) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] -Condition : isnotnull(ss_customer_sk#26) +Input [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] +Condition : isnotnull(ss_customer_sk#70) (80) Scan parquet default.customer -Output [1]: [c_customer_sk#29] +Output [1]: [c_customer_sk#75] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (81) ColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#29] +Input [1]: [c_customer_sk#75] (82) Filter [codegen id : 1] -Input [1]: [c_customer_sk#29] -Condition : isnotnull(c_customer_sk#29) +Input [1]: [c_customer_sk#75] +Condition : isnotnull(c_customer_sk#75) (83) BroadcastExchange -Input [1]: [c_customer_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#62] +Input [1]: [c_customer_sk#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] (84) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ss_customer_sk#70] +Right keys [1]: [c_customer_sk#75] Join condition: None (85) Project [codegen id : 3] -Output [4]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] -Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] +Output [4]: [ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75] +Input [5]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#12] +Output [2]: [d_date_sk#77, d_year#78] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#10, d_year#12] +Input [2]: [d_date_sk#77, d_year#78] (88) Filter [codegen id : 2] -Input [2]: [d_date_sk#10, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +Input [2]: [d_date_sk#77, d_year#78] +Condition : (d_year#78 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#77)) (89) Project [codegen id : 2] -Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_year#12] +Output [1]: [d_date_sk#77] +Input [2]: [d_date_sk#77, d_year#78] (90) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] +Input [1]: [d_date_sk#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#79] (91) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#73] +Right keys [1]: [d_date_sk#77] Join condition: None (92) Project [codegen id : 3] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Input [5]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29, d_date_sk#10] +Output [3]: [ss_quantity#71, ss_sales_price#72, c_customer_sk#75] +Input [5]: [ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75, d_date_sk#77] (93) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#64, isEmpty#65] -Results [3]: [c_customer_sk#29, sum#66, isEmpty#67] +Input [3]: [ss_quantity#71, ss_sales_price#72, c_customer_sk#75] +Keys [1]: [c_customer_sk#75] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#80, isEmpty#81] +Results [3]: [c_customer_sk#75, sum#82, isEmpty#83] (94) Exchange -Input [3]: [c_customer_sk#29, sum#66, isEmpty#67] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [3]: [c_customer_sk#75, sum#82, isEmpty#83] +Arguments: hashpartitioning(c_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#84] (95) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#29, sum#66, isEmpty#67] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS csales#70] +Input [3]: [c_customer_sk#75, sum#82, isEmpty#83] +Keys [1]: [c_customer_sk#75] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))#85] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))#85 AS csales#86] (96) HashAggregate [codegen id : 4] -Input [1]: [csales#70] +Input [1]: [csales#86] Keys: [] -Functions [1]: [partial_max(csales#70)] -Aggregate Attributes [1]: [max#71] -Results [1]: [max#72] +Functions [1]: [partial_max(csales#86)] +Aggregate Attributes [1]: [max#87] +Results [1]: [max#88] (97) Exchange -Input [1]: [max#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#73] +Input [1]: [max#88] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#89] (98) HashAggregate [codegen id : 5] -Input [1]: [max#72] +Input [1]: [max#88] Keys: [] -Functions [1]: [max(csales#70)] -Aggregate Attributes [1]: [max(csales#70)#74] -Results [1]: [max(csales#70)#74 AS tpcds_cmax#75] +Functions [1]: [max(csales#86)] +Aggregate Attributes [1]: [max(csales#86)#90] +Results [1]: [max(csales#86)#90 AS tpcds_cmax#91] -Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#61 +Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 ReusedExchange (99) (99) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#77] -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:6 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] 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 35a42fa256325..e75b223ea599a 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 @@ -290,145 +290,145 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (34) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (36) Filter [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Condition : isnotnull(ss_customer_sk#26) (37) Project [codegen id : 11] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (38) Exchange Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#29] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#30] (39) Sort [codegen id : 12] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (40) Scan parquet default.customer -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 13] -Input [1]: [c_customer_sk#30] +Input [1]: [c_customer_sk#31] (42) Filter [codegen id : 13] -Input [1]: [c_customer_sk#30] -Condition : isnotnull(c_customer_sk#30) +Input [1]: [c_customer_sk#31] +Condition : isnotnull(c_customer_sk#31) (43) Exchange -Input [1]: [c_customer_sk#30] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [1]: [c_customer_sk#31] +Arguments: hashpartitioning(c_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] (44) Sort [codegen id : 14] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 15] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#31] Join condition: None (46) Project [codegen id : 15] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#31] (47) HashAggregate [codegen id : 15] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Keys [1]: [c_customer_sk#31] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Aggregate Attributes [2]: [sum#33, isEmpty#34] +Results [3]: [c_customer_sk#31, sum#35, isEmpty#36] (48) HashAggregate [codegen id : 15] -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Keys [1]: [c_customer_sk#30] +Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] +Keys [1]: [c_customer_sk#31] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (49) Filter [codegen id : 15] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (50) Project [codegen id : 15] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#31] +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (51) Sort [codegen id : 15] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (52) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#31] Join condition: None (53) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_year#13, d_moy#40] +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 16] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (55) Filter [codegen id : 16] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#40)) AND (d_year#13 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#11)) +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) (56) Project [codegen id : 16] -Output [1]: [d_date_sk#11] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] +Output [1]: [d_date_sk#43] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (57) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] (58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#11] +Right keys [1]: [d_date_sk#43] Join condition: None (59) Project [codegen id : 17] Output [3]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4] -Input [5]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#11] +Input [5]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] (60) Scan parquet default.customer -Output [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Output [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 18] -Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] (62) Filter [codegen id : 18] -Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] -Condition : isnotnull(c_customer_sk#30) +Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] +Condition : isnotnull(c_customer_sk#47) (63) Exchange -Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] +Arguments: hashpartitioning(c_customer_sk#47, 5), ENSURE_REQUIREMENTS, [id=#50] (64) Sort [codegen id : 19] -Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] +Arguments: [c_customer_sk#47 ASC NULLS FIRST], false, 0 (65) ReusedExchange [Reuses operator id: 38] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] @@ -438,327 +438,327 @@ Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (67) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#31] (68) Sort [codegen id : 23] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (69) SortMergeJoin [codegen id : 24] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#31] Join condition: None (70) Project [codegen id : 24] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#31] (71) HashAggregate [codegen id : 24] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Keys [1]: [c_customer_sk#31] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Aggregate Attributes [2]: [sum#33, isEmpty#34] +Results [3]: [c_customer_sk#31, sum#35, isEmpty#36] (72) HashAggregate [codegen id : 24] -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Keys [1]: [c_customer_sk#30] +Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] +Keys [1]: [c_customer_sk#31] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (73) Filter [codegen id : 24] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (74) Project [codegen id : 24] -Output [1]: [c_customer_sk#30 AS c_customer_sk#30#45] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#31] +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (75) Sort [codegen id : 24] -Input [1]: [c_customer_sk#30#45] -Arguments: [c_customer_sk#30#45 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (76) SortMergeJoin -Left keys [1]: [c_customer_sk#30] -Right keys [1]: [c_customer_sk#30#45] +Left keys [1]: [c_customer_sk#47] +Right keys [1]: [c_customer_sk#31] Join condition: None (77) SortMergeJoin [codegen id : 25] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#47] Join condition: None (78) Project [codegen id : 25] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#42, c_last_name#43] -Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#30, c_first_name#42, c_last_name#43] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#48, c_last_name#49] +Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#47, c_first_name#48, c_last_name#49] (79) HashAggregate [codegen id : 25] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#42, c_last_name#43] -Keys [2]: [c_last_name#43, c_first_name#42] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#48, c_last_name#49] +Keys [2]: [c_last_name#49, c_first_name#48] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#46, isEmpty#47] -Results [4]: [c_last_name#43, c_first_name#42, sum#48, isEmpty#49] +Aggregate Attributes [2]: [sum#51, isEmpty#52] +Results [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] (80) Exchange -Input [4]: [c_last_name#43, c_first_name#42, sum#48, isEmpty#49] -Arguments: hashpartitioning(c_last_name#43, c_first_name#42, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] +Arguments: hashpartitioning(c_last_name#49, c_first_name#48, 5), ENSURE_REQUIREMENTS, [id=#55] (81) HashAggregate [codegen id : 26] -Input [4]: [c_last_name#43, c_first_name#42, sum#48, isEmpty#49] -Keys [2]: [c_last_name#43, c_first_name#42] +Input [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] +Keys [2]: [c_last_name#49, c_first_name#48] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#51] -Results [3]: [c_last_name#43, c_first_name#42, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#51 AS sales#52] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#56] +Results [3]: [c_last_name#49, c_first_name#48, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sales#57] (82) Scan parquet default.web_sales -Output [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Output [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#57), dynamicpruningexpression(ws_sold_date_sk#57 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (83) ColumnarToRow [codegen id : 27] -Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] (84) Filter [codegen id : 27] -Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Condition : isnotnull(ws_bill_customer_sk#54) +Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Condition : isnotnull(ws_bill_customer_sk#59) (85) Exchange -Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Arguments: hashpartitioning(ws_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Arguments: hashpartitioning(ws_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#63] (86) Sort [codegen id : 28] -Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Arguments: [ws_item_sk#53 ASC NULLS FIRST], false, 0 +Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Arguments: [ws_item_sk#58 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: unknown] -Output [2]: [ss_item_sk#8, d_date#12] +Output [2]: [ss_item_sk#64, d_date#65] (88) Sort [codegen id : 31] -Input [2]: [ss_item_sk#8, d_date#12] -Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 +Input [2]: [ss_item_sk#64, d_date#65] +Arguments: [ss_item_sk#64 ASC NULLS FIRST], false, 0 (89) ReusedExchange [Reuses operator id: 21] -Output [2]: [i_item_sk#16, i_item_desc#17] +Output [2]: [i_item_sk#66, i_item_desc#67] (90) Sort [codegen id : 33] -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16 ASC NULLS FIRST], false, 0 +Input [2]: [i_item_sk#66, i_item_desc#67] +Arguments: [i_item_sk#66 ASC NULLS FIRST], false, 0 (91) SortMergeJoin [codegen id : 34] -Left keys [1]: [ss_item_sk#8] -Right keys [1]: [i_item_sk#16] +Left keys [1]: [ss_item_sk#64] +Right keys [1]: [i_item_sk#66] Join condition: None (92) Project [codegen id : 34] -Output [3]: [d_date#12, i_item_sk#16, i_item_desc#17] -Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#16, i_item_desc#17] +Output [3]: [d_date#65, i_item_sk#66, i_item_desc#67] +Input [4]: [ss_item_sk#64, d_date#65, i_item_sk#66, i_item_desc#67] (93) HashAggregate [codegen id : 34] -Input [3]: [d_date#12, i_item_sk#16, i_item_desc#17] -Keys [3]: [substr(i_item_desc#17, 1, 30) AS substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12] +Input [3]: [d_date#65, i_item_sk#66, i_item_desc#67] +Keys [3]: [substr(i_item_desc#67, 1, 30) AS substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#60] -Results [4]: [substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12, count#61] +Aggregate Attributes [1]: [count#69] +Results [4]: [substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65, count#70] (94) HashAggregate [codegen id : 34] -Input [4]: [substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12, count#61] -Keys [3]: [substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12] +Input [4]: [substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65, count#70] +Keys [3]: [substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#62] -Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#62 AS count(1)#63] +Aggregate Attributes [1]: [count(1)#71] +Results [2]: [i_item_sk#66 AS item_sk#23, count(1)#71 AS count(1)#72] (95) Filter [codegen id : 34] -Input [2]: [item_sk#23, count(1)#63] -Condition : (count(1)#63 > 4) +Input [2]: [item_sk#23, count(1)#72] +Condition : (count(1)#72 > 4) (96) Project [codegen id : 34] Output [1]: [item_sk#23] -Input [2]: [item_sk#23, count(1)#63] +Input [2]: [item_sk#23, count(1)#72] (97) Sort [codegen id : 34] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ws_item_sk#53] +Left keys [1]: [ws_item_sk#58] Right keys [1]: [item_sk#23] Join condition: None (99) Project [codegen id : 35] -Output [4]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Output [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] (100) Exchange -Input [4]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Arguments: hashpartitioning(ws_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Arguments: hashpartitioning(ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#73] (101) Sort [codegen id : 36] -Input [4]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Arguments: [ws_bill_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Arguments: [ws_bill_customer_sk#59 ASC NULLS FIRST], false, 0 (102) ReusedExchange [Reuses operator id: 38] -Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Output [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] (103) Sort [codegen id : 38] -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] +Arguments: [ss_customer_sk#74 ASC NULLS FIRST], false, 0 (104) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#77] (105) Sort [codegen id : 40] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#77] +Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 (106) SortMergeJoin [codegen id : 41] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ss_customer_sk#74] +Right keys [1]: [c_customer_sk#77] Join condition: None (107) Project [codegen id : 41] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, c_customer_sk#77] (108) HashAggregate [codegen id : 41] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#65, isEmpty#66] -Results [3]: [c_customer_sk#30, sum#67, isEmpty#68] +Input [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Keys [1]: [c_customer_sk#77] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#78, isEmpty#79] +Results [3]: [c_customer_sk#77, sum#80, isEmpty#81] (109) HashAggregate [codegen id : 41] -Input [3]: [c_customer_sk#30, sum#67, isEmpty#68] -Keys [1]: [c_customer_sk#30] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] +Input [3]: [c_customer_sk#77, sum#80, isEmpty#81] +Keys [1]: [c_customer_sk#77] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82] +Results [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] (110) Filter [codegen id : 41] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (111) Project [codegen id : 41] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] +Output [1]: [c_customer_sk#77] +Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] (112) Sort [codegen id : 41] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#77] +Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 (113) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#54] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ws_bill_customer_sk#59] +Right keys [1]: [c_customer_sk#77] Join condition: None (114) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#84] (115) BroadcastHashJoin [codegen id : 43] -Left keys [1]: [ws_sold_date_sk#57] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ws_sold_date_sk#62] +Right keys [1]: [d_date_sk#84] Join condition: None (116) Project [codegen id : 43] -Output [3]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56] -Input [5]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57, d_date_sk#11] +Output [3]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61] +Input [5]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62, d_date_sk#84] (117) ReusedExchange [Reuses operator id: 63] -Output [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Output [3]: [c_customer_sk#85, c_first_name#86, c_last_name#87] (118) Sort [codegen id : 45] -Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#85, c_first_name#86, c_last_name#87] +Arguments: [c_customer_sk#85 ASC NULLS FIRST], false, 0 (119) ReusedExchange [Reuses operator id: 38] -Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Output [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] (120) Sort [codegen id : 47] -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] +Arguments: [ss_customer_sk#74 ASC NULLS FIRST], false, 0 (121) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#77] (122) Sort [codegen id : 49] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#77] +Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 (123) SortMergeJoin [codegen id : 50] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ss_customer_sk#74] +Right keys [1]: [c_customer_sk#77] Join condition: None (124) Project [codegen id : 50] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, c_customer_sk#77] (125) HashAggregate [codegen id : 50] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#65, isEmpty#66] -Results [3]: [c_customer_sk#30, sum#67, isEmpty#68] +Input [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Keys [1]: [c_customer_sk#77] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#78, isEmpty#79] +Results [3]: [c_customer_sk#77, sum#80, isEmpty#81] (126) HashAggregate [codegen id : 50] -Input [3]: [c_customer_sk#30, sum#67, isEmpty#68] -Keys [1]: [c_customer_sk#30] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] +Input [3]: [c_customer_sk#77, sum#80, isEmpty#81] +Keys [1]: [c_customer_sk#77] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82] +Results [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] (127) Filter [codegen id : 50] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (128) Project [codegen id : 50] -Output [1]: [c_customer_sk#30 AS c_customer_sk#30#71] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] +Output [1]: [c_customer_sk#77] +Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] (129) Sort [codegen id : 50] -Input [1]: [c_customer_sk#30#71] -Arguments: [c_customer_sk#30#71 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#77] +Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 (130) SortMergeJoin -Left keys [1]: [c_customer_sk#30] -Right keys [1]: [c_customer_sk#30#71] +Left keys [1]: [c_customer_sk#85] +Right keys [1]: [c_customer_sk#77] Join condition: None (131) SortMergeJoin [codegen id : 51] -Left keys [1]: [ws_bill_customer_sk#54] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ws_bill_customer_sk#59] +Right keys [1]: [c_customer_sk#85] Join condition: None (132) Project [codegen id : 51] -Output [4]: [ws_quantity#55, ws_list_price#56, c_first_name#42, c_last_name#43] -Input [6]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, c_customer_sk#30, c_first_name#42, c_last_name#43] +Output [4]: [ws_quantity#60, ws_list_price#61, c_first_name#86, c_last_name#87] +Input [6]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, c_customer_sk#85, c_first_name#86, c_last_name#87] (133) HashAggregate [codegen id : 51] -Input [4]: [ws_quantity#55, ws_list_price#56, c_first_name#42, c_last_name#43] -Keys [2]: [c_last_name#43, c_first_name#42] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#72, isEmpty#73] -Results [4]: [c_last_name#43, c_first_name#42, sum#74, isEmpty#75] +Input [4]: [ws_quantity#60, ws_list_price#61, c_first_name#86, c_last_name#87] +Keys [2]: [c_last_name#87, c_first_name#86] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#88, isEmpty#89] +Results [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] (134) Exchange -Input [4]: [c_last_name#43, c_first_name#42, sum#74, isEmpty#75] -Arguments: hashpartitioning(c_last_name#43, c_first_name#42, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] +Arguments: hashpartitioning(c_last_name#87, c_first_name#86, 5), ENSURE_REQUIREMENTS, [id=#92] (135) HashAggregate [codegen id : 52] -Input [4]: [c_last_name#43, c_first_name#42, sum#74, isEmpty#75] -Keys [2]: [c_last_name#43, c_first_name#42] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))#77] -Results [3]: [c_last_name#43, c_first_name#42, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))#77 AS sales#78] +Input [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] +Keys [2]: [c_last_name#87, c_first_name#86] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#93] +Results [3]: [c_last_name#87, c_first_name#86, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#93 AS sales#94] (136) Union (137) TakeOrderedAndProject -Input [3]: [c_last_name#43, c_first_name#42, sales#52] -Arguments: 100, [c_last_name#43 ASC NULLS FIRST, c_first_name#42 ASC NULLS FIRST, sales#52 ASC NULLS FIRST], [c_last_name#43, c_first_name#42, sales#52] +Input [3]: [c_last_name#49, c_first_name#48, sales#57] +Arguments: 100, [c_last_name#49 ASC NULLS FIRST, c_first_name#48 ASC NULLS FIRST, sales#57 ASC NULLS FIRST], [c_last_name#49, c_first_name#48, sales#57] ===== Subqueries ===== @@ -767,7 +767,7 @@ ReusedExchange (138) (138) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#43] Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 ReusedExchange (139) @@ -776,7 +776,7 @@ ReusedExchange (139) (139) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#11, d_date#12] -Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (163) +- Exchange (162) +- * HashAggregate (161) @@ -804,135 +804,135 @@ Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (140) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Output [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#79)] +PartitionFilters: [isnotnull(ss_sold_date_sk#98), dynamicpruningexpression(ss_sold_date_sk#98 IN dynamicpruning#99)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (141) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] (142) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] -Condition : isnotnull(ss_customer_sk#26) +Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] +Condition : isnotnull(ss_customer_sk#95) (143) Scan parquet default.date_dim -Output [2]: [d_date_sk#11, d_year#13] +Output [2]: [d_date_sk#100, d_year#101] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#11, d_year#13] +Input [2]: [d_date_sk#100, d_year#101] (145) Filter [codegen id : 1] -Input [2]: [d_date_sk#11, d_year#13] -Condition : (d_year#13 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) +Input [2]: [d_date_sk#100, d_year#101] +Condition : (d_year#101 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#100)) (146) Project [codegen id : 1] -Output [1]: [d_date_sk#11] -Input [2]: [d_date_sk#11, d_year#13] +Output [1]: [d_date_sk#100] +Input [2]: [d_date_sk#100, d_year#101] (147) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#80] +Input [1]: [d_date_sk#100] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#102] (148) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ss_sold_date_sk#98] +Right keys [1]: [d_date_sk#100] Join condition: None (149) Project [codegen id : 2] -Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9, d_date_sk#11] +Output [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] +Input [5]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98, d_date_sk#100] (150) Exchange -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] +Arguments: hashpartitioning(ss_customer_sk#95, 5), ENSURE_REQUIREMENTS, [id=#103] (151) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] +Arguments: [ss_customer_sk#95 ASC NULLS FIRST], false, 0 (152) Scan parquet default.customer -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#104] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (153) ColumnarToRow [codegen id : 4] -Input [1]: [c_customer_sk#30] +Input [1]: [c_customer_sk#104] (154) Filter [codegen id : 4] -Input [1]: [c_customer_sk#30] -Condition : isnotnull(c_customer_sk#30) +Input [1]: [c_customer_sk#104] +Condition : isnotnull(c_customer_sk#104) (155) Exchange -Input [1]: [c_customer_sk#30] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#82] +Input [1]: [c_customer_sk#104] +Arguments: hashpartitioning(c_customer_sk#104, 5), ENSURE_REQUIREMENTS, [id=#105] (156) Sort [codegen id : 5] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#104] +Arguments: [c_customer_sk#104 ASC NULLS FIRST], false, 0 (157) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ss_customer_sk#95] +Right keys [1]: [c_customer_sk#104] Join condition: None (158) Project [codegen id : 6] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#96, ss_sales_price#97, c_customer_sk#104] +Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, c_customer_sk#104] (159) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#83, isEmpty#84] -Results [3]: [c_customer_sk#30, sum#85, isEmpty#86] +Input [3]: [ss_quantity#96, ss_sales_price#97, c_customer_sk#104] +Keys [1]: [c_customer_sk#104] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#106, isEmpty#107] +Results [3]: [c_customer_sk#104, sum#108, isEmpty#109] (160) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#30, sum#85, isEmpty#86] -Keys [1]: [c_customer_sk#30] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#87] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#87 AS csales#88] +Input [3]: [c_customer_sk#104, sum#108, isEmpty#109] +Keys [1]: [c_customer_sk#104] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))#110] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))#110 AS csales#111] (161) HashAggregate [codegen id : 6] -Input [1]: [csales#88] +Input [1]: [csales#111] Keys: [] -Functions [1]: [partial_max(csales#88)] -Aggregate Attributes [1]: [max#89] -Results [1]: [max#90] +Functions [1]: [partial_max(csales#111)] +Aggregate Attributes [1]: [max#112] +Results [1]: [max#113] (162) Exchange -Input [1]: [max#90] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#91] +Input [1]: [max#113] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] (163) HashAggregate [codegen id : 7] -Input [1]: [max#90] +Input [1]: [max#113] Keys: [] -Functions [1]: [max(csales#88)] -Aggregate Attributes [1]: [max(csales#88)#92] -Results [1]: [max(csales#88)#92 AS tpcds_cmax#93] +Functions [1]: [max(csales#111)] +Aggregate Attributes [1]: [max(csales#111)#115] +Results [1]: [max(csales#111)#115 AS tpcds_cmax#116] -Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#79 +Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#98 IN dynamicpruning#99 ReusedExchange (164) (164) ReusedExchange [Reuses operator id: 147] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#100] -Subquery:5 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:5 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] -Subquery:6 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#57 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 110 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:7 Hosting operator id = 110 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] -Subquery:8 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:8 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] 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 385cf017e58c3..f5c64908c8fb9 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 @@ -240,340 +240,340 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (30) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (32) Filter [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Condition : isnotnull(ss_customer_sk#26) (33) Project [codegen id : 8] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (34) Scan parquet default.customer -Output [1]: [c_customer_sk#29] +Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 7] -Input [1]: [c_customer_sk#29] +Input [1]: [c_customer_sk#30] (36) Filter [codegen id : 7] -Input [1]: [c_customer_sk#29] -Condition : isnotnull(c_customer_sk#29) +Input [1]: [c_customer_sk#30] +Condition : isnotnull(c_customer_sk#30) (37) BroadcastExchange -Input [1]: [c_customer_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [1]: [c_customer_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (38) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#30] Join condition: None (39) Project [codegen id : 8] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (40) HashAggregate [codegen id : 8] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#31, isEmpty#32] -Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] (41) Exchange -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#36] (42) HashAggregate [codegen id : 9] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (43) Filter [codegen id : 9] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (44) Project [codegen id : 9] -Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (45) Sort [codegen id : 9] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (46) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#30] Join condition: None (47) Scan parquet default.customer -Output [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Output [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 10] -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] (49) Filter [codegen id : 10] -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Condition : isnotnull(c_customer_sk#29) +Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Condition : isnotnull(c_customer_sk#43) (50) Exchange -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Arguments: hashpartitioning(c_customer_sk#43, 5), ENSURE_REQUIREMENTS, [id=#46] (51) Sort [codegen id : 11] -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Arguments: [c_customer_sk#43 ASC NULLS FIRST], false, 0 (52) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Output [3]: [c_customer_sk#30, sum#34, isEmpty#35] (53) HashAggregate [codegen id : 14] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (54) Filter [codegen id : 14] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (55) Project [codegen id : 14] -Output [1]: [c_customer_sk#29 AS c_customer_sk#29#43] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (56) Sort [codegen id : 14] -Input [1]: [c_customer_sk#29#43] -Arguments: [c_customer_sk#29#43 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (57) SortMergeJoin -Left keys [1]: [c_customer_sk#29] -Right keys [1]: [c_customer_sk#29#43] +Left keys [1]: [c_customer_sk#43] +Right keys [1]: [c_customer_sk#30] Join condition: None (58) BroadcastExchange -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] +Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] (59) BroadcastHashJoin [codegen id : 16] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#43] Join condition: None (60) Project [codegen id : 16] -Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#40, c_last_name#41] -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#29, c_first_name#40, c_last_name#41] +Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#44, c_last_name#45] +Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#43, c_first_name#44, c_last_name#45] (61) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_year#12, d_moy#45] +Output [3]: [d_date_sk#48, d_year#49, d_moy#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 15] -Input [3]: [d_date_sk#10, d_year#12, d_moy#45] +Input [3]: [d_date_sk#48, d_year#49, d_moy#50] (63) Filter [codegen id : 15] -Input [3]: [d_date_sk#10, d_year#12, d_moy#45] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#45)) AND (d_year#12 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#10)) +Input [3]: [d_date_sk#48, d_year#49, d_moy#50] +Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2000)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#48)) (64) Project [codegen id : 15] -Output [1]: [d_date_sk#10] -Input [3]: [d_date_sk#10, d_year#12, d_moy#45] +Output [1]: [d_date_sk#48] +Input [3]: [d_date_sk#48, d_year#49, d_moy#50] (65) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Input [1]: [d_date_sk#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] (66) BroadcastHashJoin [codegen id : 16] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#10] +Right keys [1]: [d_date_sk#48] Join condition: None (67) Project [codegen id : 16] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#40, c_last_name#41] -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#40, c_last_name#41, d_date_sk#10] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#44, c_last_name#45] +Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#44, c_last_name#45, d_date_sk#48] (68) HashAggregate [codegen id : 16] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#40, c_last_name#41] -Keys [2]: [c_last_name#41, c_first_name#40] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#44, c_last_name#45] +Keys [2]: [c_last_name#45, c_first_name#44] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#47, isEmpty#48] -Results [4]: [c_last_name#41, c_first_name#40, sum#49, isEmpty#50] +Aggregate Attributes [2]: [sum#52, isEmpty#53] +Results [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] (69) Exchange -Input [4]: [c_last_name#41, c_first_name#40, sum#49, isEmpty#50] -Arguments: hashpartitioning(c_last_name#41, c_first_name#40, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] +Arguments: hashpartitioning(c_last_name#45, c_first_name#44, 5), ENSURE_REQUIREMENTS, [id=#56] (70) HashAggregate [codegen id : 17] -Input [4]: [c_last_name#41, c_first_name#40, sum#49, isEmpty#50] -Keys [2]: [c_last_name#41, c_first_name#40] +Input [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] +Keys [2]: [c_last_name#45, c_first_name#44] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#52] -Results [3]: [c_last_name#41, c_first_name#40, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#52 AS sales#53] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#57] +Results [3]: [c_last_name#45, c_first_name#44, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#57 AS sales#58] (71) Scan parquet default.web_sales -Output [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Output [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 22] -Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] (73) Filter [codegen id : 22] -Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] -Condition : isnotnull(ws_bill_customer_sk#55) +Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_bill_customer_sk#60) (74) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] (75) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_item_sk#54] +Left keys [1]: [ws_item_sk#59] Right keys [1]: [item_sk#22] Join condition: None (76) Project [codegen id : 22] -Output [4]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] -Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Output [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] (77) Exchange -Input [4]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_bill_customer_sk#55, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Arguments: hashpartitioning(ws_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#64] (78) Sort [codegen id : 23] -Input [4]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] -Arguments: [ws_bill_customer_sk#55 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Arguments: [ws_bill_customer_sk#60 ASC NULLS FIRST], false, 0 (79) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#29, sum#60, isEmpty#61] +Output [3]: [c_customer_sk#65, sum#66, isEmpty#67] (80) HashAggregate [codegen id : 26] -Input [3]: [c_customer_sk#29, sum#60, isEmpty#61] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] +Input [3]: [c_customer_sk#65, sum#66, isEmpty#67] +Keys [1]: [c_customer_sk#65] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70] +Results [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] (81) Filter [codegen id : 26] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (82) Project [codegen id : 26] -Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] +Output [1]: [c_customer_sk#65] +Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] (83) Sort [codegen id : 26] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#65] +Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 (84) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#55] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ws_bill_customer_sk#60] +Right keys [1]: [c_customer_sk#65] Join condition: None (85) ReusedExchange [Reuses operator id: 50] -Output [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Output [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] (86) Sort [codegen id : 28] -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] +Arguments: [c_customer_sk#72 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#29, sum#60, isEmpty#61] +Output [3]: [c_customer_sk#65, sum#66, isEmpty#67] (88) HashAggregate [codegen id : 31] -Input [3]: [c_customer_sk#29, sum#60, isEmpty#61] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] +Input [3]: [c_customer_sk#65, sum#66, isEmpty#67] +Keys [1]: [c_customer_sk#65] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70] +Results [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] (89) Filter [codegen id : 31] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (90) Project [codegen id : 31] -Output [1]: [c_customer_sk#29 AS c_customer_sk#29#64] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] +Output [1]: [c_customer_sk#65] +Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] (91) Sort [codegen id : 31] -Input [1]: [c_customer_sk#29#64] -Arguments: [c_customer_sk#29#64 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#65] +Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 (92) SortMergeJoin -Left keys [1]: [c_customer_sk#29] -Right keys [1]: [c_customer_sk#29#64] +Left keys [1]: [c_customer_sk#72] +Right keys [1]: [c_customer_sk#65] Join condition: None (93) BroadcastExchange -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] +Input [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] (94) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ws_bill_customer_sk#55] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ws_bill_customer_sk#60] +Right keys [1]: [c_customer_sk#72] Join condition: None (95) Project [codegen id : 33] -Output [5]: [ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58, c_first_name#40, c_last_name#41] -Input [7]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58, c_customer_sk#29, c_first_name#40, c_last_name#41] +Output [5]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74] +Input [7]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_customer_sk#72, c_first_name#73, c_last_name#74] (96) ReusedExchange [Reuses operator id: 65] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#76] (97) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ws_sold_date_sk#58] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#63] +Right keys [1]: [d_date_sk#76] Join condition: None (98) Project [codegen id : 33] -Output [4]: [ws_quantity#56, ws_list_price#57, c_first_name#40, c_last_name#41] -Input [6]: [ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58, c_first_name#40, c_last_name#41, d_date_sk#10] +Output [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74] +Input [6]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74, d_date_sk#76] (99) HashAggregate [codegen id : 33] -Input [4]: [ws_quantity#56, ws_list_price#57, c_first_name#40, c_last_name#41] -Keys [2]: [c_last_name#41, c_first_name#40] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#66, isEmpty#67] -Results [4]: [c_last_name#41, c_first_name#40, sum#68, isEmpty#69] +Input [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74] +Keys [2]: [c_last_name#74, c_first_name#73] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#77, isEmpty#78] +Results [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] (100) Exchange -Input [4]: [c_last_name#41, c_first_name#40, sum#68, isEmpty#69] -Arguments: hashpartitioning(c_last_name#41, c_first_name#40, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] +Arguments: hashpartitioning(c_last_name#74, c_first_name#73, 5), ENSURE_REQUIREMENTS, [id=#81] (101) HashAggregate [codegen id : 34] -Input [4]: [c_last_name#41, c_first_name#40, sum#68, isEmpty#69] -Keys [2]: [c_last_name#41, c_first_name#40] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))#71] -Results [3]: [c_last_name#41, c_first_name#40, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sales#72] +Input [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] +Keys [2]: [c_last_name#74, c_first_name#73] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#82] +Results [3]: [c_last_name#74, c_first_name#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sales#83] (102) Union (103) TakeOrderedAndProject -Input [3]: [c_last_name#41, c_first_name#40, sales#53] -Arguments: 100, [c_last_name#41 ASC NULLS FIRST, c_first_name#40 ASC NULLS FIRST, sales#53 ASC NULLS FIRST], [c_last_name#41, c_first_name#40, sales#53] +Input [3]: [c_last_name#45, c_first_name#44, sales#58] +Arguments: 100, [c_last_name#45 ASC NULLS FIRST, c_first_name#44 ASC NULLS FIRST, sales#58 ASC NULLS FIRST], [c_last_name#45, c_first_name#44, sales#58] ===== Subqueries ===== @@ -582,7 +582,7 @@ ReusedExchange (104) (104) ReusedExchange [Reuses operator id: 65] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#48] Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 ReusedExchange (105) @@ -591,7 +591,7 @@ ReusedExchange (105) (105) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#10, d_date#11] -Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (127) +- Exchange (126) +- * HashAggregate (125) @@ -617,127 +617,127 @@ Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquer (106) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Output [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (107) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] (108) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] -Condition : isnotnull(ss_customer_sk#26) +Input [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] +Condition : isnotnull(ss_customer_sk#84) (109) Scan parquet default.customer -Output [1]: [c_customer_sk#29] +Output [1]: [c_customer_sk#89] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (110) ColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#29] +Input [1]: [c_customer_sk#89] (111) Filter [codegen id : 1] -Input [1]: [c_customer_sk#29] -Condition : isnotnull(c_customer_sk#29) +Input [1]: [c_customer_sk#89] +Condition : isnotnull(c_customer_sk#89) (112) BroadcastExchange -Input [1]: [c_customer_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [1]: [c_customer_sk#89] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#90] (113) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ss_customer_sk#84] +Right keys [1]: [c_customer_sk#89] Join condition: None (114) Project [codegen id : 3] -Output [4]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] -Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] +Output [4]: [ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89] +Input [5]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89] (115) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#12] +Output [2]: [d_date_sk#91, d_year#92] 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 (116) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#10, d_year#12] +Input [2]: [d_date_sk#91, d_year#92] (117) Filter [codegen id : 2] -Input [2]: [d_date_sk#10, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +Input [2]: [d_date_sk#91, d_year#92] +Condition : (d_year#92 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#91)) (118) Project [codegen id : 2] -Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_year#12] +Output [1]: [d_date_sk#91] +Input [2]: [d_date_sk#91, d_year#92] (119) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#75] +Input [1]: [d_date_sk#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] (120) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#87] +Right keys [1]: [d_date_sk#91] Join condition: None (121) Project [codegen id : 3] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Input [5]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29, d_date_sk#10] +Output [3]: [ss_quantity#85, ss_sales_price#86, c_customer_sk#89] +Input [5]: [ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89, d_date_sk#91] (122) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#76, isEmpty#77] -Results [3]: [c_customer_sk#29, sum#78, isEmpty#79] +Input [3]: [ss_quantity#85, ss_sales_price#86, c_customer_sk#89] +Keys [1]: [c_customer_sk#89] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#94, isEmpty#95] +Results [3]: [c_customer_sk#89, sum#96, isEmpty#97] (123) Exchange -Input [3]: [c_customer_sk#29, sum#78, isEmpty#79] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [3]: [c_customer_sk#89, sum#96, isEmpty#97] +Arguments: hashpartitioning(c_customer_sk#89, 5), ENSURE_REQUIREMENTS, [id=#98] (124) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#29, sum#78, isEmpty#79] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#81] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#81 AS csales#82] +Input [3]: [c_customer_sk#89, sum#96, isEmpty#97] +Keys [1]: [c_customer_sk#89] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))#99] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))#99 AS csales#100] (125) HashAggregate [codegen id : 4] -Input [1]: [csales#82] +Input [1]: [csales#100] Keys: [] -Functions [1]: [partial_max(csales#82)] -Aggregate Attributes [1]: [max#83] -Results [1]: [max#84] +Functions [1]: [partial_max(csales#100)] +Aggregate Attributes [1]: [max#101] +Results [1]: [max#102] (126) Exchange -Input [1]: [max#84] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#85] +Input [1]: [max#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103] (127) HashAggregate [codegen id : 5] -Input [1]: [max#84] +Input [1]: [max#102] Keys: [] -Functions [1]: [max(csales#82)] -Aggregate Attributes [1]: [max(csales#82)#86] -Results [1]: [max(csales#82)#86 AS tpcds_cmax#87] +Functions [1]: [max(csales#100)] +Aggregate Attributes [1]: [max(csales#100)#104] +Results [1]: [max(csales#100)#104 AS tpcds_cmax#105] -Subquery:4 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#73 +Subquery:4 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#88 ReusedExchange (128) (128) ReusedExchange [Reuses operator id: 119] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#91] -Subquery:5 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:5 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] -Subquery:6 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:7 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] -Subquery:8 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:8 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] 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 431cf3ef92deb..7895ee1838ad7 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 @@ -340,248 +340,248 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (51) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] (53) Filter [codegen id : 2] -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)) +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) (54) Project [codegen id : 2] -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] +Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] (55) Scan parquet default.store -Output [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (56) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] (57) Filter [codegen id : 1] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -Condition : (((isnotnull(s_market_id#22) AND (s_market_id#22 = 8)) AND isnotnull(s_store_sk#20)) AND isnotnull(s_zip#24)) +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) (58) Project [codegen id : 1] -Output [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] (59) BroadcastExchange -Input [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] +Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] (60) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#20] +Left keys [1]: [ss_store_sk#52] +Right keys [1]: [s_store_sk#56] Join condition: None (61) Project [codegen id : 2] -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -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#20, s_store_name#21, s_state#23, s_zip#24] +Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] (62) Exchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#62] (63) Sort [codegen id : 3] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 (64) Scan parquet default.customer -Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Output [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (65) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] (66) Filter [codegen id : 4] -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_birth_country#18)) +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_birth_country#66)) (67) Exchange -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, [id=#67] (68) Sort [codegen id : 5] -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 +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: [c_customer_sk#63 ASC NULLS FIRST], false, 0 (69) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#15] +Left keys [1]: [ss_customer_sk#51] +Right keys [1]: [c_customer_sk#63] Join condition: None (70) Project [codegen id : 6] -Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Output [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Input [11]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] (71) Exchange -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#68] (72) Sort [codegen id : 7] -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 (73) Scan parquet default.item -Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (75) Filter [codegen id : 8] -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) +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Condition : isnotnull(i_item_sk#69) (76) Exchange -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=#54] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: hashpartitioning(i_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] (77) Sort [codegen id : 9] -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 +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: [i_item_sk#69 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ss_item_sk#50] +Right keys [1]: [i_item_sk#69] Join condition: None (79) Project [codegen id : 10] -Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (80) Exchange -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Arguments: hashpartitioning(c_birth_country#18, s_zip#24, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: hashpartitioning(c_birth_country#66, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] (81) Sort [codegen id : 11] -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Arguments: [c_birth_country#18 ASC NULLS FIRST, s_zip#24 ASC NULLS FIRST], false, 0 +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: [c_birth_country#66 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 (82) Scan parquet default.customer_address -Output [3]: [ca_state#26, ca_zip#27, ca_country#28] +Output [3]: [ca_state#77, ca_zip#78, ca_country#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (83) ColumnarToRow [codegen id : 12] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] (84) Filter [codegen id : 12] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Condition : (isnotnull(ca_country#28) AND isnotnull(ca_zip#27)) +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) (85) Exchange -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Arguments: hashpartitioning(upper(ca_country#28), ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: hashpartitioning(upper(ca_country#79), ca_zip#78, 5), ENSURE_REQUIREMENTS, [id=#80] (86) Sort [codegen id : 13] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Arguments: [upper(ca_country#28) ASC NULLS FIRST, ca_zip#27 ASC NULLS FIRST], false, 0 +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: [upper(ca_country#79) ASC NULLS FIRST, ca_zip#78 ASC NULLS FIRST], false, 0 (87) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#18, s_zip#24] -Right keys [2]: [upper(ca_country#28), ca_zip#27] +Left keys [2]: [c_birth_country#66, s_zip#60] +Right keys [2]: [upper(ca_country#79), ca_zip#78] Join condition: None (88) Project [codegen id : 14] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, ca_zip#27, ca_country#28] +Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, ca_zip#78, ca_country#79] (89) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#57] +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Arguments: hashpartitioning(cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint), 5), ENSURE_REQUIREMENTS, [id=#81] (90) Sort [codegen id : 15] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Arguments: [cast(ss_ticket_number#53 as bigint) ASC NULLS FIRST, cast(ss_item_sk#50 as bigint) ASC NULLS FIRST], false, 0 (91) Scan parquet default.store_returns -Output [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Output [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 16] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] (93) Filter [codegen id : 16] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] -Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Condition : (isnotnull(sr_ticket_number#83) AND isnotnull(sr_item_sk#82)) (94) Project [codegen id : 16] -Output [2]: [sr_item_sk#31, sr_ticket_number#32] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Output [2]: [sr_item_sk#82, sr_ticket_number#83] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] (95) Exchange -Input [2]: [sr_item_sk#31, sr_ticket_number#32] -Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: hashpartitioning(sr_ticket_number#83, sr_item_sk#82, 5), ENSURE_REQUIREMENTS, [id=#85] (96) Sort [codegen id : 17] -Input [2]: [sr_item_sk#31, sr_ticket_number#32] -Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 (97) SortMergeJoin [codegen id : 18] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] +Left keys [2]: [cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint)] +Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] Join condition: None (98) Project [codegen id : 18] -Output [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, sr_item_sk#31, sr_ticket_number#32] +Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, sr_item_sk#82, sr_ticket_number#83] (99) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] -Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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#59] -Results [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] +Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] +Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum#86] +Results [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] (100) Exchange -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Arguments: hashpartitioning(c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#88] (101) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] -Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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))#62] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#62,17,2) AS netpaid#39] +Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] (102) HashAggregate [codegen id : 19] Input [1]: [netpaid#39] Keys: [] Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#63, count#64] -Results [2]: [sum#65, count#66] +Aggregate Attributes [2]: [sum#90, count#91] +Results [2]: [sum#92, count#93] (103) Exchange -Input [2]: [sum#65, count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] +Input [2]: [sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] (104) HashAggregate [codegen id : 20] -Input [2]: [sum#65, count#66] +Input [2]: [sum#92, count#93] Keys: [] Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#68] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#68)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#69] +Aggregate Attributes [1]: [avg(netpaid#39)#95] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] 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 f73a5a5e052e3..0565ae76a440d 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 @@ -316,212 +316,212 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (48) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 1] -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] +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] (50) Filter [codegen id : 1] -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)) +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)) (51) Project [codegen id : 1] -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] +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] (52) Exchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#49] +Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Arguments: hashpartitioning(cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] (53) Sort [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Arguments: [cast(ss_ticket_number#52 as bigint) ASC NULLS FIRST, cast(ss_item_sk#49 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.store_returns -Output [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] (56) Filter [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] -Condition : (isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#8)) +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Condition : (isnotnull(sr_ticket_number#57) AND isnotnull(sr_item_sk#56)) (57) Project [codegen id : 3] -Output [2]: [sr_item_sk#8, sr_ticket_number#9] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [2]: [sr_item_sk#56, sr_ticket_number#57] +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] (58) Exchange -Input [2]: [sr_item_sk#8, sr_ticket_number#9] -Arguments: hashpartitioning(sr_ticket_number#9, sr_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [sr_item_sk#56, sr_ticket_number#57] +Arguments: hashpartitioning(sr_ticket_number#57, sr_item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#59] (59) Sort [codegen id : 4] -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 +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 (60) SortMergeJoin [codegen id : 9] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] +Left keys [2]: [cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint)] +Right keys [2]: [sr_ticket_number#57, sr_item_sk#56] Join condition: None (61) Project [codegen id : 9] -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] +Output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] +Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#56, sr_ticket_number#57] (62) Scan parquet default.store -Output [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (63) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] (64) Filter [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] -Condition : (((isnotnull(s_market_id#14) AND (s_market_id#14 = 8)) AND isnotnull(s_store_sk#12)) AND isnotnull(s_zip#16)) +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Condition : (((isnotnull(s_market_id#62) AND (s_market_id#62 = 8)) AND isnotnull(s_store_sk#60)) AND isnotnull(s_zip#64)) (65) Project [codegen id : 5] -Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] (66) BroadcastExchange -Input [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] +Input [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] (67) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] +Left keys [1]: [ss_store_sk#51] +Right keys [1]: [s_store_sk#60] Join condition: None (68) Project [codegen id : 9] -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] +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64] +Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] (69) Scan parquet default.item -Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Output [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] (71) Filter [codegen id : 6] -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) +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Condition : isnotnull(i_item_sk#66) (72) BroadcastExchange -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=#52] +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#72] (73) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [ss_item_sk#49] +Right keys [1]: [i_item_sk#66] Join condition: None (74) Project [codegen id : 9] -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] +Output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] (75) Scan parquet default.customer -Output [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Output [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (76) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] (77) Filter [codegen id : 7] -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_birth_country#28)) +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_birth_country#76)) (78) BroadcastExchange -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] (79) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#25] +Left keys [1]: [ss_customer_sk#50] +Right keys [1]: [c_customer_sk#73] Join condition: None (80) Project [codegen id : 9] -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] +Output [12]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76] +Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] (81) Scan parquet default.customer_address -Output [3]: [ca_state#30, ca_zip#31, ca_country#32] +Output [3]: [ca_state#78, ca_zip#79, ca_country#80] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (82) ColumnarToRow [codegen id : 8] -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] (83) Filter [codegen id : 8] -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] -Condition : (isnotnull(ca_country#32) AND isnotnull(ca_zip#31)) +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) (84) BroadcastExchange -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#54] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#81] (85) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#28, s_zip#16] -Right keys [2]: [upper(ca_country#32), ca_zip#31] +Left keys [2]: [c_birth_country#76, s_zip#64] +Right keys [2]: [upper(ca_country#80), ca_zip#79] Join condition: None (86) Project [codegen id : 9] -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] +Output [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] +Input [15]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76, ca_state#78, ca_zip#79, ca_country#80] (87) HashAggregate [codegen id : 9] -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#55] -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#56] +Input [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] +Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum#82] +Results [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] (88) Exchange -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#56] -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=#57] +Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +Arguments: hashpartitioning(c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, [id=#84] (89) HashAggregate [codegen id : 10] -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#56] -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))#58] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#58,17,2) AS netpaid#38] +Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] +Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#85] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#38] (90) HashAggregate [codegen id : 10] Input [1]: [netpaid#38] Keys: [] Functions [1]: [partial_avg(netpaid#38)] -Aggregate Attributes [2]: [sum#59, count#60] -Results [2]: [sum#61, count#62] +Aggregate Attributes [2]: [sum#86, count#87] +Results [2]: [sum#88, count#89] (91) Exchange -Input [2]: [sum#61, count#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] +Input [2]: [sum#88, count#89] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#90] (92) HashAggregate [codegen id : 11] -Input [2]: [sum#61, count#62] +Input [2]: [sum#88, count#89] Keys: [] Functions [1]: [avg(netpaid#38)] -Aggregate Attributes [1]: [avg(netpaid#38)#64] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#64)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#65] +Aggregate Attributes [1]: [avg(netpaid#38)#91] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] 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 163a4e7e3ddad..e45459d529104 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 @@ -340,248 +340,248 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (51) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] (53) Filter [codegen id : 2] -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)) +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) (54) Project [codegen id : 2] -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] +Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] (55) Scan parquet default.store -Output [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (56) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] (57) Filter [codegen id : 1] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -Condition : (((isnotnull(s_market_id#22) AND (s_market_id#22 = 8)) AND isnotnull(s_store_sk#20)) AND isnotnull(s_zip#24)) +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) (58) Project [codegen id : 1] -Output [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] (59) BroadcastExchange -Input [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] +Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] (60) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#20] +Left keys [1]: [ss_store_sk#52] +Right keys [1]: [s_store_sk#56] Join condition: None (61) Project [codegen id : 2] -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -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#20, s_store_name#21, s_state#23, s_zip#24] +Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] (62) Exchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#62] (63) Sort [codegen id : 3] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 (64) Scan parquet default.customer -Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Output [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (65) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] (66) Filter [codegen id : 4] -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_birth_country#18)) +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_birth_country#66)) (67) Exchange -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, [id=#67] (68) Sort [codegen id : 5] -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 +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: [c_customer_sk#63 ASC NULLS FIRST], false, 0 (69) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#15] +Left keys [1]: [ss_customer_sk#51] +Right keys [1]: [c_customer_sk#63] Join condition: None (70) Project [codegen id : 6] -Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Output [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Input [11]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] (71) Exchange -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#68] (72) Sort [codegen id : 7] -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 (73) Scan parquet default.item -Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (75) Filter [codegen id : 8] -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) +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Condition : isnotnull(i_item_sk#69) (76) Exchange -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=#54] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: hashpartitioning(i_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] (77) Sort [codegen id : 9] -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 +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: [i_item_sk#69 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ss_item_sk#50] +Right keys [1]: [i_item_sk#69] Join condition: None (79) Project [codegen id : 10] -Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (80) Exchange -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Arguments: hashpartitioning(c_birth_country#18, s_zip#24, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: hashpartitioning(c_birth_country#66, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] (81) Sort [codegen id : 11] -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Arguments: [c_birth_country#18 ASC NULLS FIRST, s_zip#24 ASC NULLS FIRST], false, 0 +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: [c_birth_country#66 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 (82) Scan parquet default.customer_address -Output [3]: [ca_state#26, ca_zip#27, ca_country#28] +Output [3]: [ca_state#77, ca_zip#78, ca_country#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (83) ColumnarToRow [codegen id : 12] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] (84) Filter [codegen id : 12] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Condition : (isnotnull(ca_country#28) AND isnotnull(ca_zip#27)) +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) (85) Exchange -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Arguments: hashpartitioning(upper(ca_country#28), ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: hashpartitioning(upper(ca_country#79), ca_zip#78, 5), ENSURE_REQUIREMENTS, [id=#80] (86) Sort [codegen id : 13] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Arguments: [upper(ca_country#28) ASC NULLS FIRST, ca_zip#27 ASC NULLS FIRST], false, 0 +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: [upper(ca_country#79) ASC NULLS FIRST, ca_zip#78 ASC NULLS FIRST], false, 0 (87) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#18, s_zip#24] -Right keys [2]: [upper(ca_country#28), ca_zip#27] +Left keys [2]: [c_birth_country#66, s_zip#60] +Right keys [2]: [upper(ca_country#79), ca_zip#78] Join condition: None (88) Project [codegen id : 14] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, ca_zip#27, ca_country#28] +Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, ca_zip#78, ca_country#79] (89) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#57] +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Arguments: hashpartitioning(cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint), 5), ENSURE_REQUIREMENTS, [id=#81] (90) Sort [codegen id : 15] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Arguments: [cast(ss_ticket_number#53 as bigint) ASC NULLS FIRST, cast(ss_item_sk#50 as bigint) ASC NULLS FIRST], false, 0 (91) Scan parquet default.store_returns -Output [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Output [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 16] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] (93) Filter [codegen id : 16] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] -Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Condition : (isnotnull(sr_ticket_number#83) AND isnotnull(sr_item_sk#82)) (94) Project [codegen id : 16] -Output [2]: [sr_item_sk#31, sr_ticket_number#32] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Output [2]: [sr_item_sk#82, sr_ticket_number#83] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] (95) Exchange -Input [2]: [sr_item_sk#31, sr_ticket_number#32] -Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: hashpartitioning(sr_ticket_number#83, sr_item_sk#82, 5), ENSURE_REQUIREMENTS, [id=#85] (96) Sort [codegen id : 17] -Input [2]: [sr_item_sk#31, sr_ticket_number#32] -Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 (97) SortMergeJoin [codegen id : 18] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] +Left keys [2]: [cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint)] +Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] Join condition: None (98) Project [codegen id : 18] -Output [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, sr_item_sk#31, sr_ticket_number#32] +Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, sr_item_sk#82, sr_ticket_number#83] (99) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] -Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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#59] -Results [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] +Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] +Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum#86] +Results [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] (100) Exchange -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Arguments: hashpartitioning(c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#88] (101) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] -Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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))#62] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#62,17,2) AS netpaid#39] +Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] (102) HashAggregate [codegen id : 19] Input [1]: [netpaid#39] Keys: [] Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#63, count#64] -Results [2]: [sum#65, count#66] +Aggregate Attributes [2]: [sum#90, count#91] +Results [2]: [sum#92, count#93] (103) Exchange -Input [2]: [sum#65, count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] +Input [2]: [sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] (104) HashAggregate [codegen id : 20] -Input [2]: [sum#65, count#66] +Input [2]: [sum#92, count#93] Keys: [] Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#68] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#68)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#69] +Aggregate Attributes [1]: [avg(netpaid#39)#95] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] 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 09830b1be656e..aeaf3f56b9e13 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 @@ -316,212 +316,212 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (48) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 1] -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] +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] (50) Filter [codegen id : 1] -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)) +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)) (51) Project [codegen id : 1] -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] +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] (52) Exchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#49] +Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Arguments: hashpartitioning(cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] (53) Sort [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Arguments: [cast(ss_ticket_number#52 as bigint) ASC NULLS FIRST, cast(ss_item_sk#49 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.store_returns -Output [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] (56) Filter [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] -Condition : (isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#8)) +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Condition : (isnotnull(sr_ticket_number#57) AND isnotnull(sr_item_sk#56)) (57) Project [codegen id : 3] -Output [2]: [sr_item_sk#8, sr_ticket_number#9] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [2]: [sr_item_sk#56, sr_ticket_number#57] +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] (58) Exchange -Input [2]: [sr_item_sk#8, sr_ticket_number#9] -Arguments: hashpartitioning(sr_ticket_number#9, sr_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [sr_item_sk#56, sr_ticket_number#57] +Arguments: hashpartitioning(sr_ticket_number#57, sr_item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#59] (59) Sort [codegen id : 4] -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 +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 (60) SortMergeJoin [codegen id : 9] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] +Left keys [2]: [cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint)] +Right keys [2]: [sr_ticket_number#57, sr_item_sk#56] Join condition: None (61) Project [codegen id : 9] -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] +Output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] +Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#56, sr_ticket_number#57] (62) Scan parquet default.store -Output [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (63) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] (64) Filter [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] -Condition : (((isnotnull(s_market_id#14) AND (s_market_id#14 = 8)) AND isnotnull(s_store_sk#12)) AND isnotnull(s_zip#16)) +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Condition : (((isnotnull(s_market_id#62) AND (s_market_id#62 = 8)) AND isnotnull(s_store_sk#60)) AND isnotnull(s_zip#64)) (65) Project [codegen id : 5] -Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] (66) BroadcastExchange -Input [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] +Input [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] (67) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] +Left keys [1]: [ss_store_sk#51] +Right keys [1]: [s_store_sk#60] Join condition: None (68) Project [codegen id : 9] -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] +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64] +Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] (69) Scan parquet default.item -Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Output [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] (71) Filter [codegen id : 6] -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) +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Condition : isnotnull(i_item_sk#66) (72) BroadcastExchange -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=#52] +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#72] (73) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [ss_item_sk#49] +Right keys [1]: [i_item_sk#66] Join condition: None (74) Project [codegen id : 9] -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] +Output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] (75) Scan parquet default.customer -Output [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Output [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (76) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] (77) Filter [codegen id : 7] -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_birth_country#28)) +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_birth_country#76)) (78) BroadcastExchange -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] (79) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#25] +Left keys [1]: [ss_customer_sk#50] +Right keys [1]: [c_customer_sk#73] Join condition: None (80) Project [codegen id : 9] -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] +Output [12]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76] +Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] (81) Scan parquet default.customer_address -Output [3]: [ca_state#30, ca_zip#31, ca_country#32] +Output [3]: [ca_state#78, ca_zip#79, ca_country#80] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (82) ColumnarToRow [codegen id : 8] -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] (83) Filter [codegen id : 8] -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] -Condition : (isnotnull(ca_country#32) AND isnotnull(ca_zip#31)) +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) (84) BroadcastExchange -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#54] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#81] (85) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#28, s_zip#16] -Right keys [2]: [upper(ca_country#32), ca_zip#31] +Left keys [2]: [c_birth_country#76, s_zip#64] +Right keys [2]: [upper(ca_country#80), ca_zip#79] Join condition: None (86) Project [codegen id : 9] -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] +Output [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] +Input [15]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76, ca_state#78, ca_zip#79, ca_country#80] (87) HashAggregate [codegen id : 9] -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#55] -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#56] +Input [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] +Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum#82] +Results [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] (88) Exchange -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#56] -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=#57] +Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +Arguments: hashpartitioning(c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, [id=#84] (89) HashAggregate [codegen id : 10] -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#56] -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))#58] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#58,17,2) AS netpaid#38] +Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] +Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#85] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#38] (90) HashAggregate [codegen id : 10] Input [1]: [netpaid#38] Keys: [] Functions [1]: [partial_avg(netpaid#38)] -Aggregate Attributes [2]: [sum#59, count#60] -Results [2]: [sum#61, count#62] +Aggregate Attributes [2]: [sum#86, count#87] +Results [2]: [sum#88, count#89] (91) Exchange -Input [2]: [sum#61, count#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] +Input [2]: [sum#88, count#89] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#90] (92) HashAggregate [codegen id : 11] -Input [2]: [sum#61, count#62] +Input [2]: [sum#88, count#89] Keys: [] Functions [1]: [avg(netpaid#38)] -Aggregate Attributes [1]: [avg(netpaid#38)#64] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#64)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#65] +Aggregate Attributes [1]: [avg(netpaid#38)#91] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt index 9f15199d8c8b7..eec45ea549531 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt @@ -126,306 +126,306 @@ Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#6 / 100.0) as decimal(11,6)) AS B1_LP#15, count(ss_list_price#3)#7 AS B1_CNT#16, count(ss_list_price#3)#12 AS B1_CNTD#17] (11) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] (13) Filter [codegen id : 3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 6)) AND (ss_quantity#1 <= 10)) AND ((((ss_list_price#3 >= 90.00) AND (ss_list_price#3 <= 100.00)) OR ((ss_coupon_amt#4 >= 2323.00) AND (ss_coupon_amt#4 <= 3323.00))) OR ((ss_wholesale_cost#2 >= 31.00) AND (ss_wholesale_cost#2 <= 51.00)))) +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Condition : (((isnotnull(ss_quantity#18) AND (ss_quantity#18 >= 6)) AND (ss_quantity#18 <= 10)) AND ((((ss_list_price#20 >= 90.00) AND (ss_list_price#20 <= 100.00)) OR ((ss_coupon_amt#21 >= 2323.00) AND (ss_coupon_amt#21 <= 3323.00))) OR ((ss_wholesale_cost#19 >= 31.00) AND (ss_wholesale_cost#19 <= 51.00)))) (14) Project [codegen id : 3] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#20] +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] (15) HashAggregate [codegen id : 3] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] -Results [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [1]: [ss_list_price#20] +Keys [1]: [ss_list_price#20] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#20)), partial_count(ss_list_price#20)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] +Results [4]: [ss_list_price#20, sum#25, count#26, count#27] (16) Exchange -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] +Arguments: hashpartitioning(ss_list_price#20, 5), ENSURE_REQUIREMENTS, [id=#28] (17) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] -Results [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] +Keys [1]: [ss_list_price#20] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] +Results [4]: [ss_list_price#20, sum#25, count#26, count#27] (18) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] -Results [4]: [sum#20, count#21, count#22, count#25] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20), partial_count(distinct ss_list_price#20)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] +Results [4]: [sum#25, count#26, count#27, count#30] (19) Exchange -Input [4]: [sum#20, count#21, count#22, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] +Input [4]: [sum#25, count#26, count#27, count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] (20) HashAggregate [codegen id : 5] -Input [4]: [sum#20, count#21, count#22, count#25] +Input [4]: [sum#25, count#26, count#27, count#30] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#18 / 100.0) as decimal(11,6)) AS B2_LP#27, count(ss_list_price#3)#19 AS B2_CNT#28, count(ss_list_price#3)#24 AS B2_CNTD#29] +Functions [3]: [avg(UnscaledValue(ss_list_price#20)), count(ss_list_price#20), count(distinct ss_list_price#20)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#20))#23 / 100.0) as decimal(11,6)) AS B2_LP#32, count(ss_list_price#20)#24 AS B2_CNT#33, count(ss_list_price#20)#29 AS B2_CNTD#34] (21) BroadcastExchange -Input [3]: [B2_LP#27, B2_CNT#28, B2_CNTD#29] -Arguments: IdentityBroadcastMode, [id=#30] +Input [3]: [B2_LP#32, B2_CNT#33, B2_CNTD#34] +Arguments: IdentityBroadcastMode, [id=#35] (22) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (23) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] (25) Filter [codegen id : 6] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 11)) AND (ss_quantity#1 <= 15)) AND ((((ss_list_price#3 >= 142.00) AND (ss_list_price#3 <= 152.00)) OR ((ss_coupon_amt#4 >= 12214.00) AND (ss_coupon_amt#4 <= 13214.00))) OR ((ss_wholesale_cost#2 >= 79.00) AND (ss_wholesale_cost#2 <= 99.00)))) +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Condition : (((isnotnull(ss_quantity#36) AND (ss_quantity#36 >= 11)) AND (ss_quantity#36 <= 15)) AND ((((ss_list_price#38 >= 142.00) AND (ss_list_price#38 <= 152.00)) OR ((ss_coupon_amt#39 >= 12214.00) AND (ss_coupon_amt#39 <= 13214.00))) OR ((ss_wholesale_cost#37 >= 79.00) AND (ss_wholesale_cost#37 <= 99.00)))) (26) Project [codegen id : 6] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#38] +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] (27) HashAggregate [codegen id : 6] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] -Results [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [1]: [ss_list_price#38] +Keys [1]: [ss_list_price#38] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#38)), partial_count(ss_list_price#38)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] +Results [4]: [ss_list_price#38, sum#43, count#44, count#45] (28) Exchange -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] +Arguments: hashpartitioning(ss_list_price#38, 5), ENSURE_REQUIREMENTS, [id=#46] (29) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] -Results [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] +Keys [1]: [ss_list_price#38] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] +Results [4]: [ss_list_price#38, sum#43, count#44, count#45] (30) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] -Results [4]: [sum#33, count#34, count#35, count#38] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38), partial_count(distinct ss_list_price#38)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] +Results [4]: [sum#43, count#44, count#45, count#48] (31) Exchange -Input [4]: [sum#33, count#34, count#35, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#39] +Input [4]: [sum#43, count#44, count#45, count#48] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] (32) HashAggregate [codegen id : 8] -Input [4]: [sum#33, count#34, count#35, count#38] +Input [4]: [sum#43, count#44, count#45, count#48] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#31 / 100.0) as decimal(11,6)) AS B3_LP#40, count(ss_list_price#3)#32 AS B3_CNT#41, count(ss_list_price#3)#37 AS B3_CNTD#42] +Functions [3]: [avg(UnscaledValue(ss_list_price#38)), count(ss_list_price#38), count(distinct ss_list_price#38)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#38))#41 / 100.0) as decimal(11,6)) AS B3_LP#50, count(ss_list_price#38)#42 AS B3_CNT#51, count(ss_list_price#38)#47 AS B3_CNTD#52] (33) BroadcastExchange -Input [3]: [B3_LP#40, B3_CNT#41, B3_CNTD#42] -Arguments: IdentityBroadcastMode, [id=#43] +Input [3]: [B3_LP#50, B3_CNT#51, B3_CNTD#52] +Arguments: IdentityBroadcastMode, [id=#53] (34) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (35) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] ReadSchema: struct (36) ColumnarToRow [codegen id : 9] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] (37) Filter [codegen id : 9] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 16)) AND (ss_quantity#1 <= 20)) AND ((((ss_list_price#3 >= 135.00) AND (ss_list_price#3 <= 145.00)) OR ((ss_coupon_amt#4 >= 6071.00) AND (ss_coupon_amt#4 <= 7071.00))) OR ((ss_wholesale_cost#2 >= 38.00) AND (ss_wholesale_cost#2 <= 58.00)))) +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Condition : (((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 16)) AND (ss_quantity#54 <= 20)) AND ((((ss_list_price#56 >= 135.00) AND (ss_list_price#56 <= 145.00)) OR ((ss_coupon_amt#57 >= 6071.00) AND (ss_coupon_amt#57 <= 7071.00))) OR ((ss_wholesale_cost#55 >= 38.00) AND (ss_wholesale_cost#55 <= 58.00)))) (38) Project [codegen id : 9] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#56] +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] (39) HashAggregate [codegen id : 9] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] -Results [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [1]: [ss_list_price#56] +Keys [1]: [ss_list_price#56] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#56)), partial_count(ss_list_price#56)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] +Results [4]: [ss_list_price#56, sum#61, count#62, count#63] (40) Exchange -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] +Arguments: hashpartitioning(ss_list_price#56, 5), ENSURE_REQUIREMENTS, [id=#64] (41) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] -Results [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] +Keys [1]: [ss_list_price#56] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] +Results [4]: [ss_list_price#56, sum#61, count#62, count#63] (42) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] -Results [4]: [sum#46, count#47, count#48, count#51] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56), partial_count(distinct ss_list_price#56)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] +Results [4]: [sum#61, count#62, count#63, count#66] (43) Exchange -Input [4]: [sum#46, count#47, count#48, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] +Input [4]: [sum#61, count#62, count#63, count#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] (44) HashAggregate [codegen id : 11] -Input [4]: [sum#46, count#47, count#48, count#51] +Input [4]: [sum#61, count#62, count#63, count#66] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#44 / 100.0) as decimal(11,6)) AS B4_LP#53, count(ss_list_price#3)#45 AS B4_CNT#54, count(ss_list_price#3)#50 AS B4_CNTD#55] +Functions [3]: [avg(UnscaledValue(ss_list_price#56)), count(ss_list_price#56), count(distinct ss_list_price#56)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#56))#59 / 100.0) as decimal(11,6)) AS B4_LP#68, count(ss_list_price#56)#60 AS B4_CNT#69, count(ss_list_price#56)#65 AS B4_CNTD#70] (45) BroadcastExchange -Input [3]: [B4_LP#53, B4_CNT#54, B4_CNTD#55] -Arguments: IdentityBroadcastMode, [id=#56] +Input [3]: [B4_LP#68, B4_CNT#69, B4_CNTD#70] +Arguments: IdentityBroadcastMode, [id=#71] (46) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (47) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] ReadSchema: struct (48) ColumnarToRow [codegen id : 12] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] (49) Filter [codegen id : 12] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 21)) AND (ss_quantity#1 <= 25)) AND ((((ss_list_price#3 >= 122.00) AND (ss_list_price#3 <= 132.00)) OR ((ss_coupon_amt#4 >= 836.00) AND (ss_coupon_amt#4 <= 1836.00))) OR ((ss_wholesale_cost#2 >= 17.00) AND (ss_wholesale_cost#2 <= 37.00)))) +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Condition : (((isnotnull(ss_quantity#72) AND (ss_quantity#72 >= 21)) AND (ss_quantity#72 <= 25)) AND ((((ss_list_price#74 >= 122.00) AND (ss_list_price#74 <= 132.00)) OR ((ss_coupon_amt#75 >= 836.00) AND (ss_coupon_amt#75 <= 1836.00))) OR ((ss_wholesale_cost#73 >= 17.00) AND (ss_wholesale_cost#73 <= 37.00)))) (50) Project [codegen id : 12] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#74] +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] (51) HashAggregate [codegen id : 12] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] -Results [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [1]: [ss_list_price#74] +Keys [1]: [ss_list_price#74] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#74)), partial_count(ss_list_price#74)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] +Results [4]: [ss_list_price#74, sum#79, count#80, count#81] (52) Exchange -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] +Arguments: hashpartitioning(ss_list_price#74, 5), ENSURE_REQUIREMENTS, [id=#82] (53) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] -Results [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] +Keys [1]: [ss_list_price#74] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] +Results [4]: [ss_list_price#74, sum#79, count#80, count#81] (54) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] -Results [4]: [sum#59, count#60, count#61, count#64] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74), partial_count(distinct ss_list_price#74)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] +Results [4]: [sum#79, count#80, count#81, count#84] (55) Exchange -Input [4]: [sum#59, count#60, count#61, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] +Input [4]: [sum#79, count#80, count#81, count#84] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#85] (56) HashAggregate [codegen id : 14] -Input [4]: [sum#59, count#60, count#61, count#64] +Input [4]: [sum#79, count#80, count#81, count#84] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#57 / 100.0) as decimal(11,6)) AS B5_LP#66, count(ss_list_price#3)#58 AS B5_CNT#67, count(ss_list_price#3)#63 AS B5_CNTD#68] +Functions [3]: [avg(UnscaledValue(ss_list_price#74)), count(ss_list_price#74), count(distinct ss_list_price#74)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#74))#77 / 100.0) as decimal(11,6)) AS B5_LP#86, count(ss_list_price#74)#78 AS B5_CNT#87, count(ss_list_price#74)#83 AS B5_CNTD#88] (57) BroadcastExchange -Input [3]: [B5_LP#66, B5_CNT#67, B5_CNTD#68] -Arguments: IdentityBroadcastMode, [id=#69] +Input [3]: [B5_LP#86, B5_CNT#87, B5_CNTD#88] +Arguments: IdentityBroadcastMode, [id=#89] (58) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (59) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] ReadSchema: struct (60) ColumnarToRow [codegen id : 15] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] (61) Filter [codegen id : 15] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 26)) AND (ss_quantity#1 <= 30)) AND ((((ss_list_price#3 >= 154.00) AND (ss_list_price#3 <= 164.00)) OR ((ss_coupon_amt#4 >= 7326.00) AND (ss_coupon_amt#4 <= 8326.00))) OR ((ss_wholesale_cost#2 >= 7.00) AND (ss_wholesale_cost#2 <= 27.00)))) +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Condition : (((isnotnull(ss_quantity#90) AND (ss_quantity#90 >= 26)) AND (ss_quantity#90 <= 30)) AND ((((ss_list_price#92 >= 154.00) AND (ss_list_price#92 <= 164.00)) OR ((ss_coupon_amt#93 >= 7326.00) AND (ss_coupon_amt#93 <= 8326.00))) OR ((ss_wholesale_cost#91 >= 7.00) AND (ss_wholesale_cost#91 <= 27.00)))) (62) Project [codegen id : 15] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#92] +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] (63) HashAggregate [codegen id : 15] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] -Results [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [1]: [ss_list_price#92] +Keys [1]: [ss_list_price#92] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#92)), partial_count(ss_list_price#92)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] +Results [4]: [ss_list_price#92, sum#97, count#98, count#99] (64) Exchange -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] +Arguments: hashpartitioning(ss_list_price#92, 5), ENSURE_REQUIREMENTS, [id=#100] (65) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] -Results [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] +Keys [1]: [ss_list_price#92] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] +Results [4]: [ss_list_price#92, sum#97, count#98, count#99] (66) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] -Results [4]: [sum#72, count#73, count#74, count#77] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92), partial_count(distinct ss_list_price#92)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] +Results [4]: [sum#97, count#98, count#99, count#102] (67) Exchange -Input [4]: [sum#72, count#73, count#74, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] +Input [4]: [sum#97, count#98, count#99, count#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103] (68) HashAggregate [codegen id : 17] -Input [4]: [sum#72, count#73, count#74, count#77] +Input [4]: [sum#97, count#98, count#99, count#102] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#70 / 100.0) as decimal(11,6)) AS B6_LP#79, count(ss_list_price#3)#71 AS B6_CNT#80, count(ss_list_price#3)#76 AS B6_CNTD#81] +Functions [3]: [avg(UnscaledValue(ss_list_price#92)), count(ss_list_price#92), count(distinct ss_list_price#92)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#92))#95 / 100.0) as decimal(11,6)) AS B6_LP#104, count(ss_list_price#92)#96 AS B6_CNT#105, count(ss_list_price#92)#101 AS B6_CNTD#106] (69) BroadcastExchange -Input [3]: [B6_LP#79, B6_CNT#80, B6_CNTD#81] -Arguments: IdentityBroadcastMode, [id=#82] +Input [3]: [B6_LP#104, B6_CNT#105, B6_CNTD#106] +Arguments: IdentityBroadcastMode, [id=#107] (70) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt index 9f15199d8c8b7..eec45ea549531 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt @@ -126,306 +126,306 @@ Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#6 / 100.0) as decimal(11,6)) AS B1_LP#15, count(ss_list_price#3)#7 AS B1_CNT#16, count(ss_list_price#3)#12 AS B1_CNTD#17] (11) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] (13) Filter [codegen id : 3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 6)) AND (ss_quantity#1 <= 10)) AND ((((ss_list_price#3 >= 90.00) AND (ss_list_price#3 <= 100.00)) OR ((ss_coupon_amt#4 >= 2323.00) AND (ss_coupon_amt#4 <= 3323.00))) OR ((ss_wholesale_cost#2 >= 31.00) AND (ss_wholesale_cost#2 <= 51.00)))) +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Condition : (((isnotnull(ss_quantity#18) AND (ss_quantity#18 >= 6)) AND (ss_quantity#18 <= 10)) AND ((((ss_list_price#20 >= 90.00) AND (ss_list_price#20 <= 100.00)) OR ((ss_coupon_amt#21 >= 2323.00) AND (ss_coupon_amt#21 <= 3323.00))) OR ((ss_wholesale_cost#19 >= 31.00) AND (ss_wholesale_cost#19 <= 51.00)))) (14) Project [codegen id : 3] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#20] +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] (15) HashAggregate [codegen id : 3] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] -Results [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [1]: [ss_list_price#20] +Keys [1]: [ss_list_price#20] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#20)), partial_count(ss_list_price#20)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] +Results [4]: [ss_list_price#20, sum#25, count#26, count#27] (16) Exchange -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] +Arguments: hashpartitioning(ss_list_price#20, 5), ENSURE_REQUIREMENTS, [id=#28] (17) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] -Results [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] +Keys [1]: [ss_list_price#20] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] +Results [4]: [ss_list_price#20, sum#25, count#26, count#27] (18) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] -Results [4]: [sum#20, count#21, count#22, count#25] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20), partial_count(distinct ss_list_price#20)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] +Results [4]: [sum#25, count#26, count#27, count#30] (19) Exchange -Input [4]: [sum#20, count#21, count#22, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] +Input [4]: [sum#25, count#26, count#27, count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] (20) HashAggregate [codegen id : 5] -Input [4]: [sum#20, count#21, count#22, count#25] +Input [4]: [sum#25, count#26, count#27, count#30] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#18 / 100.0) as decimal(11,6)) AS B2_LP#27, count(ss_list_price#3)#19 AS B2_CNT#28, count(ss_list_price#3)#24 AS B2_CNTD#29] +Functions [3]: [avg(UnscaledValue(ss_list_price#20)), count(ss_list_price#20), count(distinct ss_list_price#20)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#20))#23 / 100.0) as decimal(11,6)) AS B2_LP#32, count(ss_list_price#20)#24 AS B2_CNT#33, count(ss_list_price#20)#29 AS B2_CNTD#34] (21) BroadcastExchange -Input [3]: [B2_LP#27, B2_CNT#28, B2_CNTD#29] -Arguments: IdentityBroadcastMode, [id=#30] +Input [3]: [B2_LP#32, B2_CNT#33, B2_CNTD#34] +Arguments: IdentityBroadcastMode, [id=#35] (22) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (23) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] (25) Filter [codegen id : 6] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 11)) AND (ss_quantity#1 <= 15)) AND ((((ss_list_price#3 >= 142.00) AND (ss_list_price#3 <= 152.00)) OR ((ss_coupon_amt#4 >= 12214.00) AND (ss_coupon_amt#4 <= 13214.00))) OR ((ss_wholesale_cost#2 >= 79.00) AND (ss_wholesale_cost#2 <= 99.00)))) +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Condition : (((isnotnull(ss_quantity#36) AND (ss_quantity#36 >= 11)) AND (ss_quantity#36 <= 15)) AND ((((ss_list_price#38 >= 142.00) AND (ss_list_price#38 <= 152.00)) OR ((ss_coupon_amt#39 >= 12214.00) AND (ss_coupon_amt#39 <= 13214.00))) OR ((ss_wholesale_cost#37 >= 79.00) AND (ss_wholesale_cost#37 <= 99.00)))) (26) Project [codegen id : 6] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#38] +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] (27) HashAggregate [codegen id : 6] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] -Results [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [1]: [ss_list_price#38] +Keys [1]: [ss_list_price#38] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#38)), partial_count(ss_list_price#38)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] +Results [4]: [ss_list_price#38, sum#43, count#44, count#45] (28) Exchange -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] +Arguments: hashpartitioning(ss_list_price#38, 5), ENSURE_REQUIREMENTS, [id=#46] (29) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] -Results [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] +Keys [1]: [ss_list_price#38] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] +Results [4]: [ss_list_price#38, sum#43, count#44, count#45] (30) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] -Results [4]: [sum#33, count#34, count#35, count#38] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38), partial_count(distinct ss_list_price#38)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] +Results [4]: [sum#43, count#44, count#45, count#48] (31) Exchange -Input [4]: [sum#33, count#34, count#35, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#39] +Input [4]: [sum#43, count#44, count#45, count#48] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] (32) HashAggregate [codegen id : 8] -Input [4]: [sum#33, count#34, count#35, count#38] +Input [4]: [sum#43, count#44, count#45, count#48] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#31 / 100.0) as decimal(11,6)) AS B3_LP#40, count(ss_list_price#3)#32 AS B3_CNT#41, count(ss_list_price#3)#37 AS B3_CNTD#42] +Functions [3]: [avg(UnscaledValue(ss_list_price#38)), count(ss_list_price#38), count(distinct ss_list_price#38)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#38))#41 / 100.0) as decimal(11,6)) AS B3_LP#50, count(ss_list_price#38)#42 AS B3_CNT#51, count(ss_list_price#38)#47 AS B3_CNTD#52] (33) BroadcastExchange -Input [3]: [B3_LP#40, B3_CNT#41, B3_CNTD#42] -Arguments: IdentityBroadcastMode, [id=#43] +Input [3]: [B3_LP#50, B3_CNT#51, B3_CNTD#52] +Arguments: IdentityBroadcastMode, [id=#53] (34) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (35) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] ReadSchema: struct (36) ColumnarToRow [codegen id : 9] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] (37) Filter [codegen id : 9] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 16)) AND (ss_quantity#1 <= 20)) AND ((((ss_list_price#3 >= 135.00) AND (ss_list_price#3 <= 145.00)) OR ((ss_coupon_amt#4 >= 6071.00) AND (ss_coupon_amt#4 <= 7071.00))) OR ((ss_wholesale_cost#2 >= 38.00) AND (ss_wholesale_cost#2 <= 58.00)))) +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Condition : (((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 16)) AND (ss_quantity#54 <= 20)) AND ((((ss_list_price#56 >= 135.00) AND (ss_list_price#56 <= 145.00)) OR ((ss_coupon_amt#57 >= 6071.00) AND (ss_coupon_amt#57 <= 7071.00))) OR ((ss_wholesale_cost#55 >= 38.00) AND (ss_wholesale_cost#55 <= 58.00)))) (38) Project [codegen id : 9] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#56] +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] (39) HashAggregate [codegen id : 9] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] -Results [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [1]: [ss_list_price#56] +Keys [1]: [ss_list_price#56] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#56)), partial_count(ss_list_price#56)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] +Results [4]: [ss_list_price#56, sum#61, count#62, count#63] (40) Exchange -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] +Arguments: hashpartitioning(ss_list_price#56, 5), ENSURE_REQUIREMENTS, [id=#64] (41) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] -Results [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] +Keys [1]: [ss_list_price#56] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] +Results [4]: [ss_list_price#56, sum#61, count#62, count#63] (42) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] -Results [4]: [sum#46, count#47, count#48, count#51] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56), partial_count(distinct ss_list_price#56)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] +Results [4]: [sum#61, count#62, count#63, count#66] (43) Exchange -Input [4]: [sum#46, count#47, count#48, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] +Input [4]: [sum#61, count#62, count#63, count#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] (44) HashAggregate [codegen id : 11] -Input [4]: [sum#46, count#47, count#48, count#51] +Input [4]: [sum#61, count#62, count#63, count#66] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#44 / 100.0) as decimal(11,6)) AS B4_LP#53, count(ss_list_price#3)#45 AS B4_CNT#54, count(ss_list_price#3)#50 AS B4_CNTD#55] +Functions [3]: [avg(UnscaledValue(ss_list_price#56)), count(ss_list_price#56), count(distinct ss_list_price#56)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#56))#59 / 100.0) as decimal(11,6)) AS B4_LP#68, count(ss_list_price#56)#60 AS B4_CNT#69, count(ss_list_price#56)#65 AS B4_CNTD#70] (45) BroadcastExchange -Input [3]: [B4_LP#53, B4_CNT#54, B4_CNTD#55] -Arguments: IdentityBroadcastMode, [id=#56] +Input [3]: [B4_LP#68, B4_CNT#69, B4_CNTD#70] +Arguments: IdentityBroadcastMode, [id=#71] (46) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (47) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] ReadSchema: struct (48) ColumnarToRow [codegen id : 12] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] (49) Filter [codegen id : 12] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 21)) AND (ss_quantity#1 <= 25)) AND ((((ss_list_price#3 >= 122.00) AND (ss_list_price#3 <= 132.00)) OR ((ss_coupon_amt#4 >= 836.00) AND (ss_coupon_amt#4 <= 1836.00))) OR ((ss_wholesale_cost#2 >= 17.00) AND (ss_wholesale_cost#2 <= 37.00)))) +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Condition : (((isnotnull(ss_quantity#72) AND (ss_quantity#72 >= 21)) AND (ss_quantity#72 <= 25)) AND ((((ss_list_price#74 >= 122.00) AND (ss_list_price#74 <= 132.00)) OR ((ss_coupon_amt#75 >= 836.00) AND (ss_coupon_amt#75 <= 1836.00))) OR ((ss_wholesale_cost#73 >= 17.00) AND (ss_wholesale_cost#73 <= 37.00)))) (50) Project [codegen id : 12] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#74] +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] (51) HashAggregate [codegen id : 12] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] -Results [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [1]: [ss_list_price#74] +Keys [1]: [ss_list_price#74] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#74)), partial_count(ss_list_price#74)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] +Results [4]: [ss_list_price#74, sum#79, count#80, count#81] (52) Exchange -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] +Arguments: hashpartitioning(ss_list_price#74, 5), ENSURE_REQUIREMENTS, [id=#82] (53) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] -Results [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] +Keys [1]: [ss_list_price#74] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] +Results [4]: [ss_list_price#74, sum#79, count#80, count#81] (54) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] -Results [4]: [sum#59, count#60, count#61, count#64] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74), partial_count(distinct ss_list_price#74)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] +Results [4]: [sum#79, count#80, count#81, count#84] (55) Exchange -Input [4]: [sum#59, count#60, count#61, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] +Input [4]: [sum#79, count#80, count#81, count#84] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#85] (56) HashAggregate [codegen id : 14] -Input [4]: [sum#59, count#60, count#61, count#64] +Input [4]: [sum#79, count#80, count#81, count#84] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#57 / 100.0) as decimal(11,6)) AS B5_LP#66, count(ss_list_price#3)#58 AS B5_CNT#67, count(ss_list_price#3)#63 AS B5_CNTD#68] +Functions [3]: [avg(UnscaledValue(ss_list_price#74)), count(ss_list_price#74), count(distinct ss_list_price#74)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#74))#77 / 100.0) as decimal(11,6)) AS B5_LP#86, count(ss_list_price#74)#78 AS B5_CNT#87, count(ss_list_price#74)#83 AS B5_CNTD#88] (57) BroadcastExchange -Input [3]: [B5_LP#66, B5_CNT#67, B5_CNTD#68] -Arguments: IdentityBroadcastMode, [id=#69] +Input [3]: [B5_LP#86, B5_CNT#87, B5_CNTD#88] +Arguments: IdentityBroadcastMode, [id=#89] (58) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (59) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] ReadSchema: struct (60) ColumnarToRow [codegen id : 15] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] (61) Filter [codegen id : 15] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 26)) AND (ss_quantity#1 <= 30)) AND ((((ss_list_price#3 >= 154.00) AND (ss_list_price#3 <= 164.00)) OR ((ss_coupon_amt#4 >= 7326.00) AND (ss_coupon_amt#4 <= 8326.00))) OR ((ss_wholesale_cost#2 >= 7.00) AND (ss_wholesale_cost#2 <= 27.00)))) +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Condition : (((isnotnull(ss_quantity#90) AND (ss_quantity#90 >= 26)) AND (ss_quantity#90 <= 30)) AND ((((ss_list_price#92 >= 154.00) AND (ss_list_price#92 <= 164.00)) OR ((ss_coupon_amt#93 >= 7326.00) AND (ss_coupon_amt#93 <= 8326.00))) OR ((ss_wholesale_cost#91 >= 7.00) AND (ss_wholesale_cost#91 <= 27.00)))) (62) Project [codegen id : 15] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#92] +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] (63) HashAggregate [codegen id : 15] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] -Results [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [1]: [ss_list_price#92] +Keys [1]: [ss_list_price#92] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#92)), partial_count(ss_list_price#92)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] +Results [4]: [ss_list_price#92, sum#97, count#98, count#99] (64) Exchange -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] +Arguments: hashpartitioning(ss_list_price#92, 5), ENSURE_REQUIREMENTS, [id=#100] (65) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] -Results [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] +Keys [1]: [ss_list_price#92] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] +Results [4]: [ss_list_price#92, sum#97, count#98, count#99] (66) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] -Results [4]: [sum#72, count#73, count#74, count#77] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92), partial_count(distinct ss_list_price#92)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] +Results [4]: [sum#97, count#98, count#99, count#102] (67) Exchange -Input [4]: [sum#72, count#73, count#74, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] +Input [4]: [sum#97, count#98, count#99, count#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103] (68) HashAggregate [codegen id : 17] -Input [4]: [sum#72, count#73, count#74, count#77] +Input [4]: [sum#97, count#98, count#99, count#102] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#70 / 100.0) as decimal(11,6)) AS B6_LP#79, count(ss_list_price#3)#71 AS B6_CNT#80, count(ss_list_price#3)#76 AS B6_CNTD#81] +Functions [3]: [avg(UnscaledValue(ss_list_price#92)), count(ss_list_price#92), count(distinct ss_list_price#92)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#92))#95 / 100.0) as decimal(11,6)) AS B6_LP#104, count(ss_list_price#92)#96 AS B6_CNT#105, count(ss_list_price#92)#101 AS B6_CNTD#106] (69) BroadcastExchange -Input [3]: [B6_LP#79, B6_CNT#80, B6_CNTD#81] -Arguments: IdentityBroadcastMode, [id=#82] +Input [3]: [B6_LP#104, B6_CNT#105, B6_CNTD#106] +Arguments: IdentityBroadcastMode, [id=#107] (70) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None 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 ca09ec6ea1a42..9e84e385be302 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 @@ -168,170 +168,170 @@ Input [3]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19] Condition : isnotnull(ctr_total_return#19) (24) Scan parquet default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +Output [4]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#23), dynamicpruningexpression(wr_returned_date_sk#23 IN dynamicpruning#5)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 8] -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +Input [4]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23] (26) Filter [codegen id : 8] -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) +Input [4]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23] +Condition : isnotnull(wr_returning_addr_sk#21) (27) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#24] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_returned_date_sk#4] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#23] +Right keys [1]: [cast(d_date_sk#24 as bigint)] Join condition: None (29) Project [codegen id : 8] -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] +Output [3]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22] +Input [5]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23, d_date_sk#24] (30) Exchange -Input [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Arguments: hashpartitioning(wr_returning_addr_sk#2, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [3]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22] +Arguments: hashpartitioning(wr_returning_addr_sk#21, 5), ENSURE_REQUIREMENTS, [id=#25] (31) Sort [codegen id : 9] -Input [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Arguments: [wr_returning_addr_sk#2 ASC NULLS FIRST], false, 0 +Input [3]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22] +Arguments: [wr_returning_addr_sk#21 ASC NULLS FIRST], false, 0 (32) ReusedExchange [Reuses operator id: 16] -Output [2]: [ca_address_sk#10, ca_state#11] +Output [2]: [ca_address_sk#26, ca_state#27] (33) Sort [codegen id : 11] -Input [2]: [ca_address_sk#10, ca_state#11] -Arguments: [cast(ca_address_sk#10 as bigint) ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#26, ca_state#27] +Arguments: [cast(ca_address_sk#26 as bigint) ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 12] -Left keys [1]: [wr_returning_addr_sk#2] -Right keys [1]: [cast(ca_address_sk#10 as bigint)] +Left keys [1]: [wr_returning_addr_sk#21] +Right keys [1]: [cast(ca_address_sk#26 as bigint)] Join condition: None (35) Project [codegen id : 12] -Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#11] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#10, ca_state#11] +Output [3]: [wr_returning_customer_sk#20, wr_return_amt#22, ca_state#27] +Input [5]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, ca_address_sk#26, ca_state#27] (36) HashAggregate [codegen id : 12] -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#11] -Keys [2]: [wr_returning_customer_sk#1, ca_state#11] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum#21] -Results [3]: [wr_returning_customer_sk#1, ca_state#11, sum#22] +Input [3]: [wr_returning_customer_sk#20, wr_return_amt#22, ca_state#27] +Keys [2]: [wr_returning_customer_sk#20, ca_state#27] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#22))] +Aggregate Attributes [1]: [sum#28] +Results [3]: [wr_returning_customer_sk#20, ca_state#27, sum#29] (37) Exchange -Input [3]: [wr_returning_customer_sk#1, ca_state#11, sum#22] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#11, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [wr_returning_customer_sk#20, ca_state#27, sum#29] +Arguments: hashpartitioning(wr_returning_customer_sk#20, ca_state#27, 5), ENSURE_REQUIREMENTS, [id=#30] (38) HashAggregate [codegen id : 13] -Input [3]: [wr_returning_customer_sk#1, ca_state#11, sum#22] -Keys [2]: [wr_returning_customer_sk#1, ca_state#11] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#24] -Results [2]: [ca_state#11 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#24,17,2) AS ctr_total_return#19] +Input [3]: [wr_returning_customer_sk#20, ca_state#27, sum#29] +Keys [2]: [wr_returning_customer_sk#20, ca_state#27] +Functions [1]: [sum(UnscaledValue(wr_return_amt#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#22))#31] +Results [2]: [ca_state#27 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(wr_return_amt#22))#31,17,2) AS ctr_total_return#19] (39) HashAggregate [codegen id : 13] Input [2]: [ctr_state#18, ctr_total_return#19] Keys [1]: [ctr_state#18] Functions [1]: [partial_avg(ctr_total_return#19)] -Aggregate Attributes [2]: [sum#25, count#26] -Results [3]: [ctr_state#18, sum#27, count#28] +Aggregate Attributes [2]: [sum#32, count#33] +Results [3]: [ctr_state#18, sum#34, count#35] (40) Exchange -Input [3]: [ctr_state#18, sum#27, count#28] -Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ctr_state#18, sum#34, count#35] +Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#36] (41) HashAggregate [codegen id : 14] -Input [3]: [ctr_state#18, sum#27, count#28] +Input [3]: [ctr_state#18, sum#34, count#35] Keys [1]: [ctr_state#18] Functions [1]: [avg(ctr_total_return#19)] -Aggregate Attributes [1]: [avg(ctr_total_return#19)#30] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#18 AS ctr_state#18#32] +Aggregate Attributes [1]: [avg(ctr_total_return#19)#37] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#37) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#38, ctr_state#18 AS ctr_state#18#39] (42) Filter [codegen id : 14] -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) +Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#38) (43) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#33] +Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#40] (44) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ctr_state#18] -Right keys [1]: [ctr_state#18#32] -Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) +Right keys [1]: [ctr_state#18#39] +Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#38) (45) Project [codegen id : 17] Output [2]: [ctr_customer_sk#17, ctr_total_return#19] -Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] +Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] (46) Scan parquet default.customer -Output [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Output [14]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 16] -Input [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Input [14]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] (48) Filter [codegen id : 16] -Input [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] -Condition : (isnotnull(c_customer_sk#34) AND isnotnull(c_current_addr_sk#36)) +Input [14]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] +Condition : (isnotnull(c_customer_sk#41) AND isnotnull(c_current_addr_sk#43)) (49) Scan parquet default.customer_address -Output [2]: [ca_address_sk#10, ca_state#11] +Output [2]: [ca_address_sk#55, ca_state#56] 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 (50) ColumnarToRow [codegen id : 15] -Input [2]: [ca_address_sk#10, ca_state#11] +Input [2]: [ca_address_sk#55, ca_state#56] (51) Filter [codegen id : 15] -Input [2]: [ca_address_sk#10, ca_state#11] -Condition : ((isnotnull(ca_state#11) AND (ca_state#11 = GA)) AND isnotnull(ca_address_sk#10)) +Input [2]: [ca_address_sk#55, ca_state#56] +Condition : ((isnotnull(ca_state#56) AND (ca_state#56 = GA)) AND isnotnull(ca_address_sk#55)) (52) Project [codegen id : 15] -Output [1]: [ca_address_sk#10] -Input [2]: [ca_address_sk#10, ca_state#11] +Output [1]: [ca_address_sk#55] +Input [2]: [ca_address_sk#55, ca_state#56] (53) BroadcastExchange -Input [1]: [ca_address_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] +Input [1]: [ca_address_sk#55] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#57] (54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_addr_sk#36] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [c_current_addr_sk#43] +Right keys [1]: [ca_address_sk#55] Join condition: None (55) Project [codegen id : 16] -Output [13]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] -Input [15]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ca_address_sk#10] +Output [13]: [c_customer_sk#41, c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] +Input [15]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ca_address_sk#55] (56) BroadcastExchange -Input [13]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] +Input [13]: [c_customer_sk#41, c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] (57) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ctr_customer_sk#17] -Right keys [1]: [cast(c_customer_sk#34 as bigint)] +Right keys [1]: [cast(c_customer_sk#41 as bigint)] Join condition: None (58) Project [codegen id : 17] -Output [13]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#19] -Input [15]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Output [13]: [c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ctr_total_return#19] +Input [15]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#41, c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] (59) TakeOrderedAndProject -Input [13]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#19] -Arguments: 100, [c_customer_id#35 ASC NULLS FIRST, c_salutation#37 ASC NULLS FIRST, c_first_name#38 ASC NULLS FIRST, c_last_name#39 ASC NULLS FIRST, c_preferred_cust_flag#40 ASC NULLS FIRST, c_birth_day#41 ASC NULLS FIRST, c_birth_month#42 ASC NULLS FIRST, c_birth_year#43 ASC NULLS FIRST, c_birth_country#44 ASC NULLS FIRST, c_login#45 ASC NULLS FIRST, c_email_address#46 ASC NULLS FIRST, c_last_review_date#47 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#19] +Input [13]: [c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ctr_total_return#19] +Arguments: 100, [c_customer_id#42 ASC NULLS FIRST, c_salutation#44 ASC NULLS FIRST, c_first_name#45 ASC NULLS FIRST, c_last_name#46 ASC NULLS FIRST, c_preferred_cust_flag#47 ASC NULLS FIRST, c_birth_day#48 ASC NULLS FIRST, c_birth_month#49 ASC NULLS FIRST, c_birth_year#50 ASC NULLS FIRST, c_birth_country#51 ASC NULLS FIRST, c_login#52 ASC NULLS FIRST, c_email_address#53 ASC NULLS FIRST, c_last_review_date#54 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ctr_total_return#19] ===== Subqueries ===== @@ -342,6 +342,6 @@ ReusedExchange (60) (60) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 24 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 24 Hosting Expression = wr_returned_date_sk#23 IN dynamicpruning#5 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 57b85bc9db340..1aaebad9f1920 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 @@ -150,158 +150,158 @@ Input [3]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18] Condition : isnotnull(ctr_total_return#18) (21) Scan parquet default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +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#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#22), dynamicpruningexpression(wr_returned_date_sk#22 IN dynamicpruning#5)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct (22) ColumnarToRow [codegen id : 6] -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +Input [4]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, wr_returned_date_sk#22] (23) Filter [codegen id : 6] -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) +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) (24) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#23] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returned_date_sk#4] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#22] +Right keys [1]: [cast(d_date_sk#23 as bigint)] Join condition: None (26) Project [codegen id : 6] -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] +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#23] (27) ReusedExchange [Reuses operator id: 14] -Output [2]: [ca_address_sk#9, ca_state#10] +Output [2]: [ca_address_sk#24, ca_state#25] (28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returning_addr_sk#2] -Right keys [1]: [cast(ca_address_sk#9 as bigint)] +Left keys [1]: [wr_returning_addr_sk#20] +Right keys [1]: [cast(ca_address_sk#24 as bigint)] Join condition: None (29) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#9, ca_state#10] +Output [3]: [wr_returning_customer_sk#19, wr_return_amt#21, ca_state#25] +Input [5]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, ca_address_sk#24, ca_state#25] (30) HashAggregate [codegen id : 6] -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [wr_returning_customer_sk#1, ca_state#10, sum#20] +Input [3]: [wr_returning_customer_sk#19, wr_return_amt#21, ca_state#25] +Keys [2]: [wr_returning_customer_sk#19, ca_state#25] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#21))] +Aggregate Attributes [1]: [sum#26] +Results [3]: [wr_returning_customer_sk#19, ca_state#25, sum#27] (31) Exchange -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#20] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [3]: [wr_returning_customer_sk#19, ca_state#25, sum#27] +Arguments: hashpartitioning(wr_returning_customer_sk#19, ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#28] (32) HashAggregate [codegen id : 7] -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#20] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#22] -Results [2]: [ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#22,17,2) AS ctr_total_return#18] +Input [3]: [wr_returning_customer_sk#19, ca_state#25, sum#27] +Keys [2]: [wr_returning_customer_sk#19, ca_state#25] +Functions [1]: [sum(UnscaledValue(wr_return_amt#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#21))#29] +Results [2]: [ca_state#25 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(wr_return_amt#21))#29,17,2) AS ctr_total_return#18] (33) HashAggregate [codegen id : 7] Input [2]: [ctr_state#17, ctr_total_return#18] Keys [1]: [ctr_state#17] Functions [1]: [partial_avg(ctr_total_return#18)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ctr_state#17, sum#25, count#26] +Aggregate Attributes [2]: [sum#30, count#31] +Results [3]: [ctr_state#17, sum#32, count#33] (34) Exchange -Input [3]: [ctr_state#17, sum#25, count#26] -Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ctr_state#17, sum#32, count#33] +Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#34] (35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#17, sum#25, count#26] +Input [3]: [ctr_state#17, sum#32, count#33] Keys [1]: [ctr_state#17] Functions [1]: [avg(ctr_total_return#18)] -Aggregate Attributes [1]: [avg(ctr_total_return#18)#28] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#28) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#29, ctr_state#17 AS ctr_state#17#30] +Aggregate Attributes [1]: [avg(ctr_total_return#18)#35] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#35) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#36, ctr_state#17 AS ctr_state#17#37] (36) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) +Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#36) (37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#31] +Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#38] (38) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#17] -Right keys [1]: [ctr_state#17#30] -Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) +Right keys [1]: [ctr_state#17#37] +Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#36) (39) Project [codegen id : 11] Output [2]: [ctr_customer_sk#16, ctr_total_return#18] -Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] +Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] (40) Scan parquet default.customer -Output [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +Output [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +Input [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] (42) Filter [codegen id : 9] -Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34)) +Input [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] +Condition : (isnotnull(c_customer_sk#39) AND isnotnull(c_current_addr_sk#41)) (43) BroadcastExchange -Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#46] +Input [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] (44) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#16] -Right keys [1]: [cast(c_customer_sk#32 as bigint)] +Right keys [1]: [cast(c_customer_sk#39 as bigint)] Join condition: None (45) Project [codegen id : 11] -Output [14]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] -Input [16]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +Output [14]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] +Input [16]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] (46) Scan parquet default.customer_address -Output [2]: [ca_address_sk#9, ca_state#10] +Output [2]: [ca_address_sk#54, ca_state#55] 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 (47) ColumnarToRow [codegen id : 10] -Input [2]: [ca_address_sk#9, ca_state#10] +Input [2]: [ca_address_sk#54, ca_state#55] (48) Filter [codegen id : 10] -Input [2]: [ca_address_sk#9, ca_state#10] -Condition : ((isnotnull(ca_state#10) AND (ca_state#10 = GA)) AND isnotnull(ca_address_sk#9)) +Input [2]: [ca_address_sk#54, ca_state#55] +Condition : ((isnotnull(ca_state#55) AND (ca_state#55 = GA)) AND isnotnull(ca_address_sk#54)) (49) Project [codegen id : 10] -Output [1]: [ca_address_sk#9] -Input [2]: [ca_address_sk#9, ca_state#10] +Output [1]: [ca_address_sk#54] +Input [2]: [ca_address_sk#54, ca_state#55] (50) BroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] +Input [1]: [ca_address_sk#54] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] (51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#34] -Right keys [1]: [ca_address_sk#9] +Left keys [1]: [c_current_addr_sk#41] +Right keys [1]: [ca_address_sk#54] Join condition: None (52) Project [codegen id : 11] -Output [13]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#18] -Input [15]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ca_address_sk#9] +Output [13]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ctr_total_return#18] +Input [15]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ca_address_sk#54] (53) TakeOrderedAndProject -Input [13]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#18] -Arguments: 100, [c_customer_id#33 ASC NULLS FIRST, c_salutation#35 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, c_last_name#37 ASC NULLS FIRST, c_preferred_cust_flag#38 ASC NULLS FIRST, c_birth_day#39 ASC NULLS FIRST, c_birth_month#40 ASC NULLS FIRST, c_birth_year#41 ASC NULLS FIRST, c_birth_country#42 ASC NULLS FIRST, c_login#43 ASC NULLS FIRST, c_email_address#44 ASC NULLS FIRST, c_last_review_date#45 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#18] +Input [13]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ctr_total_return#18] +Arguments: 100, [c_customer_id#40 ASC NULLS FIRST, c_salutation#42 ASC NULLS FIRST, c_first_name#43 ASC NULLS FIRST, c_last_name#44 ASC NULLS FIRST, c_preferred_cust_flag#45 ASC NULLS FIRST, c_birth_day#46 ASC NULLS FIRST, c_birth_month#47 ASC NULLS FIRST, c_birth_year#48 ASC NULLS FIRST, c_birth_country#49 ASC NULLS FIRST, c_login#50 ASC NULLS FIRST, c_email_address#51 ASC NULLS FIRST, c_last_review_date#52 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ctr_total_return#18] ===== Subqueries ===== @@ -312,6 +312,6 @@ ReusedExchange (54) (54) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 21 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 21 Hosting Expression = wr_returned_date_sk#22 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 61aa5ea5e7e5c..1703da84e7914 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 @@ -220,452 +220,452 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#16] Results [2]: [ca_county#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#16,17,2) AS store_sales#17] (22) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#18)] +PartitionFilters: [isnotnull(ss_sold_date_sk#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 8] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20] (24) Filter [codegen id : 8] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) +Input [3]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20] +Condition : isnotnull(ss_addr_sk#18) (25) Scan parquet default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_qoy#21] +Output [3]: [d_date_sk#22, d_year#23, d_qoy#24] 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 (26) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] +Input [3]: [d_date_sk#22, d_year#23, d_qoy#24] (27) Filter [codegen id : 7] -Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] -Condition : ((((isnotnull(d_qoy#21) AND isnotnull(d_year#20)) AND (d_qoy#21 = 3)) AND (d_year#20 = 2000)) AND isnotnull(d_date_sk#19)) +Input [3]: [d_date_sk#22, d_year#23, d_qoy#24] +Condition : ((((isnotnull(d_qoy#24) AND isnotnull(d_year#23)) AND (d_qoy#24 = 3)) AND (d_year#23 = 2000)) AND isnotnull(d_date_sk#22)) (28) BroadcastExchange -Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +Input [3]: [d_date_sk#22, d_year#23, d_qoy#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] (29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#19] +Left keys [1]: [ss_sold_date_sk#20] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 8] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#19, d_year#20, d_qoy#21] +Output [4]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24] +Input [6]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23, d_qoy#24] (31) Exchange -Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21] -Arguments: hashpartitioning(ss_addr_sk#1, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [4]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24] +Arguments: hashpartitioning(ss_addr_sk#18, 5), ENSURE_REQUIREMENTS, [id=#26] (32) Sort [codegen id : 9] -Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21] -Arguments: [ss_addr_sk#1 ASC NULLS FIRST], false, 0 +Input [4]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24] +Arguments: [ss_addr_sk#18 ASC NULLS FIRST], false, 0 (33) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#24, ca_county#25] +Output [2]: [ca_address_sk#27, ca_county#28] (34) Sort [codegen id : 11] -Input [2]: [ca_address_sk#24, ca_county#25] -Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#27, ca_county#28] +Arguments: [ca_address_sk#27 ASC NULLS FIRST], false, 0 (35) SortMergeJoin [codegen id : 12] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [ss_addr_sk#18] +Right keys [1]: [ca_address_sk#27] Join condition: None (36) Project [codegen id : 12] -Output [4]: [ss_ext_sales_price#2, d_year#20, d_qoy#21, ca_county#25] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21, ca_address_sk#24, ca_county#25] +Output [4]: [ss_ext_sales_price#19, d_year#23, d_qoy#24, ca_county#28] +Input [6]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24, ca_address_sk#27, ca_county#28] (37) HashAggregate [codegen id : 12] -Input [4]: [ss_ext_sales_price#2, d_year#20, d_qoy#21, ca_county#25] -Keys [3]: [ca_county#25, d_qoy#21, d_year#20] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#26] -Results [4]: [ca_county#25, d_qoy#21, d_year#20, sum#27] +Input [4]: [ss_ext_sales_price#19, d_year#23, d_qoy#24, ca_county#28] +Keys [3]: [ca_county#28, d_qoy#24, d_year#23] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#19))] +Aggregate Attributes [1]: [sum#29] +Results [4]: [ca_county#28, d_qoy#24, d_year#23, sum#30] (38) Exchange -Input [4]: [ca_county#25, d_qoy#21, d_year#20, sum#27] -Arguments: hashpartitioning(ca_county#25, d_qoy#21, d_year#20, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [4]: [ca_county#28, d_qoy#24, d_year#23, sum#30] +Arguments: hashpartitioning(ca_county#28, d_qoy#24, d_year#23, 5), ENSURE_REQUIREMENTS, [id=#31] (39) HashAggregate [codegen id : 13] -Input [4]: [ca_county#25, d_qoy#21, d_year#20, sum#27] -Keys [3]: [ca_county#25, d_qoy#21, d_year#20] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#29] -Results [2]: [ca_county#25, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#29,17,2) AS store_sales#30] +Input [4]: [ca_county#28, d_qoy#24, d_year#23, sum#30] +Keys [3]: [ca_county#28, d_qoy#24, d_year#23] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#19))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#19))#32] +Results [2]: [ca_county#28, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#19))#32,17,2) AS store_sales#33] (40) BroadcastExchange -Input [2]: [ca_county#25, store_sales#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#31] +Input [2]: [ca_county#28, store_sales#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] (41) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ca_county#11] -Right keys [1]: [ca_county#25] +Right keys [1]: [ca_county#28] Join condition: None (42) Project [codegen id : 42] -Output [3]: [ca_county#11, store_sales#17, store_sales#30] -Input [4]: [ca_county#11, store_sales#17, ca_county#25, store_sales#30] +Output [3]: [ca_county#11, store_sales#17, store_sales#33] +Input [4]: [ca_county#11, store_sales#17, ca_county#28, store_sales#33] (43) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(ss_sold_date_sk#37), dynamicpruningexpression(ss_sold_date_sk#37 IN dynamicpruning#38)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 15] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37] (45) Filter [codegen id : 15] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) +Input [3]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37] +Condition : isnotnull(ss_addr_sk#35) (46) Scan parquet default.date_dim -Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Output [3]: [d_date_sk#39, d_year#40, d_qoy#41] 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 (47) ColumnarToRow [codegen id : 14] -Input [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] (48) Filter [codegen id : 14] -Input [3]: [d_date_sk#33, d_year#34, d_qoy#35] -Condition : ((((isnotnull(d_qoy#35) AND isnotnull(d_year#34)) AND (d_qoy#35 = 1)) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) +Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] +Condition : ((((isnotnull(d_qoy#41) AND isnotnull(d_year#40)) AND (d_qoy#41 = 1)) AND (d_year#40 = 2000)) AND isnotnull(d_date_sk#39)) (49) BroadcastExchange -Input [3]: [d_date_sk#33, d_year#34, d_qoy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] +Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#42] (50) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [ss_sold_date_sk#37] +Right keys [1]: [d_date_sk#39] Join condition: None (51) Project [codegen id : 15] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#33, d_year#34, d_qoy#35] +Output [4]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41] +Input [6]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37, d_date_sk#39, d_year#40, d_qoy#41] (52) Exchange -Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35] -Arguments: hashpartitioning(ss_addr_sk#1, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41] +Arguments: hashpartitioning(ss_addr_sk#35, 5), ENSURE_REQUIREMENTS, [id=#43] (53) Sort [codegen id : 16] -Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35] -Arguments: [ss_addr_sk#1 ASC NULLS FIRST], false, 0 +Input [4]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41] +Arguments: [ss_addr_sk#35 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#38, ca_county#39] +Output [2]: [ca_address_sk#44, ca_county#45] (55) Sort [codegen id : 18] -Input [2]: [ca_address_sk#38, ca_county#39] -Arguments: [ca_address_sk#38 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#44, ca_county#45] +Arguments: [ca_address_sk#44 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 19] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#38] +Left keys [1]: [ss_addr_sk#35] +Right keys [1]: [ca_address_sk#44] Join condition: None (57) Project [codegen id : 19] -Output [4]: [ss_ext_sales_price#2, d_year#34, d_qoy#35, ca_county#39] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35, ca_address_sk#38, ca_county#39] +Output [4]: [ss_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#45] +Input [6]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41, ca_address_sk#44, ca_county#45] (58) HashAggregate [codegen id : 19] -Input [4]: [ss_ext_sales_price#2, d_year#34, d_qoy#35, ca_county#39] -Keys [3]: [ca_county#39, d_qoy#35, d_year#34] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#40] -Results [4]: [ca_county#39, d_qoy#35, d_year#34, sum#41] +Input [4]: [ss_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#45] +Keys [3]: [ca_county#45, d_qoy#41, d_year#40] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#36))] +Aggregate Attributes [1]: [sum#46] +Results [4]: [ca_county#45, d_qoy#41, d_year#40, sum#47] (59) Exchange -Input [4]: [ca_county#39, d_qoy#35, d_year#34, sum#41] -Arguments: hashpartitioning(ca_county#39, d_qoy#35, d_year#34, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [4]: [ca_county#45, d_qoy#41, d_year#40, sum#47] +Arguments: hashpartitioning(ca_county#45, d_qoy#41, d_year#40, 5), ENSURE_REQUIREMENTS, [id=#48] (60) HashAggregate [codegen id : 20] -Input [4]: [ca_county#39, d_qoy#35, d_year#34, sum#41] -Keys [3]: [ca_county#39, d_qoy#35, d_year#34] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#43] -Results [3]: [ca_county#39, d_year#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#43,17,2) AS store_sales#44] +Input [4]: [ca_county#45, d_qoy#41, d_year#40, sum#47] +Keys [3]: [ca_county#45, d_qoy#41, d_year#40] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#36))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#36))#49] +Results [3]: [ca_county#45, d_year#40, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#36))#49,17,2) AS store_sales#50] (61) BroadcastExchange -Input [3]: [ca_county#39, d_year#34, store_sales#44] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#45] +Input [3]: [ca_county#45, d_year#40, store_sales#50] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#51] (62) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ca_county#11] -Right keys [1]: [ca_county#39] +Right keys [1]: [ca_county#45] Join condition: None (63) Project [codegen id : 42] -Output [5]: [store_sales#17, store_sales#30, ca_county#39, d_year#34, store_sales#44] -Input [6]: [ca_county#11, store_sales#17, store_sales#30, ca_county#39, d_year#34, store_sales#44] +Output [5]: [store_sales#17, store_sales#33, ca_county#45, d_year#40, store_sales#50] +Input [6]: [ca_county#11, store_sales#17, store_sales#33, ca_county#45, d_year#40, store_sales#50] (64) 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#52, ws_ext_sales_price#53, ws_sold_date_sk#54] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#38)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Input [3]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, ws_sold_date_sk#54] (66) Filter [codegen id : 22] -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#52, ws_ext_sales_price#53, ws_sold_date_sk#54] +Condition : isnotnull(ws_bill_addr_sk#52) (67) ReusedExchange [Reuses operator id: 49] -Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] +Output [3]: [d_date_sk#55, d_year#56, d_qoy#57] (68) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#48] -Right keys [1]: [d_date_sk#49] +Left keys [1]: [ws_sold_date_sk#54] +Right keys [1]: [d_date_sk#55] Join condition: None (69) Project [codegen id : 22] -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#52, ws_ext_sales_price#53, d_year#56, d_qoy#57] +Input [6]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, ws_sold_date_sk#54, d_date_sk#55, d_year#56, d_qoy#57] (70) Exchange -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51] -Arguments: hashpartitioning(ws_bill_addr_sk#46, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [4]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57] +Arguments: hashpartitioning(ws_bill_addr_sk#52, 5), ENSURE_REQUIREMENTS, [id=#58] (71) Sort [codegen id : 23] -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51] -Arguments: [ws_bill_addr_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57] +Arguments: [ws_bill_addr_sk#52 ASC NULLS FIRST], false, 0 (72) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#53, ca_county#54] +Output [2]: [ca_address_sk#59, ca_county#60] (73) Sort [codegen id : 25] -Input [2]: [ca_address_sk#53, ca_county#54] -Arguments: [ca_address_sk#53 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#59, ca_county#60] +Arguments: [ca_address_sk#59 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 26] -Left keys [1]: [ws_bill_addr_sk#46] -Right keys [1]: [ca_address_sk#53] +Left keys [1]: [ws_bill_addr_sk#52] +Right keys [1]: [ca_address_sk#59] Join condition: None (75) Project [codegen id : 26] -Output [4]: [ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_county#54] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_address_sk#53, ca_county#54] +Output [4]: [ws_ext_sales_price#53, d_year#56, d_qoy#57, ca_county#60] +Input [6]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57, ca_address_sk#59, ca_county#60] (76) HashAggregate [codegen id : 26] -Input [4]: [ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_county#54] -Keys [3]: [ca_county#54, d_qoy#51, d_year#50] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum#55] -Results [4]: [ca_county#54, d_qoy#51, d_year#50, sum#56] +Input [4]: [ws_ext_sales_price#53, d_year#56, d_qoy#57, ca_county#60] +Keys [3]: [ca_county#60, d_qoy#57, d_year#56] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#53))] +Aggregate Attributes [1]: [sum#61] +Results [4]: [ca_county#60, d_qoy#57, d_year#56, sum#62] (77) Exchange -Input [4]: [ca_county#54, d_qoy#51, d_year#50, sum#56] -Arguments: hashpartitioning(ca_county#54, d_qoy#51, d_year#50, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [4]: [ca_county#60, d_qoy#57, d_year#56, sum#62] +Arguments: hashpartitioning(ca_county#60, d_qoy#57, d_year#56, 5), ENSURE_REQUIREMENTS, [id=#63] (78) HashAggregate [codegen id : 41] -Input [4]: [ca_county#54, d_qoy#51, d_year#50, sum#56] -Keys [3]: [ca_county#54, 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))#58] -Results [2]: [ca_county#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#47))#58,17,2) AS web_sales#59] +Input [4]: [ca_county#60, d_qoy#57, d_year#56, sum#62] +Keys [3]: [ca_county#60, d_qoy#57, d_year#56] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#53))#64] +Results [2]: [ca_county#60, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#53))#64,17,2) AS web_sales#65] (79) 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#66, ws_ext_sales_price#67, ws_sold_date_sk#68] 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#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (80) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Input [3]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, ws_sold_date_sk#68] (81) Filter [codegen id : 28] -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#66, ws_ext_sales_price#67, ws_sold_date_sk#68] +Condition : isnotnull(ws_bill_addr_sk#66) (82) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#60, d_year#61, d_qoy#62] +Output [3]: [d_date_sk#69, d_year#70, d_qoy#71] (83) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#48] -Right keys [1]: [d_date_sk#60] +Left keys [1]: [ws_sold_date_sk#68] +Right keys [1]: [d_date_sk#69] Join condition: None (84) Project [codegen id : 28] -Output [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48, d_date_sk#60, d_year#61, d_qoy#62] +Output [4]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71] +Input [6]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, ws_sold_date_sk#68, d_date_sk#69, d_year#70, d_qoy#71] (85) Exchange -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62] -Arguments: hashpartitioning(ws_bill_addr_sk#46, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [4]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71] +Arguments: hashpartitioning(ws_bill_addr_sk#66, 5), ENSURE_REQUIREMENTS, [id=#72] (86) Sort [codegen id : 29] -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62] -Arguments: [ws_bill_addr_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71] +Arguments: [ws_bill_addr_sk#66 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#64, ca_county#65] +Output [2]: [ca_address_sk#73, ca_county#74] (88) Sort [codegen id : 31] -Input [2]: [ca_address_sk#64, ca_county#65] -Arguments: [ca_address_sk#64 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#73, ca_county#74] +Arguments: [ca_address_sk#73 ASC NULLS FIRST], false, 0 (89) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_addr_sk#46] -Right keys [1]: [ca_address_sk#64] +Left keys [1]: [ws_bill_addr_sk#66] +Right keys [1]: [ca_address_sk#73] Join condition: None (90) Project [codegen id : 32] -Output [4]: [ws_ext_sales_price#47, d_year#61, d_qoy#62, ca_county#65] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62, ca_address_sk#64, ca_county#65] +Output [4]: [ws_ext_sales_price#67, d_year#70, d_qoy#71, ca_county#74] +Input [6]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71, ca_address_sk#73, ca_county#74] (91) HashAggregate [codegen id : 32] -Input [4]: [ws_ext_sales_price#47, d_year#61, d_qoy#62, ca_county#65] -Keys [3]: [ca_county#65, d_qoy#62, d_year#61] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum#66] -Results [4]: [ca_county#65, d_qoy#62, d_year#61, sum#67] +Input [4]: [ws_ext_sales_price#67, d_year#70, d_qoy#71, ca_county#74] +Keys [3]: [ca_county#74, d_qoy#71, d_year#70] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#67))] +Aggregate Attributes [1]: [sum#75] +Results [4]: [ca_county#74, d_qoy#71, d_year#70, sum#76] (92) Exchange -Input [4]: [ca_county#65, d_qoy#62, d_year#61, sum#67] -Arguments: hashpartitioning(ca_county#65, d_qoy#62, d_year#61, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [4]: [ca_county#74, d_qoy#71, d_year#70, sum#76] +Arguments: hashpartitioning(ca_county#74, d_qoy#71, d_year#70, 5), ENSURE_REQUIREMENTS, [id=#77] (93) HashAggregate [codegen id : 33] -Input [4]: [ca_county#65, d_qoy#62, d_year#61, sum#67] -Keys [3]: [ca_county#65, d_qoy#62, d_year#61] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#47))#69] -Results [2]: [ca_county#65, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#47))#69,17,2) AS web_sales#70] +Input [4]: [ca_county#74, d_qoy#71, d_year#70, sum#76] +Keys [3]: [ca_county#74, d_qoy#71, d_year#70] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#67))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#67))#78] +Results [2]: [ca_county#74, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#67))#78,17,2) AS web_sales#79] (94) BroadcastExchange -Input [2]: [ca_county#65, web_sales#70] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#71] +Input [2]: [ca_county#74, web_sales#79] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#80] (95) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#54] -Right keys [1]: [ca_county#65] +Left keys [1]: [ca_county#60] +Right keys [1]: [ca_county#74] Join condition: None (96) Project [codegen id : 41] -Output [3]: [ca_county#54, web_sales#59, web_sales#70] -Input [4]: [ca_county#54, web_sales#59, ca_county#65, web_sales#70] +Output [3]: [ca_county#60, web_sales#65, web_sales#79] +Input [4]: [ca_county#60, web_sales#65, ca_county#74, web_sales#79] (97) 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#81, ws_ext_sales_price#82, ws_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#18)] +PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (98) ColumnarToRow [codegen id : 35] -Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Input [3]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, ws_sold_date_sk#83] (99) Filter [codegen id : 35] -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#81, ws_ext_sales_price#82, ws_sold_date_sk#83] +Condition : isnotnull(ws_bill_addr_sk#81) (100) ReusedExchange [Reuses operator id: 28] -Output [3]: [d_date_sk#72, d_year#73, d_qoy#74] +Output [3]: [d_date_sk#84, d_year#85, d_qoy#86] (101) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_sold_date_sk#48] -Right keys [1]: [d_date_sk#72] +Left keys [1]: [ws_sold_date_sk#83] +Right keys [1]: [d_date_sk#84] Join condition: None (102) Project [codegen id : 35] -Output [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48, d_date_sk#72, d_year#73, d_qoy#74] +Output [4]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86] +Input [6]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, ws_sold_date_sk#83, d_date_sk#84, d_year#85, d_qoy#86] (103) Exchange -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74] -Arguments: hashpartitioning(ws_bill_addr_sk#46, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [4]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86] +Arguments: hashpartitioning(ws_bill_addr_sk#81, 5), ENSURE_REQUIREMENTS, [id=#87] (104) Sort [codegen id : 36] -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74] -Arguments: [ws_bill_addr_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86] +Arguments: [ws_bill_addr_sk#81 ASC NULLS FIRST], false, 0 (105) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#76, ca_county#77] +Output [2]: [ca_address_sk#88, ca_county#89] (106) Sort [codegen id : 38] -Input [2]: [ca_address_sk#76, ca_county#77] -Arguments: [ca_address_sk#76 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#88, ca_county#89] +Arguments: [ca_address_sk#88 ASC NULLS FIRST], false, 0 (107) SortMergeJoin [codegen id : 39] -Left keys [1]: [ws_bill_addr_sk#46] -Right keys [1]: [ca_address_sk#76] +Left keys [1]: [ws_bill_addr_sk#81] +Right keys [1]: [ca_address_sk#88] Join condition: None (108) Project [codegen id : 39] -Output [4]: [ws_ext_sales_price#47, d_year#73, d_qoy#74, ca_county#77] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74, ca_address_sk#76, ca_county#77] +Output [4]: [ws_ext_sales_price#82, d_year#85, d_qoy#86, ca_county#89] +Input [6]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86, ca_address_sk#88, ca_county#89] (109) HashAggregate [codegen id : 39] -Input [4]: [ws_ext_sales_price#47, d_year#73, d_qoy#74, ca_county#77] -Keys [3]: [ca_county#77, d_qoy#74, d_year#73] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum#78] -Results [4]: [ca_county#77, d_qoy#74, d_year#73, sum#79] +Input [4]: [ws_ext_sales_price#82, d_year#85, d_qoy#86, ca_county#89] +Keys [3]: [ca_county#89, d_qoy#86, d_year#85] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#82))] +Aggregate Attributes [1]: [sum#90] +Results [4]: [ca_county#89, d_qoy#86, d_year#85, sum#91] (110) Exchange -Input [4]: [ca_county#77, d_qoy#74, d_year#73, sum#79] -Arguments: hashpartitioning(ca_county#77, d_qoy#74, d_year#73, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [4]: [ca_county#89, d_qoy#86, d_year#85, sum#91] +Arguments: hashpartitioning(ca_county#89, d_qoy#86, d_year#85, 5), ENSURE_REQUIREMENTS, [id=#92] (111) HashAggregate [codegen id : 40] -Input [4]: [ca_county#77, d_qoy#74, d_year#73, sum#79] -Keys [3]: [ca_county#77, d_qoy#74, d_year#73] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#47))#81] -Results [2]: [ca_county#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#47))#81,17,2) AS web_sales#82] +Input [4]: [ca_county#89, d_qoy#86, d_year#85, sum#91] +Keys [3]: [ca_county#89, d_qoy#86, d_year#85] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#82))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#82))#93] +Results [2]: [ca_county#89, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#82))#93,17,2) AS web_sales#94] (112) BroadcastExchange -Input [2]: [ca_county#77, web_sales#82] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#83] +Input [2]: [ca_county#89, web_sales#94] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#95] (113) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#54] -Right keys [1]: [ca_county#77] +Left keys [1]: [ca_county#60] +Right keys [1]: [ca_county#89] Join condition: None (114) Project [codegen id : 41] -Output [4]: [ca_county#54, web_sales#59, web_sales#70, web_sales#82] -Input [5]: [ca_county#54, web_sales#59, web_sales#70, ca_county#77, web_sales#82] +Output [4]: [ca_county#60, web_sales#65, web_sales#79, web_sales#94] +Input [5]: [ca_county#60, web_sales#65, web_sales#79, ca_county#89, web_sales#94] (115) BroadcastExchange -Input [4]: [ca_county#54, web_sales#59, web_sales#70, web_sales#82] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#84] +Input [4]: [ca_county#60, web_sales#65, web_sales#79, web_sales#94] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#96] (116) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ca_county#39] -Right keys [1]: [ca_county#54] -Join condition: ((CASE WHEN (web_sales#59 > 0.00) THEN CheckOverflow((promote_precision(web_sales#70) / promote_precision(web_sales#59)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#44 > 0.00) THEN CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#44)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#70 > 0.00) THEN CheckOverflow((promote_precision(web_sales#82) / promote_precision(web_sales#70)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#17 > 0.00) THEN CheckOverflow((promote_precision(store_sales#30) / promote_precision(store_sales#17)), DecimalType(37,20), true) ELSE null END)) +Left keys [1]: [ca_county#45] +Right keys [1]: [ca_county#60] +Join condition: ((CASE WHEN (web_sales#65 > 0.00) THEN CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#65)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#50 > 0.00) THEN CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#50)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#79 > 0.00) THEN CheckOverflow((promote_precision(web_sales#94) / promote_precision(web_sales#79)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#17 > 0.00) THEN CheckOverflow((promote_precision(store_sales#33) / promote_precision(store_sales#17)), DecimalType(37,20), true) ELSE null END)) (117) Project [codegen id : 42] -Output [6]: [ca_county#39, d_year#34, CheckOverflow((promote_precision(web_sales#70) / promote_precision(web_sales#59)), DecimalType(37,20), true) AS web_q1_q2_increase#85, CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#44)), DecimalType(37,20), true) AS store_q1_q2_increase#86, CheckOverflow((promote_precision(web_sales#82) / promote_precision(web_sales#70)), DecimalType(37,20), true) AS web_q2_q3_increase#87, CheckOverflow((promote_precision(store_sales#30) / promote_precision(store_sales#17)), DecimalType(37,20), true) AS store_q2_q3_increase#88] -Input [9]: [store_sales#17, store_sales#30, ca_county#39, d_year#34, store_sales#44, ca_county#54, web_sales#59, web_sales#70, web_sales#82] +Output [6]: [ca_county#45, d_year#40, CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#65)), DecimalType(37,20), true) AS web_q1_q2_increase#97, CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#50)), DecimalType(37,20), true) AS store_q1_q2_increase#98, CheckOverflow((promote_precision(web_sales#94) / promote_precision(web_sales#79)), DecimalType(37,20), true) AS web_q2_q3_increase#99, CheckOverflow((promote_precision(store_sales#33) / promote_precision(store_sales#17)), DecimalType(37,20), true) AS store_q2_q3_increase#100] +Input [9]: [store_sales#17, store_sales#33, ca_county#45, d_year#40, store_sales#50, ca_county#60, web_sales#65, web_sales#79, web_sales#94] (118) Exchange -Input [6]: [ca_county#39, d_year#34, web_q1_q2_increase#85, store_q1_q2_increase#86, web_q2_q3_increase#87, store_q2_q3_increase#88] -Arguments: rangepartitioning(ca_county#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [6]: [ca_county#45, d_year#40, web_q1_q2_increase#97, store_q1_q2_increase#98, web_q2_q3_increase#99, store_q2_q3_increase#100] +Arguments: rangepartitioning(ca_county#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#101] (119) Sort [codegen id : 43] -Input [6]: [ca_county#39, d_year#34, web_q1_q2_increase#85, store_q1_q2_increase#86, web_q2_q3_increase#87, store_q2_q3_increase#88] -Arguments: [ca_county#39 ASC NULLS FIRST], true, 0 +Input [6]: [ca_county#45, d_year#40, web_q1_q2_increase#97, store_q1_q2_increase#98, web_q2_q3_increase#99, store_q2_q3_increase#100] +Arguments: [ca_county#45 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -676,24 +676,24 @@ ReusedExchange (120) (120) ReusedExchange [Reuses operator id: 7] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#18 +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 ReusedExchange (121) (121) ReusedExchange [Reuses operator id: 28] -Output [3]: [d_date_sk#19, d_year#20, d_qoy#21] +Output [3]: [d_date_sk#22, d_year#23, d_qoy#24] -Subquery:3 Hosting operator id = 43 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#32 +Subquery:3 Hosting operator id = 43 Hosting Expression = ss_sold_date_sk#37 IN dynamicpruning#38 ReusedExchange (122) (122) ReusedExchange [Reuses operator id: 49] -Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Output [3]: [d_date_sk#39, d_year#40, d_qoy#41] -Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#32 +Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#38 -Subquery:5 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#4 +Subquery:5 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#4 -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#18 +Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#21 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 81a4a7fed99b2..f47a7af5a15e3 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 @@ -188,383 +188,383 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#15] Results [3]: [ca_county#10, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#15,17,2) AS store_sales#16] (19) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(ss_sold_date_sk#19), dynamicpruningexpression(ss_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (20) ColumnarToRow [codegen id : 6] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19] (21) Filter [codegen id : 6] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) +Input [3]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19] +Condition : isnotnull(ss_addr_sk#17) (22) Scan parquet default.date_dim -Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 4] -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] (24) Filter [codegen id : 4] -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) +Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] +Condition : ((((isnotnull(d_qoy#23) AND isnotnull(d_year#22)) AND (d_qoy#23 = 2)) AND (d_year#22 = 2000)) AND isnotnull(d_date_sk#21)) (25) BroadcastExchange -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] (26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#18] +Left keys [1]: [ss_sold_date_sk#19] +Right keys [1]: [d_date_sk#21] Join condition: None (27) Project [codegen id : 6] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#19, d_qoy#20] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#18, d_year#19, d_qoy#20] +Output [4]: [ss_addr_sk#17, ss_ext_sales_price#18, d_year#22, d_qoy#23] +Input [6]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19, d_date_sk#21, d_year#22, d_qoy#23] (28) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#22, ca_county#23] +Output [2]: [ca_address_sk#25, ca_county#26] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#22] +Left keys [1]: [ss_addr_sk#17] +Right keys [1]: [ca_address_sk#25] Join condition: None (30) Project [codegen id : 6] -Output [4]: [ss_ext_sales_price#2, d_year#19, d_qoy#20, ca_county#23] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#19, d_qoy#20, ca_address_sk#22, ca_county#23] +Output [4]: [ss_ext_sales_price#18, d_year#22, d_qoy#23, ca_county#26] +Input [6]: [ss_addr_sk#17, ss_ext_sales_price#18, d_year#22, d_qoy#23, ca_address_sk#25, ca_county#26] (31) HashAggregate [codegen id : 6] -Input [4]: [ss_ext_sales_price#2, d_year#19, d_qoy#20, ca_county#23] -Keys [3]: [ca_county#23, d_qoy#20, d_year#19] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#24] -Results [4]: [ca_county#23, d_qoy#20, d_year#19, sum#25] +Input [4]: [ss_ext_sales_price#18, d_year#22, d_qoy#23, ca_county#26] +Keys [3]: [ca_county#26, d_qoy#23, d_year#22] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#18))] +Aggregate Attributes [1]: [sum#27] +Results [4]: [ca_county#26, d_qoy#23, d_year#22, sum#28] (32) Exchange -Input [4]: [ca_county#23, d_qoy#20, d_year#19, sum#25] -Arguments: hashpartitioning(ca_county#23, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [4]: [ca_county#26, d_qoy#23, d_year#22, sum#28] +Arguments: hashpartitioning(ca_county#26, d_qoy#23, d_year#22, 5), ENSURE_REQUIREMENTS, [id=#29] (33) HashAggregate [codegen id : 7] -Input [4]: [ca_county#23, d_qoy#20, d_year#19, sum#25] -Keys [3]: [ca_county#23, d_qoy#20, d_year#19] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#27] -Results [2]: [ca_county#23, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#27,17,2) AS store_sales#28] +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] (34) BroadcastExchange -Input [2]: [ca_county#23, store_sales#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#29] +Input [2]: [ca_county#26, store_sales#31] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] (35) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#10] -Right keys [1]: [ca_county#23] +Right keys [1]: [ca_county#26] Join condition: None (36) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_addr_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ss_sold_date_sk#35), dynamicpruningexpression(ss_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 10] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_addr_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] (38) Filter [codegen id : 10] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) +Input [3]: [ss_addr_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] +Condition : isnotnull(ss_addr_sk#33) (39) Scan parquet default.date_dim -Output [3]: [d_date_sk#31, d_year#32, d_qoy#33] +Output [3]: [d_date_sk#37, d_year#38, d_qoy#39] 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 (40) ColumnarToRow [codegen id : 8] -Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] +Input [3]: [d_date_sk#37, d_year#38, d_qoy#39] (41) Filter [codegen id : 8] -Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] -Condition : ((((isnotnull(d_qoy#33) AND isnotnull(d_year#32)) AND (d_qoy#33 = 3)) AND (d_year#32 = 2000)) AND isnotnull(d_date_sk#31)) +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 = 3)) AND (d_year#38 = 2000)) AND isnotnull(d_date_sk#37)) (42) BroadcastExchange -Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [3]: [d_date_sk#37, d_year#38, d_qoy#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] (43) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#31] +Left keys [1]: [ss_sold_date_sk#35] +Right keys [1]: [d_date_sk#37] Join condition: None (44) Project [codegen id : 10] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#32, d_qoy#33] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#31, d_year#32, d_qoy#33] +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] (45) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#35, ca_county#36] +Output [2]: [ca_address_sk#41, ca_county#42] (46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#35] +Left keys [1]: [ss_addr_sk#33] +Right keys [1]: [ca_address_sk#41] Join condition: None (47) Project [codegen id : 10] -Output [4]: [ss_ext_sales_price#2, d_year#32, d_qoy#33, ca_county#36] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#32, d_qoy#33, ca_address_sk#35, ca_county#36] +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] (48) HashAggregate [codegen id : 10] -Input [4]: [ss_ext_sales_price#2, d_year#32, d_qoy#33, ca_county#36] -Keys [3]: [ca_county#36, d_qoy#33, d_year#32] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#37] -Results [4]: [ca_county#36, d_qoy#33, d_year#32, sum#38] +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] (49) Exchange -Input [4]: [ca_county#36, d_qoy#33, d_year#32, sum#38] -Arguments: hashpartitioning(ca_county#36, d_qoy#33, d_year#32, 5), ENSURE_REQUIREMENTS, [id=#39] +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] (50) HashAggregate [codegen id : 11] -Input [4]: [ca_county#36, d_qoy#33, d_year#32, sum#38] -Keys [3]: [ca_county#36, d_qoy#33, d_year#32] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#40] -Results [2]: [ca_county#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#40,17,2) AS store_sales#41] +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 [2]: [ca_county#42, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#46,17,2) AS store_sales#47] (51) BroadcastExchange -Input [2]: [ca_county#36, store_sales#41] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#42] +Input [2]: [ca_county#42, store_sales#47] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] (52) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#23] -Right keys [1]: [ca_county#36] +Left keys [1]: [ca_county#26] +Right keys [1]: [ca_county#42] Join condition: None (53) Project [codegen id : 24] -Output [5]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41] -Input [7]: [ca_county#10, d_year#6, store_sales#16, ca_county#23, store_sales#28, ca_county#36, store_sales#41] +Output [5]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47] +Input [7]: [ca_county#10, d_year#6, store_sales#16, ca_county#26, store_sales#31, ca_county#42, store_sales#47] (54) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Output [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 14] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] (56) Filter [codegen id : 14] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_bill_addr_sk#43) +Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_bill_addr_sk#49) (57) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#46, d_year#47, d_qoy#48] +Output [3]: [d_date_sk#52, d_year#53, d_qoy#54] (58) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#46] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#52] Join condition: None (59) Project [codegen id : 14] -Output [4]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#47, d_qoy#48] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45, d_date_sk#46, d_year#47, d_qoy#48] +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] (60) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#49, ca_county#50] +Output [2]: [ca_address_sk#55, ca_county#56] (61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#43] -Right keys [1]: [ca_address_sk#49] +Left keys [1]: [ws_bill_addr_sk#49] +Right keys [1]: [ca_address_sk#55] Join condition: None (62) Project [codegen id : 14] -Output [4]: [ws_ext_sales_price#44, d_year#47, d_qoy#48, ca_county#50] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#47, d_qoy#48, ca_address_sk#49, ca_county#50] +Output [4]: [ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_county#56] +Input [6]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_address_sk#55, ca_county#56] (63) HashAggregate [codegen id : 14] -Input [4]: [ws_ext_sales_price#44, d_year#47, d_qoy#48, ca_county#50] -Keys [3]: [ca_county#50, d_qoy#48, d_year#47] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum#51] -Results [4]: [ca_county#50, d_qoy#48, d_year#47, sum#52] +Input [4]: [ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_county#56] +Keys [3]: [ca_county#56, d_qoy#54, d_year#53] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#50))] +Aggregate Attributes [1]: [sum#57] +Results [4]: [ca_county#56, d_qoy#54, d_year#53, sum#58] (64) Exchange -Input [4]: [ca_county#50, d_qoy#48, d_year#47, sum#52] -Arguments: hashpartitioning(ca_county#50, d_qoy#48, d_year#47, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [4]: [ca_county#56, d_qoy#54, d_year#53, sum#58] +Arguments: hashpartitioning(ca_county#56, d_qoy#54, d_year#53, 5), ENSURE_REQUIREMENTS, [id=#59] (65) HashAggregate [codegen id : 15] -Input [4]: [ca_county#50, d_qoy#48, d_year#47, sum#52] -Keys [3]: [ca_county#50, d_qoy#48, d_year#47] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#44))#54] -Results [2]: [ca_county#50, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#44))#54,17,2) AS web_sales#55] +Input [4]: [ca_county#56, d_qoy#54, d_year#53, sum#58] +Keys [3]: [ca_county#56, 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))#60] +Results [2]: [ca_county#56, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#50))#60,17,2) AS web_sales#61] (66) BroadcastExchange -Input [2]: [ca_county#50, web_sales#55] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] +Input [2]: [ca_county#56, web_sales#61] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#62] (67) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#10] -Right keys [1]: [ca_county#50] +Right keys [1]: [ca_county#56] Join condition: None (68) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Output [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#20)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 18] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] (70) Filter [codegen id : 18] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_bill_addr_sk#43) +Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +Condition : isnotnull(ws_bill_addr_sk#63) (71) ReusedExchange [Reuses operator id: 25] -Output [3]: [d_date_sk#57, d_year#58, d_qoy#59] +Output [3]: [d_date_sk#66, d_year#67, d_qoy#68] (72) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#57] +Left keys [1]: [ws_sold_date_sk#65] +Right keys [1]: [d_date_sk#66] Join condition: None (73) Project [codegen id : 18] -Output [4]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#58, d_qoy#59] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45, d_date_sk#57, d_year#58, d_qoy#59] +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] (74) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#60, ca_county#61] +Output [2]: [ca_address_sk#69, ca_county#70] (75) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_addr_sk#43] -Right keys [1]: [ca_address_sk#60] +Left keys [1]: [ws_bill_addr_sk#63] +Right keys [1]: [ca_address_sk#69] Join condition: None (76) Project [codegen id : 18] -Output [4]: [ws_ext_sales_price#44, d_year#58, d_qoy#59, ca_county#61] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#58, d_qoy#59, ca_address_sk#60, ca_county#61] +Output [4]: [ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_county#70] +Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_address_sk#69, ca_county#70] (77) HashAggregate [codegen id : 18] -Input [4]: [ws_ext_sales_price#44, d_year#58, d_qoy#59, ca_county#61] -Keys [3]: [ca_county#61, d_qoy#59, d_year#58] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum#62] -Results [4]: [ca_county#61, d_qoy#59, d_year#58, sum#63] +Input [4]: [ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_county#70] +Keys [3]: [ca_county#70, d_qoy#68, d_year#67] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#64))] +Aggregate Attributes [1]: [sum#71] +Results [4]: [ca_county#70, d_qoy#68, d_year#67, sum#72] (78) Exchange -Input [4]: [ca_county#61, d_qoy#59, d_year#58, sum#63] -Arguments: hashpartitioning(ca_county#61, d_qoy#59, d_year#58, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [4]: [ca_county#70, d_qoy#68, d_year#67, sum#72] +Arguments: hashpartitioning(ca_county#70, d_qoy#68, d_year#67, 5), ENSURE_REQUIREMENTS, [id=#73] (79) HashAggregate [codegen id : 19] -Input [4]: [ca_county#61, d_qoy#59, d_year#58, sum#63] -Keys [3]: [ca_county#61, d_qoy#59, d_year#58] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#44))#65] -Results [2]: [ca_county#61, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#44))#65,17,2) AS web_sales#66] +Input [4]: [ca_county#70, d_qoy#68, d_year#67, sum#72] +Keys [3]: [ca_county#70, 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))#74] +Results [2]: [ca_county#70, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#64))#74,17,2) AS web_sales#75] (80) BroadcastExchange -Input [2]: [ca_county#61, web_sales#66] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#67] +Input [2]: [ca_county#70, web_sales#75] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#76] (81) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#50] -Right keys [1]: [ca_county#61] -Join condition: (CASE WHEN (web_sales#55 > 0.00) THEN CheckOverflow((promote_precision(web_sales#66) / promote_precision(web_sales#55)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#16 > 0.00) THEN CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#16)), DecimalType(37,20), true) ELSE null END) +Left keys [1]: [ca_county#56] +Right keys [1]: [ca_county#70] +Join condition: (CASE WHEN (web_sales#61 > 0.00) THEN CheckOverflow((promote_precision(web_sales#75) / promote_precision(web_sales#61)), 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) (82) Project [codegen id : 24] -Output [8]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41, ca_county#50, web_sales#55, web_sales#66] -Input [9]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41, ca_county#50, web_sales#55, ca_county#61, web_sales#66] +Output [8]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47, ca_county#56, web_sales#61, web_sales#75] +Input [9]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47, ca_county#56, web_sales#61, ca_county#70, web_sales#75] (83) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Output [3]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (84) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Input [3]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79] (85) Filter [codegen id : 22] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_bill_addr_sk#43) +Input [3]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79] +Condition : isnotnull(ws_bill_addr_sk#77) (86) ReusedExchange [Reuses operator id: 42] -Output [3]: [d_date_sk#68, d_year#69, d_qoy#70] +Output [3]: [d_date_sk#80, d_year#81, d_qoy#82] (87) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#68] +Left keys [1]: [ws_sold_date_sk#79] +Right keys [1]: [d_date_sk#80] Join condition: None (88) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#69, d_qoy#70] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45, d_date_sk#68, d_year#69, d_qoy#70] +Output [4]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, d_year#81, d_qoy#82] +Input [6]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79, d_date_sk#80, d_year#81, d_qoy#82] (89) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#71, ca_county#72] +Output [2]: [ca_address_sk#83, ca_county#84] (90) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_bill_addr_sk#43] -Right keys [1]: [ca_address_sk#71] +Left keys [1]: [ws_bill_addr_sk#77] +Right keys [1]: [ca_address_sk#83] Join condition: None (91) Project [codegen id : 22] -Output [4]: [ws_ext_sales_price#44, d_year#69, d_qoy#70, ca_county#72] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#69, d_qoy#70, ca_address_sk#71, ca_county#72] +Output [4]: [ws_ext_sales_price#78, d_year#81, d_qoy#82, ca_county#84] +Input [6]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, d_year#81, d_qoy#82, ca_address_sk#83, ca_county#84] (92) HashAggregate [codegen id : 22] -Input [4]: [ws_ext_sales_price#44, d_year#69, d_qoy#70, ca_county#72] -Keys [3]: [ca_county#72, d_qoy#70, d_year#69] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum#73] -Results [4]: [ca_county#72, d_qoy#70, d_year#69, sum#74] +Input [4]: [ws_ext_sales_price#78, d_year#81, d_qoy#82, ca_county#84] +Keys [3]: [ca_county#84, d_qoy#82, d_year#81] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#78))] +Aggregate Attributes [1]: [sum#85] +Results [4]: [ca_county#84, d_qoy#82, d_year#81, sum#86] (93) Exchange -Input [4]: [ca_county#72, d_qoy#70, d_year#69, sum#74] -Arguments: hashpartitioning(ca_county#72, d_qoy#70, d_year#69, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [4]: [ca_county#84, d_qoy#82, d_year#81, sum#86] +Arguments: hashpartitioning(ca_county#84, d_qoy#82, d_year#81, 5), ENSURE_REQUIREMENTS, [id=#87] (94) HashAggregate [codegen id : 23] -Input [4]: [ca_county#72, d_qoy#70, d_year#69, sum#74] -Keys [3]: [ca_county#72, d_qoy#70, d_year#69] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#44))#76] -Results [2]: [ca_county#72, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#44))#76,17,2) AS web_sales#77] +Input [4]: [ca_county#84, d_qoy#82, d_year#81, sum#86] +Keys [3]: [ca_county#84, d_qoy#82, d_year#81] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#78))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#78))#88] +Results [2]: [ca_county#84, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#78))#88,17,2) AS web_sales#89] (95) BroadcastExchange -Input [2]: [ca_county#72, web_sales#77] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#78] +Input [2]: [ca_county#84, web_sales#89] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#90] (96) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#50] -Right keys [1]: [ca_county#72] -Join condition: (CASE WHEN (web_sales#66 > 0.00) THEN CheckOverflow((promote_precision(web_sales#77) / promote_precision(web_sales#66)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#28 > 0.00) THEN CheckOverflow((promote_precision(store_sales#41) / promote_precision(store_sales#28)), DecimalType(37,20), true) ELSE null END) +Left keys [1]: [ca_county#56] +Right keys [1]: [ca_county#84] +Join condition: (CASE WHEN (web_sales#75 > 0.00) THEN CheckOverflow((promote_precision(web_sales#89) / promote_precision(web_sales#75)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#31 > 0.00) THEN CheckOverflow((promote_precision(store_sales#47) / promote_precision(store_sales#31)), DecimalType(37,20), true) ELSE null END) (97) Project [codegen id : 24] -Output [6]: [ca_county#10, d_year#6, CheckOverflow((promote_precision(web_sales#66) / promote_precision(web_sales#55)), DecimalType(37,20), true) AS web_q1_q2_increase#79, CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q1_q2_increase#80, CheckOverflow((promote_precision(web_sales#77) / promote_precision(web_sales#66)), DecimalType(37,20), true) AS web_q2_q3_increase#81, CheckOverflow((promote_precision(store_sales#41) / promote_precision(store_sales#28)), DecimalType(37,20), true) AS store_q2_q3_increase#82] -Input [10]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41, ca_county#50, web_sales#55, web_sales#66, ca_county#72, web_sales#77] +Output [6]: [ca_county#10, d_year#6, CheckOverflow((promote_precision(web_sales#75) / promote_precision(web_sales#61)), DecimalType(37,20), true) AS web_q1_q2_increase#91, CheckOverflow((promote_precision(store_sales#31) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q1_q2_increase#92, CheckOverflow((promote_precision(web_sales#89) / promote_precision(web_sales#75)), DecimalType(37,20), true) AS web_q2_q3_increase#93, CheckOverflow((promote_precision(store_sales#47) / promote_precision(store_sales#31)), DecimalType(37,20), true) AS store_q2_q3_increase#94] +Input [10]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47, ca_county#56, web_sales#61, web_sales#75, ca_county#84, web_sales#89] (98) Exchange -Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#79, store_q1_q2_increase#80, web_q2_q3_increase#81, store_q2_q3_increase#82] -Arguments: rangepartitioning(ca_county#10 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#91, store_q1_q2_increase#92, web_q2_q3_increase#93, store_q2_q3_increase#94] +Arguments: rangepartitioning(ca_county#10 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#95] (99) Sort [codegen id : 25] -Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#79, store_q1_q2_increase#80, web_q2_q3_increase#81, store_q2_q3_increase#82] +Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#91, store_q1_q2_increase#92, web_q2_q3_increase#93, store_q2_q3_increase#94] Arguments: [ca_county#10 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -576,24 +576,24 @@ ReusedExchange (100) (100) ReusedExchange [Reuses operator id: 7] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#17 +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#19 IN dynamicpruning#20 ReusedExchange (101) (101) ReusedExchange [Reuses operator id: 25] -Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] -Subquery:3 Hosting operator id = 36 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#30 +Subquery:3 Hosting operator id = 36 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 ReusedExchange (102) (102) ReusedExchange [Reuses operator id: 42] -Output [3]: [d_date_sk#31, d_year#32, d_qoy#33] +Output [3]: [d_date_sk#37, d_year#38, d_qoy#39] -Subquery:4 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#17 +Subquery:5 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#20 -Subquery:6 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#30 +Subquery:6 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#36 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index 0f35745fc2144..ad918310a918a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -104,13 +104,8 @@ ReadSchema: struct Input [2]: [d_date_sk#11, d_date#12] (16) Filter [codegen id : 2] -<<<<<<< HEAD Input [2]: [d_date_sk#11, d_date#12] Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) -======= -Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) ->>>>>>> abfd9b23cd7c21e9525df85a16e0611ef0f35908 (17) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index 1349aa54f3cc2..eeda8611876d6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -160,30 +160,30 @@ Input [2]: [i_item_sk#13, i_manufact_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_category#15, i_manufact_id#14] +Output [2]: [i_category#15, i_manufact_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#14] +Input [2]: [i_category#15, i_manufact_id#16] (23) Filter [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#14] +Input [2]: [i_category#15, i_manufact_id#16] Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] -Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] -Input [2]: [i_category#15, i_manufact_id#14] +Output [1]: [i_manufact_id#16] +Input [2]: [i_category#15, i_manufact_id#16] (25) BroadcastExchange -Input [1]: [i_manufact_id#14#16] +Input [1]: [i_manufact_id#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_manufact_id#14] -Right keys [1]: [i_manufact_id#14#16] +Right keys [1]: [i_manufact_id#16] Join condition: None (27) BroadcastExchange @@ -233,127 +233,127 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_manufact_id#14] +Output [2]: [i_item_sk#30, i_manufact_id#31] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_manufact_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_manufact_id#14] +Output [2]: [cs_ext_sales_price#26, i_manufact_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_manufact_id#31] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_manufact_id#14] -Keys [1]: [i_manufact_id#14] +Input [2]: [cs_ext_sales_price#26, i_manufact_id#31] +Keys [1]: [i_manufact_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_manufact_id#14, sum#29] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_manufact_id#31, sum#33] (46) Exchange -Input [2]: [i_manufact_id#14, sum#29] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [i_manufact_id#31, sum#33] +Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#14, sum#29] -Keys [1]: [i_manufact_id#14] +Input [2]: [i_manufact_id#31, sum#33] +Keys [1]: [i_manufact_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] (48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] (50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#41] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#42] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_manufact_id#14] +Output [2]: [i_item_sk#43, i_manufact_id#44] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_manufact_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_manufact_id#14] +Output [2]: [ws_ext_sales_price#39, i_manufact_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_manufact_id#44] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_manufact_id#14] -Keys [1]: [i_manufact_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_manufact_id#14, sum#38] +Input [2]: [ws_ext_sales_price#39, i_manufact_id#44] +Keys [1]: [i_manufact_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_manufact_id#44, sum#46] (61) Exchange -Input [2]: [i_manufact_id#14, sum#38] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [2]: [i_manufact_id#44, sum#46] +Arguments: hashpartitioning(i_manufact_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] (62) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#14, sum#38] -Keys [1]: [i_manufact_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +Input [2]: [i_manufact_id#44, sum#46] +Keys [1]: [i_manufact_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_manufact_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] (63) Union @@ -361,23 +361,23 @@ Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price Input [2]: [i_manufact_id#14, total_sales#23] Keys [1]: [i_manufact_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_manufact_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_manufact_id#14, sum#52, isEmpty#53] (65) Exchange -Input [3]: [i_manufact_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] (66) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#14, sum#44, isEmpty#45] +Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] Keys [1]: [i_manufact_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_manufact_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_manufact_id#14, sum(total_sales#23)#55 AS total_sales#56] (67) TakeOrderedAndProject -Input [2]: [i_manufact_id#14, total_sales#48] -Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_manufact_id#14, total_sales#48] +Input [2]: [i_manufact_id#14, total_sales#56] +Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_manufact_id#14, total_sales#56] ===== Subqueries ===== @@ -390,6 +390,6 @@ Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index 1349aa54f3cc2..eeda8611876d6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -160,30 +160,30 @@ Input [2]: [i_item_sk#13, i_manufact_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_category#15, i_manufact_id#14] +Output [2]: [i_category#15, i_manufact_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#14] +Input [2]: [i_category#15, i_manufact_id#16] (23) Filter [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#14] +Input [2]: [i_category#15, i_manufact_id#16] Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] -Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] -Input [2]: [i_category#15, i_manufact_id#14] +Output [1]: [i_manufact_id#16] +Input [2]: [i_category#15, i_manufact_id#16] (25) BroadcastExchange -Input [1]: [i_manufact_id#14#16] +Input [1]: [i_manufact_id#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_manufact_id#14] -Right keys [1]: [i_manufact_id#14#16] +Right keys [1]: [i_manufact_id#16] Join condition: None (27) BroadcastExchange @@ -233,127 +233,127 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_manufact_id#14] +Output [2]: [i_item_sk#30, i_manufact_id#31] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_manufact_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_manufact_id#14] +Output [2]: [cs_ext_sales_price#26, i_manufact_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_manufact_id#31] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_manufact_id#14] -Keys [1]: [i_manufact_id#14] +Input [2]: [cs_ext_sales_price#26, i_manufact_id#31] +Keys [1]: [i_manufact_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_manufact_id#14, sum#29] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_manufact_id#31, sum#33] (46) Exchange -Input [2]: [i_manufact_id#14, sum#29] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [i_manufact_id#31, sum#33] +Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#14, sum#29] -Keys [1]: [i_manufact_id#14] +Input [2]: [i_manufact_id#31, sum#33] +Keys [1]: [i_manufact_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] (48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] (50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#41] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#42] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_manufact_id#14] +Output [2]: [i_item_sk#43, i_manufact_id#44] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_manufact_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_manufact_id#14] +Output [2]: [ws_ext_sales_price#39, i_manufact_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_manufact_id#44] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_manufact_id#14] -Keys [1]: [i_manufact_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_manufact_id#14, sum#38] +Input [2]: [ws_ext_sales_price#39, i_manufact_id#44] +Keys [1]: [i_manufact_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_manufact_id#44, sum#46] (61) Exchange -Input [2]: [i_manufact_id#14, sum#38] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [2]: [i_manufact_id#44, sum#46] +Arguments: hashpartitioning(i_manufact_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] (62) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#14, sum#38] -Keys [1]: [i_manufact_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +Input [2]: [i_manufact_id#44, sum#46] +Keys [1]: [i_manufact_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_manufact_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] (63) Union @@ -361,23 +361,23 @@ Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price Input [2]: [i_manufact_id#14, total_sales#23] Keys [1]: [i_manufact_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_manufact_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_manufact_id#14, sum#52, isEmpty#53] (65) Exchange -Input [3]: [i_manufact_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] (66) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#14, sum#44, isEmpty#45] +Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] Keys [1]: [i_manufact_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_manufact_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_manufact_id#14, sum(total_sales#23)#55 AS total_sales#56] (67) TakeOrderedAndProject -Input [2]: [i_manufact_id#14, total_sales#48] -Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_manufact_id#14, total_sales#48] +Input [2]: [i_manufact_id#14, total_sales#56] +Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_manufact_id#14, total_sales#56] ===== Subqueries ===== @@ -390,6 +390,6 @@ Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt index 7098389df6776..9da5f36eb0553 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt @@ -145,20 +145,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#17] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#10] +Right keys [1]: [d_date_sk#17] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] -Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] +Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] (23) Exchange Input [1]: [ws_bill_customer_sk#15] -Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] @@ -170,38 +170,38 @@ Right keys [1]: [ws_bill_customer_sk#15] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Output [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#21] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [cs_sold_date_sk#20] +Right keys [1]: [d_date_sk#21] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#18] -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] +Output [1]: [cs_ship_customer_sk#19] +Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] (31) Exchange -Input [1]: [cs_ship_customer_sk#18] -Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [cs_ship_customer_sk#19] +Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#18] -Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#19] +Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#19] Join condition: None (34) Filter [codegen id : 12] @@ -214,103 +214,103 @@ Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2 (36) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#21] +Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#23] (37) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 (38) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_state#23] +Output [2]: [ca_address_sk#24, ca_state#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 14] -Input [2]: [ca_address_sk#22, ca_state#23] +Input [2]: [ca_address_sk#24, ca_state#25] (40) Filter [codegen id : 14] -Input [2]: [ca_address_sk#22, ca_state#23] -Condition : isnotnull(ca_address_sk#22) +Input [2]: [ca_address_sk#24, ca_state#25] +Condition : isnotnull(ca_address_sk#24) (41) Exchange -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: hashpartitioning(ca_address_sk#22, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] (42) Sort [codegen id : 15] -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#24] Join condition: None (44) Project [codegen id : 16] -Output [2]: [c_current_cdemo_sk#4, ca_state#23] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] +Output [2]: [c_current_cdemo_sk#4, ca_state#25] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24, ca_state#25] (45) Exchange -Input [2]: [c_current_cdemo_sk#4, ca_state#23] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [2]: [c_current_cdemo_sk#4, ca_state#25] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#27] (46) Sort [codegen id : 17] -Input [2]: [c_current_cdemo_sk#4, ca_state#23] +Input [2]: [c_current_cdemo_sk#4, ca_state#25] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 (47) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 18] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (49) Filter [codegen id : 18] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Condition : isnotnull(cd_demo_sk#26) +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Condition : isnotnull(cd_demo_sk#28) (50) Exchange -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: hashpartitioning(cd_demo_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#34] (51) Sort [codegen id : 19] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: [cd_demo_sk#26 ASC NULLS FIRST], false, 0 +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#26] +Right keys [1]: [cd_demo_sk#28] Join condition: None (53) Project [codegen id : 20] -Output [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [8]: [c_current_cdemo_sk#4, ca_state#25, cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (54) HashAggregate [codegen id : 20] -Input [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#29), partial_max(cd_dep_count#29), partial_avg(cd_dep_count#29), partial_min(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_avg(cd_dep_employed_count#30), partial_min(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_avg(cd_dep_college_count#31)] -Aggregate Attributes [13]: [count#33, min#34, max#35, sum#36, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45] -Results [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] +Input [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Functions [10]: [partial_count(1), partial_min(cd_dep_count#31), partial_max(cd_dep_count#31), partial_avg(cd_dep_count#31), partial_min(cd_dep_employed_count#32), partial_max(cd_dep_employed_count#32), partial_avg(cd_dep_employed_count#32), partial_min(cd_dep_college_count#33), partial_max(cd_dep_college_count#33), partial_avg(cd_dep_college_count#33)] +Aggregate Attributes [13]: [count#35, min#36, max#37, sum#38, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47] +Results [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] (55) Exchange -Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] -Arguments: hashpartitioning(ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] +Arguments: hashpartitioning(ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#61] (56) HashAggregate [codegen id : 21] -Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] -Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [count(1), min(cd_dep_count#29), max(cd_dep_count#29), avg(cd_dep_count#29), min(cd_dep_employed_count#30), max(cd_dep_employed_count#30), avg(cd_dep_employed_count#30), min(cd_dep_college_count#31), max(cd_dep_college_count#31), avg(cd_dep_college_count#31)] -Aggregate Attributes [10]: [count(1)#60, min(cd_dep_count#29)#61, max(cd_dep_count#29)#62, avg(cd_dep_count#29)#63, min(cd_dep_employed_count#30)#64, max(cd_dep_employed_count#30)#65, avg(cd_dep_employed_count#30)#66, min(cd_dep_college_count#31)#67, max(cd_dep_college_count#31)#68, avg(cd_dep_college_count#31)#69] -Results [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, count(1)#60 AS cnt1#70, min(cd_dep_count#29)#61 AS min(cd_dep_count)#71, max(cd_dep_count#29)#62 AS max(cd_dep_count)#72, avg(cd_dep_count#29)#63 AS avg(cd_dep_count)#73, cd_dep_employed_count#30, count(1)#60 AS cnt2#74, min(cd_dep_employed_count#30)#64 AS min(cd_dep_employed_count)#75, max(cd_dep_employed_count#30)#65 AS max(cd_dep_employed_count)#76, avg(cd_dep_employed_count#30)#66 AS avg(cd_dep_employed_count)#77, cd_dep_college_count#31, count(1)#60 AS cnt3#78, min(cd_dep_college_count#31)#67 AS min(cd_dep_college_count)#79, max(cd_dep_college_count#31)#68 AS max(cd_dep_college_count)#80, avg(cd_dep_college_count#31)#69 AS avg(cd_dep_college_count)#81, cd_dep_count#29 AS aggOrder#82] +Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] +Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Functions [10]: [count(1), min(cd_dep_count#31), max(cd_dep_count#31), avg(cd_dep_count#31), min(cd_dep_employed_count#32), max(cd_dep_employed_count#32), avg(cd_dep_employed_count#32), min(cd_dep_college_count#33), max(cd_dep_college_count#33), avg(cd_dep_college_count#33)] +Aggregate Attributes [10]: [count(1)#62, min(cd_dep_count#31)#63, max(cd_dep_count#31)#64, avg(cd_dep_count#31)#65, min(cd_dep_employed_count#32)#66, max(cd_dep_employed_count#32)#67, avg(cd_dep_employed_count#32)#68, min(cd_dep_college_count#33)#69, max(cd_dep_college_count#33)#70, avg(cd_dep_college_count#33)#71] +Results [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, count(1)#62 AS cnt1#72, min(cd_dep_count#31)#63 AS min(cd_dep_count)#73, max(cd_dep_count#31)#64 AS max(cd_dep_count)#74, avg(cd_dep_count#31)#65 AS avg(cd_dep_count)#75, cd_dep_employed_count#32, count(1)#62 AS cnt2#76, min(cd_dep_employed_count#32)#66 AS min(cd_dep_employed_count)#77, max(cd_dep_employed_count#32)#67 AS max(cd_dep_employed_count)#78, avg(cd_dep_employed_count#32)#68 AS avg(cd_dep_employed_count)#79, cd_dep_college_count#33, count(1)#62 AS cnt3#80, min(cd_dep_college_count#33)#69 AS min(cd_dep_college_count)#81, max(cd_dep_college_count#33)#70 AS max(cd_dep_college_count)#82, avg(cd_dep_college_count#33)#71 AS avg(cd_dep_college_count)#83, cd_dep_count#31 AS aggOrder#84] (57) TakeOrderedAndProject -Input [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cnt1#70, min(cd_dep_count)#71, max(cd_dep_count)#72, avg(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, min(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, avg(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, min(cd_dep_college_count)#79, max(cd_dep_college_count)#80, avg(cd_dep_college_count)#81, aggOrder#82] -Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, aggOrder#82 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#23, cd_gender#27, cd_marital_status#28, cnt1#70, min(cd_dep_count)#71, max(cd_dep_count)#72, avg(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, min(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, avg(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, min(cd_dep_college_count)#79, max(cd_dep_college_count)#80, avg(cd_dep_college_count)#81] +Input [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cnt1#72, min(cd_dep_count)#73, max(cd_dep_count)#74, avg(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, min(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, avg(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, min(cd_dep_college_count)#81, max(cd_dep_college_count)#82, avg(cd_dep_college_count)#83, aggOrder#84] +Arguments: 100, [ca_state#25 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, aggOrder#84 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [ca_state#25, cd_gender#29, cd_marital_status#30, cnt1#72, min(cd_dep_count)#73, max(cd_dep_count)#74, avg(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, min(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, avg(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, min(cd_dep_college_count)#81, max(cd_dep_college_count)#82, avg(cd_dep_college_count)#83] ===== Subqueries ===== @@ -323,6 +323,6 @@ Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index 4dab13491784a..48ae824834450 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -122,20 +122,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#16] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] -Right keys [1]: [d_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] +Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#16] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] @@ -143,34 +143,34 @@ Right keys [1]: [ws_bill_customer_sk#14] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#8)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#20] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#9] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#20] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#17] -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] +Output [1]: [cs_ship_customer_sk#18] +Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#20] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [cs_ship_customer_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#17] +Right keys [1]: [cs_ship_customer_sk#18] Join condition: None (29) Filter [codegen id : 9] @@ -182,80 +182,80 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +Output [2]: [ca_address_sk#22, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_state#21] +Input [2]: [ca_address_sk#22, ca_state#23] (33) Filter [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) +Input [2]: [ca_address_sk#22, ca_state#23] +Condition : isnotnull(ca_address_sk#22) (34) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +Input [2]: [ca_address_sk#22, ca_state#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] (35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#22] Join condition: None (36) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#21] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] +Output [2]: [c_current_cdemo_sk#4, ca_state#23] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] (37) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (39) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#25) (40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#25] Join condition: None (42) Project [codegen id : 9] -Output [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Output [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (43) HashAggregate [codegen id : 9] -Input [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#26), partial_max(cd_dep_count#26), partial_avg(cd_dep_count#26), partial_min(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_avg(cd_dep_employed_count#27), partial_min(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_avg(cd_dep_college_count#28)] -Aggregate Attributes [13]: [count#30, min#31, max#32, sum#33, count#34, min#35, max#36, sum#37, count#38, min#39, max#40, sum#41, count#42] -Results [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] +Input [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Functions [10]: [partial_count(1), partial_min(cd_dep_count#28), partial_max(cd_dep_count#28), partial_avg(cd_dep_count#28), partial_min(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_avg(cd_dep_employed_count#29), partial_min(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_avg(cd_dep_college_count#30)] +Aggregate Attributes [13]: [count#32, min#33, max#34, sum#35, count#36, min#37, max#38, sum#39, count#40, min#41, max#42, sum#43, count#44] +Results [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] (44) Exchange -Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] -Arguments: hashpartitioning(ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] +Arguments: hashpartitioning(ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] (45) HashAggregate [codegen id : 10] -Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] -Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), min(cd_dep_count#26), max(cd_dep_count#26), avg(cd_dep_count#26), min(cd_dep_employed_count#27), max(cd_dep_employed_count#27), avg(cd_dep_employed_count#27), min(cd_dep_college_count#28), max(cd_dep_college_count#28), avg(cd_dep_college_count#28)] -Aggregate Attributes [10]: [count(1)#57, min(cd_dep_count#26)#58, max(cd_dep_count#26)#59, avg(cd_dep_count#26)#60, min(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, avg(cd_dep_employed_count#27)#63, min(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, avg(cd_dep_college_count#28)#66] -Results [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, count(1)#57 AS cnt1#67, min(cd_dep_count#26)#58 AS min(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, avg(cd_dep_count#26)#60 AS avg(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, min(cd_dep_employed_count#27)#61 AS min(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, avg(cd_dep_employed_count#27)#63 AS avg(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, min(cd_dep_college_count#28)#64 AS min(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, avg(cd_dep_college_count#28)#66 AS avg(cd_dep_college_count)#78, cd_dep_count#26 AS aggOrder#79] +Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] +Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Functions [10]: [count(1), min(cd_dep_count#28), max(cd_dep_count#28), avg(cd_dep_count#28), min(cd_dep_employed_count#29), max(cd_dep_employed_count#29), avg(cd_dep_employed_count#29), min(cd_dep_college_count#30), max(cd_dep_college_count#30), avg(cd_dep_college_count#30)] +Aggregate Attributes [10]: [count(1)#59, min(cd_dep_count#28)#60, max(cd_dep_count#28)#61, avg(cd_dep_count#28)#62, min(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, avg(cd_dep_employed_count#29)#65, min(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, avg(cd_dep_college_count#30)#68] +Results [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, count(1)#59 AS cnt1#69, min(cd_dep_count#28)#60 AS min(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, avg(cd_dep_count#28)#62 AS avg(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, min(cd_dep_employed_count#29)#63 AS min(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, avg(cd_dep_employed_count#29)#65 AS avg(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, min(cd_dep_college_count#30)#66 AS min(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, avg(cd_dep_college_count#30)#68 AS avg(cd_dep_college_count)#80, cd_dep_count#28 AS aggOrder#81] (46) TakeOrderedAndProject -Input [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78, aggOrder#79] -Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, aggOrder#79 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#21, cd_gender#24, cd_marital_status#25, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78] +Input [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80, aggOrder#81] +Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, aggOrder#81 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#23, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80] ===== Subqueries ===== @@ -268,6 +268,6 @@ Output [1]: [d_date_sk#9] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 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 68ab7a8bbecab..5eb63f2a046cf 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 @@ -191,121 +191,121 @@ Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30] Arguments: [i_item_sk#10 ASC NULLS FIRST, w_warehouse_sk#12 ASC NULLS FIRST], false, 0 (30) Scan parquet default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(inv_date_sk#35), dynamicpruningexpression(inv_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 10] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] (32) Filter [codegen id : 10] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] +Condition : (isnotnull(inv_item_sk#32) AND isnotnull(inv_warehouse_sk#33)) (33) Scan parquet default.date_dim -Output [3]: [d_date_sk#33, d_year#34, d_moy#35] +Output [3]: [d_date_sk#37, d_year#38, d_moy#39] 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 (34) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] (35) Filter [codegen id : 7] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] -Condition : ((((isnotnull(d_year#34) AND isnotnull(d_moy#35)) AND (d_year#34 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#33)) +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] +Condition : ((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2001)) AND (d_moy#39 = 2)) AND isnotnull(d_date_sk#37)) (36) Project [codegen id : 7] -Output [2]: [d_date_sk#33, d_moy#35] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] +Output [2]: [d_date_sk#37, d_moy#39] +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] (37) BroadcastExchange -Input [2]: [d_date_sk#33, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [2]: [d_date_sk#37, d_moy#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [inv_date_sk#35] +Right keys [1]: [d_date_sk#37] Join condition: None (39) Project [codegen id : 10] -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35] -Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#33, d_moy#35] +Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39] +Input [6]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35, d_date_sk#37, d_moy#39] (40) ReusedExchange [Reuses operator id: 14] -Output [1]: [i_item_sk#37] +Output [1]: [i_item_sk#41] (41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#37] +Left keys [1]: [inv_item_sk#32] +Right keys [1]: [i_item_sk#41] Join condition: None (42) Project [codegen id : 10] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37] +Output [4]: [inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41] +Input [5]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41] (43) ReusedExchange [Reuses operator id: 20] -Output [2]: [w_warehouse_sk#38, w_warehouse_name#39] +Output [2]: [w_warehouse_sk#42, w_warehouse_name#43] (44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#38] +Left keys [1]: [inv_warehouse_sk#33] +Right keys [1]: [w_warehouse_sk#42] Join condition: None (45) Project [codegen id : 10] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39, d_moy#35] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39] +Output [5]: [inv_quantity_on_hand#34, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43, d_moy#39] +Input [6]: [inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43] (46) HashAggregate [codegen id : 10] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39, d_moy#35] -Keys [4]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#40, avg#41, m2#42, sum#43, count#44] -Results [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, n#45, avg#46, m2#47, sum#48, count#49] +Input [5]: [inv_quantity_on_hand#34, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43, d_moy#39] +Keys [4]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#34 as double)), partial_avg(inv_quantity_on_hand#34)] +Aggregate Attributes [5]: [n#44, avg#45, m2#46, sum#47, count#48] +Results [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] (47) Exchange -Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, n#45, avg#46, m2#47, sum#48, count#49] -Arguments: hashpartitioning(w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] +Arguments: hashpartitioning(w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, 5), ENSURE_REQUIREMENTS, [id=#54] (48) HashAggregate [codegen id : 11] -Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#51, avg(inv_quantity_on_hand#3)#52] -Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#3 as double))#51 AS stdev#53, avg(inv_quantity_on_hand#3)#52 AS mean#54] +Input [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] +Keys [4]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#34 as double)), avg(inv_quantity_on_hand#34)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#34 as double))#55, avg(inv_quantity_on_hand#34)#56] +Results [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stddev_samp(cast(inv_quantity_on_hand#34 as double))#55 AS stdev#57, avg(inv_quantity_on_hand#34)#56 AS mean#58] (49) Filter [codegen id : 11] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#42, i_item_sk#41, d_moy#39, stdev#57, mean#58] +Condition : ((isnotnull(mean#58) AND isnotnull(stdev#57)) AND (NOT (mean#58 = 0.0) AND ((stdev#57 / mean#58) > 1.0))) (50) Project [codegen id : 11] -Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#35, stdev#53, mean#54] +Output [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, CASE WHEN (mean#58 = 0.0) THEN null ELSE (stdev#57 / mean#58) END AS cov#59] +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stdev#57, mean#58] (51) Exchange -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: hashpartitioning(i_item_sk#37, w_warehouse_sk#38, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: hashpartitioning(i_item_sk#41, w_warehouse_sk#42, 5), ENSURE_REQUIREMENTS, [id=#60] (52) Sort [codegen id : 12] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: [i_item_sk#37 ASC NULLS FIRST, w_warehouse_sk#38 ASC NULLS FIRST], false, 0 +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: [i_item_sk#41 ASC NULLS FIRST, w_warehouse_sk#42 ASC NULLS FIRST], false, 0 (53) SortMergeJoin [codegen id : 13] Left keys [2]: [i_item_sk#10, w_warehouse_sk#12] -Right keys [2]: [i_item_sk#37, w_warehouse_sk#38] +Right keys [2]: [i_item_sk#41, w_warehouse_sk#42] Join condition: None (54) Exchange -Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: rangepartitioning(w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: rangepartitioning(w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, mean#58 ASC NULLS FIRST, cov#59 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#61] (55) Sort [codegen id : 14] -Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: [w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: [w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, mean#58 ASC NULLS FIRST, cov#59 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -316,11 +316,11 @@ ReusedExchange (56) (56) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#6, d_moy#8] -Subquery:2 Hosting operator id = 30 Hosting Expression = inv_date_sk#4 IN dynamicpruning#32 +Subquery:2 Hosting operator id = 30 Hosting Expression = inv_date_sk#35 IN dynamicpruning#36 ReusedExchange (57) (57) ReusedExchange [Reuses operator id: 37] -Output [2]: [d_date_sk#33, d_moy#35] +Output [2]: [d_date_sk#37, d_moy#39] 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 7abd3bb1acb11..e77de53c5adcb 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 @@ -180,117 +180,117 @@ Output [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, CASE WHEN (mean#2 Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] (28) Scan parquet default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Output [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(inv_date_sk#34), dynamicpruningexpression(inv_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] (30) Filter [codegen id : 8] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) +Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] +Condition : (isnotnull(inv_item_sk#31) AND isnotnull(inv_warehouse_sk#32)) (31) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#32] +Output [1]: [i_item_sk#36] (32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [inv_item_sk#31] +Right keys [1]: [i_item_sk#36] Join condition: None (33) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32] +Output [4]: [inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36] +Input [5]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36] (34) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#33, w_warehouse_name#34] +Output [2]: [w_warehouse_sk#37, w_warehouse_name#38] (35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#33] +Left keys [1]: [inv_warehouse_sk#32] +Right keys [1]: [w_warehouse_sk#37] Join condition: None (36) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] +Output [5]: [inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38] +Input [6]: [inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38] (37) Scan parquet default.date_dim -Output [3]: [d_date_sk#35, d_year#36, d_moy#37] +Output [3]: [d_date_sk#39, d_year#40, d_moy#41] 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 (38) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] (39) Filter [codegen id : 7] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2001)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] +Condition : ((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2001)) AND (d_moy#41 = 2)) AND isnotnull(d_date_sk#39)) (40) Project [codegen id : 7] -Output [2]: [d_date_sk#35, d_moy#37] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Output [2]: [d_date_sk#39, d_moy#41] +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] (41) BroadcastExchange -Input [2]: [d_date_sk#35, d_moy#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] +Input [2]: [d_date_sk#39, d_moy#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (42) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#35] +Left keys [1]: [inv_date_sk#34] +Right keys [1]: [d_date_sk#39] Join condition: None (43) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_date_sk#35, d_moy#37] +Output [5]: [inv_quantity_on_hand#33, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_moy#41] +Input [7]: [inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_date_sk#39, d_moy#41] (44) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] -Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#39, avg#40, m2#41, sum#42, count#43] -Results [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] +Input [5]: [inv_quantity_on_hand#33, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_moy#41] +Keys [4]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#33 as double)), partial_avg(inv_quantity_on_hand#33)] +Aggregate Attributes [5]: [n#43, avg#44, m2#45, sum#46, count#47] +Results [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] (45) Exchange -Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] -Arguments: hashpartitioning(w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] +Arguments: hashpartitioning(w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, 5), ENSURE_REQUIREMENTS, [id=#53] (46) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] -Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#50, avg(inv_quantity_on_hand#3)#51] -Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_quantity_on_hand#3 as double))#50 AS stdev#52, avg(inv_quantity_on_hand#3)#51 AS mean#53] +Input [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] +Keys [4]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#33 as double)), avg(inv_quantity_on_hand#33)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#33 as double))#54, avg(inv_quantity_on_hand#33)#55] +Results [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stddev_samp(cast(inv_quantity_on_hand#33 as double))#54 AS stdev#56, avg(inv_quantity_on_hand#33)#55 AS mean#57] (47) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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#37, i_item_sk#36, d_moy#41, stdev#56, mean#57] +Condition : ((isnotnull(mean#57) AND isnotnull(stdev#56)) AND (NOT (mean#57 = 0.0) AND ((stdev#56 / mean#57) > 1.0))) (48) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#52, mean#53] +Output [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, CASE WHEN (mean#57 = 0.0) THEN null ELSE (stdev#56 / mean#57) END AS cov#58] +Input [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stdev#56, mean#57] (49) BroadcastExchange -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [id=#59] (50) BroadcastHashJoin [codegen id : 10] Left keys [2]: [i_item_sk#6, w_warehouse_sk#8] -Right keys [2]: [i_item_sk#32, w_warehouse_sk#33] +Right keys [2]: [i_item_sk#36, w_warehouse_sk#37] Join condition: None (51) Exchange -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, cov#54] -Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#37 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#13, mean#29, cov#30, w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, mean#57 ASC NULLS FIRST, cov#58 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#60] (52) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, cov#54] -Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#37 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#13, mean#29, cov#30, w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, mean#57 ASC NULLS FIRST, cov#58 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -301,11 +301,11 @@ ReusedExchange (53) (53) ReusedExchange [Reuses operator id: 20] Output [2]: [d_date_sk#11, d_moy#13] -Subquery:2 Hosting operator id = 28 Hosting Expression = inv_date_sk#4 IN dynamicpruning#31 +Subquery:2 Hosting operator id = 28 Hosting Expression = inv_date_sk#34 IN dynamicpruning#35 ReusedExchange (54) (54) ReusedExchange [Reuses operator id: 41] -Output [2]: [d_date_sk#35, d_moy#37] +Output [2]: [d_date_sk#39, d_moy#41] 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 f0dfbac4f3a0f..e2e3760fe2d03 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 @@ -191,121 +191,121 @@ Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30] Arguments: [i_item_sk#10 ASC NULLS FIRST, w_warehouse_sk#12 ASC NULLS FIRST], false, 0 (30) Scan parquet default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(inv_date_sk#35), dynamicpruningexpression(inv_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 10] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] (32) Filter [codegen id : 10] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] +Condition : (isnotnull(inv_item_sk#32) AND isnotnull(inv_warehouse_sk#33)) (33) Scan parquet default.date_dim -Output [3]: [d_date_sk#33, d_year#34, d_moy#35] +Output [3]: [d_date_sk#37, d_year#38, d_moy#39] 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 (34) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] (35) Filter [codegen id : 7] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] -Condition : ((((isnotnull(d_year#34) AND isnotnull(d_moy#35)) AND (d_year#34 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#33)) +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] +Condition : ((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2001)) AND (d_moy#39 = 2)) AND isnotnull(d_date_sk#37)) (36) Project [codegen id : 7] -Output [2]: [d_date_sk#33, d_moy#35] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] +Output [2]: [d_date_sk#37, d_moy#39] +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] (37) BroadcastExchange -Input [2]: [d_date_sk#33, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [2]: [d_date_sk#37, d_moy#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [inv_date_sk#35] +Right keys [1]: [d_date_sk#37] Join condition: None (39) Project [codegen id : 10] -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35] -Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#33, d_moy#35] +Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39] +Input [6]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35, d_date_sk#37, d_moy#39] (40) ReusedExchange [Reuses operator id: 14] -Output [1]: [i_item_sk#37] +Output [1]: [i_item_sk#41] (41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#37] +Left keys [1]: [inv_item_sk#32] +Right keys [1]: [i_item_sk#41] Join condition: None (42) Project [codegen id : 10] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37] +Output [4]: [inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41] +Input [5]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41] (43) ReusedExchange [Reuses operator id: 20] -Output [2]: [w_warehouse_sk#38, w_warehouse_name#39] +Output [2]: [w_warehouse_sk#42, w_warehouse_name#43] (44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#38] +Left keys [1]: [inv_warehouse_sk#33] +Right keys [1]: [w_warehouse_sk#42] Join condition: None (45) Project [codegen id : 10] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39, d_moy#35] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39] +Output [5]: [inv_quantity_on_hand#34, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43, d_moy#39] +Input [6]: [inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43] (46) HashAggregate [codegen id : 10] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39, d_moy#35] -Keys [4]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#40, avg#41, m2#42, sum#43, count#44] -Results [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, n#45, avg#46, m2#47, sum#48, count#49] +Input [5]: [inv_quantity_on_hand#34, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43, d_moy#39] +Keys [4]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#34 as double)), partial_avg(inv_quantity_on_hand#34)] +Aggregate Attributes [5]: [n#44, avg#45, m2#46, sum#47, count#48] +Results [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] (47) Exchange -Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, n#45, avg#46, m2#47, sum#48, count#49] -Arguments: hashpartitioning(w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] +Arguments: hashpartitioning(w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, 5), ENSURE_REQUIREMENTS, [id=#54] (48) HashAggregate [codegen id : 11] -Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#51, avg(inv_quantity_on_hand#3)#52] -Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#3 as double))#51 AS stdev#53, avg(inv_quantity_on_hand#3)#52 AS mean#54] +Input [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] +Keys [4]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#34 as double)), avg(inv_quantity_on_hand#34)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#34 as double))#55, avg(inv_quantity_on_hand#34)#56] +Results [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stddev_samp(cast(inv_quantity_on_hand#34 as double))#55 AS stdev#57, avg(inv_quantity_on_hand#34)#56 AS mean#58] (49) Filter [codegen id : 11] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#42, i_item_sk#41, d_moy#39, stdev#57, mean#58] +Condition : ((isnotnull(mean#58) AND isnotnull(stdev#57)) AND (NOT (mean#58 = 0.0) AND ((stdev#57 / mean#58) > 1.0))) (50) Project [codegen id : 11] -Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#35, stdev#53, mean#54] +Output [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, CASE WHEN (mean#58 = 0.0) THEN null ELSE (stdev#57 / mean#58) END AS cov#59] +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stdev#57, mean#58] (51) Exchange -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: hashpartitioning(i_item_sk#37, w_warehouse_sk#38, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: hashpartitioning(i_item_sk#41, w_warehouse_sk#42, 5), ENSURE_REQUIREMENTS, [id=#60] (52) Sort [codegen id : 12] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: [i_item_sk#37 ASC NULLS FIRST, w_warehouse_sk#38 ASC NULLS FIRST], false, 0 +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: [i_item_sk#41 ASC NULLS FIRST, w_warehouse_sk#42 ASC NULLS FIRST], false, 0 (53) SortMergeJoin [codegen id : 13] Left keys [2]: [i_item_sk#10, w_warehouse_sk#12] -Right keys [2]: [i_item_sk#37, w_warehouse_sk#38] +Right keys [2]: [i_item_sk#41, w_warehouse_sk#42] Join condition: None (54) Exchange -Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: rangepartitioning(w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: rangepartitioning(w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, mean#58 ASC NULLS FIRST, cov#59 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#61] (55) Sort [codegen id : 14] -Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: [w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: [w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, mean#58 ASC NULLS FIRST, cov#59 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -316,11 +316,11 @@ ReusedExchange (56) (56) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#6, d_moy#8] -Subquery:2 Hosting operator id = 30 Hosting Expression = inv_date_sk#4 IN dynamicpruning#32 +Subquery:2 Hosting operator id = 30 Hosting Expression = inv_date_sk#35 IN dynamicpruning#36 ReusedExchange (57) (57) ReusedExchange [Reuses operator id: 37] -Output [2]: [d_date_sk#33, d_moy#35] +Output [2]: [d_date_sk#37, d_moy#39] 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 d1a5ecaa6a4d8..d60cd37ce7bf1 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 @@ -180,117 +180,117 @@ Output [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, CASE WHEN (mean#2 Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] (28) Scan parquet default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Output [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(inv_date_sk#34), dynamicpruningexpression(inv_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] (30) Filter [codegen id : 8] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) +Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] +Condition : (isnotnull(inv_item_sk#31) AND isnotnull(inv_warehouse_sk#32)) (31) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#32] +Output [1]: [i_item_sk#36] (32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [inv_item_sk#31] +Right keys [1]: [i_item_sk#36] Join condition: None (33) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32] +Output [4]: [inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36] +Input [5]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36] (34) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#33, w_warehouse_name#34] +Output [2]: [w_warehouse_sk#37, w_warehouse_name#38] (35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#33] +Left keys [1]: [inv_warehouse_sk#32] +Right keys [1]: [w_warehouse_sk#37] Join condition: None (36) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] +Output [5]: [inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38] +Input [6]: [inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38] (37) Scan parquet default.date_dim -Output [3]: [d_date_sk#35, d_year#36, d_moy#37] +Output [3]: [d_date_sk#39, d_year#40, d_moy#41] 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 (38) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] (39) Filter [codegen id : 7] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2001)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] +Condition : ((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2001)) AND (d_moy#41 = 2)) AND isnotnull(d_date_sk#39)) (40) Project [codegen id : 7] -Output [2]: [d_date_sk#35, d_moy#37] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Output [2]: [d_date_sk#39, d_moy#41] +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] (41) BroadcastExchange -Input [2]: [d_date_sk#35, d_moy#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] +Input [2]: [d_date_sk#39, d_moy#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (42) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#35] +Left keys [1]: [inv_date_sk#34] +Right keys [1]: [d_date_sk#39] Join condition: None (43) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_date_sk#35, d_moy#37] +Output [5]: [inv_quantity_on_hand#33, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_moy#41] +Input [7]: [inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_date_sk#39, d_moy#41] (44) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] -Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#39, avg#40, m2#41, sum#42, count#43] -Results [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] +Input [5]: [inv_quantity_on_hand#33, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_moy#41] +Keys [4]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#33 as double)), partial_avg(inv_quantity_on_hand#33)] +Aggregate Attributes [5]: [n#43, avg#44, m2#45, sum#46, count#47] +Results [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] (45) Exchange -Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] -Arguments: hashpartitioning(w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] +Arguments: hashpartitioning(w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, 5), ENSURE_REQUIREMENTS, [id=#53] (46) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] -Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#50, avg(inv_quantity_on_hand#3)#51] -Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_quantity_on_hand#3 as double))#50 AS stdev#52, avg(inv_quantity_on_hand#3)#51 AS mean#53] +Input [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] +Keys [4]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#33 as double)), avg(inv_quantity_on_hand#33)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#33 as double))#54, avg(inv_quantity_on_hand#33)#55] +Results [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stddev_samp(cast(inv_quantity_on_hand#33 as double))#54 AS stdev#56, avg(inv_quantity_on_hand#33)#55 AS mean#57] (47) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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#37, i_item_sk#36, d_moy#41, stdev#56, mean#57] +Condition : ((isnotnull(mean#57) AND isnotnull(stdev#56)) AND (NOT (mean#57 = 0.0) AND ((stdev#56 / mean#57) > 1.0))) (48) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#52, mean#53] +Output [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, CASE WHEN (mean#57 = 0.0) THEN null ELSE (stdev#56 / mean#57) END AS cov#58] +Input [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stdev#56, mean#57] (49) BroadcastExchange -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [id=#59] (50) BroadcastHashJoin [codegen id : 10] Left keys [2]: [i_item_sk#6, w_warehouse_sk#8] -Right keys [2]: [i_item_sk#32, w_warehouse_sk#33] +Right keys [2]: [i_item_sk#36, w_warehouse_sk#37] Join condition: None (51) Exchange -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, cov#54] -Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#37 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#13, mean#29, cov#30, w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, mean#57 ASC NULLS FIRST, cov#58 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#60] (52) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, cov#54] -Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#37 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#13, mean#29, cov#30, w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, mean#57 ASC NULLS FIRST, cov#58 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -301,11 +301,11 @@ ReusedExchange (53) (53) ReusedExchange [Reuses operator id: 20] Output [2]: [d_date_sk#11, d_moy#13] -Subquery:2 Hosting operator id = 28 Hosting Expression = inv_date_sk#4 IN dynamicpruning#31 +Subquery:2 Hosting operator id = 28 Hosting Expression = inv_date_sk#34 IN dynamicpruning#35 ReusedExchange (54) (54) ReusedExchange [Reuses operator id: 41] -Output [2]: [d_date_sk#35, d_moy#37] +Output [2]: [d_date_sk#39, d_moy#41] 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 7b6bd35bfe180..ce8b8bed5a26e 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 @@ -239,465 +239,465 @@ Input [2]: [customer_id#27, year_total#28] Arguments: [customer_id#27 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6] +Output [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ss_sold_date_sk#35), dynamicpruningexpression(ss_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6] +Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] (27) Filter [codegen id : 10] -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_customer_sk#1) +Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Condition : isnotnull(ss_customer_sk#30) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#37, d_year#38] 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 (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#8, d_year#9] +Input [2]: [d_date_sk#37, d_year#38] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#8, d_year#9] -Condition : ((isnotnull(d_year#9) AND (d_year#9 = 2002)) AND isnotnull(d_date_sk#8)) +Input [2]: [d_date_sk#37, d_year#38] +Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_sk#37)) (31) BroadcastExchange -Input [2]: [d_date_sk#8, d_year#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [2]: [d_date_sk#37, d_year#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#39] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ss_sold_date_sk#35] +Right keys [1]: [d_date_sk#37] Join condition: None (33) Project [codegen id : 10] -Output [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] -Input [8]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6, d_date_sk#8, d_year#9] +Output [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Input [8]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#37, d_year#38] (34) Exchange -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] -Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Arguments: hashpartitioning(ss_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#40] (35) Sort [codegen id : 11] -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] -Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Arguments: [ss_customer_sk#30 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] (37) Sort [codegen id : 13] -Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] -Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +Arguments: [c_customer_sk#41 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#12] +Left keys [1]: [ss_customer_sk#30] +Right keys [1]: [c_customer_sk#41] Join condition: None (39) Project [codegen id : 14] -Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] -Input [14]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Input [14]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38, c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] (40) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#35, isEmpty#36] +Input [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#49, isEmpty#50] +Results [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, sum#51, isEmpty#52] (41) Exchange -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#35, isEmpty#36] -Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, sum#51, isEmpty#52] +Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, 5), ENSURE_REQUIREMENTS, [id=#53] (42) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#35, isEmpty#36] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 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#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#38] -Results [8]: [c_customer_id#13 AS customer_id#39, c_first_name#14 AS customer_first_name#40, c_last_name#15 AS customer_last_name#41, c_preferred_cust_flag#16 AS customer_preferred_cust_flag#42, c_birth_country#17 AS customer_birth_country#43, c_login#18 AS customer_login#44, c_email_address#19 AS customer_email_address#45, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#38 AS year_total#46] +Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, sum#51, isEmpty#52] +Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 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#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#54] +Results [8]: [c_customer_id#42 AS customer_id#55, c_first_name#43 AS customer_first_name#56, c_last_name#44 AS customer_last_name#57, c_preferred_cust_flag#45 AS customer_preferred_cust_flag#58, c_birth_country#46 AS customer_birth_country#59, c_login#47 AS customer_login#60, c_email_address#48 AS customer_email_address#61, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#54 AS year_total#62] (43) Exchange -Input [8]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46] -Arguments: hashpartitioning(customer_id#39, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [8]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62] +Arguments: hashpartitioning(customer_id#55, 5), ENSURE_REQUIREMENTS, [id=#63] (44) Sort [codegen id : 16] -Input [8]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46] -Arguments: [customer_id#39 ASC NULLS FIRST], false, 0 +Input [8]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62] +Arguments: [customer_id#55 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#39] +Right keys [1]: [customer_id#55] Join condition: None (46) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] +Output [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#53), dynamicpruningexpression(cs_sold_date_sk#53 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#69), dynamicpruningexpression(cs_sold_date_sk#69 IN dynamicpruning#7)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] +Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69] (48) Filter [codegen id : 19] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] -Condition : isnotnull(cs_bill_customer_sk#48) +Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69] +Condition : isnotnull(cs_bill_customer_sk#64) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#70, d_year#71] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#69] +Right keys [1]: [d_date_sk#70] Join condition: None (51) Project [codegen id : 19] -Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Input [8]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53, d_date_sk#8, d_year#9] +Output [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] +Input [8]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69, d_date_sk#70, d_year#71] (52) Exchange -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Arguments: hashpartitioning(cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] +Arguments: hashpartitioning(cs_bill_customer_sk#64, 5), ENSURE_REQUIREMENTS, [id=#72] (53) Sort [codegen id : 20] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Arguments: [cs_bill_customer_sk#48 ASC NULLS FIRST], false, 0 +Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] +Arguments: [cs_bill_customer_sk#64 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] (55) Sort [codegen id : 22] -Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] -Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Arguments: [c_customer_sk#73 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#48] -Right keys [1]: [c_customer_sk#12] +Left keys [1]: [cs_bill_customer_sk#64] +Right keys [1]: [c_customer_sk#73] Join condition: None (57) Project [codegen id : 23] -Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Input [14]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] +Input [14]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71, c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] (58) HashAggregate [codegen id : 23] -Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#55, isEmpty#56] -Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#57, isEmpty#58] +Input [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] +Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#81, isEmpty#82] +Results [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, sum#83, isEmpty#84] (59) Exchange -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#57, isEmpty#58] -Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, sum#83, isEmpty#84] +Arguments: hashpartitioning(c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, 5), ENSURE_REQUIREMENTS, [id=#85] (60) HashAggregate [codegen id : 24] -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#57, isEmpty#58] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 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#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#60] -Results [2]: [c_customer_id#13 AS customer_id#61, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#60 AS year_total#62] +Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, sum#83, isEmpty#84] +Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 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#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86] +Results [2]: [c_customer_id#74 AS customer_id#87, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86 AS year_total#88] (61) Filter [codegen id : 24] -Input [2]: [customer_id#61, year_total#62] -Condition : (isnotnull(year_total#62) AND (year_total#62 > 0.000000)) +Input [2]: [customer_id#87, year_total#88] +Condition : (isnotnull(year_total#88) AND (year_total#88 > 0.000000)) (62) Project [codegen id : 24] -Output [2]: [customer_id#61 AS customer_id#63, year_total#62 AS year_total#64] -Input [2]: [customer_id#61, year_total#62] +Output [2]: [customer_id#87 AS customer_id#89, year_total#88 AS year_total#90] +Input [2]: [customer_id#87, year_total#88] (63) Exchange -Input [2]: [customer_id#63, year_total#64] -Arguments: hashpartitioning(customer_id#63, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [2]: [customer_id#89, year_total#90] +Arguments: hashpartitioning(customer_id#89, 5), ENSURE_REQUIREMENTS, [id=#91] (64) Sort [codegen id : 25] -Input [2]: [customer_id#63, year_total#64] -Arguments: [customer_id#63 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#89, year_total#90] +Arguments: [customer_id#89 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#63] +Right keys [1]: [customer_id#89] Join condition: None (66) Project [codegen id : 26] -Output [11]: [customer_id#27, year_total#28, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46, year_total#64] -Input [12]: [customer_id#27, year_total#28, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46, customer_id#63, year_total#64] +Output [11]: [customer_id#27, year_total#28, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62, year_total#90] +Input [12]: [customer_id#27, year_total#28, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62, customer_id#89, year_total#90] (67) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] +Output [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#53), dynamicpruningexpression(cs_sold_date_sk#53 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(cs_sold_date_sk#97), dynamicpruningexpression(cs_sold_date_sk#97 IN dynamicpruning#36)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] +Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97] (69) Filter [codegen id : 28] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] -Condition : isnotnull(cs_bill_customer_sk#48) +Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97] +Condition : isnotnull(cs_bill_customer_sk#92) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#98, d_year#99] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#97] +Right keys [1]: [d_date_sk#98] Join condition: None (72) Project [codegen id : 28] -Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Input [8]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53, d_date_sk#8, d_year#9] +Output [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] +Input [8]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97, d_date_sk#98, d_year#99] (73) Exchange -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Arguments: hashpartitioning(cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] +Arguments: hashpartitioning(cs_bill_customer_sk#92, 5), ENSURE_REQUIREMENTS, [id=#100] (74) Sort [codegen id : 29] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Arguments: [cs_bill_customer_sk#48 ASC NULLS FIRST], false, 0 +Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] +Arguments: [cs_bill_customer_sk#92 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] (76) Sort [codegen id : 31] -Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] -Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] +Arguments: [c_customer_sk#101 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [cs_bill_customer_sk#48] -Right keys [1]: [c_customer_sk#12] +Left keys [1]: [cs_bill_customer_sk#92] +Right keys [1]: [c_customer_sk#101] Join condition: None (78) Project [codegen id : 32] -Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Input [14]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] +Input [14]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99, c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] (79) HashAggregate [codegen id : 32] -Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#67, isEmpty#68] -Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#69, isEmpty#70] +Input [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] +Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#109, isEmpty#110] +Results [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, sum#111, isEmpty#112] (80) Exchange -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#69, isEmpty#70] -Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, sum#111, isEmpty#112] +Arguments: hashpartitioning(c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, 5), ENSURE_REQUIREMENTS, [id=#113] (81) HashAggregate [codegen id : 33] -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#69, isEmpty#70] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 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#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#72] -Results [2]: [c_customer_id#13 AS customer_id#73, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#72 AS year_total#74] +Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, sum#111, isEmpty#112] +Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 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#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#114] +Results [2]: [c_customer_id#102 AS customer_id#115, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#114 AS year_total#116] (82) Exchange -Input [2]: [customer_id#73, year_total#74] -Arguments: hashpartitioning(customer_id#73, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [2]: [customer_id#115, year_total#116] +Arguments: hashpartitioning(customer_id#115, 5), ENSURE_REQUIREMENTS, [id=#117] (83) Sort [codegen id : 34] -Input [2]: [customer_id#73, year_total#74] -Arguments: [customer_id#73 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#115, year_total#116] +Arguments: [customer_id#115 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#73] -Join condition: (CASE WHEN (year_total#64 > 0.000000) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#64)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#28 > 0.000000) THEN CheckOverflow((promote_precision(year_total#46) / promote_precision(year_total#28)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#115] +Join condition: (CASE WHEN (year_total#90 > 0.000000) THEN CheckOverflow((promote_precision(year_total#116) / promote_precision(year_total#90)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#28 > 0.000000) THEN CheckOverflow((promote_precision(year_total#62) / promote_precision(year_total#28)), DecimalType(38,14), true) ELSE null END) (85) Project [codegen id : 35] -Output [10]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74] -Input [13]: [customer_id#27, year_total#28, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46, year_total#64, customer_id#73, year_total#74] +Output [10]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116] +Input [13]: [customer_id#27, year_total#28, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62, year_total#90, customer_id#115, year_total#116] (86) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] +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#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ws_sold_date_sk#123), dynamicpruningexpression(ws_sold_date_sk#123 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 37] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] +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] (88) Filter [codegen id : 37] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_bill_customer_sk#76) +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) (89) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#124, d_year#125] (90) BroadcastHashJoin [codegen id : 37] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ws_sold_date_sk#123] +Right keys [1]: [d_date_sk#124] Join condition: None (91) Project [codegen id : 37] -Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Input [8]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#8, d_year#9] +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, d_year#125] +Input [8]: [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, d_date_sk#124, d_year#125] (92) Exchange -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Arguments: hashpartitioning(ws_bill_customer_sk#76, 5), ENSURE_REQUIREMENTS, [id=#82] +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, d_year#125] +Arguments: hashpartitioning(ws_bill_customer_sk#118, 5), ENSURE_REQUIREMENTS, [id=#126] (93) Sort [codegen id : 38] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Arguments: [ws_bill_customer_sk#76 ASC NULLS FIRST], false, 0 +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, d_year#125] +Arguments: [ws_bill_customer_sk#118 ASC NULLS FIRST], false, 0 (94) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [8]: [c_customer_sk#127, c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134] (95) Sort [codegen id : 40] -Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] -Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#127, c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134] +Arguments: [c_customer_sk#127 ASC NULLS FIRST], false, 0 (96) SortMergeJoin [codegen id : 41] -Left keys [1]: [ws_bill_customer_sk#76] -Right keys [1]: [c_customer_sk#12] +Left keys [1]: [ws_bill_customer_sk#118] +Right keys [1]: [c_customer_sk#127] Join condition: None (97) Project [codegen id : 41] -Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Input [14]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [12]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, d_year#125] +Input [14]: [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, d_year#125, c_customer_sk#127, c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134] (98) HashAggregate [codegen id : 41] -Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#83, isEmpty#84] -Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#85, isEmpty#86] +Input [12]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, d_year#125] +Keys [8]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125] +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#135, isEmpty#136] +Results [10]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, sum#137, isEmpty#138] (99) Exchange -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#85, isEmpty#86] -Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#87] +Input [10]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, sum#137, isEmpty#138] +Arguments: hashpartitioning(c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, 5), ENSURE_REQUIREMENTS, [id=#139] (100) HashAggregate [codegen id : 42] -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#85, isEmpty#86] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 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#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#88] -Results [2]: [c_customer_id#13 AS customer_id#89, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#88 AS year_total#90] +Input [10]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, sum#137, isEmpty#138] +Keys [8]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125] +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))#140] +Results [2]: [c_customer_id#128 AS customer_id#141, 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))#140 AS year_total#142] (101) Filter [codegen id : 42] -Input [2]: [customer_id#89, year_total#90] -Condition : (isnotnull(year_total#90) AND (year_total#90 > 0.000000)) +Input [2]: [customer_id#141, year_total#142] +Condition : (isnotnull(year_total#142) AND (year_total#142 > 0.000000)) (102) Project [codegen id : 42] -Output [2]: [customer_id#89 AS customer_id#91, year_total#90 AS year_total#92] -Input [2]: [customer_id#89, year_total#90] +Output [2]: [customer_id#141 AS customer_id#143, year_total#142 AS year_total#144] +Input [2]: [customer_id#141, year_total#142] (103) Exchange -Input [2]: [customer_id#91, year_total#92] -Arguments: hashpartitioning(customer_id#91, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [2]: [customer_id#143, year_total#144] +Arguments: hashpartitioning(customer_id#143, 5), ENSURE_REQUIREMENTS, [id=#145] (104) Sort [codegen id : 43] -Input [2]: [customer_id#91, year_total#92] -Arguments: [customer_id#91 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#143, year_total#144] +Arguments: [customer_id#143 ASC NULLS FIRST], false, 0 (105) SortMergeJoin [codegen id : 44] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#91] +Right keys [1]: [customer_id#143] Join condition: None (106) Project [codegen id : 44] -Output [11]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74, year_total#92] -Input [12]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74, customer_id#91, year_total#92] +Output [11]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116, year_total#144] +Input [12]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116, customer_id#143, year_total#144] (107) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] +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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ws_sold_date_sk#151), dynamicpruningexpression(ws_sold_date_sk#151 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 46] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] +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] (109) Filter [codegen id : 46] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_bill_customer_sk#76) +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) (110) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#152, d_year#153] (111) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ws_sold_date_sk#151] +Right keys [1]: [d_date_sk#152] Join condition: None (112) Project [codegen id : 46] -Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Input [8]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#8, d_year#9] +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, d_year#153] +Input [8]: [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, d_date_sk#152, d_year#153] (113) Exchange -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Arguments: hashpartitioning(ws_bill_customer_sk#76, 5), ENSURE_REQUIREMENTS, [id=#94] +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, d_year#153] +Arguments: hashpartitioning(ws_bill_customer_sk#146, 5), ENSURE_REQUIREMENTS, [id=#154] (114) Sort [codegen id : 47] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Arguments: [ws_bill_customer_sk#76 ASC NULLS FIRST], false, 0 +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, d_year#153] +Arguments: [ws_bill_customer_sk#146 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] (116) Sort [codegen id : 49] -Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] -Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] +Arguments: [c_customer_sk#155 ASC NULLS FIRST], false, 0 (117) SortMergeJoin [codegen id : 50] -Left keys [1]: [ws_bill_customer_sk#76] -Right keys [1]: [c_customer_sk#12] +Left keys [1]: [ws_bill_customer_sk#146] +Right keys [1]: [c_customer_sk#155] Join condition: None (118) Project [codegen id : 50] -Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Input [14]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] +Input [14]: [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, d_year#153, c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] (119) HashAggregate [codegen id : 50] -Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#95, isEmpty#96] -Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#97, isEmpty#98] +Input [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] +Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] +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#163, isEmpty#164] +Results [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] (120) Exchange -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#97, isEmpty#98] -Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#99] +Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] +Arguments: hashpartitioning(c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, 5), ENSURE_REQUIREMENTS, [id=#167] (121) HashAggregate [codegen id : 51] -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#97, isEmpty#98] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 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#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#100] -Results [2]: [c_customer_id#13 AS customer_id#101, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#100 AS year_total#102] +Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] +Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] +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))#168] +Results [2]: [c_customer_id#156 AS customer_id#169, 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))#168 AS year_total#170] (122) Exchange -Input [2]: [customer_id#101, year_total#102] -Arguments: hashpartitioning(customer_id#101, 5), ENSURE_REQUIREMENTS, [id=#103] +Input [2]: [customer_id#169, year_total#170] +Arguments: hashpartitioning(customer_id#169, 5), ENSURE_REQUIREMENTS, [id=#171] (123) Sort [codegen id : 52] -Input [2]: [customer_id#101, year_total#102] -Arguments: [customer_id#101 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#169, year_total#170] +Arguments: [customer_id#169 ASC NULLS FIRST], false, 0 (124) SortMergeJoin [codegen id : 53] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#101] -Join condition: (CASE WHEN (year_total#64 > 0.000000) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#64)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#92 > 0.000000) THEN CheckOverflow((promote_precision(year_total#102) / promote_precision(year_total#92)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#169] +Join condition: (CASE WHEN (year_total#90 > 0.000000) THEN CheckOverflow((promote_precision(year_total#116) / promote_precision(year_total#90)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#144 > 0.000000) THEN CheckOverflow((promote_precision(year_total#170) / promote_precision(year_total#144)), DecimalType(38,14), true) ELSE null END) (125) Project [codegen id : 53] -Output [7]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45] -Input [13]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74, year_total#92, customer_id#101, year_total#102] +Output [7]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61] +Input [13]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116, year_total#144, customer_id#169, year_total#170] (126) TakeOrderedAndProject -Input [7]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45] -Arguments: 100, [customer_id#39 ASC NULLS FIRST, customer_first_name#40 ASC NULLS FIRST, customer_last_name#41 ASC NULLS FIRST, customer_preferred_cust_flag#42 ASC NULLS FIRST, customer_birth_country#43 ASC NULLS FIRST, customer_login#44 ASC NULLS FIRST, customer_email_address#45 ASC NULLS FIRST], [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45] +Input [7]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61] +Arguments: 100, [customer_id#55 ASC NULLS FIRST, customer_first_name#56 ASC NULLS FIRST, customer_last_name#57 ASC NULLS FIRST, customer_preferred_cust_flag#58 ASC NULLS FIRST, customer_birth_country#59 ASC NULLS FIRST, customer_login#60 ASC NULLS FIRST, customer_email_address#61 ASC NULLS FIRST], [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61] ===== Subqueries ===== @@ -708,19 +708,19 @@ ReusedExchange (127) (127) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#8, d_year#9] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#30 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 ReusedExchange (128) (128) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#37, d_year#38] -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#69 IN dynamicpruning#7 -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#30 +Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#97 IN dynamicpruning#36 -Subquery:5 Hosting operator id = 86 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#7 +Subquery:5 Hosting operator id = 86 Hosting Expression = ws_sold_date_sk#123 IN dynamicpruning#7 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#30 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#151 IN dynamicpruning#36 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 80dee3d164511..1cf27630d2994 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 @@ -209,460 +209,460 @@ Input [2]: [customer_id#26, year_total#27] Condition : (isnotnull(year_total#27) AND (year_total#27 > 0.000000)) (20) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] (22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] +Condition : (isnotnull(c_customer_sk#28) AND isnotnull(c_customer_id#29)) (23) Scan parquet default.store_sales -Output [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Output [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(ss_sold_date_sk#41), dynamicpruningexpression(ss_sold_date_sk#41 IN dynamicpruning#42)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] (25) Filter [codegen id : 4] -Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -Condition : isnotnull(ss_customer_sk#9) +Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] +Condition : isnotnull(ss_customer_sk#36) (26) BroadcastExchange -Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#9] +Left keys [1]: [c_customer_sk#28] +Right keys [1]: [ss_customer_sk#36] Join condition: None (28) Project [codegen id : 6] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Output [12]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] +Input [14]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#44, d_year#45] 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 (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#17, d_year#18] +Input [2]: [d_date_sk#44, d_year#45] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#17, d_year#18] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2002)) AND isnotnull(d_date_sk#17)) +Input [2]: [d_date_sk#44, d_year#45] +Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) (32) BroadcastExchange -Input [2]: [d_date_sk#17, d_year#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [2]: [d_date_sk#44, d_year#45] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#46] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#14] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ss_sold_date_sk#41] +Right keys [1]: [d_date_sk#44] Join condition: None (34) Project [codegen id : 6] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#18] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14, d_date_sk#17, d_year#18] +Output [12]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#45] +Input [14]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41, d_date_sk#44, d_year#45] (35) HashAggregate [codegen id : 6] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#18] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#31, isEmpty#32] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#33, isEmpty#34] +Input [12]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#45] +Keys [8]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#47, isEmpty#48] +Results [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, sum#49, isEmpty#50] (36) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#33, isEmpty#34] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, sum#49, isEmpty#50] +Arguments: hashpartitioning(c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, 5), ENSURE_REQUIREMENTS, [id=#51] (37) HashAggregate [codegen id : 7] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#33, isEmpty#34] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 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#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#36] -Results [8]: [c_customer_id#2 AS customer_id#37, c_first_name#3 AS customer_first_name#38, c_last_name#4 AS customer_last_name#39, c_preferred_cust_flag#5 AS customer_preferred_cust_flag#40, c_birth_country#6 AS customer_birth_country#41, c_login#7 AS customer_login#42, c_email_address#8 AS customer_email_address#43, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#36 AS year_total#44] +Input [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, sum#49, isEmpty#50] +Keys [8]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 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#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#52] +Results [8]: [c_customer_id#29 AS customer_id#53, c_first_name#30 AS customer_first_name#54, c_last_name#31 AS customer_last_name#55, c_preferred_cust_flag#32 AS customer_preferred_cust_flag#56, c_birth_country#33 AS customer_birth_country#57, c_login#34 AS customer_login#58, c_email_address#35 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#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#52 AS year_total#60] (38) BroadcastExchange -Input [8]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#45] +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: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#61] (39) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#37] +Right keys [1]: [customer_id#53] Join condition: None (40) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] (42) Filter [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] +Condition : (isnotnull(c_customer_sk#62) AND isnotnull(c_customer_id#63)) (43) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Output [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#51), dynamicpruningexpression(cs_sold_date_sk#51 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cs_sold_date_sk#75), dynamicpruningexpression(cs_sold_date_sk#75 IN dynamicpruning#15)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Input [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] (45) Filter [codegen id : 8] -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Condition : isnotnull(cs_bill_customer_sk#46) +Input [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] +Condition : isnotnull(cs_bill_customer_sk#70) (46) BroadcastExchange -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] +Input [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_bill_customer_sk#46] +Left keys [1]: [c_customer_sk#62] +Right keys [1]: [cs_bill_customer_sk#70] Join condition: None (48) Project [codegen id : 10] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] +Input [14]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#77, d_year#78] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#51] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [cs_sold_date_sk#75] +Right keys [1]: [d_date_sk#77] Join condition: None (51) Project [codegen id : 10] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51, d_date_sk#17, d_year#18] +Output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, d_year#78] +Input [14]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75, d_date_sk#77, d_year#78] (52) HashAggregate [codegen id : 10] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#53, isEmpty#54] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#55, isEmpty#56] +Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, d_year#78] +Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 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#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, sum#81, isEmpty#82] (53) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#55, isEmpty#56] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, sum#81, isEmpty#82] +Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#83] (54) HashAggregate [codegen id : 11] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#55, isEmpty#56] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 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#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#58] -Results [2]: [c_customer_id#2 AS customer_id#59, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#58 AS year_total#60] +Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, sum#81, isEmpty#82] +Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 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#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#84] +Results [2]: [c_customer_id#63 AS customer_id#85, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#84 AS year_total#86] (55) Filter [codegen id : 11] -Input [2]: [customer_id#59, year_total#60] -Condition : (isnotnull(year_total#60) AND (year_total#60 > 0.000000)) +Input [2]: [customer_id#85, year_total#86] +Condition : (isnotnull(year_total#86) AND (year_total#86 > 0.000000)) (56) Project [codegen id : 11] -Output [2]: [customer_id#59 AS customer_id#61, year_total#60 AS year_total#62] -Input [2]: [customer_id#59, year_total#60] +Output [2]: [customer_id#85 AS customer_id#87, year_total#86 AS year_total#88] +Input [2]: [customer_id#85, year_total#86] (57) BroadcastExchange -Input [2]: [customer_id#61, year_total#62] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#63] +Input [2]: [customer_id#87, year_total#88] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#89] (58) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#61] +Right keys [1]: [customer_id#87] Join condition: None (59) Project [codegen id : 24] -Output [11]: [customer_id#26, year_total#27, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44, year_total#62] -Input [12]: [customer_id#26, year_total#27, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44, customer_id#61, year_total#62] +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#88] +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#87, year_total#88] (60) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97] (62) Filter [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97] +Condition : (isnotnull(c_customer_sk#90) AND isnotnull(c_customer_id#91)) (63) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Output [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#51), dynamicpruningexpression(cs_sold_date_sk#51 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(cs_sold_date_sk#103), dynamicpruningexpression(cs_sold_date_sk#103 IN dynamicpruning#42)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Input [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] (65) Filter [codegen id : 12] -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Condition : isnotnull(cs_bill_customer_sk#46) +Input [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] +Condition : isnotnull(cs_bill_customer_sk#98) (66) BroadcastExchange -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] +Input [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#104] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_bill_customer_sk#46] +Left keys [1]: [c_customer_sk#90] +Right keys [1]: [cs_bill_customer_sk#98] Join condition: None (68) Project [codegen id : 14] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Output [12]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] +Input [14]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#105, d_year#106] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#51] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [cs_sold_date_sk#103] +Right keys [1]: [d_date_sk#105] Join condition: None (71) Project [codegen id : 14] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51, d_date_sk#17, d_year#18] +Output [12]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, d_year#106] +Input [14]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103, d_date_sk#105, d_year#106] (72) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#65, isEmpty#66] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#67, isEmpty#68] +Input [12]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, d_year#106] +Keys [8]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#107, isEmpty#108] +Results [10]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, sum#109, isEmpty#110] (73) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#67, isEmpty#68] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [10]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, sum#109, isEmpty#110] +Arguments: hashpartitioning(c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, 5), ENSURE_REQUIREMENTS, [id=#111] (74) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#67, isEmpty#68] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 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#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#70] -Results [2]: [c_customer_id#2 AS customer_id#71, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#70 AS year_total#72] +Input [10]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, sum#109, isEmpty#110] +Keys [8]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 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#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112] +Results [2]: [c_customer_id#91 AS customer_id#113, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112 AS year_total#114] (75) BroadcastExchange -Input [2]: [customer_id#71, year_total#72] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#73] +Input [2]: [customer_id#113, year_total#114] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#115] (76) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#71] -Join condition: (CASE WHEN (year_total#62 > 0.000000) THEN CheckOverflow((promote_precision(year_total#72) / promote_precision(year_total#62)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#27 > 0.000000) THEN CheckOverflow((promote_precision(year_total#44) / promote_precision(year_total#27)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#113] +Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), 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) (77) Project [codegen id : 24] -Output [10]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72] -Input [13]: [customer_id#26, year_total#27, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44, year_total#62, customer_id#71, year_total#72] +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#88, year_total#114] +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#88, customer_id#113, year_total#114] (78) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (79) ColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123] (80) Filter [codegen id : 18] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123] +Condition : (isnotnull(c_customer_sk#116) AND isnotnull(c_customer_id#117)) (81) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Output [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(ws_sold_date_sk#129), dynamicpruningexpression(ws_sold_date_sk#129 IN dynamicpruning#15)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (82) ColumnarToRow [codegen id : 16] -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Input [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] (83) Filter [codegen id : 16] -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Condition : isnotnull(ws_bill_customer_sk#74) +Input [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] +Condition : isnotnull(ws_bill_customer_sk#124) (84) BroadcastExchange -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#80] +Input [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#130] (85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#74] +Left keys [1]: [c_customer_sk#116] +Right keys [1]: [ws_bill_customer_sk#124] Join condition: None (86) Project [codegen id : 18] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Output [12]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] +Input [14]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] (87) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#131, d_year#132] (88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#79] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ws_sold_date_sk#129] +Right keys [1]: [d_date_sk#131] Join condition: None (89) Project [codegen id : 18] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#17, d_year#18] +Output [12]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, d_year#132] +Input [14]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129, d_date_sk#131, d_year#132] (90) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#81, isEmpty#82] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#83, isEmpty#84] +Input [12]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, d_year#132] +Keys [8]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#133, isEmpty#134] +Results [10]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, sum#135, isEmpty#136] (91) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#83, isEmpty#84] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [10]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, sum#135, isEmpty#136] +Arguments: hashpartitioning(c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, 5), ENSURE_REQUIREMENTS, [id=#137] (92) HashAggregate [codegen id : 19] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#83, isEmpty#84] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 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#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86] -Results [2]: [c_customer_id#2 AS customer_id#87, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86 AS year_total#88] +Input [10]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, sum#135, isEmpty#136] +Keys [8]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 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#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138] +Results [2]: [c_customer_id#117 AS customer_id#139, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138 AS year_total#140] (93) Filter [codegen id : 19] -Input [2]: [customer_id#87, year_total#88] -Condition : (isnotnull(year_total#88) AND (year_total#88 > 0.000000)) +Input [2]: [customer_id#139, year_total#140] +Condition : (isnotnull(year_total#140) AND (year_total#140 > 0.000000)) (94) Project [codegen id : 19] -Output [2]: [customer_id#87 AS customer_id#89, year_total#88 AS year_total#90] -Input [2]: [customer_id#87, year_total#88] +Output [2]: [customer_id#139 AS customer_id#141, year_total#140 AS year_total#142] +Input [2]: [customer_id#139, year_total#140] (95) BroadcastExchange -Input [2]: [customer_id#89, year_total#90] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#91] +Input [2]: [customer_id#141, year_total#142] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#143] (96) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#89] +Right keys [1]: [customer_id#141] Join condition: None (97) Project [codegen id : 24] -Output [11]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72, year_total#90] -Input [12]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72, customer_id#89, year_total#90] +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#88, year_total#114, year_total#142] +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#88, year_total#114, customer_id#141, year_total#142] (98) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (99) ColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] (100) Filter [codegen id : 22] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] +Condition : (isnotnull(c_customer_sk#144) AND isnotnull(c_customer_id#145)) (101) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Output [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(ws_sold_date_sk#157), dynamicpruningexpression(ws_sold_date_sk#157 IN dynamicpruning#42)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 20] -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] (103) Filter [codegen id : 20] -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Condition : isnotnull(ws_bill_customer_sk#74) +Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Condition : isnotnull(ws_bill_customer_sk#152) (104) BroadcastExchange -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] +Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#158] (105) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#74] +Left keys [1]: [c_customer_sk#144] +Right keys [1]: [ws_bill_customer_sk#152] Join condition: None (106) Project [codegen id : 22] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Output [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Input [14]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] (107) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#159, d_year#160] (108) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#79] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ws_sold_date_sk#157] +Right keys [1]: [d_date_sk#159] Join condition: None (109) Project [codegen id : 22] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#17, d_year#18] +Output [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, d_year#160] +Input [14]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157, d_date_sk#159, d_year#160] (110) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#93, isEmpty#94] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#95, isEmpty#96] +Input [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, d_year#160] +Keys [8]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#161, isEmpty#162] +Results [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] (111) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#95, isEmpty#96] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#97] +Input [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] +Arguments: hashpartitioning(c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, 5), ENSURE_REQUIREMENTS, [id=#165] (112) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#95, isEmpty#96] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 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#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#98] -Results [2]: [c_customer_id#2 AS customer_id#99, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#98 AS year_total#100] +Input [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] +Keys [8]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 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#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166] +Results [2]: [c_customer_id#145 AS customer_id#167, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166 AS year_total#168] (113) BroadcastExchange -Input [2]: [customer_id#99, year_total#100] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#101] +Input [2]: [customer_id#167, year_total#168] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#169] (114) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#99] -Join condition: (CASE WHEN (year_total#62 > 0.000000) THEN CheckOverflow((promote_precision(year_total#72) / promote_precision(year_total#62)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#90 > 0.000000) THEN CheckOverflow((promote_precision(year_total#100) / promote_precision(year_total#90)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#167] +Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#142 > 0.000000) THEN CheckOverflow((promote_precision(year_total#168) / promote_precision(year_total#142)), DecimalType(38,14), true) ELSE null END) (115) Project [codegen id : 24] -Output [7]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43] -Input [13]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72, year_total#90, customer_id#99, year_total#100] +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#88, year_total#114, year_total#142, customer_id#167, year_total#168] (116) TakeOrderedAndProject -Input [7]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43] -Arguments: 100, [customer_id#37 ASC NULLS FIRST, customer_first_name#38 ASC NULLS FIRST, customer_last_name#39 ASC NULLS FIRST, customer_preferred_cust_flag#40 ASC NULLS FIRST, customer_birth_country#41 ASC NULLS FIRST, customer_login#42 ASC NULLS FIRST, customer_email_address#43 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43] +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] ===== Subqueries ===== @@ -673,19 +673,19 @@ ReusedExchange (117) (117) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#17, d_year#18] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#28 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#41 IN dynamicpruning#42 ReusedExchange (118) (118) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#44, d_year#45] -Subquery:3 Hosting operator id = 43 Hosting Expression = cs_sold_date_sk#51 IN dynamicpruning#15 +Subquery:3 Hosting operator id = 43 Hosting Expression = cs_sold_date_sk#75 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 63 Hosting Expression = cs_sold_date_sk#51 IN dynamicpruning#28 +Subquery:4 Hosting operator id = 63 Hosting Expression = cs_sold_date_sk#103 IN dynamicpruning#42 -Subquery:5 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#129 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#28 +Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#157 IN dynamicpruning#42 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt index 0c227d75c67c4..034c0fe9eb535 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt @@ -65,13 +65,8 @@ Aggregate Attributes [1]: [count#9] Results [2]: [i_manufact#5, count#10] (10) Exchange -<<<<<<< HEAD Input [2]: [i_manufact#5, count#10] Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [id=#11] -======= -Input [2]: [i_manufact#2, count#9] -Arguments: hashpartitioning(i_manufact#2, 5), ENSURE_REQUIREMENTS, [id=#10] ->>>>>>> abfd9b23cd7c21e9525df85a16e0611ef0f35908 (11) HashAggregate [codegen id : 2] Input [2]: [i_manufact#5, count#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index 0c227d75c67c4..034c0fe9eb535 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -65,13 +65,8 @@ Aggregate Attributes [1]: [count#9] Results [2]: [i_manufact#5, count#10] (10) Exchange -<<<<<<< HEAD Input [2]: [i_manufact#5, count#10] Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [id=#11] -======= -Input [2]: [i_manufact#2, count#9] -Arguments: hashpartitioning(i_manufact#2, 5), ENSURE_REQUIREMENTS, [id=#10] ->>>>>>> abfd9b23cd7c21e9525df85a16e0611ef0f35908 (11) HashAggregate [codegen id : 2] Input [2]: [i_manufact#5, count#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt index fdb95bd5de9d3..84a8547e3f6ef 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt @@ -102,98 +102,98 @@ Output [2]: [item_sk#11, rnk#17] Input [3]: [item_sk#11, rank_col#12, rnk#17] (15) ReusedExchange [Reuses operator id: 6] -Output [3]: [ss_item_sk#1, sum#18, count#19] +Output [3]: [ss_item_sk#18, sum#19, count#20] (16) HashAggregate [codegen id : 5] -Input [3]: [ss_item_sk#1, sum#18, count#19] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#20] -Results [3]: [ss_item_sk#1 AS item_sk#21, cast((avg(UnscaledValue(ss_net_profit#3))#20 / 100.0) as decimal(11,6)) AS rank_col#22, cast((avg(UnscaledValue(ss_net_profit#3))#20 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#3)#23] +Input [3]: [ss_item_sk#18, sum#19, count#20] +Keys [1]: [ss_item_sk#18] +Functions [1]: [avg(UnscaledValue(ss_net_profit#21))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#21))#22] +Results [3]: [ss_item_sk#18 AS item_sk#23, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS rank_col#24, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#21)#25] (17) Filter [codegen id : 5] -Input [3]: [item_sk#21, rank_col#22, avg(ss_net_profit#3)#23] -Condition : (isnotnull(avg(ss_net_profit#3)#23) AND (cast(avg(ss_net_profit#3)#23 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) +Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] +Condition : (isnotnull(avg(ss_net_profit#21)#25) AND (cast(avg(ss_net_profit#21)#25 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) (18) Project [codegen id : 5] -Output [2]: [item_sk#21, rank_col#22] -Input [3]: [item_sk#21, rank_col#22, avg(ss_net_profit#3)#23] +Output [2]: [item_sk#23, rank_col#24] +Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] (19) Exchange -Input [2]: [item_sk#21, rank_col#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [2]: [item_sk#23, rank_col#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] (20) Sort [codegen id : 6] -Input [2]: [item_sk#21, rank_col#22] -Arguments: [rank_col#22 DESC NULLS LAST], false, 0 +Input [2]: [item_sk#23, rank_col#24] +Arguments: [rank_col#24 DESC NULLS LAST], false, 0 (21) Window -Input [2]: [item_sk#21, rank_col#22] -Arguments: [rank(rank_col#22) windowspecdefinition(rank_col#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#25], [rank_col#22 DESC NULLS LAST] +Input [2]: [item_sk#23, rank_col#24] +Arguments: [rank(rank_col#24) windowspecdefinition(rank_col#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#27], [rank_col#24 DESC NULLS LAST] (22) Filter [codegen id : 7] -Input [3]: [item_sk#21, rank_col#22, rnk#25] -Condition : ((rnk#25 < 11) AND isnotnull(item_sk#21)) +Input [3]: [item_sk#23, rank_col#24, rnk#27] +Condition : ((rnk#27 < 11) AND isnotnull(item_sk#23)) (23) Project [codegen id : 7] -Output [2]: [item_sk#21, rnk#25] -Input [3]: [item_sk#21, rank_col#22, rnk#25] +Output [2]: [item_sk#23, rnk#27] +Input [3]: [item_sk#23, rank_col#24, rnk#27] (24) BroadcastExchange -Input [2]: [item_sk#21, rnk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#26] +Input [2]: [item_sk#23, rnk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#28] (25) BroadcastHashJoin [codegen id : 10] Left keys [1]: [rnk#17] -Right keys [1]: [rnk#25] +Right keys [1]: [rnk#27] Join condition: None (26) Project [codegen id : 10] -Output [3]: [item_sk#11, rnk#17, item_sk#21] -Input [4]: [item_sk#11, rnk#17, item_sk#21, rnk#25] +Output [3]: [item_sk#11, rnk#17, item_sk#23] +Input [4]: [item_sk#11, rnk#17, item_sk#23, rnk#27] (27) Scan parquet default.item -Output [2]: [i_item_sk#27, i_product_name#28] +Output [2]: [i_item_sk#29, i_product_name#30] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 8] -Input [2]: [i_item_sk#27, i_product_name#28] +Input [2]: [i_item_sk#29, i_product_name#30] (29) Filter [codegen id : 8] -Input [2]: [i_item_sk#27, i_product_name#28] -Condition : isnotnull(i_item_sk#27) +Input [2]: [i_item_sk#29, i_product_name#30] +Condition : isnotnull(i_item_sk#29) (30) BroadcastExchange -Input [2]: [i_item_sk#27, i_product_name#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [2]: [i_item_sk#29, i_product_name#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [item_sk#11] -Right keys [1]: [i_item_sk#27] +Right keys [1]: [i_item_sk#29] Join condition: None (32) Project [codegen id : 10] -Output [3]: [rnk#17, item_sk#21, i_product_name#28] -Input [5]: [item_sk#11, rnk#17, item_sk#21, i_item_sk#27, i_product_name#28] +Output [3]: [rnk#17, item_sk#23, i_product_name#30] +Input [5]: [item_sk#11, rnk#17, item_sk#23, i_item_sk#29, i_product_name#30] (33) ReusedExchange [Reuses operator id: 30] -Output [2]: [i_item_sk#30, i_product_name#31] +Output [2]: [i_item_sk#32, i_product_name#33] (34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [item_sk#21] -Right keys [1]: [i_item_sk#30] +Left keys [1]: [item_sk#23] +Right keys [1]: [i_item_sk#32] Join condition: None (35) Project [codegen id : 10] -Output [3]: [rnk#17, i_product_name#28 AS best_performing#32, i_product_name#31 AS worst_performing#33] -Input [5]: [rnk#17, item_sk#21, i_product_name#28, i_item_sk#30, i_product_name#31] +Output [3]: [rnk#17, i_product_name#30 AS best_performing#34, i_product_name#33 AS worst_performing#35] +Input [5]: [rnk#17, item_sk#23, i_product_name#30, i_item_sk#32, i_product_name#33] (36) TakeOrderedAndProject -Input [3]: [rnk#17, best_performing#32, worst_performing#33] -Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#32, worst_performing#33] +Input [3]: [rnk#17, best_performing#34, worst_performing#35] +Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#34, worst_performing#35] ===== Subqueries ===== @@ -208,40 +208,40 @@ Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery (37) Scan parquet default.store_sales -Output [4]: [ss_addr_sk#34, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Output [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 1] -Input [4]: [ss_addr_sk#34, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] (39) Filter [codegen id : 1] -Input [4]: [ss_addr_sk#34, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) AND isnull(ss_addr_sk#34)) +Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] +Condition : ((isnotnull(ss_store_sk#37) AND (ss_store_sk#37 = 4)) AND isnull(ss_addr_sk#36)) (40) Project [codegen id : 1] -Output [2]: [ss_store_sk#2, ss_net_profit#3] -Input [4]: [ss_addr_sk#34, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Output [2]: [ss_store_sk#37, ss_net_profit#38] +Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] (41) HashAggregate [codegen id : 1] -Input [2]: [ss_store_sk#2, ss_net_profit#3] -Keys [1]: [ss_store_sk#2] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#35, count#36] -Results [3]: [ss_store_sk#2, sum#37, count#38] +Input [2]: [ss_store_sk#37, ss_net_profit#38] +Keys [1]: [ss_store_sk#37] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#38))] +Aggregate Attributes [2]: [sum#40, count#41] +Results [3]: [ss_store_sk#37, sum#42, count#43] (42) Exchange -Input [3]: [ss_store_sk#2, sum#37, count#38] -Arguments: hashpartitioning(ss_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [3]: [ss_store_sk#37, sum#42, count#43] +Arguments: hashpartitioning(ss_store_sk#37, 5), ENSURE_REQUIREMENTS, [id=#44] (43) HashAggregate [codegen id : 2] -Input [3]: [ss_store_sk#2, sum#37, count#38] -Keys [1]: [ss_store_sk#2] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#40] -Results [1]: [cast((avg(UnscaledValue(ss_net_profit#3))#40 / 100.0) as decimal(11,6)) AS rank_col#41] +Input [3]: [ss_store_sk#37, sum#42, count#43] +Keys [1]: [ss_store_sk#37] +Functions [1]: [avg(UnscaledValue(ss_net_profit#38))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#38))#45] +Results [1]: [cast((avg(UnscaledValue(ss_net_profit#38))#45 / 100.0) as decimal(11,6)) AS rank_col#46] Subquery:2 Hosting operator id = 17 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index f52ad2e63762e..b42ddd8db7836 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -107,98 +107,98 @@ Input [2]: [item_sk#11, rnk#17] Arguments: [rnk#17 ASC NULLS FIRST], false, 0 (16) ReusedExchange [Reuses operator id: 6] -Output [3]: [ss_item_sk#1, sum#18, count#19] +Output [3]: [ss_item_sk#18, sum#19, count#20] (17) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#1, sum#18, count#19] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#20] -Results [3]: [ss_item_sk#1 AS item_sk#21, cast((avg(UnscaledValue(ss_net_profit#3))#20 / 100.0) as decimal(11,6)) AS rank_col#22, cast((avg(UnscaledValue(ss_net_profit#3))#20 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#3)#23] +Input [3]: [ss_item_sk#18, sum#19, count#20] +Keys [1]: [ss_item_sk#18] +Functions [1]: [avg(UnscaledValue(ss_net_profit#21))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#21))#22] +Results [3]: [ss_item_sk#18 AS item_sk#23, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS rank_col#24, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#21)#25] (18) Filter [codegen id : 6] -Input [3]: [item_sk#21, rank_col#22, avg(ss_net_profit#3)#23] -Condition : (isnotnull(avg(ss_net_profit#3)#23) AND (cast(avg(ss_net_profit#3)#23 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) +Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] +Condition : (isnotnull(avg(ss_net_profit#21)#25) AND (cast(avg(ss_net_profit#21)#25 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) (19) Project [codegen id : 6] -Output [2]: [item_sk#21, rank_col#22] -Input [3]: [item_sk#21, rank_col#22, avg(ss_net_profit#3)#23] +Output [2]: [item_sk#23, rank_col#24] +Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] (20) Exchange -Input [2]: [item_sk#21, rank_col#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [2]: [item_sk#23, rank_col#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] (21) Sort [codegen id : 7] -Input [2]: [item_sk#21, rank_col#22] -Arguments: [rank_col#22 DESC NULLS LAST], false, 0 +Input [2]: [item_sk#23, rank_col#24] +Arguments: [rank_col#24 DESC NULLS LAST], false, 0 (22) Window -Input [2]: [item_sk#21, rank_col#22] -Arguments: [rank(rank_col#22) windowspecdefinition(rank_col#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#25], [rank_col#22 DESC NULLS LAST] +Input [2]: [item_sk#23, rank_col#24] +Arguments: [rank(rank_col#24) windowspecdefinition(rank_col#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#27], [rank_col#24 DESC NULLS LAST] (23) Filter [codegen id : 8] -Input [3]: [item_sk#21, rank_col#22, rnk#25] -Condition : ((rnk#25 < 11) AND isnotnull(item_sk#21)) +Input [3]: [item_sk#23, rank_col#24, rnk#27] +Condition : ((rnk#27 < 11) AND isnotnull(item_sk#23)) (24) Project [codegen id : 8] -Output [2]: [item_sk#21, rnk#25] -Input [3]: [item_sk#21, rank_col#22, rnk#25] +Output [2]: [item_sk#23, rnk#27] +Input [3]: [item_sk#23, rank_col#24, rnk#27] (25) Sort [codegen id : 8] -Input [2]: [item_sk#21, rnk#25] -Arguments: [rnk#25 ASC NULLS FIRST], false, 0 +Input [2]: [item_sk#23, rnk#27] +Arguments: [rnk#27 ASC NULLS FIRST], false, 0 (26) SortMergeJoin [codegen id : 11] Left keys [1]: [rnk#17] -Right keys [1]: [rnk#25] +Right keys [1]: [rnk#27] Join condition: None (27) Project [codegen id : 11] -Output [3]: [item_sk#11, rnk#17, item_sk#21] -Input [4]: [item_sk#11, rnk#17, item_sk#21, rnk#25] +Output [3]: [item_sk#11, rnk#17, item_sk#23] +Input [4]: [item_sk#11, rnk#17, item_sk#23, rnk#27] (28) Scan parquet default.item -Output [2]: [i_item_sk#26, i_product_name#27] +Output [2]: [i_item_sk#28, i_product_name#29] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_sk#26, i_product_name#27] +Input [2]: [i_item_sk#28, i_product_name#29] (30) Filter [codegen id : 9] -Input [2]: [i_item_sk#26, i_product_name#27] -Condition : isnotnull(i_item_sk#26) +Input [2]: [i_item_sk#28, i_product_name#29] +Condition : isnotnull(i_item_sk#28) (31) BroadcastExchange -Input [2]: [i_item_sk#26, i_product_name#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] +Input [2]: [i_item_sk#28, i_product_name#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [item_sk#11] -Right keys [1]: [i_item_sk#26] +Right keys [1]: [i_item_sk#28] Join condition: None (33) Project [codegen id : 11] -Output [3]: [rnk#17, item_sk#21, i_product_name#27] -Input [5]: [item_sk#11, rnk#17, item_sk#21, i_item_sk#26, i_product_name#27] +Output [3]: [rnk#17, item_sk#23, i_product_name#29] +Input [5]: [item_sk#11, rnk#17, item_sk#23, i_item_sk#28, i_product_name#29] (34) ReusedExchange [Reuses operator id: 31] -Output [2]: [i_item_sk#29, i_product_name#30] +Output [2]: [i_item_sk#31, i_product_name#32] (35) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [item_sk#21] -Right keys [1]: [i_item_sk#29] +Left keys [1]: [item_sk#23] +Right keys [1]: [i_item_sk#31] Join condition: None (36) Project [codegen id : 11] -Output [3]: [rnk#17, i_product_name#27 AS best_performing#31, i_product_name#30 AS worst_performing#32] -Input [5]: [rnk#17, item_sk#21, i_product_name#27, i_item_sk#29, i_product_name#30] +Output [3]: [rnk#17, i_product_name#29 AS best_performing#33, i_product_name#32 AS worst_performing#34] +Input [5]: [rnk#17, item_sk#23, i_product_name#29, i_item_sk#31, i_product_name#32] (37) TakeOrderedAndProject -Input [3]: [rnk#17, best_performing#31, worst_performing#32] -Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#31, worst_performing#32] +Input [3]: [rnk#17, best_performing#33, worst_performing#34] +Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#33, worst_performing#34] ===== Subqueries ===== @@ -213,40 +213,40 @@ Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery (38) Scan parquet default.store_sales -Output [4]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Output [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 1] -Input [4]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] (40) Filter [codegen id : 1] -Input [4]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) AND isnull(ss_addr_sk#33)) +Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_store_sk#36) AND (ss_store_sk#36 = 4)) AND isnull(ss_addr_sk#35)) (41) Project [codegen id : 1] -Output [2]: [ss_store_sk#2, ss_net_profit#3] -Input [4]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Output [2]: [ss_store_sk#36, ss_net_profit#37] +Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] (42) HashAggregate [codegen id : 1] -Input [2]: [ss_store_sk#2, ss_net_profit#3] -Keys [1]: [ss_store_sk#2] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#34, count#35] -Results [3]: [ss_store_sk#2, sum#36, count#37] +Input [2]: [ss_store_sk#36, ss_net_profit#37] +Keys [1]: [ss_store_sk#36] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#37))] +Aggregate Attributes [2]: [sum#39, count#40] +Results [3]: [ss_store_sk#36, sum#41, count#42] (43) Exchange -Input [3]: [ss_store_sk#2, sum#36, count#37] -Arguments: hashpartitioning(ss_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [3]: [ss_store_sk#36, sum#41, count#42] +Arguments: hashpartitioning(ss_store_sk#36, 5), ENSURE_REQUIREMENTS, [id=#43] (44) HashAggregate [codegen id : 2] -Input [3]: [ss_store_sk#2, sum#36, count#37] -Keys [1]: [ss_store_sk#2] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#39] -Results [1]: [cast((avg(UnscaledValue(ss_net_profit#3))#39 / 100.0) as decimal(11,6)) AS rank_col#40] +Input [3]: [ss_store_sk#36, sum#41, count#42] +Keys [1]: [ss_store_sk#36] +Functions [1]: [avg(UnscaledValue(ss_net_profit#37))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#37))#44] +Results [1]: [cast((avg(UnscaledValue(ss_net_profit#37))#44 / 100.0) as decimal(11,6)) AS rank_col#45] Subquery:2 Hosting operator id = 18 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt index 22f746eb1d91a..701414b22eb80 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt @@ -199,30 +199,30 @@ Output [4]: [ws_sales_price#4, ca_city#19, ca_zip#20, i_item_id#12] Input [6]: [ws_bill_customer_sk#3, ws_sales_price#4, i_item_id#12, c_customer_sk#15, ca_city#19, ca_zip#20] (35) Scan parquet default.item -Output [2]: [i_item_sk#11, i_item_id#12] +Output [2]: [i_item_sk#23, i_item_id#24] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_item_sk, [2,3,5,7,11,13,17,19,23,29])] ReadSchema: struct (36) ColumnarToRow [codegen id : 11] -Input [2]: [i_item_sk#11, i_item_id#12] +Input [2]: [i_item_sk#23, i_item_id#24] (37) Filter [codegen id : 11] -Input [2]: [i_item_sk#11, i_item_id#12] -Condition : i_item_sk#11 IN (2,3,5,7,11,13,17,19,23,29) +Input [2]: [i_item_sk#23, i_item_id#24] +Condition : i_item_sk#23 IN (2,3,5,7,11,13,17,19,23,29) (38) Project [codegen id : 11] -Output [1]: [i_item_id#12 AS i_item_id#12#23] -Input [2]: [i_item_sk#11, i_item_id#12] +Output [1]: [i_item_id#24] +Input [2]: [i_item_sk#23, i_item_id#24] (39) BroadcastExchange -Input [1]: [i_item_id#12#23] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#24] +Input [1]: [i_item_id#24] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#25] (40) BroadcastHashJoin [codegen id : 12] Left keys [1]: [i_item_id#12] -Right keys [1]: [i_item_id#12#23] +Right keys [1]: [i_item_id#24] Join condition: None (41) Filter [codegen id : 12] @@ -237,23 +237,23 @@ Input [5]: [ws_sales_price#4, ca_city#19, ca_zip#20, i_item_id#12, exists#1] Input [3]: [ws_sales_price#4, ca_city#19, ca_zip#20] Keys [2]: [ca_zip#20, ca_city#19] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#25] -Results [3]: [ca_zip#20, ca_city#19, sum#26] +Aggregate Attributes [1]: [sum#26] +Results [3]: [ca_zip#20, ca_city#19, sum#27] (44) Exchange -Input [3]: [ca_zip#20, ca_city#19, sum#26] -Arguments: hashpartitioning(ca_zip#20, ca_city#19, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ca_zip#20, ca_city#19, sum#27] +Arguments: hashpartitioning(ca_zip#20, ca_city#19, 5), ENSURE_REQUIREMENTS, [id=#28] (45) HashAggregate [codegen id : 13] -Input [3]: [ca_zip#20, ca_city#19, sum#26] +Input [3]: [ca_zip#20, ca_city#19, sum#27] Keys [2]: [ca_zip#20, ca_city#19] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#28] -Results [3]: [ca_zip#20, ca_city#19, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#28,17,2) AS sum(ws_sales_price)#29] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#29] +Results [3]: [ca_zip#20, ca_city#19, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#29,17,2) AS sum(ws_sales_price)#30] (46) TakeOrderedAndProject -Input [3]: [ca_zip#20, ca_city#19, sum(ws_sales_price)#29] -Arguments: 100, [ca_zip#20 ASC NULLS FIRST, ca_city#19 ASC NULLS FIRST], [ca_zip#20, ca_city#19, sum(ws_sales_price)#29] +Input [3]: [ca_zip#20, ca_city#19, sum(ws_sales_price)#30] +Arguments: 100, [ca_zip#20 ASC NULLS FIRST, ca_city#19 ASC NULLS FIRST], [ca_zip#20, ca_city#19, sum(ws_sales_price)#30] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt index b81498ddcea34..f3a37f9e8767e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt @@ -169,30 +169,30 @@ Output [4]: [ws_sales_price#4, ca_city#11, ca_zip#12, i_item_id#19] Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#11, ca_zip#12, i_item_sk#18, i_item_id#19] (29) Scan parquet default.item -Output [2]: [i_item_sk#18, i_item_id#19] +Output [2]: [i_item_sk#21, i_item_id#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_item_sk, [2,3,5,7,11,13,17,19,23,29])] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] -Input [2]: [i_item_sk#18, i_item_id#19] +Input [2]: [i_item_sk#21, i_item_id#22] (31) Filter [codegen id : 5] -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : i_item_sk#18 IN (2,3,5,7,11,13,17,19,23,29) +Input [2]: [i_item_sk#21, i_item_id#22] +Condition : i_item_sk#21 IN (2,3,5,7,11,13,17,19,23,29) (32) Project [codegen id : 5] -Output [1]: [i_item_id#19 AS i_item_id#19#21] -Input [2]: [i_item_sk#18, i_item_id#19] +Output [1]: [i_item_id#22] +Input [2]: [i_item_sk#21, i_item_id#22] (33) BroadcastExchange -Input [1]: [i_item_id#19#21] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#22] +Input [1]: [i_item_id#22] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#23] (34) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_id#19] -Right keys [1]: [i_item_id#19#21] +Right keys [1]: [i_item_id#22] Join condition: None (35) Filter [codegen id : 6] @@ -207,23 +207,23 @@ Input [5]: [ws_sales_price#4, ca_city#11, ca_zip#12, i_item_id#19, exists#1] Input [3]: [ws_sales_price#4, ca_city#11, ca_zip#12] Keys [2]: [ca_zip#12, ca_city#11] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#23] -Results [3]: [ca_zip#12, ca_city#11, sum#24] +Aggregate Attributes [1]: [sum#24] +Results [3]: [ca_zip#12, ca_city#11, sum#25] (38) Exchange -Input [3]: [ca_zip#12, ca_city#11, sum#24] -Arguments: hashpartitioning(ca_zip#12, ca_city#11, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [ca_zip#12, ca_city#11, sum#25] +Arguments: hashpartitioning(ca_zip#12, ca_city#11, 5), ENSURE_REQUIREMENTS, [id=#26] (39) HashAggregate [codegen id : 7] -Input [3]: [ca_zip#12, ca_city#11, sum#24] +Input [3]: [ca_zip#12, ca_city#11, sum#25] Keys [2]: [ca_zip#12, ca_city#11] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#26] -Results [3]: [ca_zip#12, ca_city#11, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#26,17,2) AS sum(ws_sales_price)#27] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#27] +Results [3]: [ca_zip#12, ca_city#11, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#27,17,2) AS sum(ws_sales_price)#28] (40) TakeOrderedAndProject -Input [3]: [ca_zip#12, ca_city#11, sum(ws_sales_price)#27] -Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#11 ASC NULLS FIRST], [ca_zip#12, ca_city#11, sum(ws_sales_price)#27] +Input [3]: [ca_zip#12, ca_city#11, sum(ws_sales_price)#28] +Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#11 ASC NULLS FIRST], [ca_zip#12, ca_city#11, sum(ws_sales_price)#28] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt index 78ade994364a7..ac35640c7b004 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt @@ -230,55 +230,55 @@ Input [5]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt Arguments: [ss_addr_sk#12 ASC NULLS FIRST], false, 0 (41) ReusedExchange [Reuses operator id: 9] -Output [2]: [ca_address_sk#6, ca_city#7] +Output [2]: [ca_address_sk#31, ca_city#32] (42) Sort [codegen id : 13] -Input [2]: [ca_address_sk#6, ca_city#7] -Arguments: [ca_address_sk#6 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#31, ca_city#32] +Arguments: [ca_address_sk#31 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 14] Left keys [1]: [ss_addr_sk#12] -Right keys [1]: [ca_address_sk#6] +Right keys [1]: [ca_address_sk#31] Join condition: None (44) Project [codegen id : 14] -Output [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#7] -Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_address_sk#6, ca_city#7] +Output [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#32] +Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_address_sk#31, ca_city#32] (45) HashAggregate [codegen id : 14] -Input [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#7] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] +Input [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#32] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32] Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#15)), partial_sum(UnscaledValue(ss_net_profit#16))] -Aggregate Attributes [2]: [sum#31, sum#32] -Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#33, sum#34] +Aggregate Attributes [2]: [sum#33, sum#34] +Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32, sum#35, sum#36] (46) HashAggregate [codegen id : 14] -Input [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#33, sum#34] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] +Input [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32, sum#35, sum#36] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#15)), sum(UnscaledValue(ss_net_profit#16))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#15))#35, sum(UnscaledValue(ss_net_profit#16))#36] -Results [5]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#7 AS bought_city#37, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#15))#35,17,2) AS amt#38, MakeDecimal(sum(UnscaledValue(ss_net_profit#16))#36,17,2) AS profit#39] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#15))#37, sum(UnscaledValue(ss_net_profit#16))#38] +Results [5]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#32 AS bought_city#39, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#15))#37,17,2) AS amt#40, MakeDecimal(sum(UnscaledValue(ss_net_profit#16))#38,17,2) AS profit#41] (47) Exchange -Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#37, amt#38, profit#39] -Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#39, amt#40, profit#41] +Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#42] (48) Sort [codegen id : 15] -Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#37, amt#38, profit#39] +Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#39, amt#40, profit#41] Arguments: [ss_customer_sk#10 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 16] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#10] -Join condition: NOT (ca_city#7 = bought_city#37) +Join condition: NOT (ca_city#7 = bought_city#39) (50) Project [codegen id : 16] -Output [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#37, ss_ticket_number#14, amt#38, profit#39] -Input [9]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#37, amt#38, profit#39] +Output [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#39, ss_ticket_number#14, amt#40, profit#41] +Input [9]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#39, amt#40, profit#41] (51) TakeOrderedAndProject -Input [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#37, ss_ticket_number#14, amt#38, profit#39] -Arguments: 100, [c_last_name#4 ASC NULLS FIRST, c_first_name#3 ASC NULLS FIRST, ca_city#7 ASC NULLS FIRST, bought_city#37 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#37, ss_ticket_number#14, amt#38, profit#39] +Input [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#39, ss_ticket_number#14, amt#40, profit#41] +Arguments: 100, [c_last_name#4 ASC NULLS FIRST, c_first_name#3 ASC NULLS FIRST, ca_city#7 ASC NULLS FIRST, bought_city#39 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#39, ss_ticket_number#14, amt#40, profit#41] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt index e9bc22389d8a6..2532a14d7907f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt @@ -225,20 +225,20 @@ Output [7]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_ad Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#31, amt#32, profit#33, c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#21, ca_city#22] +Output [2]: [ca_address_sk#39, ca_city#40] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#21] -Join condition: NOT (ca_city#22 = bought_city#31) +Right keys [1]: [ca_address_sk#39] +Join condition: NOT (ca_city#40 = bought_city#31) (42) Project [codegen id : 8] -Output [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#21, ca_city#22] +Output [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#39, ca_city#40] (43) TakeOrderedAndProject -Input [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#22 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#40 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] ===== Subqueries ===== 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 dc19996f9957b..b7ffba01f92c6 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 @@ -229,78 +229,78 @@ Output [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_ye (39) HashAggregate [codegen id : 21] Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum#36] Keys [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35] -Functions [1]: [sum(UnscaledValue(ss_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#3))#37] -Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#3))#37,17,2) AS sum_sales#38] +Functions [1]: [sum(UnscaledValue(ss_sales_price#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#37))#38] +Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#37))#38,17,2) AS sum_sales#39] (40) Exchange -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#40] (41) Sort [codegen id : 22] -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST], false, 0 (42) Window -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] (43) Project [codegen id : 23] -Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38, rn#40] +Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] (44) Exchange -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1), 5), ENSURE_REQUIREMENTS, [id=#41] +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1), 5), ENSURE_REQUIREMENTS, [id=#42] (45) Sort [codegen id : 24] -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#40 + 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#41 + 1) ASC NULLS FIRST], false, 0 (46) SortMergeJoin [codegen id : 25] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1)] +Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1)] Join condition: None (47) Project [codegen id : 25] -Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38] -Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] +Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39] +Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] (48) ReusedExchange [Reuses operator id: 40] -Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Output [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] (49) Sort [codegen id : 34] -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] +Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST], false, 0 (50) Window -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] +Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] +Arguments: [rank(d_year#47, d_moy#48) windowspecdefinition(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#50], [i_category#43, i_brand#44, s_store_name#45, s_company_name#46], [d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST] (51) Project [codegen id : 35] -Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] +Output [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] (52) Exchange -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1), 5), ENSURE_REQUIREMENTS, [id=#50] +Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Arguments: hashpartitioning(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1), 5), ENSURE_REQUIREMENTS, [id=#51] (53) Sort [codegen id : 36] -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (rn#49 - 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, (rn#50 - 1) ASC NULLS FIRST], false, 0 (54) SortMergeJoin [codegen id : 37] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1)] +Right keys [5]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1)] Join condition: None (55) Project [codegen id : 37] -Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#38 AS psum#51, sum_sales#48 AS nsum#52] -Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] +Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#39 AS psum#52, sum_sales#49 AS nsum#53] +Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39, i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] (56) TakeOrderedAndProject -Input [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#11 ASC NULLS FIRST], [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] +Input [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#11 ASC NULLS FIRST], [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] ===== Subqueries ===== 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 3d58d26e6a3a4..917b21c247449 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 @@ -202,70 +202,70 @@ Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_ye (34) HashAggregate [codegen id : 13] 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#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#35] -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#6))#35,17,2) AS sum_sales#36] +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] (35) 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#36] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#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#37] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] (36) Sort [codegen id : 14] -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36] +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: [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 (37) 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#36] -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#38], [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#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] (38) Project [codegen id : 15] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] -Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] +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] (39) BroadcastExchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] -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=#39] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +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=#40] (40) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#38 + 1)] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] Join condition: None (41) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] -Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] +Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] +Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] (42) ReusedExchange [Reuses operator id: 35] -Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] +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] (43) Sort [codegen id : 21] -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#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, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 +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 (44) Window -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] -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#47], [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] +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] (45) Project [codegen id : 22] -Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] -Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] +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] (46) BroadcastExchange -Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] -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=#48] +Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +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=#49] (47) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -Right keys [5]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#47 - 1)] +Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] Join condition: None (48) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#49, sum_sales#46 AS nsum#50] -Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] +Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] +Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] (49) TakeOrderedAndProject -Input [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] +Input [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, psum#50, nsum#51] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt index e93d0e9503493..3f30183af3a94 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt @@ -250,225 +250,225 @@ Output [5]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_net_profit#47, cs_sold_date_sk#48] (36) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#49] (37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_sold_date_sk#48] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#49] Join condition: None (38) Project [codegen id : 12] Output [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#8] +Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#49] (39) Exchange Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#49] +Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#50] (40) Sort [codegen id : 13] Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] Arguments: [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST], false, 0 (41) Scan parquet default.catalog_returns -Output [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Output [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (42) ColumnarToRow [codegen id : 14] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] (43) Filter [codegen id : 14] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] -Condition : (((isnotnull(cr_return_amount#53) AND (cr_return_amount#53 > 10000.00)) AND isnotnull(cr_order_number#51)) AND isnotnull(cr_item_sk#50)) +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Condition : (((isnotnull(cr_return_amount#54) AND (cr_return_amount#54 > 10000.00)) AND isnotnull(cr_order_number#52)) AND isnotnull(cr_item_sk#51)) (44) Project [codegen id : 14] -Output [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Output [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] (45) Exchange -Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Arguments: hashpartitioning(cr_order_number#52, cr_item_sk#51, 5), ENSURE_REQUIREMENTS, [id=#56] (46) Sort [codegen id : 15] -Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Arguments: [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Arguments: [cr_order_number#52 ASC NULLS FIRST, cr_item_sk#51 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 16] Left keys [2]: [cs_order_number#44, cs_item_sk#43] -Right keys [2]: [cr_order_number#51, cr_item_sk#50] +Right keys [2]: [cr_order_number#52, cr_item_sk#51] Join condition: None (48) Project [codegen id : 16] -Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] -Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] +Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] (49) HashAggregate [codegen id : 16] -Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] +Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] Keys [1]: [cs_item_sk#43] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#52, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#56, sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] -Results [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#53, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] +Results [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] (50) Exchange -Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] -Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#69] (51) HashAggregate [codegen id : 17] -Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] Keys [1]: [cs_item_sk#43] -Functions [4]: [sum(coalesce(cr_return_quantity#52, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#52, 0))#69, sum(coalesce(cs_quantity#45, 0))#70, sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72] -Results [3]: [cs_item_sk#43 AS item#73, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#52, 0))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#70 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#74, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#75] +Functions [4]: [sum(coalesce(cr_return_quantity#53, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#53, 0))#70, sum(coalesce(cs_quantity#45, 0))#71, sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73] +Results [3]: [cs_item_sk#43 AS item#74, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#53, 0))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#71 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#75, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#76] (52) Exchange -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#77] (53) Sort [codegen id : 18] -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: [return_ratio#74 ASC NULLS FIRST], false, 0 +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: [return_ratio#75 ASC NULLS FIRST], false, 0 (54) Window -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: [rank(return_ratio#74) windowspecdefinition(return_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#77], [return_ratio#74 ASC NULLS FIRST] +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: [rank(return_ratio#75) windowspecdefinition(return_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#78], [return_ratio#75 ASC NULLS FIRST] (55) Sort [codegen id : 19] -Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] -Arguments: [currency_ratio#75 ASC NULLS FIRST], false, 0 +Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] +Arguments: [currency_ratio#76 ASC NULLS FIRST], false, 0 (56) Window -Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] -Arguments: [rank(currency_ratio#75) windowspecdefinition(currency_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#78], [currency_ratio#75 ASC NULLS FIRST] +Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] +Arguments: [rank(currency_ratio#76) windowspecdefinition(currency_ratio#76 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#79], [currency_ratio#76 ASC NULLS FIRST] (57) Filter [codegen id : 20] -Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] -Condition : ((return_rank#77 <= 10) OR (currency_rank#78 <= 10)) +Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] +Condition : ((return_rank#78 <= 10) OR (currency_rank#79 <= 10)) (58) Project [codegen id : 20] -Output [5]: [catalog AS channel#79, item#73, return_ratio#74, return_rank#77, currency_rank#78] -Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] +Output [5]: [catalog AS channel#80, item#74, return_ratio#75, return_rank#78, currency_rank#79] +Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] (59) Scan parquet default.store_sales -Output [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Output [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_sold_date_sk#85 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 22] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] (61) Filter [codegen id : 22] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] -Condition : (((((((isnotnull(ss_net_profit#84) AND isnotnull(ss_net_paid#83)) AND isnotnull(ss_quantity#82)) AND (ss_net_profit#84 > 1.00)) AND (ss_net_paid#83 > 0.00)) AND (ss_quantity#82 > 0)) AND isnotnull(ss_ticket_number#81)) AND isnotnull(ss_item_sk#80)) +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Condition : (((((((isnotnull(ss_net_profit#85) AND isnotnull(ss_net_paid#84)) AND isnotnull(ss_quantity#83)) AND (ss_net_profit#85 > 1.00)) AND (ss_net_paid#84 > 0.00)) AND (ss_quantity#83 > 0)) AND isnotnull(ss_ticket_number#82)) AND isnotnull(ss_item_sk#81)) (62) Project [codegen id : 22] -Output [5]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Output [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] (63) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#87] (64) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ss_sold_date_sk#85] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ss_sold_date_sk#86] +Right keys [1]: [d_date_sk#87] Join condition: None (65) Project [codegen id : 22] -Output [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85, d_date_sk#8] +Output [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86, d_date_sk#87] (66) Exchange -Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Arguments: hashpartitioning(cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint), 5), ENSURE_REQUIREMENTS, [id=#86] +Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Arguments: hashpartitioning(cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint), 5), ENSURE_REQUIREMENTS, [id=#88] (67) Sort [codegen id : 23] -Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Arguments: [cast(ss_ticket_number#81 as bigint) ASC NULLS FIRST, cast(ss_item_sk#80 as bigint) ASC NULLS FIRST], false, 0 +Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Arguments: [cast(ss_ticket_number#82 as bigint) ASC NULLS FIRST, cast(ss_item_sk#81 as bigint) ASC NULLS FIRST], false, 0 (68) Scan parquet default.store_returns -Output [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Output [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 24] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] (70) Filter [codegen id : 24] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] -Condition : (((isnotnull(sr_return_amt#90) AND (sr_return_amt#90 > 10000.00)) AND isnotnull(sr_ticket_number#88)) AND isnotnull(sr_item_sk#87)) +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Condition : (((isnotnull(sr_return_amt#92) AND (sr_return_amt#92 > 10000.00)) AND isnotnull(sr_ticket_number#90)) AND isnotnull(sr_item_sk#89)) (71) Project [codegen id : 24] -Output [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Output [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] (72) Exchange -Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Arguments: hashpartitioning(sr_ticket_number#88, sr_item_sk#87, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Arguments: hashpartitioning(sr_ticket_number#90, sr_item_sk#89, 5), ENSURE_REQUIREMENTS, [id=#94] (73) Sort [codegen id : 25] -Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Arguments: [sr_ticket_number#88 ASC NULLS FIRST, sr_item_sk#87 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Arguments: [sr_ticket_number#90 ASC NULLS FIRST, sr_item_sk#89 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 26] -Left keys [2]: [cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint)] -Right keys [2]: [sr_ticket_number#88, sr_item_sk#87] +Left keys [2]: [cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint)] +Right keys [2]: [sr_ticket_number#90, sr_item_sk#89] Join condition: None (75) Project [codegen id : 26] -Output [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] -Input [8]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Output [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] +Input [8]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] (76) HashAggregate [codegen id : 26] -Input [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] -Keys [1]: [ss_item_sk#80] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#89, 0)), partial_sum(coalesce(ss_quantity#82, 0)), partial_sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#93, sum#94, sum#95, isEmpty#96, sum#97, isEmpty#98] -Results [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Input [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] +Keys [1]: [ss_item_sk#81] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#91, 0)), partial_sum(coalesce(ss_quantity#83, 0)), partial_sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#95, sum#96, sum#97, isEmpty#98, sum#99, isEmpty#100] +Results [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] (77) Exchange -Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Arguments: hashpartitioning(ss_item_sk#80, 5), ENSURE_REQUIREMENTS, [id=#105] +Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] +Arguments: hashpartitioning(ss_item_sk#81, 5), ENSURE_REQUIREMENTS, [id=#107] (78) HashAggregate [codegen id : 27] -Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Keys [1]: [ss_item_sk#80] -Functions [4]: [sum(coalesce(sr_return_quantity#89, 0)), sum(coalesce(ss_quantity#82, 0)), sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#89, 0))#106, sum(coalesce(ss_quantity#82, 0))#107, sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108, sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109] -Results [3]: [ss_item_sk#80 AS item#110, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#89, 0))#106 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#82, 0))#107 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#111, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#112] +Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] +Keys [1]: [ss_item_sk#81] +Functions [4]: [sum(coalesce(sr_return_quantity#91, 0)), sum(coalesce(ss_quantity#83, 0)), sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#91, 0))#108, sum(coalesce(ss_quantity#83, 0))#109, sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110, sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111] +Results [3]: [ss_item_sk#81 AS item#112, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#91, 0))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#83, 0))#109 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#113, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#114] (79) Exchange -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] (80) Sort [codegen id : 28] -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: [return_ratio#111 ASC NULLS FIRST], false, 0 +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: [return_ratio#113 ASC NULLS FIRST], false, 0 (81) Window -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: [rank(return_ratio#111) windowspecdefinition(return_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#114], [return_ratio#111 ASC NULLS FIRST] +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: [rank(return_ratio#113) windowspecdefinition(return_ratio#113 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#116], [return_ratio#113 ASC NULLS FIRST] (82) Sort [codegen id : 29] -Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] -Arguments: [currency_ratio#112 ASC NULLS FIRST], false, 0 +Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] +Arguments: [currency_ratio#114 ASC NULLS FIRST], false, 0 (83) Window -Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] -Arguments: [rank(currency_ratio#112) windowspecdefinition(currency_ratio#112 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#115], [currency_ratio#112 ASC NULLS FIRST] +Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] +Arguments: [rank(currency_ratio#114) windowspecdefinition(currency_ratio#114 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#117], [currency_ratio#114 ASC NULLS FIRST] (84) Filter [codegen id : 30] -Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] -Condition : ((return_rank#114 <= 10) OR (currency_rank#115 <= 10)) +Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] +Condition : ((return_rank#116 <= 10) OR (currency_rank#117 <= 10)) (85) Project [codegen id : 30] -Output [5]: [store AS channel#116, item#110, return_ratio#111, return_rank#114, currency_rank#115] -Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] +Output [5]: [store AS channel#118, item#112, return_ratio#113, return_rank#116, currency_rank#117] +Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] (86) Union @@ -481,7 +481,7 @@ Results [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_ran (88) Exchange Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] -Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#117] +Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#119] (89) HashAggregate [codegen id : 32] Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] @@ -505,6 +505,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#85 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index aeaca4deaae0c..a98f59f84be7d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -260,170 +260,170 @@ Output [6]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, Input [9]: [cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_item_sk#49, cr_order_number#50, cr_return_quantity#51, cr_return_amount#52] (40) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#54] (41) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#47] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#54] Join condition: None (42) Project [codegen id : 10] Output [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] -Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#14] +Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#54] (43) HashAggregate [codegen id : 10] Input [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] Keys [1]: [cs_item_sk#42] Functions [4]: [partial_sum(coalesce(cr_return_quantity#51, 0)), partial_sum(coalesce(cs_quantity#44, 0)), partial_sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Results [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] +Aggregate Attributes [6]: [sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Results [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] (44) Exchange -Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] -Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#67] (45) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] +Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] Keys [1]: [cs_item_sk#42] Functions [4]: [sum(coalesce(cr_return_quantity#51, 0)), sum(coalesce(cs_quantity#44, 0)), sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#67, sum(coalesce(cs_quantity#44, 0))#68, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70] -Results [3]: [cs_item_sk#42 AS item#71, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#67 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#68 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#72, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#73] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#68, sum(coalesce(cs_quantity#44, 0))#69, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71] +Results [3]: [cs_item_sk#42 AS item#72, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#68 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#69 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#73, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#74] (46) Exchange -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#74] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#75] (47) Sort [codegen id : 12] -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: [return_ratio#72 ASC NULLS FIRST], false, 0 +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [return_ratio#73 ASC NULLS FIRST], false, 0 (48) Window -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: [rank(return_ratio#72) windowspecdefinition(return_ratio#72 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#72 ASC NULLS FIRST] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#76], [return_ratio#73 ASC NULLS FIRST] (49) Sort [codegen id : 13] -Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] -Arguments: [currency_ratio#73 ASC NULLS FIRST], false, 0 +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 (50) Window -Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] -Arguments: [rank(currency_ratio#73) windowspecdefinition(currency_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#73 ASC NULLS FIRST] +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#77], [currency_ratio#74 ASC NULLS FIRST] (51) Filter [codegen id : 14] -Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] +Condition : ((return_rank#76 <= 10) OR (currency_rank#77 <= 10)) (52) Project [codegen id : 14] -Output [5]: [catalog AS channel#77, item#71, return_ratio#72, return_rank#75, currency_rank#76] -Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] +Output [5]: [catalog AS channel#78, item#72, return_ratio#73, return_rank#76, currency_rank#77] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] (53) Scan parquet default.store_sales -Output [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Output [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#83), dynamicpruningexpression(ss_sold_date_sk#83 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#84), dynamicpruningexpression(ss_sold_date_sk#84 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 15] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] (55) Filter [codegen id : 15] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] -Condition : (((((((isnotnull(ss_net_profit#82) AND isnotnull(ss_net_paid#81)) AND isnotnull(ss_quantity#80)) AND (ss_net_profit#82 > 1.00)) AND (ss_net_paid#81 > 0.00)) AND (ss_quantity#80 > 0)) AND isnotnull(ss_ticket_number#79)) AND isnotnull(ss_item_sk#78)) +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Condition : (((((((isnotnull(ss_net_profit#83) AND isnotnull(ss_net_paid#82)) AND isnotnull(ss_quantity#81)) AND (ss_net_profit#83 > 1.00)) AND (ss_net_paid#82 > 0.00)) AND (ss_quantity#81 > 0)) AND isnotnull(ss_ticket_number#80)) AND isnotnull(ss_item_sk#79)) (56) Project [codegen id : 15] -Output [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Output [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] (57) BroadcastExchange -Input [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#84] +Input [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#85] (58) Scan parquet default.store_returns -Output [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Output [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (59) ColumnarToRow -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] (60) Filter -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] -Condition : (((isnotnull(sr_return_amt#88) AND (sr_return_amt#88 > 10000.00)) AND isnotnull(sr_ticket_number#86)) AND isnotnull(sr_item_sk#85)) +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Condition : (((isnotnull(sr_return_amt#89) AND (sr_return_amt#89 > 10000.00)) AND isnotnull(sr_ticket_number#87)) AND isnotnull(sr_item_sk#86)) (61) Project -Output [4]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Output [4]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] (62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [cast(ss_ticket_number#79 as bigint), cast(ss_item_sk#78 as bigint)] -Right keys [2]: [sr_ticket_number#86, sr_item_sk#85] +Left keys [2]: [cast(ss_ticket_number#80 as bigint), cast(ss_item_sk#79 as bigint)] +Right keys [2]: [sr_ticket_number#87, sr_item_sk#86] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88] -Input [9]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] +Output [6]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89] +Input [9]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] (64) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#91] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#83] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ss_sold_date_sk#84] +Right keys [1]: [d_date_sk#91] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] -Input [7]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88, d_date_sk#14] +Output [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] +Input [7]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89, d_date_sk#91] (67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] -Keys [1]: [ss_item_sk#78] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#87, 0)), partial_sum(coalesce(ss_quantity#80, 0)), partial_sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Results [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Input [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] +Keys [1]: [ss_item_sk#79] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#88, 0)), partial_sum(coalesce(ss_quantity#81, 0)), partial_sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] +Results [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] (68) Exchange -Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Arguments: hashpartitioning(ss_item_sk#78, 5), ENSURE_REQUIREMENTS, [id=#102] +Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Arguments: hashpartitioning(ss_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#104] (69) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Keys [1]: [ss_item_sk#78] -Functions [4]: [sum(coalesce(sr_return_quantity#87, 0)), sum(coalesce(ss_quantity#80, 0)), sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#87, 0))#103, sum(coalesce(ss_quantity#80, 0))#104, sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105, sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106] -Results [3]: [ss_item_sk#78 AS item#107, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#87, 0))#103 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#80, 0))#104 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#108, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#109] +Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Keys [1]: [ss_item_sk#79] +Functions [4]: [sum(coalesce(sr_return_quantity#88, 0)), sum(coalesce(ss_quantity#81, 0)), sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#88, 0))#105, sum(coalesce(ss_quantity#81, 0))#106, sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107, sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108] +Results [3]: [ss_item_sk#79 AS item#109, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#88, 0))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#81, 0))#106 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#110, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#111] (70) Exchange -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#112] (71) Sort [codegen id : 19] -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: [return_ratio#108 ASC NULLS FIRST], false, 0 +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: [return_ratio#110 ASC NULLS FIRST], false, 0 (72) Window -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: [rank(return_ratio#108) windowspecdefinition(return_ratio#108 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#111], [return_ratio#108 ASC NULLS FIRST] +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: [rank(return_ratio#110) windowspecdefinition(return_ratio#110 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#113], [return_ratio#110 ASC NULLS FIRST] (73) Sort [codegen id : 20] -Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] -Arguments: [currency_ratio#109 ASC NULLS FIRST], false, 0 +Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] +Arguments: [currency_ratio#111 ASC NULLS FIRST], false, 0 (74) Window -Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] -Arguments: [rank(currency_ratio#109) windowspecdefinition(currency_ratio#109 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#112], [currency_ratio#109 ASC NULLS FIRST] +Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] +Arguments: [rank(currency_ratio#111) windowspecdefinition(currency_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#114], [currency_ratio#111 ASC NULLS FIRST] (75) Filter [codegen id : 21] -Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] -Condition : ((return_rank#111 <= 10) OR (currency_rank#112 <= 10)) +Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] +Condition : ((return_rank#113 <= 10) OR (currency_rank#114 <= 10)) (76) Project [codegen id : 21] -Output [5]: [store AS channel#113, item#107, return_ratio#108, return_rank#111, currency_rank#112] -Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] +Output [5]: [store AS channel#115, item#109, return_ratio#110, return_rank#113, currency_rank#114] +Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] (77) Union @@ -436,7 +436,7 @@ Results [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_ran (79) Exchange Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] -Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#114] +Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#116] (80) HashAggregate [codegen id : 23] Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] @@ -460,6 +460,6 @@ Output [1]: [d_date_sk#14] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#83 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#84 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index a754cef8f2962..28a457258eff7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -265,193 +265,193 @@ Output [6]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, Input [8]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] (41) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#70] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#25] +Right keys [1]: [d_date_sk#70] Join condition: None (43) Project [codegen id : 11] Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#25] +Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#70] (44) HashAggregate [codegen id : 11] Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] -Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] +Results [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] (45) Exchange -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] +Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#79] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#83, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS RETURNS#84, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#85, catalog channel AS channel#86, concat(catalog_page, cp_catalog_page_id#68) AS id#87] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#84, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS RETURNS#85, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#86, catalog channel AS channel#87, concat(catalog_page, cp_catalog_page_id#68) AS id#88] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] -Condition : isnotnull(ws_web_site_sk#88) +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : isnotnull(ws_web_site_sk#89) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] (53) Exchange -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: hashpartitioning(wr_item_sk#98, wr_order_number#99, 5), ENSURE_REQUIREMENTS, [id=#103] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: hashpartitioning(wr_item_sk#99, wr_order_number#100, 5), ENSURE_REQUIREMENTS, [id=#104] (54) Sort [codegen id : 15] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: [wr_item_sk#98 ASC NULLS FIRST, wr_order_number#99 ASC NULLS FIRST], false, 0 +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: [wr_item_sk#99 ASC NULLS FIRST, wr_order_number#100 ASC NULLS FIRST], false, 0 (55) Scan parquet default.web_sales -Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 16] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (57) Filter [codegen id : 16] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) (58) Project [codegen id : 16] -Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (59) Exchange -Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Arguments: hashpartitioning(cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint), 5), ENSURE_REQUIREMENTS, [id=#106] +Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Arguments: hashpartitioning(cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint), 5), ENSURE_REQUIREMENTS, [id=#109] (60) Sort [codegen id : 17] -Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Arguments: [cast(ws_item_sk#104 as bigint) ASC NULLS FIRST, cast(ws_order_number#105 as bigint) ASC NULLS FIRST], false, 0 +Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Arguments: [cast(ws_item_sk#105 as bigint) ASC NULLS FIRST, cast(ws_order_number#107 as bigint) ASC NULLS FIRST], false, 0 (61) SortMergeJoin [codegen id : 18] -Left keys [2]: [wr_item_sk#98, wr_order_number#99] -Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] +Left keys [2]: [wr_item_sk#99, wr_order_number#100] +Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] Join condition: None (62) Project [codegen id : 18] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#107, wr_returned_date_sk#102 AS date_sk#108, 0.00 AS sales_price#109, 0.00 AS profit#110, wr_return_amt#100 AS return_amt#111, wr_net_loss#101 AS net_loss#112] -Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#110, wr_returned_date_sk#103 AS date_sk#111, 0.00 AS sales_price#112, 0.00 AS profit#113, wr_return_amt#101 AS return_amt#114, wr_net_loss#102 AS net_loss#115] +Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] (63) Union (64) Scan parquet default.web_site -Output [2]: [web_site_sk#113, web_site_id#114] +Output [2]: [web_site_sk#116, web_site_id#117] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 19] -Input [2]: [web_site_sk#113, web_site_id#114] +Input [2]: [web_site_sk#116, web_site_id#117] (66) Filter [codegen id : 19] -Input [2]: [web_site_sk#113, web_site_id#114] -Condition : isnotnull(web_site_sk#113) +Input [2]: [web_site_sk#116, web_site_id#117] +Condition : isnotnull(web_site_sk#116) (67) BroadcastExchange -Input [2]: [web_site_sk#113, web_site_id#114] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] +Input [2]: [web_site_sk#116, web_site_id#117] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] (68) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [wsr_web_site_sk#92] -Right keys [1]: [web_site_sk#113] +Left keys [1]: [wsr_web_site_sk#93] +Right keys [1]: [web_site_sk#116] Join condition: None (69) Project [codegen id : 21] -Output [6]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Input [8]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#113, web_site_id#114] +Output [6]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [8]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] (70) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#119] (71) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [date_sk#93] -Right keys [1]: [cast(d_date_sk#25 as bigint)] +Left keys [1]: [date_sk#94] +Right keys [1]: [cast(d_date_sk#119 as bigint)] Join condition: None (72) Project [codegen id : 21] -Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Input [7]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114, d_date_sk#25] +Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [7]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117, d_date_sk#119] (73) HashAggregate [codegen id : 21] -Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Keys [1]: [web_site_id#114] -Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] -Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] +Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Keys [1]: [web_site_id#117] +Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum#120, sum#121, sum#122, sum#123] +Results [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] (74) Exchange -Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] +Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#128] (75) HashAggregate [codegen id : 22] -Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -Keys [1]: [web_site_id#114] -Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#125, sum(UnscaledValue(return_amt#96))#126, sum(UnscaledValue(profit#95))#127, sum(UnscaledValue(net_loss#97))#128] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#94))#125,17,2) AS sales#129, MakeDecimal(sum(UnscaledValue(return_amt#96))#126,17,2) AS RETURNS#130, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#131, web channel AS channel#132, concat(web_site, web_site_id#114) AS id#133] +Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] +Keys [1]: [web_site_id#117] +Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#129, sum(UnscaledValue(return_amt#97))#130, sum(UnscaledValue(profit#96))#131, sum(UnscaledValue(net_loss#98))#132] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#95))#129,17,2) AS sales#133, MakeDecimal(sum(UnscaledValue(return_amt#97))#130,17,2) AS RETURNS#134, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#131,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#132,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#135, web channel AS channel#136, concat(web_site, web_site_id#117) AS id#137] (76) Union (77) Expand [codegen id : 23] Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45] -Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#134, id#135, spark_grouping_id#136] +Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#138, id#139, spark_grouping_id#140] (78) HashAggregate [codegen id : 23] -Input [6]: [sales#41, returns#42, profit#43, channel#134, id#135, spark_grouping_id#136] -Keys [3]: [channel#134, id#135, spark_grouping_id#136] +Input [6]: [sales#41, returns#42, profit#43, channel#138, id#139, spark_grouping_id#140] +Keys [3]: [channel#138, id#139, spark_grouping_id#140] Functions [3]: [partial_sum(sales#41), partial_sum(returns#42), partial_sum(profit#43)] -Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] -Results [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Aggregate Attributes [6]: [sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Results [9]: [channel#138, id#139, spark_grouping_id#140, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] (79) Exchange -Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -Arguments: hashpartitioning(channel#134, id#135, spark_grouping_id#136, 5), ENSURE_REQUIREMENTS, [id=#149] +Input [9]: [channel#138, id#139, spark_grouping_id#140, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] +Arguments: hashpartitioning(channel#138, id#139, spark_grouping_id#140, 5), ENSURE_REQUIREMENTS, [id=#153] (80) HashAggregate [codegen id : 24] -Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -Keys [3]: [channel#134, id#135, spark_grouping_id#136] +Input [9]: [channel#138, id#139, spark_grouping_id#140, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] +Keys [3]: [channel#138, id#139, spark_grouping_id#140] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] -Aggregate Attributes [3]: [sum(sales#41)#150, sum(returns#42)#151, sum(profit#43)#152] -Results [5]: [channel#134, id#135, sum(sales#41)#150 AS sales#153, sum(returns#42)#151 AS returns#154, sum(profit#43)#152 AS profit#155] +Aggregate Attributes [3]: [sum(sales#41)#154, sum(returns#42)#155, sum(profit#43)#156] +Results [5]: [channel#138, id#139, sum(sales#41)#154 AS sales#157, sum(returns#42)#155 AS returns#158, sum(profit#43)#156 AS profit#159] (81) TakeOrderedAndProject -Input [5]: [channel#134, id#135, sales#153, returns#154, profit#155] -Arguments: 100, [channel#134 ASC NULLS FIRST, id#135 ASC NULLS FIRST], [channel#134, id#135, sales#153, returns#154, profit#155] +Input [5]: [channel#138, id#139, sales#157, returns#158, profit#159] +Arguments: 100, [channel#138 ASC NULLS FIRST, id#139 ASC NULLS FIRST], [channel#138, id#139, sales#157, returns#158, profit#159] ===== Subqueries ===== @@ -469,12 +469,12 @@ ReusedExchange (83) (83) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#70] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index b3eb1f9592ff3..cb130ce17795a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -235,208 +235,208 @@ Input [4]: [cr_catalog_page_sk#57, cr_return_amount#58, cr_net_loss#59, cr_retur (34) Union (35) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#67] (36) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#67] Join condition: None (37) Project [codegen id : 11] Output [5]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56] -Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#22] +Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#67] (38) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Output [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] (40) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] -Condition : isnotnull(cp_catalog_page_sk#67) +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Condition : isnotnull(cp_catalog_page_sk#68) (41) BroadcastExchange -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#69] +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#70] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [page_sk#51] -Right keys [1]: [cp_catalog_page_sk#67] +Right keys [1]: [cp_catalog_page_sk#68] Join condition: None (43) Project [codegen id : 11] -Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] +Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] +Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#68, cp_catalog_page_id#69] (44) HashAggregate [codegen id : 11] -Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Keys [1]: [cp_catalog_page_id#68] +Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] +Keys [1]: [cp_catalog_page_id#69] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] -Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] +Results [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] (45) Exchange -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] +Arguments: hashpartitioning(cp_catalog_page_id#69, 5), ENSURE_REQUIREMENTS, [id=#79] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Keys [1]: [cp_catalog_page_id#68] +Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] +Keys [1]: [cp_catalog_page_id#69] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#83, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS RETURNS#84, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#85, catalog channel AS channel#86, concat(catalog_page, cp_catalog_page_id#68) AS id#87] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#84, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS RETURNS#85, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#86, catalog channel AS channel#87, concat(catalog_page, cp_catalog_page_id#69) AS id#88] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] -Condition : isnotnull(ws_web_site_sk#88) +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : isnotnull(ws_web_site_sk#89) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] (53) BroadcastExchange -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#103] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#104] (54) Scan parquet default.web_sales -Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (55) ColumnarToRow -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (56) Filter -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) (57) Project -Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#98, wr_order_number#99] -Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] +Left keys [2]: [wr_item_sk#99, wr_order_number#100] +Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] Join condition: None (59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#106, wr_returned_date_sk#102 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#100 AS return_amt#110, wr_net_loss#101 AS net_loss#111] -Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#109, wr_returned_date_sk#103 AS date_sk#110, 0.00 AS sales_price#111, 0.00 AS profit#112, wr_return_amt#101 AS return_amt#113, wr_net_loss#102 AS net_loss#114] +Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] (60) Union (61) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#115] (62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#93] -Right keys [1]: [cast(d_date_sk#22 as bigint)] +Left keys [1]: [date_sk#94] +Right keys [1]: [cast(d_date_sk#115 as bigint)] Join condition: None (63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97] -Input [7]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, d_date_sk#22] +Output [5]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98] +Input [7]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, d_date_sk#115] (64) Scan parquet default.web_site -Output [2]: [web_site_sk#112, web_site_id#113] +Output [2]: [web_site_sk#116, web_site_id#117] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#112, web_site_id#113] +Input [2]: [web_site_sk#116, web_site_id#117] (66) Filter [codegen id : 17] -Input [2]: [web_site_sk#112, web_site_id#113] -Condition : isnotnull(web_site_sk#112) +Input [2]: [web_site_sk#116, web_site_id#117] +Condition : isnotnull(web_site_sk#116) (67) BroadcastExchange -Input [2]: [web_site_sk#112, web_site_id#113] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#114] +Input [2]: [web_site_sk#116, web_site_id#117] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] (68) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#92] -Right keys [1]: [web_site_sk#112] +Left keys [1]: [wsr_web_site_sk#93] +Right keys [1]: [web_site_sk#116] Join condition: None (69) Project [codegen id : 18] -Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] -Input [7]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#112, web_site_id#113] +Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [7]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] (70) HashAggregate [codegen id : 18] -Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] -Keys [1]: [web_site_id#113] -Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] -Results [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] +Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Keys [1]: [web_site_id#117] +Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum#119, sum#120, sum#121, sum#122] +Results [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] (71) Exchange -Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -Arguments: hashpartitioning(web_site_id#113, 5), ENSURE_REQUIREMENTS, [id=#123] +Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] +Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#127] (72) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -Keys [1]: [web_site_id#113] -Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#124, sum(UnscaledValue(return_amt#96))#125, sum(UnscaledValue(profit#95))#126, sum(UnscaledValue(net_loss#97))#127] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#94))#124,17,2) AS sales#128, MakeDecimal(sum(UnscaledValue(return_amt#96))#125,17,2) AS RETURNS#129, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#126,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#127,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#130, web channel AS channel#131, concat(web_site, web_site_id#113) AS id#132] +Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] +Keys [1]: [web_site_id#117] +Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#128, sum(UnscaledValue(return_amt#97))#129, sum(UnscaledValue(profit#96))#130, sum(UnscaledValue(net_loss#98))#131] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#95))#128,17,2) AS sales#132, MakeDecimal(sum(UnscaledValue(return_amt#97))#129,17,2) AS RETURNS#133, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#130,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#131,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#134, web channel AS channel#135, concat(web_site, web_site_id#117) AS id#136] (73) Union (74) Expand [codegen id : 20] Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45] -Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#133, id#134, spark_grouping_id#135] +Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#137, id#138, spark_grouping_id#139] (75) HashAggregate [codegen id : 20] -Input [6]: [sales#41, returns#42, profit#43, channel#133, id#134, spark_grouping_id#135] -Keys [3]: [channel#133, id#134, spark_grouping_id#135] +Input [6]: [sales#41, returns#42, profit#43, channel#137, id#138, spark_grouping_id#139] +Keys [3]: [channel#137, id#138, spark_grouping_id#139] Functions [3]: [partial_sum(sales#41), partial_sum(returns#42), partial_sum(profit#43)] -Aggregate Attributes [6]: [sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Results [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Results [9]: [channel#137, id#138, spark_grouping_id#139, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] (76) Exchange -Input [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -Arguments: hashpartitioning(channel#133, id#134, spark_grouping_id#135, 5), ENSURE_REQUIREMENTS, [id=#148] +Input [9]: [channel#137, id#138, spark_grouping_id#139, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Arguments: hashpartitioning(channel#137, id#138, spark_grouping_id#139, 5), ENSURE_REQUIREMENTS, [id=#152] (77) HashAggregate [codegen id : 21] -Input [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -Keys [3]: [channel#133, id#134, spark_grouping_id#135] +Input [9]: [channel#137, id#138, spark_grouping_id#139, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Keys [3]: [channel#137, id#138, spark_grouping_id#139] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] -Aggregate Attributes [3]: [sum(sales#41)#149, sum(returns#42)#150, sum(profit#43)#151] -Results [5]: [channel#133, id#134, sum(sales#41)#149 AS sales#152, sum(returns#42)#150 AS returns#153, sum(profit#43)#151 AS profit#154] +Aggregate Attributes [3]: [sum(sales#41)#153, sum(returns#42)#154, sum(profit#43)#155] +Results [5]: [channel#137, id#138, sum(sales#41)#153 AS sales#156, sum(returns#42)#154 AS returns#157, sum(profit#43)#155 AS profit#158] (78) TakeOrderedAndProject -Input [5]: [channel#133, id#134, sales#152, returns#153, profit#154] -Arguments: 100, [channel#133 ASC NULLS FIRST, id#134 ASC NULLS FIRST], [channel#133, id#134, sales#152, returns#153, profit#154] +Input [5]: [channel#137, id#138, sales#156, returns#157, profit#158] +Arguments: 100, [channel#137 ASC NULLS FIRST, id#138 ASC NULLS FIRST], [channel#137, id#138, sales#156, returns#157, profit#158] ===== Subqueries ===== @@ -454,12 +454,12 @@ ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#67] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt index 85906beeff369..b149bdd3e1e3f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt @@ -297,39 +297,39 @@ Input [3]: [ss_customer_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] Condition : isnotnull(ss_customer_sk#33) (51) Scan parquet default.date_dim -Output [2]: [d_date_sk#21, d_month_seq#37] +Output [2]: [d_date_sk#37, d_month_seq#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#21, d_month_seq#37] +Input [2]: [d_date_sk#37, d_month_seq#38] (53) Filter [codegen id : 12] -Input [2]: [d_date_sk#21, d_month_seq#37] -Condition : (((isnotnull(d_month_seq#37) AND (d_month_seq#37 >= Subquery scalar-subquery#38, [id=#39])) AND (d_month_seq#37 <= Subquery scalar-subquery#40, [id=#41])) AND isnotnull(d_date_sk#21)) +Input [2]: [d_date_sk#37, d_month_seq#38] +Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= Subquery scalar-subquery#39, [id=#40])) AND (d_month_seq#38 <= Subquery scalar-subquery#41, [id=#42])) AND isnotnull(d_date_sk#37)) (54) Project [codegen id : 12] -Output [1]: [d_date_sk#21] -Input [2]: [d_date_sk#21, d_month_seq#37] +Output [1]: [d_date_sk#37] +Input [2]: [d_date_sk#37, d_month_seq#38] (55) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [d_date_sk#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#43] (56) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ss_sold_date_sk#35] -Right keys [1]: [d_date_sk#21] +Right keys [1]: [d_date_sk#37] Join condition: None (57) Project [codegen id : 13] Output [2]: [ss_customer_sk#33, ss_ext_sales_price#34] -Input [4]: [ss_customer_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35, d_date_sk#21] +Input [4]: [ss_customer_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35, d_date_sk#37] (58) Exchange Input [2]: [ss_customer_sk#33, ss_ext_sales_price#34] -Arguments: hashpartitioning(ss_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#43] +Arguments: hashpartitioning(ss_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#44] (59) Sort [codegen id : 14] Input [2]: [ss_customer_sk#33, ss_ext_sales_price#34] @@ -348,37 +348,37 @@ Input [3]: [c_customer_sk#30, ss_customer_sk#33, ss_ext_sales_price#34] Input [2]: [c_customer_sk#30, ss_ext_sales_price#34] Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#34))] -Aggregate Attributes [1]: [sum#44] -Results [2]: [c_customer_sk#30, sum#45] +Aggregate Attributes [1]: [sum#45] +Results [2]: [c_customer_sk#30, sum#46] (63) HashAggregate [codegen id : 15] -Input [2]: [c_customer_sk#30, sum#45] +Input [2]: [c_customer_sk#30, sum#46] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#34))#46] -Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#46,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#47] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#34))#47] +Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#47,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#48] (64) HashAggregate [codegen id : 15] -Input [1]: [segment#47] -Keys [1]: [segment#47] +Input [1]: [segment#48] +Keys [1]: [segment#48] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#48] -Results [2]: [segment#47, count#49] +Aggregate Attributes [1]: [count#49] +Results [2]: [segment#48, count#50] (65) Exchange -Input [2]: [segment#47, count#49] -Arguments: hashpartitioning(segment#47, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [segment#48, count#50] +Arguments: hashpartitioning(segment#48, 5), ENSURE_REQUIREMENTS, [id=#51] (66) HashAggregate [codegen id : 16] -Input [2]: [segment#47, count#49] -Keys [1]: [segment#47] +Input [2]: [segment#48, count#50] +Keys [1]: [segment#48] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#51] -Results [3]: [segment#47, count(1)#51 AS num_customers#52, (segment#47 * 50) AS segment_base#53] +Aggregate Attributes [1]: [count(1)#52] +Results [3]: [segment#48, count(1)#52 AS num_customers#53, (segment#48 * 50) AS segment_base#54] (67) TakeOrderedAndProject -Input [3]: [segment#47, num_customers#52, segment_base#53] -Arguments: 100, [segment#47 ASC NULLS FIRST, num_customers#52 ASC NULLS FIRST], [segment#47, num_customers#52, segment_base#53] +Input [3]: [segment#48, num_customers#53, segment_base#54] +Arguments: 100, [segment#48 ASC NULLS FIRST, num_customers#53 ASC NULLS FIRST], [segment#48, num_customers#53, segment_base#54] ===== Subqueries ===== @@ -396,9 +396,9 @@ ReusedExchange (69) (69) ReusedExchange [Reuses operator id: 55] -Output [1]: [d_date_sk#21] +Output [1]: [d_date_sk#37] -Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#39, [id=#40] * HashAggregate (76) +- Exchange (75) +- * HashAggregate (74) @@ -409,42 +409,42 @@ Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (70) Scan parquet default.date_dim -Output [3]: [d_month_seq#37, d_year#22, d_moy#23] +Output [3]: [d_month_seq#55, 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,1998), EqualTo(d_moy,12)] ReadSchema: struct (71) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] +Input [3]: [d_month_seq#55, d_year#56, d_moy#57] (72) Filter [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] -Condition : (((isnotnull(d_year#22) AND isnotnull(d_moy#23)) AND (d_year#22 = 1998)) AND (d_moy#23 = 12)) +Input [3]: [d_month_seq#55, d_year#56, d_moy#57] +Condition : (((isnotnull(d_year#56) AND isnotnull(d_moy#57)) AND (d_year#56 = 1998)) AND (d_moy#57 = 12)) (73) Project [codegen id : 1] -Output [1]: [(d_month_seq#37 + 1) AS (d_month_seq + 1)#54] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] +Output [1]: [(d_month_seq#55 + 1) AS (d_month_seq + 1)#58] +Input [3]: [d_month_seq#55, d_year#56, d_moy#57] (74) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 1)#54] -Keys [1]: [(d_month_seq + 1)#54] +Input [1]: [(d_month_seq + 1)#58] +Keys [1]: [(d_month_seq + 1)#58] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#54] +Results [1]: [(d_month_seq + 1)#58] (75) Exchange -Input [1]: [(d_month_seq + 1)#54] -Arguments: hashpartitioning((d_month_seq + 1)#54, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [1]: [(d_month_seq + 1)#58] +Arguments: hashpartitioning((d_month_seq + 1)#58, 5), ENSURE_REQUIREMENTS, [id=#59] (76) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#54] -Keys [1]: [(d_month_seq + 1)#54] +Input [1]: [(d_month_seq + 1)#58] +Keys [1]: [(d_month_seq + 1)#58] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#54] +Results [1]: [(d_month_seq + 1)#58] -Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#40, [id=#41] +Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (83) +- Exchange (82) +- * HashAggregate (81) @@ -455,39 +455,39 @@ Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (77) Scan parquet default.date_dim -Output [3]: [d_month_seq#37, d_year#22, d_moy#23] +Output [3]: [d_month_seq#60, d_year#61, d_moy#62] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (78) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] +Input [3]: [d_month_seq#60, d_year#61, d_moy#62] (79) Filter [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] -Condition : (((isnotnull(d_year#22) AND isnotnull(d_moy#23)) AND (d_year#22 = 1998)) AND (d_moy#23 = 12)) +Input [3]: [d_month_seq#60, d_year#61, d_moy#62] +Condition : (((isnotnull(d_year#61) AND isnotnull(d_moy#62)) AND (d_year#61 = 1998)) AND (d_moy#62 = 12)) (80) Project [codegen id : 1] -Output [1]: [(d_month_seq#37 + 3) AS (d_month_seq + 3)#56] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] +Output [1]: [(d_month_seq#60 + 3) AS (d_month_seq + 3)#63] +Input [3]: [d_month_seq#60, d_year#61, d_moy#62] (81) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 3)#56] -Keys [1]: [(d_month_seq + 3)#56] +Input [1]: [(d_month_seq + 3)#63] +Keys [1]: [(d_month_seq + 3)#63] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#56] +Results [1]: [(d_month_seq + 3)#63] (82) Exchange -Input [1]: [(d_month_seq + 3)#56] -Arguments: hashpartitioning((d_month_seq + 3)#56, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [1]: [(d_month_seq + 3)#63] +Arguments: hashpartitioning((d_month_seq + 3)#63, 5), ENSURE_REQUIREMENTS, [id=#64] (83) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#56] -Keys [1]: [(d_month_seq + 3)#56] +Input [1]: [(d_month_seq + 3)#63] +Keys [1]: [(d_month_seq + 3)#63] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#56] +Results [1]: [(d_month_seq + 3)#63] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index 2c1f7c6195ba7..d0638d7606bb5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -295,75 +295,75 @@ Output [3]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28] Input [7]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28, ca_county#32, ca_state#33, s_county#35, s_state#36] (51) Scan parquet default.date_dim -Output [2]: [d_date_sk#18, d_month_seq#38] +Output [2]: [d_date_sk#38, d_month_seq#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 10] -Input [2]: [d_date_sk#18, d_month_seq#38] +Input [2]: [d_date_sk#38, d_month_seq#39] (53) Filter [codegen id : 10] -Input [2]: [d_date_sk#18, d_month_seq#38] -Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= Subquery scalar-subquery#39, [id=#40])) AND (d_month_seq#38 <= Subquery scalar-subquery#41, [id=#42])) AND isnotnull(d_date_sk#18)) +Input [2]: [d_date_sk#38, d_month_seq#39] +Condition : (((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= Subquery scalar-subquery#40, [id=#41])) AND (d_month_seq#39 <= Subquery scalar-subquery#42, [id=#43])) AND isnotnull(d_date_sk#38)) (54) Project [codegen id : 10] -Output [1]: [d_date_sk#18] -Input [2]: [d_date_sk#18, d_month_seq#38] +Output [1]: [d_date_sk#38] +Input [2]: [d_date_sk#38, d_month_seq#39] (55) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#43] +Input [1]: [d_date_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (56) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#38] Join condition: None (57) Project [codegen id : 11] Output [2]: [c_customer_sk#22, ss_ext_sales_price#27] -Input [4]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#18] +Input [4]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#38] (58) HashAggregate [codegen id : 11] Input [2]: [c_customer_sk#22, ss_ext_sales_price#27] Keys [1]: [c_customer_sk#22] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum#44] -Results [2]: [c_customer_sk#22, sum#45] +Aggregate Attributes [1]: [sum#45] +Results [2]: [c_customer_sk#22, sum#46] (59) Exchange -Input [2]: [c_customer_sk#22, sum#45] -Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [2]: [c_customer_sk#22, sum#46] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#47] (60) HashAggregate [codegen id : 12] -Input [2]: [c_customer_sk#22, sum#45] +Input [2]: [c_customer_sk#22, sum#46] Keys [1]: [c_customer_sk#22] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#47] -Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#47,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#48] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#48] +Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#48,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#49] (61) HashAggregate [codegen id : 12] -Input [1]: [segment#48] -Keys [1]: [segment#48] +Input [1]: [segment#49] +Keys [1]: [segment#49] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#49] -Results [2]: [segment#48, count#50] +Aggregate Attributes [1]: [count#50] +Results [2]: [segment#49, count#51] (62) Exchange -Input [2]: [segment#48, count#50] -Arguments: hashpartitioning(segment#48, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [2]: [segment#49, count#51] +Arguments: hashpartitioning(segment#49, 5), ENSURE_REQUIREMENTS, [id=#52] (63) HashAggregate [codegen id : 13] -Input [2]: [segment#48, count#50] -Keys [1]: [segment#48] +Input [2]: [segment#49, count#51] +Keys [1]: [segment#49] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#52] -Results [3]: [segment#48, count(1)#52 AS num_customers#53, (segment#48 * 50) AS segment_base#54] +Aggregate Attributes [1]: [count(1)#53] +Results [3]: [segment#49, count(1)#53 AS num_customers#54, (segment#49 * 50) AS segment_base#55] (64) TakeOrderedAndProject -Input [3]: [segment#48, num_customers#53, segment_base#54] -Arguments: 100, [segment#48 ASC NULLS FIRST, num_customers#53 ASC NULLS FIRST], [segment#48, num_customers#53, segment_base#54] +Input [3]: [segment#49, num_customers#54, segment_base#55] +Arguments: 100, [segment#49 ASC NULLS FIRST, num_customers#54 ASC NULLS FIRST], [segment#49, num_customers#54, segment_base#55] ===== Subqueries ===== @@ -381,9 +381,9 @@ ReusedExchange (66) (66) ReusedExchange [Reuses operator id: 55] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#38] -Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#39, [id=#40] +Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#40, [id=#41] * HashAggregate (73) +- Exchange (72) +- * HashAggregate (71) @@ -394,42 +394,42 @@ Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (67) Scan parquet default.date_dim -Output [3]: [d_month_seq#38, d_year#19, d_moy#20] +Output [3]: [d_month_seq#56, d_year#57, d_moy#58] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (68) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] +Input [3]: [d_month_seq#56, d_year#57, d_moy#58] (69) Filter [codegen id : 1] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] -Condition : (((isnotnull(d_year#19) AND isnotnull(d_moy#20)) AND (d_year#19 = 1998)) AND (d_moy#20 = 12)) +Input [3]: [d_month_seq#56, d_year#57, d_moy#58] +Condition : (((isnotnull(d_year#57) AND isnotnull(d_moy#58)) AND (d_year#57 = 1998)) AND (d_moy#58 = 12)) (70) Project [codegen id : 1] -Output [1]: [(d_month_seq#38 + 1) AS (d_month_seq + 1)#55] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] +Output [1]: [(d_month_seq#56 + 1) AS (d_month_seq + 1)#59] +Input [3]: [d_month_seq#56, d_year#57, d_moy#58] (71) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 1)#55] -Keys [1]: [(d_month_seq + 1)#55] +Input [1]: [(d_month_seq + 1)#59] +Keys [1]: [(d_month_seq + 1)#59] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#55] +Results [1]: [(d_month_seq + 1)#59] (72) Exchange -Input [1]: [(d_month_seq + 1)#55] -Arguments: hashpartitioning((d_month_seq + 1)#55, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [1]: [(d_month_seq + 1)#59] +Arguments: hashpartitioning((d_month_seq + 1)#59, 5), ENSURE_REQUIREMENTS, [id=#60] (73) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#55] -Keys [1]: [(d_month_seq + 1)#55] +Input [1]: [(d_month_seq + 1)#59] +Keys [1]: [(d_month_seq + 1)#59] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#55] +Results [1]: [(d_month_seq + 1)#59] -Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#42, [id=#43] * HashAggregate (80) +- Exchange (79) +- * HashAggregate (78) @@ -440,39 +440,39 @@ Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (74) Scan parquet default.date_dim -Output [3]: [d_month_seq#38, d_year#19, d_moy#20] +Output [3]: [d_month_seq#61, d_year#62, d_moy#63] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (75) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] +Input [3]: [d_month_seq#61, d_year#62, d_moy#63] (76) Filter [codegen id : 1] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] -Condition : (((isnotnull(d_year#19) AND isnotnull(d_moy#20)) AND (d_year#19 = 1998)) AND (d_moy#20 = 12)) +Input [3]: [d_month_seq#61, d_year#62, d_moy#63] +Condition : (((isnotnull(d_year#62) AND isnotnull(d_moy#63)) AND (d_year#62 = 1998)) AND (d_moy#63 = 12)) (77) Project [codegen id : 1] -Output [1]: [(d_month_seq#38 + 3) AS (d_month_seq + 3)#57] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] +Output [1]: [(d_month_seq#61 + 3) AS (d_month_seq + 3)#64] +Input [3]: [d_month_seq#61, d_year#62, d_moy#63] (78) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 3)#57] -Keys [1]: [(d_month_seq + 3)#57] +Input [1]: [(d_month_seq + 3)#64] +Keys [1]: [(d_month_seq + 3)#64] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#57] +Results [1]: [(d_month_seq + 3)#64] (79) Exchange -Input [1]: [(d_month_seq + 3)#57] -Arguments: hashpartitioning((d_month_seq + 3)#57, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [1]: [(d_month_seq + 3)#64] +Arguments: hashpartitioning((d_month_seq + 3)#64, 5), ENSURE_REQUIREMENTS, [id=#65] (80) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#57] -Keys [1]: [(d_month_seq + 3)#57] +Input [1]: [(d_month_seq + 3)#64] +Keys [1]: [(d_month_seq + 3)#64] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#57] +Results [1]: [(d_month_seq + 3)#64] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt index 8e3bf5bf57bcf..47eda1483a06c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt @@ -160,30 +160,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#14, i_color#15] +Output [2]: [i_item_id#15, i_color#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#14, i_color#15] +Input [2]: [i_item_id#15, i_color#16] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#14, i_color#15] -Condition : i_color#15 IN (slate ,blanched ,burnished ) +Input [2]: [i_item_id#15, i_color#16] +Condition : i_color#16 IN (slate ,blanched ,burnished ) (24) Project [codegen id : 3] -Output [1]: [i_item_id#14 AS i_item_id#14#16] -Input [2]: [i_item_id#14, i_color#15] +Output [1]: [i_item_id#15] +Input [2]: [i_item_id#15, i_color#16] (25) BroadcastExchange -Input [1]: [i_item_id#14#16] +Input [1]: [i_item_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#14#16] +Right keys [1]: [i_item_id#15] Join condition: None (27) BroadcastExchange @@ -233,127 +233,127 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#30, i_item_id#31] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] +Output [2]: [cs_ext_sales_price#26, i_item_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#14] -Keys [1]: [i_item_id#14] +Input [2]: [cs_ext_sales_price#26, i_item_id#31] +Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#14, sum#29] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#31, sum#33] (46) Exchange -Input [2]: [i_item_id#14, sum#29] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [i_item_id#31, sum#33] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#14, sum#29] -Keys [1]: [i_item_id#14] +Input [2]: [i_item_id#31, sum#33] +Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] (48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] (50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#41] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#42] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#43, i_item_id#44] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_item_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] +Output [2]: [ws_ext_sales_price#39, i_item_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_item_id#14, sum#38] +Input [2]: [ws_ext_sales_price#39, i_item_id#44] +Keys [1]: [i_item_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_item_id#44, sum#46] (61) Exchange -Input [2]: [i_item_id#14, sum#38] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [2]: [i_item_id#44, sum#46] +Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] (62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#38] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +Input [2]: [i_item_id#44, sum#46] +Keys [1]: [i_item_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] (63) Union @@ -361,23 +361,23 @@ Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35) Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_item_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_item_id#14, sum#52, isEmpty#53] (65) Exchange -Input [3]: [i_item_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] (66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#44, isEmpty#45] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] (67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#48] -Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] +Input [2]: [i_item_id#14, total_sales#56] +Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] ===== Subqueries ===== @@ -390,6 +390,6 @@ Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 8e3bf5bf57bcf..47eda1483a06c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -160,30 +160,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#14, i_color#15] +Output [2]: [i_item_id#15, i_color#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#14, i_color#15] +Input [2]: [i_item_id#15, i_color#16] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#14, i_color#15] -Condition : i_color#15 IN (slate ,blanched ,burnished ) +Input [2]: [i_item_id#15, i_color#16] +Condition : i_color#16 IN (slate ,blanched ,burnished ) (24) Project [codegen id : 3] -Output [1]: [i_item_id#14 AS i_item_id#14#16] -Input [2]: [i_item_id#14, i_color#15] +Output [1]: [i_item_id#15] +Input [2]: [i_item_id#15, i_color#16] (25) BroadcastExchange -Input [1]: [i_item_id#14#16] +Input [1]: [i_item_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#14#16] +Right keys [1]: [i_item_id#15] Join condition: None (27) BroadcastExchange @@ -233,127 +233,127 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#30, i_item_id#31] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] +Output [2]: [cs_ext_sales_price#26, i_item_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#14] -Keys [1]: [i_item_id#14] +Input [2]: [cs_ext_sales_price#26, i_item_id#31] +Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#14, sum#29] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#31, sum#33] (46) Exchange -Input [2]: [i_item_id#14, sum#29] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [i_item_id#31, sum#33] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#14, sum#29] -Keys [1]: [i_item_id#14] +Input [2]: [i_item_id#31, sum#33] +Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] (48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] (50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#41] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#42] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#43, i_item_id#44] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_item_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] +Output [2]: [ws_ext_sales_price#39, i_item_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_item_id#14, sum#38] +Input [2]: [ws_ext_sales_price#39, i_item_id#44] +Keys [1]: [i_item_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_item_id#44, sum#46] (61) Exchange -Input [2]: [i_item_id#14, sum#38] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [2]: [i_item_id#44, sum#46] +Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] (62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#38] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +Input [2]: [i_item_id#44, sum#46] +Keys [1]: [i_item_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] (63) Union @@ -361,23 +361,23 @@ Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35) Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_item_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_item_id#14, sum#52, isEmpty#53] (65) Exchange -Input [3]: [i_item_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] (66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#44, isEmpty#45] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] (67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#48] -Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] +Input [2]: [i_item_id#14, total_sales#56] +Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] ===== Subqueries ===== @@ -390,6 +390,6 @@ Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 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 f0be099379418..6e39745703215 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 @@ -229,78 +229,78 @@ Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] (39) HashAggregate [codegen id : 21] Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] Keys [5]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33] -Functions [1]: [sum(UnscaledValue(cs_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#3))#35] -Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#3))#35,17,2) AS sum_sales#36] +Functions [1]: [sum(UnscaledValue(cs_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#35))#36] +Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#35))#36,17,2) AS sum_sales#37] (40) Exchange -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] (41) Sort [codegen id : 22] -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 (42) Window -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] -Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] (43) Project [codegen id : 23] -Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] +Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] (44) Exchange -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#38 + 1), 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] (45) Sort [codegen id : 24] -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#38 + 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 (46) SortMergeJoin [codegen id : 25] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#38 + 1)] +Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#39 + 1)] Join condition: None (47) Project [codegen id : 25] -Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] -Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] +Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] +Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] (48) ReusedExchange [Reuses operator id: 40] -Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Output [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] (49) Sort [codegen id : 34] -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 (50) Window -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] +Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] +Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#41, i_brand#42, cc_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#41, i_brand#42, cc_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] (51) Project [codegen id : 35] -Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] +Output [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] (52) Exchange -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (rn#46 - 1), 5), ENSURE_REQUIREMENTS, [id=#47] +Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Arguments: hashpartitioning(i_category#41, i_brand#42, cc_name#43, (rn#47 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] (53) Sort [codegen id : 36] -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, (rn#47 - 1) ASC NULLS FIRST], false, 0 (54) SortMergeJoin [codegen id : 37] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (rn#46 - 1)] +Right keys [4]: [i_category#41, i_brand#42, cc_name#43, (rn#47 - 1)] Join condition: None (55) Project [codegen id : 37] -Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#48, sum_sales#45 AS nsum#49] -Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] +Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#49, sum_sales#46 AS nsum#50] +Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] (56) TakeOrderedAndProject -Input [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] +Input [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] ===== Subqueries ===== 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 1eb7dbe89d6e1..fa24f5ede5453 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 @@ -202,70 +202,70 @@ Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] (34) HashAggregate [codegen id : 13] 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#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#33] -Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#33,17,2) AS sum_sales#34] +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] (35) Exchange -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#35] +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] (36) Sort [codegen id : 14] -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35] 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 (37) Window -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] -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#36], [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#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] (38) Project [codegen id : 15] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] -Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34, rn#36] +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] (39) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#37] +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#38] (40) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#36 + 1)] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] Join condition: None (41) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34] -Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] +Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35] +Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] (42) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] +Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] (43) Sort [codegen id : 21] -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] -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 +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 (44) Window -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] -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#44], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] +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] (45) Project [codegen id : 22] -Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] -Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43, rn#44] +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] (46) BroadcastExchange -Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#45] +Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#46] (47) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -Right keys [4]: [i_category#38, i_brand#39, cc_name#40, (rn#44 - 1)] +Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] Join condition: None (48) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#34 AS psum#46, sum_sales#43 AS nsum#47] -Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34, i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] +Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] +Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] (49) TakeOrderedAndProject -Input [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, 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#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, psum#46, nsum#47] +Input [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, 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#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, psum#47, nsum#48] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index f6dd5a59de4d4..b339df0707d2f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -110,30 +110,30 @@ Input [2]: [d_date_sk#4, d_date#5] Condition : isnotnull(d_date_sk#4) (7) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#6, d_week_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#6, d_week_seq#7] (9) Filter [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = Subquery scalar-subquery#7, [id=#8])) +Input [2]: [d_date#6, d_week_seq#7] +Condition : (isnotnull(d_week_seq#7) AND (d_week_seq#7 = Subquery scalar-subquery#8, [id=#9])) (10) Project [codegen id : 1] -Output [1]: [d_date#5 AS d_date#5#9] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_date#6] +Input [2]: [d_date#6, d_week_seq#7] (11) BroadcastExchange -Input [1]: [d_date#5#9] +Input [1]: [d_date#6] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] (12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [d_date#5] -Right keys [1]: [d_date#5#9] +Right keys [1]: [d_date#6] Join condition: None (13) Project [codegen id : 2] @@ -218,237 +218,237 @@ Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Condition : isnotnull(cs_item_sk#21) (30) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] +Output [2]: [d_date_sk#24, d_date#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#4, d_date#5] +Input [2]: [d_date_sk#24, d_date#25] (32) Filter [codegen id : 6] -Input [2]: [d_date_sk#4, d_date#5] -Condition : isnotnull(d_date_sk#4) +Input [2]: [d_date_sk#24, d_date#25] +Condition : isnotnull(d_date_sk#24) (33) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#26, d_week_seq#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (34) ColumnarToRow [codegen id : 5] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#26, d_week_seq#27] (35) Filter [codegen id : 5] -Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) +Input [2]: [d_date#26, d_week_seq#27] +Condition : (isnotnull(d_week_seq#27) AND (d_week_seq#27 = ReusedSubquery Subquery scalar-subquery#8, [id=#9])) (36) Project [codegen id : 5] -Output [1]: [d_date#5 AS d_date#5#24] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_date#26] +Input [2]: [d_date#26, d_week_seq#27] (37) BroadcastExchange -Input [1]: [d_date#5#24] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#25] +Input [1]: [d_date#26] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#28] (38) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [d_date#5] -Right keys [1]: [d_date#5#24] +Left keys [1]: [d_date#25] +Right keys [1]: [d_date#26] Join condition: None (39) Project [codegen id : 6] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_date#25] (40) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#4] +Right keys [1]: [d_date_sk#24] Join condition: None (42) Project [codegen id : 8] Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#4] +Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] (43) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#12, i_item_id#13] +Output [2]: [i_item_sk#30, i_item_id#31] (44) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#12] +Right keys [1]: [i_item_sk#30] Join condition: None (45) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#22, i_item_id#13] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#12, i_item_id#13] +Output [2]: [cs_ext_sales_price#22, i_item_id#31] +Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#30, i_item_id#31] (46) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#22, i_item_id#13] -Keys [1]: [i_item_id#13] +Input [2]: [cs_ext_sales_price#22, i_item_id#31] +Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_item_id#13, sum#28] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#31, sum#33] (47) Exchange -Input [2]: [i_item_id#13, sum#28] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [2]: [i_item_id#31, sum#33] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] (48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#13, sum#28] -Keys [1]: [i_item_id#13] +Input [2]: [i_item_id#31, sum#33] +Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#13 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#35] +Results [2]: [i_item_id#31 AS item_id#36, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#35,17,2) AS cs_item_rev#37] (49) Filter [codegen id : 9] -Input [2]: [item_id#31, cs_item_rev#32] -Condition : isnotnull(cs_item_rev#32) +Input [2]: [item_id#36, cs_item_rev#37] +Condition : isnotnull(cs_item_rev#37) (50) BroadcastExchange -Input [2]: [item_id#31, cs_item_rev#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] +Input [2]: [item_id#36, cs_item_rev#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#38] (51) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#31] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) +Right keys [1]: [item_id#36] +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) (52) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] -Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] +Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#37] +Input [4]: [item_id#19, ss_item_rev#20, item_id#36, cs_item_rev#37] (53) Scan parquet default.web_sales -Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] (55) Filter [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#34) +Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] +Condition : isnotnull(ws_item_sk#39) (56) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] +Output [2]: [d_date_sk#42, d_date#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 11] -Input [2]: [d_date_sk#4, d_date#5] +Input [2]: [d_date_sk#42, d_date#43] (58) Filter [codegen id : 11] -Input [2]: [d_date_sk#4, d_date#5] -Condition : isnotnull(d_date_sk#4) +Input [2]: [d_date_sk#42, d_date#43] +Condition : isnotnull(d_date_sk#42) (59) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#44, d_week_seq#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (60) ColumnarToRow [codegen id : 10] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#44, d_week_seq#45] (61) Filter [codegen id : 10] -Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) +Input [2]: [d_date#44, d_week_seq#45] +Condition : (isnotnull(d_week_seq#45) AND (d_week_seq#45 = ReusedSubquery Subquery scalar-subquery#8, [id=#9])) (62) Project [codegen id : 10] -Output [1]: [d_date#5 AS d_date#5#37] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_date#44] +Input [2]: [d_date#44, d_week_seq#45] (63) BroadcastExchange -Input [1]: [d_date#5#37] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#38] +Input [1]: [d_date#44] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#46] (64) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_date#5] -Right keys [1]: [d_date#5#37] +Left keys [1]: [d_date#43] +Right keys [1]: [d_date#44] Join condition: None (65) Project [codegen id : 11] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +Output [1]: [d_date_sk#42] +Input [2]: [d_date_sk#42, d_date#43] (66) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Input [1]: [d_date_sk#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] (67) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#4] +Left keys [1]: [ws_sold_date_sk#41] +Right keys [1]: [d_date_sk#42] Join condition: None (68) Project [codegen id : 13] -Output [2]: [ws_item_sk#34, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#4] +Output [2]: [ws_item_sk#39, ws_ext_sales_price#40] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41, d_date_sk#42] (69) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#12, i_item_id#13] +Output [2]: [i_item_sk#48, i_item_id#49] (70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#12] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#48] Join condition: None (71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#35, i_item_id#13] -Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, i_item_sk#12, i_item_id#13] +Output [2]: [ws_ext_sales_price#40, i_item_id#49] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#40, i_item_sk#48, i_item_id#49] (72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#35, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#13, sum#41] +Input [2]: [ws_ext_sales_price#40, i_item_id#49] +Keys [1]: [i_item_id#49] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#40))] +Aggregate Attributes [1]: [sum#50] +Results [2]: [i_item_id#49, sum#51] (73) Exchange -Input [2]: [i_item_id#13, sum#41] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [2]: [i_item_id#49, sum#51] +Arguments: hashpartitioning(i_item_id#49, 5), ENSURE_REQUIREMENTS, [id=#52] (74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#13, sum#41] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] -Results [2]: [i_item_id#13 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] +Input [2]: [i_item_id#49, sum#51] +Keys [1]: [i_item_id#49] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#40))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#40))#53] +Results [2]: [i_item_id#49 AS item_id#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#40))#53,17,2) AS ws_item_rev#55] (75) Filter [codegen id : 14] -Input [2]: [item_id#44, ws_item_rev#45] -Condition : isnotnull(ws_item_rev#45) +Input [2]: [item_id#54, ws_item_rev#55] +Condition : isnotnull(ws_item_rev#55) (76) BroadcastExchange -Input [2]: [item_id#44, ws_item_rev#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] +Input [2]: [item_id#54, ws_item_rev#55] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] (77) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#44] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) +Right keys [1]: [item_id#54] +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) (78) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#57, cs_item_rev#37, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#37 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#58, ws_item_rev#55, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#55 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#59, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#60] +Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#37, item_id#54, ws_item_rev#55] (79) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] ===== Subqueries ===== -Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#7, [id=#8] +Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#8, [id=#9] * Project (83) +- * Filter (82) +- * ColumnarToRow (81) @@ -456,25 +456,25 @@ Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery (80) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#61, d_week_seq#62] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct (81) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#61, d_week_seq#62] (82) Filter [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_date#5) AND (d_date#5 = 2000-01-03)) +Input [2]: [d_date#61, d_week_seq#62] +Condition : (isnotnull(d_date#61) AND (d_date#61 = 2000-01-03)) (83) Project [codegen id : 1] -Output [1]: [d_week_seq#6] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_week_seq#62] +Input [2]: [d_date#61, d_week_seq#62] -Subquery:2 Hosting operator id = 35 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] +Subquery:2 Hosting operator id = 35 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] -Subquery:3 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] +Subquery:3 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index ab4c4ad4ae65b..1f3e6853a3c41 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -137,30 +137,30 @@ Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) (13) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#9, d_week_seq#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (14) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#9, d_week_seq#10] (15) Filter [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = Subquery scalar-subquery#10, [id=#11])) +Input [2]: [d_date#9, d_week_seq#10] +Condition : (isnotnull(d_week_seq#10) AND (d_week_seq#10 = Subquery scalar-subquery#11, [id=#12])) (16) Project [codegen id : 2] -Output [1]: [d_date#8 AS d_date#8#12] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_date#9] +Input [2]: [d_date#9, d_week_seq#10] (17) BroadcastExchange -Input [1]: [d_date#8#12] +Input [1]: [d_date#9] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] (18) BroadcastHashJoin [codegen id : 3] Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#12] +Right keys [1]: [d_date#9] Join condition: None (19) Project [codegen id : 3] @@ -218,237 +218,237 @@ Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Condition : isnotnull(cs_item_sk#21) (30) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#24, i_item_id#25] (31) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#4] +Right keys [1]: [i_item_sk#24] Join condition: None (32) Project [codegen id : 8] -Output [3]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#5] -Input [5]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_sk#4, i_item_id#5] +Output [3]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25] +Input [5]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_sk#24, i_item_id#25] (33) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] +Output [2]: [d_date_sk#26, d_date#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#7, d_date#8] +Input [2]: [d_date_sk#26, d_date#27] (35) Filter [codegen id : 7] -Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) +Input [2]: [d_date_sk#26, d_date#27] +Condition : isnotnull(d_date_sk#26) (36) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#28, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#28, d_week_seq#29] (38) Filter [codegen id : 6] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) +Input [2]: [d_date#28, d_week_seq#29] +Condition : (isnotnull(d_week_seq#29) AND (d_week_seq#29 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) (39) Project [codegen id : 6] -Output [1]: [d_date#8 AS d_date#8#24] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_date#28] +Input [2]: [d_date#28, d_week_seq#29] (40) BroadcastExchange -Input [1]: [d_date#8#24] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#25] +Input [1]: [d_date#28] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#30] (41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#24] +Left keys [1]: [d_date#27] +Right keys [1]: [d_date#28] Join condition: None (42) Project [codegen id : 7] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] +Output [1]: [d_date_sk#26] +Input [2]: [d_date_sk#26, d_date#27] (43) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (44) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#26] Join condition: None (45) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#22, i_item_id#5] -Input [4]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#5, d_date_sk#7] +Output [2]: [cs_ext_sales_price#22, i_item_id#25] +Input [4]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25, d_date_sk#26] (46) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#22, i_item_id#5] -Keys [1]: [i_item_id#5] +Input [2]: [cs_ext_sales_price#22, i_item_id#25] +Keys [1]: [i_item_id#25] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_item_id#5, sum#28] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#25, sum#33] (47) Exchange -Input [2]: [i_item_id#5, sum#28] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [2]: [i_item_id#25, sum#33] +Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [id=#34] (48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#5, sum#28] -Keys [1]: [i_item_id#5] +Input [2]: [i_item_id#25, sum#33] +Keys [1]: [i_item_id#25] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#5 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#35] +Results [2]: [i_item_id#25 AS item_id#36, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#35,17,2) AS cs_item_rev#37] (49) Filter [codegen id : 9] -Input [2]: [item_id#31, cs_item_rev#32] -Condition : isnotnull(cs_item_rev#32) +Input [2]: [item_id#36, cs_item_rev#37] +Condition : isnotnull(cs_item_rev#37) (50) BroadcastExchange -Input [2]: [item_id#31, cs_item_rev#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] +Input [2]: [item_id#36, cs_item_rev#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#38] (51) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#31] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) +Right keys [1]: [item_id#36] +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) (52) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] -Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] +Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#37] +Input [4]: [item_id#19, ss_item_rev#20, item_id#36, cs_item_rev#37] (53) Scan parquet default.web_sales -Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] (55) Filter [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#34) +Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] +Condition : isnotnull(ws_item_sk#39) (56) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#42, i_item_id#43] (57) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#4] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#42] Join condition: None (58) Project [codegen id : 13] -Output [3]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#5] -Input [5]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_sk#4, i_item_id#5] +Output [3]: [ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_id#43] +Input [5]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_sk#42, i_item_id#43] (59) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] +Output [2]: [d_date_sk#44, d_date#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#7, d_date#8] +Input [2]: [d_date_sk#44, d_date#45] (61) Filter [codegen id : 12] -Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) +Input [2]: [d_date_sk#44, d_date#45] +Condition : isnotnull(d_date_sk#44) (62) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#46, d_week_seq#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (63) ColumnarToRow [codegen id : 11] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#46, d_week_seq#47] (64) Filter [codegen id : 11] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) +Input [2]: [d_date#46, d_week_seq#47] +Condition : (isnotnull(d_week_seq#47) AND (d_week_seq#47 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) (65) Project [codegen id : 11] -Output [1]: [d_date#8 AS d_date#8#37] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_date#46] +Input [2]: [d_date#46, d_week_seq#47] (66) BroadcastExchange -Input [1]: [d_date#8#37] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#38] +Input [1]: [d_date#46] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#48] (67) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#37] +Left keys [1]: [d_date#45] +Right keys [1]: [d_date#46] Join condition: None (68) Project [codegen id : 12] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] +Output [1]: [d_date_sk#44] +Input [2]: [d_date_sk#44, d_date#45] (69) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Input [1]: [d_date_sk#44] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] (70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [ws_sold_date_sk#41] +Right keys [1]: [d_date_sk#44] Join condition: None (71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#35, i_item_id#5] -Input [4]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#5, d_date_sk#7] +Output [2]: [ws_ext_sales_price#40, i_item_id#43] +Input [4]: [ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_id#43, d_date_sk#44] (72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#35, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#5, sum#41] +Input [2]: [ws_ext_sales_price#40, i_item_id#43] +Keys [1]: [i_item_id#43] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#40))] +Aggregate Attributes [1]: [sum#50] +Results [2]: [i_item_id#43, sum#51] (73) Exchange -Input [2]: [i_item_id#5, sum#41] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [2]: [i_item_id#43, sum#51] +Arguments: hashpartitioning(i_item_id#43, 5), ENSURE_REQUIREMENTS, [id=#52] (74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#5, sum#41] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] -Results [2]: [i_item_id#5 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] +Input [2]: [i_item_id#43, sum#51] +Keys [1]: [i_item_id#43] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#40))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#40))#53] +Results [2]: [i_item_id#43 AS item_id#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#40))#53,17,2) AS ws_item_rev#55] (75) Filter [codegen id : 14] -Input [2]: [item_id#44, ws_item_rev#45] -Condition : isnotnull(ws_item_rev#45) +Input [2]: [item_id#54, ws_item_rev#55] +Condition : isnotnull(ws_item_rev#55) (76) BroadcastExchange -Input [2]: [item_id#44, ws_item_rev#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] +Input [2]: [item_id#54, ws_item_rev#55] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] (77) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#44] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) +Right keys [1]: [item_id#54] +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) (78) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#57, cs_item_rev#37, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#37 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#58, ws_item_rev#55, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#55 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#59, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#60] +Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#37, item_id#54, ws_item_rev#55] (79) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] ===== Subqueries ===== -Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#10, [id=#11] +Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#11, [id=#12] * Project (83) +- * Filter (82) +- * ColumnarToRow (81) @@ -456,25 +456,25 @@ Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquer (80) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#61, d_week_seq#62] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct (81) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#61, d_week_seq#62] (82) Filter [codegen id : 1] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_date#8) AND (d_date#8 = 2000-01-03)) +Input [2]: [d_date#61, d_week_seq#62] +Condition : (isnotnull(d_date#61) AND (d_date#61 = 2000-01-03)) (83) Project [codegen id : 1] -Output [1]: [d_week_seq#9] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_week_seq#62] +Input [2]: [d_date#61, d_week_seq#62] -Subquery:2 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] +Subquery:2 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] -Subquery:3 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] +Subquery:3 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] 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 fb57c1b00ecf3..cf22f178dd33f 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#5, ss_store_sk#1, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] +Output [9]: [d_week_seq#54, ss_store_sk#55, sum#56, sum#57, sum#58, sum#59, sum#60, sum#61, sum#62] (27) HashAggregate [codegen id : 9] -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))#61, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67] -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))#61,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63,17,2) AS tue_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67,17,2) AS sat_sales#36] +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] (28) Scan parquet default.store -Output [2]: [s_store_sk#37, s_store_id#38] +Output [2]: [s_store_sk#72, s_store_id#73] 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#37, s_store_id#38] +Input [2]: [s_store_sk#72, s_store_id#73] (30) Filter [codegen id : 7] -Input [2]: [s_store_sk#37, s_store_id#38] -Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) +Input [2]: [s_store_sk#72, s_store_id#73] +Condition : (isnotnull(s_store_sk#72) AND isnotnull(s_store_id#73)) (31) BroadcastExchange -Input [2]: [s_store_sk#37, s_store_id#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#68] +Input [2]: [s_store_sk#72, s_store_id#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#37] +Left keys [1]: [ss_store_sk#55] +Right keys [1]: [s_store_sk#72] Join condition: None (33) Project [codegen id : 9] -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#38] -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#37, s_store_id#38] +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] (34) Scan parquet default.date_dim -Output [2]: [d_month_seq#69, d_week_seq#70] +Output [2]: [d_month_seq#75, d_week_seq#76] 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#69, d_week_seq#70] +Input [2]: [d_month_seq#75, d_week_seq#76] (36) Filter [codegen id : 8] -Input [2]: [d_month_seq#69, d_week_seq#70] -Condition : (((isnotnull(d_month_seq#69) AND (d_month_seq#69 >= 1224)) AND (d_month_seq#69 <= 1235)) AND isnotnull(d_week_seq#70)) +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)) (37) Project [codegen id : 8] -Output [1]: [d_week_seq#70] -Input [2]: [d_month_seq#69, d_week_seq#70] +Output [1]: [d_week_seq#76] +Input [2]: [d_month_seq#75, d_week_seq#76] (38) BroadcastExchange -Input [1]: [d_week_seq#70] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] +Input [1]: [d_week_seq#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] (39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#70] +Left keys [1]: [d_week_seq#54] +Right keys [1]: [d_week_seq#76] Join condition: None (40) Project [codegen id : 9] -Output [9]: [d_week_seq#5 AS d_week_seq2#72, s_store_id#38 AS s_store_id2#73, sun_sales#30 AS sun_sales2#74, mon_sales#31 AS mon_sales2#75, tue_sales#32 AS tue_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 [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#38, d_week_seq#70] +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] (41) BroadcastExchange -Input [9]: [d_week_seq2#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_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] +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] (42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#73, (d_week_seq2#72 - 52)] +Right keys [2]: [s_store_id2#79, (d_week_seq2#78 - 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#74)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#82, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#75)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#83, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#76)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#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 [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#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] +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] (44) TakeOrderedAndProject -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_sales2)#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_sales2)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] +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] 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 fb57c1b00ecf3..cf22f178dd33f 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#5, ss_store_sk#1, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] +Output [9]: [d_week_seq#54, ss_store_sk#55, sum#56, sum#57, sum#58, sum#59, sum#60, sum#61, sum#62] (27) HashAggregate [codegen id : 9] -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))#61, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67] -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))#61,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63,17,2) AS tue_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67,17,2) AS sat_sales#36] +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] (28) Scan parquet default.store -Output [2]: [s_store_sk#37, s_store_id#38] +Output [2]: [s_store_sk#72, s_store_id#73] 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#37, s_store_id#38] +Input [2]: [s_store_sk#72, s_store_id#73] (30) Filter [codegen id : 7] -Input [2]: [s_store_sk#37, s_store_id#38] -Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) +Input [2]: [s_store_sk#72, s_store_id#73] +Condition : (isnotnull(s_store_sk#72) AND isnotnull(s_store_id#73)) (31) BroadcastExchange -Input [2]: [s_store_sk#37, s_store_id#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#68] +Input [2]: [s_store_sk#72, s_store_id#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#37] +Left keys [1]: [ss_store_sk#55] +Right keys [1]: [s_store_sk#72] Join condition: None (33) Project [codegen id : 9] -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#38] -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#37, s_store_id#38] +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] (34) Scan parquet default.date_dim -Output [2]: [d_month_seq#69, d_week_seq#70] +Output [2]: [d_month_seq#75, d_week_seq#76] 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#69, d_week_seq#70] +Input [2]: [d_month_seq#75, d_week_seq#76] (36) Filter [codegen id : 8] -Input [2]: [d_month_seq#69, d_week_seq#70] -Condition : (((isnotnull(d_month_seq#69) AND (d_month_seq#69 >= 1224)) AND (d_month_seq#69 <= 1235)) AND isnotnull(d_week_seq#70)) +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)) (37) Project [codegen id : 8] -Output [1]: [d_week_seq#70] -Input [2]: [d_month_seq#69, d_week_seq#70] +Output [1]: [d_week_seq#76] +Input [2]: [d_month_seq#75, d_week_seq#76] (38) BroadcastExchange -Input [1]: [d_week_seq#70] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] +Input [1]: [d_week_seq#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] (39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#70] +Left keys [1]: [d_week_seq#54] +Right keys [1]: [d_week_seq#76] Join condition: None (40) Project [codegen id : 9] -Output [9]: [d_week_seq#5 AS d_week_seq2#72, s_store_id#38 AS s_store_id2#73, sun_sales#30 AS sun_sales2#74, mon_sales#31 AS mon_sales2#75, tue_sales#32 AS tue_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 [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#38, d_week_seq#70] +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] (41) BroadcastExchange -Input [9]: [d_week_seq2#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_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] +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] (42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#73, (d_week_seq2#72 - 52)] +Right keys [2]: [s_store_id2#79, (d_week_seq2#78 - 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#74)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#82, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#75)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#83, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#76)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#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 [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#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] +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] (44) TakeOrderedAndProject -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_sales2)#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_sales2)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] +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] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt index 10b8d0d9e7f05..49cfd232239aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt @@ -160,30 +160,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#14, i_category#15] +Output [2]: [i_item_id#15, i_category#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#14, i_category#15] +Input [2]: [i_item_id#15, i_category#16] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#14, i_category#15] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) +Input [2]: [i_item_id#15, i_category#16] +Condition : (isnotnull(i_category#16) AND (i_category#16 = Music )) (24) Project [codegen id : 3] -Output [1]: [i_item_id#14 AS i_item_id#14#16] -Input [2]: [i_item_id#14, i_category#15] +Output [1]: [i_item_id#15] +Input [2]: [i_item_id#15, i_category#16] (25) BroadcastExchange -Input [1]: [i_item_id#14#16] +Input [1]: [i_item_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#14#16] +Right keys [1]: [i_item_id#15] Join condition: None (27) BroadcastExchange @@ -233,127 +233,127 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#30, i_item_id#31] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] +Output [2]: [cs_ext_sales_price#26, i_item_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#14] -Keys [1]: [i_item_id#14] +Input [2]: [cs_ext_sales_price#26, i_item_id#31] +Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#14, sum#29] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#31, sum#33] (46) Exchange -Input [2]: [i_item_id#14, sum#29] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [i_item_id#31, sum#33] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#14, sum#29] -Keys [1]: [i_item_id#14] +Input [2]: [i_item_id#31, sum#33] +Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] (48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] (50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#41] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#42] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#43, i_item_id#44] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_item_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] +Output [2]: [ws_ext_sales_price#39, i_item_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_item_id#14, sum#38] +Input [2]: [ws_ext_sales_price#39, i_item_id#44] +Keys [1]: [i_item_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_item_id#44, sum#46] (61) Exchange -Input [2]: [i_item_id#14, sum#38] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [2]: [i_item_id#44, sum#46] +Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] (62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#38] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +Input [2]: [i_item_id#44, sum#46] +Keys [1]: [i_item_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] (63) Union @@ -361,23 +361,23 @@ Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35) Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_item_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_item_id#14, sum#52, isEmpty#53] (65) Exchange -Input [3]: [i_item_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] (66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#44, isEmpty#45] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] (67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#48] -Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] +Input [2]: [i_item_id#14, total_sales#56] +Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] ===== Subqueries ===== @@ -390,6 +390,6 @@ Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 10b8d0d9e7f05..49cfd232239aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -160,30 +160,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#14, i_category#15] +Output [2]: [i_item_id#15, i_category#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#14, i_category#15] +Input [2]: [i_item_id#15, i_category#16] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#14, i_category#15] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) +Input [2]: [i_item_id#15, i_category#16] +Condition : (isnotnull(i_category#16) AND (i_category#16 = Music )) (24) Project [codegen id : 3] -Output [1]: [i_item_id#14 AS i_item_id#14#16] -Input [2]: [i_item_id#14, i_category#15] +Output [1]: [i_item_id#15] +Input [2]: [i_item_id#15, i_category#16] (25) BroadcastExchange -Input [1]: [i_item_id#14#16] +Input [1]: [i_item_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#14#16] +Right keys [1]: [i_item_id#15] Join condition: None (27) BroadcastExchange @@ -233,127 +233,127 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#30, i_item_id#31] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] +Output [2]: [cs_ext_sales_price#26, i_item_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#14] -Keys [1]: [i_item_id#14] +Input [2]: [cs_ext_sales_price#26, i_item_id#31] +Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#14, sum#29] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#31, sum#33] (46) Exchange -Input [2]: [i_item_id#14, sum#29] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [i_item_id#31, sum#33] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#14, sum#29] -Keys [1]: [i_item_id#14] +Input [2]: [i_item_id#31, sum#33] +Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] (48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] (50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#41] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#42] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#43, i_item_id#44] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_item_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] +Output [2]: [ws_ext_sales_price#39, i_item_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_item_id#14, sum#38] +Input [2]: [ws_ext_sales_price#39, i_item_id#44] +Keys [1]: [i_item_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_item_id#44, sum#46] (61) Exchange -Input [2]: [i_item_id#14, sum#38] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [2]: [i_item_id#44, sum#46] +Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] (62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#38] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +Input [2]: [i_item_id#44, sum#46] +Keys [1]: [i_item_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] (63) Union @@ -361,23 +361,23 @@ Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35) Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_item_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_item_id#14, sum#52, isEmpty#53] (65) Exchange -Input [3]: [i_item_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] (66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#44, isEmpty#45] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] (67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#48] -Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] +Input [2]: [i_item_id#14, total_sales#56] +Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] ===== Subqueries ===== @@ -390,6 +390,6 @@ Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt index 23807f0561551..410fd9bc3d4e7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt @@ -286,100 +286,100 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#32] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#32,17,2) AS promotions#33] (48) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] +Output [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] (50) Filter [codegen id : 13] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_store_sk#36) AND isnotnull(ss_customer_sk#35)) AND isnotnull(ss_item_sk#34)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#39] (52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ss_sold_date_sk#38] +Right keys [1]: [d_date_sk#39] Join condition: None (53) Project [codegen id : 13] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] +Output [4]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37] +Input [6]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38, d_date_sk#39] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [i_item_sk#12] +Output [1]: [i_item_sk#40] (55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#12] +Left keys [1]: [ss_item_sk#34] +Right keys [1]: [i_item_sk#40] Join condition: None (56) Project [codegen id : 13] -Output [3]: [ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, i_item_sk#12] +Output [3]: [ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37] +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, i_item_sk#40] (57) ReusedExchange [Reuses operator id: 29] -Output [1]: [s_store_sk#20] +Output [1]: [s_store_sk#41] (58) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#20] +Left keys [1]: [ss_store_sk#36] +Right keys [1]: [s_store_sk#41] Join condition: None (59) Project [codegen id : 13] -Output [2]: [ss_customer_sk#2, ss_ext_sales_price#5] -Input [4]: [ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, s_store_sk#20] +Output [2]: [ss_customer_sk#35, ss_ext_sales_price#37] +Input [4]: [ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, s_store_sk#41] (60) ReusedExchange [Reuses operator id: 42] -Output [1]: [c_customer_sk#23] +Output [1]: [c_customer_sk#42] (61) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#23] +Left keys [1]: [ss_customer_sk#35] +Right keys [1]: [c_customer_sk#42] Join condition: None (62) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#5] -Input [3]: [ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#23] +Output [1]: [ss_ext_sales_price#37] +Input [3]: [ss_customer_sk#35, ss_ext_sales_price#37, c_customer_sk#42] (63) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#5] +Input [1]: [ss_ext_sales_price#37] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum#34] -Results [1]: [sum#35] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#37))] +Aggregate Attributes [1]: [sum#43] +Results [1]: [sum#44] (64) Exchange -Input [1]: [sum#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#36] +Input [1]: [sum#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45] (65) HashAggregate [codegen id : 14] -Input [1]: [sum#35] +Input [1]: [sum#44] Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#37] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#37,17,2) AS total#38] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#37))#46] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#37))#46,17,2) AS total#47] (66) BroadcastExchange -Input [1]: [total#38] -Arguments: IdentityBroadcastMode, [id=#39] +Input [1]: [total#47] +Arguments: IdentityBroadcastMode, [id=#48] (67) BroadcastNestedLoopJoin [codegen id : 15] Join condition: None (68) Project [codegen id : 15] -Output [3]: [promotions#33, total#38, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#38 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] -Input [2]: [promotions#33, total#38] +Output [3]: [promotions#33, total#47, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#47 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#49] +Input [2]: [promotions#33, total#47] (69) Sort [codegen id : 15] -Input [3]: [promotions#33, total#38, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] -Arguments: [promotions#33 ASC NULLS FIRST, total#38 ASC NULLS FIRST], true, 0 +Input [3]: [promotions#33, total#47, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#49] +Arguments: [promotions#33 ASC NULLS FIRST, total#47 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -390,6 +390,6 @@ ReusedExchange (70) (70) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt index c0fad04ce4026..e0bee37e047cb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt @@ -289,112 +289,112 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#32] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#32,17,2) AS promotions#33] (48) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] +Output [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] (50) Filter [codegen id : 13] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_store_sk#36) AND isnotnull(ss_customer_sk#35)) AND isnotnull(ss_item_sk#34)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [s_store_sk#8] +Output [1]: [s_store_sk#39] (52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] +Left keys [1]: [ss_store_sk#36] +Right keys [1]: [s_store_sk#39] Join condition: None (53) Project [codegen id : 13] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] +Output [4]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37, ss_sold_date_sk#38] +Input [6]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38, s_store_sk#39] (54) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#40] (55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#16] +Left keys [1]: [ss_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] Join condition: None (56) Project [codegen id : 13] -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#16] +Output [3]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37] +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37, ss_sold_date_sk#38, d_date_sk#40] (57) ReusedExchange [Reuses operator id: 28] -Output [2]: [c_customer_sk#20, c_current_addr_sk#21] +Output [2]: [c_customer_sk#41, c_current_addr_sk#42] (58) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#20] +Left keys [1]: [ss_customer_sk#35] +Right keys [1]: [c_customer_sk#41] Join condition: None (59) Project [codegen id : 13] -Output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#21] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#20, c_current_addr_sk#21] +Output [3]: [ss_item_sk#34, ss_ext_sales_price#37, c_current_addr_sk#42] +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37, c_customer_sk#41, c_current_addr_sk#42] (60) ReusedExchange [Reuses operator id: 35] -Output [1]: [ca_address_sk#23] +Output [1]: [ca_address_sk#43] (61) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [c_current_addr_sk#21] -Right keys [1]: [ca_address_sk#23] +Left keys [1]: [c_current_addr_sk#42] +Right keys [1]: [ca_address_sk#43] Join condition: None (62) Project [codegen id : 13] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#5] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#21, ca_address_sk#23] +Output [2]: [ss_item_sk#34, ss_ext_sales_price#37] +Input [4]: [ss_item_sk#34, ss_ext_sales_price#37, c_current_addr_sk#42, ca_address_sk#43] (63) ReusedExchange [Reuses operator id: 42] -Output [1]: [i_item_sk#26] +Output [1]: [i_item_sk#44] (64) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#26] +Left keys [1]: [ss_item_sk#34] +Right keys [1]: [i_item_sk#44] Join condition: None (65) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#5] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#26] +Output [1]: [ss_ext_sales_price#37] +Input [3]: [ss_item_sk#34, ss_ext_sales_price#37, i_item_sk#44] (66) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#5] +Input [1]: [ss_ext_sales_price#37] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum#34] -Results [1]: [sum#35] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#37))] +Aggregate Attributes [1]: [sum#45] +Results [1]: [sum#46] (67) Exchange -Input [1]: [sum#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#36] +Input [1]: [sum#46] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#47] (68) HashAggregate [codegen id : 14] -Input [1]: [sum#35] +Input [1]: [sum#46] Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#37] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#37,17,2) AS total#38] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#37))#48] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#37))#48,17,2) AS total#49] (69) BroadcastExchange -Input [1]: [total#38] -Arguments: IdentityBroadcastMode, [id=#39] +Input [1]: [total#49] +Arguments: IdentityBroadcastMode, [id=#50] (70) BroadcastNestedLoopJoin [codegen id : 15] Join condition: None (71) Project [codegen id : 15] -Output [3]: [promotions#33, total#38, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#38 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] -Input [2]: [promotions#33, total#38] +Output [3]: [promotions#33, total#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#49 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#51] +Input [2]: [promotions#33, total#49] (72) Sort [codegen id : 15] -Input [3]: [promotions#33, total#38, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] -Arguments: [promotions#33 ASC NULLS FIRST, total#38 ASC NULLS FIRST], true, 0 +Input [3]: [promotions#33, total#49, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#51] +Arguments: [promotions#33 ASC NULLS FIRST, total#49 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -405,6 +405,6 @@ ReusedExchange (73) (73) ReusedExchange [Reuses operator id: 22] Output [1]: [d_date_sk#16] -Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt index bd8dc07f42967..931e72bba7182 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt @@ -795,353 +795,353 @@ Input [17]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_str Arguments: [item_sk#112 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, store_zip#114 ASC NULLS FIRST], false, 0 (132) Scan parquet default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#129)] +PartitionFilters: [isnotnull(ss_sold_date_sk#140), dynamicpruningexpression(ss_sold_date_sk#140 IN dynamicpruning#141)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (133) ColumnarToRow [codegen id : 44] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] (134) Filter [codegen id : 44] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Condition : (((((((isnotnull(ss_item_sk#129) AND isnotnull(ss_ticket_number#136)) AND isnotnull(ss_store_sk#134)) AND isnotnull(ss_customer_sk#130)) AND isnotnull(ss_cdemo_sk#131)) AND isnotnull(ss_promo_sk#135)) AND isnotnull(ss_hdemo_sk#132)) AND isnotnull(ss_addr_sk#133)) (135) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint), 5), ENSURE_REQUIREMENTS, [id=#130] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Arguments: hashpartitioning(cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint), 5), ENSURE_REQUIREMENTS, [id=#142] (136) Sort [codegen id : 45] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [cast(ss_item_sk#1 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#8 as bigint) ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Arguments: [cast(ss_item_sk#129 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#136 as bigint) ASC NULLS FIRST], false, 0 (137) ReusedExchange [Reuses operator id: 10] -Output [2]: [sr_item_sk#15, sr_ticket_number#16] +Output [2]: [sr_item_sk#143, sr_ticket_number#144] (138) Sort [codegen id : 47] -Input [2]: [sr_item_sk#15, sr_ticket_number#16] -Arguments: [sr_item_sk#15 ASC NULLS FIRST, sr_ticket_number#16 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#143, sr_ticket_number#144] +Arguments: [sr_item_sk#143 ASC NULLS FIRST, sr_ticket_number#144 ASC NULLS FIRST], false, 0 (139) SortMergeJoin [codegen id : 56] -Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] -Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] +Left keys [2]: [cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint)] +Right keys [2]: [sr_item_sk#143, sr_ticket_number#144] Join condition: None (140) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, sr_item_sk#143, sr_ticket_number#144] (141) ReusedExchange [Reuses operator id: 33] -Output [1]: [cs_item_sk#19] +Output [1]: [cs_item_sk#145] (142) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#19] +Left keys [1]: [ss_item_sk#129] +Right keys [1]: [cs_item_sk#145] Join condition: None (143) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, cs_item_sk#145] (144) Scan parquet default.date_dim -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#146, d_year#147] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (145) ColumnarToRow [codegen id : 54] -Input [2]: [d_date_sk#43, d_year#44] +Input [2]: [d_date_sk#146, d_year#147] (146) Filter [codegen id : 54] -Input [2]: [d_date_sk#43, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#43)) +Input [2]: [d_date_sk#146, d_year#147] +Condition : ((isnotnull(d_year#147) AND (d_year#147 = 2000)) AND isnotnull(d_date_sk#146)) (147) BroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#131] +Input [2]: [d_date_sk#146, d_year#147] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#148] (148) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#43] +Left keys [1]: [ss_sold_date_sk#140] +Right keys [1]: [d_date_sk#146] Join condition: None (149) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#43, d_year#44] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147] +Input [13]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, d_date_sk#146, d_year#147] (150) ReusedExchange [Reuses operator id: 45] -Output [3]: [s_store_sk#46, s_store_name#47, s_zip#48] +Output [3]: [s_store_sk#149, s_store_name#150, s_zip#151] (151) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#46] +Left keys [1]: [ss_store_sk#134] +Right keys [1]: [s_store_sk#149] Join condition: None (152) Project [codegen id : 56] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_sk#46, s_store_name#47, s_zip#48] +Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_sk#149, s_store_name#150, s_zip#151] (153) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#132] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Arguments: hashpartitioning(ss_customer_sk#130, 5), ENSURE_REQUIREMENTS, [id=#152] (154) Sort [codegen id : 57] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Arguments: [ss_customer_sk#130 ASC NULLS FIRST], false, 0 (155) ReusedExchange [Reuses operator id: 53] -Output [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Output [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] (156) Sort [codegen id : 59] -Input [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] -Arguments: [c_customer_sk#51 ASC NULLS FIRST], false, 0 +Input [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Arguments: [c_customer_sk#153 ASC NULLS FIRST], false, 0 (157) SortMergeJoin [codegen id : 62] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#51] +Left keys [1]: [ss_customer_sk#130] +Right keys [1]: [c_customer_sk#153] Join condition: None (158) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Input [18]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] (159) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#133, d_year#134] +Output [2]: [d_date_sk#159, d_year#160] (160) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_sales_date_sk#56] -Right keys [1]: [d_date_sk#133] +Left keys [1]: [c_first_sales_date_sk#158] +Right keys [1]: [d_date_sk#159] Join condition: None (161) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56, d_date_sk#133, d_year#134] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158, d_date_sk#159, d_year#160] (162) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#135, d_year#136] +Output [2]: [d_date_sk#161, d_year#162] (163) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_shipto_date_sk#55] -Right keys [1]: [d_date_sk#135] +Left keys [1]: [c_first_shipto_date_sk#157] +Right keys [1]: [d_date_sk#161] Join condition: None (164) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134, d_date_sk#135, d_year#136] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160, d_date_sk#161, d_year#162] (165) Exchange -Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Arguments: hashpartitioning(ss_cdemo_sk#3, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Arguments: hashpartitioning(ss_cdemo_sk#131, 5), ENSURE_REQUIREMENTS, [id=#163] (166) Sort [codegen id : 63] -Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Arguments: [ss_cdemo_sk#3 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Arguments: [ss_cdemo_sk#131 ASC NULLS FIRST], false, 0 (167) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#64, cd_marital_status#65] +Output [2]: [cd_demo_sk#164, cd_marital_status#165] (168) Sort [codegen id : 65] -Input [2]: [cd_demo_sk#64, cd_marital_status#65] -Arguments: [cd_demo_sk#64 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#164, cd_marital_status#165] +Arguments: [cd_demo_sk#164 ASC NULLS FIRST], false, 0 (169) SortMergeJoin [codegen id : 66] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#64] +Left keys [1]: [ss_cdemo_sk#131] +Right keys [1]: [cd_demo_sk#164] Join condition: None (170) Project [codegen id : 66] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_demo_sk#64, cd_marital_status#65] +Output [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_demo_sk#164, cd_marital_status#165] (171) Exchange -Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Arguments: hashpartitioning(c_current_cdemo_sk#52, 5), ENSURE_REQUIREMENTS, [id=#138] +Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Arguments: hashpartitioning(c_current_cdemo_sk#154, 5), ENSURE_REQUIREMENTS, [id=#166] (172) Sort [codegen id : 67] -Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Arguments: [c_current_cdemo_sk#52 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Arguments: [c_current_cdemo_sk#154 ASC NULLS FIRST], false, 0 (173) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#139, cd_marital_status#140] +Output [2]: [cd_demo_sk#167, cd_marital_status#168] (174) Sort [codegen id : 69] -Input [2]: [cd_demo_sk#139, cd_marital_status#140] -Arguments: [cd_demo_sk#139 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#167, cd_marital_status#168] +Arguments: [cd_demo_sk#167 ASC NULLS FIRST], false, 0 (175) SortMergeJoin [codegen id : 73] -Left keys [1]: [c_current_cdemo_sk#52] -Right keys [1]: [cd_demo_sk#139] -Join condition: NOT (cd_marital_status#65 = cd_marital_status#140) +Left keys [1]: [c_current_cdemo_sk#154] +Right keys [1]: [cd_demo_sk#167] +Join condition: NOT (cd_marital_status#165 = cd_marital_status#168) (176) Project [codegen id : 73] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65, cd_demo_sk#139, cd_marital_status#140] +Output [14]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [18]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165, cd_demo_sk#167, cd_marital_status#168] (177) ReusedExchange [Reuses operator id: 84] -Output [1]: [p_promo_sk#70] +Output [1]: [p_promo_sk#169] (178) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#70] +Left keys [1]: [ss_promo_sk#135] +Right keys [1]: [p_promo_sk#169] Join condition: None (179) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, p_promo_sk#70] +Output [13]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, p_promo_sk#169] (180) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#72, hd_income_band_sk#73] +Output [2]: [hd_demo_sk#170, hd_income_band_sk#171] (181) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#72] +Left keys [1]: [ss_hdemo_sk#132] +Right keys [1]: [hd_demo_sk#170] Join condition: None (182) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_demo_sk#72, hd_income_band_sk#73] +Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171] +Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_demo_sk#170, hd_income_band_sk#171] (183) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] +Output [2]: [hd_demo_sk#172, hd_income_band_sk#173] (184) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [c_current_hdemo_sk#53] -Right keys [1]: [hd_demo_sk#141] +Left keys [1]: [c_current_hdemo_sk#155] +Right keys [1]: [hd_demo_sk#172] Join condition: None (185) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_demo_sk#141, hd_income_band_sk#142] +Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Input [15]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_demo_sk#172, hd_income_band_sk#173] (186) Exchange -Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Arguments: hashpartitioning(ss_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#143] +Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Arguments: hashpartitioning(ss_addr_sk#133, 5), ENSURE_REQUIREMENTS, [id=#174] (187) Sort [codegen id : 74] -Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Arguments: [ss_addr_sk#5 ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Arguments: [ss_addr_sk#133 ASC NULLS FIRST], false, 0 (188) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Output [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] (189) Sort [codegen id : 76] -Input [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: [ca_address_sk#78 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: [ca_address_sk#175 ASC NULLS FIRST], false, 0 (190) SortMergeJoin [codegen id : 77] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#78] +Left keys [1]: [ss_addr_sk#133] +Right keys [1]: [ca_address_sk#175] Join condition: None (191) Project [codegen id : 77] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Output [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Input [18]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] (192) Exchange -Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: hashpartitioning(c_current_addr_sk#54, 5), ENSURE_REQUIREMENTS, [id=#144] +Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: hashpartitioning(c_current_addr_sk#156, 5), ENSURE_REQUIREMENTS, [id=#180] (193) Sort [codegen id : 78] -Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: [c_current_addr_sk#54 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: [c_current_addr_sk#156 ASC NULLS FIRST], false, 0 (194) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Output [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] (195) Sort [codegen id : 80] -Input [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Arguments: [ca_address_sk#145 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Arguments: [ca_address_sk#181 ASC NULLS FIRST], false, 0 (196) SortMergeJoin [codegen id : 84] -Left keys [1]: [c_current_addr_sk#54] -Right keys [1]: [ca_address_sk#145] +Left keys [1]: [c_current_addr_sk#156] +Right keys [1]: [ca_address_sk#181] Join condition: None (197) Project [codegen id : 84] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Output [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [21]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] (198) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#90] +Output [1]: [ib_income_band_sk#186] (199) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#73] -Right keys [1]: [ib_income_band_sk#90] +Left keys [1]: [hd_income_band_sk#171] +Right keys [1]: [ib_income_band_sk#186] Join condition: None (200) Project [codegen id : 84] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#90] +Output [18]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [20]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#186] (201) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#150] +Output [1]: [ib_income_band_sk#187] (202) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#142] -Right keys [1]: [ib_income_band_sk#150] +Left keys [1]: [hd_income_band_sk#173] +Right keys [1]: [ib_income_band_sk#187] Join condition: None (203) Project [codegen id : 84] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#150] +Output [17]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#187] (204) ReusedExchange [Reuses operator id: 124] -Output [2]: [i_item_sk#93, i_product_name#96] +Output [2]: [i_item_sk#188, i_product_name#189] (205) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#93] +Left keys [1]: [ss_item_sk#129] +Right keys [1]: [i_item_sk#188] Join condition: None (206) Project [codegen id : 84] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] +Output [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] +Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] (207) HashAggregate [codegen id : 84] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] -Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#151, sum#152, sum#153, sum#154] -Results [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] +Input [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] +Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#137)), partial_sum(UnscaledValue(ss_list_price#138)), partial_sum(UnscaledValue(ss_coupon_amt#139))] +Aggregate Attributes [4]: [count#190, sum#191, sum#192, sum#193] +Results [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] (208) Exchange -Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] -Arguments: hashpartitioning(i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, 5), ENSURE_REQUIREMENTS, [id=#159] +Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] +Arguments: hashpartitioning(i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, 5), ENSURE_REQUIREMENTS, [id=#198] (209) HashAggregate [codegen id : 85] -Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] -Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#160, sum(UnscaledValue(ss_wholesale_cost#9))#161, sum(UnscaledValue(ss_list_price#10))#162, sum(UnscaledValue(ss_coupon_amt#11))#163] -Results [8]: [i_item_sk#93 AS item_sk#164, s_store_name#47 AS store_name#165, s_zip#48 AS store_zip#166, d_year#44 AS syear#167, count(1)#160 AS cnt#168, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#161,17,2) AS s1#169, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#162,17,2) AS s2#170, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#163,17,2) AS s3#171] +Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] +Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#137)), sum(UnscaledValue(ss_list_price#138)), sum(UnscaledValue(ss_coupon_amt#139))] +Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#137))#200, sum(UnscaledValue(ss_list_price#138))#201, sum(UnscaledValue(ss_coupon_amt#139))#202] +Results [8]: [i_item_sk#188 AS item_sk#203, s_store_name#150 AS store_name#204, s_zip#151 AS store_zip#205, d_year#147 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#137))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#138))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#139))#202,17,2) AS s3#210] (210) Exchange -Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] -Arguments: hashpartitioning(item_sk#164, store_name#165, store_zip#166, 5), ENSURE_REQUIREMENTS, [id=#172] +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] (211) Sort [codegen id : 86] -Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] -Arguments: [item_sk#164 ASC NULLS FIRST, store_name#165 ASC NULLS FIRST, store_zip#166 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 (212) SortMergeJoin [codegen id : 87] Left keys [3]: [item_sk#112, store_name#113, store_zip#114] -Right keys [3]: [item_sk#164, store_name#165, store_zip#166] -Join condition: (cnt#168 <= cnt#124) +Right keys [3]: [item_sk#203, store_name#204, store_zip#205] +Join condition: (cnt#207 <= cnt#124) (213) Project [codegen id : 87] -Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] +Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] (214) Exchange -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#173] +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] (215) Sort [codegen id : 88] -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1152,11 +1152,11 @@ ReusedExchange (216) (216) ReusedExchange [Reuses operator id: 39] Output [2]: [d_date_sk#43, d_year#44] -Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#129 +Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#140 IN dynamicpruning#141 ReusedExchange (217) (217) ReusedExchange [Reuses operator id: 147] -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#146, d_year#147] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index 216e7f5530128..a866c8aa9b486 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt @@ -701,323 +701,323 @@ Input [17]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_str Arguments: [item_sk#105 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, store_zip#107 ASC NULLS FIRST], false, 0 (115) Scan parquet default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#122)] +PartitionFilters: [isnotnull(ss_sold_date_sk#133), dynamicpruningexpression(ss_sold_date_sk#133 IN dynamicpruning#134)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (116) ColumnarToRow [codegen id : 27] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] (117) Filter [codegen id : 27] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Condition : (((((((isnotnull(ss_item_sk#122) AND isnotnull(ss_ticket_number#129)) AND isnotnull(ss_store_sk#127)) AND isnotnull(ss_customer_sk#123)) AND isnotnull(ss_cdemo_sk#124)) AND isnotnull(ss_promo_sk#128)) AND isnotnull(ss_hdemo_sk#125)) AND isnotnull(ss_addr_sk#126)) (118) BroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#123] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#135] (119) Scan parquet default.store_returns -Output [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Output [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct (120) ColumnarToRow -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] (121) Filter -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] -Condition : (isnotnull(sr_item_sk#15) AND isnotnull(sr_ticket_number#16)) +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Condition : (isnotnull(sr_item_sk#136) AND isnotnull(sr_ticket_number#137)) (122) Project -Output [2]: [sr_item_sk#15, sr_ticket_number#16] -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Output [2]: [sr_item_sk#136, sr_ticket_number#137] +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] (123) BroadcastHashJoin [codegen id : 28] -Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] -Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] +Left keys [2]: [cast(ss_item_sk#122 as bigint), cast(ss_ticket_number#129 as bigint)] +Right keys [2]: [sr_item_sk#136, sr_ticket_number#137] Join condition: None (124) Project [codegen id : 28] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, sr_item_sk#136, sr_ticket_number#137] (125) Exchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: hashpartitioning(ss_item_sk#122, 5), ENSURE_REQUIREMENTS, [id=#139] (126) Sort [codegen id : 29] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: [ss_item_sk#122 ASC NULLS FIRST], false, 0 (127) ReusedExchange [Reuses operator id: 28] -Output [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] +Output [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] (128) HashAggregate [codegen id : 35] -Input [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] -Keys [1]: [cs_item_sk#19] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#21)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#21))#128, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129] -Results [3]: [cs_item_sk#19, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#21))#128,17,2) AS sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] +Input [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] +Keys [1]: [cs_item_sk#140] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#144)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#144))#148, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149] +Results [3]: [cs_item_sk#140, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#144))#148,17,2) AS sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] (129) Filter [codegen id : 35] -Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] -Condition : (isnotnull(sum(cs_ext_list_price#21)#130) AND (cast(sum(cs_ext_list_price#21)#130 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131)), DecimalType(21,2), true))) +Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] +Condition : (isnotnull(sum(cs_ext_list_price#144)#150) AND (cast(sum(cs_ext_list_price#144)#150 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151)), DecimalType(21,2), true))) (130) Project [codegen id : 35] -Output [1]: [cs_item_sk#19] -Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] +Output [1]: [cs_item_sk#140] +Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] (131) Sort [codegen id : 35] -Input [1]: [cs_item_sk#19] -Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 +Input [1]: [cs_item_sk#140] +Arguments: [cs_item_sk#140 ASC NULLS FIRST], false, 0 (132) SortMergeJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#19] +Left keys [1]: [ss_item_sk#122] +Right keys [1]: [cs_item_sk#140] Join condition: None (133) Project [codegen id : 51] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, cs_item_sk#140] (134) Scan parquet default.date_dim -Output [2]: [d_date_sk#42, d_year#43] +Output [2]: [d_date_sk#152, d_year#153] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (135) ColumnarToRow [codegen id : 36] -Input [2]: [d_date_sk#42, d_year#43] +Input [2]: [d_date_sk#152, d_year#153] (136) Filter [codegen id : 36] -Input [2]: [d_date_sk#42, d_year#43] -Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2000)) AND isnotnull(d_date_sk#42)) +Input [2]: [d_date_sk#152, d_year#153] +Condition : ((isnotnull(d_year#153) AND (d_year#153 = 2000)) AND isnotnull(d_date_sk#152)) (137) BroadcastExchange -Input [2]: [d_date_sk#42, d_year#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#132] +Input [2]: [d_date_sk#152, d_year#153] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#154] (138) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#42] +Left keys [1]: [ss_sold_date_sk#133] +Right keys [1]: [d_date_sk#152] Join condition: None (139) Project [codegen id : 51] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#42, d_year#43] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153] +Input [13]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, d_date_sk#152, d_year#153] (140) ReusedExchange [Reuses operator id: 44] -Output [3]: [s_store_sk#45, s_store_name#46, s_zip#47] +Output [3]: [s_store_sk#155, s_store_name#156, s_zip#157] (141) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#45] +Left keys [1]: [ss_store_sk#127] +Right keys [1]: [s_store_sk#155] Join condition: None (142) Project [codegen id : 51] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_sk#45, s_store_name#46, s_zip#47] +Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157] +Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_sk#155, s_store_name#156, s_zip#157] (143) ReusedExchange [Reuses operator id: 50] -Output [6]: [c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] +Output [6]: [c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] (144) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#49] +Left keys [1]: [ss_customer_sk#123] +Right keys [1]: [c_customer_sk#158] Join condition: None (145) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] +Input [18]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] (146) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#133, d_year#134] +Output [2]: [d_date_sk#164, d_year#165] (147) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_sales_date_sk#54] -Right keys [1]: [d_date_sk#133] +Left keys [1]: [c_first_sales_date_sk#163] +Right keys [1]: [d_date_sk#164] Join condition: None (148) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54, d_date_sk#133, d_year#134] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163, d_date_sk#164, d_year#165] (149) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#135, d_year#136] +Output [2]: [d_date_sk#166, d_year#167] (150) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_shipto_date_sk#53] -Right keys [1]: [d_date_sk#135] +Left keys [1]: [c_first_shipto_date_sk#162] +Right keys [1]: [d_date_sk#166] Join condition: None (151) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134, d_date_sk#135, d_year#136] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165, d_date_sk#166, d_year#167] (152) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#61, cd_marital_status#62] +Output [2]: [cd_demo_sk#168, cd_marital_status#169] (153) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#61] +Left keys [1]: [ss_cdemo_sk#124] +Right keys [1]: [cd_demo_sk#168] Join condition: None (154) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_demo_sk#61, cd_marital_status#62] +Output [16]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_demo_sk#168, cd_marital_status#169] (155) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#137, cd_marital_status#138] +Output [2]: [cd_demo_sk#170, cd_marital_status#171] (156) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_cdemo_sk#50] -Right keys [1]: [cd_demo_sk#137] -Join condition: NOT (cd_marital_status#62 = cd_marital_status#138) +Left keys [1]: [c_current_cdemo_sk#159] +Right keys [1]: [cd_demo_sk#170] +Join condition: NOT (cd_marital_status#169 = cd_marital_status#171) (157) Project [codegen id : 51] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62, cd_demo_sk#137, cd_marital_status#138] +Output [14]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [18]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169, cd_demo_sk#170, cd_marital_status#171] (158) ReusedExchange [Reuses operator id: 74] -Output [1]: [p_promo_sk#66] +Output [1]: [p_promo_sk#172] (159) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#66] +Left keys [1]: [ss_promo_sk#128] +Right keys [1]: [p_promo_sk#172] Join condition: None (160) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, p_promo_sk#66] +Output [13]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, p_promo_sk#172] (161) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#68, hd_income_band_sk#69] +Output [2]: [hd_demo_sk#173, hd_income_band_sk#174] (162) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#68] +Left keys [1]: [ss_hdemo_sk#125] +Right keys [1]: [hd_demo_sk#173] Join condition: None (163) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_demo_sk#68, hd_income_band_sk#69] +Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174] +Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_demo_sk#173, hd_income_band_sk#174] (164) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] +Output [2]: [hd_demo_sk#175, hd_income_band_sk#176] (165) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_hdemo_sk#51] -Right keys [1]: [hd_demo_sk#139] +Left keys [1]: [c_current_hdemo_sk#160] +Right keys [1]: [hd_demo_sk#175] Join condition: None (166) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_demo_sk#139, hd_income_band_sk#140] +Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176] +Input [15]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_demo_sk#175, hd_income_band_sk#176] (167) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] +Output [5]: [ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] (168) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#73] +Left keys [1]: [ss_addr_sk#126] +Right keys [1]: [ca_address_sk#177] Join condition: None (169) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] +Output [16]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] +Input [18]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] (170) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Output [5]: [ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] (171) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_addr_sk#52] -Right keys [1]: [ca_address_sk#141] +Left keys [1]: [c_current_addr_sk#161] +Right keys [1]: [ca_address_sk#182] Join condition: None (172) Project [codegen id : 51] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Output [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [21]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] (173) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#84] +Output [1]: [ib_income_band_sk#187] (174) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#69] -Right keys [1]: [ib_income_band_sk#84] +Left keys [1]: [hd_income_band_sk#174] +Right keys [1]: [ib_income_band_sk#187] Join condition: None (175) Project [codegen id : 51] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#84] +Output [18]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [20]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#187] (176) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#146] +Output [1]: [ib_income_band_sk#188] (177) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#140] -Right keys [1]: [ib_income_band_sk#146] +Left keys [1]: [hd_income_band_sk#176] +Right keys [1]: [ib_income_band_sk#188] Join condition: None (178) Project [codegen id : 51] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#146] +Output [17]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#188] (179) ReusedExchange [Reuses operator id: 108] -Output [2]: [i_item_sk#87, i_product_name#90] +Output [2]: [i_item_sk#189, i_product_name#190] (180) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#87] +Left keys [1]: [ss_item_sk#122] +Right keys [1]: [i_item_sk#189] Join condition: None (181) Project [codegen id : 51] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] +Output [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] (182) HashAggregate [codegen id : 51] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] -Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#147, sum#148, sum#149, sum#150] -Results [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] +Input [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#130)), partial_sum(UnscaledValue(ss_list_price#131)), partial_sum(UnscaledValue(ss_coupon_amt#132))] +Aggregate Attributes [4]: [count#191, sum#192, sum#193, sum#194] +Results [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] (183) HashAggregate [codegen id : 51] -Input [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#155, sum(UnscaledValue(ss_wholesale_cost#9))#156, sum(UnscaledValue(ss_list_price#10))#157, sum(UnscaledValue(ss_coupon_amt#11))#158] -Results [8]: [i_item_sk#87 AS item_sk#159, s_store_name#46 AS store_name#160, s_zip#47 AS store_zip#161, d_year#43 AS syear#162, count(1)#155 AS cnt#163, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#156,17,2) AS s1#164, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#157,17,2) AS s2#165, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#158,17,2) AS s3#166] +Input [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] +Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#130)), sum(UnscaledValue(ss_list_price#131)), sum(UnscaledValue(ss_coupon_amt#132))] +Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#130))#200, sum(UnscaledValue(ss_list_price#131))#201, sum(UnscaledValue(ss_coupon_amt#132))#202] +Results [8]: [i_item_sk#189 AS item_sk#203, s_store_name#156 AS store_name#204, s_zip#157 AS store_zip#205, d_year#153 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#130))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#131))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#132))#202,17,2) AS s3#210] (184) Exchange -Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] -Arguments: hashpartitioning(item_sk#159, store_name#160, store_zip#161, 5), ENSURE_REQUIREMENTS, [id=#167] +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] (185) Sort [codegen id : 52] -Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] -Arguments: [item_sk#159 ASC NULLS FIRST, store_name#160 ASC NULLS FIRST, store_zip#161 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 (186) SortMergeJoin [codegen id : 53] Left keys [3]: [item_sk#105, store_name#106, store_zip#107] -Right keys [3]: [item_sk#159, store_name#160, store_zip#161] -Join condition: (cnt#163 <= cnt#117) +Right keys [3]: [item_sk#203, store_name#204, store_zip#205] +Join condition: (cnt#207 <= cnt#117) (187) Project [codegen id : 53] -Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] +Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] (188) Exchange -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#168] +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] (189) Sort [codegen id : 54] -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1028,11 +1028,11 @@ ReusedExchange (190) (190) ReusedExchange [Reuses operator id: 38] Output [2]: [d_date_sk#42, d_year#43] -Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#122 +Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#133 IN dynamicpruning#134 ReusedExchange (191) (191) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#42, d_year#43] +Output [2]: [d_date_sk#152, d_year#153] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt index 41c2ffcc75e7a..84e49ab9373e4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt @@ -130,135 +130,135 @@ Input [4]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17 Condition : isnotnull(ss_store_sk#15) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#18] (19) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#17] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#18] Join condition: None (20) Project [codegen id : 4] Output [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] -Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#6] +Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#18] (21) HashAggregate [codegen id : 4] Input [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] +Aggregate Attributes [1]: [sum#19] +Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] (22) Exchange -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] -Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] +Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#21] (23) HashAggregate [codegen id : 5] -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#21] -Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#21,17,2) AS revenue#22] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#22] +Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#22,17,2) AS revenue#23] (24) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#15, revenue#22] +Input [2]: [ss_store_sk#15, revenue#23] Keys [1]: [ss_store_sk#15] -Functions [1]: [partial_avg(revenue#22)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ss_store_sk#15, sum#25, count#26] +Functions [1]: [partial_avg(revenue#23)] +Aggregate Attributes [2]: [sum#24, count#25] +Results [3]: [ss_store_sk#15, sum#26, count#27] (25) Exchange -Input [3]: [ss_store_sk#15, sum#25, count#26] -Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ss_store_sk#15, sum#26, count#27] +Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#28] (26) HashAggregate [codegen id : 6] -Input [3]: [ss_store_sk#15, sum#25, count#26] +Input [3]: [ss_store_sk#15, sum#26, count#27] Keys [1]: [ss_store_sk#15] -Functions [1]: [avg(revenue#22)] -Aggregate Attributes [1]: [avg(revenue#22)#28] -Results [2]: [ss_store_sk#15, avg(revenue#22)#28 AS ave#29] +Functions [1]: [avg(revenue#23)] +Aggregate Attributes [1]: [avg(revenue#23)#29] +Results [2]: [ss_store_sk#15, avg(revenue#23)#29 AS ave#30] (27) BroadcastExchange -Input [2]: [ss_store_sk#15, ave#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] +Input [2]: [ss_store_sk#15, ave#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (28) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] Right keys [1]: [ss_store_sk#15] -Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#29)), DecimalType(23,7), true)) +Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#30)), DecimalType(23,7), true)) (29) Project [codegen id : 8] Output [3]: [ss_store_sk#2, ss_item_sk#1, revenue#13] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#29] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#30] (30) Scan parquet default.store -Output [2]: [s_store_sk#31, s_store_name#32] +Output [2]: [s_store_sk#32, s_store_name#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#31, s_store_name#32] +Input [2]: [s_store_sk#32, s_store_name#33] (32) Filter [codegen id : 7] -Input [2]: [s_store_sk#31, s_store_name#32] -Condition : isnotnull(s_store_sk#31) +Input [2]: [s_store_sk#32, s_store_name#33] +Condition : isnotnull(s_store_sk#32) (33) BroadcastExchange -Input [2]: [s_store_sk#31, s_store_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +Input [2]: [s_store_sk#32, s_store_name#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#31] +Right keys [1]: [s_store_sk#32] Join condition: None (35) Project [codegen id : 8] -Output [3]: [ss_item_sk#1, revenue#13, s_store_name#32] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#31, s_store_name#32] +Output [3]: [ss_item_sk#1, revenue#13, s_store_name#33] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#32, s_store_name#33] (36) Exchange -Input [3]: [ss_item_sk#1, revenue#13, s_store_name#32] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [ss_item_sk#1, revenue#13, s_store_name#33] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#35] (37) Sort [codegen id : 9] -Input [3]: [ss_item_sk#1, revenue#13, s_store_name#32] +Input [3]: [ss_item_sk#1, revenue#13, s_store_name#33] Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (38) Scan parquet default.item -Output [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Output [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 10] -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] (40) Filter [codegen id : 10] -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Condition : isnotnull(i_item_sk#35) +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Condition : isnotnull(i_item_sk#36) (41) Exchange -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Arguments: hashpartitioning(i_item_sk#35, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Arguments: hashpartitioning(i_item_sk#36, 5), ENSURE_REQUIREMENTS, [id=#41] (42) Sort [codegen id : 11] -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Arguments: [i_item_sk#35 ASC NULLS FIRST], false, 0 +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Arguments: [i_item_sk#36 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 12] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#35] +Right keys [1]: [i_item_sk#36] Join condition: None (44) Project [codegen id : 12] -Output [6]: [s_store_name#32, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Input [8]: [ss_item_sk#1, revenue#13, s_store_name#32, i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Output [6]: [s_store_name#33, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Input [8]: [ss_item_sk#1, revenue#13, s_store_name#33, i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] (45) TakeOrderedAndProject -Input [6]: [s_store_name#32, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Arguments: 100, [s_store_name#32 ASC NULLS FIRST, i_item_desc#36 ASC NULLS FIRST], [s_store_name#32, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [6]: [s_store_name#33, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Arguments: 100, [s_store_name#33 ASC NULLS FIRST, i_item_desc#37 ASC NULLS FIRST], [s_store_name#33, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt index 52de9873db590..45c7c051601c5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt @@ -181,65 +181,65 @@ Input [4]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26 Condition : isnotnull(ss_store_sk#24) (30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#27] (31) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#27] Join condition: None (32) Project [codegen id : 6] Output [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] -Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#8] +Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#27] (33) HashAggregate [codegen id : 6] Input [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#27] -Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] +Aggregate Attributes [1]: [sum#28] +Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] (34) Exchange -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] -Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] +Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#30] (35) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#30] -Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#30,17,2) AS revenue#31] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] +Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS revenue#32] (36) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#24, revenue#31] +Input [2]: [ss_store_sk#24, revenue#32] Keys [1]: [ss_store_sk#24] -Functions [1]: [partial_avg(revenue#31)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [3]: [ss_store_sk#24, sum#34, count#35] +Functions [1]: [partial_avg(revenue#32)] +Aggregate Attributes [2]: [sum#33, count#34] +Results [3]: [ss_store_sk#24, sum#35, count#36] (37) Exchange -Input [3]: [ss_store_sk#24, sum#34, count#35] -Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ss_store_sk#24, sum#35, count#36] +Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#37] (38) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#24, sum#34, count#35] +Input [3]: [ss_store_sk#24, sum#35, count#36] Keys [1]: [ss_store_sk#24] -Functions [1]: [avg(revenue#31)] -Aggregate Attributes [1]: [avg(revenue#31)#37] -Results [2]: [ss_store_sk#24, avg(revenue#31)#37 AS ave#38] +Functions [1]: [avg(revenue#32)] +Aggregate Attributes [1]: [avg(revenue#32)#38] +Results [2]: [ss_store_sk#24, avg(revenue#32)#38 AS ave#39] (39) BroadcastExchange -Input [2]: [ss_store_sk#24, ave#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Input [2]: [ss_store_sk#24, ave#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#24] -Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#38)), DecimalType(23,7), true)) +Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#39)), DecimalType(23,7), true)) (41) Project [codegen id : 9] Output [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#38] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#39] (42) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt index 51298b80bbbbe..8c342961cf970 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt @@ -221,70 +221,70 @@ Input [7]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_qu Condition : ((isnotnull(cs_warehouse_sk#176) AND isnotnull(cs_sold_time_sk#174)) AND isnotnull(cs_ship_mode_sk#175)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [sm_ship_mode_sk#9] +Output [1]: [sm_ship_mode_sk#181] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_mode_sk#175] -Right keys [1]: [sm_ship_mode_sk#9] +Right keys [1]: [sm_ship_mode_sk#181] Join condition: None (38) Project [codegen id : 11] Output [6]: [cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180] -Input [8]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, sm_ship_mode_sk#9] +Input [8]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, sm_ship_mode_sk#181] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [t_time_sk#12] +Output [1]: [t_time_sk#182] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_time_sk#174] -Right keys [1]: [t_time_sk#12] +Right keys [1]: [t_time_sk#182] Join condition: None (41) Project [codegen id : 11] Output [5]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180] -Input [7]: [cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, t_time_sk#12] +Input [7]: [cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, t_time_sk#182] (42) ReusedExchange [Reuses operator id: 21] -Output [3]: [d_date_sk#15, d_year#16, d_moy#17] +Output [3]: [d_date_sk#183, d_year#184, d_moy#185] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#180] -Right keys [1]: [d_date_sk#15] +Right keys [1]: [d_date_sk#183] Join condition: None (44) Project [codegen id : 11] -Output [6]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#16, d_moy#17] -Input [8]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, d_date_sk#15, d_year#16, d_moy#17] +Output [6]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#184, d_moy#185] +Input [8]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, d_date_sk#183, d_year#184, d_moy#185] (45) ReusedExchange [Reuses operator id: 27] -Output [7]: [w_warehouse_sk#19, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25] +Output [7]: [w_warehouse_sk#186, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192] (46) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_warehouse_sk#176] -Right keys [1]: [w_warehouse_sk#19] +Right keys [1]: [w_warehouse_sk#186] Join condition: None (47) Project [codegen id : 11] -Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, d_moy#17] -Input [13]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#16, d_moy#17, w_warehouse_sk#19, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25] +Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, d_moy#185] +Input [13]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#184, d_moy#185, w_warehouse_sk#186, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192] (48) HashAggregate [codegen id : 11] -Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, d_moy#17] -Keys [7]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16] -Functions [24]: [partial_sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228] -Results [55]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] +Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, d_moy#185] +Keys [7]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184] +Functions [24]: [partial_sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [48]: [sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240] +Results [55]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] (49) Exchange -Input [55]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] -Arguments: hashpartitioning(w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#277] +Input [55]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] +Arguments: hashpartitioning(w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, 5), ENSURE_REQUIREMENTS, [id=#289] (50) HashAggregate [codegen id : 12] -Input [55]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] -Keys [7]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16] -Functions [24]: [sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301] -Results [32]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, DHL,BARIAN AS ship_carriers#302, d_year#16 AS year#303, sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278 AS jan_sales#304, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279 AS feb_sales#305, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280 AS mar_sales#306, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281 AS apr_sales#307, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282 AS may_sales#308, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283 AS jun_sales#309, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284 AS jul_sales#310, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285 AS aug_sales#311, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286 AS sep_sales#312, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287 AS oct_sales#313, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288 AS nov_sales#314, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289 AS dec_sales#315, sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_net#316, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_net#317, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_net#318, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_net#319, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_net#320, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_net#321, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_net#322, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_net#323, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_net#324, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_net#325, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_net#326, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_net#327] +Input [55]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] +Keys [7]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184] +Functions [24]: [sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313] +Results [32]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, DHL,BARIAN AS ship_carriers#314, d_year#184 AS year#315, sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_sales#316, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_sales#317, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_sales#318, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_sales#319, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_sales#320, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_sales#321, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_sales#322, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_sales#323, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_sales#324, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_sales#325, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_sales#326, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_sales#327, sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302 AS jan_net#328, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303 AS feb_net#329, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304 AS mar_net#330, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305 AS apr_net#331, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306 AS may_net#332, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307 AS jun_net#333, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308 AS jul_net#334, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309 AS aug_net#335, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310 AS sep_net#336, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311 AS oct_net#337, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312 AS nov_net#338, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313 AS dec_net#339] (51) Union @@ -292,23 +292,23 @@ Results [32]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23 Input [32]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#150, feb_sales#151, mar_sales#152, apr_sales#153, may_sales#154, jun_sales#155, jul_sales#156, aug_sales#157, sep_sales#158, oct_sales#159, nov_sales#160, dec_sales#161, jan_net#162, feb_net#163, mar_net#164, apr_net#165, may_net#166, jun_net#167, jul_net#168, aug_net#169, sep_net#170, oct_net#171, nov_net#172, dec_net#173] Keys [8]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149] Functions [36]: [partial_sum(jan_sales#150), partial_sum(feb_sales#151), partial_sum(mar_sales#152), partial_sum(apr_sales#153), partial_sum(may_sales#154), partial_sum(jun_sales#155), partial_sum(jul_sales#156), partial_sum(aug_sales#157), partial_sum(sep_sales#158), partial_sum(oct_sales#159), partial_sum(nov_sales#160), partial_sum(dec_sales#161), partial_sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(jan_net#162), partial_sum(feb_net#163), partial_sum(mar_net#164), partial_sum(apr_net#165), partial_sum(may_net#166), partial_sum(jun_net#167), partial_sum(jul_net#168), partial_sum(aug_net#169), partial_sum(sep_net#170), partial_sum(oct_net#171), partial_sum(nov_net#172), partial_sum(dec_net#173)] -Aggregate Attributes [72]: [sum#328, isEmpty#329, sum#330, isEmpty#331, sum#332, isEmpty#333, sum#334, isEmpty#335, sum#336, isEmpty#337, sum#338, isEmpty#339, sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399] -Results [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] +Aggregate Attributes [72]: [sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411] +Results [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] (53) Exchange -Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] -Arguments: hashpartitioning(w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#472] +Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] +Arguments: hashpartitioning(w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#484] (54) HashAggregate [codegen id : 14] -Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] +Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] Keys [8]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149] Functions [36]: [sum(jan_sales#150), sum(feb_sales#151), sum(mar_sales#152), sum(apr_sales#153), sum(may_sales#154), sum(jun_sales#155), sum(jul_sales#156), sum(aug_sales#157), sum(sep_sales#158), sum(oct_sales#159), sum(nov_sales#160), sum(dec_sales#161), sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(jan_net#162), sum(feb_net#163), sum(mar_net#164), sum(apr_net#165), sum(may_net#166), sum(jun_net#167), sum(jul_net#168), sum(aug_net#169), sum(sep_net#170), sum(oct_net#171), sum(nov_net#172), sum(dec_net#173)] -Aggregate Attributes [36]: [sum(jan_sales#150)#473, sum(feb_sales#151)#474, sum(mar_sales#152)#475, sum(apr_sales#153)#476, sum(may_sales#154)#477, sum(jun_sales#155)#478, sum(jul_sales#156)#479, sum(aug_sales#157)#480, sum(sep_sales#158)#481, sum(oct_sales#159)#482, sum(nov_sales#160)#483, sum(dec_sales#161)#484, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496, sum(jan_net#162)#497, sum(feb_net#163)#498, sum(mar_net#164)#499, sum(apr_net#165)#500, sum(may_net#166)#501, sum(jun_net#167)#502, sum(jul_net#168)#503, sum(aug_net#169)#504, sum(sep_net#170)#505, sum(oct_net#171)#506, sum(nov_net#172)#507, sum(dec_net#173)#508] -Results [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum(jan_sales#150)#473 AS jan_sales#509, sum(feb_sales#151)#474 AS feb_sales#510, sum(mar_sales#152)#475 AS mar_sales#511, sum(apr_sales#153)#476 AS apr_sales#512, sum(may_sales#154)#477 AS may_sales#513, sum(jun_sales#155)#478 AS jun_sales#514, sum(jul_sales#156)#479 AS jul_sales#515, sum(aug_sales#157)#480 AS aug_sales#516, sum(sep_sales#158)#481 AS sep_sales#517, sum(oct_sales#159)#482 AS oct_sales#518, sum(nov_sales#160)#483 AS nov_sales#519, sum(dec_sales#161)#484 AS dec_sales#520, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485 AS jan_sales_per_sq_foot#521, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486 AS feb_sales_per_sq_foot#522, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487 AS mar_sales_per_sq_foot#523, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488 AS apr_sales_per_sq_foot#524, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489 AS may_sales_per_sq_foot#525, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490 AS jun_sales_per_sq_foot#526, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491 AS jul_sales_per_sq_foot#527, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492 AS aug_sales_per_sq_foot#528, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493 AS sep_sales_per_sq_foot#529, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494 AS oct_sales_per_sq_foot#530, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495 AS nov_sales_per_sq_foot#531, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496 AS dec_sales_per_sq_foot#532, sum(jan_net#162)#497 AS jan_net#533, sum(feb_net#163)#498 AS feb_net#534, sum(mar_net#164)#499 AS mar_net#535, sum(apr_net#165)#500 AS apr_net#536, sum(may_net#166)#501 AS may_net#537, sum(jun_net#167)#502 AS jun_net#538, sum(jul_net#168)#503 AS jul_net#539, sum(aug_net#169)#504 AS aug_net#540, sum(sep_net#170)#505 AS sep_net#541, sum(oct_net#171)#506 AS oct_net#542, sum(nov_net#172)#507 AS nov_net#543, sum(dec_net#173)#508 AS dec_net#544] +Aggregate Attributes [36]: [sum(jan_sales#150)#485, sum(feb_sales#151)#486, sum(mar_sales#152)#487, sum(apr_sales#153)#488, sum(may_sales#154)#489, sum(jun_sales#155)#490, sum(jul_sales#156)#491, sum(aug_sales#157)#492, sum(sep_sales#158)#493, sum(oct_sales#159)#494, sum(nov_sales#160)#495, sum(dec_sales#161)#496, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508, sum(jan_net#162)#509, sum(feb_net#163)#510, sum(mar_net#164)#511, sum(apr_net#165)#512, sum(may_net#166)#513, sum(jun_net#167)#514, sum(jul_net#168)#515, sum(aug_net#169)#516, sum(sep_net#170)#517, sum(oct_net#171)#518, sum(nov_net#172)#519, sum(dec_net#173)#520] +Results [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum(jan_sales#150)#485 AS jan_sales#521, sum(feb_sales#151)#486 AS feb_sales#522, sum(mar_sales#152)#487 AS mar_sales#523, sum(apr_sales#153)#488 AS apr_sales#524, sum(may_sales#154)#489 AS may_sales#525, sum(jun_sales#155)#490 AS jun_sales#526, sum(jul_sales#156)#491 AS jul_sales#527, sum(aug_sales#157)#492 AS aug_sales#528, sum(sep_sales#158)#493 AS sep_sales#529, sum(oct_sales#159)#494 AS oct_sales#530, sum(nov_sales#160)#495 AS nov_sales#531, sum(dec_sales#161)#496 AS dec_sales#532, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497 AS jan_sales_per_sq_foot#533, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498 AS feb_sales_per_sq_foot#534, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499 AS mar_sales_per_sq_foot#535, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500 AS apr_sales_per_sq_foot#536, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501 AS may_sales_per_sq_foot#537, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502 AS jun_sales_per_sq_foot#538, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503 AS jul_sales_per_sq_foot#539, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504 AS aug_sales_per_sq_foot#540, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505 AS sep_sales_per_sq_foot#541, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506 AS oct_sales_per_sq_foot#542, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507 AS nov_sales_per_sq_foot#543, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508 AS dec_sales_per_sq_foot#544, sum(jan_net#162)#509 AS jan_net#545, sum(feb_net#163)#510 AS feb_net#546, sum(mar_net#164)#511 AS mar_net#547, sum(apr_net#165)#512 AS apr_net#548, sum(may_net#166)#513 AS may_net#549, sum(jun_net#167)#514 AS jun_net#550, sum(jul_net#168)#515 AS jul_net#551, sum(aug_net#169)#516 AS aug_net#552, sum(sep_net#170)#517 AS sep_net#553, sum(oct_net#171)#518 AS oct_net#554, sum(nov_net#172)#519 AS nov_net#555, sum(dec_net#173)#520 AS dec_net#556] (55) TakeOrderedAndProject -Input [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, may_sales_per_sq_foot#525, jun_sales_per_sq_foot#526, jul_sales_per_sq_foot#527, aug_sales_per_sq_foot#528, sep_sales_per_sq_foot#529, oct_sales_per_sq_foot#530, nov_sales_per_sq_foot#531, dec_sales_per_sq_foot#532, jan_net#533, feb_net#534, mar_net#535, apr_net#536, may_net#537, jun_net#538, jul_net#539, aug_net#540, sep_net#541, oct_net#542, nov_net#543, dec_net#544] -Arguments: 100, [w_warehouse_name#20 ASC NULLS FIRST], [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, ... 20 more fields] +Input [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, may_sales_per_sq_foot#537, jun_sales_per_sq_foot#538, jul_sales_per_sq_foot#539, aug_sales_per_sq_foot#540, sep_sales_per_sq_foot#541, oct_sales_per_sq_foot#542, nov_sales_per_sq_foot#543, dec_sales_per_sq_foot#544, jan_net#545, feb_net#546, mar_net#547, apr_net#548, may_net#549, jun_net#550, jul_net#551, aug_net#552, sep_net#553, oct_net#554, nov_net#555, dec_net#556] +Arguments: 100, [w_warehouse_name#20 ASC NULLS FIRST], [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, ... 20 more fields] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt index 3d44b22396486..832965c1aaa31 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt @@ -221,70 +221,70 @@ Input [7]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_qu Condition : ((isnotnull(cs_warehouse_sk#176) AND isnotnull(cs_sold_time_sk#174)) AND isnotnull(cs_ship_mode_sk#175)) (36) ReusedExchange [Reuses operator id: 7] -Output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Output [7]: [w_warehouse_sk#181, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_warehouse_sk#176] -Right keys [1]: [w_warehouse_sk#9] +Right keys [1]: [w_warehouse_sk#181] Join condition: None (38) Project [codegen id : 11] -Output [12]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Output [12]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187] +Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_sk#181, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187] (39) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] +Output [3]: [d_date_sk#188, d_year#189, d_moy#190] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#180] -Right keys [1]: [d_date_sk#17] +Right keys [1]: [d_date_sk#188] Join condition: None (41) Project [codegen id : 11] -Output [13]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] -Input [15]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_date_sk#17, d_year#18, d_moy#19] +Output [13]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] +Input [15]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_date_sk#188, d_year#189, d_moy#190] (42) ReusedExchange [Reuses operator id: 20] -Output [1]: [t_time_sk#21] +Output [1]: [t_time_sk#191] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_time_sk#174] -Right keys [1]: [t_time_sk#21] +Right keys [1]: [t_time_sk#191] Join condition: None (44) Project [codegen id : 11] -Output [12]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] -Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19, t_time_sk#21] +Output [12]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] +Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190, t_time_sk#191] (45) ReusedExchange [Reuses operator id: 27] -Output [1]: [sm_ship_mode_sk#24] +Output [1]: [sm_ship_mode_sk#192] (46) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_mode_sk#175] -Right keys [1]: [sm_ship_mode_sk#24] +Right keys [1]: [sm_ship_mode_sk#192] Join condition: None (47) Project [codegen id : 11] -Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] -Input [13]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#24] +Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] +Input [13]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190, sm_ship_mode_sk#192] (48) HashAggregate [codegen id : 11] -Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18] -Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228] -Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] +Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] +Keys [7]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189] +Functions [24]: [partial_sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [48]: [sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240] +Results [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] (49) Exchange -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#277] +Input [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] +Arguments: hashpartitioning(w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, 5), ENSURE_REQUIREMENTS, [id=#289] (50) HashAggregate [codegen id : 12] -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18] -Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301] -Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, DHL,BARIAN AS ship_carriers#302, d_year#18 AS year#303, sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278 AS jan_sales#304, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279 AS feb_sales#305, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280 AS mar_sales#306, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281 AS apr_sales#307, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282 AS may_sales#308, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283 AS jun_sales#309, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284 AS jul_sales#310, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285 AS aug_sales#311, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286 AS sep_sales#312, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287 AS oct_sales#313, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288 AS nov_sales#314, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289 AS dec_sales#315, sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_net#316, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_net#317, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_net#318, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_net#319, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_net#320, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_net#321, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_net#322, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_net#323, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_net#324, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_net#325, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_net#326, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_net#327] +Input [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] +Keys [7]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189] +Functions [24]: [sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313] +Results [32]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, DHL,BARIAN AS ship_carriers#314, d_year#189 AS year#315, sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_sales#316, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_sales#317, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_sales#318, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_sales#319, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_sales#320, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_sales#321, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_sales#322, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_sales#323, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_sales#324, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_sales#325, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_sales#326, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_sales#327, sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302 AS jan_net#328, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303 AS feb_net#329, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304 AS mar_net#330, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305 AS apr_net#331, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306 AS may_net#332, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307 AS jun_net#333, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308 AS jul_net#334, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309 AS aug_net#335, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310 AS sep_net#336, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311 AS oct_net#337, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312 AS nov_net#338, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313 AS dec_net#339] (51) Union @@ -292,23 +292,23 @@ Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13 Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#150, feb_sales#151, mar_sales#152, apr_sales#153, may_sales#154, jun_sales#155, jul_sales#156, aug_sales#157, sep_sales#158, oct_sales#159, nov_sales#160, dec_sales#161, jan_net#162, feb_net#163, mar_net#164, apr_net#165, may_net#166, jun_net#167, jul_net#168, aug_net#169, sep_net#170, oct_net#171, nov_net#172, dec_net#173] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149] Functions [36]: [partial_sum(jan_sales#150), partial_sum(feb_sales#151), partial_sum(mar_sales#152), partial_sum(apr_sales#153), partial_sum(may_sales#154), partial_sum(jun_sales#155), partial_sum(jul_sales#156), partial_sum(aug_sales#157), partial_sum(sep_sales#158), partial_sum(oct_sales#159), partial_sum(nov_sales#160), partial_sum(dec_sales#161), partial_sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(jan_net#162), partial_sum(feb_net#163), partial_sum(mar_net#164), partial_sum(apr_net#165), partial_sum(may_net#166), partial_sum(jun_net#167), partial_sum(jul_net#168), partial_sum(aug_net#169), partial_sum(sep_net#170), partial_sum(oct_net#171), partial_sum(nov_net#172), partial_sum(dec_net#173)] -Aggregate Attributes [72]: [sum#328, isEmpty#329, sum#330, isEmpty#331, sum#332, isEmpty#333, sum#334, isEmpty#335, sum#336, isEmpty#337, sum#338, isEmpty#339, sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399] -Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] +Aggregate Attributes [72]: [sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411] +Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] (53) Exchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#472] +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#484] (54) HashAggregate [codegen id : 14] -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149] Functions [36]: [sum(jan_sales#150), sum(feb_sales#151), sum(mar_sales#152), sum(apr_sales#153), sum(may_sales#154), sum(jun_sales#155), sum(jul_sales#156), sum(aug_sales#157), sum(sep_sales#158), sum(oct_sales#159), sum(nov_sales#160), sum(dec_sales#161), sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(jan_net#162), sum(feb_net#163), sum(mar_net#164), sum(apr_net#165), sum(may_net#166), sum(jun_net#167), sum(jul_net#168), sum(aug_net#169), sum(sep_net#170), sum(oct_net#171), sum(nov_net#172), sum(dec_net#173)] -Aggregate Attributes [36]: [sum(jan_sales#150)#473, sum(feb_sales#151)#474, sum(mar_sales#152)#475, sum(apr_sales#153)#476, sum(may_sales#154)#477, sum(jun_sales#155)#478, sum(jul_sales#156)#479, sum(aug_sales#157)#480, sum(sep_sales#158)#481, sum(oct_sales#159)#482, sum(nov_sales#160)#483, sum(dec_sales#161)#484, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496, sum(jan_net#162)#497, sum(feb_net#163)#498, sum(mar_net#164)#499, sum(apr_net#165)#500, sum(may_net#166)#501, sum(jun_net#167)#502, sum(jul_net#168)#503, sum(aug_net#169)#504, sum(sep_net#170)#505, sum(oct_net#171)#506, sum(nov_net#172)#507, sum(dec_net#173)#508] -Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum(jan_sales#150)#473 AS jan_sales#509, sum(feb_sales#151)#474 AS feb_sales#510, sum(mar_sales#152)#475 AS mar_sales#511, sum(apr_sales#153)#476 AS apr_sales#512, sum(may_sales#154)#477 AS may_sales#513, sum(jun_sales#155)#478 AS jun_sales#514, sum(jul_sales#156)#479 AS jul_sales#515, sum(aug_sales#157)#480 AS aug_sales#516, sum(sep_sales#158)#481 AS sep_sales#517, sum(oct_sales#159)#482 AS oct_sales#518, sum(nov_sales#160)#483 AS nov_sales#519, sum(dec_sales#161)#484 AS dec_sales#520, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485 AS jan_sales_per_sq_foot#521, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486 AS feb_sales_per_sq_foot#522, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487 AS mar_sales_per_sq_foot#523, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488 AS apr_sales_per_sq_foot#524, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489 AS may_sales_per_sq_foot#525, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490 AS jun_sales_per_sq_foot#526, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491 AS jul_sales_per_sq_foot#527, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492 AS aug_sales_per_sq_foot#528, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493 AS sep_sales_per_sq_foot#529, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494 AS oct_sales_per_sq_foot#530, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495 AS nov_sales_per_sq_foot#531, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496 AS dec_sales_per_sq_foot#532, sum(jan_net#162)#497 AS jan_net#533, sum(feb_net#163)#498 AS feb_net#534, sum(mar_net#164)#499 AS mar_net#535, sum(apr_net#165)#500 AS apr_net#536, sum(may_net#166)#501 AS may_net#537, sum(jun_net#167)#502 AS jun_net#538, sum(jul_net#168)#503 AS jul_net#539, sum(aug_net#169)#504 AS aug_net#540, sum(sep_net#170)#505 AS sep_net#541, sum(oct_net#171)#506 AS oct_net#542, sum(nov_net#172)#507 AS nov_net#543, sum(dec_net#173)#508 AS dec_net#544] +Aggregate Attributes [36]: [sum(jan_sales#150)#485, sum(feb_sales#151)#486, sum(mar_sales#152)#487, sum(apr_sales#153)#488, sum(may_sales#154)#489, sum(jun_sales#155)#490, sum(jul_sales#156)#491, sum(aug_sales#157)#492, sum(sep_sales#158)#493, sum(oct_sales#159)#494, sum(nov_sales#160)#495, sum(dec_sales#161)#496, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508, sum(jan_net#162)#509, sum(feb_net#163)#510, sum(mar_net#164)#511, sum(apr_net#165)#512, sum(may_net#166)#513, sum(jun_net#167)#514, sum(jul_net#168)#515, sum(aug_net#169)#516, sum(sep_net#170)#517, sum(oct_net#171)#518, sum(nov_net#172)#519, sum(dec_net#173)#520] +Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum(jan_sales#150)#485 AS jan_sales#521, sum(feb_sales#151)#486 AS feb_sales#522, sum(mar_sales#152)#487 AS mar_sales#523, sum(apr_sales#153)#488 AS apr_sales#524, sum(may_sales#154)#489 AS may_sales#525, sum(jun_sales#155)#490 AS jun_sales#526, sum(jul_sales#156)#491 AS jul_sales#527, sum(aug_sales#157)#492 AS aug_sales#528, sum(sep_sales#158)#493 AS sep_sales#529, sum(oct_sales#159)#494 AS oct_sales#530, sum(nov_sales#160)#495 AS nov_sales#531, sum(dec_sales#161)#496 AS dec_sales#532, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497 AS jan_sales_per_sq_foot#533, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498 AS feb_sales_per_sq_foot#534, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499 AS mar_sales_per_sq_foot#535, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500 AS apr_sales_per_sq_foot#536, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501 AS may_sales_per_sq_foot#537, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502 AS jun_sales_per_sq_foot#538, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503 AS jul_sales_per_sq_foot#539, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504 AS aug_sales_per_sq_foot#540, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505 AS sep_sales_per_sq_foot#541, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506 AS oct_sales_per_sq_foot#542, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507 AS nov_sales_per_sq_foot#543, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508 AS dec_sales_per_sq_foot#544, sum(jan_net#162)#509 AS jan_net#545, sum(feb_net#163)#510 AS feb_net#546, sum(mar_net#164)#511 AS mar_net#547, sum(apr_net#165)#512 AS apr_net#548, sum(may_net#166)#513 AS may_net#549, sum(jun_net#167)#514 AS jun_net#550, sum(jul_net#168)#515 AS jul_net#551, sum(aug_net#169)#516 AS aug_net#552, sum(sep_net#170)#517 AS sep_net#553, sum(oct_net#171)#518 AS oct_net#554, sum(nov_net#172)#519 AS nov_net#555, sum(dec_net#173)#520 AS dec_net#556] (55) TakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, may_sales_per_sq_foot#525, jun_sales_per_sq_foot#526, jul_sales_per_sq_foot#527, aug_sales_per_sq_foot#528, sep_sales_per_sq_foot#529, oct_sales_per_sq_foot#530, nov_sales_per_sq_foot#531, dec_sales_per_sq_foot#532, jan_net#533, feb_net#534, mar_net#535, apr_net#536, may_net#537, jun_net#538, jul_net#539, aug_net#540, sep_net#541, oct_net#542, nov_net#543, dec_net#544] -Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, ... 20 more fields] +Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, may_sales_per_sq_foot#537, jun_sales_per_sq_foot#538, jul_sales_per_sq_foot#539, aug_sales_per_sq_foot#540, sep_sales_per_sq_foot#541, oct_sales_per_sq_foot#542, nov_sales_per_sq_foot#543, dec_sales_per_sq_foot#544, jan_net#545, feb_net#546, mar_net#547, apr_net#548, may_net#549, jun_net#550, jul_net#551, aug_net#552, sep_net#553, oct_net#554, nov_net#555, dec_net#556] +Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, ... 20 more fields] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt index 744fe4b5a594d..b6a609ec193b4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt @@ -230,55 +230,55 @@ Input [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_ Arguments: [ss_addr_sk#12 ASC NULLS FIRST], false, 0 (41) ReusedExchange [Reuses operator id: 9] -Output [2]: [ca_address_sk#6, ca_city#7] +Output [2]: [ca_address_sk#32, ca_city#33] (42) Sort [codegen id : 13] -Input [2]: [ca_address_sk#6, ca_city#7] -Arguments: [ca_address_sk#6 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#32, ca_city#33] +Arguments: [ca_address_sk#32 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 14] Left keys [1]: [ss_addr_sk#12] -Right keys [1]: [ca_address_sk#6] +Right keys [1]: [ca_address_sk#32] Join condition: None (44) Project [codegen id : 14] -Output [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#7] -Input [8]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_address_sk#6, ca_city#7] +Output [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#33] +Input [8]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_address_sk#32, ca_city#33] (45) HashAggregate [codegen id : 14] -Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#7] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] +Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#33] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#15)), partial_sum(UnscaledValue(ss_ext_list_price#16)), partial_sum(UnscaledValue(ss_ext_tax#17))] -Aggregate Attributes [3]: [sum#32, sum#33, sum#34] -Results [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#35, sum#36, sum#37] +Aggregate Attributes [3]: [sum#34, sum#35, sum#36] +Results [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33, sum#37, sum#38, sum#39] (46) HashAggregate [codegen id : 14] -Input [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#35, sum#36, sum#37] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] +Input [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33, sum#37, sum#38, sum#39] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#15)), sum(UnscaledValue(ss_ext_list_price#16)), sum(UnscaledValue(ss_ext_tax#17))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#15))#38, sum(UnscaledValue(ss_ext_list_price#16))#39, sum(UnscaledValue(ss_ext_tax#17))#40] -Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#7 AS bought_city#41, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#38,17,2) AS extended_price#42, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#16))#39,17,2) AS list_price#43, MakeDecimal(sum(UnscaledValue(ss_ext_tax#17))#40,17,2) AS extended_tax#44] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#15))#40, sum(UnscaledValue(ss_ext_list_price#16))#41, sum(UnscaledValue(ss_ext_tax#17))#42] +Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#33 AS bought_city#43, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#40,17,2) AS extended_price#44, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#16))#41,17,2) AS list_price#45, MakeDecimal(sum(UnscaledValue(ss_ext_tax#17))#42,17,2) AS extended_tax#46] (47) Exchange -Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#41, extended_price#42, list_price#43, extended_tax#44] -Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#43, extended_price#44, list_price#45, extended_tax#46] +Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#47] (48) Sort [codegen id : 15] -Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#41, extended_price#42, list_price#43, extended_tax#44] +Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#43, extended_price#44, list_price#45, extended_tax#46] Arguments: [ss_customer_sk#10 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 16] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#10] -Join condition: NOT (ca_city#7 = bought_city#41) +Join condition: NOT (ca_city#7 = bought_city#43) (50) Project [codegen id : 16] -Output [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#41, ss_ticket_number#14, extended_price#42, extended_tax#44, list_price#43] -Input [10]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#41, extended_price#42, list_price#43, extended_tax#44] +Output [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#43, ss_ticket_number#14, extended_price#44, extended_tax#46, list_price#45] +Input [10]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#43, extended_price#44, list_price#45, extended_tax#46] (51) TakeOrderedAndProject -Input [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#41, ss_ticket_number#14, extended_price#42, extended_tax#44, list_price#43] -Arguments: 100, [c_last_name#4 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#41, ss_ticket_number#14, extended_price#42, extended_tax#44, list_price#43] +Input [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#43, ss_ticket_number#14, extended_price#44, extended_tax#46, list_price#45] +Arguments: 100, [c_last_name#4 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#43, ss_ticket_number#14, extended_price#44, extended_tax#46, list_price#45] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt index 1a9fafda6250b..101e4f272f98a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt @@ -225,20 +225,20 @@ Output [8]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#3 Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_customer_sk#39, c_current_addr_sk#40, c_first_name#41, c_last_name#42] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#22, ca_city#23] +Output [2]: [ca_address_sk#44, ca_city#45] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#40] -Right keys [1]: [ca_address_sk#22] -Join condition: NOT (ca_city#23 = bought_city#35) +Right keys [1]: [ca_address_sk#44] +Join condition: NOT (ca_city#45 = bought_city#35) (42) Project [codegen id : 8] -Output [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#22, ca_city#23] +Output [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#44, ca_city#45] (43) TakeOrderedAndProject -Input [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt index 97d7a81baf2a7..a0532462eff64 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt @@ -139,20 +139,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#15] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#13] -Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] +Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] (23) Exchange Input [1]: [ws_bill_customer_sk#13] -Arguments: hashpartitioning(ws_bill_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#15] +Arguments: hashpartitioning(ws_bill_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#16] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#13] @@ -164,38 +164,38 @@ Right keys [1]: [ws_bill_customer_sk#13] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#19] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#16] -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] +Output [1]: [cs_ship_customer_sk#17] +Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] (31) Exchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: hashpartitioning(cs_ship_customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [1]: [cs_ship_customer_sk#17] +Arguments: hashpartitioning(cs_ship_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#20] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#16] -Arguments: [cs_ship_customer_sk#16 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#17] +Arguments: [cs_ship_customer_sk#17 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#16] +Right keys [1]: [cs_ship_customer_sk#17] Join condition: None (34) Project [codegen id : 13] @@ -203,84 +203,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (35) Scan parquet default.customer_address -Output [2]: [ca_address_sk#19, ca_state#20] +Output [2]: [ca_address_sk#21, ca_state#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 12] -Input [2]: [ca_address_sk#19, ca_state#20] +Input [2]: [ca_address_sk#21, ca_state#22] (37) Filter [codegen id : 12] -Input [2]: [ca_address_sk#19, ca_state#20] -Condition : (ca_state#20 IN (KY,GA,NM) AND isnotnull(ca_address_sk#19)) +Input [2]: [ca_address_sk#21, ca_state#22] +Condition : (ca_state#22 IN (KY,GA,NM) AND isnotnull(ca_address_sk#21)) (38) Project [codegen id : 12] -Output [1]: [ca_address_sk#19] -Input [2]: [ca_address_sk#19, ca_state#20] +Output [1]: [ca_address_sk#21] +Input [2]: [ca_address_sk#21, ca_state#22] (39) BroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [ca_address_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#19] +Right keys [1]: [ca_address_sk#21] Join condition: None (41) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21] (42) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (43) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (44) ColumnarToRow -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] (45) Filter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Condition : isnotnull(cd_demo_sk#23) +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Condition : isnotnull(cd_demo_sk#25) (46) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#25] Join condition: None (47) Project [codegen id : 14] -Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Output [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] (48) HashAggregate [codegen id : 14] -Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] +Aggregate Attributes [1]: [count#31] +Results [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] (49) Exchange -Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] -Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] +Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, 5), ENSURE_REQUIREMENTS, [id=#33] (50) HashAggregate [codegen id : 15] -Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] -Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] +Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#32 AS cnt1#33, cd_purchase_estimate#27, count(1)#32 AS cnt2#34, cd_credit_rating#28, count(1)#32 AS cnt3#35] +Aggregate Attributes [1]: [count(1)#34] +Results [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#34 AS cnt1#35, cd_purchase_estimate#29, count(1)#34 AS cnt2#36, cd_credit_rating#30, count(1)#34 AS cnt3#37] (51) TakeOrderedAndProject -Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#33, cd_purchase_estimate#27, cnt2#34, cd_credit_rating#28, cnt3#35] -Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#33, cd_purchase_estimate#27, cnt2#34, cd_credit_rating#28, cnt3#35] +Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#35, cd_purchase_estimate#29, cnt2#36, cd_credit_rating#30, cnt3#37] +Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#35, cd_purchase_estimate#29, cnt2#36, cd_credit_rating#30, cnt3#37] ===== Subqueries ===== @@ -293,6 +293,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index b5307abaf688c..d0b3c2231d997 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -122,20 +122,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#12] -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] @@ -143,34 +143,34 @@ Right keys [1]: [ws_bill_customer_sk#12] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#6)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#18] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#18] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#15] -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] +Output [1]: [cs_ship_customer_sk#16] +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#18] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Input [1]: [cs_ship_customer_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#15] +Right keys [1]: [cs_ship_customer_sk#16] Join condition: None (29) Project [codegen id : 9] @@ -178,84 +178,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (30) Scan parquet default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_state#19] +Input [2]: [ca_address_sk#20, ca_state#21] (32) Filter [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : (ca_state#21 IN (KY,GA,NM) AND isnotnull(ca_address_sk#20)) (33) Project [codegen id : 7] -Output [1]: [ca_address_sk#18] -Input [2]: [ca_address_sk#18, ca_state#19] +Output [1]: [ca_address_sk#20] +Input [2]: [ca_address_sk#20, ca_state#21] (34) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +Input [1]: [ca_address_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#18] +Right keys [1]: [ca_address_sk#20] Join condition: None (36) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] (37) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] (39) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -Condition : isnotnull(cd_demo_sk#21) +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Condition : isnotnull(cd_demo_sk#23) (40) BroadcastExchange -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#21] +Right keys [1]: [cd_demo_sk#23] Join condition: None (42) Project [codegen id : 9] -Output [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] (43) HashAggregate [codegen id : 9] -Input [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] +Aggregate Attributes [1]: [count#30] +Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#31] (44) Exchange -Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] -Arguments: hashpartitioning(cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#31] +Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, 5), ENSURE_REQUIREMENTS, [id=#32] (45) HashAggregate [codegen id : 10] -Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] -Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#31] +Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, count(1)#31 AS cnt1#32, cd_purchase_estimate#25, count(1)#31 AS cnt2#33, cd_credit_rating#26, count(1)#31 AS cnt3#34] +Aggregate Attributes [1]: [count(1)#33] +Results [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#33 AS cnt1#34, cd_purchase_estimate#27, count(1)#33 AS cnt2#35, cd_credit_rating#28, count(1)#33 AS cnt3#36] (46) TakeOrderedAndProject -Input [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#32, cd_purchase_estimate#25, cnt2#33, cd_credit_rating#26, cnt3#34] -Arguments: 100, [cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_education_status#24 ASC NULLS FIRST, cd_purchase_estimate#25 ASC NULLS FIRST, cd_credit_rating#26 ASC NULLS FIRST], [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#32, cd_purchase_estimate#25, cnt2#33, cd_credit_rating#26, cnt3#34] +Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#34, cd_purchase_estimate#27, cnt2#35, cd_credit_rating#28, cnt3#36] +Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#34, cd_purchase_estimate#27, cnt2#35, cd_credit_rating#28, cnt3#36] ===== Subqueries ===== @@ -268,6 +268,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index 9e2d33f2bca19..7b8445a46ac54 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -109,109 +109,109 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] (16) Filter [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_store_sk#11) (17) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#13] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] -Output [2]: [ss_store_sk#1, ss_net_profit#2] -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] +Output [2]: [ss_store_sk#11, ss_net_profit#12] +Input [4]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13, d_date_sk#14] (20) Scan parquet default.store -Output [2]: [s_store_sk#8, s_state#10] +Output [2]: [s_store_sk#15, s_state#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#8, s_state#10] +Input [2]: [s_store_sk#15, s_state#16] (22) Filter [codegen id : 3] -Input [2]: [s_store_sk#8, s_state#10] -Condition : isnotnull(s_store_sk#8) +Input [2]: [s_store_sk#15, s_state#16] +Condition : isnotnull(s_store_sk#15) (23) BroadcastExchange -Input [2]: [s_store_sk#8, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +Input [2]: [s_store_sk#15, s_state#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#8] +Left keys [1]: [ss_store_sk#11] +Right keys [1]: [s_store_sk#15] Join condition: None (25) Project [codegen id : 4] -Output [2]: [ss_net_profit#2, s_state#10] -Input [4]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_state#10] +Output [2]: [ss_net_profit#12, s_state#16] +Input [4]: [ss_store_sk#11, ss_net_profit#12, s_store_sk#15, s_state#16] (26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#2, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#12] -Results [2]: [s_state#10, sum#13] +Input [2]: [ss_net_profit#12, s_state#16] +Keys [1]: [s_state#16] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum#18] +Results [2]: [s_state#16, sum#19] (27) Exchange -Input [2]: [s_state#10, sum#13] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [2]: [s_state#16, sum#19] +Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, [id=#20] (28) HashAggregate [codegen id : 5] -Input [2]: [s_state#10, sum#13] -Keys [1]: [s_state#10] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#15] -Results [3]: [s_state#10 AS s_state#16, s_state#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#15,17,2) AS _w2#17] +Input [2]: [s_state#16, sum#19] +Keys [1]: [s_state#16] +Functions [1]: [sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#12))#21] +Results [3]: [s_state#16 AS s_state#22, s_state#16, MakeDecimal(sum(UnscaledValue(ss_net_profit#12))#21,17,2) AS _w2#23] (29) Exchange -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, [id=#24] (30) Sort [codegen id : 6] -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [s_state#10 ASC NULLS FIRST, _w2#17 DESC NULLS LAST], false, 0 +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: [s_state#16 ASC NULLS FIRST, _w2#23 DESC NULLS LAST], false, 0 (31) Window -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#10], [_w2#17 DESC NULLS LAST] +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: [rank(_w2#23) windowspecdefinition(s_state#16, _w2#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#25], [s_state#16], [_w2#23 DESC NULLS LAST] (32) Filter [codegen id : 7] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (ranking#19 <= 5) +Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] +Condition : (ranking#25 <= 5) (33) Project [codegen id : 7] -Output [1]: [s_state#16] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] +Output [1]: [s_state#22] +Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] (34) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#20] +Input [1]: [s_state#22] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#26] (35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#10] -Right keys [1]: [s_state#16] +Right keys [1]: [s_state#22] Join condition: None (36) BroadcastExchange Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] @@ -224,45 +224,45 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_county#9, s_state#10 (39) Expand [codegen id : 9] Input [3]: [ss_net_profit#2, s_state#10, s_county#9] -Arguments: [List(ss_net_profit#2, s_state#10, s_county#9, 0), List(ss_net_profit#2, s_state#10, null, 1), List(ss_net_profit#2, null, null, 3)], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] +Arguments: [List(ss_net_profit#2, s_state#10, s_county#9, 0), List(ss_net_profit#2, s_state#10, null, 1), List(ss_net_profit#2, null, null, 3)], [ss_net_profit#2, s_state#28, s_county#29, spark_grouping_id#30] (40) HashAggregate [codegen id : 9] -Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] +Input [4]: [ss_net_profit#2, s_state#28, s_county#29, spark_grouping_id#30] +Keys [3]: [s_state#28, s_county#29, spark_grouping_id#30] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#25] -Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Aggregate Attributes [1]: [sum#31] +Results [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] (41) Exchange -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] +Arguments: hashpartitioning(s_state#28, s_county#29, spark_grouping_id#30, 5), ENSURE_REQUIREMENTS, [id=#33] (42) HashAggregate [codegen id : 10] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] +Input [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] +Keys [3]: [s_state#28, s_county#29, spark_grouping_id#30] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#28] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS total_sum#29, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS _w3#33] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#34] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS total_sum#35, s_state#28, s_county#29, (cast((shiftright(spark_grouping_id#30, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint)) AS lochierarchy#36, (cast((shiftright(spark_grouping_id#30, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint)) AS _w1#37, CASE WHEN (cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint) = 0) THEN s_state#28 END AS _w2#38, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS _w3#39] (43) Exchange -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: hashpartitioning(_w1#37, _w2#38, 5), ENSURE_REQUIREMENTS, [id=#40] (44) Sort [codegen id : 11] -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w3#33 DESC NULLS LAST], false, 0 +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: [_w1#37 ASC NULLS FIRST, _w2#38 ASC NULLS FIRST, _w3#39 DESC NULLS LAST], false, 0 (45) Window -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: [rank(_w3#33) windowspecdefinition(_w1#31, _w2#32, _w3#33 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#35], [_w1#31, _w2#32], [_w3#33 DESC NULLS LAST] +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: [rank(_w3#39) windowspecdefinition(_w1#37, _w2#38, _w3#39 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [_w1#37, _w2#38], [_w3#39 DESC NULLS LAST] (46) Project [codegen id : 12] -Output [5]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] -Input [8]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33, rank_within_parent#35] +Output [5]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] +Input [8]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39, rank_within_parent#41] (47) TakeOrderedAndProject -Input [5]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#35 ASC NULLS FIRST], [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] +Input [5]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] +Arguments: 100, [lochierarchy#36 DESC NULLS LAST, CASE WHEN (lochierarchy#36 = 0) THEN s_state#28 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] ===== Subqueries ===== @@ -273,6 +273,6 @@ ReusedExchange (48) (48) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#5] -Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index cd12e362c44f0..91cfba59bb5bc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -109,109 +109,109 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] (16) Filter [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_store_sk#11) (17) Scan parquet default.store -Output [2]: [s_store_sk#8, s_state#10] +Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#8, s_state#10] +Input [2]: [s_store_sk#14, s_state#15] (19) Filter [codegen id : 2] -Input [2]: [s_store_sk#8, s_state#10] -Condition : isnotnull(s_store_sk#8) +Input [2]: [s_store_sk#14, s_state#15] +Condition : isnotnull(s_store_sk#14) (20) BroadcastExchange -Input [2]: [s_store_sk#8, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +Input [2]: [s_store_sk#14, s_state#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#16] (21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#8] +Left keys [1]: [ss_store_sk#11] +Right keys [1]: [s_store_sk#14] Join condition: None (22) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, ss_sold_date_sk#3, s_state#10] -Input [5]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, s_store_sk#8, s_state#10] +Output [3]: [ss_net_profit#12, ss_sold_date_sk#13, s_state#15] +Input [5]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13, s_store_sk#14, s_state#15] (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#17] (24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#13] +Right keys [1]: [d_date_sk#17] Join condition: None (25) Project [codegen id : 4] -Output [2]: [ss_net_profit#2, s_state#10] -Input [4]: [ss_net_profit#2, ss_sold_date_sk#3, s_state#10, d_date_sk#5] +Output [2]: [ss_net_profit#12, s_state#15] +Input [4]: [ss_net_profit#12, ss_sold_date_sk#13, s_state#15, d_date_sk#17] (26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#2, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#12] -Results [2]: [s_state#10, sum#13] +Input [2]: [ss_net_profit#12, s_state#15] +Keys [1]: [s_state#15] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum#18] +Results [2]: [s_state#15, sum#19] (27) Exchange -Input [2]: [s_state#10, sum#13] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [2]: [s_state#15, sum#19] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [id=#20] (28) HashAggregate [codegen id : 5] -Input [2]: [s_state#10, sum#13] -Keys [1]: [s_state#10] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#15] -Results [3]: [s_state#10 AS s_state#16, s_state#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#15,17,2) AS _w2#17] +Input [2]: [s_state#15, sum#19] +Keys [1]: [s_state#15] +Functions [1]: [sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#12))#21] +Results [3]: [s_state#15 AS s_state#22, s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#12))#21,17,2) AS _w2#23] (29) Exchange -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [id=#24] (30) Sort [codegen id : 6] -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [s_state#10 ASC NULLS FIRST, _w2#17 DESC NULLS LAST], false, 0 +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: [s_state#15 ASC NULLS FIRST, _w2#23 DESC NULLS LAST], false, 0 (31) Window -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#10], [_w2#17 DESC NULLS LAST] +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: [rank(_w2#23) windowspecdefinition(s_state#15, _w2#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#25], [s_state#15], [_w2#23 DESC NULLS LAST] (32) Filter [codegen id : 7] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (ranking#19 <= 5) +Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] +Condition : (ranking#25 <= 5) (33) Project [codegen id : 7] -Output [1]: [s_state#16] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] +Output [1]: [s_state#22] +Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] (34) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#20] +Input [1]: [s_state#22] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#26] (35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#10] -Right keys [1]: [s_state#16] +Right keys [1]: [s_state#22] Join condition: None (36) BroadcastExchange Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] @@ -224,45 +224,45 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_county#9, s_state#10 (39) Expand [codegen id : 9] Input [3]: [ss_net_profit#2, s_state#10, s_county#9] -Arguments: [List(ss_net_profit#2, s_state#10, s_county#9, 0), List(ss_net_profit#2, s_state#10, null, 1), List(ss_net_profit#2, null, null, 3)], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] +Arguments: [List(ss_net_profit#2, s_state#10, s_county#9, 0), List(ss_net_profit#2, s_state#10, null, 1), List(ss_net_profit#2, null, null, 3)], [ss_net_profit#2, s_state#28, s_county#29, spark_grouping_id#30] (40) HashAggregate [codegen id : 9] -Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] +Input [4]: [ss_net_profit#2, s_state#28, s_county#29, spark_grouping_id#30] +Keys [3]: [s_state#28, s_county#29, spark_grouping_id#30] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#25] -Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Aggregate Attributes [1]: [sum#31] +Results [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] (41) Exchange -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] +Arguments: hashpartitioning(s_state#28, s_county#29, spark_grouping_id#30, 5), ENSURE_REQUIREMENTS, [id=#33] (42) HashAggregate [codegen id : 10] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] +Input [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] +Keys [3]: [s_state#28, s_county#29, spark_grouping_id#30] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#28] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS total_sum#29, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS _w3#33] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#34] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS total_sum#35, s_state#28, s_county#29, (cast((shiftright(spark_grouping_id#30, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint)) AS lochierarchy#36, (cast((shiftright(spark_grouping_id#30, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint)) AS _w1#37, CASE WHEN (cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint) = 0) THEN s_state#28 END AS _w2#38, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS _w3#39] (43) Exchange -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: hashpartitioning(_w1#37, _w2#38, 5), ENSURE_REQUIREMENTS, [id=#40] (44) Sort [codegen id : 11] -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w3#33 DESC NULLS LAST], false, 0 +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: [_w1#37 ASC NULLS FIRST, _w2#38 ASC NULLS FIRST, _w3#39 DESC NULLS LAST], false, 0 (45) Window -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: [rank(_w3#33) windowspecdefinition(_w1#31, _w2#32, _w3#33 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#35], [_w1#31, _w2#32], [_w3#33 DESC NULLS LAST] +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: [rank(_w3#39) windowspecdefinition(_w1#37, _w2#38, _w3#39 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [_w1#37, _w2#38], [_w3#39 DESC NULLS LAST] (46) Project [codegen id : 12] -Output [5]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] -Input [8]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33, rank_within_parent#35] +Output [5]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] +Input [8]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39, rank_within_parent#41] (47) TakeOrderedAndProject -Input [5]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#35 ASC NULLS FIRST], [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] +Input [5]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] +Arguments: 100, [lochierarchy#36 DESC NULLS LAST, CASE WHEN (lochierarchy#36 = 0) THEN s_state#28 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] ===== Subqueries ===== @@ -273,6 +273,6 @@ ReusedExchange (48) (48) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#5] -Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt index cd88a84ede266..274d0c02b5583 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt @@ -127,43 +127,43 @@ Input [4]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_da Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_sold_time_sk#18)) (19) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#22] (20) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#11] +Right keys [1]: [d_date_sk#22] Join condition: None (21) Project [codegen id : 5] -Output [3]: [cs_ext_sales_price#20 AS ext_price#22, cs_item_sk#19 AS sold_item_sk#23, cs_sold_time_sk#18 AS time_sk#24] -Input [5]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#11] +Output [3]: [cs_ext_sales_price#20 AS ext_price#23, cs_item_sk#19 AS sold_item_sk#24, cs_sold_time_sk#18 AS time_sk#25] +Input [5]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#22] (22) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 7] -Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] (24) Filter [codegen id : 7] -Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -Condition : (isnotnull(ss_item_sk#26) AND isnotnull(ss_sold_time_sk#25)) +Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Condition : (isnotnull(ss_item_sk#27) AND isnotnull(ss_sold_time_sk#26)) (25) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#30] (26) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ss_sold_date_sk#29] +Right keys [1]: [d_date_sk#30] Join condition: None (27) Project [codegen id : 7] -Output [3]: [ss_ext_sales_price#27 AS ext_price#29, ss_item_sk#26 AS sold_item_sk#30, ss_sold_time_sk#25 AS time_sk#31] -Input [5]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#11] +Output [3]: [ss_ext_sales_price#28 AS ext_price#31, ss_item_sk#27 AS sold_item_sk#32, ss_sold_time_sk#26 AS time_sk#33] +Input [5]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#30] (28) Union @@ -177,61 +177,61 @@ Output [4]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17] Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#15, sold_item_sk#16, time_sk#17] (31) Scan parquet default.time_dim -Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Output [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] (33) Filter [codegen id : 8] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Condition : (((t_meal_time#35 = breakfast ) OR (t_meal_time#35 = dinner )) AND isnotnull(t_time_sk#32)) +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Condition : (((t_meal_time#37 = breakfast ) OR (t_meal_time#37 = dinner )) AND isnotnull(t_time_sk#34)) (34) Project [codegen id : 8] -Output [3]: [t_time_sk#32, t_hour#33, t_minute#34] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Output [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] (35) BroadcastExchange -Input [3]: [t_time_sk#32, t_hour#33, t_minute#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [time_sk#17] -Right keys [1]: [t_time_sk#32] +Right keys [1]: [t_time_sk#34] Join condition: None (37) Project [codegen id : 9] -Output [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#33, t_minute#34] -Input [7]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17, t_time_sk#32, t_hour#33, t_minute#34] +Output [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#35, t_minute#36] +Input [7]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17, t_time_sk#34, t_hour#35, t_minute#36] (38) HashAggregate [codegen id : 9] -Input [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#33, t_minute#34] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] +Input [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#35, t_minute#36] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] Functions [1]: [partial_sum(UnscaledValue(ext_price#15))] -Aggregate Attributes [1]: [sum#37] -Results [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] +Aggregate Attributes [1]: [sum#39] +Results [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] (39) Exchange -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] -Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] +Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, 5), ENSURE_REQUIREMENTS, [id=#41] (40) HashAggregate [codegen id : 10] -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] Functions [1]: [sum(UnscaledValue(ext_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#15))#40] -Results [5]: [i_brand_id#2 AS brand_id#41, i_brand#3 AS brand#42, t_hour#33, t_minute#34, MakeDecimal(sum(UnscaledValue(ext_price#15))#40,17,2) AS ext_price#43] +Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#15))#42] +Results [5]: [i_brand_id#2 AS brand_id#43, i_brand#3 AS brand#44, t_hour#35, t_minute#36, MakeDecimal(sum(UnscaledValue(ext_price#15))#42,17,2) AS ext_price#45] (41) Exchange -Input [5]: [brand_id#41, brand#42, t_hour#33, t_minute#34, ext_price#43] -Arguments: rangepartitioning(ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [5]: [brand_id#43, brand#44, t_hour#35, t_minute#36, ext_price#45] +Arguments: rangepartitioning(ext_price#45 DESC NULLS LAST, brand_id#43 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#46] (42) Sort [codegen id : 11] -Input [5]: [brand_id#41, brand#42, t_hour#33, t_minute#34, ext_price#43] -Arguments: [ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST], true, 0 +Input [5]: [brand_id#43, brand#44, t_hour#35, t_minute#36, ext_price#45] +Arguments: [ext_price#45 DESC NULLS LAST, brand_id#43 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -244,6 +244,6 @@ Output [1]: [d_date_sk#11] Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index cd88a84ede266..274d0c02b5583 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt @@ -127,43 +127,43 @@ Input [4]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_da Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_sold_time_sk#18)) (19) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#22] (20) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#11] +Right keys [1]: [d_date_sk#22] Join condition: None (21) Project [codegen id : 5] -Output [3]: [cs_ext_sales_price#20 AS ext_price#22, cs_item_sk#19 AS sold_item_sk#23, cs_sold_time_sk#18 AS time_sk#24] -Input [5]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#11] +Output [3]: [cs_ext_sales_price#20 AS ext_price#23, cs_item_sk#19 AS sold_item_sk#24, cs_sold_time_sk#18 AS time_sk#25] +Input [5]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#22] (22) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 7] -Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] (24) Filter [codegen id : 7] -Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -Condition : (isnotnull(ss_item_sk#26) AND isnotnull(ss_sold_time_sk#25)) +Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Condition : (isnotnull(ss_item_sk#27) AND isnotnull(ss_sold_time_sk#26)) (25) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#30] (26) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ss_sold_date_sk#29] +Right keys [1]: [d_date_sk#30] Join condition: None (27) Project [codegen id : 7] -Output [3]: [ss_ext_sales_price#27 AS ext_price#29, ss_item_sk#26 AS sold_item_sk#30, ss_sold_time_sk#25 AS time_sk#31] -Input [5]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#11] +Output [3]: [ss_ext_sales_price#28 AS ext_price#31, ss_item_sk#27 AS sold_item_sk#32, ss_sold_time_sk#26 AS time_sk#33] +Input [5]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#30] (28) Union @@ -177,61 +177,61 @@ Output [4]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17] Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#15, sold_item_sk#16, time_sk#17] (31) Scan parquet default.time_dim -Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Output [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] (33) Filter [codegen id : 8] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Condition : (((t_meal_time#35 = breakfast ) OR (t_meal_time#35 = dinner )) AND isnotnull(t_time_sk#32)) +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Condition : (((t_meal_time#37 = breakfast ) OR (t_meal_time#37 = dinner )) AND isnotnull(t_time_sk#34)) (34) Project [codegen id : 8] -Output [3]: [t_time_sk#32, t_hour#33, t_minute#34] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Output [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] (35) BroadcastExchange -Input [3]: [t_time_sk#32, t_hour#33, t_minute#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [time_sk#17] -Right keys [1]: [t_time_sk#32] +Right keys [1]: [t_time_sk#34] Join condition: None (37) Project [codegen id : 9] -Output [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#33, t_minute#34] -Input [7]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17, t_time_sk#32, t_hour#33, t_minute#34] +Output [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#35, t_minute#36] +Input [7]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17, t_time_sk#34, t_hour#35, t_minute#36] (38) HashAggregate [codegen id : 9] -Input [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#33, t_minute#34] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] +Input [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#35, t_minute#36] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] Functions [1]: [partial_sum(UnscaledValue(ext_price#15))] -Aggregate Attributes [1]: [sum#37] -Results [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] +Aggregate Attributes [1]: [sum#39] +Results [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] (39) Exchange -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] -Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] +Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, 5), ENSURE_REQUIREMENTS, [id=#41] (40) HashAggregate [codegen id : 10] -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] Functions [1]: [sum(UnscaledValue(ext_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#15))#40] -Results [5]: [i_brand_id#2 AS brand_id#41, i_brand#3 AS brand#42, t_hour#33, t_minute#34, MakeDecimal(sum(UnscaledValue(ext_price#15))#40,17,2) AS ext_price#43] +Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#15))#42] +Results [5]: [i_brand_id#2 AS brand_id#43, i_brand#3 AS brand#44, t_hour#35, t_minute#36, MakeDecimal(sum(UnscaledValue(ext_price#15))#42,17,2) AS ext_price#45] (41) Exchange -Input [5]: [brand_id#41, brand#42, t_hour#33, t_minute#34, ext_price#43] -Arguments: rangepartitioning(ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [5]: [brand_id#43, brand#44, t_hour#35, t_minute#36, ext_price#45] +Arguments: rangepartitioning(ext_price#45 DESC NULLS LAST, brand_id#43 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#46] (42) Sort [codegen id : 11] -Input [5]: [brand_id#41, brand#42, t_hour#33, t_minute#34, ext_price#43] -Arguments: [ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST], true, 0 +Input [5]: [brand_id#43, brand#44, t_hour#35, t_minute#36, ext_price#45] +Arguments: [ext_price#45 DESC NULLS LAST, brand_id#43 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -244,6 +244,6 @@ Output [1]: [d_date_sk#11] Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt index 74cbccc79fdc5..9b2ead7ea96f7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt @@ -199,285 +199,285 @@ Input [2]: [customer_id#18, year_total#19] Arguments: [customer_id#18 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] +Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] +Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] (27) Filter [codegen id : 10] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_customer_sk#1) +Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Condition : isnotnull(ss_customer_sk#21) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#25, d_year#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#5, d_year#6] +Input [2]: [d_date_sk#25, d_year#26] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#5, d_year#6] -Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2002)) AND d_year#6 IN (2001,2002)) AND isnotnull(d_date_sk#5)) +Input [2]: [d_date_sk#25, d_year#26] +Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) (31) BroadcastExchange -Input [2]: [d_date_sk#5, d_year#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +Input [2]: [d_date_sk#25, d_year#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#23] +Right keys [1]: [d_date_sk#25] Join condition: None (33) Project [codegen id : 10] -Output [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Input [5]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6] +Output [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Input [5]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] (34) Exchange -Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Arguments: hashpartitioning(ss_customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#28] (35) Sort [codegen id : 11] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Arguments: [ss_customer_sk#21 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] (37) Sort [codegen id : 13] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ss_customer_sk#21] +Right keys [1]: [c_customer_sk#29] Join condition: None (39) Project [codegen id : 14] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] -Input [7]: [ss_customer_sk#1, ss_net_paid#2, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] +Input [7]: [ss_customer_sk#21, ss_net_paid#22, d_year#26, c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] (40) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#2))] -Aggregate Attributes [1]: [sum#24] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] +Aggregate Attributes [1]: [sum#33] +Results [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] (41) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] +Arguments: hashpartitioning(c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, 5), ENSURE_REQUIREMENTS, [id=#35] (42) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ss_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#2))#27] -Results [4]: [c_customer_id#10 AS customer_id#28, c_first_name#11 AS customer_first_name#29, c_last_name#12 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#2))#27,17,2) AS year_total#31] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] +Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#22))#36] +Results [4]: [c_customer_id#30 AS customer_id#37, c_first_name#31 AS customer_first_name#38, c_last_name#32 AS customer_last_name#39, MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#36,17,2) AS year_total#40] (43) Exchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: hashpartitioning(customer_id#28, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] +Arguments: hashpartitioning(customer_id#37, 5), ENSURE_REQUIREMENTS, [id=#41] (44) Sort [codegen id : 16] -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#28 ASC NULLS FIRST], false, 0 +Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] +Arguments: [customer_id#37 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#28] +Right keys [1]: [customer_id#37] Join condition: None (46) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] (48) Filter [codegen id : 19] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_bill_customer_sk#33) +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_bill_customer_sk#42) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#45, d_year#46] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (51) Project [codegen id : 19] -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] +Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Input [5]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] (52) Exchange -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#47] (53) Sort [codegen id : 20] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Arguments: [ws_bill_customer_sk#42 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] (55) Sort [codegen id : 22] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Arguments: [c_customer_sk#48 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#33] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ws_bill_customer_sk#42] +Right keys [1]: [c_customer_sk#48] Join condition: None (57) Project [codegen id : 23] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] +Input [7]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46, c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] (58) HashAggregate [codegen id : 23] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum#37] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] +Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] +Aggregate Attributes [1]: [sum#52] +Results [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] (59) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] +Arguments: hashpartitioning(c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, 5), ENSURE_REQUIREMENTS, [id=#54] (60) HashAggregate [codegen id : 24] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#40] -Results [2]: [c_customer_id#10 AS customer_id#41, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#40,17,2) AS year_total#42] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] +Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] +Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#55] +Results [2]: [c_customer_id#49 AS customer_id#56, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#55,17,2) AS year_total#57] (61) Filter [codegen id : 24] -Input [2]: [customer_id#41, year_total#42] -Condition : (isnotnull(year_total#42) AND (year_total#42 > 0.00)) +Input [2]: [customer_id#56, year_total#57] +Condition : (isnotnull(year_total#57) AND (year_total#57 > 0.00)) (62) Project [codegen id : 24] -Output [2]: [customer_id#41 AS customer_id#43, year_total#42 AS year_total#44] -Input [2]: [customer_id#41, year_total#42] +Output [2]: [customer_id#56 AS customer_id#58, year_total#57 AS year_total#59] +Input [2]: [customer_id#56, year_total#57] (63) Exchange -Input [2]: [customer_id#43, year_total#44] -Arguments: hashpartitioning(customer_id#43, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [2]: [customer_id#58, year_total#59] +Arguments: hashpartitioning(customer_id#58, 5), ENSURE_REQUIREMENTS, [id=#60] (64) Sort [codegen id : 25] -Input [2]: [customer_id#43, year_total#44] -Arguments: [customer_id#43 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#58, year_total#59] +Arguments: [customer_id#58 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#43] +Right keys [1]: [customer_id#58] Join condition: None (66) Project [codegen id : 26] -Output [7]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44] -Input [8]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#43, year_total#44] +Output [7]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59] +Input [8]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, customer_id#58, year_total#59] (67) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] (69) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_bill_customer_sk#33) +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_bill_customer_sk#61) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#64, d_year#65] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#63] +Right keys [1]: [d_date_sk#64] Join condition: None (72) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] +Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Input [5]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65] (73) Exchange -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Arguments: hashpartitioning(ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#66] (74) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Arguments: [ws_bill_customer_sk#61 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] (76) Sort [codegen id : 31] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] +Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#33] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ws_bill_customer_sk#61] +Right keys [1]: [c_customer_sk#67] Join condition: None (78) Project [codegen id : 32] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] +Input [7]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65, c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] (79) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum#47] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] +Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#62))] +Aggregate Attributes [1]: [sum#71] +Results [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] (80) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] +Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#73] (81) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#50] -Results [2]: [c_customer_id#10 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#50,17,2) AS year_total#52] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] +Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] +Functions [1]: [sum(UnscaledValue(ws_net_paid#62))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#62))#74] +Results [2]: [c_customer_id#68 AS customer_id#75, MakeDecimal(sum(UnscaledValue(ws_net_paid#62))#74,17,2) AS year_total#76] (82) Exchange -Input [2]: [customer_id#51, year_total#52] -Arguments: hashpartitioning(customer_id#51, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [2]: [customer_id#75, year_total#76] +Arguments: hashpartitioning(customer_id#75, 5), ENSURE_REQUIREMENTS, [id=#77] (83) Sort [codegen id : 34] -Input [2]: [customer_id#51, year_total#52] -Arguments: [customer_id#51 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#75, year_total#76] +Arguments: [customer_id#75 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#51] -Join condition: (CASE WHEN (year_total#44 > 0.00) THEN CheckOverflow((promote_precision(year_total#52) / promote_precision(year_total#44)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#31) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#75] +Join condition: (CASE WHEN (year_total#59 > 0.00) THEN CheckOverflow((promote_precision(year_total#76) / promote_precision(year_total#59)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#40) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) (85) Project [codegen id : 35] -Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Input [9]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44, customer_id#51, year_total#52] +Output [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] +Input [9]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59, customer_id#75, year_total#76] (86) TakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: 100, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] +Input [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] +Arguments: 100, [customer_id#37 ASC NULLS FIRST, customer_id#37 ASC NULLS FIRST, customer_id#37 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39] ===== Subqueries ===== @@ -488,15 +488,15 @@ ReusedExchange (87) (87) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#5, d_year#6] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#21 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#25, d_year#26] -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#21 +Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt index db3a0a43be591..9fccc4c4ba66d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt @@ -171,282 +171,282 @@ Input [2]: [customer_id#17, year_total#18] Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) (20) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] (22) Filter [codegen id : 6] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Condition : (isnotnull(c_customer_sk#19) AND isnotnull(c_customer_id#20)) (23) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] (25) Filter [codegen id : 4] -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_customer_sk#5) +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_customer_sk#23) (26) BroadcastExchange -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#5] +Left keys [1]: [c_customer_sk#19] +Right keys [1]: [ss_customer_sk#23] Join condition: None (28) Project [codegen id : 6] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25] +Input [7]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#28, d_year#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#10, d_year#11] +Input [2]: [d_date_sk#28, d_year#29] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#10, d_year#11] -Condition : (((isnotnull(d_year#11) AND (d_year#11 = 2002)) AND d_year#11 IN (2001,2002)) AND isnotnull(d_date_sk#10)) +Input [2]: [d_date_sk#28, d_year#29] +Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) (32) BroadcastExchange -Input [2]: [d_date_sk#10, d_year#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Input [2]: [d_date_sk#28, d_year#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#25] +Right keys [1]: [d_date_sk#28] Join condition: None (34) Project [codegen id : 6] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] +Input [7]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#28, d_year#29] (35) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum#22] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] +Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] +Aggregate Attributes [1]: [sum#31] +Results [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] (36) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] +Arguments: hashpartitioning(c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, 5), ENSURE_REQUIREMENTS, [id=#33] (37) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#25] -Results [4]: [c_customer_id#2 AS customer_id#26, c_first_name#3 AS customer_first_name#27, c_last_name#4 AS customer_last_name#28, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#25,17,2) AS year_total#29] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] +Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] +Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#34] +Results [4]: [c_customer_id#20 AS customer_id#35, c_first_name#21 AS customer_first_name#36, c_last_name#22 AS customer_last_name#37, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#34,17,2) AS year_total#38] (38) BroadcastExchange -Input [4]: [customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] +Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#39] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#26] +Right keys [1]: [customer_id#35] Join condition: None (40) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] (42) Filter [codegen id : 10] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] +Condition : (isnotnull(c_customer_sk#40) AND isnotnull(c_customer_id#41)) (43) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] (45) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_bill_customer_sk#31) +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Condition : isnotnull(ws_bill_customer_sk#44) (46) BroadcastExchange -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#31] +Left keys [1]: [c_customer_sk#40] +Right keys [1]: [ws_bill_customer_sk#44] Join condition: None (48) Project [codegen id : 10] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46] +Input [7]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#48, d_year#49] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#46] +Right keys [1]: [d_date_sk#48] Join condition: None (51) Project [codegen id : 10] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] +Input [7]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46, d_date_sk#48, d_year#49] (52) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum#35] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] +Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#45))] +Aggregate Attributes [1]: [sum#50] +Results [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] (53) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] +Arguments: hashpartitioning(c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, 5), ENSURE_REQUIREMENTS, [id=#52] (54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#38] -Results [2]: [c_customer_id#2 AS customer_id#39, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#38,17,2) AS year_total#40] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] +Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] +Functions [1]: [sum(UnscaledValue(ws_net_paid#45))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#45))#53] +Results [2]: [c_customer_id#41 AS customer_id#54, MakeDecimal(sum(UnscaledValue(ws_net_paid#45))#53,17,2) AS year_total#55] (55) Filter [codegen id : 11] -Input [2]: [customer_id#39, year_total#40] -Condition : (isnotnull(year_total#40) AND (year_total#40 > 0.00)) +Input [2]: [customer_id#54, year_total#55] +Condition : (isnotnull(year_total#55) AND (year_total#55 > 0.00)) (56) Project [codegen id : 11] -Output [2]: [customer_id#39 AS customer_id#41, year_total#40 AS year_total#42] -Input [2]: [customer_id#39, year_total#40] +Output [2]: [customer_id#54 AS customer_id#56, year_total#55 AS year_total#57] +Input [2]: [customer_id#54, year_total#55] (57) BroadcastExchange -Input [2]: [customer_id#41, year_total#42] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#43] +Input [2]: [customer_id#56, year_total#57] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#58] (58) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#41] +Right keys [1]: [customer_id#56] Join condition: None (59) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42] -Input [8]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, customer_id#41, year_total#42] +Output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57] +Input [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, customer_id#56, year_total#57] (60) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] (62) Filter [codegen id : 14] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] +Condition : (isnotnull(c_customer_sk#59) AND isnotnull(c_customer_id#60)) (63) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] (65) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_bill_customer_sk#31) +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Condition : isnotnull(ws_bill_customer_sk#63) (66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#31] +Left keys [1]: [c_customer_sk#59] +Right keys [1]: [ws_bill_customer_sk#63] Join condition: None (68) Project [codegen id : 14] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65] +Input [7]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#67, d_year#68] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#65] +Right keys [1]: [d_date_sk#67] Join condition: None (71) Project [codegen id : 14] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] +Input [7]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65, d_date_sk#67, d_year#68] (72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum#45] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] +Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#64))] +Aggregate Attributes [1]: [sum#69] +Results [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] (73) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] +Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, 5), ENSURE_REQUIREMENTS, [id=#71] (74) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#48] -Results [2]: [c_customer_id#2 AS customer_id#49, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#48,17,2) AS year_total#50] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] +Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] +Functions [1]: [sum(UnscaledValue(ws_net_paid#64))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#64))#72] +Results [2]: [c_customer_id#60 AS customer_id#73, MakeDecimal(sum(UnscaledValue(ws_net_paid#64))#72,17,2) AS year_total#74] (75) BroadcastExchange -Input [2]: [customer_id#49, year_total#50] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#51] +Input [2]: [customer_id#73, year_total#74] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#75] (76) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#49] -Join condition: (CASE WHEN (year_total#42 > 0.00) THEN CheckOverflow((promote_precision(year_total#50) / promote_precision(year_total#42)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#29) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#73] +Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#57)), 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) (77) Project [codegen id : 16] -Output [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] -Input [9]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42, customer_id#49, year_total#50] +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#57, customer_id#73, year_total#74] (78) TakeOrderedAndProject -Input [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] -Arguments: 100, [customer_id#26 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST], [customer_id#26, customer_first_name#27, customer_last_name#28] +Input [3]: [customer_id#35, customer_first_name#36, customer_last_name#37] +Arguments: 100, [customer_id#35 ASC NULLS FIRST, customer_id#35 ASC NULLS FIRST, customer_id#35 ASC NULLS FIRST], [customer_id#35, customer_first_name#36, customer_last_name#37] ===== Subqueries ===== @@ -457,15 +457,15 @@ ReusedExchange (79) (79) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#10, d_year#11] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#19 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#28, d_year#29] -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#19 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#26 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt index f87f822166e23..a06e41562f8ff 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#31] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#36, d_year#37] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#36] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] -Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (42) Exchange -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_item_sk#47) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ws_item_sk#47] +Right keys [1]: [i_item_sk#52] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#57, d_year#58] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#57] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] (55) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] -Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (61) Exchange -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] +Right keys [2]: [wr_order_number#61, wr_item_sk#60] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] +Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#55, sum#56] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Aggregate Attributes [2]: [sum#69, sum#70] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] +Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Condition : isnotnull(cs_item_sk#79) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [cs_item_sk#79] +Right keys [1]: [i_item_sk#85] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] +Input [2]: [d_date_sk#90, d_year#91] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] -Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) +Input [2]: [d_date_sk#90, d_year#91] +Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) (83) BroadcastExchange -Input [2]: [d_date_sk#71, d_year#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [d_date_sk#90, d_year#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [cs_sold_date_sk#83] +Right keys [1]: [d_date_sk#90] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] (86) Exchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] -Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#2, cs_item_sk#1] -Right keys [2]: [cr_order_number#19, cr_item_sk#18] +Left keys [2]: [cs_order_number#80, cs_item_sk#79] +Right keys [2]: [cr_order_number#95, cr_item_sk#94] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Condition : isnotnull(ss_item_sk#26) +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Condition : isnotnull(ss_item_sk#98) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#98] +Right keys [1]: [i_item_sk#103] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#108, d_year#109] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ss_sold_date_sk#102] +Right keys [1]: [d_date_sk#108] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] (101) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] +Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] +Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Condition : isnotnull(ws_item_sk#116) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ws_item_sk#116] +Right keys [1]: [i_item_sk#121] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#126, d_year#127] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ws_sold_date_sk#120] +Right keys [1]: [d_date_sk#126] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] (116) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] +Right keys [2]: [wr_order_number#130, wr_item_sk#129] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] +Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Aggregate Attributes [2]: [sum#135, sum#136] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] (127) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] +Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] (129) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] (130) Sort [codegen id : 50] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 (131) 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#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] (133) TakeOrderedAndProject -Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] -Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt index f87f822166e23..a06e41562f8ff 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#31] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#36, d_year#37] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#36] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] -Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (42) Exchange -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_item_sk#47) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ws_item_sk#47] +Right keys [1]: [i_item_sk#52] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#57, d_year#58] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#57] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] (55) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] -Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (61) Exchange -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] +Right keys [2]: [wr_order_number#61, wr_item_sk#60] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] +Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#55, sum#56] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Aggregate Attributes [2]: [sum#69, sum#70] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] +Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Condition : isnotnull(cs_item_sk#79) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [cs_item_sk#79] +Right keys [1]: [i_item_sk#85] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] +Input [2]: [d_date_sk#90, d_year#91] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] -Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) +Input [2]: [d_date_sk#90, d_year#91] +Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) (83) BroadcastExchange -Input [2]: [d_date_sk#71, d_year#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [d_date_sk#90, d_year#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [cs_sold_date_sk#83] +Right keys [1]: [d_date_sk#90] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] (86) Exchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] -Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#2, cs_item_sk#1] -Right keys [2]: [cr_order_number#19, cr_item_sk#18] +Left keys [2]: [cs_order_number#80, cs_item_sk#79] +Right keys [2]: [cr_order_number#95, cr_item_sk#94] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Condition : isnotnull(ss_item_sk#26) +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Condition : isnotnull(ss_item_sk#98) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#98] +Right keys [1]: [i_item_sk#103] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#108, d_year#109] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ss_sold_date_sk#102] +Right keys [1]: [d_date_sk#108] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] (101) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] +Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] +Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Condition : isnotnull(ws_item_sk#116) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ws_item_sk#116] +Right keys [1]: [i_item_sk#121] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#126, d_year#127] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ws_sold_date_sk#120] +Right keys [1]: [d_date_sk#126] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] (116) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] +Right keys [2]: [wr_order_number#130, wr_item_sk#129] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] +Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Aggregate Attributes [2]: [sum#135, sum#136] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] (127) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] +Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] (129) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] (130) Sort [codegen id : 50] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 (131) 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#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] (133) TakeOrderedAndProject -Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] -Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt index 0092354e54be0..9284172139688 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt @@ -134,93 +134,93 @@ Input [4]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sol Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [id=#19] (20) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] (22) Filter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : isnotnull(d_date_sk#5) +Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] +Condition : isnotnull(d_date_sk#20) (23) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_sold_date_sk#18] -Right keys [1]: [d_date_sk#5] +Right keys [1]: [d_date_sk#20] Join condition: None (24) Project [codegen id : 5] -Output [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#6, d_qoy#7] -Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, d_date_sk#5, d_year#6, d_qoy#7] +Output [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#21, d_qoy#22] +Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, d_date_sk#20, d_year#21, d_qoy#22] (25) BroadcastExchange -Input [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +Input [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#21, d_qoy#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (26) Scan parquet default.item -Output [2]: [i_item_sk#9, i_category#10] +Output [2]: [i_item_sk#24, i_category#25] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (27) ColumnarToRow -Input [2]: [i_item_sk#9, i_category#10] +Input [2]: [i_item_sk#24, i_category#25] (28) Filter -Input [2]: [i_item_sk#9, i_category#10] -Condition : isnotnull(i_item_sk#9) +Input [2]: [i_item_sk#24, i_category#25] +Condition : isnotnull(i_item_sk#24) (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#15] -Right keys [1]: [i_item_sk#9] +Right keys [1]: [i_item_sk#24] Join condition: None (30) Project [codegen id : 6] -Output [6]: [web AS channel#21, ws_ship_customer_sk#16 AS col_name#22, d_year#6, d_qoy#7, i_category#10, ws_ext_sales_price#17 AS ext_sales_price#23] -Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#6, d_qoy#7, i_item_sk#9, i_category#10] +Output [6]: [web AS channel#26, ws_ship_customer_sk#16 AS col_name#27, d_year#21, d_qoy#22, i_category#25, ws_ext_sales_price#17 AS ext_sales_price#28] +Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#21, d_qoy#22, i_item_sk#24, i_category#25] (31) Scan parquet default.catalog_sales -Output [4]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +Output [4]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#27)] +PartitionFilters: [isnotnull(cs_sold_date_sk#32)] PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 9] -Input [4]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +Input [4]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32] (33) Filter [codegen id : 9] -Input [4]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] -Condition : (isnull(cs_ship_addr_sk#24) AND isnotnull(cs_item_sk#25)) +Input [4]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32] +Condition : (isnull(cs_ship_addr_sk#29) AND isnotnull(cs_item_sk#30)) (34) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] (35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [cs_sold_date_sk#32] +Right keys [1]: [d_date_sk#33] Join condition: None (36) Project [codegen id : 9] -Output [5]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_year#6, d_qoy#7] -Input [7]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#5, d_year#6, d_qoy#7] +Output [5]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, d_year#34, d_qoy#35] +Input [7]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32, d_date_sk#33, d_year#34, d_qoy#35] (37) ReusedExchange [Reuses operator id: 13] -Output [2]: [i_item_sk#9, i_category#10] +Output [2]: [i_item_sk#36, i_category#37] (38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#9] +Left keys [1]: [cs_item_sk#30] +Right keys [1]: [i_item_sk#36] Join condition: None (39) Project [codegen id : 9] -Output [6]: [catalog AS channel#28, cs_ship_addr_sk#24 AS col_name#29, d_year#6, d_qoy#7, i_category#10, cs_ext_sales_price#26 AS ext_sales_price#30] -Input [7]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_year#6, d_qoy#7, i_item_sk#9, i_category#10] +Output [6]: [catalog AS channel#38, cs_ship_addr_sk#29 AS col_name#39, d_year#34, d_qoy#35, i_category#37, cs_ext_sales_price#31 AS ext_sales_price#40] +Input [7]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, d_year#34, d_qoy#35, i_item_sk#36, i_category#37] (40) Union @@ -228,21 +228,21 @@ Input [7]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_year#6, Input [6]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, ext_sales_price#14] Keys [5]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10] Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count#31, sum#32] -Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#33, sum#34] +Aggregate Attributes [2]: [count#41, sum#42] +Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#43, sum#44] (42) Exchange -Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#33, sum#34] -Arguments: hashpartitioning(channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#43, sum#44] +Arguments: hashpartitioning(channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, 5), ENSURE_REQUIREMENTS, [id=#45] (43) HashAggregate [codegen id : 11] -Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#33, sum#34] +Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#43, sum#44] Keys [5]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count(1)#36, sum(UnscaledValue(ext_sales_price#14))#37] -Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count(1)#36 AS sales_cnt#38, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#37,17,2) AS sales_amt#39] +Aggregate Attributes [2]: [count(1)#46, sum(UnscaledValue(ext_sales_price#14))#47] +Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count(1)#46 AS sales_cnt#48, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#47,17,2) AS sales_amt#49] (44) TakeOrderedAndProject -Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#38, sales_amt#39] -Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#6 ASC NULLS FIRST, d_qoy#7 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#38, sales_amt#39] +Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#48, sales_amt#49] +Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#6 ASC NULLS FIRST, d_qoy#7 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#48, sales_amt#49] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt index 1250725d1782e..026c9396cd025 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt @@ -124,67 +124,67 @@ Input [4]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sol Condition : (isnull(ws_ship_customer_sk#16) AND isnotnull(ws_item_sk#15)) (19) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#5, i_category#6] +Output [2]: [i_item_sk#19, i_category#20] (20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#15] -Right keys [1]: [i_item_sk#5] +Right keys [1]: [i_item_sk#19] Join condition: None (21) Project [codegen id : 6] -Output [4]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#6] -Input [6]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_item_sk#5, i_category#6] +Output [4]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#20] +Input [6]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_item_sk#19, i_category#20] (22) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] +Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] (23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#18] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#21] Join condition: None (24) Project [codegen id : 6] -Output [6]: [web AS channel#19, ws_ship_customer_sk#16 AS col_name#20, d_year#9, d_qoy#10, i_category#6, ws_ext_sales_price#17 AS ext_sales_price#21] -Input [7]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#6, d_date_sk#8, d_year#9, d_qoy#10] +Output [6]: [web AS channel#24, ws_ship_customer_sk#16 AS col_name#25, d_year#22, d_qoy#23, i_category#20, ws_ext_sales_price#17 AS ext_sales_price#26] +Input [7]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#20, d_date_sk#21, d_year#22, d_qoy#23] (25) Scan parquet default.catalog_sales -Output [4]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25] +Output [4]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#25)] +PartitionFilters: [isnotnull(cs_sold_date_sk#30)] PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 9] -Input [4]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25] +Input [4]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30] (27) Filter [codegen id : 9] -Input [4]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25] -Condition : (isnull(cs_ship_addr_sk#22) AND isnotnull(cs_item_sk#23)) +Input [4]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30] +Condition : (isnull(cs_ship_addr_sk#27) AND isnotnull(cs_item_sk#28)) (28) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#5, i_category#6] +Output [2]: [i_item_sk#31, i_category#32] (29) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#23] -Right keys [1]: [i_item_sk#5] +Left keys [1]: [cs_item_sk#28] +Right keys [1]: [i_item_sk#31] Join condition: None (30) Project [codegen id : 9] -Output [4]: [cs_ship_addr_sk#22, cs_ext_sales_price#24, cs_sold_date_sk#25, i_category#6] -Input [6]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25, i_item_sk#5, i_category#6] +Output [4]: [cs_ship_addr_sk#27, cs_ext_sales_price#29, cs_sold_date_sk#30, i_category#32] +Input [6]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30, i_item_sk#31, i_category#32] (31) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] +Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#25] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#30] +Right keys [1]: [d_date_sk#33] Join condition: None (33) Project [codegen id : 9] -Output [6]: [catalog AS channel#26, cs_ship_addr_sk#22 AS col_name#27, d_year#9, d_qoy#10, i_category#6, cs_ext_sales_price#24 AS ext_sales_price#28] -Input [7]: [cs_ship_addr_sk#22, cs_ext_sales_price#24, cs_sold_date_sk#25, i_category#6, d_date_sk#8, d_year#9, d_qoy#10] +Output [6]: [catalog AS channel#36, cs_ship_addr_sk#27 AS col_name#37, d_year#34, d_qoy#35, i_category#32, cs_ext_sales_price#29 AS ext_sales_price#38] +Input [7]: [cs_ship_addr_sk#27, cs_ext_sales_price#29, cs_sold_date_sk#30, i_category#32, d_date_sk#33, d_year#34, d_qoy#35] (34) Union @@ -192,21 +192,21 @@ Input [7]: [cs_ship_addr_sk#22, cs_ext_sales_price#24, cs_sold_date_sk#25, i_cat Input [6]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, ext_sales_price#14] Keys [5]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6] Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count#29, sum#30] -Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] +Aggregate Attributes [2]: [count#39, sum#40] +Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#41, sum#42] (36) Exchange -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] -Arguments: hashpartitioning(channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#41, sum#42] +Arguments: hashpartitioning(channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, 5), ENSURE_REQUIREMENTS, [id=#43] (37) HashAggregate [codegen id : 11] -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] +Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#41, sum#42] Keys [5]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count(1)#34, sum(UnscaledValue(ext_sales_price#14))#35] -Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count(1)#34 AS sales_cnt#36, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#35,17,2) AS sales_amt#37] +Aggregate Attributes [2]: [count(1)#44, sum(UnscaledValue(ext_sales_price#14))#45] +Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count(1)#44 AS sales_cnt#46, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#45,17,2) AS sales_amt#47] (38) TakeOrderedAndProject -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#36, sales_amt#37] -Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#36, sales_amt#37] +Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#46, sales_amt#47] +Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#46, sales_amt#47] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt index a3626dfe3293d..4b2299ca2e749 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt @@ -209,38 +209,38 @@ Output [4]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_s Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25] (26) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#26] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Right keys [1]: [cast(d_date_sk#26 as bigint)] Join condition: None (28) Project [codegen id : 6] Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#6] +Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#26] (29) HashAggregate [codegen id : 6] Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] Keys [1]: [s_store_sk#25] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#26, sum#27] -Results [3]: [s_store_sk#25, sum#28, sum#29] +Aggregate Attributes [2]: [sum#27, sum#28] +Results [3]: [s_store_sk#25, sum#29, sum#30] (30) Exchange -Input [3]: [s_store_sk#25, sum#28, sum#29] -Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [s_store_sk#25, sum#29, sum#30] +Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#25, sum#28, sum#29] +Input [3]: [s_store_sk#25, sum#29, sum#30] Keys [1]: [s_store_sk#25] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] -Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] +Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] (32) BroadcastExchange -Input [3]: [s_store_sk#25, returns#33, profit_loss#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [3]: [s_store_sk#25, returns#34, profit_loss#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] @@ -248,269 +248,269 @@ Right keys [1]: [s_store_sk#25] Join condition: None (34) Project [codegen id : 8] -Output [5]: [sales#18, coalesce(returns#33, 0.00) AS returns#36, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#37, store channel AS channel#38, s_store_sk#9 AS id#39] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] +Output [5]: [sales#18, coalesce(returns#34, 0.00) AS returns#37, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#38, store channel AS channel#39, s_store_sk#9 AS id#40] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#34, profit_loss#35] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#45] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#43] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cs_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] +Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum#44, sum#45] -Results [3]: [cs_call_center_sk#40, sum#46, sum#47] +Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum#46, sum#47] +Results [3]: [cs_call_center_sk#41, sum#48, sum#49] (41) Exchange -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] (42) HashAggregate [codegen id : 14] -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] -Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] +Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] (43) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +Output [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#55), dynamicpruningexpression(cr_returned_date_sk#55 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#5)] ReadSchema: struct (44) ColumnarToRow [codegen id : 12] -Input [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +Input [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] (45) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#58] (46) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cr_returned_date_sk#55] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#57] +Right keys [1]: [d_date_sk#58] Join condition: None (47) Project [codegen id : 12] -Output [2]: [cr_return_amount#53, cr_net_loss#54] -Input [4]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55, d_date_sk#6] +Output [2]: [cr_return_amount#55, cr_net_loss#56] +Input [4]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57, d_date_sk#58] (48) HashAggregate [codegen id : 12] -Input [2]: [cr_return_amount#53, cr_net_loss#54] +Input [2]: [cr_return_amount#55, cr_net_loss#56] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#53)), partial_sum(UnscaledValue(cr_net_loss#54))] -Aggregate Attributes [2]: [sum#56, sum#57] -Results [2]: [sum#58, sum#59] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#55)), partial_sum(UnscaledValue(cr_net_loss#56))] +Aggregate Attributes [2]: [sum#59, sum#60] +Results [2]: [sum#61, sum#62] (49) Exchange -Input [2]: [sum#58, sum#59] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#60] +Input [2]: [sum#61, sum#62] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] (50) HashAggregate [codegen id : 13] -Input [2]: [sum#58, sum#59] +Input [2]: [sum#61, sum#62] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#53)), sum(UnscaledValue(cr_net_loss#54))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#53))#61, sum(UnscaledValue(cr_net_loss#54))#62] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#53))#61,17,2) AS returns#63, MakeDecimal(sum(UnscaledValue(cr_net_loss#54))#62,17,2) AS profit_loss#64] +Functions [2]: [sum(UnscaledValue(cr_return_amount#55)), sum(UnscaledValue(cr_net_loss#56))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#55))#64, sum(UnscaledValue(cr_net_loss#56))#65] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#55))#64,17,2) AS returns#66, MakeDecimal(sum(UnscaledValue(cr_net_loss#56))#65,17,2) AS profit_loss#67] (51) BroadcastExchange -Input [2]: [returns#63, profit_loss#64] -Arguments: IdentityBroadcastMode, [id=#65] +Input [2]: [returns#66, profit_loss#67] +Arguments: IdentityBroadcastMode, [id=#68] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [sales#51, returns#63, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#64 as decimal(18,2)))), DecimalType(18,2), true) AS profit#66, catalog channel AS channel#67, cs_call_center_sk#40 AS id#68] -Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#63, profit_loss#64] +Output [5]: [sales#53, returns#66, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#67 as decimal(18,2)))), DecimalType(18,2), true) AS profit#69, catalog channel AS channel#70, cs_call_center_sk#41 AS id#71] +Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#66, profit_loss#67] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] -Condition : isnotnull(ws_web_page_sk#69) +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Condition : isnotnull(ws_web_page_sk#72) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#76] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#72] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#75] +Right keys [1]: [d_date_sk#76] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] -Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] +Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] +Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#73] +Output [1]: [wp_web_page_sk#77] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#73] +Input [1]: [wp_web_page_sk#77] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#73] -Condition : isnotnull(wp_web_page_sk#73) +Input [1]: [wp_web_page_sk#77] +Condition : isnotnull(wp_web_page_sk#77) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [1]: [wp_web_page_sk#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#69] -Right keys [1]: [wp_web_page_sk#73] +Left keys [1]: [ws_web_page_sk#72] +Right keys [1]: [wp_web_page_sk#77] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum#75, sum#76] -Results [3]: [wp_web_page_sk#73, sum#77, sum#78] +Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum#79, sum#80] +Results [3]: [wp_web_page_sk#77, sum#81, sum#82] (67) Exchange -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] -Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] +Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : isnotnull(wr_web_page_sk#84) +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Condition : isnotnull(wr_web_page_sk#88) (72) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#88] +Output [1]: [wp_web_page_sk#92] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#84] -Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] +Left keys [1]: [wr_web_page_sk#88] +Right keys [1]: [cast(wp_web_page_sk#92 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [4]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] -Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] +Output [4]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] +Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] (75) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#93] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#87] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#91] +Right keys [1]: [cast(d_date_sk#93 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Input [5]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88, d_date_sk#6] +Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] +Input [5]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92, d_date_sk#93] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum#89, sum#90] -Results [3]: [wp_web_page_sk#88, sum#91, sum#92] +Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] +Keys [1]: [wp_web_page_sk#92] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum#94, sum#95] +Results [3]: [wp_web_page_sk#92, sum#96, sum#97] (79) Exchange -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [3]: [wp_web_page_sk#92, sum#96, sum#97] +Arguments: hashpartitioning(wp_web_page_sk#92, 5), ENSURE_REQUIREMENTS, [id=#98] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] -Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] +Input [3]: [wp_web_page_sk#92, sum#96, sum#97] +Keys [1]: [wp_web_page_sk#92] +Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] +Results [3]: [wp_web_page_sk#92, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] +Input [3]: [wp_web_page_sk#92, returns#101, profit_loss#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#73] -Right keys [1]: [wp_web_page_sk#88] +Left keys [1]: [wp_web_page_sk#77] +Right keys [1]: [wp_web_page_sk#92] Join condition: None (83) Project [codegen id : 22] -Output [5]: [sales#82, coalesce(returns#96, 0.00) AS returns#99, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#100, web channel AS channel#101, wp_web_page_sk#73 AS id#102] -Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] +Output [5]: [sales#86, coalesce(returns#101, 0.00) AS returns#104, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#105, web channel AS channel#106, wp_web_page_sk#77 AS id#107] +Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#92, returns#101, profit_loss#102] (84) Union (85) Expand [codegen id : 23] -Input [5]: [sales#18, returns#36, profit#37, channel#38, id#39] -Arguments: [List(sales#18, returns#36, profit#37, channel#38, id#39, 0), List(sales#18, returns#36, profit#37, channel#38, null, 1), List(sales#18, returns#36, profit#37, null, null, 3)], [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] +Input [5]: [sales#18, returns#37, profit#38, channel#39, id#40] +Arguments: [List(sales#18, returns#37, profit#38, channel#39, id#40, 0), List(sales#18, returns#37, profit#38, channel#39, null, 1), List(sales#18, returns#37, profit#38, null, null, 3)], [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] (86) HashAggregate [codegen id : 23] -Input [6]: [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] -Keys [3]: [channel#103, id#104, spark_grouping_id#105] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#36), partial_sum(profit#37)] -Aggregate Attributes [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Results [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Input [6]: [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] +Keys [3]: [channel#108, id#109, spark_grouping_id#110] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#37), partial_sum(profit#38)] +Aggregate Attributes [6]: [sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] +Results [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] (87) Exchange -Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Arguments: hashpartitioning(channel#103, id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, [id=#118] +Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Arguments: hashpartitioning(channel#108, id#109, spark_grouping_id#110, 5), ENSURE_REQUIREMENTS, [id=#123] (88) HashAggregate [codegen id : 24] -Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Keys [3]: [channel#103, id#104, spark_grouping_id#105] -Functions [3]: [sum(sales#18), sum(returns#36), sum(profit#37)] -Aggregate Attributes [3]: [sum(sales#18)#119, sum(returns#36)#120, sum(profit#37)#121] -Results [5]: [channel#103, id#104, sum(sales#18)#119 AS sales#122, sum(returns#36)#120 AS returns#123, sum(profit#37)#121 AS profit#124] +Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Keys [3]: [channel#108, id#109, spark_grouping_id#110] +Functions [3]: [sum(sales#18), sum(returns#37), sum(profit#38)] +Aggregate Attributes [3]: [sum(sales#18)#124, sum(returns#37)#125, sum(profit#38)#126] +Results [5]: [channel#108, id#109, sum(sales#18)#124 AS sales#127, sum(returns#37)#125 AS returns#128, sum(profit#38)#126 AS profit#129] (89) TakeOrderedAndProject -Input [5]: [channel#103, id#104, sales#122, returns#123, profit#124] -Arguments: 100, [channel#103 ASC NULLS FIRST, id#104 ASC NULLS FIRST], [channel#103, id#104, sales#122, returns#123, profit#124] +Input [5]: [channel#108, id#109, sales#127, returns#128, profit#129] +Arguments: 100, [channel#108 ASC NULLS FIRST, id#109 ASC NULLS FIRST], [channel#108, id#109, sales#127, returns#128, profit#129] ===== Subqueries ===== @@ -526,14 +526,14 @@ ReusedExchange (91) (91) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#26] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#55 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt index 7a277d5362e32..618da39637e23 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt @@ -197,320 +197,320 @@ Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_s Condition : isnotnull(sr_store_sk#20) (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#25] (24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None (25) Project [codegen id : 6] Output [3]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22] -Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#6] +Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#25] (26) ReusedExchange [Reuses operator id: 14] -Output [1]: [s_store_sk#25] +Output [1]: [s_store_sk#26] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_store_sk#20] -Right keys [1]: [cast(s_store_sk#25 as bigint)] +Right keys [1]: [cast(s_store_sk#26 as bigint)] Join condition: None (28) Project [codegen id : 6] -Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#25] +Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] +Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#26] (29) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Keys [1]: [s_store_sk#25] +Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] +Keys [1]: [s_store_sk#26] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#26, sum#27] -Results [3]: [s_store_sk#25, sum#28, sum#29] +Aggregate Attributes [2]: [sum#27, sum#28] +Results [3]: [s_store_sk#26, sum#29, sum#30] (30) Exchange -Input [3]: [s_store_sk#25, sum#28, sum#29] -Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [s_store_sk#26, sum#29, sum#30] +Arguments: hashpartitioning(s_store_sk#26, 5), ENSURE_REQUIREMENTS, [id=#31] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#25, sum#28, sum#29] -Keys [1]: [s_store_sk#25] +Input [3]: [s_store_sk#26, sum#29, sum#30] +Keys [1]: [s_store_sk#26] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] -Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] +Results [3]: [s_store_sk#26, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] (32) BroadcastExchange -Input [3]: [s_store_sk#25, returns#33, profit_loss#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [3]: [s_store_sk#26, returns#34, profit_loss#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] -Right keys [1]: [s_store_sk#25] +Right keys [1]: [s_store_sk#26] Join condition: None (34) Project [codegen id : 8] -Output [5]: [sales#18, coalesce(returns#33, 0.00) AS returns#36, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#37, store channel AS channel#38, s_store_sk#9 AS id#39] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] +Output [5]: [sales#18, coalesce(returns#34, 0.00) AS returns#37, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#38, store channel AS channel#39, s_store_sk#9 AS id#40] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#26, returns#34, profit_loss#35] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#45] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#43] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cs_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] +Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum#44, sum#45] -Results [3]: [cs_call_center_sk#40, sum#46, sum#47] +Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum#46, sum#47] +Results [3]: [cs_call_center_sk#41, sum#48, sum#49] (41) Exchange -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] -Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] +Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] (43) BroadcastExchange -Input [3]: [cs_call_center_sk#40, sales#51, profit#52] -Arguments: IdentityBroadcastMode, [id=#53] +Input [3]: [cs_call_center_sk#41, sales#53, profit#54] +Arguments: IdentityBroadcastMode, [id=#55] (44) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Output [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#58), dynamicpruningexpression(cr_returned_date_sk#58 IN dynamicpruning#5)] ReadSchema: struct (45) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Input [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] (46) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#59] (47) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#56] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#58] +Right keys [1]: [d_date_sk#59] Join condition: None (48) Project [codegen id : 13] -Output [2]: [cr_return_amount#54, cr_net_loss#55] -Input [4]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56, d_date_sk#6] +Output [2]: [cr_return_amount#56, cr_net_loss#57] +Input [4]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58, d_date_sk#59] (49) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#54, cr_net_loss#55] +Input [2]: [cr_return_amount#56, cr_net_loss#57] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#54)), partial_sum(UnscaledValue(cr_net_loss#55))] -Aggregate Attributes [2]: [sum#57, sum#58] -Results [2]: [sum#59, sum#60] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#56)), partial_sum(UnscaledValue(cr_net_loss#57))] +Aggregate Attributes [2]: [sum#60, sum#61] +Results [2]: [sum#62, sum#63] (50) Exchange -Input [2]: [sum#59, sum#60] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] +Input [2]: [sum#62, sum#63] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#64] (51) HashAggregate -Input [2]: [sum#59, sum#60] +Input [2]: [sum#62, sum#63] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#54)), sum(UnscaledValue(cr_net_loss#55))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#54))#62, sum(UnscaledValue(cr_net_loss#55))#63] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#54))#62,17,2) AS returns#64, MakeDecimal(sum(UnscaledValue(cr_net_loss#55))#63,17,2) AS profit_loss#65] +Functions [2]: [sum(UnscaledValue(cr_return_amount#56)), sum(UnscaledValue(cr_net_loss#57))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#56))#65, sum(UnscaledValue(cr_net_loss#57))#66] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#56))#65,17,2) AS returns#67, MakeDecimal(sum(UnscaledValue(cr_net_loss#57))#66,17,2) AS profit_loss#68] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [sales#51, returns#64, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#65 as decimal(18,2)))), DecimalType(18,2), true) AS profit#66, catalog channel AS channel#67, cs_call_center_sk#40 AS id#68] -Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#64, profit_loss#65] +Output [5]: [sales#53, returns#67, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#68 as decimal(18,2)))), DecimalType(18,2), true) AS profit#69, catalog channel AS channel#70, cs_call_center_sk#41 AS id#71] +Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#67, profit_loss#68] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] -Condition : isnotnull(ws_web_page_sk#69) +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Condition : isnotnull(ws_web_page_sk#72) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#76] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#72] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#75] +Right keys [1]: [d_date_sk#76] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] -Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] +Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] +Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#73] +Output [1]: [wp_web_page_sk#77] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#73] +Input [1]: [wp_web_page_sk#77] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#73] -Condition : isnotnull(wp_web_page_sk#73) +Input [1]: [wp_web_page_sk#77] +Condition : isnotnull(wp_web_page_sk#77) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [1]: [wp_web_page_sk#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#69] -Right keys [1]: [wp_web_page_sk#73] +Left keys [1]: [ws_web_page_sk#72] +Right keys [1]: [wp_web_page_sk#77] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum#75, sum#76] -Results [3]: [wp_web_page_sk#73, sum#77, sum#78] +Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum#79, sum#80] +Results [3]: [wp_web_page_sk#77, sum#81, sum#82] (67) Exchange -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] -Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] +Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : isnotnull(wr_web_page_sk#84) +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Condition : isnotnull(wr_web_page_sk#88) (72) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#92] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#87] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#91] +Right keys [1]: [cast(d_date_sk#92 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86] -Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, d_date_sk#6] +Output [3]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90] +Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, d_date_sk#92] (75) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#88] +Output [1]: [wp_web_page_sk#93] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#84] -Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] +Left keys [1]: [wr_web_page_sk#88] +Right keys [1]: [cast(wp_web_page_sk#93 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum#89, sum#90] -Results [3]: [wp_web_page_sk#88, sum#91, sum#92] +Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] +Keys [1]: [wp_web_page_sk#93] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum#94, sum#95] +Results [3]: [wp_web_page_sk#93, sum#96, sum#97] (79) Exchange -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [3]: [wp_web_page_sk#93, sum#96, sum#97] +Arguments: hashpartitioning(wp_web_page_sk#93, 5), ENSURE_REQUIREMENTS, [id=#98] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] -Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] +Input [3]: [wp_web_page_sk#93, sum#96, sum#97] +Keys [1]: [wp_web_page_sk#93] +Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] +Results [3]: [wp_web_page_sk#93, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] +Input [3]: [wp_web_page_sk#93, returns#101, profit_loss#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#73] -Right keys [1]: [wp_web_page_sk#88] +Left keys [1]: [wp_web_page_sk#77] +Right keys [1]: [wp_web_page_sk#93] Join condition: None (83) Project [codegen id : 22] -Output [5]: [sales#82, coalesce(returns#96, 0.00) AS returns#99, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#100, web channel AS channel#101, wp_web_page_sk#73 AS id#102] -Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] +Output [5]: [sales#86, coalesce(returns#101, 0.00) AS returns#104, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#105, web channel AS channel#106, wp_web_page_sk#77 AS id#107] +Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#93, returns#101, profit_loss#102] (84) Union (85) Expand [codegen id : 23] -Input [5]: [sales#18, returns#36, profit#37, channel#38, id#39] -Arguments: [List(sales#18, returns#36, profit#37, channel#38, id#39, 0), List(sales#18, returns#36, profit#37, channel#38, null, 1), List(sales#18, returns#36, profit#37, null, null, 3)], [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] +Input [5]: [sales#18, returns#37, profit#38, channel#39, id#40] +Arguments: [List(sales#18, returns#37, profit#38, channel#39, id#40, 0), List(sales#18, returns#37, profit#38, channel#39, null, 1), List(sales#18, returns#37, profit#38, null, null, 3)], [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] (86) HashAggregate [codegen id : 23] -Input [6]: [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] -Keys [3]: [channel#103, id#104, spark_grouping_id#105] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#36), partial_sum(profit#37)] -Aggregate Attributes [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Results [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Input [6]: [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] +Keys [3]: [channel#108, id#109, spark_grouping_id#110] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#37), partial_sum(profit#38)] +Aggregate Attributes [6]: [sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] +Results [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] (87) Exchange -Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Arguments: hashpartitioning(channel#103, id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, [id=#118] +Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Arguments: hashpartitioning(channel#108, id#109, spark_grouping_id#110, 5), ENSURE_REQUIREMENTS, [id=#123] (88) HashAggregate [codegen id : 24] -Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Keys [3]: [channel#103, id#104, spark_grouping_id#105] -Functions [3]: [sum(sales#18), sum(returns#36), sum(profit#37)] -Aggregate Attributes [3]: [sum(sales#18)#119, sum(returns#36)#120, sum(profit#37)#121] -Results [5]: [channel#103, id#104, sum(sales#18)#119 AS sales#122, sum(returns#36)#120 AS returns#123, sum(profit#37)#121 AS profit#124] +Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Keys [3]: [channel#108, id#109, spark_grouping_id#110] +Functions [3]: [sum(sales#18), sum(returns#37), sum(profit#38)] +Aggregate Attributes [3]: [sum(sales#18)#124, sum(returns#37)#125, sum(profit#38)#126] +Results [5]: [channel#108, id#109, sum(sales#18)#124 AS sales#127, sum(returns#37)#125 AS returns#128, sum(profit#38)#126 AS profit#129] (89) TakeOrderedAndProject -Input [5]: [channel#103, id#104, sales#122, returns#123, profit#124] -Arguments: 100, [channel#103 ASC NULLS FIRST, id#104 ASC NULLS FIRST], [channel#103, id#104, sales#122, returns#123, profit#124] +Input [5]: [channel#108, id#109, sales#127, returns#128, profit#129] +Arguments: 100, [channel#108 ASC NULLS FIRST, id#109 ASC NULLS FIRST], [channel#108, id#109, sales#127, returns#128, profit#129] ===== Subqueries ===== @@ -526,14 +526,14 @@ ReusedExchange (91) (91) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#25] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#58 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt index ec22a1048c116..b74247d7aef3f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt @@ -248,164 +248,164 @@ Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale Input [9]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#43, d_year#44] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#43] Join condition: None (41) Project [codegen id : 13] -Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] -Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#14, d_year#15] +Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] +Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#43, d_year#44] (42) HashAggregate [codegen id : 13] -Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] -Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] +Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [partial_sum(cs_quantity#34), partial_sum(UnscaledValue(cs_wholesale_cost#35)), partial_sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum#43, sum#44, sum#45] -Results [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] +Aggregate Attributes [3]: [sum#45, sum#46, sum#47] +Results [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] (43) Exchange -Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] -Arguments: hashpartitioning(d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] +Arguments: hashpartitioning(d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#51] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] -Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] +Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [sum(cs_quantity#34), sum(UnscaledValue(cs_wholesale_cost#35)), sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum(cs_quantity#34)#50, sum(UnscaledValue(cs_wholesale_cost#35))#51, sum(UnscaledValue(cs_sales_price#36))#52] -Results [6]: [d_year#15 AS cs_sold_year#53, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#54, sum(cs_quantity#34)#50 AS cs_qty#55, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#51,17,2) AS cs_wc#56, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#52,17,2) AS cs_sp#57] +Aggregate Attributes [3]: [sum(cs_quantity#34)#52, sum(UnscaledValue(cs_wholesale_cost#35))#53, sum(UnscaledValue(cs_sales_price#36))#54] +Results [6]: [d_year#44 AS cs_sold_year#55, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#56, sum(cs_quantity#34)#52 AS cs_qty#57, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#53,17,2) AS cs_wc#58, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#54,17,2) AS cs_sp#59] (45) Filter [codegen id : 14] -Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] -Condition : (coalesce(cs_qty#55, 0) > 0) +Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] +Condition : (coalesce(cs_qty#57, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] -Arguments: [cs_sold_year#53 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] +Arguments: [cs_sold_year#55 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#56 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54] +Right keys [3]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] (49) Scan parquet default.web_sales -Output [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Output [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] (51) Filter [codegen id : 16] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Condition : (isnotnull(ws_item_sk#58) AND isnotnull(ws_bill_customer_sk#59)) +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Condition : (isnotnull(ws_item_sk#60) AND isnotnull(ws_bill_customer_sk#61)) (52) Exchange -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Arguments: hashpartitioning(cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint), 5), ENSURE_REQUIREMENTS, [id=#65] +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Arguments: hashpartitioning(cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint), 5), ENSURE_REQUIREMENTS, [id=#67] (53) Sort [codegen id : 17] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Arguments: [cast(ws_order_number#60 as bigint) ASC NULLS FIRST, cast(ws_item_sk#58 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Arguments: [cast(ws_order_number#62 as bigint) ASC NULLS FIRST, cast(ws_item_sk#60 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.web_returns -Output [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Output [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] (56) Filter [codegen id : 18] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] -Condition : (isnotnull(wr_order_number#67) AND isnotnull(wr_item_sk#66)) +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Condition : (isnotnull(wr_order_number#69) AND isnotnull(wr_item_sk#68)) (57) Project [codegen id : 18] -Output [2]: [wr_item_sk#66, wr_order_number#67] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Output [2]: [wr_item_sk#68, wr_order_number#69] +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] (58) Exchange -Input [2]: [wr_item_sk#66, wr_order_number#67] -Arguments: hashpartitioning(wr_order_number#67, wr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [wr_item_sk#68, wr_order_number#69] +Arguments: hashpartitioning(wr_order_number#69, wr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] (59) Sort [codegen id : 19] -Input [2]: [wr_item_sk#66, wr_order_number#67] -Arguments: [wr_order_number#67 ASC NULLS FIRST, wr_item_sk#66 ASC NULLS FIRST], false, 0 +Input [2]: [wr_item_sk#68, wr_order_number#69] +Arguments: [wr_order_number#69 ASC NULLS FIRST, wr_item_sk#68 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint)] -Right keys [2]: [wr_order_number#67, wr_item_sk#66] +Left keys [2]: [cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint)] +Right keys [2]: [wr_order_number#69, wr_item_sk#68] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] -Condition : isnull(wr_order_number#67) +Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] +Condition : isnull(wr_order_number#69) (62) Project [codegen id : 21] -Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] +Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#72, d_year#73] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [ws_sold_date_sk#64] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#66] +Right keys [1]: [d_date_sk#72] Join condition: None (65) Project [codegen id : 21] -Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] -Input [8]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, d_date_sk#14, d_year#15] +Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] +Input [8]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, d_date_sk#72, d_year#73] (66) HashAggregate [codegen id : 21] -Input [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] -Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] -Functions [3]: [partial_sum(ws_quantity#61), partial_sum(UnscaledValue(ws_wholesale_cost#62)), partial_sum(UnscaledValue(ws_sales_price#63))] -Aggregate Attributes [3]: [sum#70, sum#71, sum#72] -Results [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] +Input [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] +Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] +Functions [3]: [partial_sum(ws_quantity#63), partial_sum(UnscaledValue(ws_wholesale_cost#64)), partial_sum(UnscaledValue(ws_sales_price#65))] +Aggregate Attributes [3]: [sum#74, sum#75, sum#76] +Results [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] (67) Exchange -Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] +Arguments: hashpartitioning(d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#80] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] -Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] -Functions [3]: [sum(ws_quantity#61), sum(UnscaledValue(ws_wholesale_cost#62)), sum(UnscaledValue(ws_sales_price#63))] -Aggregate Attributes [3]: [sum(ws_quantity#61)#77, sum(UnscaledValue(ws_wholesale_cost#62))#78, sum(UnscaledValue(ws_sales_price#63))#79] -Results [6]: [d_year#15 AS ws_sold_year#80, ws_item_sk#58, ws_bill_customer_sk#59 AS ws_customer_sk#81, sum(ws_quantity#61)#77 AS ws_qty#82, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#62))#78,17,2) AS ws_wc#83, MakeDecimal(sum(UnscaledValue(ws_sales_price#63))#79,17,2) AS ws_sp#84] +Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] +Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] +Functions [3]: [sum(ws_quantity#63), sum(UnscaledValue(ws_wholesale_cost#64)), sum(UnscaledValue(ws_sales_price#65))] +Aggregate Attributes [3]: [sum(ws_quantity#63)#81, sum(UnscaledValue(ws_wholesale_cost#64))#82, sum(UnscaledValue(ws_sales_price#65))#83] +Results [6]: [d_year#73 AS ws_sold_year#84, ws_item_sk#60, ws_bill_customer_sk#61 AS ws_customer_sk#85, sum(ws_quantity#63)#81 AS ws_qty#86, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#64))#82,17,2) AS ws_wc#87, MakeDecimal(sum(UnscaledValue(ws_sales_price#65))#83,17,2) AS ws_sp#88] (69) Filter [codegen id : 22] -Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] -Condition : (coalesce(ws_qty#82, 0) > 0) +Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] +Condition : (coalesce(ws_qty#86, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] -Arguments: [ws_sold_year#80 ASC NULLS FIRST, ws_item_sk#58 ASC NULLS FIRST, ws_customer_sk#81 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] +Arguments: [ws_sold_year#84 ASC NULLS FIRST, ws_item_sk#60 ASC NULLS FIRST, ws_customer_sk#85 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81] +Right keys [3]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85] Join condition: None (72) Project [codegen id : 23] -Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#82 + cs_qty#55), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#82, 0) + coalesce(cs_qty#55, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#83, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#56, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#84, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#57, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#82, cs_qty#55] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57, ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] +Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#86 + cs_qty#57), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#86, 0) + coalesce(cs_qty#57, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#87, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#58, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#88, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#59, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#86, cs_qty#57] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59, ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] (73) TakeOrderedAndProject -Input [12]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#82, cs_qty#55] -Arguments: 100, [ratio#85 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#82 + cs_qty#55), 1) as double)), 2) ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] +Input [12]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#86, cs_qty#57] +Arguments: 100, [ratio#89 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#86 + cs_qty#57), 1) as double)), 2) ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt index f5eff516f9b36..c7da3a1e23c62 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt @@ -248,164 +248,164 @@ Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale Input [9]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_order_number#33, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, wr_item_sk#39, wr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#43, d_year#44] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#37] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#43] Join condition: None (41) Project [codegen id : 13] -Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] -Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#14, d_year#15] +Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] +Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#43, d_year#44] (42) HashAggregate [codegen id : 13] -Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] -Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] +Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [partial_sum(ws_quantity#34), partial_sum(UnscaledValue(ws_wholesale_cost#35)), partial_sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum#43, sum#44, sum#45] -Results [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] +Aggregate Attributes [3]: [sum#45, sum#46, sum#47] +Results [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] (43) Exchange -Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] -Arguments: hashpartitioning(d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] +Arguments: hashpartitioning(d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#51] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] -Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] +Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [sum(ws_quantity#34), sum(UnscaledValue(ws_wholesale_cost#35)), sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum(ws_quantity#34)#50, sum(UnscaledValue(ws_wholesale_cost#35))#51, sum(UnscaledValue(ws_sales_price#36))#52] -Results [6]: [d_year#15 AS ws_sold_year#53, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#54, sum(ws_quantity#34)#50 AS ws_qty#55, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#51,17,2) AS ws_wc#56, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#52,17,2) AS ws_sp#57] +Aggregate Attributes [3]: [sum(ws_quantity#34)#52, sum(UnscaledValue(ws_wholesale_cost#35))#53, sum(UnscaledValue(ws_sales_price#36))#54] +Results [6]: [d_year#44 AS ws_sold_year#55, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#56, sum(ws_quantity#34)#52 AS ws_qty#57, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#53,17,2) AS ws_wc#58, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#54,17,2) AS ws_sp#59] (45) Filter [codegen id : 14] -Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] -Condition : (coalesce(ws_qty#55, 0) > 0) +Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] +Condition : (coalesce(ws_qty#57, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] -Arguments: [ws_sold_year#53 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] +Arguments: [ws_sold_year#55 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#56 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54] +Right keys [3]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] (49) Scan parquet default.catalog_sales -Output [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Output [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#66), dynamicpruningexpression(cs_sold_date_sk#66 IN dynamicpruning#8)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] (51) Filter [codegen id : 16] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Condition : (isnotnull(cs_item_sk#59) AND isnotnull(cs_bill_customer_sk#58)) +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Condition : (isnotnull(cs_item_sk#61) AND isnotnull(cs_bill_customer_sk#60)) (52) Exchange -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Arguments: hashpartitioning(cs_order_number#60, cs_item_sk#59, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Arguments: hashpartitioning(cs_order_number#62, cs_item_sk#61, 5), ENSURE_REQUIREMENTS, [id=#67] (53) Sort [codegen id : 17] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Arguments: [cs_order_number#60 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST], false, 0 +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Arguments: [cs_order_number#62 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST], false, 0 (54) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Output [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] (56) Filter [codegen id : 18] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] -Condition : (isnotnull(cr_order_number#67) AND isnotnull(cr_item_sk#66)) +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Condition : (isnotnull(cr_order_number#69) AND isnotnull(cr_item_sk#68)) (57) Project [codegen id : 18] -Output [2]: [cr_item_sk#66, cr_order_number#67] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Output [2]: [cr_item_sk#68, cr_order_number#69] +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] (58) Exchange -Input [2]: [cr_item_sk#66, cr_order_number#67] -Arguments: hashpartitioning(cr_order_number#67, cr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [cr_item_sk#68, cr_order_number#69] +Arguments: hashpartitioning(cr_order_number#69, cr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] (59) Sort [codegen id : 19] -Input [2]: [cr_item_sk#66, cr_order_number#67] -Arguments: [cr_order_number#67 ASC NULLS FIRST, cr_item_sk#66 ASC NULLS FIRST], false, 0 +Input [2]: [cr_item_sk#68, cr_order_number#69] +Arguments: [cr_order_number#69 ASC NULLS FIRST, cr_item_sk#68 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cs_order_number#60, cs_item_sk#59] -Right keys [2]: [cr_order_number#67, cr_item_sk#66] +Left keys [2]: [cs_order_number#62, cs_item_sk#61] +Right keys [2]: [cr_order_number#69, cr_item_sk#68] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] -Condition : isnull(cr_order_number#67) +Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] +Condition : isnull(cr_order_number#69) (62) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] +Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#72, d_year#73] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [cs_sold_date_sk#64] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [cs_sold_date_sk#66] +Right keys [1]: [d_date_sk#72] Join condition: None (65) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] -Input [8]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, d_date_sk#14, d_year#15] +Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] +Input [8]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, d_date_sk#72, d_year#73] (66) HashAggregate [codegen id : 21] -Input [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] -Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] -Functions [3]: [partial_sum(cs_quantity#61), partial_sum(UnscaledValue(cs_wholesale_cost#62)), partial_sum(UnscaledValue(cs_sales_price#63))] -Aggregate Attributes [3]: [sum#70, sum#71, sum#72] -Results [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] +Input [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] +Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] +Functions [3]: [partial_sum(cs_quantity#63), partial_sum(UnscaledValue(cs_wholesale_cost#64)), partial_sum(UnscaledValue(cs_sales_price#65))] +Aggregate Attributes [3]: [sum#74, sum#75, sum#76] +Results [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] (67) Exchange -Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] +Arguments: hashpartitioning(d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#80] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] -Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] -Functions [3]: [sum(cs_quantity#61), sum(UnscaledValue(cs_wholesale_cost#62)), sum(UnscaledValue(cs_sales_price#63))] -Aggregate Attributes [3]: [sum(cs_quantity#61)#77, sum(UnscaledValue(cs_wholesale_cost#62))#78, sum(UnscaledValue(cs_sales_price#63))#79] -Results [6]: [d_year#15 AS cs_sold_year#80, cs_item_sk#59, cs_bill_customer_sk#58 AS cs_customer_sk#81, sum(cs_quantity#61)#77 AS cs_qty#82, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#62))#78,17,2) AS cs_wc#83, MakeDecimal(sum(UnscaledValue(cs_sales_price#63))#79,17,2) AS cs_sp#84] +Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] +Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] +Functions [3]: [sum(cs_quantity#63), sum(UnscaledValue(cs_wholesale_cost#64)), sum(UnscaledValue(cs_sales_price#65))] +Aggregate Attributes [3]: [sum(cs_quantity#63)#81, sum(UnscaledValue(cs_wholesale_cost#64))#82, sum(UnscaledValue(cs_sales_price#65))#83] +Results [6]: [d_year#73 AS cs_sold_year#84, cs_item_sk#61, cs_bill_customer_sk#60 AS cs_customer_sk#85, sum(cs_quantity#63)#81 AS cs_qty#86, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#64))#82,17,2) AS cs_wc#87, MakeDecimal(sum(UnscaledValue(cs_sales_price#65))#83,17,2) AS cs_sp#88] (69) Filter [codegen id : 22] -Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] -Condition : (coalesce(cs_qty#82, 0) > 0) +Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] +Condition : (coalesce(cs_qty#86, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] -Arguments: [cs_sold_year#80 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST, cs_customer_sk#81 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] +Arguments: [cs_sold_year#84 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST, cs_customer_sk#85 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81] +Right keys [3]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85] Join condition: None (72) Project [codegen id : 23] -Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#55 + cs_qty#82), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#55, 0) + coalesce(cs_qty#82, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#56, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#83, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#57, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#84, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, cs_qty#82] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57, cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] +Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#57 + cs_qty#86), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#57, 0) + coalesce(cs_qty#86, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#58, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#87, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#59, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#88, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, cs_qty#86] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59, cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] (73) TakeOrderedAndProject -Input [12]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, cs_qty#82] -Arguments: 100, [ratio#85 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#55 + cs_qty#82), 1) as double)), 2) ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] +Input [12]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, cs_qty#86] +Arguments: 100, [ratio#89 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#57 + cs_qty#86), 1) as double)), 2) ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt index 48521c72281f4..d13b9623d1b34 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt @@ -149,157 +149,157 @@ Input [1]: [ca_zip#14] Condition : (substr(ca_zip#14, 1, 5) INSET (56910,69952,63792,39371,74351,11101,25003,97189,57834,73134,62377,51200,32754,22752,86379,14171,91110,40162,98569,28709,13394,66162,25733,25782,26065,18383,51949,87343,50298,83849,33786,64528,23470,67030,46136,25280,46820,77721,99076,18426,31880,17871,98235,45748,49156,18652,72013,51622,43848,78567,41248,13695,44165,67853,54917,53179,64034,10567,71791,68908,55565,59402,64147,85816,57855,61547,27700,68100,28810,58263,15723,83933,51103,58058,90578,82276,81096,81426,96451,77556,38607,76638,18906,62971,57047,48425,35576,11928,30625,83444,73520,51650,57647,60099,30122,94983,24128,10445,41368,26233,26859,21756,24676,19849,36420,38193,58470,39127,13595,87501,24317,15455,69399,98025,81019,48033,11376,39516,67875,92712,14867,38122,29741,42961,30469,51211,56458,15559,16021,33123,33282,33515,72823,54601,76698,56240,72175,60279,20004,68806,72325,28488,43933,50412,45200,22246,78668,79777,96765,67301,73273,49448,82636,23932,47305,29839,39192,18799,61265,37125,58943,64457,88424,24610,84935,89360,68893,30431,28898,10336,90257,59166,46081,26105,96888,36634,86284,35258,39972,22927,73241,53268,24206,27385,99543,31671,14663,30903,39861,24996,63089,88086,83921,21076,67897,66708,45721,60576,25103,52867,30450,36233,30010,96576,73171,56571,56575,64544,13955,78451,43285,18119,16725,83041,76107,79994,54364,35942,56691,19769,63435,34102,18845,22744,13354,75691,45549,23968,31387,83144,13375,15765,28577,88190,19736,73650,37930,25989,83926,94898,51798,39736,22437,55253,38415,71256,18376,42029,25858,44438,19515,38935,51649,71954,15882,18767,63193,25486,49130,37126,40604,34425,17043,12305,11634,26653,94167,36446,10516,67473,66864,72425,63981,18842,22461,42666,47770,69035,70372,28587,45266,15371,15798,45375,90225,16807,31016,68014,21337,19505,50016,10144,84093,21286,19430,34322,91068,94945,72305,24671,58048,65084,28545,21195,20548,22245,77191,96976,48583,76231,15734,61810,11356,68621,68786,98359,41367,26689,69913,76614,68101,88885,50308,79077,18270,28915,29178,53672,62878,10390,14922,68341,56529,41766,68309,56616,15126,61860,97789,11489,45692,41918,72151,72550,27156,36495,70738,17879,53535,17920,68880,78890,35850,14089,58078,65164,27068,26231,13376,57665,32213,77610,87816,21309,15146,86198,91137,55307,67467,40558,94627,82136,22351,89091,20260,23006,91393,47537,62496,98294,18840,71286,81312,31029,70466,35458,14060,22685,28286,25631,19512,40081,63837,14328,35474,22152,76232,51061,86057,17183) AND isnotnull(substr(ca_zip#14, 1, 5))) (22) Scan parquet default.customer_address -Output [2]: [ca_address_sk#15, ca_zip#14] +Output [2]: [ca_address_sk#15, ca_zip#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 5] -Input [2]: [ca_address_sk#15, ca_zip#14] +Input [2]: [ca_address_sk#15, ca_zip#16] (24) Filter [codegen id : 5] -Input [2]: [ca_address_sk#15, ca_zip#14] +Input [2]: [ca_address_sk#15, ca_zip#16] Condition : isnotnull(ca_address_sk#15) (25) Exchange -Input [2]: [ca_address_sk#15, ca_zip#14] -Arguments: hashpartitioning(ca_address_sk#15, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [2]: [ca_address_sk#15, ca_zip#16] +Arguments: hashpartitioning(ca_address_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] (26) Sort [codegen id : 6] -Input [2]: [ca_address_sk#15, ca_zip#14] +Input [2]: [ca_address_sk#15, ca_zip#16] Arguments: [ca_address_sk#15 ASC NULLS FIRST], false, 0 (27) Scan parquet default.customer -Output [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] +Output [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 7] -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] +Input [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] (29) Filter [codegen id : 7] -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Condition : ((isnotnull(c_preferred_cust_flag#18) AND (c_preferred_cust_flag#18 = Y)) AND isnotnull(c_current_addr_sk#17)) +Input [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] +Condition : ((isnotnull(c_preferred_cust_flag#19) AND (c_preferred_cust_flag#19 = Y)) AND isnotnull(c_current_addr_sk#18)) (30) Project [codegen id : 7] -Output [1]: [c_current_addr_sk#17] -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] +Output [1]: [c_current_addr_sk#18] +Input [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] (31) Exchange -Input [1]: [c_current_addr_sk#17] -Arguments: hashpartitioning(c_current_addr_sk#17, 5), ENSURE_REQUIREMENTS, [id=#19] +Input [1]: [c_current_addr_sk#18] +Arguments: hashpartitioning(c_current_addr_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] (32) Sort [codegen id : 8] -Input [1]: [c_current_addr_sk#17] -Arguments: [c_current_addr_sk#17 ASC NULLS FIRST], false, 0 +Input [1]: [c_current_addr_sk#18] +Arguments: [c_current_addr_sk#18 ASC NULLS FIRST], false, 0 (33) SortMergeJoin [codegen id : 9] Left keys [1]: [ca_address_sk#15] -Right keys [1]: [c_current_addr_sk#17] +Right keys [1]: [c_current_addr_sk#18] Join condition: None (34) Project [codegen id : 9] -Output [1]: [ca_zip#14] -Input [3]: [ca_address_sk#15, ca_zip#14, c_current_addr_sk#17] +Output [1]: [ca_zip#16] +Input [3]: [ca_address_sk#15, ca_zip#16, c_current_addr_sk#18] (35) HashAggregate [codegen id : 9] -Input [1]: [ca_zip#14] -Keys [1]: [ca_zip#14] +Input [1]: [ca_zip#16] +Keys [1]: [ca_zip#16] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#20] -Results [2]: [ca_zip#14, count#21] +Aggregate Attributes [1]: [count#21] +Results [2]: [ca_zip#16, count#22] (36) Exchange -Input [2]: [ca_zip#14, count#21] -Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [2]: [ca_zip#16, count#22] +Arguments: hashpartitioning(ca_zip#16, 5), ENSURE_REQUIREMENTS, [id=#23] (37) HashAggregate [codegen id : 10] -Input [2]: [ca_zip#14, count#21] -Keys [1]: [ca_zip#14] +Input [2]: [ca_zip#16, count#22] +Keys [1]: [ca_zip#16] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#23] -Results [2]: [substr(ca_zip#14, 1, 5) AS ca_zip#24, count(1)#23 AS count(1)#25] +Aggregate Attributes [1]: [count(1)#24] +Results [2]: [substr(ca_zip#16, 1, 5) AS ca_zip#25, count(1)#24 AS count(1)#26] (38) Filter [codegen id : 10] -Input [2]: [ca_zip#24, count(1)#25] -Condition : (count(1)#25 > 10) +Input [2]: [ca_zip#25, count(1)#26] +Condition : (count(1)#26 > 10) (39) Project [codegen id : 10] -Output [1]: [ca_zip#24] -Input [2]: [ca_zip#24, count(1)#25] +Output [1]: [ca_zip#25] +Input [2]: [ca_zip#25, count(1)#26] (40) BroadcastExchange -Input [1]: [ca_zip#24] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#26] +Input [1]: [ca_zip#25] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#27] (41) BroadcastHashJoin [codegen id : 11] Left keys [2]: [coalesce(substr(ca_zip#14, 1, 5), ), isnull(substr(ca_zip#14, 1, 5))] -Right keys [2]: [coalesce(ca_zip#24, ), isnull(ca_zip#24)] +Right keys [2]: [coalesce(ca_zip#25, ), isnull(ca_zip#25)] Join condition: None (42) Project [codegen id : 11] -Output [1]: [substr(ca_zip#14, 1, 5) AS ca_zip#27] +Output [1]: [substr(ca_zip#14, 1, 5) AS ca_zip#28] Input [1]: [ca_zip#14] (43) HashAggregate [codegen id : 11] -Input [1]: [ca_zip#27] -Keys [1]: [ca_zip#27] +Input [1]: [ca_zip#28] +Keys [1]: [ca_zip#28] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#27] +Results [1]: [ca_zip#28] (44) Exchange -Input [1]: [ca_zip#27] -Arguments: hashpartitioning(ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [1]: [ca_zip#28] +Arguments: hashpartitioning(ca_zip#28, 5), ENSURE_REQUIREMENTS, [id=#29] (45) HashAggregate [codegen id : 12] -Input [1]: [ca_zip#27] -Keys [1]: [ca_zip#27] +Input [1]: [ca_zip#28] +Keys [1]: [ca_zip#28] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#27] +Results [1]: [ca_zip#28] (46) Exchange -Input [1]: [ca_zip#27] -Arguments: hashpartitioning(substr(ca_zip#27, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#29] +Input [1]: [ca_zip#28] +Arguments: hashpartitioning(substr(ca_zip#28, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#30] (47) Sort [codegen id : 13] -Input [1]: [ca_zip#27] -Arguments: [substr(ca_zip#27, 1, 2) ASC NULLS FIRST], false, 0 +Input [1]: [ca_zip#28] +Arguments: [substr(ca_zip#28, 1, 2) ASC NULLS FIRST], false, 0 (48) SortMergeJoin [codegen id : 14] Left keys [1]: [substr(s_zip#11, 1, 2)] -Right keys [1]: [substr(ca_zip#27, 1, 2)] +Right keys [1]: [substr(ca_zip#28, 1, 2)] Join condition: None (49) Project [codegen id : 14] Output [2]: [ss_net_profit#2, s_store_name#10] -Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#27] +Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#28] (50) HashAggregate [codegen id : 14] Input [2]: [ss_net_profit#2, s_store_name#10] Keys [1]: [s_store_name#10] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#30] -Results [2]: [s_store_name#10, sum#31] +Aggregate Attributes [1]: [sum#31] +Results [2]: [s_store_name#10, sum#32] (51) Exchange -Input [2]: [s_store_name#10, sum#31] -Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [2]: [s_store_name#10, sum#32] +Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#33] (52) HashAggregate [codegen id : 15] -Input [2]: [s_store_name#10, sum#31] +Input [2]: [s_store_name#10, sum#32] Keys [1]: [s_store_name#10] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#33] -Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#33,17,2) AS sum(ss_net_profit)#34] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#34] +Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS sum(ss_net_profit)#35] (53) TakeOrderedAndProject -Input [2]: [s_store_name#10, sum(ss_net_profit)#34] -Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#34] +Input [2]: [s_store_name#10, sum(ss_net_profit)#35] +Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#35] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt index 09e1189003ed6..dbb5e1f606a8d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt @@ -135,141 +135,141 @@ Input [1]: [ca_zip#13] Condition : (substr(ca_zip#13, 1, 5) INSET (56910,69952,63792,39371,74351,11101,25003,97189,57834,73134,62377,51200,32754,22752,86379,14171,91110,40162,98569,28709,13394,66162,25733,25782,26065,18383,51949,87343,50298,83849,33786,64528,23470,67030,46136,25280,46820,77721,99076,18426,31880,17871,98235,45748,49156,18652,72013,51622,43848,78567,41248,13695,44165,67853,54917,53179,64034,10567,71791,68908,55565,59402,64147,85816,57855,61547,27700,68100,28810,58263,15723,83933,51103,58058,90578,82276,81096,81426,96451,77556,38607,76638,18906,62971,57047,48425,35576,11928,30625,83444,73520,51650,57647,60099,30122,94983,24128,10445,41368,26233,26859,21756,24676,19849,36420,38193,58470,39127,13595,87501,24317,15455,69399,98025,81019,48033,11376,39516,67875,92712,14867,38122,29741,42961,30469,51211,56458,15559,16021,33123,33282,33515,72823,54601,76698,56240,72175,60279,20004,68806,72325,28488,43933,50412,45200,22246,78668,79777,96765,67301,73273,49448,82636,23932,47305,29839,39192,18799,61265,37125,58943,64457,88424,24610,84935,89360,68893,30431,28898,10336,90257,59166,46081,26105,96888,36634,86284,35258,39972,22927,73241,53268,24206,27385,99543,31671,14663,30903,39861,24996,63089,88086,83921,21076,67897,66708,45721,60576,25103,52867,30450,36233,30010,96576,73171,56571,56575,64544,13955,78451,43285,18119,16725,83041,76107,79994,54364,35942,56691,19769,63435,34102,18845,22744,13354,75691,45549,23968,31387,83144,13375,15765,28577,88190,19736,73650,37930,25989,83926,94898,51798,39736,22437,55253,38415,71256,18376,42029,25858,44438,19515,38935,51649,71954,15882,18767,63193,25486,49130,37126,40604,34425,17043,12305,11634,26653,94167,36446,10516,67473,66864,72425,63981,18842,22461,42666,47770,69035,70372,28587,45266,15371,15798,45375,90225,16807,31016,68014,21337,19505,50016,10144,84093,21286,19430,34322,91068,94945,72305,24671,58048,65084,28545,21195,20548,22245,77191,96976,48583,76231,15734,61810,11356,68621,68786,98359,41367,26689,69913,76614,68101,88885,50308,79077,18270,28915,29178,53672,62878,10390,14922,68341,56529,41766,68309,56616,15126,61860,97789,11489,45692,41918,72151,72550,27156,36495,70738,17879,53535,17920,68880,78890,35850,14089,58078,65164,27068,26231,13376,57665,32213,77610,87816,21309,15146,86198,91137,55307,67467,40558,94627,82136,22351,89091,20260,23006,91393,47537,62496,98294,18840,71286,81312,31029,70466,35458,14060,22685,28286,25631,19512,40081,63837,14328,35474,22152,76232,51061,86057,17183) AND isnotnull(substr(ca_zip#13, 1, 5))) (20) Scan parquet default.customer_address -Output [2]: [ca_address_sk#14, ca_zip#13] +Output [2]: [ca_address_sk#14, ca_zip#15] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#14, ca_zip#13] +Input [2]: [ca_address_sk#14, ca_zip#15] (22) Filter [codegen id : 4] -Input [2]: [ca_address_sk#14, ca_zip#13] +Input [2]: [ca_address_sk#14, ca_zip#15] Condition : isnotnull(ca_address_sk#14) (23) Scan parquet default.customer -Output [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Output [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 3] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] (25) Filter [codegen id : 3] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -Condition : ((isnotnull(c_preferred_cust_flag#16) AND (c_preferred_cust_flag#16 = Y)) AND isnotnull(c_current_addr_sk#15)) +Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Condition : ((isnotnull(c_preferred_cust_flag#17) AND (c_preferred_cust_flag#17 = Y)) AND isnotnull(c_current_addr_sk#16)) (26) Project [codegen id : 3] -Output [1]: [c_current_addr_sk#15] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Output [1]: [c_current_addr_sk#16] +Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] (27) BroadcastExchange -Input [1]: [c_current_addr_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Input [1]: [c_current_addr_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (28) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ca_address_sk#14] -Right keys [1]: [c_current_addr_sk#15] +Right keys [1]: [c_current_addr_sk#16] Join condition: None (29) Project [codegen id : 4] -Output [1]: [ca_zip#13] -Input [3]: [ca_address_sk#14, ca_zip#13, c_current_addr_sk#15] +Output [1]: [ca_zip#15] +Input [3]: [ca_address_sk#14, ca_zip#15, c_current_addr_sk#16] (30) HashAggregate [codegen id : 4] -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] +Input [1]: [ca_zip#15] +Keys [1]: [ca_zip#15] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#18] -Results [2]: [ca_zip#13, count#19] +Aggregate Attributes [1]: [count#19] +Results [2]: [ca_zip#15, count#20] (31) Exchange -Input [2]: [ca_zip#13, count#19] -Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [2]: [ca_zip#15, count#20] +Arguments: hashpartitioning(ca_zip#15, 5), ENSURE_REQUIREMENTS, [id=#21] (32) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#13, count#19] -Keys [1]: [ca_zip#13] +Input [2]: [ca_zip#15, count#20] +Keys [1]: [ca_zip#15] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#21] -Results [2]: [substr(ca_zip#13, 1, 5) AS ca_zip#22, count(1)#21 AS count(1)#23] +Aggregate Attributes [1]: [count(1)#22] +Results [2]: [substr(ca_zip#15, 1, 5) AS ca_zip#23, count(1)#22 AS count(1)#24] (33) Filter [codegen id : 5] -Input [2]: [ca_zip#22, count(1)#23] -Condition : (count(1)#23 > 10) +Input [2]: [ca_zip#23, count(1)#24] +Condition : (count(1)#24 > 10) (34) Project [codegen id : 5] -Output [1]: [ca_zip#22] -Input [2]: [ca_zip#22, count(1)#23] +Output [1]: [ca_zip#23] +Input [2]: [ca_zip#23, count(1)#24] (35) BroadcastExchange -Input [1]: [ca_zip#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#24] +Input [1]: [ca_zip#23] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#25] (36) BroadcastHashJoin [codegen id : 6] Left keys [2]: [coalesce(substr(ca_zip#13, 1, 5), ), isnull(substr(ca_zip#13, 1, 5))] -Right keys [2]: [coalesce(ca_zip#22, ), isnull(ca_zip#22)] +Right keys [2]: [coalesce(ca_zip#23, ), isnull(ca_zip#23)] Join condition: None (37) Project [codegen id : 6] -Output [1]: [substr(ca_zip#13, 1, 5) AS ca_zip#25] +Output [1]: [substr(ca_zip#13, 1, 5) AS ca_zip#26] Input [1]: [ca_zip#13] (38) HashAggregate [codegen id : 6] -Input [1]: [ca_zip#25] -Keys [1]: [ca_zip#25] +Input [1]: [ca_zip#26] +Keys [1]: [ca_zip#26] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#25] +Results [1]: [ca_zip#26] (39) Exchange -Input [1]: [ca_zip#25] -Arguments: hashpartitioning(ca_zip#25, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [1]: [ca_zip#26] +Arguments: hashpartitioning(ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#27] (40) HashAggregate [codegen id : 7] -Input [1]: [ca_zip#25] -Keys [1]: [ca_zip#25] +Input [1]: [ca_zip#26] +Keys [1]: [ca_zip#26] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#25] +Results [1]: [ca_zip#26] (41) BroadcastExchange -Input [1]: [ca_zip#25] -Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [id=#27] +Input [1]: [ca_zip#26] +Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [id=#28] (42) BroadcastHashJoin [codegen id : 8] Left keys [1]: [substr(s_zip#11, 1, 2)] -Right keys [1]: [substr(ca_zip#25, 1, 2)] +Right keys [1]: [substr(ca_zip#26, 1, 2)] Join condition: None (43) Project [codegen id : 8] Output [2]: [ss_net_profit#2, s_store_name#10] -Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#25] +Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#26] (44) HashAggregate [codegen id : 8] Input [2]: [ss_net_profit#2, s_store_name#10] Keys [1]: [s_store_name#10] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [s_store_name#10, sum#29] +Aggregate Attributes [1]: [sum#29] +Results [2]: [s_store_name#10, sum#30] (45) Exchange -Input [2]: [s_store_name#10, sum#29] -Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [s_store_name#10, sum#30] +Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#31] (46) HashAggregate [codegen id : 9] -Input [2]: [s_store_name#10, sum#29] +Input [2]: [s_store_name#10, sum#30] Keys [1]: [s_store_name#10] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] -Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS sum(ss_net_profit)#32] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#32] +Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#32,17,2) AS sum(ss_net_profit)#33] (47) TakeOrderedAndProject -Input [2]: [s_store_name#10, sum(ss_net_profit)#32] -Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#32] +Input [2]: [s_store_name#10, sum(ss_net_profit)#33] +Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#33] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt index e5c59a0ecfaad..bdb1a52a18f2d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt @@ -367,252 +367,252 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#16] +Output [1]: [i_item_sk#61] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#16] +Right keys [1]: [i_item_sk#61] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#16] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#61] (60) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#19] +Output [1]: [p_promo_sk#62] (61) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#19] +Right keys [1]: [p_promo_sk#62] Join condition: None (62) Project [codegen id : 19] Output [6]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#19] +Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#62] (63) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#63] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#63] Join condition: None (65) Project [codegen id : 19] Output [5]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#22] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#63] (66) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 18] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] (68) Filter [codegen id : 18] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Condition : isnotnull(cp_catalog_page_sk#61) +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Condition : isnotnull(cp_catalog_page_sk#64) (69) BroadcastExchange -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#61] +Right keys [1]: [cp_catalog_page_sk#64] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#64, cp_catalog_page_id#65] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Keys [1]: [cp_catalog_page_id#62] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] +Keys [1]: [cp_catalog_page_id#65] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Results [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] (73) Exchange -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [id=#77] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [cp_catalog_page_id#62] +Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Keys [1]: [cp_catalog_page_id#65] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#80, catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#62) AS id#82] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] +Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#81, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#82, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#83, catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#65) AS id#85] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) (78) Exchange -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] -Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (84) Exchange -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] -Right keys [2]: [wr_item_sk#91, wr_order_number#92] +Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] +Right keys [2]: [wr_item_sk#94, wr_order_number#95] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#16] +Output [1]: [i_item_sk#100] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#83] -Right keys [1]: [i_item_sk#16] +Left keys [1]: [ws_item_sk#86] +Right keys [1]: [i_item_sk#100] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, i_item_sk#16] +Output [7]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, i_item_sk#100] (91) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#19] +Output [1]: [p_promo_sk#101] (92) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#85] -Right keys [1]: [p_promo_sk#19] +Left keys [1]: [ws_promo_sk#88] +Right keys [1]: [p_promo_sk#101] Join condition: None (93) Project [codegen id : 29] -Output [6]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [8]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, p_promo_sk#19] +Output [6]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [8]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, p_promo_sk#101] (94) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#102] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#89] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [ws_sold_date_sk#92] +Right keys [1]: [d_date_sk#102] Join condition: None (96) Project [codegen id : 29] -Output [5]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] -Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#22] +Output [5]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] +Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#102] (97) Scan parquet default.web_site -Output [2]: [web_site_sk#97, web_site_id#98] +Output [2]: [web_site_sk#103, web_site_id#104] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (98) ColumnarToRow [codegen id : 28] -Input [2]: [web_site_sk#97, web_site_id#98] +Input [2]: [web_site_sk#103, web_site_id#104] (99) Filter [codegen id : 28] -Input [2]: [web_site_sk#97, web_site_id#98] -Condition : isnotnull(web_site_sk#97) +Input [2]: [web_site_sk#103, web_site_id#104] +Condition : isnotnull(web_site_sk#103) (100) BroadcastExchange -Input [2]: [web_site_sk#97, web_site_id#98] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Input [2]: [web_site_sk#103, web_site_id#104] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#105] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#84] -Right keys [1]: [web_site_sk#97] +Left keys [1]: [ws_web_site_sk#87] +Right keys [1]: [web_site_sk#103] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] +Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] +Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#103, web_site_id#104] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Keys [1]: [web_site_id#98] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] +Keys [1]: [web_site_id#104] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Results [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] (104) Exchange -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] +Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, [id=#116] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Keys [1]: [web_site_id#98] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#114, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#115, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#116, web channel AS channel#117, concat(web_site, web_site_id#98) AS id#118] +Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Keys [1]: [web_site_id#104] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] +Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#120, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#121, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#122, web channel AS channel#123, concat(web_site, web_site_id#104) AS id#124] (106) Union (107) Expand [codegen id : 31] Input [5]: [sales#42, returns#43, profit#44, channel#45, id#46] -Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] +Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] (108) HashAggregate [codegen id : 31] -Input [6]: [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] -Keys [3]: [channel#119, id#120, spark_grouping_id#121] +Input [6]: [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] +Keys [3]: [channel#125, id#126, spark_grouping_id#127] Functions [3]: [partial_sum(sales#42), partial_sum(returns#43), partial_sum(profit#44)] -Aggregate Attributes [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Results [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Aggregate Attributes [6]: [sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Results [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] (109) Exchange -Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] -Arguments: hashpartitioning(channel#119, id#120, spark_grouping_id#121, 5), ENSURE_REQUIREMENTS, [id=#134] +Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Arguments: hashpartitioning(channel#125, id#126, spark_grouping_id#127, 5), ENSURE_REQUIREMENTS, [id=#140] (110) HashAggregate [codegen id : 32] -Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] -Keys [3]: [channel#119, id#120, spark_grouping_id#121] +Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Keys [3]: [channel#125, id#126, spark_grouping_id#127] Functions [3]: [sum(sales#42), sum(returns#43), sum(profit#44)] -Aggregate Attributes [3]: [sum(sales#42)#135, sum(returns#43)#136, sum(profit#44)#137] -Results [5]: [channel#119, id#120, sum(sales#42)#135 AS sales#138, sum(returns#43)#136 AS returns#139, sum(profit#44)#137 AS profit#140] +Aggregate Attributes [3]: [sum(sales#42)#141, sum(returns#43)#142, sum(profit#44)#143] +Results [5]: [channel#125, id#126, sum(sales#42)#141 AS sales#144, sum(returns#43)#142 AS returns#145, sum(profit#44)#143 AS profit#146] (111) TakeOrderedAndProject -Input [5]: [channel#119, id#120, sales#138, returns#139, profit#140] -Arguments: 100, [channel#119 ASC NULLS FIRST, id#120 ASC NULLS FIRST], [channel#119, id#120, sales#138, returns#139, profit#140] +Input [5]: [channel#125, id#126, sales#144, returns#145, profit#146] +Arguments: 100, [channel#125 ASC NULLS FIRST, id#126 ASC NULLS FIRST], [channel#125, id#126, sales#144, returns#145, profit#146] ===== Subqueries ===== @@ -625,6 +625,6 @@ Output [1]: [d_date_sk#22] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt index 3d087efe472bb..aa15d27d4e562 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt @@ -367,252 +367,252 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#61] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#61] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#16] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#61] (60) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] (62) Filter [codegen id : 16] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Condition : isnotnull(cp_catalog_page_sk#61) +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Condition : isnotnull(cp_catalog_page_sk#62) (63) BroadcastExchange -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#61] +Right keys [1]: [cp_catalog_page_sk#62] Join condition: None (65) Project [codegen id : 19] -Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#62, cp_catalog_page_id#63] (66) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#22] +Output [1]: [i_item_sk#65] (67) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#22] +Right keys [1]: [i_item_sk#65] Join condition: None (68) Project [codegen id : 19] -Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, i_item_sk#22] +Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, i_item_sk#65] (69) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#25] +Output [1]: [p_promo_sk#66] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#25] +Right keys [1]: [p_promo_sk#66] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, p_promo_sk#25] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, p_promo_sk#66] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Keys [1]: [cp_catalog_page_id#62] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Keys [1]: [cp_catalog_page_id#63] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Results [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] (73) Exchange -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Arguments: hashpartitioning(cp_catalog_page_id#63, 5), ENSURE_REQUIREMENTS, [id=#77] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [cp_catalog_page_id#62] +Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Keys [1]: [cp_catalog_page_id#63] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#80, catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#62) AS id#82] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] +Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#81, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#82, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#83, catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#63) AS id#85] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) (78) Exchange -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] -Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (84) Exchange -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] -Right keys [2]: [wr_item_sk#91, wr_order_number#92] +Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] +Right keys [2]: [wr_item_sk#94, wr_order_number#95] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#100] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#89] -Right keys [1]: [d_date_sk#16] +Left keys [1]: [ws_sold_date_sk#92] +Right keys [1]: [d_date_sk#100] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#16] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#100] (91) Scan parquet default.web_site -Output [2]: [web_site_sk#97, web_site_id#98] +Output [2]: [web_site_sk#101, web_site_id#102] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 26] -Input [2]: [web_site_sk#97, web_site_id#98] +Input [2]: [web_site_sk#101, web_site_id#102] (93) Filter [codegen id : 26] -Input [2]: [web_site_sk#97, web_site_id#98] -Condition : isnotnull(web_site_sk#97) +Input [2]: [web_site_sk#101, web_site_id#102] +Condition : isnotnull(web_site_sk#101) (94) BroadcastExchange -Input [2]: [web_site_sk#97, web_site_id#98] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Input [2]: [web_site_sk#101, web_site_id#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#84] -Right keys [1]: [web_site_sk#97] +Left keys [1]: [ws_web_site_sk#87] +Right keys [1]: [web_site_sk#101] Join condition: None (96) Project [codegen id : 29] -Output [7]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] +Output [7]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#101, web_site_id#102] (97) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#22] +Output [1]: [i_item_sk#104] (98) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#83] -Right keys [1]: [i_item_sk#22] +Left keys [1]: [ws_item_sk#86] +Right keys [1]: [i_item_sk#104] Join condition: None (99) Project [codegen id : 29] -Output [6]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [8]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, i_item_sk#22] +Output [6]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [8]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, i_item_sk#104] (100) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#25] +Output [1]: [p_promo_sk#105] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#85] -Right keys [1]: [p_promo_sk#25] +Left keys [1]: [ws_promo_sk#88] +Right keys [1]: [p_promo_sk#105] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [7]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, p_promo_sk#25] +Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [7]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, p_promo_sk#105] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Keys [1]: [web_site_id#98] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Keys [1]: [web_site_id#102] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Results [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] (104) Exchange -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] +Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Arguments: hashpartitioning(web_site_id#102, 5), ENSURE_REQUIREMENTS, [id=#116] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Keys [1]: [web_site_id#98] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#114, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#115, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#116, web channel AS channel#117, concat(web_site, web_site_id#98) AS id#118] +Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Keys [1]: [web_site_id#102] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] +Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#120, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#121, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#122, web channel AS channel#123, concat(web_site, web_site_id#102) AS id#124] (106) Union (107) Expand [codegen id : 31] Input [5]: [sales#42, returns#43, profit#44, channel#45, id#46] -Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] +Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] (108) HashAggregate [codegen id : 31] -Input [6]: [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] -Keys [3]: [channel#119, id#120, spark_grouping_id#121] +Input [6]: [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] +Keys [3]: [channel#125, id#126, spark_grouping_id#127] Functions [3]: [partial_sum(sales#42), partial_sum(returns#43), partial_sum(profit#44)] -Aggregate Attributes [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Results [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Aggregate Attributes [6]: [sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Results [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] (109) Exchange -Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] -Arguments: hashpartitioning(channel#119, id#120, spark_grouping_id#121, 5), ENSURE_REQUIREMENTS, [id=#134] +Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Arguments: hashpartitioning(channel#125, id#126, spark_grouping_id#127, 5), ENSURE_REQUIREMENTS, [id=#140] (110) HashAggregate [codegen id : 32] -Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] -Keys [3]: [channel#119, id#120, spark_grouping_id#121] +Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Keys [3]: [channel#125, id#126, spark_grouping_id#127] Functions [3]: [sum(sales#42), sum(returns#43), sum(profit#44)] -Aggregate Attributes [3]: [sum(sales#42)#135, sum(returns#43)#136, sum(profit#44)#137] -Results [5]: [channel#119, id#120, sum(sales#42)#135 AS sales#138, sum(returns#43)#136 AS returns#139, sum(profit#44)#137 AS profit#140] +Aggregate Attributes [3]: [sum(sales#42)#141, sum(returns#43)#142, sum(profit#44)#143] +Results [5]: [channel#125, id#126, sum(sales#42)#141 AS sales#144, sum(returns#43)#142 AS returns#145, sum(profit#44)#143 AS profit#146] (111) TakeOrderedAndProject -Input [5]: [channel#119, id#120, sales#138, returns#139, profit#140] -Arguments: 100, [channel#119 ASC NULLS FIRST, id#120 ASC NULLS FIRST], [channel#119, id#120, sales#138, returns#139, profit#140] +Input [5]: [channel#125, id#126, sales#144, returns#145, profit#146] +Arguments: 100, [channel#125 ASC NULLS FIRST, id#126 ASC NULLS FIRST], [channel#125, id#126, sales#144, returns#145, profit#146] ===== Subqueries ===== @@ -625,6 +625,6 @@ Output [1]: [d_date_sk#16] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#8 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 167e142598ae8..d2d70bef8c3ee 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 @@ -170,178 +170,178 @@ Input [3]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19] Condition : isnotnull(ctr_total_return#19) (24) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Output [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(cr_returned_date_sk#23 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 8] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Input [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] (26) Filter [codegen id : 8] -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) +Input [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] +Condition : isnotnull(cr_returning_addr_sk#21) (27) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#24] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None (29) Project [codegen id : 8] -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] +Output [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] +Input [5]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23, d_date_sk#24] (30) Exchange -Input [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Arguments: hashpartitioning(cr_returning_addr_sk#2, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] +Arguments: hashpartitioning(cr_returning_addr_sk#21, 5), ENSURE_REQUIREMENTS, [id=#25] (31) Sort [codegen id : 9] -Input [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Arguments: [cr_returning_addr_sk#2 ASC NULLS FIRST], false, 0 +Input [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] +Arguments: [cr_returning_addr_sk#21 ASC NULLS FIRST], false, 0 (32) ReusedExchange [Reuses operator id: 16] -Output [2]: [ca_address_sk#10, ca_state#11] +Output [2]: [ca_address_sk#26, ca_state#27] (33) Sort [codegen id : 11] -Input [2]: [ca_address_sk#10, ca_state#11] -Arguments: [ca_address_sk#10 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#26, ca_state#27] +Arguments: [ca_address_sk#26 ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 12] -Left keys [1]: [cr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [cr_returning_addr_sk#21] +Right keys [1]: [ca_address_sk#26] Join condition: None (35) Project [codegen id : 12] -Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#11] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#10, ca_state#11] +Output [3]: [cr_returning_customer_sk#20, cr_return_amt_inc_tax#22, ca_state#27] +Input [5]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, ca_address_sk#26, ca_state#27] (36) HashAggregate [codegen id : 12] -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#11] -Keys [2]: [cr_returning_customer_sk#1, ca_state#11] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum#21] -Results [3]: [cr_returning_customer_sk#1, ca_state#11, sum#22] +Input [3]: [cr_returning_customer_sk#20, cr_return_amt_inc_tax#22, ca_state#27] +Keys [2]: [cr_returning_customer_sk#20, ca_state#27] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#22))] +Aggregate Attributes [1]: [sum#28] +Results [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] (37) Exchange -Input [3]: [cr_returning_customer_sk#1, ca_state#11, sum#22] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#11, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] +Arguments: hashpartitioning(cr_returning_customer_sk#20, ca_state#27, 5), ENSURE_REQUIREMENTS, [id=#30] (38) HashAggregate [codegen id : 13] -Input [3]: [cr_returning_customer_sk#1, ca_state#11, sum#22] -Keys [2]: [cr_returning_customer_sk#1, ca_state#11] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#24] -Results [2]: [ca_state#11 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#24,17,2) AS ctr_total_return#19] +Input [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] +Keys [2]: [cr_returning_customer_sk#20, ca_state#27] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#22))#31] +Results [2]: [ca_state#27 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#22))#31,17,2) AS ctr_total_return#19] (39) HashAggregate [codegen id : 13] Input [2]: [ctr_state#18, ctr_total_return#19] Keys [1]: [ctr_state#18] Functions [1]: [partial_avg(ctr_total_return#19)] -Aggregate Attributes [2]: [sum#25, count#26] -Results [3]: [ctr_state#18, sum#27, count#28] +Aggregate Attributes [2]: [sum#32, count#33] +Results [3]: [ctr_state#18, sum#34, count#35] (40) Exchange -Input [3]: [ctr_state#18, sum#27, count#28] -Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ctr_state#18, sum#34, count#35] +Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#36] (41) HashAggregate [codegen id : 14] -Input [3]: [ctr_state#18, sum#27, count#28] +Input [3]: [ctr_state#18, sum#34, count#35] Keys [1]: [ctr_state#18] Functions [1]: [avg(ctr_total_return#19)] -Aggregate Attributes [1]: [avg(ctr_total_return#19)#30] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#18 AS ctr_state#18#32] +Aggregate Attributes [1]: [avg(ctr_total_return#19)#37] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#37) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#38, ctr_state#18 AS ctr_state#18#39] (42) Filter [codegen id : 14] -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) +Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#38) (43) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#33] +Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#40] (44) BroadcastHashJoin [codegen id : 15] Left keys [1]: [ctr_state#18] -Right keys [1]: [ctr_state#18#32] -Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) +Right keys [1]: [ctr_state#18#39] +Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#38) (45) Project [codegen id : 15] Output [2]: [ctr_customer_sk#17, ctr_total_return#19] -Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] +Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] (46) Exchange Input [2]: [ctr_customer_sk#17, ctr_total_return#19] -Arguments: hashpartitioning(ctr_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#34] +Arguments: hashpartitioning(ctr_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#41] (47) Sort [codegen id : 16] Input [2]: [ctr_customer_sk#17, ctr_total_return#19] Arguments: [ctr_customer_sk#17 ASC NULLS FIRST], false, 0 (48) Scan parquet default.customer -Output [6]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40] +Output [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 18] -Input [6]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40] +Input [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] (50) Filter [codegen id : 18] -Input [6]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40] -Condition : (isnotnull(c_customer_sk#35) AND isnotnull(c_current_addr_sk#37)) +Input [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] +Condition : (isnotnull(c_customer_sk#42) AND isnotnull(c_current_addr_sk#44)) (51) Scan parquet default.customer_address -Output [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Output [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] 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 (52) ColumnarToRow [codegen id : 17] -Input [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] (53) Filter [codegen id : 17] -Input [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Condition : ((isnotnull(ca_state#11) AND (ca_state#11 = GA)) AND isnotnull(ca_address_sk#10)) +Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Condition : ((isnotnull(ca_state#55) AND (ca_state#55 = GA)) AND isnotnull(ca_address_sk#48)) (54) BroadcastExchange -Input [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] +Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#60] (55) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#37] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [c_current_addr_sk#44] +Right keys [1]: [ca_address_sk#48] Join condition: None (56) Project [codegen id : 18] -Output [16]: [c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Input [18]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40, ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Output [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Input [18]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47, ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] (57) Exchange -Input [16]: [c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Arguments: hashpartitioning(c_customer_sk#35, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Arguments: hashpartitioning(c_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#61] (58) Sort [codegen id : 19] -Input [16]: [c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Arguments: [c_customer_sk#35 ASC NULLS FIRST], false, 0 +Input [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Arguments: [c_customer_sk#42 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 20] Left keys [1]: [ctr_customer_sk#17] -Right keys [1]: [c_customer_sk#35] +Right keys [1]: [c_customer_sk#42] Join condition: None (60) Project [codegen id : 20] -Output [16]: [c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#19] -Input [18]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Output [16]: [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] +Input [18]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] (61) TakeOrderedAndProject -Input [16]: [c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#19] -Arguments: 100, [c_customer_id#36 ASC NULLS FIRST, c_salutation#38 ASC NULLS FIRST, c_first_name#39 ASC NULLS FIRST, c_last_name#40 ASC NULLS FIRST, ca_street_number#41 ASC NULLS FIRST, ca_street_name#42 ASC NULLS FIRST, ca_street_type#43 ASC NULLS FIRST, ca_suite_number#44 ASC NULLS FIRST, ca_city#45 ASC NULLS FIRST, ca_county#46 ASC NULLS FIRST, ca_state#11 ASC NULLS FIRST, ca_zip#47 ASC NULLS FIRST, ca_country#48 ASC NULLS FIRST, ca_gmt_offset#49 ASC NULLS FIRST, ca_location_type#50 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#19] +Input [16]: [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] +Arguments: 100, [c_customer_id#43 ASC NULLS FIRST, c_salutation#45 ASC NULLS FIRST, c_first_name#46 ASC NULLS FIRST, c_last_name#47 ASC NULLS FIRST, ca_street_number#49 ASC NULLS FIRST, ca_street_name#50 ASC NULLS FIRST, ca_street_type#51 ASC NULLS FIRST, ca_suite_number#52 ASC NULLS FIRST, ca_city#53 ASC NULLS FIRST, ca_county#54 ASC NULLS FIRST, ca_state#55 ASC NULLS FIRST, ca_zip#56 ASC NULLS FIRST, ca_country#57 ASC NULLS FIRST, ca_gmt_offset#58 ASC NULLS FIRST, ca_location_type#59 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] ===== Subqueries ===== @@ -352,6 +352,6 @@ ReusedExchange (62) (62) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#23 IN dynamicpruning#5 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 6a8f9e5264d49..04371a7f43d2a 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 @@ -149,154 +149,154 @@ Input [3]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18] Condition : isnotnull(ctr_total_return#18) (21) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Output [4]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#22), dynamicpruningexpression(cr_returned_date_sk#22 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct (22) ColumnarToRow [codegen id : 6] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Input [4]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22] (23) Filter [codegen id : 6] -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) +Input [4]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22] +Condition : isnotnull(cr_returning_addr_sk#20) (24) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#23] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#22] +Right keys [1]: [d_date_sk#23] Join condition: None (26) Project [codegen id : 6] -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] +Output [3]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21] +Input [5]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22, d_date_sk#23] (27) ReusedExchange [Reuses operator id: 14] -Output [2]: [ca_address_sk#9, ca_state#10] +Output [2]: [ca_address_sk#24, ca_state#25] (28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#9] +Left keys [1]: [cr_returning_addr_sk#20] +Right keys [1]: [ca_address_sk#24] Join condition: None (29) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#9, ca_state#10] +Output [3]: [cr_returning_customer_sk#19, cr_return_amt_inc_tax#21, ca_state#25] +Input [5]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, ca_address_sk#24, ca_state#25] (30) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [cr_returning_customer_sk#1, ca_state#10, sum#20] +Input [3]: [cr_returning_customer_sk#19, cr_return_amt_inc_tax#21, ca_state#25] +Keys [2]: [cr_returning_customer_sk#19, ca_state#25] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#21))] +Aggregate Attributes [1]: [sum#26] +Results [3]: [cr_returning_customer_sk#19, ca_state#25, sum#27] (31) Exchange -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#20] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [3]: [cr_returning_customer_sk#19, ca_state#25, sum#27] +Arguments: hashpartitioning(cr_returning_customer_sk#19, ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#28] (32) HashAggregate [codegen id : 7] -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#20] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#22] -Results [2]: [ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#22,17,2) AS ctr_total_return#18] +Input [3]: [cr_returning_customer_sk#19, ca_state#25, sum#27] +Keys [2]: [cr_returning_customer_sk#19, ca_state#25] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#21))#29] +Results [2]: [ca_state#25 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#21))#29,17,2) AS ctr_total_return#18] (33) HashAggregate [codegen id : 7] Input [2]: [ctr_state#17, ctr_total_return#18] Keys [1]: [ctr_state#17] Functions [1]: [partial_avg(ctr_total_return#18)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ctr_state#17, sum#25, count#26] +Aggregate Attributes [2]: [sum#30, count#31] +Results [3]: [ctr_state#17, sum#32, count#33] (34) Exchange -Input [3]: [ctr_state#17, sum#25, count#26] -Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ctr_state#17, sum#32, count#33] +Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#34] (35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#17, sum#25, count#26] +Input [3]: [ctr_state#17, sum#32, count#33] Keys [1]: [ctr_state#17] Functions [1]: [avg(ctr_total_return#18)] -Aggregate Attributes [1]: [avg(ctr_total_return#18)#28] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#28) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#29, ctr_state#17 AS ctr_state#17#30] +Aggregate Attributes [1]: [avg(ctr_total_return#18)#35] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#35) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#36, ctr_state#17 AS ctr_state#17#37] (36) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) +Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#36) (37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#31] +Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#38] (38) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#17] -Right keys [1]: [ctr_state#17#30] -Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) +Right keys [1]: [ctr_state#17#37] +Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#36) (39) Project [codegen id : 11] Output [2]: [ctr_customer_sk#16, ctr_total_return#18] -Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] +Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] (40) Scan parquet default.customer -Output [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +Output [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +Input [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] (42) Filter [codegen id : 9] -Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34)) +Input [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] +Condition : (isnotnull(c_customer_sk#39) AND isnotnull(c_current_addr_sk#41)) (43) BroadcastExchange -Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] +Input [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] (44) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#16] -Right keys [1]: [c_customer_sk#32] +Right keys [1]: [c_customer_sk#39] Join condition: None (45) Project [codegen id : 11] -Output [6]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] -Input [8]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +Output [6]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] +Input [8]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] (46) Scan parquet default.customer_address -Output [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] +Output [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] 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 (47) ColumnarToRow [codegen id : 10] -Input [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] +Input [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] (48) Filter [codegen id : 10] -Input [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] -Condition : ((isnotnull(ca_state#10) AND (ca_state#10 = GA)) AND isnotnull(ca_address_sk#9)) +Input [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] +Condition : ((isnotnull(ca_state#53) AND (ca_state#53 = GA)) AND isnotnull(ca_address_sk#46)) (49) BroadcastExchange -Input [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Input [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#34] -Right keys [1]: [ca_address_sk#9] +Left keys [1]: [c_current_addr_sk#41] +Right keys [1]: [ca_address_sk#46] Join condition: None (51) Project [codegen id : 11] -Output [16]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48, ctr_total_return#18] -Input [18]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] +Output [16]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57, ctr_total_return#18] +Input [18]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] (52) TakeOrderedAndProject -Input [16]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48, ctr_total_return#18] -Arguments: 100, [c_customer_id#33 ASC NULLS FIRST, c_salutation#35 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, c_last_name#37 ASC NULLS FIRST, ca_street_number#39 ASC NULLS FIRST, ca_street_name#40 ASC NULLS FIRST, ca_street_type#41 ASC NULLS FIRST, ca_suite_number#42 ASC NULLS FIRST, ca_city#43 ASC NULLS FIRST, ca_county#44 ASC NULLS FIRST, ca_state#10 ASC NULLS FIRST, ca_zip#45 ASC NULLS FIRST, ca_country#46 ASC NULLS FIRST, ca_gmt_offset#47 ASC NULLS FIRST, ca_location_type#48 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48, ctr_total_return#18] +Input [16]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57, ctr_total_return#18] +Arguments: 100, [c_customer_id#40 ASC NULLS FIRST, c_salutation#42 ASC NULLS FIRST, c_first_name#43 ASC NULLS FIRST, c_last_name#44 ASC NULLS FIRST, ca_street_number#47 ASC NULLS FIRST, ca_street_name#48 ASC NULLS FIRST, ca_street_type#49 ASC NULLS FIRST, ca_suite_number#50 ASC NULLS FIRST, ca_city#51 ASC NULLS FIRST, ca_county#52 ASC NULLS FIRST, ca_state#53 ASC NULLS FIRST, ca_zip#54 ASC NULLS FIRST, ca_country#55 ASC NULLS FIRST, ca_gmt_offset#56 ASC NULLS FIRST, ca_location_type#57 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57, ctr_total_return#18] ===== Subqueries ===== @@ -307,6 +307,6 @@ ReusedExchange (53) (53) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 21 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 21 Hosting Expression = cr_returned_date_sk#22 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt index d006b61d20c33..a7c71a0668676 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt @@ -92,51 +92,51 @@ Input [2]: [d_date_sk#4, d_date#5] Condition : isnotnull(d_date_sk#4) (7) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#6, d_week_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (8) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#6, d_week_seq#7] (9) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (10) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#8, d_week_seq#9] (11) Filter [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] -Condition : cast(d_date#5 as string) IN (2000-06-30,2000-09-27,2000-11-17) +Input [2]: [d_date#8, d_week_seq#9] +Condition : cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) (12) Project [codegen id : 1] -Output [1]: [d_week_seq#6 AS d_week_seq#6#7] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_week_seq#9] +Input [2]: [d_date#8, d_week_seq#9] (13) BroadcastExchange -Input [1]: [d_week_seq#6#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] +Input [1]: [d_week_seq#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] (14) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_week_seq#6] -Right keys [1]: [d_week_seq#6#7] +Left keys [1]: [d_week_seq#7] +Right keys [1]: [d_week_seq#9] Join condition: None (15) Project [codegen id : 2] -Output [1]: [d_date#5 AS d_date#5#9] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_date#6] +Input [2]: [d_date#6, d_week_seq#7] (16) BroadcastExchange -Input [1]: [d_date#5#9] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] +Input [1]: [d_date#6] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#11] (17) BroadcastHashJoin [codegen id : 3] Left keys [1]: [d_date#5] -Right keys [1]: [d_date#5#9] +Right keys [1]: [d_date#6] Join condition: None (18) Project [codegen id : 3] @@ -145,7 +145,7 @@ Input [2]: [d_date_sk#4, d_date#5] (19) BroadcastExchange Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (20) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] @@ -157,191 +157,191 @@ Output [2]: [sr_item_sk#1, sr_return_quantity#2] Input [4]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, d_date_sk#4] (22) Scan parquet default.item -Output [2]: [i_item_sk#12, i_item_id#13] +Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (23) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#12, i_item_id#13] +Input [2]: [i_item_sk#13, i_item_id#14] (24) Filter [codegen id : 4] -Input [2]: [i_item_sk#12, i_item_id#13] -Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) +Input [2]: [i_item_sk#13, i_item_id#14] +Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) (25) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] +Input [2]: [i_item_sk#13, i_item_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] (26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_item_sk#1] -Right keys [1]: [cast(i_item_sk#12 as bigint)] +Right keys [1]: [cast(i_item_sk#13 as bigint)] Join condition: None (27) Project [codegen id : 5] -Output [2]: [sr_return_quantity#2, i_item_id#13] -Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#12, i_item_id#13] +Output [2]: [sr_return_quantity#2, i_item_id#14] +Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#13, i_item_id#14] (28) HashAggregate [codegen id : 5] -Input [2]: [sr_return_quantity#2, i_item_id#13] -Keys [1]: [i_item_id#13] +Input [2]: [sr_return_quantity#2, i_item_id#14] +Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#13, sum#16] +Aggregate Attributes [1]: [sum#16] +Results [2]: [i_item_id#14, sum#17] (29) Exchange -Input [2]: [i_item_id#13, sum#16] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [2]: [i_item_id#14, sum#17] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#18] (30) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#13, sum#16] -Keys [1]: [i_item_id#13] +Input [2]: [i_item_id#14, sum#17] +Keys [1]: [i_item_id#14] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#18] -Results [2]: [i_item_id#13 AS item_id#19, sum(sr_return_quantity#2)#18 AS sr_item_qty#20] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] +Results [2]: [i_item_id#14 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] (31) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] (33) Filter [codegen id : 10] -Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] -Condition : isnotnull(cr_item_sk#21) +Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +Condition : isnotnull(cr_item_sk#22) (34) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#4] +Output [1]: [d_date_sk#25] (35) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#23] -Right keys [1]: [d_date_sk#4] +Left keys [1]: [cr_returned_date_sk#24] +Right keys [1]: [d_date_sk#25] Join condition: None (36) Project [codegen id : 10] -Output [2]: [cr_item_sk#21, cr_return_quantity#22] -Input [4]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, d_date_sk#4] +Output [2]: [cr_item_sk#22, cr_return_quantity#23] +Input [4]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, d_date_sk#25] (37) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#12, i_item_id#13] +Output [2]: [i_item_sk#26, i_item_id#27] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#21] -Right keys [1]: [i_item_sk#12] +Left keys [1]: [cr_item_sk#22] +Right keys [1]: [i_item_sk#26] Join condition: None (39) Project [codegen id : 10] -Output [2]: [cr_return_quantity#22, i_item_id#13] -Input [4]: [cr_item_sk#21, cr_return_quantity#22, i_item_sk#12, i_item_id#13] +Output [2]: [cr_return_quantity#23, i_item_id#27] +Input [4]: [cr_item_sk#22, cr_return_quantity#23, i_item_sk#26, i_item_id#27] (40) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#22, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum#24] -Results [2]: [i_item_id#13, sum#25] +Input [2]: [cr_return_quantity#23, i_item_id#27] +Keys [1]: [i_item_id#27] +Functions [1]: [partial_sum(cr_return_quantity#23)] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_item_id#27, sum#29] (41) Exchange -Input [2]: [i_item_id#13, sum#25] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [i_item_id#27, sum#29] +Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, [id=#30] (42) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#13, sum#25] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum(cr_return_quantity#22)#27] -Results [2]: [i_item_id#13 AS item_id#28, sum(cr_return_quantity#22)#27 AS cr_item_qty#29] +Input [2]: [i_item_id#27, sum#29] +Keys [1]: [i_item_id#27] +Functions [1]: [sum(cr_return_quantity#23)] +Aggregate Attributes [1]: [sum(cr_return_quantity#23)#31] +Results [2]: [i_item_id#27 AS item_id#32, sum(cr_return_quantity#23)#31 AS cr_item_qty#33] (43) BroadcastExchange -Input [2]: [item_id#28, cr_item_qty#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] +Input [2]: [item_id#32, cr_item_qty#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] (44) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#28] +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#32] Join condition: None (45) Project [codegen id : 18] -Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] -Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] +Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#33] +Input [4]: [item_id#20, sr_item_qty#21, item_id#32, cr_item_qty#33] (46) Scan parquet default.web_returns -Output [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +Output [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#33), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(wr_returned_date_sk#37), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] (48) Filter [codegen id : 16] -Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] -Condition : isnotnull(wr_item_sk#31) +Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +Condition : isnotnull(wr_item_sk#35) (49) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#4] +Output [1]: [d_date_sk#38] (50) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#33] -Right keys [1]: [cast(d_date_sk#4 as bigint)] +Left keys [1]: [wr_returned_date_sk#37] +Right keys [1]: [cast(d_date_sk#38 as bigint)] Join condition: None (51) Project [codegen id : 16] -Output [2]: [wr_item_sk#31, wr_return_quantity#32] -Input [4]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33, d_date_sk#4] +Output [2]: [wr_item_sk#35, wr_return_quantity#36] +Input [4]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37, d_date_sk#38] (52) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#12, i_item_id#13] +Output [2]: [i_item_sk#39, i_item_id#40] (53) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#31] -Right keys [1]: [cast(i_item_sk#12 as bigint)] +Left keys [1]: [wr_item_sk#35] +Right keys [1]: [cast(i_item_sk#39 as bigint)] Join condition: None (54) Project [codegen id : 16] -Output [2]: [wr_return_quantity#32, i_item_id#13] -Input [4]: [wr_item_sk#31, wr_return_quantity#32, i_item_sk#12, i_item_id#13] +Output [2]: [wr_return_quantity#36, i_item_id#40] +Input [4]: [wr_item_sk#35, wr_return_quantity#36, i_item_sk#39, i_item_id#40] (55) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#32, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(wr_return_quantity#32)] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#13, sum#35] +Input [2]: [wr_return_quantity#36, i_item_id#40] +Keys [1]: [i_item_id#40] +Functions [1]: [partial_sum(wr_return_quantity#36)] +Aggregate Attributes [1]: [sum#41] +Results [2]: [i_item_id#40, sum#42] (56) Exchange -Input [2]: [i_item_id#13, sum#35] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [2]: [i_item_id#40, sum#42] +Arguments: hashpartitioning(i_item_id#40, 5), ENSURE_REQUIREMENTS, [id=#43] (57) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#13, sum#35] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(wr_return_quantity#32)] -Aggregate Attributes [1]: [sum(wr_return_quantity#32)#37] -Results [2]: [i_item_id#13 AS item_id#38, sum(wr_return_quantity#32)#37 AS wr_item_qty#39] +Input [2]: [i_item_id#40, sum#42] +Keys [1]: [i_item_id#40] +Functions [1]: [sum(wr_return_quantity#36)] +Aggregate Attributes [1]: [sum(wr_return_quantity#36)#44] +Results [2]: [i_item_id#40 AS item_id#45, sum(wr_return_quantity#36)#44 AS wr_item_qty#46] (58) BroadcastExchange -Input [2]: [item_id#38, wr_item_qty#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] +Input [2]: [item_id#45, wr_item_qty#46] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#47] (59) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#38] +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#45] Join condition: None (60) Project [codegen id : 18] -Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] -Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] +Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS sr_dev#48, cr_item_qty#33, (((cast(cr_item_qty#33 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS cr_dev#49, wr_item_qty#46, (((cast(wr_item_qty#46 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS wr_dev#50, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#51] +Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#33, item_id#45, wr_item_qty#46] (61) TakeOrderedAndProject -Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] -Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +Input [8]: [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] +Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index c926ab7e3ccb6..fd73cee998f71 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -119,51 +119,51 @@ Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) (13) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#9, d_week_seq#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#9, d_week_seq#10] (15) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#11, d_week_seq#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (16) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#11, d_week_seq#12] (17) Filter [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] -Condition : cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) +Input [2]: [d_date#11, d_week_seq#12] +Condition : cast(d_date#11 as string) IN (2000-06-30,2000-09-27,2000-11-17) (18) Project [codegen id : 2] -Output [1]: [d_week_seq#9 AS d_week_seq#9#10] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_week_seq#12] +Input [2]: [d_date#11, d_week_seq#12] (19) BroadcastExchange -Input [1]: [d_week_seq#9#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] +Input [1]: [d_week_seq#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] (20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_week_seq#9] -Right keys [1]: [d_week_seq#9#10] +Left keys [1]: [d_week_seq#10] +Right keys [1]: [d_week_seq#12] Join condition: None (21) Project [codegen id : 3] -Output [1]: [d_date#8 AS d_date#8#12] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_date#9] +Input [2]: [d_date#9, d_week_seq#10] (22) BroadcastExchange -Input [1]: [d_date#8#12] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] +Input [1]: [d_date#9] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#14] (23) BroadcastHashJoin [codegen id : 4] Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#12] +Right keys [1]: [d_date#9] Join condition: None (24) Project [codegen id : 4] @@ -172,7 +172,7 @@ Input [2]: [d_date_sk#7, d_date#8] (25) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] @@ -187,161 +187,161 @@ Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#5, d_date_sk# Input [2]: [sr_return_quantity#2, i_item_id#5] Keys [1]: [i_item_id#5] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#5, sum#16] +Aggregate Attributes [1]: [sum#16] +Results [2]: [i_item_id#5, sum#17] (29) Exchange -Input [2]: [i_item_id#5, sum#16] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [2]: [i_item_id#5, sum#17] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#18] (30) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#5, sum#16] +Input [2]: [i_item_id#5, sum#17] Keys [1]: [i_item_id#5] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#18] -Results [2]: [i_item_id#5 AS item_id#19, sum(sr_return_quantity#2)#18 AS sr_item_qty#20] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] +Results [2]: [i_item_id#5 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] (31) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] (33) Filter [codegen id : 10] -Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] -Condition : isnotnull(cr_item_sk#21) +Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +Condition : isnotnull(cr_item_sk#22) (34) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#25, i_item_id#26] (35) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#21] -Right keys [1]: [i_item_sk#4] +Left keys [1]: [cr_item_sk#22] +Right keys [1]: [i_item_sk#25] Join condition: None (36) Project [codegen id : 10] -Output [3]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#5] -Input [5]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, i_item_sk#4, i_item_id#5] +Output [3]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#26] +Input [5]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, i_item_sk#25, i_item_id#26] (37) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#27] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#23] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [cr_returned_date_sk#24] +Right keys [1]: [d_date_sk#27] Join condition: None (39) Project [codegen id : 10] -Output [2]: [cr_return_quantity#22, i_item_id#5] -Input [4]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#5, d_date_sk#7] +Output [2]: [cr_return_quantity#23, i_item_id#26] +Input [4]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#26, d_date_sk#27] (40) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#22, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum#24] -Results [2]: [i_item_id#5, sum#25] +Input [2]: [cr_return_quantity#23, i_item_id#26] +Keys [1]: [i_item_id#26] +Functions [1]: [partial_sum(cr_return_quantity#23)] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_item_id#26, sum#29] (41) Exchange -Input [2]: [i_item_id#5, sum#25] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [i_item_id#26, sum#29] +Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [id=#30] (42) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#5, sum#25] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum(cr_return_quantity#22)#27] -Results [2]: [i_item_id#5 AS item_id#28, sum(cr_return_quantity#22)#27 AS cr_item_qty#29] +Input [2]: [i_item_id#26, sum#29] +Keys [1]: [i_item_id#26] +Functions [1]: [sum(cr_return_quantity#23)] +Aggregate Attributes [1]: [sum(cr_return_quantity#23)#31] +Results [2]: [i_item_id#26 AS item_id#32, sum(cr_return_quantity#23)#31 AS cr_item_qty#33] (43) BroadcastExchange -Input [2]: [item_id#28, cr_item_qty#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] +Input [2]: [item_id#32, cr_item_qty#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] (44) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#28] +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#32] Join condition: None (45) Project [codegen id : 18] -Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] -Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] +Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#33] +Input [4]: [item_id#20, sr_item_qty#21, item_id#32, cr_item_qty#33] (46) Scan parquet default.web_returns -Output [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +Output [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#33), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(wr_returned_date_sk#37), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] (48) Filter [codegen id : 16] -Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] -Condition : isnotnull(wr_item_sk#31) +Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +Condition : isnotnull(wr_item_sk#35) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#38, i_item_id#39] (50) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#31] -Right keys [1]: [cast(i_item_sk#4 as bigint)] +Left keys [1]: [wr_item_sk#35] +Right keys [1]: [cast(i_item_sk#38 as bigint)] Join condition: None (51) Project [codegen id : 16] -Output [3]: [wr_return_quantity#32, wr_returned_date_sk#33, i_item_id#5] -Input [5]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33, i_item_sk#4, i_item_id#5] +Output [3]: [wr_return_quantity#36, wr_returned_date_sk#37, i_item_id#39] +Input [5]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37, i_item_sk#38, i_item_id#39] (52) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#40] (53) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#33] -Right keys [1]: [cast(d_date_sk#7 as bigint)] +Left keys [1]: [wr_returned_date_sk#37] +Right keys [1]: [cast(d_date_sk#40 as bigint)] Join condition: None (54) Project [codegen id : 16] -Output [2]: [wr_return_quantity#32, i_item_id#5] -Input [4]: [wr_return_quantity#32, wr_returned_date_sk#33, i_item_id#5, d_date_sk#7] +Output [2]: [wr_return_quantity#36, i_item_id#39] +Input [4]: [wr_return_quantity#36, wr_returned_date_sk#37, i_item_id#39, d_date_sk#40] (55) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#32, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(wr_return_quantity#32)] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#5, sum#35] +Input [2]: [wr_return_quantity#36, i_item_id#39] +Keys [1]: [i_item_id#39] +Functions [1]: [partial_sum(wr_return_quantity#36)] +Aggregate Attributes [1]: [sum#41] +Results [2]: [i_item_id#39, sum#42] (56) Exchange -Input [2]: [i_item_id#5, sum#35] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [2]: [i_item_id#39, sum#42] +Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, [id=#43] (57) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#5, sum#35] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(wr_return_quantity#32)] -Aggregate Attributes [1]: [sum(wr_return_quantity#32)#37] -Results [2]: [i_item_id#5 AS item_id#38, sum(wr_return_quantity#32)#37 AS wr_item_qty#39] +Input [2]: [i_item_id#39, sum#42] +Keys [1]: [i_item_id#39] +Functions [1]: [sum(wr_return_quantity#36)] +Aggregate Attributes [1]: [sum(wr_return_quantity#36)#44] +Results [2]: [i_item_id#39 AS item_id#45, sum(wr_return_quantity#36)#44 AS wr_item_qty#46] (58) BroadcastExchange -Input [2]: [item_id#38, wr_item_qty#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] +Input [2]: [item_id#45, wr_item_qty#46] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#47] (59) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#38] +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#45] Join condition: None (60) Project [codegen id : 18] -Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] -Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] +Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS sr_dev#48, cr_item_qty#33, (((cast(cr_item_qty#33 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS cr_dev#49, wr_item_qty#46, (((cast(wr_item_qty#46 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS wr_dev#50, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#51] +Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#33, item_id#45, wr_item_qty#46] (61) TakeOrderedAndProject -Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] -Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +Input [8]: [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] +Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt index f9d73c29db3d8..e72928545d080 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt @@ -313,687 +313,687 @@ Aggregate Attributes [1]: [count(1)#19] Results [1]: [count(1)#19 AS h8_30_to_9#20] (29) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] (31) Filter [codegen id : 8] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Condition : ((isnotnull(ss_hdemo_sk#22) AND isnotnull(ss_sold_time_sk#21)) AND isnotnull(ss_store_sk#23)) (32) Project [codegen id : 8] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] (33) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#25, t_hour#26, t_minute#27] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#25, t_hour#26, t_minute#27] (35) Filter [codegen id : 5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 9)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#25, t_hour#26, t_minute#27] +Condition : ((((isnotnull(t_hour#26) AND isnotnull(t_minute#27)) AND (t_hour#26 = 9)) AND (t_minute#27 < 30)) AND isnotnull(t_time_sk#25)) (36) Project [codegen id : 5] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#25] +Input [3]: [t_time_sk#25, t_hour#26, t_minute#27] (37) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [t_time_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (38) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#21] +Right keys [1]: [t_time_sk#25] Join condition: None (39) Project [codegen id : 8] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#22, ss_store_sk#23] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, t_time_sk#25] (40) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#29] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#23] +Right keys [1]: [s_store_sk#29] Join condition: None (42) Project [codegen id : 8] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#22] +Input [3]: [ss_hdemo_sk#22, ss_store_sk#23, s_store_sk#29] (43) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#30] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#22] +Right keys [1]: [hd_demo_sk#30] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#22, hd_demo_sk#30] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#22] -Results [1]: [count#23] +Aggregate Attributes [1]: [count#31] +Results [1]: [count#32] (47) Exchange -Input [1]: [count#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [1]: [count#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] (48) HashAggregate [codegen id : 9] -Input [1]: [count#23] +Input [1]: [count#32] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#25] -Results [1]: [count(1)#25 AS h9_to_9_30#26] +Aggregate Attributes [1]: [count(1)#34] +Results [1]: [count(1)#34 AS h9_to_9_30#35] (49) BroadcastExchange -Input [1]: [h9_to_9_30#26] -Arguments: IdentityBroadcastMode, [id=#27] +Input [1]: [h9_to_9_30#35] +Arguments: IdentityBroadcastMode, [id=#36] (50) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (51) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 13] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] (53) Filter [codegen id : 13] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) (54) Project [codegen id : 13] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] (55) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#41, t_hour#42, t_minute#43] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 10] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#41, t_hour#42, t_minute#43] (57) Filter [codegen id : 10] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 9)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#41, t_hour#42, t_minute#43] +Condition : ((((isnotnull(t_hour#42) AND isnotnull(t_minute#43)) AND (t_hour#42 = 9)) AND (t_minute#43 >= 30)) AND isnotnull(t_time_sk#41)) (58) Project [codegen id : 10] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#41] +Input [3]: [t_time_sk#41, t_hour#42, t_minute#43] (59) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [t_time_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (60) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#37] +Right keys [1]: [t_time_sk#41] Join condition: None (61) Project [codegen id : 13] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#38, ss_store_sk#39] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, t_time_sk#41] (62) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#45] (63) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#39] +Right keys [1]: [s_store_sk#45] Join condition: None (64) Project [codegen id : 13] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#38] +Input [3]: [ss_hdemo_sk#38, ss_store_sk#39, s_store_sk#45] (65) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#46] (66) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#38] +Right keys [1]: [hd_demo_sk#46] Join condition: None (67) Project [codegen id : 13] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#38, hd_demo_sk#46] (68) HashAggregate [codegen id : 13] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [1]: [count#30] +Aggregate Attributes [1]: [count#47] +Results [1]: [count#48] (69) Exchange -Input [1]: [count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] +Input [1]: [count#48] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] (70) HashAggregate [codegen id : 14] -Input [1]: [count#30] +Input [1]: [count#48] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [1]: [count(1)#32 AS h9_30_to_10#33] +Aggregate Attributes [1]: [count(1)#50] +Results [1]: [count(1)#50 AS h9_30_to_10#51] (71) BroadcastExchange -Input [1]: [h9_30_to_10#33] -Arguments: IdentityBroadcastMode, [id=#34] +Input [1]: [h9_30_to_10#51] +Arguments: IdentityBroadcastMode, [id=#52] (72) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (73) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 18] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] (75) Filter [codegen id : 18] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Condition : ((isnotnull(ss_hdemo_sk#54) AND isnotnull(ss_sold_time_sk#53)) AND isnotnull(ss_store_sk#55)) (76) Project [codegen id : 18] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] (77) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#57, t_hour#58, t_minute#59] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (78) ColumnarToRow [codegen id : 15] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] (79) Filter [codegen id : 15] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 10)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Condition : ((((isnotnull(t_hour#58) AND isnotnull(t_minute#59)) AND (t_hour#58 = 10)) AND (t_minute#59 < 30)) AND isnotnull(t_time_sk#57)) (80) Project [codegen id : 15] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#57] +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] (81) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [1]: [t_time_sk#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] (82) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#53] +Right keys [1]: [t_time_sk#57] Join condition: None (83) Project [codegen id : 18] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#54, ss_store_sk#55] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, t_time_sk#57] (84) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#61] (85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#55] +Right keys [1]: [s_store_sk#61] Join condition: None (86) Project [codegen id : 18] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#54] +Input [3]: [ss_hdemo_sk#54, ss_store_sk#55, s_store_sk#61] (87) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#62] (88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#54] +Right keys [1]: [hd_demo_sk#62] Join condition: None (89) Project [codegen id : 18] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#54, hd_demo_sk#62] (90) HashAggregate [codegen id : 18] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#36] -Results [1]: [count#37] +Aggregate Attributes [1]: [count#63] +Results [1]: [count#64] (91) Exchange -Input [1]: [count#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#38] +Input [1]: [count#64] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] (92) HashAggregate [codegen id : 19] -Input [1]: [count#37] +Input [1]: [count#64] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#39] -Results [1]: [count(1)#39 AS h10_to_10_30#40] +Aggregate Attributes [1]: [count(1)#66] +Results [1]: [count(1)#66 AS h10_to_10_30#67] (93) BroadcastExchange -Input [1]: [h10_to_10_30#40] -Arguments: IdentityBroadcastMode, [id=#41] +Input [1]: [h10_to_10_30#67] +Arguments: IdentityBroadcastMode, [id=#68] (94) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (95) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (96) ColumnarToRow [codegen id : 23] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] (97) Filter [codegen id : 23] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Condition : ((isnotnull(ss_hdemo_sk#70) AND isnotnull(ss_sold_time_sk#69)) AND isnotnull(ss_store_sk#71)) (98) Project [codegen id : 23] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] (99) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#73, t_hour#74, t_minute#75] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (100) ColumnarToRow [codegen id : 20] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#73, t_hour#74, t_minute#75] (101) Filter [codegen id : 20] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 10)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#73, t_hour#74, t_minute#75] +Condition : ((((isnotnull(t_hour#74) AND isnotnull(t_minute#75)) AND (t_hour#74 = 10)) AND (t_minute#75 >= 30)) AND isnotnull(t_time_sk#73)) (102) Project [codegen id : 20] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#73] +Input [3]: [t_time_sk#73, t_hour#74, t_minute#75] (103) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [t_time_sk#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] (104) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#69] +Right keys [1]: [t_time_sk#73] Join condition: None (105) Project [codegen id : 23] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#70, ss_store_sk#71] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, t_time_sk#73] (106) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#77] (107) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#71] +Right keys [1]: [s_store_sk#77] Join condition: None (108) Project [codegen id : 23] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#70] +Input [3]: [ss_hdemo_sk#70, ss_store_sk#71, s_store_sk#77] (109) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#78] (110) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#70] +Right keys [1]: [hd_demo_sk#78] Join condition: None (111) Project [codegen id : 23] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#70, hd_demo_sk#78] (112) HashAggregate [codegen id : 23] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#43] -Results [1]: [count#44] +Aggregate Attributes [1]: [count#79] +Results [1]: [count#80] (113) Exchange -Input [1]: [count#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45] +Input [1]: [count#80] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] (114) HashAggregate [codegen id : 24] -Input [1]: [count#44] +Input [1]: [count#80] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#46] -Results [1]: [count(1)#46 AS h10_30_to_11#47] +Aggregate Attributes [1]: [count(1)#82] +Results [1]: [count(1)#82 AS h10_30_to_11#83] (115) BroadcastExchange -Input [1]: [h10_30_to_11#47] -Arguments: IdentityBroadcastMode, [id=#48] +Input [1]: [h10_30_to_11#83] +Arguments: IdentityBroadcastMode, [id=#84] (116) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (117) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (118) ColumnarToRow [codegen id : 28] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] (119) Filter [codegen id : 28] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Condition : ((isnotnull(ss_hdemo_sk#86) AND isnotnull(ss_sold_time_sk#85)) AND isnotnull(ss_store_sk#87)) (120) Project [codegen id : 28] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] (121) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#89, t_hour#90, t_minute#91] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (122) ColumnarToRow [codegen id : 25] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#89, t_hour#90, t_minute#91] (123) Filter [codegen id : 25] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 11)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#89, t_hour#90, t_minute#91] +Condition : ((((isnotnull(t_hour#90) AND isnotnull(t_minute#91)) AND (t_hour#90 = 11)) AND (t_minute#91 < 30)) AND isnotnull(t_time_sk#89)) (124) Project [codegen id : 25] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#89] +Input [3]: [t_time_sk#89, t_hour#90, t_minute#91] (125) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] +Input [1]: [t_time_sk#89] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#92] (126) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#85] +Right keys [1]: [t_time_sk#89] Join condition: None (127) Project [codegen id : 28] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#86, ss_store_sk#87] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, t_time_sk#89] (128) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#93] (129) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#87] +Right keys [1]: [s_store_sk#93] Join condition: None (130) Project [codegen id : 28] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#86] +Input [3]: [ss_hdemo_sk#86, ss_store_sk#87, s_store_sk#93] (131) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#94] (132) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#86] +Right keys [1]: [hd_demo_sk#94] Join condition: None (133) Project [codegen id : 28] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#86, hd_demo_sk#94] (134) HashAggregate [codegen id : 28] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#50] -Results [1]: [count#51] +Aggregate Attributes [1]: [count#95] +Results [1]: [count#96] (135) Exchange -Input [1]: [count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] +Input [1]: [count#96] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#97] (136) HashAggregate [codegen id : 29] -Input [1]: [count#51] +Input [1]: [count#96] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#53] -Results [1]: [count(1)#53 AS h11_to_11_30#54] +Aggregate Attributes [1]: [count(1)#98] +Results [1]: [count(1)#98 AS h11_to_11_30#99] (137) BroadcastExchange -Input [1]: [h11_to_11_30#54] -Arguments: IdentityBroadcastMode, [id=#55] +Input [1]: [h11_to_11_30#99] +Arguments: IdentityBroadcastMode, [id=#100] (138) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (139) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (140) ColumnarToRow [codegen id : 33] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] (141) Filter [codegen id : 33] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Condition : ((isnotnull(ss_hdemo_sk#102) AND isnotnull(ss_sold_time_sk#101)) AND isnotnull(ss_store_sk#103)) (142) Project [codegen id : 33] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] (143) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#105, t_hour#106, t_minute#107] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (144) ColumnarToRow [codegen id : 30] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#105, t_hour#106, t_minute#107] (145) Filter [codegen id : 30] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 11)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#105, t_hour#106, t_minute#107] +Condition : ((((isnotnull(t_hour#106) AND isnotnull(t_minute#107)) AND (t_hour#106 = 11)) AND (t_minute#107 >= 30)) AND isnotnull(t_time_sk#105)) (146) Project [codegen id : 30] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#105] +Input [3]: [t_time_sk#105, t_hour#106, t_minute#107] (147) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +Input [1]: [t_time_sk#105] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#108] (148) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#101] +Right keys [1]: [t_time_sk#105] Join condition: None (149) Project [codegen id : 33] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#102, ss_store_sk#103] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, t_time_sk#105] (150) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#109] (151) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#103] +Right keys [1]: [s_store_sk#109] Join condition: None (152) Project [codegen id : 33] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#102] +Input [3]: [ss_hdemo_sk#102, ss_store_sk#103, s_store_sk#109] (153) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#110] (154) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#102] +Right keys [1]: [hd_demo_sk#110] Join condition: None (155) Project [codegen id : 33] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#102, hd_demo_sk#110] (156) HashAggregate [codegen id : 33] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#57] -Results [1]: [count#58] +Aggregate Attributes [1]: [count#111] +Results [1]: [count#112] (157) Exchange -Input [1]: [count#58] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#59] +Input [1]: [count#112] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] (158) HashAggregate [codegen id : 34] -Input [1]: [count#58] +Input [1]: [count#112] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#60] -Results [1]: [count(1)#60 AS h11_30_to_12#61] +Aggregate Attributes [1]: [count(1)#114] +Results [1]: [count(1)#114 AS h11_30_to_12#115] (159) BroadcastExchange -Input [1]: [h11_30_to_12#61] -Arguments: IdentityBroadcastMode, [id=#62] +Input [1]: [h11_30_to_12#115] +Arguments: IdentityBroadcastMode, [id=#116] (160) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (161) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (162) ColumnarToRow [codegen id : 38] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] (163) Filter [codegen id : 38] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Condition : ((isnotnull(ss_hdemo_sk#118) AND isnotnull(ss_sold_time_sk#117)) AND isnotnull(ss_store_sk#119)) (164) Project [codegen id : 38] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] (165) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#121, t_hour#122, t_minute#123] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (166) ColumnarToRow [codegen id : 35] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#121, t_hour#122, t_minute#123] (167) Filter [codegen id : 35] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 12)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#121, t_hour#122, t_minute#123] +Condition : ((((isnotnull(t_hour#122) AND isnotnull(t_minute#123)) AND (t_hour#122 = 12)) AND (t_minute#123 < 30)) AND isnotnull(t_time_sk#121)) (168) Project [codegen id : 35] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#121] +Input [3]: [t_time_sk#121, t_hour#122, t_minute#123] (169) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] +Input [1]: [t_time_sk#121] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#124] (170) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#117] +Right keys [1]: [t_time_sk#121] Join condition: None (171) Project [codegen id : 38] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#118, ss_store_sk#119] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, t_time_sk#121] (172) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#125] (173) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#119] +Right keys [1]: [s_store_sk#125] Join condition: None (174) Project [codegen id : 38] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#118] +Input [3]: [ss_hdemo_sk#118, ss_store_sk#119, s_store_sk#125] (175) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#126] (176) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#118] +Right keys [1]: [hd_demo_sk#126] Join condition: None (177) Project [codegen id : 38] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#118, hd_demo_sk#126] (178) HashAggregate [codegen id : 38] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#64] -Results [1]: [count#65] +Aggregate Attributes [1]: [count#127] +Results [1]: [count#128] (179) Exchange -Input [1]: [count#65] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] +Input [1]: [count#128] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] (180) HashAggregate [codegen id : 39] -Input [1]: [count#65] +Input [1]: [count#128] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#67] -Results [1]: [count(1)#67 AS h12_to_12_30#68] +Aggregate Attributes [1]: [count(1)#130] +Results [1]: [count(1)#130 AS h12_to_12_30#131] (181) BroadcastExchange -Input [1]: [h12_to_12_30#68] -Arguments: IdentityBroadcastMode, [id=#69] +Input [1]: [h12_to_12_30#131] +Arguments: IdentityBroadcastMode, [id=#132] (182) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt index 453906c2b5f34..9f56c71154a66 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt @@ -313,687 +313,687 @@ Aggregate Attributes [1]: [count(1)#19] Results [1]: [count(1)#19 AS h8_30_to_9#20] (29) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] (31) Filter [codegen id : 8] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Condition : ((isnotnull(ss_hdemo_sk#22) AND isnotnull(ss_sold_time_sk#21)) AND isnotnull(ss_store_sk#23)) (32) Project [codegen id : 8] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] (33) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#25] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#22] +Right keys [1]: [hd_demo_sk#25] Join condition: None (35) Project [codegen id : 8] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#21, ss_store_sk#23] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, hd_demo_sk#25] (36) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#26, t_hour#27, t_minute#28] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#26, t_hour#27, t_minute#28] (38) Filter [codegen id : 6] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 9)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#26, t_hour#27, t_minute#28] +Condition : ((((isnotnull(t_hour#27) AND isnotnull(t_minute#28)) AND (t_hour#27 = 9)) AND (t_minute#28 < 30)) AND isnotnull(t_time_sk#26)) (39) Project [codegen id : 6] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#26] +Input [3]: [t_time_sk#26, t_hour#27, t_minute#28] (40) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [t_time_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#21] +Right keys [1]: [t_time_sk#26] Join condition: None (42) Project [codegen id : 8] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#23] +Input [3]: [ss_sold_time_sk#21, ss_store_sk#23, t_time_sk#26] (43) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#30] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#23] +Right keys [1]: [s_store_sk#30] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#23, s_store_sk#30] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#22] -Results [1]: [count#23] +Aggregate Attributes [1]: [count#31] +Results [1]: [count#32] (47) Exchange -Input [1]: [count#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [1]: [count#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] (48) HashAggregate [codegen id : 9] -Input [1]: [count#23] +Input [1]: [count#32] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#25] -Results [1]: [count(1)#25 AS h9_to_9_30#26] +Aggregate Attributes [1]: [count(1)#34] +Results [1]: [count(1)#34 AS h9_to_9_30#35] (49) BroadcastExchange -Input [1]: [h9_to_9_30#26] -Arguments: IdentityBroadcastMode, [id=#27] +Input [1]: [h9_to_9_30#35] +Arguments: IdentityBroadcastMode, [id=#36] (50) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (51) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 13] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] (53) Filter [codegen id : 13] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) (54) Project [codegen id : 13] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] (55) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#41] (56) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#38] +Right keys [1]: [hd_demo_sk#41] Join condition: None (57) Project [codegen id : 13] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#37, ss_store_sk#39] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] (58) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 11] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] (60) Filter [codegen id : 11] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 9)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] +Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 9)) AND (t_minute#44 >= 30)) AND isnotnull(t_time_sk#42)) (61) Project [codegen id : 11] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#42] +Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] (62) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [t_time_sk#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#45] (63) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#37] +Right keys [1]: [t_time_sk#42] Join condition: None (64) Project [codegen id : 13] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#39] +Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] (65) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#46] (66) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#39] +Right keys [1]: [s_store_sk#46] Join condition: None (67) Project [codegen id : 13] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#39, s_store_sk#46] (68) HashAggregate [codegen id : 13] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [1]: [count#30] +Aggregate Attributes [1]: [count#47] +Results [1]: [count#48] (69) Exchange -Input [1]: [count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] +Input [1]: [count#48] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] (70) HashAggregate [codegen id : 14] -Input [1]: [count#30] +Input [1]: [count#48] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [1]: [count(1)#32 AS h9_30_to_10#33] +Aggregate Attributes [1]: [count(1)#50] +Results [1]: [count(1)#50 AS h9_30_to_10#51] (71) BroadcastExchange -Input [1]: [h9_30_to_10#33] -Arguments: IdentityBroadcastMode, [id=#34] +Input [1]: [h9_30_to_10#51] +Arguments: IdentityBroadcastMode, [id=#52] (72) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (73) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 18] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] (75) Filter [codegen id : 18] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Condition : ((isnotnull(ss_hdemo_sk#54) AND isnotnull(ss_sold_time_sk#53)) AND isnotnull(ss_store_sk#55)) (76) Project [codegen id : 18] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] (77) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#57] (78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#54] +Right keys [1]: [hd_demo_sk#57] Join condition: None (79) Project [codegen id : 18] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#53, ss_store_sk#55] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, hd_demo_sk#57] (80) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#58, t_hour#59, t_minute#60] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (81) ColumnarToRow [codegen id : 16] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#58, t_hour#59, t_minute#60] (82) Filter [codegen id : 16] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 10)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#58, t_hour#59, t_minute#60] +Condition : ((((isnotnull(t_hour#59) AND isnotnull(t_minute#60)) AND (t_hour#59 = 10)) AND (t_minute#60 < 30)) AND isnotnull(t_time_sk#58)) (83) Project [codegen id : 16] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#58] +Input [3]: [t_time_sk#58, t_hour#59, t_minute#60] (84) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [1]: [t_time_sk#58] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] (85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#53] +Right keys [1]: [t_time_sk#58] Join condition: None (86) Project [codegen id : 18] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#55] +Input [3]: [ss_sold_time_sk#53, ss_store_sk#55, t_time_sk#58] (87) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#62] (88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#55] +Right keys [1]: [s_store_sk#62] Join condition: None (89) Project [codegen id : 18] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#55, s_store_sk#62] (90) HashAggregate [codegen id : 18] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#36] -Results [1]: [count#37] +Aggregate Attributes [1]: [count#63] +Results [1]: [count#64] (91) Exchange -Input [1]: [count#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#38] +Input [1]: [count#64] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] (92) HashAggregate [codegen id : 19] -Input [1]: [count#37] +Input [1]: [count#64] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#39] -Results [1]: [count(1)#39 AS h10_to_10_30#40] +Aggregate Attributes [1]: [count(1)#66] +Results [1]: [count(1)#66 AS h10_to_10_30#67] (93) BroadcastExchange -Input [1]: [h10_to_10_30#40] -Arguments: IdentityBroadcastMode, [id=#41] +Input [1]: [h10_to_10_30#67] +Arguments: IdentityBroadcastMode, [id=#68] (94) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (95) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (96) ColumnarToRow [codegen id : 23] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] (97) Filter [codegen id : 23] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Condition : ((isnotnull(ss_hdemo_sk#70) AND isnotnull(ss_sold_time_sk#69)) AND isnotnull(ss_store_sk#71)) (98) Project [codegen id : 23] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] (99) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#73] (100) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#70] +Right keys [1]: [hd_demo_sk#73] Join condition: None (101) Project [codegen id : 23] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#69, ss_store_sk#71] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, hd_demo_sk#73] (102) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#74, t_hour#75, t_minute#76] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (103) ColumnarToRow [codegen id : 21] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] (104) Filter [codegen id : 21] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 10)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] +Condition : ((((isnotnull(t_hour#75) AND isnotnull(t_minute#76)) AND (t_hour#75 = 10)) AND (t_minute#76 >= 30)) AND isnotnull(t_time_sk#74)) (105) Project [codegen id : 21] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#74] +Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] (106) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [t_time_sk#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] (107) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#69] +Right keys [1]: [t_time_sk#74] Join condition: None (108) Project [codegen id : 23] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#71] +Input [3]: [ss_sold_time_sk#69, ss_store_sk#71, t_time_sk#74] (109) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#78] (110) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#71] +Right keys [1]: [s_store_sk#78] Join condition: None (111) Project [codegen id : 23] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#71, s_store_sk#78] (112) HashAggregate [codegen id : 23] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#43] -Results [1]: [count#44] +Aggregate Attributes [1]: [count#79] +Results [1]: [count#80] (113) Exchange -Input [1]: [count#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45] +Input [1]: [count#80] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] (114) HashAggregate [codegen id : 24] -Input [1]: [count#44] +Input [1]: [count#80] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#46] -Results [1]: [count(1)#46 AS h10_30_to_11#47] +Aggregate Attributes [1]: [count(1)#82] +Results [1]: [count(1)#82 AS h10_30_to_11#83] (115) BroadcastExchange -Input [1]: [h10_30_to_11#47] -Arguments: IdentityBroadcastMode, [id=#48] +Input [1]: [h10_30_to_11#83] +Arguments: IdentityBroadcastMode, [id=#84] (116) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (117) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (118) ColumnarToRow [codegen id : 28] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] (119) Filter [codegen id : 28] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Condition : ((isnotnull(ss_hdemo_sk#86) AND isnotnull(ss_sold_time_sk#85)) AND isnotnull(ss_store_sk#87)) (120) Project [codegen id : 28] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] (121) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#89] (122) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#86] +Right keys [1]: [hd_demo_sk#89] Join condition: None (123) Project [codegen id : 28] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#85, ss_store_sk#87] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, hd_demo_sk#89] (124) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#90, t_hour#91, t_minute#92] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (125) ColumnarToRow [codegen id : 26] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#90, t_hour#91, t_minute#92] (126) Filter [codegen id : 26] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 11)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#90, t_hour#91, t_minute#92] +Condition : ((((isnotnull(t_hour#91) AND isnotnull(t_minute#92)) AND (t_hour#91 = 11)) AND (t_minute#92 < 30)) AND isnotnull(t_time_sk#90)) (127) Project [codegen id : 26] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#90] +Input [3]: [t_time_sk#90, t_hour#91, t_minute#92] (128) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] +Input [1]: [t_time_sk#90] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] (129) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#85] +Right keys [1]: [t_time_sk#90] Join condition: None (130) Project [codegen id : 28] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#87] +Input [3]: [ss_sold_time_sk#85, ss_store_sk#87, t_time_sk#90] (131) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#94] (132) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#87] +Right keys [1]: [s_store_sk#94] Join condition: None (133) Project [codegen id : 28] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#87, s_store_sk#94] (134) HashAggregate [codegen id : 28] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#50] -Results [1]: [count#51] +Aggregate Attributes [1]: [count#95] +Results [1]: [count#96] (135) Exchange -Input [1]: [count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] +Input [1]: [count#96] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#97] (136) HashAggregate [codegen id : 29] -Input [1]: [count#51] +Input [1]: [count#96] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#53] -Results [1]: [count(1)#53 AS h11_to_11_30#54] +Aggregate Attributes [1]: [count(1)#98] +Results [1]: [count(1)#98 AS h11_to_11_30#99] (137) BroadcastExchange -Input [1]: [h11_to_11_30#54] -Arguments: IdentityBroadcastMode, [id=#55] +Input [1]: [h11_to_11_30#99] +Arguments: IdentityBroadcastMode, [id=#100] (138) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (139) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (140) ColumnarToRow [codegen id : 33] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] (141) Filter [codegen id : 33] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Condition : ((isnotnull(ss_hdemo_sk#102) AND isnotnull(ss_sold_time_sk#101)) AND isnotnull(ss_store_sk#103)) (142) Project [codegen id : 33] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] (143) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#105] (144) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#102] +Right keys [1]: [hd_demo_sk#105] Join condition: None (145) Project [codegen id : 33] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#101, ss_store_sk#103] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, hd_demo_sk#105] (146) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#106, t_hour#107, t_minute#108] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (147) ColumnarToRow [codegen id : 31] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#106, t_hour#107, t_minute#108] (148) Filter [codegen id : 31] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 11)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#106, t_hour#107, t_minute#108] +Condition : ((((isnotnull(t_hour#107) AND isnotnull(t_minute#108)) AND (t_hour#107 = 11)) AND (t_minute#108 >= 30)) AND isnotnull(t_time_sk#106)) (149) Project [codegen id : 31] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#106] +Input [3]: [t_time_sk#106, t_hour#107, t_minute#108] (150) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +Input [1]: [t_time_sk#106] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#109] (151) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#101] +Right keys [1]: [t_time_sk#106] Join condition: None (152) Project [codegen id : 33] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#103] +Input [3]: [ss_sold_time_sk#101, ss_store_sk#103, t_time_sk#106] (153) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#110] (154) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#103] +Right keys [1]: [s_store_sk#110] Join condition: None (155) Project [codegen id : 33] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#103, s_store_sk#110] (156) HashAggregate [codegen id : 33] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#57] -Results [1]: [count#58] +Aggregate Attributes [1]: [count#111] +Results [1]: [count#112] (157) Exchange -Input [1]: [count#58] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#59] +Input [1]: [count#112] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] (158) HashAggregate [codegen id : 34] -Input [1]: [count#58] +Input [1]: [count#112] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#60] -Results [1]: [count(1)#60 AS h11_30_to_12#61] +Aggregate Attributes [1]: [count(1)#114] +Results [1]: [count(1)#114 AS h11_30_to_12#115] (159) BroadcastExchange -Input [1]: [h11_30_to_12#61] -Arguments: IdentityBroadcastMode, [id=#62] +Input [1]: [h11_30_to_12#115] +Arguments: IdentityBroadcastMode, [id=#116] (160) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (161) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (162) ColumnarToRow [codegen id : 38] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] (163) Filter [codegen id : 38] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Condition : ((isnotnull(ss_hdemo_sk#118) AND isnotnull(ss_sold_time_sk#117)) AND isnotnull(ss_store_sk#119)) (164) Project [codegen id : 38] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] (165) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#121] (166) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#118] +Right keys [1]: [hd_demo_sk#121] Join condition: None (167) Project [codegen id : 38] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#117, ss_store_sk#119] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, hd_demo_sk#121] (168) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#122, t_hour#123, t_minute#124] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (169) ColumnarToRow [codegen id : 36] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#122, t_hour#123, t_minute#124] (170) Filter [codegen id : 36] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 12)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#122, t_hour#123, t_minute#124] +Condition : ((((isnotnull(t_hour#123) AND isnotnull(t_minute#124)) AND (t_hour#123 = 12)) AND (t_minute#124 < 30)) AND isnotnull(t_time_sk#122)) (171) Project [codegen id : 36] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#122] +Input [3]: [t_time_sk#122, t_hour#123, t_minute#124] (172) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] +Input [1]: [t_time_sk#122] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#125] (173) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#117] +Right keys [1]: [t_time_sk#122] Join condition: None (174) Project [codegen id : 38] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#119] +Input [3]: [ss_sold_time_sk#117, ss_store_sk#119, t_time_sk#122] (175) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#126] (176) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#119] +Right keys [1]: [s_store_sk#126] Join condition: None (177) Project [codegen id : 38] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#119, s_store_sk#126] (178) HashAggregate [codegen id : 38] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#64] -Results [1]: [count#65] +Aggregate Attributes [1]: [count#127] +Results [1]: [count#128] (179) Exchange -Input [1]: [count#65] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] +Input [1]: [count#128] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] (180) HashAggregate [codegen id : 39] -Input [1]: [count#65] +Input [1]: [count#128] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#67] -Results [1]: [count(1)#67 AS h12_to_12_30#68] +Aggregate Attributes [1]: [count(1)#130] +Results [1]: [count(1)#130 AS h12_to_12_30#131] (181) BroadcastExchange -Input [1]: [h12_to_12_30#68] -Arguments: IdentityBroadcastMode, [id=#69] +Input [1]: [h12_to_12_30#131] +Arguments: IdentityBroadcastMode, [id=#132] (182) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt index 65c66eb083f55..8736c9861a5ce 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt @@ -82,40 +82,40 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (12) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (13) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] (14) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Condition : ((isnotnull(ss_quantity#44) AND (ss_quantity#44 >= 1)) AND (ss_quantity#44 <= 20)) (15) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#45] +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] (16) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#45] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#45, count#46] -Results [2]: [sum#47, count#48] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#45))] +Aggregate Attributes [2]: [sum#47, count#48] +Results [2]: [sum#49, count#50] (17) Exchange -Input [2]: [sum#47, count#48] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] +Input [2]: [sum#49, count#50] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#51] (18) HashAggregate [codegen id : 2] -Input [2]: [sum#47, count#48] +Input [2]: [sum#49, count#50] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#50] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#50 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#51] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))#52] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#45))#52 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#53] Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#6, [id=#7] * HashAggregate (25) @@ -128,40 +128,40 @@ Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (19) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (20) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] (21) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Condition : ((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 1)) AND (ss_quantity#54 <= 20)) (22) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#55] +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] (23) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#55] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#53, count#54] -Results [2]: [sum#55, count#56] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [2]: [sum#57, count#58] +Results [2]: [sum#59, count#60] (24) Exchange -Input [2]: [sum#55, count#56] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#57] +Input [2]: [sum#59, count#60] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] (25) HashAggregate [codegen id : 2] -Input [2]: [sum#55, count#56] +Input [2]: [sum#59, count#60] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#58] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#58 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#59] +Functions [1]: [avg(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#55))#62] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#55))#62 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#63] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#9, [id=#10] * HashAggregate (32) @@ -174,40 +174,40 @@ Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (26) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#64, ss_sold_date_sk#65] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (27) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] (28) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +Condition : ((isnotnull(ss_quantity#64) AND (ss_quantity#64 >= 21)) AND (ss_quantity#64 <= 40)) (29) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] (30) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#60] -Results [1]: [count#61] +Aggregate Attributes [1]: [count#66] +Results [1]: [count#67] (31) Exchange -Input [1]: [count#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] +Input [1]: [count#67] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#68] (32) HashAggregate [codegen id : 2] -Input [1]: [count#61] +Input [1]: [count#67] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#63] -Results [1]: [count(1)#63 AS count(1)#64] +Aggregate Attributes [1]: [count(1)#69] +Results [1]: [count(1)#69 AS count(1)#70] Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] * HashAggregate (39) @@ -220,40 +220,40 @@ Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (33) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (34) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] (35) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Condition : ((isnotnull(ss_quantity#71) AND (ss_quantity#71 >= 21)) AND (ss_quantity#71 <= 40)) (36) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#72] +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] (37) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#72] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#65, count#66] -Results [2]: [sum#67, count#68] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#72))] +Aggregate Attributes [2]: [sum#74, count#75] +Results [2]: [sum#76, count#77] (38) Exchange -Input [2]: [sum#67, count#68] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [sum#76, count#77] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] (39) HashAggregate [codegen id : 2] -Input [2]: [sum#67, count#68] +Input [2]: [sum#76, count#77] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#70] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#70 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#71] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))#79] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#72))#79 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#80] Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#13, [id=#14] * HashAggregate (46) @@ -266,40 +266,40 @@ Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (40) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (41) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] (42) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Condition : ((isnotnull(ss_quantity#81) AND (ss_quantity#81 >= 21)) AND (ss_quantity#81 <= 40)) (43) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#82] +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] (44) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#82] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#72, count#73] -Results [2]: [sum#74, count#75] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#82))] +Aggregate Attributes [2]: [sum#84, count#85] +Results [2]: [sum#86, count#87] (45) Exchange -Input [2]: [sum#74, count#75] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] +Input [2]: [sum#86, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] (46) HashAggregate [codegen id : 2] -Input [2]: [sum#74, count#75] +Input [2]: [sum#86, count#87] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#77] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#77 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#78] +Functions [1]: [avg(UnscaledValue(ss_net_paid#82))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#82))#89] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#82))#89 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#90] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#16, [id=#17] * HashAggregate (53) @@ -312,40 +312,40 @@ Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (47) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#91, ss_sold_date_sk#92] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (48) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] (49) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +Condition : ((isnotnull(ss_quantity#91) AND (ss_quantity#91 >= 41)) AND (ss_quantity#91 <= 60)) (50) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] (51) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#79] -Results [1]: [count#80] +Aggregate Attributes [1]: [count#93] +Results [1]: [count#94] (52) Exchange -Input [1]: [count#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +Input [1]: [count#94] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] (53) HashAggregate [codegen id : 2] -Input [1]: [count#80] +Input [1]: [count#94] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#82] -Results [1]: [count(1)#82 AS count(1)#83] +Aggregate Attributes [1]: [count(1)#96] +Results [1]: [count(1)#96 AS count(1)#97] Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#18, [id=#19] * HashAggregate (60) @@ -358,40 +358,40 @@ Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (54) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (55) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] (56) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Condition : ((isnotnull(ss_quantity#98) AND (ss_quantity#98 >= 41)) AND (ss_quantity#98 <= 60)) (57) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#99] +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] (58) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#99] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#84, count#85] -Results [2]: [sum#86, count#87] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#99))] +Aggregate Attributes [2]: [sum#101, count#102] +Results [2]: [sum#103, count#104] (59) Exchange -Input [2]: [sum#86, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] +Input [2]: [sum#103, count#104] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] (60) HashAggregate [codegen id : 2] -Input [2]: [sum#86, count#87] +Input [2]: [sum#103, count#104] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#89] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#89 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#90] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))#106] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#99))#106 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#107] Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#20, [id=#21] * HashAggregate (67) @@ -404,40 +404,40 @@ Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (61) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (62) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] (63) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Condition : ((isnotnull(ss_quantity#108) AND (ss_quantity#108 >= 41)) AND (ss_quantity#108 <= 60)) (64) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#109] +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] (65) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#109] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#91, count#92] -Results [2]: [sum#93, count#94] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#109))] +Aggregate Attributes [2]: [sum#111, count#112] +Results [2]: [sum#113, count#114] (66) Exchange -Input [2]: [sum#93, count#94] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] +Input [2]: [sum#113, count#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] (67) HashAggregate [codegen id : 2] -Input [2]: [sum#93, count#94] +Input [2]: [sum#113, count#114] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#96] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#96 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#97] +Functions [1]: [avg(UnscaledValue(ss_net_paid#109))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#109))#116] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#109))#116 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#117] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#23, [id=#24] * HashAggregate (74) @@ -450,40 +450,40 @@ Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (68) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#118, ss_sold_date_sk#119] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (69) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] (70) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +Condition : ((isnotnull(ss_quantity#118) AND (ss_quantity#118 >= 61)) AND (ss_quantity#118 <= 80)) (71) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] (72) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#98] -Results [1]: [count#99] +Aggregate Attributes [1]: [count#120] +Results [1]: [count#121] (73) Exchange -Input [1]: [count#99] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#100] +Input [1]: [count#121] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#122] (74) HashAggregate [codegen id : 2] -Input [1]: [count#99] +Input [1]: [count#121] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#101] -Results [1]: [count(1)#101 AS count(1)#102] +Aggregate Attributes [1]: [count(1)#123] +Results [1]: [count(1)#123 AS count(1)#124] Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#25, [id=#26] * HashAggregate (81) @@ -496,40 +496,40 @@ Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (75) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (76) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] (77) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Condition : ((isnotnull(ss_quantity#125) AND (ss_quantity#125 >= 61)) AND (ss_quantity#125 <= 80)) (78) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#126] +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] (79) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#126] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#103, count#104] -Results [2]: [sum#105, count#106] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#126))] +Aggregate Attributes [2]: [sum#128, count#129] +Results [2]: [sum#130, count#131] (80) Exchange -Input [2]: [sum#105, count#106] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#107] +Input [2]: [sum#130, count#131] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#132] (81) HashAggregate [codegen id : 2] -Input [2]: [sum#105, count#106] +Input [2]: [sum#130, count#131] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#108] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#108 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#109] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))#133] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#126))#133 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#134] Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#27, [id=#28] * HashAggregate (88) @@ -542,40 +542,40 @@ Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (82) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (83) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] (84) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Condition : ((isnotnull(ss_quantity#135) AND (ss_quantity#135 >= 61)) AND (ss_quantity#135 <= 80)) (85) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#136] +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] (86) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#136] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#110, count#111] -Results [2]: [sum#112, count#113] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#136))] +Aggregate Attributes [2]: [sum#138, count#139] +Results [2]: [sum#140, count#141] (87) Exchange -Input [2]: [sum#112, count#113] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] +Input [2]: [sum#140, count#141] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#142] (88) HashAggregate [codegen id : 2] -Input [2]: [sum#112, count#113] +Input [2]: [sum#140, count#141] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#115] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#115 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#116] +Functions [1]: [avg(UnscaledValue(ss_net_paid#136))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#136))#143] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#136))#143 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#144] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#30, [id=#31] * HashAggregate (95) @@ -588,40 +588,40 @@ Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (89) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#145, ss_sold_date_sk#146] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (90) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] (91) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] +Condition : ((isnotnull(ss_quantity#145) AND (ss_quantity#145 >= 81)) AND (ss_quantity#145 <= 100)) (92) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] (93) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#117] -Results [1]: [count#118] +Aggregate Attributes [1]: [count#147] +Results [1]: [count#148] (94) Exchange -Input [1]: [count#118] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#119] +Input [1]: [count#148] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#149] (95) HashAggregate [codegen id : 2] -Input [1]: [count#118] +Input [1]: [count#148] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#120] -Results [1]: [count(1)#120 AS count(1)#121] +Aggregate Attributes [1]: [count(1)#150] +Results [1]: [count(1)#150 AS count(1)#151] Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#32, [id=#33] * HashAggregate (102) @@ -634,40 +634,40 @@ Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (96) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (97) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] (98) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Condition : ((isnotnull(ss_quantity#152) AND (ss_quantity#152 >= 81)) AND (ss_quantity#152 <= 100)) (99) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#153] +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] (100) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#153] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#122, count#123] -Results [2]: [sum#124, count#125] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#153))] +Aggregate Attributes [2]: [sum#155, count#156] +Results [2]: [sum#157, count#158] (101) Exchange -Input [2]: [sum#124, count#125] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#126] +Input [2]: [sum#157, count#158] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#159] (102) HashAggregate [codegen id : 2] -Input [2]: [sum#124, count#125] +Input [2]: [sum#157, count#158] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#127] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#127 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#128] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))#160] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#153))#160 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#161] Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#34, [id=#35] * HashAggregate (109) @@ -680,39 +680,39 @@ Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (103) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] (105) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Condition : ((isnotnull(ss_quantity#162) AND (ss_quantity#162 >= 81)) AND (ss_quantity#162 <= 100)) (106) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#163] +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] (107) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#163] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#129, count#130] -Results [2]: [sum#131, count#132] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#163))] +Aggregate Attributes [2]: [sum#165, count#166] +Results [2]: [sum#167, count#168] (108) Exchange -Input [2]: [sum#131, count#132] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#133] +Input [2]: [sum#167, count#168] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#169] (109) HashAggregate [codegen id : 2] -Input [2]: [sum#131, count#132] +Input [2]: [sum#167, count#168] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#134] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#134 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#135] +Functions [1]: [avg(UnscaledValue(ss_net_paid#163))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#163))#170] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#163))#170 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#171] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt index 65c66eb083f55..8736c9861a5ce 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt @@ -82,40 +82,40 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (12) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (13) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] (14) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Condition : ((isnotnull(ss_quantity#44) AND (ss_quantity#44 >= 1)) AND (ss_quantity#44 <= 20)) (15) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#45] +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] (16) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#45] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#45, count#46] -Results [2]: [sum#47, count#48] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#45))] +Aggregate Attributes [2]: [sum#47, count#48] +Results [2]: [sum#49, count#50] (17) Exchange -Input [2]: [sum#47, count#48] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] +Input [2]: [sum#49, count#50] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#51] (18) HashAggregate [codegen id : 2] -Input [2]: [sum#47, count#48] +Input [2]: [sum#49, count#50] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#50] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#50 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#51] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))#52] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#45))#52 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#53] Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#6, [id=#7] * HashAggregate (25) @@ -128,40 +128,40 @@ Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (19) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (20) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] (21) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Condition : ((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 1)) AND (ss_quantity#54 <= 20)) (22) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#55] +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] (23) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#55] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#53, count#54] -Results [2]: [sum#55, count#56] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [2]: [sum#57, count#58] +Results [2]: [sum#59, count#60] (24) Exchange -Input [2]: [sum#55, count#56] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#57] +Input [2]: [sum#59, count#60] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] (25) HashAggregate [codegen id : 2] -Input [2]: [sum#55, count#56] +Input [2]: [sum#59, count#60] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#58] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#58 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#59] +Functions [1]: [avg(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#55))#62] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#55))#62 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#63] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#9, [id=#10] * HashAggregate (32) @@ -174,40 +174,40 @@ Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (26) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#64, ss_sold_date_sk#65] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (27) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] (28) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +Condition : ((isnotnull(ss_quantity#64) AND (ss_quantity#64 >= 21)) AND (ss_quantity#64 <= 40)) (29) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] (30) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#60] -Results [1]: [count#61] +Aggregate Attributes [1]: [count#66] +Results [1]: [count#67] (31) Exchange -Input [1]: [count#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] +Input [1]: [count#67] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#68] (32) HashAggregate [codegen id : 2] -Input [1]: [count#61] +Input [1]: [count#67] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#63] -Results [1]: [count(1)#63 AS count(1)#64] +Aggregate Attributes [1]: [count(1)#69] +Results [1]: [count(1)#69 AS count(1)#70] Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] * HashAggregate (39) @@ -220,40 +220,40 @@ Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (33) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (34) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] (35) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Condition : ((isnotnull(ss_quantity#71) AND (ss_quantity#71 >= 21)) AND (ss_quantity#71 <= 40)) (36) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#72] +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] (37) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#72] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#65, count#66] -Results [2]: [sum#67, count#68] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#72))] +Aggregate Attributes [2]: [sum#74, count#75] +Results [2]: [sum#76, count#77] (38) Exchange -Input [2]: [sum#67, count#68] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [sum#76, count#77] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] (39) HashAggregate [codegen id : 2] -Input [2]: [sum#67, count#68] +Input [2]: [sum#76, count#77] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#70] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#70 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#71] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))#79] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#72))#79 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#80] Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#13, [id=#14] * HashAggregate (46) @@ -266,40 +266,40 @@ Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (40) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (41) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] (42) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Condition : ((isnotnull(ss_quantity#81) AND (ss_quantity#81 >= 21)) AND (ss_quantity#81 <= 40)) (43) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#82] +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] (44) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#82] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#72, count#73] -Results [2]: [sum#74, count#75] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#82))] +Aggregate Attributes [2]: [sum#84, count#85] +Results [2]: [sum#86, count#87] (45) Exchange -Input [2]: [sum#74, count#75] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] +Input [2]: [sum#86, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] (46) HashAggregate [codegen id : 2] -Input [2]: [sum#74, count#75] +Input [2]: [sum#86, count#87] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#77] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#77 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#78] +Functions [1]: [avg(UnscaledValue(ss_net_paid#82))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#82))#89] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#82))#89 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#90] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#16, [id=#17] * HashAggregate (53) @@ -312,40 +312,40 @@ Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (47) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#91, ss_sold_date_sk#92] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (48) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] (49) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +Condition : ((isnotnull(ss_quantity#91) AND (ss_quantity#91 >= 41)) AND (ss_quantity#91 <= 60)) (50) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] (51) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#79] -Results [1]: [count#80] +Aggregate Attributes [1]: [count#93] +Results [1]: [count#94] (52) Exchange -Input [1]: [count#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +Input [1]: [count#94] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] (53) HashAggregate [codegen id : 2] -Input [1]: [count#80] +Input [1]: [count#94] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#82] -Results [1]: [count(1)#82 AS count(1)#83] +Aggregate Attributes [1]: [count(1)#96] +Results [1]: [count(1)#96 AS count(1)#97] Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#18, [id=#19] * HashAggregate (60) @@ -358,40 +358,40 @@ Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (54) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (55) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] (56) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Condition : ((isnotnull(ss_quantity#98) AND (ss_quantity#98 >= 41)) AND (ss_quantity#98 <= 60)) (57) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#99] +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] (58) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#99] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#84, count#85] -Results [2]: [sum#86, count#87] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#99))] +Aggregate Attributes [2]: [sum#101, count#102] +Results [2]: [sum#103, count#104] (59) Exchange -Input [2]: [sum#86, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] +Input [2]: [sum#103, count#104] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] (60) HashAggregate [codegen id : 2] -Input [2]: [sum#86, count#87] +Input [2]: [sum#103, count#104] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#89] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#89 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#90] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))#106] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#99))#106 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#107] Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#20, [id=#21] * HashAggregate (67) @@ -404,40 +404,40 @@ Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (61) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (62) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] (63) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Condition : ((isnotnull(ss_quantity#108) AND (ss_quantity#108 >= 41)) AND (ss_quantity#108 <= 60)) (64) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#109] +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] (65) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#109] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#91, count#92] -Results [2]: [sum#93, count#94] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#109))] +Aggregate Attributes [2]: [sum#111, count#112] +Results [2]: [sum#113, count#114] (66) Exchange -Input [2]: [sum#93, count#94] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] +Input [2]: [sum#113, count#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] (67) HashAggregate [codegen id : 2] -Input [2]: [sum#93, count#94] +Input [2]: [sum#113, count#114] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#96] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#96 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#97] +Functions [1]: [avg(UnscaledValue(ss_net_paid#109))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#109))#116] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#109))#116 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#117] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#23, [id=#24] * HashAggregate (74) @@ -450,40 +450,40 @@ Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (68) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#118, ss_sold_date_sk#119] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (69) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] (70) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +Condition : ((isnotnull(ss_quantity#118) AND (ss_quantity#118 >= 61)) AND (ss_quantity#118 <= 80)) (71) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] (72) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#98] -Results [1]: [count#99] +Aggregate Attributes [1]: [count#120] +Results [1]: [count#121] (73) Exchange -Input [1]: [count#99] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#100] +Input [1]: [count#121] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#122] (74) HashAggregate [codegen id : 2] -Input [1]: [count#99] +Input [1]: [count#121] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#101] -Results [1]: [count(1)#101 AS count(1)#102] +Aggregate Attributes [1]: [count(1)#123] +Results [1]: [count(1)#123 AS count(1)#124] Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#25, [id=#26] * HashAggregate (81) @@ -496,40 +496,40 @@ Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (75) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (76) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] (77) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Condition : ((isnotnull(ss_quantity#125) AND (ss_quantity#125 >= 61)) AND (ss_quantity#125 <= 80)) (78) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#126] +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] (79) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#126] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#103, count#104] -Results [2]: [sum#105, count#106] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#126))] +Aggregate Attributes [2]: [sum#128, count#129] +Results [2]: [sum#130, count#131] (80) Exchange -Input [2]: [sum#105, count#106] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#107] +Input [2]: [sum#130, count#131] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#132] (81) HashAggregate [codegen id : 2] -Input [2]: [sum#105, count#106] +Input [2]: [sum#130, count#131] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#108] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#108 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#109] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))#133] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#126))#133 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#134] Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#27, [id=#28] * HashAggregate (88) @@ -542,40 +542,40 @@ Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (82) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (83) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] (84) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Condition : ((isnotnull(ss_quantity#135) AND (ss_quantity#135 >= 61)) AND (ss_quantity#135 <= 80)) (85) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#136] +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] (86) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#136] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#110, count#111] -Results [2]: [sum#112, count#113] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#136))] +Aggregate Attributes [2]: [sum#138, count#139] +Results [2]: [sum#140, count#141] (87) Exchange -Input [2]: [sum#112, count#113] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] +Input [2]: [sum#140, count#141] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#142] (88) HashAggregate [codegen id : 2] -Input [2]: [sum#112, count#113] +Input [2]: [sum#140, count#141] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#115] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#115 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#116] +Functions [1]: [avg(UnscaledValue(ss_net_paid#136))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#136))#143] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#136))#143 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#144] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#30, [id=#31] * HashAggregate (95) @@ -588,40 +588,40 @@ Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (89) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#145, ss_sold_date_sk#146] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (90) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] (91) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] +Condition : ((isnotnull(ss_quantity#145) AND (ss_quantity#145 >= 81)) AND (ss_quantity#145 <= 100)) (92) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] (93) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#117] -Results [1]: [count#118] +Aggregate Attributes [1]: [count#147] +Results [1]: [count#148] (94) Exchange -Input [1]: [count#118] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#119] +Input [1]: [count#148] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#149] (95) HashAggregate [codegen id : 2] -Input [1]: [count#118] +Input [1]: [count#148] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#120] -Results [1]: [count(1)#120 AS count(1)#121] +Aggregate Attributes [1]: [count(1)#150] +Results [1]: [count(1)#150 AS count(1)#151] Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#32, [id=#33] * HashAggregate (102) @@ -634,40 +634,40 @@ Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (96) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (97) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] (98) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Condition : ((isnotnull(ss_quantity#152) AND (ss_quantity#152 >= 81)) AND (ss_quantity#152 <= 100)) (99) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#153] +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] (100) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#153] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#122, count#123] -Results [2]: [sum#124, count#125] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#153))] +Aggregate Attributes [2]: [sum#155, count#156] +Results [2]: [sum#157, count#158] (101) Exchange -Input [2]: [sum#124, count#125] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#126] +Input [2]: [sum#157, count#158] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#159] (102) HashAggregate [codegen id : 2] -Input [2]: [sum#124, count#125] +Input [2]: [sum#157, count#158] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#127] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#127 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#128] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))#160] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#153))#160 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#161] Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#34, [id=#35] * HashAggregate (109) @@ -680,39 +680,39 @@ Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (103) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] (105) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Condition : ((isnotnull(ss_quantity#162) AND (ss_quantity#162 >= 81)) AND (ss_quantity#162 <= 100)) (106) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#163] +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] (107) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#163] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#129, count#130] -Results [2]: [sum#131, count#132] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#163))] +Aggregate Attributes [2]: [sum#165, count#166] +Results [2]: [sum#167, count#168] (108) Exchange -Input [2]: [sum#131, count#132] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#133] +Input [2]: [sum#167, count#168] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#169] (109) HashAggregate [codegen id : 2] -Input [2]: [sum#131, count#132] +Input [2]: [sum#167, count#168] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#134] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#134 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#135] +Functions [1]: [avg(UnscaledValue(ss_net_paid#163))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#163))#170] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#163))#170 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#171] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt index a74b232e8c667..5226fce4ef512 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt @@ -183,108 +183,108 @@ Aggregate Attributes [1]: [count(1)#17] Results [1]: [count(1)#17 AS amc#18] (29) Scan parquet default.web_sales -Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Output [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] (31) Filter [codegen id : 8] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Condition : ((isnotnull(ws_ship_hdemo_sk#20) AND isnotnull(ws_sold_time_sk#19)) AND isnotnull(ws_web_page_sk#21)) (32) Project [codegen id : 8] -Output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Output [3]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] (33) ReusedExchange [Reuses operator id: 9] -Output [1]: [wp_web_page_sk#5] +Output [1]: [wp_web_page_sk#23] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#3] -Right keys [1]: [wp_web_page_sk#5] +Left keys [1]: [ws_web_page_sk#21] +Right keys [1]: [wp_web_page_sk#23] Join condition: None (35) Project [codegen id : 8] -Output [2]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, wp_web_page_sk#5] +Output [2]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, wp_web_page_sk#23] (36) ReusedExchange [Reuses operator id: 16] -Output [1]: [hd_demo_sk#8] +Output [1]: [hd_demo_sk#24] (37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_ship_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#8] +Left keys [1]: [ws_ship_hdemo_sk#20] +Right keys [1]: [hd_demo_sk#24] Join condition: None (38) Project [codegen id : 8] -Output [1]: [ws_sold_time_sk#1] -Input [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, hd_demo_sk#8] +Output [1]: [ws_sold_time_sk#19] +Input [3]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, hd_demo_sk#24] (39) Scan parquet default.time_dim -Output [2]: [t_time_sk#11, t_hour#12] +Output [2]: [t_time_sk#25, t_hour#26] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 7] -Input [2]: [t_time_sk#11, t_hour#12] +Input [2]: [t_time_sk#25, t_hour#26] (41) Filter [codegen id : 7] -Input [2]: [t_time_sk#11, t_hour#12] -Condition : (((isnotnull(t_hour#12) AND (t_hour#12 >= 19)) AND (t_hour#12 <= 20)) AND isnotnull(t_time_sk#11)) +Input [2]: [t_time_sk#25, t_hour#26] +Condition : (((isnotnull(t_hour#26) AND (t_hour#26 >= 19)) AND (t_hour#26 <= 20)) AND isnotnull(t_time_sk#25)) (42) Project [codegen id : 7] -Output [1]: [t_time_sk#11] -Input [2]: [t_time_sk#11, t_hour#12] +Output [1]: [t_time_sk#25] +Input [2]: [t_time_sk#25, t_hour#26] (43) BroadcastExchange -Input [1]: [t_time_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [t_time_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_time_sk#1] -Right keys [1]: [t_time_sk#11] +Left keys [1]: [ws_sold_time_sk#19] +Right keys [1]: [t_time_sk#25] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ws_sold_time_sk#1, t_time_sk#11] +Input [2]: [ws_sold_time_sk#19, t_time_sk#25] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#20] -Results [1]: [count#21] +Aggregate Attributes [1]: [count#28] +Results [1]: [count#29] (47) Exchange -Input [1]: [count#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [count#29] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#30] (48) HashAggregate [codegen id : 9] -Input [1]: [count#21] +Input [1]: [count#29] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#23] -Results [1]: [count(1)#23 AS pmc#24] +Aggregate Attributes [1]: [count(1)#31] +Results [1]: [count(1)#31 AS pmc#32] (49) BroadcastExchange -Input [1]: [pmc#24] -Arguments: IdentityBroadcastMode, [id=#25] +Input [1]: [pmc#32] +Arguments: IdentityBroadcastMode, [id=#33] (50) BroadcastNestedLoopJoin [codegen id : 10] Join condition: None (51) Project [codegen id : 10] -Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#24 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#26] -Input [2]: [amc#18, pmc#24] +Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#32 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#34] +Input [2]: [amc#18, pmc#32] (52) Sort [codegen id : 10] -Input [1]: [am_pm_ratio#26] -Arguments: [am_pm_ratio#26 ASC NULLS FIRST], true, 0 +Input [1]: [am_pm_ratio#34] +Arguments: [am_pm_ratio#34 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt index 070b497ac4d18..e369a027040d3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt @@ -183,108 +183,108 @@ Aggregate Attributes [1]: [count(1)#17] Results [1]: [count(1)#17 AS amc#18] (29) Scan parquet default.web_sales -Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Output [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] (31) Filter [codegen id : 8] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Condition : ((isnotnull(ws_ship_hdemo_sk#20) AND isnotnull(ws_sold_time_sk#19)) AND isnotnull(ws_web_page_sk#21)) (32) Project [codegen id : 8] -Output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Output [3]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] (33) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#23] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_ship_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ws_ship_hdemo_sk#20] +Right keys [1]: [hd_demo_sk#23] Join condition: None (35) Project [codegen id : 8] -Output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] +Output [2]: [ws_sold_time_sk#19, ws_web_page_sk#21] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, hd_demo_sk#23] (36) Scan parquet default.time_dim -Output [2]: [t_time_sk#8, t_hour#9] +Output [2]: [t_time_sk#24, t_hour#25] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [2]: [t_time_sk#8, t_hour#9] +Input [2]: [t_time_sk#24, t_hour#25] (38) Filter [codegen id : 6] -Input [2]: [t_time_sk#8, t_hour#9] -Condition : (((isnotnull(t_hour#9) AND (t_hour#9 >= 19)) AND (t_hour#9 <= 20)) AND isnotnull(t_time_sk#8)) +Input [2]: [t_time_sk#24, t_hour#25] +Condition : (((isnotnull(t_hour#25) AND (t_hour#25 >= 19)) AND (t_hour#25 <= 20)) AND isnotnull(t_time_sk#24)) (39) Project [codegen id : 6] -Output [1]: [t_time_sk#8] -Input [2]: [t_time_sk#8, t_hour#9] +Output [1]: [t_time_sk#24] +Input [2]: [t_time_sk#24, t_hour#25] (40) BroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [t_time_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_time_sk#1] -Right keys [1]: [t_time_sk#8] +Left keys [1]: [ws_sold_time_sk#19] +Right keys [1]: [t_time_sk#24] Join condition: None (42) Project [codegen id : 8] -Output [1]: [ws_web_page_sk#3] -Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#8] +Output [1]: [ws_web_page_sk#21] +Input [3]: [ws_sold_time_sk#19, ws_web_page_sk#21, t_time_sk#24] (43) ReusedExchange [Reuses operator id: 23] -Output [1]: [wp_web_page_sk#11] +Output [1]: [wp_web_page_sk#27] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#3] -Right keys [1]: [wp_web_page_sk#11] +Left keys [1]: [ws_web_page_sk#21] +Right keys [1]: [wp_web_page_sk#27] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ws_web_page_sk#3, wp_web_page_sk#11] +Input [2]: [ws_web_page_sk#21, wp_web_page_sk#27] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#20] -Results [1]: [count#21] +Aggregate Attributes [1]: [count#28] +Results [1]: [count#29] (47) Exchange -Input [1]: [count#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [count#29] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#30] (48) HashAggregate [codegen id : 9] -Input [1]: [count#21] +Input [1]: [count#29] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#23] -Results [1]: [count(1)#23 AS pmc#24] +Aggregate Attributes [1]: [count(1)#31] +Results [1]: [count(1)#31 AS pmc#32] (49) BroadcastExchange -Input [1]: [pmc#24] -Arguments: IdentityBroadcastMode, [id=#25] +Input [1]: [pmc#32] +Arguments: IdentityBroadcastMode, [id=#33] (50) BroadcastNestedLoopJoin [codegen id : 10] Join condition: None (51) Project [codegen id : 10] -Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#24 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#26] -Input [2]: [amc#18, pmc#24] +Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#32 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#34] +Input [2]: [amc#18, pmc#32] (52) Sort [codegen id : 10] -Input [1]: [am_pm_ratio#26] -Arguments: [am_pm_ratio#26 ASC NULLS FIRST], true, 0 +Input [1]: [am_pm_ratio#34] +Arguments: [am_pm_ratio#34 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index 7d525630d8a9a..46953fe184f28 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -107,13 +107,8 @@ ReadSchema: struct Input [2]: [d_date_sk#11, d_date#12] (16) Filter [codegen id : 2] -<<<<<<< HEAD Input [2]: [d_date_sk#11, d_date#12] Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) -======= -Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) ->>>>>>> abfd9b23cd7c21e9525df85a16e0611ef0f35908 (17) Project [codegen id : 2] Output [1]: [d_date_sk#11] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index c77590bf71044..8a5b567285939 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -77,30 +77,30 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet default.web_sales -Output [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Output [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Input [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] (9) Project [codegen id : 3] -Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#10, ws_order_number#5 AS ws_order_number#5#11] -Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Output [2]: [ws_warehouse_sk#10, ws_order_number#11] +Input [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] (10) Exchange -Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] -Arguments: hashpartitioning(ws_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [2]: [ws_warehouse_sk#10, ws_order_number#11] +Arguments: hashpartitioning(ws_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 4] -Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] -Arguments: [ws_order_number#5#11 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#10, ws_order_number#11] +Arguments: [ws_order_number#11 ASC NULLS FIRST], false, 0 (12) SortMergeJoin Left keys [1]: [ws_order_number#5] -Right keys [1]: [ws_order_number#5#11] -Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#10) +Right keys [1]: [ws_order_number#11] +Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#10) (13) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -108,168 +108,168 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (14) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#13] +Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#14] (15) Sort [codegen id : 6] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: [cast(ws_order_number#5 as bigint) ASC NULLS FIRST], false, 0 (16) Scan parquet default.web_returns -Output [2]: [wr_order_number#14, wr_returned_date_sk#15] +Output [2]: [wr_order_number#15, wr_returned_date_sk#16] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] ReadSchema: struct (17) ColumnarToRow [codegen id : 7] -Input [2]: [wr_order_number#14, wr_returned_date_sk#15] +Input [2]: [wr_order_number#15, wr_returned_date_sk#16] (18) Project [codegen id : 7] -Output [1]: [wr_order_number#14] -Input [2]: [wr_order_number#14, wr_returned_date_sk#15] +Output [1]: [wr_order_number#15] +Input [2]: [wr_order_number#15, wr_returned_date_sk#16] (19) Exchange -Input [1]: [wr_order_number#14] -Arguments: hashpartitioning(wr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [1]: [wr_order_number#15] +Arguments: hashpartitioning(wr_order_number#15, 5), ENSURE_REQUIREMENTS, [id=#17] (20) Sort [codegen id : 8] -Input [1]: [wr_order_number#14] -Arguments: [wr_order_number#14 ASC NULLS FIRST], false, 0 +Input [1]: [wr_order_number#15] +Arguments: [wr_order_number#15 ASC NULLS FIRST], false, 0 (21) SortMergeJoin Left keys [1]: [cast(ws_order_number#5 as bigint)] -Right keys [1]: [wr_order_number#14] +Right keys [1]: [wr_order_number#15] Join condition: None (22) Scan parquet default.customer_address -Output [2]: [ca_address_sk#17, ca_state#18] +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 (23) ColumnarToRow [codegen id : 9] -Input [2]: [ca_address_sk#17, ca_state#18] +Input [2]: [ca_address_sk#18, ca_state#19] (24) Filter [codegen id : 9] -Input [2]: [ca_address_sk#17, ca_state#18] -Condition : ((isnotnull(ca_state#18) AND (ca_state#18 = IL)) AND isnotnull(ca_address_sk#17)) +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)) (25) Project [codegen id : 9] -Output [1]: [ca_address_sk#17] -Input [2]: [ca_address_sk#17, ca_state#18] +Output [1]: [ca_address_sk#18] +Input [2]: [ca_address_sk#18, ca_state#19] (26) BroadcastExchange -Input [1]: [ca_address_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [ca_address_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (27) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#17] +Right keys [1]: [ca_address_sk#18] Join condition: None (28) Project [codegen id : 12] Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#17] +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#18] (29) Scan parquet default.web_site -Output [2]: [web_site_sk#20, web_company_name#21] +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 (30) ColumnarToRow [codegen id : 10] -Input [2]: [web_site_sk#20, web_company_name#21] +Input [2]: [web_site_sk#21, web_company_name#22] (31) Filter [codegen id : 10] -Input [2]: [web_site_sk#20, web_company_name#21] -Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri )) AND isnotnull(web_site_sk#20)) +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)) (32) Project [codegen id : 10] -Output [1]: [web_site_sk#20] -Input [2]: [web_site_sk#20, web_company_name#21] +Output [1]: [web_site_sk#21] +Input [2]: [web_site_sk#21, web_company_name#22] (33) BroadcastExchange -Input [1]: [web_site_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [web_site_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#20] +Right keys [1]: [web_site_sk#21] Join condition: None (35) Project [codegen id : 12] Output [4]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#20] +Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#21] (36) Scan parquet default.date_dim -Output [2]: [d_date_sk#23, d_date#24] +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 (37) ColumnarToRow [codegen id : 11] -Input [2]: [d_date_sk#23, d_date#24] +Input [2]: [d_date_sk#24, d_date#25] (38) Filter [codegen id : 11] -Input [2]: [d_date_sk#23, d_date#24] -Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 1999-02-01)) AND (d_date#24 <= 1999-04-02)) AND isnotnull(d_date_sk#23)) +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)) (39) Project [codegen id : 11] -Output [1]: [d_date_sk#23] -Input [2]: [d_date_sk#23, d_date#24] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_date#25] (40) BroadcastExchange -Input [1]: [d_date_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None (42) Project [codegen id : 12] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [5]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#23] +Input [5]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#24] (43) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] -Results [3]: [ws_order_number#5, sum#28, sum#29] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28] +Results [3]: [ws_order_number#5, sum#29, sum#30] (44) Exchange -Input [3]: [ws_order_number#5, sum#28, sum#29] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [ws_order_number#5, sum#29, sum#30] +Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#31] (45) HashAggregate [codegen id : 13] -Input [3]: [ws_order_number#5, sum#28, sum#29] +Input [3]: [ws_order_number#5, sum#29, sum#30] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] -Results [3]: [ws_order_number#5, sum#28, sum#29] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28] +Results [3]: [ws_order_number#5, sum#29, sum#30] (46) HashAggregate [codegen id : 13] -Input [3]: [ws_order_number#5, sum#28, sum#29] +Input [3]: [ws_order_number#5, sum#29, sum#30] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] -Results [3]: [sum#28, sum#29, count#32] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28, count(ws_order_number#5)#32] +Results [3]: [sum#29, sum#30, count#33] (47) Exchange -Input [3]: [sum#28, sum#29, count#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [sum#29, sum#30, count#33] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#34] (48) HashAggregate [codegen id : 14] -Input [3]: [sum#28, sum#29, count#32] +Input [3]: [sum#29, sum#30, count#33] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] -Results [3]: [count(ws_order_number#5)#31 AS order count #34, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#26,17,2) AS total shipping cost #35, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#27,17,2) AS total net profit #36] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28, count(ws_order_number#5)#32] +Results [3]: [count(ws_order_number#5)#32 AS order count #35, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#27,17,2) AS total shipping cost #36, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#28,17,2) AS total net profit #37] (49) Sort [codegen id : 14] -Input [3]: [order count #34, total shipping cost #35, total net profit #36] -Arguments: [order count #34 ASC NULLS FIRST], true, 0 +Input [3]: [order count #35, total shipping cost #36, total net profit #37] +Arguments: [order count #35 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index 9558a01423452..042d4a3e347a2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -77,30 +77,30 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet default.web_sales -Output [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Output [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Input [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] (9) Project [codegen id : 3] -Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#10, ws_order_number#5 AS ws_order_number#5#11] -Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Output [2]: [ws_warehouse_sk#10, ws_order_number#11] +Input [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] (10) Exchange -Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] -Arguments: hashpartitioning(ws_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [2]: [ws_warehouse_sk#10, ws_order_number#11] +Arguments: hashpartitioning(ws_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 4] -Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] -Arguments: [ws_order_number#5#11 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#10, ws_order_number#11] +Arguments: [ws_order_number#11 ASC NULLS FIRST], false, 0 (12) SortMergeJoin Left keys [1]: [ws_order_number#5] -Right keys [1]: [ws_order_number#5#11] -Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#10) +Right keys [1]: [ws_order_number#11] +Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#10) (13) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -108,168 +108,168 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (14) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#13] +Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#14] (15) Sort [codegen id : 6] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: [cast(ws_order_number#5 as bigint) ASC NULLS FIRST], false, 0 (16) Scan parquet default.web_returns -Output [2]: [wr_order_number#14, wr_returned_date_sk#15] +Output [2]: [wr_order_number#15, wr_returned_date_sk#16] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] ReadSchema: struct (17) ColumnarToRow [codegen id : 7] -Input [2]: [wr_order_number#14, wr_returned_date_sk#15] +Input [2]: [wr_order_number#15, wr_returned_date_sk#16] (18) Project [codegen id : 7] -Output [1]: [wr_order_number#14] -Input [2]: [wr_order_number#14, wr_returned_date_sk#15] +Output [1]: [wr_order_number#15] +Input [2]: [wr_order_number#15, wr_returned_date_sk#16] (19) Exchange -Input [1]: [wr_order_number#14] -Arguments: hashpartitioning(wr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [1]: [wr_order_number#15] +Arguments: hashpartitioning(wr_order_number#15, 5), ENSURE_REQUIREMENTS, [id=#17] (20) Sort [codegen id : 8] -Input [1]: [wr_order_number#14] -Arguments: [wr_order_number#14 ASC NULLS FIRST], false, 0 +Input [1]: [wr_order_number#15] +Arguments: [wr_order_number#15 ASC NULLS FIRST], false, 0 (21) SortMergeJoin Left keys [1]: [cast(ws_order_number#5 as bigint)] -Right keys [1]: [wr_order_number#14] +Right keys [1]: [wr_order_number#15] Join condition: None (22) Scan parquet default.date_dim -Output [2]: [d_date_sk#17, d_date#18] +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 (23) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#17, d_date#18] +Input [2]: [d_date_sk#18, d_date#19] (24) Filter [codegen id : 9] -Input [2]: [d_date_sk#17, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-01)) AND (d_date#18 <= 1999-04-02)) AND isnotnull(d_date_sk#17)) +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)) (25) Project [codegen id : 9] -Output [1]: [d_date_sk#17] -Input [2]: [d_date_sk#17, d_date#18] +Output [1]: [d_date_sk#18] +Input [2]: [d_date_sk#18, d_date#19] (26) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [d_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (27) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#17] +Right keys [1]: [d_date_sk#18] Join condition: None (28) Project [codegen id : 12] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#17] +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#18] (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +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 (30) ColumnarToRow [codegen id : 10] -Input [2]: [ca_address_sk#20, ca_state#21] +Input [2]: [ca_address_sk#21, ca_state#22] (31) Filter [codegen id : 10] -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : ((isnotnull(ca_state#21) AND (ca_state#21 = IL)) AND isnotnull(ca_address_sk#20)) +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)) (32) Project [codegen id : 10] -Output [1]: [ca_address_sk#20] -Input [2]: [ca_address_sk#20, ca_state#21] +Output [1]: [ca_address_sk#21] +Input [2]: [ca_address_sk#21, ca_state#22] (33) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [ca_address_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#21] Join condition: None (35) Project [codegen id : 12] Output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#20] +Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#21] (36) Scan parquet default.web_site -Output [2]: [web_site_sk#23, web_company_name#24] +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 (37) ColumnarToRow [codegen id : 11] -Input [2]: [web_site_sk#23, web_company_name#24] +Input [2]: [web_site_sk#24, web_company_name#25] (38) Filter [codegen id : 11] -Input [2]: [web_site_sk#23, web_company_name#24] -Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri )) AND isnotnull(web_site_sk#23)) +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)) (39) Project [codegen id : 11] -Output [1]: [web_site_sk#23] -Input [2]: [web_site_sk#23, web_company_name#24] +Output [1]: [web_site_sk#24] +Input [2]: [web_site_sk#24, web_company_name#25] (40) BroadcastExchange -Input [1]: [web_site_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] +Input [1]: [web_site_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#23] +Right keys [1]: [web_site_sk#24] Join condition: None (42) Project [codegen id : 12] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#23] +Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#24] (43) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] -Results [3]: [ws_order_number#5, sum#28, sum#29] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28] +Results [3]: [ws_order_number#5, sum#29, sum#30] (44) Exchange -Input [3]: [ws_order_number#5, sum#28, sum#29] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [ws_order_number#5, sum#29, sum#30] +Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#31] (45) HashAggregate [codegen id : 13] -Input [3]: [ws_order_number#5, sum#28, sum#29] +Input [3]: [ws_order_number#5, sum#29, sum#30] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] -Results [3]: [ws_order_number#5, sum#28, sum#29] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28] +Results [3]: [ws_order_number#5, sum#29, sum#30] (46) HashAggregate [codegen id : 13] -Input [3]: [ws_order_number#5, sum#28, sum#29] +Input [3]: [ws_order_number#5, sum#29, sum#30] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] -Results [3]: [sum#28, sum#29, count#32] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28, count(ws_order_number#5)#32] +Results [3]: [sum#29, sum#30, count#33] (47) Exchange -Input [3]: [sum#28, sum#29, count#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [sum#29, sum#30, count#33] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#34] (48) HashAggregate [codegen id : 14] -Input [3]: [sum#28, sum#29, count#32] +Input [3]: [sum#29, sum#30, count#33] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] -Results [3]: [count(ws_order_number#5)#31 AS order count #34, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#26,17,2) AS total shipping cost #35, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#27,17,2) AS total net profit #36] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28, count(ws_order_number#5)#32] +Results [3]: [count(ws_order_number#5)#32 AS order count #35, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#27,17,2) AS total shipping cost #36, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#28,17,2) AS total net profit #37] (49) Sort [codegen id : 14] -Input [3]: [order count #34, total shipping cost #35, total net profit #36] -Arguments: [order count #34 ASC NULLS FIRST], true, 0 +Input [3]: [order count #35, total shipping cost #36, total net profit #37] +Arguments: [order count #35 ASC NULLS FIRST], true, 0 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 320a93e19bb27..ee82855f8c010 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 @@ -92,261 +92,261 @@ Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (7) Scan parquet default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] (9) Filter [codegen id : 3] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] -Condition : (isnotnull(ws_order_number#4) AND isnotnull(ws_warehouse_sk#9)) +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] +Condition : (isnotnull(ws_order_number#10) AND isnotnull(ws_warehouse_sk#9)) (10) Project [codegen id : 3] -Output [2]: [ws_warehouse_sk#9, ws_order_number#4] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Output [2]: [ws_warehouse_sk#9, ws_order_number#10] +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] (11) Exchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#10] +Input [2]: [ws_warehouse_sk#9, ws_order_number#10] +Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, [id=#12] (12) Sort [codegen id : 4] -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: [ws_order_number#4 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 (13) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] +Output [2]: [ws_warehouse_sk#13, ws_order_number#14] (14) Sort [codegen id : 6] -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#12 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 (15) SortMergeJoin [codegen id : 7] -Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#12] -Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#11) +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) (16) Project [codegen id : 7] -Output [1]: [ws_order_number#4 AS ws_order_number#4#13] -Input [4]: [ws_warehouse_sk#9, ws_order_number#4, ws_warehouse_sk#11, ws_order_number#12] +Output [1]: [ws_order_number#10] +Input [4]: [ws_warehouse_sk#9, ws_order_number#10, ws_warehouse_sk#13, ws_order_number#14] (17) SortMergeJoin Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#4#13] +Right keys [1]: [ws_order_number#10] Join condition: None (18) Exchange Input [6]: [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] -Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#15] (19) Sort [codegen id : 8] Input [6]: [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] Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 (20) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#9, ws_order_number#4] +Output [2]: [ws_warehouse_sk#16, ws_order_number#17] (21) Sort [codegen id : 10] -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#16, ws_order_number#17] +Arguments: [ws_order_number#17 ASC NULLS FIRST], false, 0 (22) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#15, ws_order_number#16] +Output [2]: [ws_warehouse_sk#18, ws_order_number#19] (23) Sort [codegen id : 12] -Input [2]: [ws_warehouse_sk#15, ws_order_number#16] -Arguments: [ws_order_number#16 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#18, ws_order_number#19] +Arguments: [ws_order_number#19 ASC NULLS FIRST], false, 0 (24) SortMergeJoin [codegen id : 13] -Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#16] -Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#15) +Left keys [1]: [ws_order_number#17] +Right keys [1]: [ws_order_number#19] +Join condition: NOT (ws_warehouse_sk#16 = ws_warehouse_sk#18) (25) Project [codegen id : 13] -Output [1]: [ws_order_number#4] -Input [4]: [ws_warehouse_sk#9, ws_order_number#4, ws_warehouse_sk#15, ws_order_number#16] +Output [1]: [ws_order_number#17] +Input [4]: [ws_warehouse_sk#16, ws_order_number#17, ws_warehouse_sk#18, ws_order_number#19] (26) Exchange -Input [1]: [ws_order_number#4] -Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#17] +Input [1]: [ws_order_number#17] +Arguments: hashpartitioning(cast(ws_order_number#17 as bigint), 5), ENSURE_REQUIREMENTS, [id=#20] (27) Sort [codegen id : 14] -Input [1]: [ws_order_number#4] -Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 +Input [1]: [ws_order_number#17] +Arguments: [cast(ws_order_number#17 as bigint) ASC NULLS FIRST], false, 0 (28) Scan parquet default.web_returns -Output [2]: [wr_order_number#18, wr_returned_date_sk#19] +Output [2]: [wr_order_number#21, wr_returned_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct (29) ColumnarToRow [codegen id : 15] -Input [2]: [wr_order_number#18, wr_returned_date_sk#19] +Input [2]: [wr_order_number#21, wr_returned_date_sk#22] (30) Filter [codegen id : 15] -Input [2]: [wr_order_number#18, wr_returned_date_sk#19] -Condition : isnotnull(wr_order_number#18) +Input [2]: [wr_order_number#21, wr_returned_date_sk#22] +Condition : isnotnull(wr_order_number#21) (31) Project [codegen id : 15] -Output [1]: [wr_order_number#18] -Input [2]: [wr_order_number#18, wr_returned_date_sk#19] +Output [1]: [wr_order_number#21] +Input [2]: [wr_order_number#21, wr_returned_date_sk#22] (32) Exchange -Input [1]: [wr_order_number#18] -Arguments: hashpartitioning(wr_order_number#18, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [wr_order_number#21] +Arguments: hashpartitioning(wr_order_number#21, 5), ENSURE_REQUIREMENTS, [id=#23] (33) Sort [codegen id : 16] -Input [1]: [wr_order_number#18] -Arguments: [wr_order_number#18 ASC NULLS FIRST], false, 0 +Input [1]: [wr_order_number#21] +Arguments: [wr_order_number#21 ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 17] -Left keys [1]: [cast(ws_order_number#4 as bigint)] -Right keys [1]: [wr_order_number#18] +Left keys [1]: [cast(ws_order_number#17 as bigint)] +Right keys [1]: [wr_order_number#21] Join condition: None (35) Project [codegen id : 17] -Output [1]: [wr_order_number#18] -Input [2]: [ws_order_number#4, wr_order_number#18] +Output [1]: [wr_order_number#21] +Input [2]: [ws_order_number#17, wr_order_number#21] (36) SortMergeJoin Left keys [1]: [cast(ws_order_number#4 as bigint)] -Right keys [1]: [wr_order_number#18] +Right keys [1]: [wr_order_number#21] Join condition: None (37) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] +Output [2]: [ca_address_sk#24, ca_state#25] 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 (38) ColumnarToRow [codegen id : 18] -Input [2]: [ca_address_sk#21, ca_state#22] +Input [2]: [ca_address_sk#24, ca_state#25] (39) Filter [codegen id : 18] -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)) +Input [2]: [ca_address_sk#24, ca_state#25] +Condition : ((isnotnull(ca_state#25) AND (ca_state#25 = IL)) AND isnotnull(ca_address_sk#24)) (40) Project [codegen id : 18] -Output [1]: [ca_address_sk#21] -Input [2]: [ca_address_sk#21, ca_state#22] +Output [1]: [ca_address_sk#24] +Input [2]: [ca_address_sk#24, ca_state#25] (41) BroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [ca_address_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (42) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#24] Join condition: None (43) Project [codegen id : 21] 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#21] +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#24] (44) Scan parquet default.web_site -Output [2]: [web_site_sk#24, web_company_name#25] +Output [2]: [web_site_sk#27, web_company_name#28] 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 (45) ColumnarToRow [codegen id : 19] -Input [2]: [web_site_sk#24, web_company_name#25] +Input [2]: [web_site_sk#27, web_company_name#28] (46) Filter [codegen id : 19] -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)) +Input [2]: [web_site_sk#27, web_company_name#28] +Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri )) AND isnotnull(web_site_sk#27)) (47) Project [codegen id : 19] -Output [1]: [web_site_sk#24] -Input [2]: [web_site_sk#24, web_company_name#25] +Output [1]: [web_site_sk#27] +Input [2]: [web_site_sk#27, web_company_name#28] (48) BroadcastExchange -Input [1]: [web_site_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [web_site_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (49) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#24] +Right keys [1]: [web_site_sk#27] Join condition: None (50) Project [codegen id : 21] 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#24] +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#27] (51) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_date#28] +Output [2]: [d_date_sk#30, d_date#31] 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 (52) ColumnarToRow [codegen id : 20] -Input [2]: [d_date_sk#27, d_date#28] +Input [2]: [d_date_sk#30, d_date#31] (53) Filter [codegen id : 20] -Input [2]: [d_date_sk#27, d_date#28] -Condition : (((isnotnull(d_date#28) AND (d_date#28 >= 1999-02-01)) AND (d_date#28 <= 1999-04-02)) AND isnotnull(d_date_sk#27)) +Input [2]: [d_date_sk#30, d_date#31] +Condition : (((isnotnull(d_date#31) AND (d_date#31 >= 1999-02-01)) AND (d_date#31 <= 1999-04-02)) AND isnotnull(d_date_sk#30)) (54) Project [codegen id : 20] -Output [1]: [d_date_sk#27] -Input [2]: [d_date_sk#27, d_date#28] +Output [1]: [d_date_sk#30] +Input [2]: [d_date_sk#30, d_date#31] (55) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] +Input [1]: [d_date_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] (56) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#30] Join condition: None (57) Project [codegen id : 21] 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#27] +Input [5]: [ws_ship_date_sk#1, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#30] (58) HashAggregate [codegen id : 21] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31] -Results [3]: [ws_order_number#4, sum#32, sum#33] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34] +Results [3]: [ws_order_number#4, sum#35, sum#36] (59) Exchange -Input [3]: [ws_order_number#4, sum#32, sum#33] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [ws_order_number#4, sum#35, sum#36] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#37] (60) HashAggregate [codegen id : 22] -Input [3]: [ws_order_number#4, sum#32, sum#33] +Input [3]: [ws_order_number#4, sum#35, sum#36] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31] -Results [3]: [ws_order_number#4, sum#32, sum#33] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34] +Results [3]: [ws_order_number#4, sum#35, sum#36] (61) HashAggregate [codegen id : 22] -Input [3]: [ws_order_number#4, sum#32, sum#33] +Input [3]: [ws_order_number#4, sum#35, sum#36] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31, count(ws_order_number#4)#35] -Results [3]: [sum#32, sum#33, count#36] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34, count(ws_order_number#4)#38] +Results [3]: [sum#35, sum#36, count#39] (62) Exchange -Input [3]: [sum#32, sum#33, count#36] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [sum#35, sum#36, count#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] (63) HashAggregate [codegen id : 23] -Input [3]: [sum#32, sum#33, count#36] +Input [3]: [sum#35, sum#36, count#39] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31, 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))#30,17,2) AS total shipping cost #39, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#31,17,2) AS total net profit #40] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34, count(ws_order_number#4)#38] +Results [3]: [count(ws_order_number#4)#38 AS order count #41, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#33,17,2) AS total shipping cost #42, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#34,17,2) AS total net profit #43] (64) Sort [codegen id : 23] -Input [3]: [order count #38, total shipping cost #39, total net profit #40] -Arguments: [order count #38 ASC NULLS FIRST], true, 0 +Input [3]: [order count #41, total shipping cost #42, total net profit #43] +Arguments: [order count #41 ASC NULLS FIRST], true, 0 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 2f719a4be1630..73f7f88aeb68d 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 @@ -92,261 +92,261 @@ Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (7) Scan parquet default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] (9) Filter [codegen id : 3] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] -Condition : (isnotnull(ws_order_number#4) AND isnotnull(ws_warehouse_sk#9)) +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] +Condition : (isnotnull(ws_order_number#10) AND isnotnull(ws_warehouse_sk#9)) (10) Project [codegen id : 3] -Output [2]: [ws_warehouse_sk#9, ws_order_number#4] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Output [2]: [ws_warehouse_sk#9, ws_order_number#10] +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] (11) Exchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#10] +Input [2]: [ws_warehouse_sk#9, ws_order_number#10] +Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, [id=#12] (12) Sort [codegen id : 4] -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: [ws_order_number#4 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 (13) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] +Output [2]: [ws_warehouse_sk#13, ws_order_number#14] (14) Sort [codegen id : 6] -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#12 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 (15) SortMergeJoin [codegen id : 7] -Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#12] -Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#11) +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) (16) Project [codegen id : 7] -Output [1]: [ws_order_number#4 AS ws_order_number#4#13] -Input [4]: [ws_warehouse_sk#9, ws_order_number#4, ws_warehouse_sk#11, ws_order_number#12] +Output [1]: [ws_order_number#10] +Input [4]: [ws_warehouse_sk#9, ws_order_number#10, ws_warehouse_sk#13, ws_order_number#14] (17) SortMergeJoin Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#4#13] +Right keys [1]: [ws_order_number#10] Join condition: None (18) Exchange Input [6]: [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] -Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#15] (19) Sort [codegen id : 8] Input [6]: [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] Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 (20) Scan parquet default.web_returns -Output [2]: [wr_order_number#15, wr_returned_date_sk#16] +Output [2]: [wr_order_number#16, wr_returned_date_sk#17] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct (21) ColumnarToRow [codegen id : 9] -Input [2]: [wr_order_number#15, wr_returned_date_sk#16] +Input [2]: [wr_order_number#16, wr_returned_date_sk#17] (22) Filter [codegen id : 9] -Input [2]: [wr_order_number#15, wr_returned_date_sk#16] -Condition : isnotnull(wr_order_number#15) +Input [2]: [wr_order_number#16, wr_returned_date_sk#17] +Condition : isnotnull(wr_order_number#16) (23) Project [codegen id : 9] -Output [1]: [wr_order_number#15] -Input [2]: [wr_order_number#15, wr_returned_date_sk#16] +Output [1]: [wr_order_number#16] +Input [2]: [wr_order_number#16, wr_returned_date_sk#17] (24) Exchange -Input [1]: [wr_order_number#15] -Arguments: hashpartitioning(wr_order_number#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [1]: [wr_order_number#16] +Arguments: hashpartitioning(wr_order_number#16, 5), ENSURE_REQUIREMENTS, [id=#18] (25) Sort [codegen id : 10] -Input [1]: [wr_order_number#15] -Arguments: [wr_order_number#15 ASC NULLS FIRST], false, 0 +Input [1]: [wr_order_number#16] +Arguments: [wr_order_number#16 ASC NULLS FIRST], false, 0 (26) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#9, ws_order_number#4] +Output [2]: [ws_warehouse_sk#19, ws_order_number#20] (27) Sort [codegen id : 12] -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#19, ws_order_number#20] +Arguments: [ws_order_number#20 ASC NULLS FIRST], false, 0 (28) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#18, ws_order_number#19] +Output [2]: [ws_warehouse_sk#21, ws_order_number#22] (29) Sort [codegen id : 14] -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#21, ws_order_number#22] +Arguments: [ws_order_number#22 ASC NULLS FIRST], false, 0 (30) SortMergeJoin [codegen id : 15] -Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#19] -Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#18) +Left keys [1]: [ws_order_number#20] +Right keys [1]: [ws_order_number#22] +Join condition: NOT (ws_warehouse_sk#19 = ws_warehouse_sk#21) (31) Project [codegen id : 15] -Output [1]: [ws_order_number#4] -Input [4]: [ws_warehouse_sk#9, ws_order_number#4, ws_warehouse_sk#18, ws_order_number#19] +Output [1]: [ws_order_number#20] +Input [4]: [ws_warehouse_sk#19, ws_order_number#20, ws_warehouse_sk#21, ws_order_number#22] (32) Exchange -Input [1]: [ws_order_number#4] -Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [ws_order_number#20] +Arguments: hashpartitioning(cast(ws_order_number#20 as bigint), 5), ENSURE_REQUIREMENTS, [id=#23] (33) Sort [codegen id : 16] -Input [1]: [ws_order_number#4] -Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 +Input [1]: [ws_order_number#20] +Arguments: [cast(ws_order_number#20 as bigint) ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 17] -Left keys [1]: [wr_order_number#15] -Right keys [1]: [cast(ws_order_number#4 as bigint)] +Left keys [1]: [wr_order_number#16] +Right keys [1]: [cast(ws_order_number#20 as bigint)] Join condition: None (35) Project [codegen id : 17] -Output [1]: [wr_order_number#15] -Input [2]: [wr_order_number#15, ws_order_number#4] +Output [1]: [wr_order_number#16] +Input [2]: [wr_order_number#16, ws_order_number#20] (36) SortMergeJoin Left keys [1]: [cast(ws_order_number#4 as bigint)] -Right keys [1]: [wr_order_number#15] +Right keys [1]: [wr_order_number#16] Join condition: None (37) Scan parquet default.date_dim -Output [2]: [d_date_sk#21, d_date#22] +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 (38) ColumnarToRow [codegen id : 18] -Input [2]: [d_date_sk#21, d_date#22] +Input [2]: [d_date_sk#24, d_date#25] (39) Filter [codegen id : 18] -Input [2]: [d_date_sk#21, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-01)) AND (d_date#22 <= 1999-04-02)) AND isnotnull(d_date_sk#21)) +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)) (40) Project [codegen id : 18] -Output [1]: [d_date_sk#21] -Input [2]: [d_date_sk#21, d_date#22] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_date#25] (41) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (42) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#21] +Right keys [1]: [d_date_sk#24] Join condition: None (43) Project [codegen id : 21] 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#21] +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#24] (44) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#27, ca_state#28] 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 (45) ColumnarToRow [codegen id : 19] -Input [2]: [ca_address_sk#24, ca_state#25] +Input [2]: [ca_address_sk#27, ca_state#28] (46) Filter [codegen id : 19] -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : ((isnotnull(ca_state#25) AND (ca_state#25 = IL)) AND isnotnull(ca_address_sk#24)) +Input [2]: [ca_address_sk#27, ca_state#28] +Condition : ((isnotnull(ca_state#28) AND (ca_state#28 = IL)) AND isnotnull(ca_address_sk#27)) (47) Project [codegen id : 19] -Output [1]: [ca_address_sk#24] -Input [2]: [ca_address_sk#24, ca_state#25] +Output [1]: [ca_address_sk#27] +Input [2]: [ca_address_sk#27, ca_state#28] (48) BroadcastExchange -Input [1]: [ca_address_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [ca_address_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (49) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#24] +Right keys [1]: [ca_address_sk#27] Join condition: None (50) Project [codegen id : 21] 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#24] +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#27] (51) Scan parquet default.web_site -Output [2]: [web_site_sk#27, web_company_name#28] +Output [2]: [web_site_sk#30, web_company_name#31] 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 (52) ColumnarToRow [codegen id : 20] -Input [2]: [web_site_sk#27, web_company_name#28] +Input [2]: [web_site_sk#30, web_company_name#31] (53) Filter [codegen id : 20] -Input [2]: [web_site_sk#27, web_company_name#28] -Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri )) AND isnotnull(web_site_sk#27)) +Input [2]: [web_site_sk#30, web_company_name#31] +Condition : ((isnotnull(web_company_name#31) AND (web_company_name#31 = pri )) AND isnotnull(web_site_sk#30)) (54) Project [codegen id : 20] -Output [1]: [web_site_sk#27] -Input [2]: [web_site_sk#27, web_company_name#28] +Output [1]: [web_site_sk#30] +Input [2]: [web_site_sk#30, web_company_name#31] (55) BroadcastExchange -Input [1]: [web_site_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] +Input [1]: [web_site_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] (56) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#27] +Right keys [1]: [web_site_sk#30] Join condition: None (57) Project [codegen id : 21] 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#27] +Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#30] (58) HashAggregate [codegen id : 21] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31] -Results [3]: [ws_order_number#4, sum#32, sum#33] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34] +Results [3]: [ws_order_number#4, sum#35, sum#36] (59) Exchange -Input [3]: [ws_order_number#4, sum#32, sum#33] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [ws_order_number#4, sum#35, sum#36] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#37] (60) HashAggregate [codegen id : 22] -Input [3]: [ws_order_number#4, sum#32, sum#33] +Input [3]: [ws_order_number#4, sum#35, sum#36] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31] -Results [3]: [ws_order_number#4, sum#32, sum#33] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34] +Results [3]: [ws_order_number#4, sum#35, sum#36] (61) HashAggregate [codegen id : 22] -Input [3]: [ws_order_number#4, sum#32, sum#33] +Input [3]: [ws_order_number#4, sum#35, sum#36] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31, count(ws_order_number#4)#35] -Results [3]: [sum#32, sum#33, count#36] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34, count(ws_order_number#4)#38] +Results [3]: [sum#35, sum#36, count#39] (62) Exchange -Input [3]: [sum#32, sum#33, count#36] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [sum#35, sum#36, count#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] (63) HashAggregate [codegen id : 23] -Input [3]: [sum#32, sum#33, count#36] +Input [3]: [sum#35, sum#36, count#39] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31, 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))#30,17,2) AS total shipping cost #39, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#31,17,2) AS total net profit #40] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34, count(ws_order_number#4)#38] +Results [3]: [count(ws_order_number#4)#38 AS order count #41, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#33,17,2) AS total shipping cost #42, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#34,17,2) AS total net profit #43] (64) Sort [codegen id : 23] -Input [3]: [order count #38, total shipping cost #39, total net profit #40] -Arguments: [order count #38 ASC NULLS FIRST], true, 0 +Input [3]: [order count #41, total shipping cost #42, total net profit #43] +Arguments: [order count #41 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt index 245357ef5560c..c01f3465ed693 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt @@ -102,16 +102,16 @@ ReadSchema: struct Input [3]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13] (16) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#14] (17) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#5] +Right keys [1]: [d_date_sk#14] Join condition: None (18) Project [codegen id : 5] Output [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#5] +Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#14] (19) HashAggregate [codegen id : 5] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] @@ -122,45 +122,45 @@ Results [2]: [cs_bill_customer_sk#11, cs_item_sk#12] (20) Exchange Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#15] (21) HashAggregate [codegen id : 6] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Keys [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Functions: [] Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#11 AS customer_sk#15, cs_item_sk#12 AS item_sk#16] +Results [2]: [cs_bill_customer_sk#11 AS customer_sk#16, cs_item_sk#12 AS item_sk#17] (22) Sort [codegen id : 6] -Input [2]: [customer_sk#15, item_sk#16] -Arguments: [customer_sk#15 ASC NULLS FIRST, item_sk#16 ASC NULLS FIRST], false, 0 +Input [2]: [customer_sk#16, item_sk#17] +Arguments: [customer_sk#16 ASC NULLS FIRST, item_sk#17 ASC NULLS FIRST], false, 0 (23) SortMergeJoin Left keys [2]: [customer_sk#9, item_sk#10] -Right keys [2]: [customer_sk#15, item_sk#16] +Right keys [2]: [customer_sk#16, item_sk#17] Join condition: None (24) Project [codegen id : 7] -Output [2]: [customer_sk#9, customer_sk#15] -Input [4]: [customer_sk#9, item_sk#10, customer_sk#15, item_sk#16] +Output [2]: [customer_sk#9, customer_sk#16] +Input [4]: [customer_sk#9, item_sk#10, customer_sk#16, item_sk#17] (25) HashAggregate [codegen id : 7] -Input [2]: [customer_sk#9, customer_sk#15] +Input [2]: [customer_sk#9, customer_sk#16] Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum#17, sum#18, sum#19] -Results [3]: [sum#20, sum#21, sum#22] +Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum#18, sum#19, sum#20] +Results [3]: [sum#21, sum#22, sum#23] (26) Exchange -Input [3]: [sum#20, sum#21, sum#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [sum#21, sum#22, sum#23] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] (27) HashAggregate [codegen id : 8] -Input [3]: [sum#20, sum#21, sum#22] +Input [3]: [sum#21, sum#22, sum#23] Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26] -Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24 AS store_only#27, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25 AS catalog_only#28, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26 AS store_and_catalog#29] +Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27] +Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25 AS store_only#28, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26 AS catalog_only#29, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27 AS store_and_catalog#30] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt index 245357ef5560c..c01f3465ed693 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt @@ -102,16 +102,16 @@ ReadSchema: struct Input [3]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13] (16) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#14] (17) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#5] +Right keys [1]: [d_date_sk#14] Join condition: None (18) Project [codegen id : 5] Output [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#5] +Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#14] (19) HashAggregate [codegen id : 5] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] @@ -122,45 +122,45 @@ Results [2]: [cs_bill_customer_sk#11, cs_item_sk#12] (20) Exchange Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#15] (21) HashAggregate [codegen id : 6] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Keys [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Functions: [] Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#11 AS customer_sk#15, cs_item_sk#12 AS item_sk#16] +Results [2]: [cs_bill_customer_sk#11 AS customer_sk#16, cs_item_sk#12 AS item_sk#17] (22) Sort [codegen id : 6] -Input [2]: [customer_sk#15, item_sk#16] -Arguments: [customer_sk#15 ASC NULLS FIRST, item_sk#16 ASC NULLS FIRST], false, 0 +Input [2]: [customer_sk#16, item_sk#17] +Arguments: [customer_sk#16 ASC NULLS FIRST, item_sk#17 ASC NULLS FIRST], false, 0 (23) SortMergeJoin Left keys [2]: [customer_sk#9, item_sk#10] -Right keys [2]: [customer_sk#15, item_sk#16] +Right keys [2]: [customer_sk#16, item_sk#17] Join condition: None (24) Project [codegen id : 7] -Output [2]: [customer_sk#9, customer_sk#15] -Input [4]: [customer_sk#9, item_sk#10, customer_sk#15, item_sk#16] +Output [2]: [customer_sk#9, customer_sk#16] +Input [4]: [customer_sk#9, item_sk#10, customer_sk#16, item_sk#17] (25) HashAggregate [codegen id : 7] -Input [2]: [customer_sk#9, customer_sk#15] +Input [2]: [customer_sk#9, customer_sk#16] Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum#17, sum#18, sum#19] -Results [3]: [sum#20, sum#21, sum#22] +Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum#18, sum#19, sum#20] +Results [3]: [sum#21, sum#22, sum#23] (26) Exchange -Input [3]: [sum#20, sum#21, sum#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [sum#21, sum#22, sum#23] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] (27) HashAggregate [codegen id : 8] -Input [3]: [sum#20, sum#21, sum#22] +Input [3]: [sum#21, sum#22, sum#23] Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26] -Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24 AS store_only#27, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25 AS catalog_only#28, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26 AS store_and_catalog#29] +Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27] +Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25 AS store_only#28, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26 AS catalog_only#29, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27 AS store_and_catalog#30] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt index 58d28b28dae10..2e710b7eda4b3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt @@ -137,52 +137,52 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#15] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (22) Project [codegen id : 7] -Output [1]: [ws_bill_customer_sk#13 AS customer_sk#15] -Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] +Output [1]: [ws_bill_customer_sk#13 AS customer_sk#16] +Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] (23) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] ReadSchema: struct (24) ColumnarToRow [codegen id : 9] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (25) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#19] (26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (27) Project [codegen id : 9] -Output [1]: [cs_ship_customer_sk#16 AS customer_sk#18] -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] +Output [1]: [cs_ship_customer_sk#17 AS customer_sk#20] +Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] (28) Union (29) Exchange -Input [1]: [customer_sk#15] -Arguments: hashpartitioning(customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#19] +Input [1]: [customer_sk#16] +Arguments: hashpartitioning(customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#21] (30) Sort [codegen id : 10] -Input [1]: [customer_sk#15] -Arguments: [customer_sk#15 ASC NULLS FIRST], false, 0 +Input [1]: [customer_sk#16] +Arguments: [customer_sk#16 ASC NULLS FIRST], false, 0 (31) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#15] +Right keys [1]: [customer_sk#16] Join condition: None (32) Project [codegen id : 12] @@ -190,84 +190,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (33) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] +Output [2]: [ca_address_sk#22, ca_county#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -Input [2]: [ca_address_sk#20, ca_county#21] +Input [2]: [ca_address_sk#22, ca_county#23] (35) Filter [codegen id : 11] -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) +Input [2]: [ca_address_sk#22, ca_county#23] +Condition : (ca_county#23 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#22)) (36) Project [codegen id : 11] -Output [1]: [ca_address_sk#20] -Input [2]: [ca_address_sk#20, ca_county#21] +Output [1]: [ca_address_sk#22] +Input [2]: [ca_address_sk#22, ca_county#23] (37) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [ca_address_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (38) BroadcastHashJoin [codegen id : 12] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#22] Join condition: None (39) Project [codegen id : 12] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#22] (40) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (41) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Output [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (42) ColumnarToRow -Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] (43) Filter -Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Condition : isnotnull(cd_demo_sk#24) +Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Condition : isnotnull(cd_demo_sk#26) (44) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#24] +Right keys [1]: [cd_demo_sk#26] Join condition: None (45) Project [codegen id : 13] -Output [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Output [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] (46) HashAggregate [codegen id : 13] -Input [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] +Aggregate Attributes [1]: [count#35] +Results [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] (47) Exchange -Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] -Arguments: hashpartitioning(cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] +Arguments: hashpartitioning(cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, 5), ENSURE_REQUIREMENTS, [id=#37] (48) HashAggregate [codegen id : 14] -Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] -Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] +Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#36] -Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count(1)#36 AS cnt1#37, cd_purchase_estimate#28, count(1)#36 AS cnt2#38, cd_credit_rating#29, count(1)#36 AS cnt3#39, cd_dep_count#30, count(1)#36 AS cnt4#40, cd_dep_employed_count#31, count(1)#36 AS cnt5#41, cd_dep_college_count#32, count(1)#36 AS cnt6#42] +Aggregate Attributes [1]: [count(1)#38] +Results [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, count(1)#38 AS cnt1#39, cd_purchase_estimate#30, count(1)#38 AS cnt2#40, cd_credit_rating#31, count(1)#38 AS cnt3#41, cd_dep_count#32, count(1)#38 AS cnt4#42, cd_dep_employed_count#33, count(1)#38 AS cnt5#43, cd_dep_college_count#34, count(1)#38 AS cnt6#44] (49) TakeOrderedAndProject -Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] -Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] +Input [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#39, cd_purchase_estimate#30, cnt2#40, cd_credit_rating#31, cnt3#41, cd_dep_count#32, cnt4#42, cd_dep_employed_count#33, cnt5#43, cd_dep_college_count#34, cnt6#44] +Arguments: 100, [cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_education_status#29 ASC NULLS FIRST, cd_purchase_estimate#30 ASC NULLS FIRST, cd_credit_rating#31 ASC NULLS FIRST, cd_dep_count#32 ASC NULLS FIRST, cd_dep_employed_count#33 ASC NULLS FIRST, cd_dep_college_count#34 ASC NULLS FIRST], [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#39, cd_purchase_estimate#30, cnt2#40, cd_credit_rating#31, cnt3#41, cd_dep_count#32, cnt4#42, cd_dep_employed_count#33, cnt5#43, cd_dep_college_count#34, cnt6#44] ===== Subqueries ===== @@ -280,6 +280,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index eac6acc3543ca..62054fe96dd5c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -121,48 +121,48 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#12 AS customer_sk#14] -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] +Output [1]: [ws_bill_customer_sk#12 AS customer_sk#15] +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] (20) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#6)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] (22) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#18] (23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#18] Join condition: None (24) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#15 AS customer_sk#17] -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] +Output [1]: [cs_ship_customer_sk#16 AS customer_sk#19] +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#18] (25) Union (26) BroadcastExchange -Input [1]: [customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Input [1]: [customer_sk#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (27) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#14] +Right keys [1]: [customer_sk#15] Join condition: None (28) Project [codegen id : 9] @@ -170,84 +170,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#19, ca_county#20] +Output [2]: [ca_address_sk#21, ca_county#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#19, ca_county#20] +Input [2]: [ca_address_sk#21, ca_county#22] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#19, ca_county#20] -Condition : (ca_county#20 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#19)) +Input [2]: [ca_address_sk#21, ca_county#22] +Condition : (ca_county#22 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#21)) (32) Project [codegen id : 7] -Output [1]: [ca_address_sk#19] -Input [2]: [ca_address_sk#19, ca_county#20] +Output [1]: [ca_address_sk#21] +Input [2]: [ca_address_sk#21, ca_county#22] (33) BroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [ca_address_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (34) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#19] +Right keys [1]: [ca_address_sk#21] Join condition: None (35) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21] (36) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] (38) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#22) +Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Condition : isnotnull(cd_demo_sk#24) (39) BroadcastExchange -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#22] +Right keys [1]: [cd_demo_sk#24] Join condition: None (41) Project [codegen id : 9] -Output [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] (42) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#32] -Results [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] +Aggregate Attributes [1]: [count#34] +Results [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#35] (43) Exchange -Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] -Arguments: hashpartitioning(cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#35] +Arguments: hashpartitioning(cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#36] (44) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] -Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#35] +Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(1)#35 AS cnt1#36, cd_purchase_estimate#26, count(1)#35 AS cnt2#37, cd_credit_rating#27, count(1)#35 AS cnt3#38, cd_dep_count#28, count(1)#35 AS cnt4#39, cd_dep_employed_count#29, count(1)#35 AS cnt5#40, cd_dep_college_count#30, count(1)#35 AS cnt6#41] +Aggregate Attributes [1]: [count(1)#37] +Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count(1)#37 AS cnt1#38, cd_purchase_estimate#28, count(1)#37 AS cnt2#39, cd_credit_rating#29, count(1)#37 AS cnt3#40, cd_dep_count#30, count(1)#37 AS cnt4#41, cd_dep_employed_count#31, count(1)#37 AS cnt5#42, cd_dep_college_count#32, count(1)#37 AS cnt6#43] (45) TakeOrderedAndProject -Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] -Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] +Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#38, cd_purchase_estimate#28, cnt2#39, cd_credit_rating#29, cnt3#40, cd_dep_count#30, cnt4#41, cd_dep_employed_count#31, cnt5#42, cd_dep_college_count#32, cnt6#43] +Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#38, cd_purchase_estimate#28, cnt2#39, cd_credit_rating#29, cnt3#40, cd_dep_count#30, cnt4#41, cd_dep_employed_count#31, cnt5#42, cd_dep_college_count#32, cnt6#43] ===== Subqueries ===== @@ -260,6 +260,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#6 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 6eb3be35a05bc..453a35d8d5175 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 @@ -199,285 +199,285 @@ Input [2]: [customer_id#23, year_total#24] Arguments: [customer_id#23 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] (27) Filter [codegen id : 10] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_customer_sk#1) +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] +Condition : isnotnull(ss_customer_sk#26) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#31, d_year#32] 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 (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#6, d_year#7] +Input [2]: [d_date_sk#31, d_year#32] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#31, d_year#32] +Condition : ((isnotnull(d_year#32) AND (d_year#32 = 2002)) AND isnotnull(d_date_sk#31)) (31) BroadcastExchange -Input [2]: [d_date_sk#6, d_year#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [2]: [d_date_sk#31, d_year#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ss_sold_date_sk#29] +Right keys [1]: [d_date_sk#31] Join condition: None (33) Project [codegen id : 10] -Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4, d_date_sk#6, d_year#7] +Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Input [6]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29, d_date_sk#31, d_year#32] (34) Exchange -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#34] (35) Sort [codegen id : 11] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] (37) Sort [codegen id : 13] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Arguments: [c_customer_sk#35 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#35] Join condition: None (39) Project [codegen id : 14] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Input [12]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Input [12]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32, c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] (40) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#29] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] +Input [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#43] +Results [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] (41) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] +Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, 5), ENSURE_REQUIREMENTS, [id=#45] (42) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32] -Results [5]: [c_customer_id#11 AS customer_id#33, c_first_name#12 AS customer_first_name#34, c_last_name#13 AS customer_last_name#35, c_email_address#17 AS customer_email_address#36, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32,18,2) AS year_total#37] +Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] +Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46] +Results [5]: [c_customer_id#36 AS customer_id#47, c_first_name#37 AS customer_first_name#48, c_last_name#38 AS customer_last_name#49, c_email_address#42 AS customer_email_address#50, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46,18,2) AS year_total#51] (43) Exchange -Input [5]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37] -Arguments: hashpartitioning(customer_id#33, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] +Arguments: hashpartitioning(customer_id#47, 5), ENSURE_REQUIREMENTS, [id=#52] (44) Sort [codegen id : 16] -Input [5]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37] -Arguments: [customer_id#33 ASC NULLS FIRST], false, 0 +Input [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] +Arguments: [customer_id#47 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#33] +Right keys [1]: [customer_id#47] Join condition: None (46) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] +Output [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#56), dynamicpruningexpression(ws_sold_date_sk#56 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] +Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56] (48) Filter [codegen id : 19] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] -Condition : isnotnull(ws_bill_customer_sk#39) +Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56] +Condition : isnotnull(ws_bill_customer_sk#53) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#57, d_year#58] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#56] +Right keys [1]: [d_date_sk#57] Join condition: None (51) Project [codegen id : 19] -Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Input [6]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42, d_date_sk#6, d_year#7] +Output [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] +Input [6]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56, d_date_sk#57, d_year#58] (52) Exchange -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Arguments: hashpartitioning(ws_bill_customer_sk#39, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] +Arguments: hashpartitioning(ws_bill_customer_sk#53, 5), ENSURE_REQUIREMENTS, [id=#59] (53) Sort [codegen id : 20] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Arguments: [ws_bill_customer_sk#39 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] +Arguments: [ws_bill_customer_sk#53 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +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] (55) Sort [codegen id : 22] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +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] +Arguments: [c_customer_sk#60 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#39] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ws_bill_customer_sk#53] +Right keys [1]: [c_customer_sk#60] Join condition: None (57) Project [codegen id : 23] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Input [12]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [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, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] +Input [12]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58, 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] (58) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#44] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#45] +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, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] +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#58] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#68] +Results [9]: [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#58, sum#69] (59) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#45] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [9]: [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#58, sum#69] +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#58, 5), ENSURE_REQUIREMENTS, [id=#70] (60) HashAggregate [codegen id : 24] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#45] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#47] -Results [2]: [c_customer_id#11 AS customer_id#48, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#47,18,2) AS year_total#49] +Input [9]: [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#58, sum#69] +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#58] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))#71] +Results [2]: [c_customer_id#61 AS customer_id#72, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))#71,18,2) AS year_total#73] (61) Filter [codegen id : 24] -Input [2]: [customer_id#48, year_total#49] -Condition : (isnotnull(year_total#49) AND (year_total#49 > 0.00)) +Input [2]: [customer_id#72, year_total#73] +Condition : (isnotnull(year_total#73) AND (year_total#73 > 0.00)) (62) Project [codegen id : 24] -Output [2]: [customer_id#48 AS customer_id#50, year_total#49 AS year_total#51] -Input [2]: [customer_id#48, year_total#49] +Output [2]: [customer_id#72 AS customer_id#74, year_total#73 AS year_total#75] +Input [2]: [customer_id#72, year_total#73] (63) Exchange -Input [2]: [customer_id#50, year_total#51] -Arguments: hashpartitioning(customer_id#50, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [2]: [customer_id#74, year_total#75] +Arguments: hashpartitioning(customer_id#74, 5), ENSURE_REQUIREMENTS, [id=#76] (64) Sort [codegen id : 25] -Input [2]: [customer_id#50, year_total#51] -Arguments: [customer_id#50 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#74, year_total#75] +Arguments: [customer_id#74 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#50] +Right keys [1]: [customer_id#74] Join condition: None (66) Project [codegen id : 26] -Output [8]: [customer_id#23, year_total#24, customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37, year_total#51] -Input [9]: [customer_id#23, year_total#24, customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37, customer_id#50, year_total#51] +Output [8]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75] +Input [9]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, customer_id#74, year_total#75] (67) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] +Output [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] +Input [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80] (69) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] -Condition : isnotnull(ws_bill_customer_sk#39) +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) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#81, d_year#82] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#80] +Right keys [1]: [d_date_sk#81] Join condition: None (72) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Input [6]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42, d_date_sk#6, d_year#7] +Output [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] +Input [6]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80, d_date_sk#81, d_year#82] (73) Exchange -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Arguments: hashpartitioning(ws_bill_customer_sk#39, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] +Arguments: hashpartitioning(ws_bill_customer_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] (74) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Arguments: [ws_bill_customer_sk#39 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] +Arguments: [ws_bill_customer_sk#77 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [8]: [c_customer_sk#84, c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91] (76) Sort [codegen id : 31] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#84, c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91] +Arguments: [c_customer_sk#84 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#39] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ws_bill_customer_sk#77] +Right keys [1]: [c_customer_sk#84] Join condition: None (78) Project [codegen id : 32] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Input [12]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [10]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] +Input [12]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82, c_customer_sk#84, c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91] (79) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#54] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#55] +Input [10]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] +Keys [8]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82] +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#92] +Results [9]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, sum#93] (80) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#55] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [9]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, sum#93] +Arguments: hashpartitioning(c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, 5), ENSURE_REQUIREMENTS, [id=#94] (81) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#55] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#57] -Results [2]: [c_customer_id#11 AS customer_id#58, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#57,18,2) AS year_total#59] +Input [9]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, sum#93] +Keys [8]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82] +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)))#95] +Results [2]: [c_customer_id#85 AS customer_id#96, 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)))#95,18,2) AS year_total#97] (82) Exchange -Input [2]: [customer_id#58, year_total#59] -Arguments: hashpartitioning(customer_id#58, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [2]: [customer_id#96, year_total#97] +Arguments: hashpartitioning(customer_id#96, 5), ENSURE_REQUIREMENTS, [id=#98] (83) Sort [codegen id : 34] -Input [2]: [customer_id#58, year_total#59] -Arguments: [customer_id#58 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#96, year_total#97] +Arguments: [customer_id#96 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#58] -Join condition: (CASE WHEN (year_total#51 > 0.00) THEN CheckOverflow((promote_precision(year_total#59) / promote_precision(year_total#51)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#37) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE 0E-20 END) +Right keys [1]: [customer_id#96] +Join condition: (CASE WHEN (year_total#75 > 0.00) THEN CheckOverflow((promote_precision(year_total#97) / promote_precision(year_total#75)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#51) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE 0E-20 END) (85) Project [codegen id : 35] -Output [4]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36] -Input [10]: [customer_id#23, year_total#24, customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37, year_total#51, customer_id#58, year_total#59] +Output [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] +Input [10]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75, customer_id#96, year_total#97] (86) TakeOrderedAndProject -Input [4]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36] -Arguments: 100, [customer_id#33 ASC NULLS FIRST, customer_first_name#34 ASC NULLS FIRST, customer_last_name#35 ASC NULLS FIRST, customer_email_address#36 ASC NULLS FIRST], [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36] +Input [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] +Arguments: 100, [customer_id#47 ASC NULLS FIRST, customer_first_name#48 ASC NULLS FIRST, customer_last_name#49 ASC NULLS FIRST, customer_email_address#50 ASC NULLS FIRST], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] ===== Subqueries ===== @@ -488,15 +488,15 @@ ReusedExchange (87) (87) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#6, d_year#7] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#26 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#30 ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#31, d_year#32] -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#56 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#26 +Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#30 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 8f7b2faaa1220..17d16033bb5be 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 @@ -171,282 +171,282 @@ Input [2]: [customer_id#22, year_total#23] Condition : (isnotnull(year_total#23) AND (year_total#23 > 0.00)) (20) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] (22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_customer_id#25)) (23) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Output [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ss_sold_date_sk#35), dynamicpruningexpression(ss_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] (25) Filter [codegen id : 4] -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_customer_sk#9) +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Condition : isnotnull(ss_customer_sk#32) (26) BroadcastExchange -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#9] +Left keys [1]: [c_customer_sk#24] +Right keys [1]: [ss_customer_sk#32] Join condition: None (28) Project [codegen id : 6] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Input [12]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#38, d_year#39] 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 (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#15, d_year#16] +Input [2]: [d_date_sk#38, d_year#39] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#15, d_year#16] -Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2002)) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#38, d_year#39] +Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) (32) BroadcastExchange -Input [2]: [d_date_sk#15, d_year#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +Input [2]: [d_date_sk#38, d_year#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#35] +Right keys [1]: [d_date_sk#38] Join condition: None (34) Project [codegen id : 6] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#15, d_year#16] +Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] +Input [12]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#38, d_year#39] (35) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#27] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] +Input [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] +Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#41] +Results [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] (36) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] +Arguments: hashpartitioning(c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, 5), ENSURE_REQUIREMENTS, [id=#43] (37) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30] -Results [5]: [c_customer_id#2 AS customer_id#31, c_first_name#3 AS customer_first_name#32, c_last_name#4 AS customer_last_name#33, c_email_address#8 AS customer_email_address#34, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30,18,2) AS year_total#35] +Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] +Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44] +Results [5]: [c_customer_id#25 AS customer_id#45, c_first_name#26 AS customer_first_name#46, c_last_name#27 AS customer_last_name#47, c_email_address#31 AS customer_email_address#48, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44,18,2) AS year_total#49] (38) BroadcastExchange -Input [5]: [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#36] +Input [5]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#31] +Right keys [1]: [customer_id#45] Join condition: None (40) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] (42) Filter [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] +Condition : (isnotnull(c_customer_sk#51) AND isnotnull(c_customer_id#52)) (43) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Input [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] (45) Filter [codegen id : 8] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_customer_sk#37) +Input [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] +Condition : isnotnull(ws_bill_customer_sk#59) (46) BroadcastExchange -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#41] +Input [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#37] +Left keys [1]: [c_customer_sk#51] +Right keys [1]: [ws_bill_customer_sk#59] Join condition: None (48) Project [codegen id : 10] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] +Input [12]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#64, d_year#65] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ws_sold_date_sk#62] +Right keys [1]: [d_date_sk#64] Join condition: None (51) Project [codegen id : 10] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#15, d_year#16] +Output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, d_year#65] +Input [12]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62, d_date_sk#64, d_year#65] (52) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#42] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#43] +Input [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, d_year#65] +Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#66] +Results [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, sum#67] (53) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#43] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, sum#67] +Arguments: hashpartitioning(c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#68] (54) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#43] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45] -Results [2]: [c_customer_id#2 AS customer_id#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45,18,2) AS year_total#47] +Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, sum#67] +Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))#69] +Results [2]: [c_customer_id#52 AS customer_id#70, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))#69,18,2) AS year_total#71] (55) Filter [codegen id : 11] -Input [2]: [customer_id#46, year_total#47] -Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) +Input [2]: [customer_id#70, year_total#71] +Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.00)) (56) Project [codegen id : 11] -Output [2]: [customer_id#46 AS customer_id#48, year_total#47 AS year_total#49] -Input [2]: [customer_id#46, year_total#47] +Output [2]: [customer_id#70 AS customer_id#72, year_total#71 AS year_total#73] +Input [2]: [customer_id#70, year_total#71] (57) BroadcastExchange -Input [2]: [customer_id#48, year_total#49] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] +Input [2]: [customer_id#72, year_total#73] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#74] (58) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#48] +Right keys [1]: [customer_id#72] Join condition: None (59) Project [codegen id : 16] -Output [8]: [customer_id#22, year_total#23, customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35, year_total#49] -Input [9]: [customer_id#22, year_total#23, customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35, customer_id#48, year_total#49] +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#73] +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#72, year_total#73] (60) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] (62) Filter [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] +Condition : (isnotnull(c_customer_sk#75) AND isnotnull(c_customer_id#76)) (63) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] (65) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_customer_sk#37) +Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Condition : isnotnull(ws_bill_customer_sk#83) (66) BroadcastExchange -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] +Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#87] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#37] +Left keys [1]: [c_customer_sk#75] +Right keys [1]: [ws_bill_customer_sk#83] Join condition: None (68) Project [codegen id : 14] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [10]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Input [12]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#88, d_year#89] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ws_sold_date_sk#86] +Right keys [1]: [d_date_sk#88] Join condition: None (71) Project [codegen id : 14] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#15, d_year#16] +Output [10]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] +Input [12]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86, d_date_sk#88, d_year#89] (72) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#52] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#53] +Input [10]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] +Keys [8]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#90] +Results [9]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, sum#91] (73) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#53] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [9]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, sum#91] +Arguments: hashpartitioning(c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, 5), ENSURE_REQUIREMENTS, [id=#92] (74) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#53] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55] -Results [2]: [c_customer_id#2 AS customer_id#56, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55,18,2) AS year_total#57] +Input [9]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, sum#91] +Keys [8]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))#93] +Results [2]: [c_customer_id#76 AS customer_id#94, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))#93,18,2) AS year_total#95] (75) BroadcastExchange -Input [2]: [customer_id#56, year_total#57] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#58] +Input [2]: [customer_id#94, year_total#95] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#96] (76) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#56] -Join condition: (CASE WHEN (year_total#49 > 0.00) THEN CheckOverflow((promote_precision(year_total#57) / promote_precision(year_total#49)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#35) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE 0E-20 END) +Right keys [1]: [customer_id#94] +Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#95) / promote_precision(year_total#73)), 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) (77) Project [codegen id : 16] -Output [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34] -Input [10]: [customer_id#22, year_total#23, customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35, year_total#49, customer_id#56, year_total#57] +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#73, customer_id#94, year_total#95] (78) TakeOrderedAndProject -Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34] -Arguments: 100, [customer_id#31 ASC NULLS FIRST, customer_first_name#32 ASC NULLS FIRST, customer_last_name#33 ASC NULLS FIRST, customer_email_address#34 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34] +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] ===== Subqueries ===== @@ -457,15 +457,15 @@ ReusedExchange (79) (79) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#15, d_year#16] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#24 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#38, d_year#39] -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#24 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#36 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 15f74a7802b27..56fa48006d894 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 @@ -393,484 +393,484 @@ Aggregate Attributes [7]: [avg(agg1#37)#73, avg(agg2#38)#74, avg(agg3#39)#75, av Results [11]: [i_item_id#20, ca_country#34, ca_state#33, ca_county#32, avg(agg1#37)#73 AS agg1#80, avg(agg2#38)#74 AS agg2#81, avg(agg3#39)#75 AS agg3#82, avg(agg4#40)#76 AS agg4#83, avg(agg5#41)#77 AS agg5#84, avg(agg6#42)#78 AS agg6#85, avg(agg7#43)#79 AS agg7#86] (52) ReusedExchange [Reuses operator id: unknown] -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#20] +Output [8]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94] (53) Sort [codegen id : 19] -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#20] -Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94] +Arguments: [cs_bill_customer_sk#87 ASC NULLS FIRST], false, 0 (54) Scan parquet default.customer -Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 21] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] (56) Filter [codegen id : 21] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] +Condition : (((c_birth_month#98 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#95)) AND isnotnull(c_current_cdemo_sk#96)) AND isnotnull(c_current_addr_sk#97)) (57) Project [codegen id : 21] -Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [4]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#99] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] (58) Scan parquet default.customer_address -Output [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Output [3]: [ca_address_sk#100, ca_state#101, ca_country#102] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] (60) Filter [codegen id : 20] -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] -Condition : (ca_state#33 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#31)) +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Condition : (ca_state#101 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#100)) (61) BroadcastExchange -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#87] +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] (62) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [c_current_addr_sk#25] -Right keys [1]: [ca_address_sk#31] +Left keys [1]: [c_current_addr_sk#97] +Right keys [1]: [ca_address_sk#100] Join condition: None (63) Project [codegen id : 21] -Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34] -Input [7]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31, ca_state#33, ca_country#34] +Output [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Input [7]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#99, ca_address_sk#100, ca_state#101, ca_country#102] (64) Exchange -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34] -Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: hashpartitioning(c_current_cdemo_sk#96, 5), ENSURE_REQUIREMENTS, [id=#104] (65) Sort [codegen id : 22] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34] -Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: [c_current_cdemo_sk#96 ASC NULLS FIRST], false, 0 (66) ReusedExchange [Reuses operator id: 35] -Output [1]: [cd_demo_sk#89] +Output [1]: [cd_demo_sk#105] (67) Sort [codegen id : 24] -Input [1]: [cd_demo_sk#89] -Arguments: [cd_demo_sk#89 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#105] +Arguments: [cd_demo_sk#105 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 25] -Left keys [1]: [c_current_cdemo_sk#24] -Right keys [1]: [cd_demo_sk#89] +Left keys [1]: [c_current_cdemo_sk#96] +Right keys [1]: [cd_demo_sk#105] Join condition: None (69) Project [codegen id : 25] -Output [4]: [c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] -Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34, cd_demo_sk#89] +Output [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Input [6]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102, cd_demo_sk#105] (70) Exchange -Input [4]: [c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: hashpartitioning(c_customer_sk#95, 5), ENSURE_REQUIREMENTS, [id=#106] (71) Sort [codegen id : 26] -Input [4]: [c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] -Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: [c_customer_sk#95 ASC NULLS FIRST], false, 0 (72) SortMergeJoin [codegen id : 27] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#23] +Left keys [1]: [cs_bill_customer_sk#87] +Right keys [1]: [c_customer_sk#95] Join condition: None (73) Project [codegen id : 27] -Output [10]: [i_item_id#20, ca_country#34, ca_state#33, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] -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#20, c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] +Output [10]: [i_item_id#94, ca_country#102, ca_state#101, cast(cs_quantity#88 as decimal(12,2)) AS agg1#37, cast(cs_list_price#89 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#91 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#90 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#92 as decimal(12,2)) AS agg5#41, cast(c_birth_year#99 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#93 as decimal(12,2)) AS agg7#43] +Input [12]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94, c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] (74) HashAggregate [codegen id : 27] -Input [10]: [i_item_id#20, ca_country#34, ca_state#33, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] -Keys [3]: [i_item_id#20, ca_country#34, ca_state#33] +Input [10]: [i_item_id#94, ca_country#102, ca_state#101, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [3]: [i_item_id#94, ca_country#102, ca_state#101] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100, sum#101, count#102, sum#103, count#104] -Results [17]: [i_item_id#20, ca_country#34, ca_state#33, 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] +Aggregate Attributes [14]: [sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] +Results [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] (75) Exchange -Input [17]: [i_item_id#20, ca_country#34, ca_state#33, 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] -Arguments: hashpartitioning(i_item_id#20, ca_country#34, ca_state#33, 5), ENSURE_REQUIREMENTS, [id=#119] +Input [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] +Arguments: hashpartitioning(i_item_id#94, ca_country#102, ca_state#101, 5), ENSURE_REQUIREMENTS, [id=#135] (76) HashAggregate [codegen id : 28] -Input [17]: [i_item_id#20, ca_country#34, ca_state#33, 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] -Keys [3]: [i_item_id#20, ca_country#34, ca_state#33] +Input [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] +Keys [3]: [i_item_id#94, ca_country#102, ca_state#101] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#120, avg(agg2#38)#121, avg(agg3#39)#122, avg(agg4#40)#123, avg(agg5#41)#124, avg(agg6#42)#125, avg(agg7#43)#126] -Results [11]: [i_item_id#20, ca_country#34, ca_state#33, null AS county#127, avg(agg1#37)#120 AS agg1#128, avg(agg2#38)#121 AS agg2#129, avg(agg3#39)#122 AS agg3#130, avg(agg4#40)#123 AS agg4#131, avg(agg5#41)#124 AS agg5#132, avg(agg6#42)#125 AS agg6#133, avg(agg7#43)#126 AS agg7#134] +Aggregate Attributes [7]: [avg(agg1#37)#136, avg(agg2#38)#137, avg(agg3#39)#138, avg(agg4#40)#139, avg(agg5#41)#140, avg(agg6#42)#141, avg(agg7#43)#142] +Results [11]: [i_item_id#94, ca_country#102, ca_state#101, null AS county#143, avg(agg1#37)#136 AS agg1#144, avg(agg2#38)#137 AS agg2#145, avg(agg3#39)#138 AS agg3#146, avg(agg4#40)#139 AS agg4#147, avg(agg5#41)#140 AS agg5#148, avg(agg6#42)#141 AS agg6#149, avg(agg7#43)#142 AS agg7#150] (77) ReusedExchange [Reuses operator id: unknown] -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#20] +Output [8]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158] (78) Sort [codegen id : 33] -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#20] -Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158] +Arguments: [cs_bill_customer_sk#151 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer -Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (80) ColumnarToRow [codegen id : 35] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] (81) Filter [codegen id : 35] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] +Condition : (((c_birth_month#162 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#159)) AND isnotnull(c_current_cdemo_sk#160)) AND isnotnull(c_current_addr_sk#161)) (82) Project [codegen id : 35] -Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_year#163] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] (83) Scan parquet default.customer_address -Output [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Output [3]: [ca_address_sk#164, ca_state#165, ca_country#166] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (84) ColumnarToRow [codegen id : 34] -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] (85) Filter [codegen id : 34] -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] -Condition : (ca_state#33 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#31)) +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] +Condition : (ca_state#165 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#164)) (86) Project [codegen id : 34] -Output [2]: [ca_address_sk#31, ca_country#34] -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Output [2]: [ca_address_sk#164, ca_country#166] +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] (87) BroadcastExchange -Input [2]: [ca_address_sk#31, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#135] +Input [2]: [ca_address_sk#164, ca_country#166] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#167] (88) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [c_current_addr_sk#25] -Right keys [1]: [ca_address_sk#31] +Left keys [1]: [c_current_addr_sk#161] +Right keys [1]: [ca_address_sk#164] Join condition: None (89) Project [codegen id : 35] -Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34] -Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31, ca_country#34] +Output [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Input [6]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_year#163, ca_address_sk#164, ca_country#166] (90) Exchange -Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34] -Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#136] +Input [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Arguments: hashpartitioning(c_current_cdemo_sk#160, 5), ENSURE_REQUIREMENTS, [id=#168] (91) Sort [codegen id : 36] -Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34] -Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Arguments: [c_current_cdemo_sk#160 ASC NULLS FIRST], false, 0 (92) ReusedExchange [Reuses operator id: 35] -Output [1]: [cd_demo_sk#137] +Output [1]: [cd_demo_sk#169] (93) Sort [codegen id : 38] -Input [1]: [cd_demo_sk#137] -Arguments: [cd_demo_sk#137 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#169] +Arguments: [cd_demo_sk#169 ASC NULLS FIRST], false, 0 (94) SortMergeJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#24] -Right keys [1]: [cd_demo_sk#137] +Left keys [1]: [c_current_cdemo_sk#160] +Right keys [1]: [cd_demo_sk#169] Join condition: None (95) Project [codegen id : 39] -Output [3]: [c_customer_sk#23, c_birth_year#27, ca_country#34] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34, cd_demo_sk#137] +Output [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166, cd_demo_sk#169] (96) Exchange -Input [3]: [c_customer_sk#23, c_birth_year#27, ca_country#34] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#138] +Input [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Arguments: hashpartitioning(c_customer_sk#159, 5), ENSURE_REQUIREMENTS, [id=#170] (97) Sort [codegen id : 40] -Input [3]: [c_customer_sk#23, c_birth_year#27, ca_country#34] -Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Arguments: [c_customer_sk#159 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 41] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#23] +Left keys [1]: [cs_bill_customer_sk#151] +Right keys [1]: [c_customer_sk#159] Join condition: None (99) Project [codegen id : 41] -Output [9]: [i_item_id#20, ca_country#34, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] -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#20, c_customer_sk#23, c_birth_year#27, ca_country#34] +Output [9]: [i_item_id#158, ca_country#166, cast(cs_quantity#152 as decimal(12,2)) AS agg1#37, cast(cs_list_price#153 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#155 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#154 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#156 as decimal(12,2)) AS agg5#41, cast(c_birth_year#163 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#157 as decimal(12,2)) AS agg7#43] +Input [11]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158, c_customer_sk#159, c_birth_year#163, ca_country#166] (100) HashAggregate [codegen id : 41] -Input [9]: [i_item_id#20, ca_country#34, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] -Keys [2]: [i_item_id#20, ca_country#34] +Input [9]: [i_item_id#158, ca_country#166, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [2]: [i_item_id#158, ca_country#166] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#139, count#140, sum#141, count#142, sum#143, count#144, sum#145, count#146, sum#147, count#148, sum#149, count#150, sum#151, count#152] -Results [16]: [i_item_id#20, ca_country#34, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] +Aggregate Attributes [14]: [sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178, sum#179, count#180, sum#181, count#182, sum#183, count#184] +Results [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] (101) Exchange -Input [16]: [i_item_id#20, ca_country#34, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] -Arguments: hashpartitioning(i_item_id#20, ca_country#34, 5), ENSURE_REQUIREMENTS, [id=#167] +Input [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] +Arguments: hashpartitioning(i_item_id#158, ca_country#166, 5), ENSURE_REQUIREMENTS, [id=#199] (102) HashAggregate [codegen id : 42] -Input [16]: [i_item_id#20, ca_country#34, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] -Keys [2]: [i_item_id#20, ca_country#34] +Input [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] +Keys [2]: [i_item_id#158, ca_country#166] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#168, avg(agg2#38)#169, avg(agg3#39)#170, avg(agg4#40)#171, avg(agg5#41)#172, avg(agg6#42)#173, avg(agg7#43)#174] -Results [11]: [i_item_id#20, ca_country#34, null AS ca_state#175, null AS county#176, avg(agg1#37)#168 AS agg1#177, avg(agg2#38)#169 AS agg2#178, avg(agg3#39)#170 AS agg3#179, avg(agg4#40)#171 AS agg4#180, avg(agg5#41)#172 AS agg5#181, avg(agg6#42)#173 AS agg6#182, avg(agg7#43)#174 AS agg7#183] +Aggregate Attributes [7]: [avg(agg1#37)#200, avg(agg2#38)#201, avg(agg3#39)#202, avg(agg4#40)#203, avg(agg5#41)#204, avg(agg6#42)#205, avg(agg7#43)#206] +Results [11]: [i_item_id#158, ca_country#166, null AS ca_state#207, null AS county#208, avg(agg1#37)#200 AS agg1#209, avg(agg2#38)#201 AS agg2#210, avg(agg3#39)#202 AS agg3#211, avg(agg4#40)#203 AS agg4#212, avg(agg5#41)#204 AS agg5#213, avg(agg6#42)#205 AS agg6#214, avg(agg7#43)#206 AS agg7#215] (103) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#224), dynamicpruningexpression(cs_sold_date_sk#224 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 49] -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] +Input [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] (105) Filter [codegen id : 49] -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)) +Input [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] +Condition : ((isnotnull(cs_bill_cdemo_sk#217) AND isnotnull(cs_bill_customer_sk#216)) AND isnotnull(cs_item_sk#218)) (106) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#225, cd_dep_count#226] (107) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#217] +Right keys [1]: [cd_demo_sk#225] Join condition: None (108) Project [codegen id : 49] -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] +Output [9]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_dep_count#226] +Input [11]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_demo_sk#225, cd_dep_count#226] (109) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#227] (110) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#16] +Left keys [1]: [cs_sold_date_sk#224] +Right keys [1]: [d_date_sk#227] Join condition: None (111) Project [codegen id : 49] -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] +Output [8]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226] +Input [10]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_dep_count#226, d_date_sk#227] (112) Scan parquet default.customer -Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (113) ColumnarToRow [codegen id : 46] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] (114) Filter [codegen id : 46] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] +Condition : (((c_birth_month#231 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#228)) AND isnotnull(c_current_cdemo_sk#229)) AND isnotnull(c_current_addr_sk#230)) (115) Project [codegen id : 46] -Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [4]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_year#232] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] (116) Scan parquet default.customer_address -Output [2]: [ca_address_sk#31, ca_state#33] +Output [2]: [ca_address_sk#233, ca_state#234] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (117) ColumnarToRow [codegen id : 45] -Input [2]: [ca_address_sk#31, ca_state#33] +Input [2]: [ca_address_sk#233, ca_state#234] (118) Filter [codegen id : 45] -Input [2]: [ca_address_sk#31, ca_state#33] -Condition : (ca_state#33 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#31)) +Input [2]: [ca_address_sk#233, ca_state#234] +Condition : (ca_state#234 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#233)) (119) Project [codegen id : 45] -Output [1]: [ca_address_sk#31] -Input [2]: [ca_address_sk#31, ca_state#33] +Output [1]: [ca_address_sk#233] +Input [2]: [ca_address_sk#233, ca_state#234] (120) BroadcastExchange -Input [1]: [ca_address_sk#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#184] +Input [1]: [ca_address_sk#233] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#235] (121) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [c_current_addr_sk#25] -Right keys [1]: [ca_address_sk#31] +Left keys [1]: [c_current_addr_sk#230] +Right keys [1]: [ca_address_sk#233] Join condition: None (122) Project [codegen id : 46] -Output [3]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31] +Output [3]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_year#232, ca_address_sk#233] (123) BroadcastExchange -Input [3]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#185] +Input [3]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#236] (124) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#186] +Output [1]: [cd_demo_sk#237] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (125) ColumnarToRow -Input [1]: [cd_demo_sk#186] +Input [1]: [cd_demo_sk#237] (126) Filter -Input [1]: [cd_demo_sk#186] -Condition : isnotnull(cd_demo_sk#186) +Input [1]: [cd_demo_sk#237] +Condition : isnotnull(cd_demo_sk#237) (127) BroadcastHashJoin [codegen id : 47] -Left keys [1]: [c_current_cdemo_sk#24] -Right keys [1]: [cd_demo_sk#186] +Left keys [1]: [c_current_cdemo_sk#229] +Right keys [1]: [cd_demo_sk#237] Join condition: None (128) Project [codegen id : 47] -Output [2]: [c_customer_sk#23, c_birth_year#27] -Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, cd_demo_sk#186] +Output [2]: [c_customer_sk#228, c_birth_year#232] +Input [4]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232, cd_demo_sk#237] (129) BroadcastExchange -Input [2]: [c_customer_sk#23, c_birth_year#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#187] +Input [2]: [c_customer_sk#228, c_birth_year#232] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#238] (130) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#23] +Left keys [1]: [cs_bill_customer_sk#216] +Right keys [1]: [c_customer_sk#228] Join condition: None (131) Project [codegen id : 49] -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#27] -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#23, c_birth_year#27] +Output [8]: [cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_birth_year#232] +Input [10]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_customer_sk#228, c_birth_year#232] (132) ReusedExchange [Reuses operator id: 21] -Output [2]: [i_item_sk#19, i_item_id#20] +Output [2]: [i_item_sk#239, i_item_id#240] (133) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#19] +Left keys [1]: [cs_item_sk#218] +Right keys [1]: [i_item_sk#239] Join condition: None (134) Project [codegen id : 49] -Output [8]: [i_item_id#20, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] -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#27, i_item_sk#19, i_item_id#20] +Output [8]: [i_item_id#240, cast(cs_quantity#219 as decimal(12,2)) AS agg1#37, cast(cs_list_price#220 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#222 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#221 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#223 as decimal(12,2)) AS agg5#41, cast(c_birth_year#232 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#226 as decimal(12,2)) AS agg7#43] +Input [10]: [cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_birth_year#232, i_item_sk#239, i_item_id#240] (135) HashAggregate [codegen id : 49] -Input [8]: [i_item_id#20, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] -Keys [1]: [i_item_id#20] +Input [8]: [i_item_id#240, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [1]: [i_item_id#240] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201] -Results [15]: [i_item_id#20, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215] +Aggregate Attributes [14]: [sum#241, count#242, sum#243, count#244, sum#245, count#246, sum#247, count#248, sum#249, count#250, sum#251, count#252, sum#253, count#254] +Results [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] (136) Exchange -Input [15]: [i_item_id#20, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215] -Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, [id=#216] +Input [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] +Arguments: hashpartitioning(i_item_id#240, 5), ENSURE_REQUIREMENTS, [id=#269] (137) HashAggregate [codegen id : 50] -Input [15]: [i_item_id#20, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215] -Keys [1]: [i_item_id#20] +Input [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] +Keys [1]: [i_item_id#240] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#217, avg(agg2#38)#218, avg(agg3#39)#219, avg(agg4#40)#220, avg(agg5#41)#221, avg(agg6#42)#222, avg(agg7#43)#223] -Results [11]: [i_item_id#20, null AS ca_country#224, null AS ca_state#225, null AS county#226, avg(agg1#37)#217 AS agg1#227, avg(agg2#38)#218 AS agg2#228, avg(agg3#39)#219 AS agg3#229, avg(agg4#40)#220 AS agg4#230, avg(agg5#41)#221 AS agg5#231, avg(agg6#42)#222 AS agg6#232, avg(agg7#43)#223 AS agg7#233] +Aggregate Attributes [7]: [avg(agg1#37)#270, avg(agg2#38)#271, avg(agg3#39)#272, avg(agg4#40)#273, avg(agg5#41)#274, avg(agg6#42)#275, avg(agg7#43)#276] +Results [11]: [i_item_id#240, null AS ca_country#277, null AS ca_state#278, null AS county#279, avg(agg1#37)#270 AS agg1#280, avg(agg2#38)#271 AS agg2#281, avg(agg3#39)#272 AS agg3#282, avg(agg4#40)#273 AS agg4#283, avg(agg5#41)#274 AS agg5#284, avg(agg6#42)#275 AS agg6#285, avg(agg7#43)#276 AS agg7#286] (138) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#295), dynamicpruningexpression(cs_sold_date_sk#295 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (139) ColumnarToRow [codegen id : 57] -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] +Input [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] (140) Filter [codegen id : 57] -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)) +Input [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] +Condition : ((isnotnull(cs_bill_cdemo_sk#288) AND isnotnull(cs_bill_customer_sk#287)) AND isnotnull(cs_item_sk#289)) (141) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#296, cd_dep_count#297] (142) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#288] +Right keys [1]: [cd_demo_sk#296] Join condition: None (143) Project [codegen id : 57] -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] +Output [9]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_dep_count#297] +Input [11]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_demo_sk#296, cd_dep_count#297] (144) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#298] (145) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#16] +Left keys [1]: [cs_sold_date_sk#295] +Right keys [1]: [d_date_sk#298] Join condition: None (146) Project [codegen id : 57] -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] +Output [8]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297] +Input [10]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_dep_count#297, d_date_sk#298] (147) Scan parquet default.item -Output [1]: [i_item_sk#19] +Output [1]: [i_item_sk#299] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (148) ColumnarToRow [codegen id : 53] -Input [1]: [i_item_sk#19] +Input [1]: [i_item_sk#299] (149) Filter [codegen id : 53] -Input [1]: [i_item_sk#19] -Condition : isnotnull(i_item_sk#19) +Input [1]: [i_item_sk#299] +Condition : isnotnull(i_item_sk#299) (150) BroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#234] +Input [1]: [i_item_sk#299] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#300] (151) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#19] +Left keys [1]: [cs_item_sk#289] +Right keys [1]: [i_item_sk#299] Join condition: None (152) Project [codegen id : 57] -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#19] +Output [7]: [cs_bill_customer_sk#287, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297] +Input [9]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297, i_item_sk#299] (153) ReusedExchange [Reuses operator id: 129] -Output [2]: [c_customer_sk#23, c_birth_year#27] +Output [2]: [c_customer_sk#301, c_birth_year#302] (154) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#23] +Left keys [1]: [cs_bill_customer_sk#287] +Right keys [1]: [c_customer_sk#301] Join condition: None (155) Project [codegen id : 57] -Output [7]: [cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] -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#23, c_birth_year#27] +Output [7]: [cast(cs_quantity#290 as decimal(12,2)) AS agg1#37, cast(cs_list_price#291 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#293 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#292 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#294 as decimal(12,2)) AS agg5#41, cast(c_birth_year#302 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#297 as decimal(12,2)) AS agg7#43] +Input [9]: [cs_bill_customer_sk#287, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297, c_customer_sk#301, c_birth_year#302] (156) HashAggregate [codegen id : 57] Input [7]: [agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] Keys: [] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#235, count#236, sum#237, count#238, sum#239, count#240, sum#241, count#242, sum#243, count#244, sum#245, count#246, sum#247, count#248] -Results [14]: [sum#249, count#250, sum#251, count#252, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262] +Aggregate Attributes [14]: [sum#303, count#304, sum#305, count#306, sum#307, count#308, sum#309, count#310, sum#311, count#312, sum#313, count#314, sum#315, count#316] +Results [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] (157) Exchange -Input [14]: [sum#249, count#250, sum#251, count#252, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#263] +Input [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#331] (158) HashAggregate [codegen id : 58] -Input [14]: [sum#249, count#250, sum#251, count#252, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262] +Input [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] Keys: [] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#264, avg(agg2#38)#265, avg(agg3#39)#266, avg(agg4#40)#267, avg(agg5#41)#268, avg(agg6#42)#269, avg(agg7#43)#270] -Results [11]: [null AS i_item_id#271, null AS ca_country#272, null AS ca_state#273, null AS county#274, avg(agg1#37)#264 AS agg1#275, avg(agg2#38)#265 AS agg2#276, avg(agg3#39)#266 AS agg3#277, avg(agg4#40)#267 AS agg4#278, avg(agg5#41)#268 AS agg5#279, avg(agg6#42)#269 AS agg6#280, avg(agg7#43)#270 AS agg7#281] +Aggregate Attributes [7]: [avg(agg1#37)#332, avg(agg2#38)#333, avg(agg3#39)#334, avg(agg4#40)#335, avg(agg5#41)#336, avg(agg6#42)#337, avg(agg7#43)#338] +Results [11]: [null AS i_item_id#339, null AS ca_country#340, null AS ca_state#341, null AS county#342, avg(agg1#37)#332 AS agg1#343, avg(agg2#38)#333 AS agg2#344, avg(agg3#39)#334 AS agg3#345, avg(agg4#40)#335 AS agg4#346, avg(agg5#41)#336 AS agg5#347, avg(agg6#42)#337 AS agg6#348, avg(agg7#43)#338 AS agg7#349] (159) Union @@ -887,8 +887,8 @@ ReusedExchange (161) (161) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#16] -Subquery:2 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#224 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 138 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 138 Hosting Expression = cs_sold_date_sk#295 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 0cb9a331b406a..8e3c5958d2831 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 @@ -366,492 +366,492 @@ Aggregate Attributes [7]: [avg(agg1#35)#71, avg(agg2#36)#72, avg(agg3#37)#73, av Results [11]: [i_item_id#33, ca_country#27, ca_state#26, ca_county#25, 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] (46) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#93), dynamicpruningexpression(cs_sold_date_sk#93 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 15] -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] +Input [9]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93] (48) Filter [codegen id : 15] -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)) +Input [9]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93] +Condition : ((isnotnull(cs_bill_cdemo_sk#86) AND isnotnull(cs_bill_customer_sk#85)) AND isnotnull(cs_item_sk#87)) (49) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#94, cd_dep_count#95] (50) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#86] +Right keys [1]: [cd_demo_sk#94] Join condition: None (51) Project [codegen id : 15] -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] +Output [9]: [cs_bill_customer_sk#85, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95] +Input [11]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_demo_sk#94, cd_dep_count#95] (52) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] +Output [4]: [c_customer_sk#96, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99] (53) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#16] +Left keys [1]: [cs_bill_customer_sk#85] +Right keys [1]: [c_customer_sk#96] Join condition: None (54) Project [codegen id : 15] -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] +Output [11]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99] +Input [13]: [cs_bill_customer_sk#85, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_customer_sk#96, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99] (55) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#85] +Output [1]: [cd_demo_sk#100] (56) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_cdemo_sk#17] -Right keys [1]: [cd_demo_sk#85] +Left keys [1]: [c_current_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#100] Join condition: None (57) Project [codegen id : 15] -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#85] +Output [10]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_addr_sk#98, c_birth_year#99] +Input [12]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99, cd_demo_sk#100] (58) Scan parquet default.customer_address -Output [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Output [3]: [ca_address_sk#101, ca_state#102, ca_country#103] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 12] -Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Input [3]: [ca_address_sk#101, ca_state#102, ca_country#103] (60) Filter [codegen id : 12] -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)) +Input [3]: [ca_address_sk#101, ca_state#102, ca_country#103] +Condition : (ca_state#102 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#101)) (61) BroadcastExchange -Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#86] +Input [3]: [ca_address_sk#101, ca_state#102, ca_country#103] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#104] (62) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_addr_sk#18] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [c_current_addr_sk#98] +Right keys [1]: [ca_address_sk#101] Join condition: None (63) Project [codegen id : 15] -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] +Output [11]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103] +Input [13]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_addr_sk#98, c_birth_year#99, ca_address_sk#101, ca_state#102, ca_country#103] (64) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#105] (65) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [cs_sold_date_sk#93] +Right keys [1]: [d_date_sk#105] Join condition: None (66) Project [codegen id : 15] -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] +Output [10]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103] +Input [12]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103, d_date_sk#105] (67) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#32, i_item_id#33] +Output [2]: [i_item_sk#106, i_item_id#107] (68) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [cs_item_sk#87] +Right keys [1]: [i_item_sk#106] Join condition: None (69) Project [codegen id : 15] -Output [10]: [i_item_id#33, ca_country#27, ca_state#26, 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#20 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] -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#32, i_item_id#33] +Output [10]: [i_item_id#107, ca_country#103, ca_state#102, cast(cs_quantity#88 as decimal(12,2)) AS agg1#35, cast(cs_list_price#89 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#91 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#90 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#92 as decimal(12,2)) AS agg5#39, cast(c_birth_year#99 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#95 as decimal(12,2)) AS agg7#41] +Input [12]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103, i_item_sk#106, i_item_id#107] (70) HashAggregate [codegen id : 15] -Input [10]: [i_item_id#33, ca_country#27, ca_state#26, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [3]: [i_item_id#33, ca_country#27, ca_state#26] +Input [10]: [i_item_id#107, ca_country#103, ca_state#102, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [3]: [i_item_id#107, ca_country#103, ca_state#102] 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#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100] -Results [17]: [i_item_id#33, ca_country#27, ca_state#26, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] +Aggregate Attributes [14]: [sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117, sum#118, count#119, sum#120, count#121] +Results [17]: [i_item_id#107, ca_country#103, ca_state#102, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] (71) Exchange -Input [17]: [i_item_id#33, ca_country#27, ca_state#26, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] -Arguments: hashpartitioning(i_item_id#33, ca_country#27, ca_state#26, 5), ENSURE_REQUIREMENTS, [id=#115] +Input [17]: [i_item_id#107, ca_country#103, ca_state#102, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] +Arguments: hashpartitioning(i_item_id#107, ca_country#103, ca_state#102, 5), ENSURE_REQUIREMENTS, [id=#136] (72) HashAggregate [codegen id : 16] -Input [17]: [i_item_id#33, ca_country#27, ca_state#26, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] -Keys [3]: [i_item_id#33, ca_country#27, ca_state#26] +Input [17]: [i_item_id#107, ca_country#103, ca_state#102, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] +Keys [3]: [i_item_id#107, ca_country#103, ca_state#102] 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)#116, avg(agg2#36)#117, avg(agg3#37)#118, avg(agg4#38)#119, avg(agg5#39)#120, avg(agg6#40)#121, avg(agg7#41)#122] -Results [11]: [i_item_id#33, ca_country#27, ca_state#26, null AS county#123, avg(agg1#35)#116 AS agg1#124, avg(agg2#36)#117 AS agg2#125, avg(agg3#37)#118 AS agg3#126, avg(agg4#38)#119 AS agg4#127, avg(agg5#39)#120 AS agg5#128, avg(agg6#40)#121 AS agg6#129, avg(agg7#41)#122 AS agg7#130] +Aggregate Attributes [7]: [avg(agg1#35)#137, avg(agg2#36)#138, avg(agg3#37)#139, avg(agg4#38)#140, avg(agg5#39)#141, avg(agg6#40)#142, avg(agg7#41)#143] +Results [11]: [i_item_id#107, ca_country#103, ca_state#102, null AS county#144, avg(agg1#35)#137 AS agg1#145, avg(agg2#36)#138 AS agg2#146, avg(agg3#37)#139 AS agg3#147, avg(agg4#38)#140 AS agg4#148, avg(agg5#39)#141 AS agg5#149, avg(agg6#40)#142 AS agg6#150, avg(agg7#41)#143 AS agg7#151] (73) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#160), dynamicpruningexpression(cs_sold_date_sk#160 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 23] -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] +Input [9]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160] (75) Filter [codegen id : 23] -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)) +Input [9]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160] +Condition : ((isnotnull(cs_bill_cdemo_sk#153) AND isnotnull(cs_bill_customer_sk#152)) AND isnotnull(cs_item_sk#154)) (76) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#161, cd_dep_count#162] (77) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#153] +Right keys [1]: [cd_demo_sk#161] Join condition: None (78) Project [codegen id : 23] -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] +Output [9]: [cs_bill_customer_sk#152, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162] +Input [11]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_demo_sk#161, cd_dep_count#162] (79) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] +Output [4]: [c_customer_sk#163, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] (80) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#16] +Left keys [1]: [cs_bill_customer_sk#152] +Right keys [1]: [c_customer_sk#163] Join condition: None (81) Project [codegen id : 23] -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] +Output [11]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] +Input [13]: [cs_bill_customer_sk#152, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_customer_sk#163, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] (82) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#131] +Output [1]: [cd_demo_sk#167] (83) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_cdemo_sk#17] -Right keys [1]: [cd_demo_sk#131] +Left keys [1]: [c_current_cdemo_sk#164] +Right keys [1]: [cd_demo_sk#167] Join condition: None (84) Project [codegen id : 23] -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#131] +Output [10]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_addr_sk#165, c_birth_year#166] +Input [12]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166, cd_demo_sk#167] (85) Scan parquet default.customer_address -Output [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Output [3]: [ca_address_sk#168, ca_state#169, ca_country#170] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (86) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] (87) Filter [codegen id : 20] -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)) +Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] +Condition : (ca_state#169 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#168)) (88) Project [codegen id : 20] -Output [2]: [ca_address_sk#24, ca_country#27] -Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Output [2]: [ca_address_sk#168, ca_country#170] +Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] (89) BroadcastExchange -Input [2]: [ca_address_sk#24, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#132] +Input [2]: [ca_address_sk#168, ca_country#170] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#171] (90) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_addr_sk#18] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [c_current_addr_sk#165] +Right keys [1]: [ca_address_sk#168] Join condition: None (91) Project [codegen id : 23] -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] +Output [10]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_birth_year#166, ca_country#170] +Input [12]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_addr_sk#165, c_birth_year#166, ca_address_sk#168, ca_country#170] (92) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#172] (93) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [cs_sold_date_sk#160] +Right keys [1]: [d_date_sk#172] Join condition: None (94) Project [codegen id : 23] -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] +Output [9]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cd_dep_count#162, c_birth_year#166, ca_country#170] +Input [11]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_birth_year#166, ca_country#170, d_date_sk#172] (95) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#32, i_item_id#33] +Output [2]: [i_item_sk#173, i_item_id#174] (96) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [cs_item_sk#154] +Right keys [1]: [i_item_sk#173] Join condition: None (97) Project [codegen id : 23] -Output [9]: [i_item_id#33, ca_country#27, 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#20 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] -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#32, i_item_id#33] +Output [9]: [i_item_id#174, ca_country#170, cast(cs_quantity#155 as decimal(12,2)) AS agg1#35, cast(cs_list_price#156 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#158 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#157 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#159 as decimal(12,2)) AS agg5#39, cast(c_birth_year#166 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#162 as decimal(12,2)) AS agg7#41] +Input [11]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cd_dep_count#162, c_birth_year#166, ca_country#170, i_item_sk#173, i_item_id#174] (98) HashAggregate [codegen id : 23] -Input [9]: [i_item_id#33, ca_country#27, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [2]: [i_item_id#33, ca_country#27] +Input [9]: [i_item_id#174, ca_country#170, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [2]: [i_item_id#174, ca_country#170] 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#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140, sum#141, count#142, sum#143, count#144, sum#145, count#146] -Results [16]: [i_item_id#33, ca_country#27, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160] +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 [16]: [i_item_id#174, ca_country#170, 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] (99) Exchange -Input [16]: [i_item_id#33, ca_country#27, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160] -Arguments: hashpartitioning(i_item_id#33, ca_country#27, 5), ENSURE_REQUIREMENTS, [id=#161] +Input [16]: [i_item_id#174, ca_country#170, 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#174, ca_country#170, 5), ENSURE_REQUIREMENTS, [id=#203] (100) HashAggregate [codegen id : 24] -Input [16]: [i_item_id#33, ca_country#27, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160] -Keys [2]: [i_item_id#33, ca_country#27] +Input [16]: [i_item_id#174, ca_country#170, 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 [2]: [i_item_id#174, ca_country#170] 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)#162, avg(agg2#36)#163, avg(agg3#37)#164, avg(agg4#38)#165, avg(agg5#39)#166, avg(agg6#40)#167, avg(agg7#41)#168] -Results [11]: [i_item_id#33, ca_country#27, null AS ca_state#169, null AS county#170, avg(agg1#35)#162 AS agg1#171, avg(agg2#36)#163 AS agg2#172, avg(agg3#37)#164 AS agg3#173, avg(agg4#38)#165 AS agg4#174, avg(agg5#39)#166 AS agg5#175, avg(agg6#40)#167 AS agg6#176, avg(agg7#41)#168 AS agg7#177] +Aggregate Attributes [7]: [avg(agg1#35)#204, avg(agg2#36)#205, avg(agg3#37)#206, avg(agg4#38)#207, avg(agg5#39)#208, avg(agg6#40)#209, avg(agg7#41)#210] +Results [11]: [i_item_id#174, ca_country#170, null AS ca_state#211, null AS county#212, avg(agg1#35)#204 AS agg1#213, avg(agg2#36)#205 AS agg2#214, avg(agg3#37)#206 AS agg3#215, avg(agg4#38)#207 AS agg4#216, avg(agg5#39)#208 AS agg5#217, avg(agg6#40)#209 AS agg6#218, avg(agg7#41)#210 AS agg7#219] (101) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#228), dynamicpruningexpression(cs_sold_date_sk#228 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 31] -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] +Input [9]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228] (103) Filter [codegen id : 31] -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)) +Input [9]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228] +Condition : ((isnotnull(cs_bill_cdemo_sk#221) AND isnotnull(cs_bill_customer_sk#220)) AND isnotnull(cs_item_sk#222)) (104) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#229, cd_dep_count#230] (105) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#221] +Right keys [1]: [cd_demo_sk#229] Join condition: None (106) Project [codegen id : 31] -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] +Output [9]: [cs_bill_customer_sk#220, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230] +Input [11]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_demo_sk#229, cd_dep_count#230] (107) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] +Output [4]: [c_customer_sk#231, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234] (108) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#16] +Left keys [1]: [cs_bill_customer_sk#220] +Right keys [1]: [c_customer_sk#231] Join condition: None (109) Project [codegen id : 31] -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] +Output [11]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234] +Input [13]: [cs_bill_customer_sk#220, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_customer_sk#231, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234] (110) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#178] +Output [1]: [cd_demo_sk#235] (111) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_cdemo_sk#17] -Right keys [1]: [cd_demo_sk#178] +Left keys [1]: [c_current_cdemo_sk#232] +Right keys [1]: [cd_demo_sk#235] Join condition: None (112) Project [codegen id : 31] -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#178] +Output [10]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_addr_sk#233, c_birth_year#234] +Input [12]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234, cd_demo_sk#235] (113) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#26] +Output [2]: [ca_address_sk#236, ca_state#237] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (114) ColumnarToRow [codegen id : 28] -Input [2]: [ca_address_sk#24, ca_state#26] +Input [2]: [ca_address_sk#236, ca_state#237] (115) Filter [codegen id : 28] -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)) +Input [2]: [ca_address_sk#236, ca_state#237] +Condition : (ca_state#237 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#236)) (116) Project [codegen id : 28] -Output [1]: [ca_address_sk#24] -Input [2]: [ca_address_sk#24, ca_state#26] +Output [1]: [ca_address_sk#236] +Input [2]: [ca_address_sk#236, ca_state#237] (117) BroadcastExchange -Input [1]: [ca_address_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#179] +Input [1]: [ca_address_sk#236] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#238] (118) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_addr_sk#18] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [c_current_addr_sk#233] +Right keys [1]: [ca_address_sk#236] Join condition: None (119) Project [codegen id : 31] -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] +Output [9]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_birth_year#234] +Input [11]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_addr_sk#233, c_birth_year#234, ca_address_sk#236] (120) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#239] (121) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [cs_sold_date_sk#228] +Right keys [1]: [d_date_sk#239] Join condition: None (122) Project [codegen id : 31] -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] +Output [8]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cd_dep_count#230, c_birth_year#234] +Input [10]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_birth_year#234, d_date_sk#239] (123) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#32, i_item_id#33] +Output [2]: [i_item_sk#240, i_item_id#241] (124) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [cs_item_sk#222] +Right keys [1]: [i_item_sk#240] Join condition: None (125) Project [codegen id : 31] -Output [8]: [i_item_id#33, 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#20 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#20, i_item_sk#32, i_item_id#33] +Output [8]: [i_item_id#241, cast(cs_quantity#223 as decimal(12,2)) AS agg1#35, cast(cs_list_price#224 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#226 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#225 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#227 as decimal(12,2)) AS agg5#39, cast(c_birth_year#234 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#230 as decimal(12,2)) AS agg7#41] +Input [10]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cd_dep_count#230, c_birth_year#234, i_item_sk#240, i_item_id#241] (126) HashAggregate [codegen id : 31] -Input [8]: [i_item_id#33, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [1]: [i_item_id#33] +Input [8]: [i_item_id#241, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [1]: [i_item_id#241] 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#180, count#181, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193] -Results [15]: [i_item_id#33, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] +Aggregate Attributes [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] +Results [15]: [i_item_id#241, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] (127) Exchange -Input [15]: [i_item_id#33, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] -Arguments: hashpartitioning(i_item_id#33, 5), ENSURE_REQUIREMENTS, [id=#208] +Input [15]: [i_item_id#241, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] +Arguments: hashpartitioning(i_item_id#241, 5), ENSURE_REQUIREMENTS, [id=#270] (128) HashAggregate [codegen id : 32] -Input [15]: [i_item_id#33, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] -Keys [1]: [i_item_id#33] +Input [15]: [i_item_id#241, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] +Keys [1]: [i_item_id#241] 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)#209, avg(agg2#36)#210, avg(agg3#37)#211, avg(agg4#38)#212, avg(agg5#39)#213, avg(agg6#40)#214, avg(agg7#41)#215] -Results [11]: [i_item_id#33, null AS ca_country#216, null AS ca_state#217, null AS county#218, avg(agg1#35)#209 AS agg1#219, avg(agg2#36)#210 AS agg2#220, avg(agg3#37)#211 AS agg3#221, avg(agg4#38)#212 AS agg4#222, avg(agg5#39)#213 AS agg5#223, avg(agg6#40)#214 AS agg6#224, avg(agg7#41)#215 AS agg7#225] +Aggregate Attributes [7]: [avg(agg1#35)#271, avg(agg2#36)#272, avg(agg3#37)#273, avg(agg4#38)#274, avg(agg5#39)#275, avg(agg6#40)#276, avg(agg7#41)#277] +Results [11]: [i_item_id#241, null AS ca_country#278, null AS ca_state#279, null AS county#280, avg(agg1#35)#271 AS agg1#281, avg(agg2#36)#272 AS agg2#282, avg(agg3#37)#273 AS agg3#283, avg(agg4#38)#274 AS agg4#284, avg(agg5#39)#275 AS agg5#285, avg(agg6#40)#276 AS agg6#286, avg(agg7#41)#277 AS agg7#287] (129) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#296), dynamicpruningexpression(cs_sold_date_sk#296 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (130) ColumnarToRow [codegen id : 39] -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] +Input [9]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296] (131) Filter [codegen id : 39] -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)) +Input [9]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296] +Condition : ((isnotnull(cs_bill_cdemo_sk#289) AND isnotnull(cs_bill_customer_sk#288)) AND isnotnull(cs_item_sk#290)) (132) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#297, cd_dep_count#298] (133) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#289] +Right keys [1]: [cd_demo_sk#297] Join condition: None (134) Project [codegen id : 39] -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] +Output [9]: [cs_bill_customer_sk#288, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298] +Input [11]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_demo_sk#297, cd_dep_count#298] (135) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] +Output [4]: [c_customer_sk#299, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302] (136) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#16] +Left keys [1]: [cs_bill_customer_sk#288] +Right keys [1]: [c_customer_sk#299] Join condition: None (137) Project [codegen id : 39] -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] +Output [11]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302] +Input [13]: [cs_bill_customer_sk#288, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_customer_sk#299, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302] (138) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#226] +Output [1]: [cd_demo_sk#303] (139) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#17] -Right keys [1]: [cd_demo_sk#226] +Left keys [1]: [c_current_cdemo_sk#300] +Right keys [1]: [cd_demo_sk#303] Join condition: None (140) Project [codegen id : 39] -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#226] +Output [10]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_addr_sk#301, c_birth_year#302] +Input [12]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302, cd_demo_sk#303] (141) ReusedExchange [Reuses operator id: 117] -Output [1]: [ca_address_sk#24] +Output [1]: [ca_address_sk#304] (142) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_addr_sk#18] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [c_current_addr_sk#301] +Right keys [1]: [ca_address_sk#304] Join condition: None (143) Project [codegen id : 39] -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] +Output [9]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_birth_year#302] +Input [11]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_addr_sk#301, c_birth_year#302, ca_address_sk#304] (144) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#305] (145) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [cs_sold_date_sk#296] +Right keys [1]: [d_date_sk#305] Join condition: None (146) Project [codegen id : 39] -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] +Output [8]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cd_dep_count#298, c_birth_year#302] +Input [10]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_birth_year#302, d_date_sk#305] (147) Scan parquet default.item -Output [1]: [i_item_sk#32] +Output [1]: [i_item_sk#306] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (148) ColumnarToRow [codegen id : 38] -Input [1]: [i_item_sk#32] +Input [1]: [i_item_sk#306] (149) Filter [codegen id : 38] -Input [1]: [i_item_sk#32] -Condition : isnotnull(i_item_sk#32) +Input [1]: [i_item_sk#306] +Condition : isnotnull(i_item_sk#306) (150) BroadcastExchange -Input [1]: [i_item_sk#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#227] +Input [1]: [i_item_sk#306] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#307] (151) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [cs_item_sk#290] +Right keys [1]: [i_item_sk#306] Join condition: None (152) Project [codegen id : 39] -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#20 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] -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#32] +Output [7]: [cast(cs_quantity#291 as decimal(12,2)) AS agg1#35, cast(cs_list_price#292 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#294 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#293 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#295 as decimal(12,2)) AS agg5#39, cast(c_birth_year#302 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#298 as decimal(12,2)) AS agg7#41] +Input [9]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cd_dep_count#298, c_birth_year#302, i_item_sk#306] (153) HashAggregate [codegen id : 39] 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#228, count#229, sum#230, count#231, sum#232, count#233, sum#234, count#235, sum#236, count#237, sum#238, count#239, sum#240, count#241] -Results [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] +Aggregate Attributes [14]: [sum#308, count#309, sum#310, count#311, sum#312, count#313, sum#314, count#315, sum#316, count#317, sum#318, count#319, sum#320, count#321] +Results [14]: [sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335] (154) Exchange -Input [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#256] +Input [14]: [sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#336] (155) HashAggregate [codegen id : 40] -Input [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] +Input [14]: [sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335] 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)#257, avg(agg2#36)#258, avg(agg3#37)#259, avg(agg4#38)#260, avg(agg5#39)#261, avg(agg6#40)#262, avg(agg7#41)#263] -Results [11]: [null AS i_item_id#264, null AS ca_country#265, null AS ca_state#266, null AS county#267, avg(agg1#35)#257 AS agg1#268, avg(agg2#36)#258 AS agg2#269, avg(agg3#37)#259 AS agg3#270, avg(agg4#38)#260 AS agg4#271, avg(agg5#39)#261 AS agg5#272, avg(agg6#40)#262 AS agg6#273, avg(agg7#41)#263 AS agg7#274] +Aggregate Attributes [7]: [avg(agg1#35)#337, avg(agg2#36)#338, avg(agg3#37)#339, avg(agg4#38)#340, avg(agg5#39)#341, avg(agg6#40)#342, avg(agg7#41)#343] +Results [11]: [null AS i_item_id#344, null AS ca_country#345, null AS ca_state#346, null AS county#347, avg(agg1#35)#337 AS agg1#348, avg(agg2#36)#338 AS agg2#349, avg(agg3#37)#339 AS agg3#350, avg(agg4#38)#340 AS agg4#351, avg(agg5#39)#341 AS agg5#352, avg(agg6#40)#342 AS agg6#353, avg(agg7#41)#343 AS agg7#354] (156) Union @@ -868,12 +868,12 @@ ReusedExchange (158) (158) ReusedExchange [Reuses operator id: 34] Output [1]: [d_date_sk#29] -Subquery:2 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#93 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#160 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 101 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 101 Hosting Expression = cs_sold_date_sk#228 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 129 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 129 Hosting Expression = cs_sold_date_sk#296 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 014b89cecd475..f5a7a9135cf29 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 @@ -198,116 +198,116 @@ Aggregate Attributes [1]: [avg(qoh#24)#29] Results [5]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, avg(qoh#24)#29 AS qoh#30] (31) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#31, count#32] +Output [6]: [i_product_name#31, i_brand#32, i_class#33, i_category#34, sum#35, count#36] (32) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#31, count#32] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#33] -Results [4]: [i_product_name#16, i_brand#13, i_class#14, avg(inv_quantity_on_hand#3)#33 AS qoh#24] +Input [6]: [i_product_name#31, i_brand#32, i_class#33, i_category#34, sum#35, count#36] +Keys [4]: [i_product_name#31, i_brand#32, i_class#33, i_category#34] +Functions [1]: [avg(inv_quantity_on_hand#37)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#37)#38] +Results [4]: [i_product_name#31, i_brand#32, i_class#33, avg(inv_quantity_on_hand#37)#38 AS qoh#24] (33) HashAggregate [codegen id : 16] -Input [4]: [i_product_name#16, i_brand#13, i_class#14, qoh#24] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] +Input [4]: [i_product_name#31, i_brand#32, i_class#33, qoh#24] +Keys [3]: [i_product_name#31, i_brand#32, i_class#33] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#34, count#35] -Results [5]: [i_product_name#16, i_brand#13, i_class#14, sum#36, count#37] +Aggregate Attributes [2]: [sum#39, count#40] +Results [5]: [i_product_name#31, i_brand#32, i_class#33, sum#41, count#42] (34) Exchange -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#36, count#37] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [5]: [i_product_name#31, i_brand#32, i_class#33, sum#41, count#42] +Arguments: hashpartitioning(i_product_name#31, i_brand#32, i_class#33, 5), ENSURE_REQUIREMENTS, [id=#43] (35) HashAggregate [codegen id : 17] -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#36, count#37] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] +Input [5]: [i_product_name#31, i_brand#32, i_class#33, sum#41, count#42] +Keys [3]: [i_product_name#31, i_brand#32, i_class#33] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#39] -Results [5]: [i_product_name#16, i_brand#13, i_class#14, null AS i_category#40, avg(qoh#24)#39 AS qoh#41] +Aggregate Attributes [1]: [avg(qoh#24)#44] +Results [5]: [i_product_name#31, i_brand#32, i_class#33, null AS i_category#45, avg(qoh#24)#44 AS qoh#46] (36) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#42, count#43] +Output [6]: [i_product_name#47, i_brand#48, i_class#49, i_category#50, sum#51, count#52] (37) HashAggregate [codegen id : 25] -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#42, count#43] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#44] -Results [3]: [i_product_name#16, i_brand#13, avg(inv_quantity_on_hand#3)#44 AS qoh#24] +Input [6]: [i_product_name#47, i_brand#48, i_class#49, i_category#50, sum#51, count#52] +Keys [4]: [i_product_name#47, i_brand#48, i_class#49, i_category#50] +Functions [1]: [avg(inv_quantity_on_hand#53)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#53)#54] +Results [3]: [i_product_name#47, i_brand#48, avg(inv_quantity_on_hand#53)#54 AS qoh#24] (38) HashAggregate [codegen id : 25] -Input [3]: [i_product_name#16, i_brand#13, qoh#24] -Keys [2]: [i_product_name#16, i_brand#13] +Input [3]: [i_product_name#47, i_brand#48, qoh#24] +Keys [2]: [i_product_name#47, i_brand#48] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#45, count#46] -Results [4]: [i_product_name#16, i_brand#13, sum#47, count#48] +Aggregate Attributes [2]: [sum#55, count#56] +Results [4]: [i_product_name#47, i_brand#48, sum#57, count#58] (39) Exchange -Input [4]: [i_product_name#16, i_brand#13, sum#47, count#48] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [i_product_name#47, i_brand#48, sum#57, count#58] +Arguments: hashpartitioning(i_product_name#47, i_brand#48, 5), ENSURE_REQUIREMENTS, [id=#59] (40) HashAggregate [codegen id : 26] -Input [4]: [i_product_name#16, i_brand#13, sum#47, count#48] -Keys [2]: [i_product_name#16, i_brand#13] +Input [4]: [i_product_name#47, i_brand#48, sum#57, count#58] +Keys [2]: [i_product_name#47, i_brand#48] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#50] -Results [5]: [i_product_name#16, i_brand#13, null AS i_class#51, null AS i_category#52, avg(qoh#24)#50 AS qoh#53] +Aggregate Attributes [1]: [avg(qoh#24)#60] +Results [5]: [i_product_name#47, i_brand#48, null AS i_class#61, null AS i_category#62, avg(qoh#24)#60 AS qoh#63] (41) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#54, count#55] +Output [6]: [i_product_name#64, i_brand#65, i_class#66, i_category#67, sum#68, count#69] (42) HashAggregate [codegen id : 34] -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#54, count#55] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#56] -Results [2]: [i_product_name#16, avg(inv_quantity_on_hand#3)#56 AS qoh#24] +Input [6]: [i_product_name#64, i_brand#65, i_class#66, i_category#67, sum#68, count#69] +Keys [4]: [i_product_name#64, i_brand#65, i_class#66, i_category#67] +Functions [1]: [avg(inv_quantity_on_hand#70)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#70)#71] +Results [2]: [i_product_name#64, avg(inv_quantity_on_hand#70)#71 AS qoh#24] (43) HashAggregate [codegen id : 34] -Input [2]: [i_product_name#16, qoh#24] -Keys [1]: [i_product_name#16] +Input [2]: [i_product_name#64, qoh#24] +Keys [1]: [i_product_name#64] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#57, count#58] -Results [3]: [i_product_name#16, sum#59, count#60] +Aggregate Attributes [2]: [sum#72, count#73] +Results [3]: [i_product_name#64, sum#74, count#75] (44) Exchange -Input [3]: [i_product_name#16, sum#59, count#60] -Arguments: hashpartitioning(i_product_name#16, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [3]: [i_product_name#64, sum#74, count#75] +Arguments: hashpartitioning(i_product_name#64, 5), ENSURE_REQUIREMENTS, [id=#76] (45) HashAggregate [codegen id : 35] -Input [3]: [i_product_name#16, sum#59, count#60] -Keys [1]: [i_product_name#16] +Input [3]: [i_product_name#64, sum#74, count#75] +Keys [1]: [i_product_name#64] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#62] -Results [5]: [i_product_name#16, null AS i_brand#63, null AS i_class#64, null AS i_category#65, avg(qoh#24)#62 AS qoh#66] +Aggregate Attributes [1]: [avg(qoh#24)#77] +Results [5]: [i_product_name#64, null AS i_brand#78, null AS i_class#79, null AS i_category#80, avg(qoh#24)#77 AS qoh#81] (46) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#67, count#68] +Output [6]: [i_product_name#82, i_brand#83, i_class#84, i_category#85, sum#86, count#87] (47) HashAggregate [codegen id : 43] -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#67, count#68] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#69] -Results [1]: [avg(inv_quantity_on_hand#3)#69 AS qoh#24] +Input [6]: [i_product_name#82, i_brand#83, i_class#84, i_category#85, sum#86, count#87] +Keys [4]: [i_product_name#82, i_brand#83, i_class#84, i_category#85] +Functions [1]: [avg(inv_quantity_on_hand#88)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#88)#89] +Results [1]: [avg(inv_quantity_on_hand#88)#89 AS qoh#24] (48) HashAggregate [codegen id : 43] Input [1]: [qoh#24] Keys: [] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#70, count#71] -Results [2]: [sum#72, count#73] +Aggregate Attributes [2]: [sum#90, count#91] +Results [2]: [sum#92, count#93] (49) Exchange -Input [2]: [sum#72, count#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#74] +Input [2]: [sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] (50) HashAggregate [codegen id : 44] -Input [2]: [sum#72, count#73] +Input [2]: [sum#92, count#93] Keys: [] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#75] -Results [5]: [null AS i_product_name#76, null AS i_brand#77, null AS i_class#78, null AS i_category#79, avg(qoh#24)#75 AS qoh#80] +Aggregate Attributes [1]: [avg(qoh#24)#95] +Results [5]: [null AS i_product_name#96, null AS i_brand#97, null AS i_class#98, null AS i_category#99, avg(qoh#24)#95 AS qoh#100] (51) Union 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 74dd076851c1d..527d2eb5e86a4 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 @@ -183,116 +183,116 @@ Aggregate Attributes [1]: [avg(qoh#23)#28] Results [5]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, avg(qoh#23)#28 AS qoh#29] (28) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#30, count#31] +Output [6]: [i_product_name#30, i_brand#31, i_class#32, i_category#33, sum#34, count#35] (29) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#30, count#31] -Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#32] -Results [4]: [i_product_name#13, i_brand#10, i_class#11, avg(inv_quantity_on_hand#3)#32 AS qoh#23] +Input [6]: [i_product_name#30, i_brand#31, i_class#32, i_category#33, sum#34, count#35] +Keys [4]: [i_product_name#30, i_brand#31, i_class#32, i_category#33] +Functions [1]: [avg(inv_quantity_on_hand#36)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#36)#37] +Results [4]: [i_product_name#30, i_brand#31, i_class#32, avg(inv_quantity_on_hand#36)#37 AS qoh#23] (30) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#13, i_brand#10, i_class#11, qoh#23] -Keys [3]: [i_product_name#13, i_brand#10, i_class#11] +Input [4]: [i_product_name#30, i_brand#31, i_class#32, qoh#23] +Keys [3]: [i_product_name#30, i_brand#31, i_class#32] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#33, count#34] -Results [5]: [i_product_name#13, i_brand#10, i_class#11, sum#35, count#36] +Aggregate Attributes [2]: [sum#38, count#39] +Results [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] (31) Exchange -Input [5]: [i_product_name#13, i_brand#10, i_class#11, sum#35, count#36] -Arguments: hashpartitioning(i_product_name#13, i_brand#10, i_class#11, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] +Arguments: hashpartitioning(i_product_name#30, i_brand#31, i_class#32, 5), ENSURE_REQUIREMENTS, [id=#42] (32) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#13, i_brand#10, i_class#11, sum#35, count#36] -Keys [3]: [i_product_name#13, i_brand#10, i_class#11] +Input [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] +Keys [3]: [i_product_name#30, i_brand#31, i_class#32] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#38] -Results [5]: [i_product_name#13, i_brand#10, i_class#11, null AS i_category#39, avg(qoh#23)#38 AS qoh#40] +Aggregate Attributes [1]: [avg(qoh#23)#43] +Results [5]: [i_product_name#30, i_brand#31, i_class#32, null AS i_category#44, avg(qoh#23)#43 AS qoh#45] (33) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#41, count#42] +Output [6]: [i_product_name#46, i_brand#47, i_class#48, i_category#49, sum#50, count#51] (34) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#41, count#42] -Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#43] -Results [3]: [i_product_name#13, i_brand#10, avg(inv_quantity_on_hand#3)#43 AS qoh#23] +Input [6]: [i_product_name#46, i_brand#47, i_class#48, i_category#49, sum#50, count#51] +Keys [4]: [i_product_name#46, i_brand#47, i_class#48, i_category#49] +Functions [1]: [avg(inv_quantity_on_hand#52)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#52)#53] +Results [3]: [i_product_name#46, i_brand#47, avg(inv_quantity_on_hand#52)#53 AS qoh#23] (35) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#13, i_brand#10, qoh#23] -Keys [2]: [i_product_name#13, i_brand#10] +Input [3]: [i_product_name#46, i_brand#47, qoh#23] +Keys [2]: [i_product_name#46, i_brand#47] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#44, count#45] -Results [4]: [i_product_name#13, i_brand#10, sum#46, count#47] +Aggregate Attributes [2]: [sum#54, count#55] +Results [4]: [i_product_name#46, i_brand#47, sum#56, count#57] (36) Exchange -Input [4]: [i_product_name#13, i_brand#10, sum#46, count#47] -Arguments: hashpartitioning(i_product_name#13, i_brand#10, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [4]: [i_product_name#46, i_brand#47, sum#56, count#57] +Arguments: hashpartitioning(i_product_name#46, i_brand#47, 5), ENSURE_REQUIREMENTS, [id=#58] (37) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#13, i_brand#10, sum#46, count#47] -Keys [2]: [i_product_name#13, i_brand#10] +Input [4]: [i_product_name#46, i_brand#47, sum#56, count#57] +Keys [2]: [i_product_name#46, i_brand#47] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#49] -Results [5]: [i_product_name#13, i_brand#10, null AS i_class#50, null AS i_category#51, avg(qoh#23)#49 AS qoh#52] +Aggregate Attributes [1]: [avg(qoh#23)#59] +Results [5]: [i_product_name#46, i_brand#47, null AS i_class#60, null AS i_category#61, avg(qoh#23)#59 AS qoh#62] (38) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#53, count#54] +Output [6]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, sum#67, count#68] (39) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#53, count#54] -Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#55] -Results [2]: [i_product_name#13, avg(inv_quantity_on_hand#3)#55 AS qoh#23] +Input [6]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, sum#67, count#68] +Keys [4]: [i_product_name#63, i_brand#64, i_class#65, i_category#66] +Functions [1]: [avg(inv_quantity_on_hand#69)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#69)#70] +Results [2]: [i_product_name#63, avg(inv_quantity_on_hand#69)#70 AS qoh#23] (40) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#13, qoh#23] -Keys [1]: [i_product_name#13] +Input [2]: [i_product_name#63, qoh#23] +Keys [1]: [i_product_name#63] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#56, count#57] -Results [3]: [i_product_name#13, sum#58, count#59] +Aggregate Attributes [2]: [sum#71, count#72] +Results [3]: [i_product_name#63, sum#73, count#74] (41) Exchange -Input [3]: [i_product_name#13, sum#58, count#59] -Arguments: hashpartitioning(i_product_name#13, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [3]: [i_product_name#63, sum#73, count#74] +Arguments: hashpartitioning(i_product_name#63, 5), ENSURE_REQUIREMENTS, [id=#75] (42) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#13, sum#58, count#59] -Keys [1]: [i_product_name#13] +Input [3]: [i_product_name#63, sum#73, count#74] +Keys [1]: [i_product_name#63] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#61] -Results [5]: [i_product_name#13, null AS i_brand#62, null AS i_class#63, null AS i_category#64, avg(qoh#23)#61 AS qoh#65] +Aggregate Attributes [1]: [avg(qoh#23)#76] +Results [5]: [i_product_name#63, null AS i_brand#77, null AS i_class#78, null AS i_category#79, avg(qoh#23)#76 AS qoh#80] (43) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#66, count#67] +Output [6]: [i_product_name#81, i_brand#82, i_class#83, i_category#84, sum#85, count#86] (44) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#66, count#67] -Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#68] -Results [1]: [avg(inv_quantity_on_hand#3)#68 AS qoh#23] +Input [6]: [i_product_name#81, i_brand#82, i_class#83, i_category#84, sum#85, count#86] +Keys [4]: [i_product_name#81, i_brand#82, i_class#83, i_category#84] +Functions [1]: [avg(inv_quantity_on_hand#87)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#87)#88] +Results [1]: [avg(inv_quantity_on_hand#87)#88 AS qoh#23] (45) HashAggregate [codegen id : 28] Input [1]: [qoh#23] Keys: [] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#69, count#70] -Results [2]: [sum#71, count#72] +Aggregate Attributes [2]: [sum#89, count#90] +Results [2]: [sum#91, count#92] (46) Exchange -Input [2]: [sum#71, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#73] +Input [2]: [sum#91, count#92] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] (47) HashAggregate [codegen id : 29] -Input [2]: [sum#71, count#72] +Input [2]: [sum#91, count#92] Keys: [] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#74] -Results [5]: [null AS i_product_name#75, null AS i_brand#76, null AS i_class#77, null AS i_category#78, avg(qoh#23)#74 AS qoh#79] +Aggregate Attributes [1]: [avg(qoh#23)#94] +Results [5]: [null AS i_product_name#95, null AS i_brand#96, null AS i_class#97, null AS i_category#98, avg(qoh#23)#94 AS qoh#99] (48) Union 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 8031170978906..1c0ed5c8c188b 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 @@ -329,224 +329,224 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (50) Scan parquet default.store -Output [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] +Output [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (51) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] +Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] (52) Filter [codegen id : 1] -Input [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] -Condition : (((isnotnull(s_market_id#3) AND (s_market_id#3 = 8)) AND isnotnull(s_store_sk#1)) AND isnotnull(s_zip#5)) +Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] +Condition : (((isnotnull(s_market_id#54) AND (s_market_id#54 = 8)) AND isnotnull(s_store_sk#52)) AND isnotnull(s_zip#56)) (53) Project [codegen id : 1] -Output [4]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5] -Input [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] +Output [4]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56] +Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] (54) BroadcastExchange -Input [4]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5] -Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#52] +Input [4]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56] +Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#57] (55) Scan parquet default.customer_address -Output [4]: [ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] +Output [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (56) ColumnarToRow -Input [4]: [ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] +Input [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] (57) Filter -Input [4]: [ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] -Condition : ((isnotnull(ca_address_sk#7) AND isnotnull(ca_country#10)) AND isnotnull(ca_zip#9)) +Input [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] +Condition : ((isnotnull(ca_address_sk#58) AND isnotnull(ca_country#61)) AND isnotnull(ca_zip#60)) (58) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [s_zip#5] -Right keys [1]: [ca_zip#9] +Left keys [1]: [s_zip#56] +Right keys [1]: [ca_zip#60] Join condition: None (59) Project [codegen id : 2] -Output [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10] -Input [8]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5, ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] +Output [6]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61] +Input [8]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56, ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] (60) BroadcastExchange -Input [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10] -Arguments: HashedRelationBroadcastMode(List(input[3, int, true], upper(input[5, string, true])),false), [id=#53] +Input [6]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61] +Arguments: HashedRelationBroadcastMode(List(input[3, int, true], upper(input[5, string, true])),false), [id=#62] (61) Scan parquet default.customer -Output [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] +Output [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct (62) ColumnarToRow -Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] +Input [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] (63) Filter -Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] -Condition : ((isnotnull(c_customer_sk#12) AND isnotnull(c_current_addr_sk#13)) AND isnotnull(c_birth_country#16)) +Input [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] +Condition : ((isnotnull(c_customer_sk#63) AND isnotnull(c_current_addr_sk#64)) AND isnotnull(c_birth_country#67)) (64) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ca_address_sk#7, upper(ca_country#10)] -Right keys [2]: [c_current_addr_sk#13, c_birth_country#16] +Left keys [2]: [ca_address_sk#58, upper(ca_country#61)] +Right keys [2]: [c_current_addr_sk#64, c_birth_country#67] Join condition: None (65) Project [codegen id : 3] -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] -Input [11]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10, c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] +Output [7]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66] +Input [11]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61, c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] (66) BroadcastExchange -Input [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] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#54] +Input [7]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#68] (67) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (68) ColumnarToRow -Input [6]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22, ss_sold_date_sk#23] +Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] (69) Filter -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)) +Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] +Condition : (((isnotnull(ss_ticket_number#72) AND isnotnull(ss_item_sk#69)) AND isnotnull(ss_store_sk#71)) AND isnotnull(ss_customer_sk#70)) (70) Project -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] +Output [5]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73] +Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] (71) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [s_store_sk#1, c_customer_sk#12] -Right keys [2]: [ss_store_sk#20, ss_customer_sk#19] +Left keys [2]: [s_store_sk#52, c_customer_sk#63] +Right keys [2]: [ss_store_sk#71, ss_customer_sk#70] Join condition: None (72) Project [codegen id : 4] -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] +Output [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] +Input [12]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73] (73) Exchange -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=#55] +Input [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] +Arguments: hashpartitioning(ss_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] (74) Sort [codegen id : 5] -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 +Input [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] +Arguments: [ss_item_sk#69 ASC NULLS FIRST], false, 0 (75) Scan parquet default.item -Output [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Output [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] (77) Filter [codegen id : 6] -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) +Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Condition : isnotnull(i_item_sk#76) (78) Exchange -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=#56] +Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Arguments: hashpartitioning(i_item_sk#76, 5), ENSURE_REQUIREMENTS, [id=#82] (79) Sort [codegen id : 7] -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 +Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Arguments: [i_item_sk#76 ASC NULLS FIRST], false, 0 (80) SortMergeJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#18] -Right keys [1]: [i_item_sk#24] +Left keys [1]: [ss_item_sk#69] +Right keys [1]: [i_item_sk#76] Join condition: None (81) Project [codegen id : 8] -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] +Output [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Input [14]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] (82) Exchange -Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] -Arguments: hashpartitioning(cast(ss_ticket_number#21 as bigint), cast(ss_item_sk#18 as bigint), 5), ENSURE_REQUIREMENTS, [id=#57] +Input [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Arguments: hashpartitioning(cast(ss_ticket_number#72 as bigint), cast(ss_item_sk#69 as bigint), 5), ENSURE_REQUIREMENTS, [id=#83] (83) Sort [codegen id : 9] -Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] -Arguments: [cast(ss_ticket_number#21 as bigint) ASC NULLS FIRST, cast(ss_item_sk#18 as bigint) ASC NULLS FIRST], false, 0 +Input [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Arguments: [cast(ss_ticket_number#72 as bigint) ASC NULLS FIRST, cast(ss_item_sk#69 as bigint) ASC NULLS FIRST], false, 0 (84) Scan parquet default.store_returns -Output [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Output [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (85) ColumnarToRow [codegen id : 10] -Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] (86) Filter [codegen id : 10] -Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] -Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) +Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] +Condition : (isnotnull(sr_ticket_number#85) AND isnotnull(sr_item_sk#84)) (87) Project [codegen id : 10] -Output [2]: [sr_item_sk#32, sr_ticket_number#33] -Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Output [2]: [sr_item_sk#84, sr_ticket_number#85] +Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] (88) Exchange -Input [2]: [sr_item_sk#32, sr_ticket_number#33] -Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [sr_item_sk#84, sr_ticket_number#85] +Arguments: hashpartitioning(sr_ticket_number#85, sr_item_sk#84, 5), ENSURE_REQUIREMENTS, [id=#87] (89) Sort [codegen id : 11] -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 +Input [2]: [sr_item_sk#84, sr_ticket_number#85] +Arguments: [sr_ticket_number#85 ASC NULLS FIRST, sr_item_sk#84 ASC NULLS FIRST], false, 0 (90) SortMergeJoin [codegen id : 12] -Left keys [2]: [cast(ss_ticket_number#21 as bigint), cast(ss_item_sk#18 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Left keys [2]: [cast(ss_ticket_number#72 as bigint), cast(ss_item_sk#69 as bigint)] +Right keys [2]: [sr_ticket_number#85, sr_item_sk#84] Join condition: None (91) Project [codegen id : 12] -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] +Output [11]: [ss_net_paid#73, s_store_name#53, s_state#55, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, c_first_name#65, c_last_name#66, ca_state#59] +Input [15]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, sr_item_sk#84, sr_ticket_number#85] (92) HashAggregate [codegen id : 12] -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#59] -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#60] +Input [11]: [ss_net_paid#73, s_store_name#53, s_state#55, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, c_first_name#65, c_last_name#66, ca_state#59] +Keys [10]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#73))] +Aggregate Attributes [1]: [sum#88] +Results [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] (93) Exchange -Input [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#60] -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=#61] +Input [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] +Arguments: hashpartitioning(c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, 5), ENSURE_REQUIREMENTS, [id=#90] (94) HashAggregate [codegen id : 13] -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#60] -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))#62] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#62,17,2) AS netpaid#40] +Input [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] +Keys [10]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78] +Functions [1]: [sum(UnscaledValue(ss_net_paid#73))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#73))#91] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#73))#91,17,2) AS netpaid#40] (95) HashAggregate [codegen id : 13] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#63, count#64] -Results [2]: [sum#65, count#66] +Aggregate Attributes [2]: [sum#92, count#93] +Results [2]: [sum#94, count#95] (96) Exchange -Input [2]: [sum#65, count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] +Input [2]: [sum#94, count#95] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#96] (97) HashAggregate [codegen id : 14] -Input [2]: [sum#65, count#66] +Input [2]: [sum#94, count#95] Keys: [] Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#68] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#68)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#69] +Aggregate Attributes [1]: [avg(netpaid#40)#97] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#97)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#98] 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 540feb0d88a3f..7259a3d829613 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 @@ -326,212 +326,212 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (50) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 1] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] (52) Filter [codegen id : 1] -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)) +Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] +Condition : (((isnotnull(ss_ticket_number#55) AND isnotnull(ss_item_sk#52)) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_customer_sk#53)) (53) Project [codegen id : 1] -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] +Output [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] +Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] (54) Exchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#52] +Input [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] +Arguments: hashpartitioning(cast(ss_ticket_number#55 as bigint), cast(ss_item_sk#52 as bigint), 5), ENSURE_REQUIREMENTS, [id=#58] (55) Sort [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +Input [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] +Arguments: [cast(ss_ticket_number#55 as bigint) ASC NULLS FIRST, cast(ss_item_sk#52 as bigint) ASC NULLS FIRST], false, 0 (56) Scan parquet default.store_returns -Output [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] (58) Filter [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] -Condition : (isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#8)) +Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] +Condition : (isnotnull(sr_ticket_number#60) AND isnotnull(sr_item_sk#59)) (59) Project [codegen id : 3] -Output [2]: [sr_item_sk#8, sr_ticket_number#9] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [2]: [sr_item_sk#59, sr_ticket_number#60] +Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] (60) Exchange -Input [2]: [sr_item_sk#8, sr_ticket_number#9] -Arguments: hashpartitioning(sr_ticket_number#9, sr_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [2]: [sr_item_sk#59, sr_ticket_number#60] +Arguments: hashpartitioning(sr_ticket_number#60, sr_item_sk#59, 5), ENSURE_REQUIREMENTS, [id=#62] (61) Sort [codegen id : 4] -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 +Input [2]: [sr_item_sk#59, sr_ticket_number#60] +Arguments: [sr_ticket_number#60 ASC NULLS FIRST, sr_item_sk#59 ASC NULLS FIRST], false, 0 (62) SortMergeJoin [codegen id : 9] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] +Left keys [2]: [cast(ss_ticket_number#55 as bigint), cast(ss_item_sk#52 as bigint)] +Right keys [2]: [sr_ticket_number#60, sr_item_sk#59] Join condition: None (63) Project [codegen id : 9] -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] +Output [4]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_net_paid#56] +Input [7]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, sr_item_sk#59, sr_ticket_number#60] (64) Scan parquet default.store -Output [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (65) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] (66) Filter [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] -Condition : (((isnotnull(s_market_id#14) AND (s_market_id#14 = 8)) AND isnotnull(s_store_sk#12)) AND isnotnull(s_zip#16)) +Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] +Condition : (((isnotnull(s_market_id#65) AND (s_market_id#65 = 8)) AND isnotnull(s_store_sk#63)) AND isnotnull(s_zip#67)) (67) Project [codegen id : 5] -Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [4]: [s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] +Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] (68) BroadcastExchange -Input [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [4]: [s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#68] (69) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] +Left keys [1]: [ss_store_sk#54] +Right keys [1]: [s_store_sk#63] Join condition: None (70) Project [codegen id : 9] -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] +Output [6]: [ss_item_sk#52, ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67] +Input [8]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_net_paid#56, s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] (71) Scan parquet default.item -Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (73) Filter [codegen id : 6] -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) +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Condition : isnotnull(i_item_sk#69) (74) BroadcastExchange -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=#55] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] (75) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [ss_item_sk#52] +Right keys [1]: [i_item_sk#69] Join condition: None (76) Project [codegen id : 9] -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] +Output [10]: [ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [12]: [ss_item_sk#52, ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (77) Scan parquet default.customer -Output [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] +Output [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct (78) ColumnarToRow [codegen id : 7] -Input [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] +Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] (79) Filter [codegen id : 7] -Input [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : ((isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#26)) AND isnotnull(c_birth_country#29)) +Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +Condition : ((isnotnull(c_customer_sk#76) AND isnotnull(c_current_addr_sk#77)) AND isnotnull(c_birth_country#80)) (80) BroadcastExchange -Input [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#56] +Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] (81) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#25] +Left keys [1]: [ss_customer_sk#53] +Right keys [1]: [c_customer_sk#76] Join condition: None (82) Project [codegen id : 9] -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] +Output [13]: [ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +Input [15]: [ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] (83) Scan parquet default.customer_address -Output [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] +Output [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (84) ColumnarToRow [codegen id : 8] -Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] +Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] (85) Filter [codegen id : 8] -Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] -Condition : ((isnotnull(ca_address_sk#31) AND isnotnull(ca_country#34)) AND isnotnull(ca_zip#33)) +Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] +Condition : ((isnotnull(ca_address_sk#82) AND isnotnull(ca_country#85)) AND isnotnull(ca_zip#84)) (86) BroadcastExchange -Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [id=#57] +Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] +Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [id=#86] (87) BroadcastHashJoin [codegen id : 9] -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] +Left keys [3]: [c_current_addr_sk#77, c_birth_country#80, s_zip#67] +Right keys [3]: [ca_address_sk#82, upper(ca_country#85), ca_zip#84] Join condition: None (88) Project [codegen id : 9] -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] +Output [11]: [ss_net_paid#56, s_store_name#64, s_state#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#78, c_last_name#79, ca_state#83] +Input [17]: [ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80, ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] (89) HashAggregate [codegen id : 9] -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#58] -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#59] +Input [11]: [ss_net_paid#56, s_store_name#64, s_state#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#78, c_last_name#79, ca_state#83] +Keys [10]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#56))] +Aggregate Attributes [1]: [sum#87] +Results [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] (90) Exchange -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#59] -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=#60] +Input [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] +Arguments: hashpartitioning(c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#89] (91) HashAggregate [codegen id : 10] -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#59] -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))#61] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#61,17,2) AS netpaid#40] +Input [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] +Keys [10]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [sum(UnscaledValue(ss_net_paid#56))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#56))#90] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#56))#90,17,2) AS netpaid#40] (92) HashAggregate [codegen id : 10] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#62, count#63] -Results [2]: [sum#64, count#65] +Aggregate Attributes [2]: [sum#91, count#92] +Results [2]: [sum#93, count#94] (93) Exchange -Input [2]: [sum#64, count#65] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] +Input [2]: [sum#93, count#94] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] (94) HashAggregate [codegen id : 11] -Input [2]: [sum#64, count#65] +Input [2]: [sum#93, count#94] Keys: [] Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#67] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#67)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#68] +Aggregate Attributes [1]: [avg(netpaid#40)#96] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#96)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#97] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt index 6c48953be5aeb..1fb9533024b2f 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#61), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -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] +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] (35) Filter [codegen id : 11] -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)) +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#62] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#55] +Right keys [1]: [cd_demo_sk#62] Join condition: None (38) Project [codegen id : 11] -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] +Output [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, cd_demo_sk#62] (39) Scan parquet default.store -Output [2]: [s_store_sk#18, s_state#19] +Output [2]: [s_store_sk#63, s_state#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 8] -Input [2]: [s_store_sk#18, s_state#19] +Input [2]: [s_store_sk#63, s_state#64] (41) Filter [codegen id : 8] -Input [2]: [s_store_sk#18, s_state#19] -Condition : ((isnotnull(s_state#19) AND (s_state#19 = TN)) AND isnotnull(s_store_sk#18)) +Input [2]: [s_store_sk#63, s_state#64] +Condition : ((isnotnull(s_state#64) AND (s_state#64 = TN)) AND isnotnull(s_store_sk#63)) (42) Project [codegen id : 8] -Output [1]: [s_store_sk#18] -Input [2]: [s_store_sk#18, s_state#19] +Output [1]: [s_store_sk#63] +Input [2]: [s_store_sk#63, s_state#64] (43) BroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [s_store_sk#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] (44) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#56] +Right keys [1]: [s_store_sk#63] Join condition: None (45) Project [codegen id : 11] -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#18] +Output [6]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Input [8]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, s_store_sk#63] (46) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#66] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#61] +Right keys [1]: [d_date_sk#66] Join condition: None (48) Project [codegen id : 11] -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] +Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#66] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#21, i_item_id#22] +Output [2]: [i_item_sk#67, i_item_id#68] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#67] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] +Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] +Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#22] +Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#68] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] +Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] (53) Exchange -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [1]: [i_item_id#22] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Keys [1]: [i_item_id#68] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] -Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] +Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] +Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] (55) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 17] -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] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] (57) Filter [codegen id : 17] -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)) +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#104] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#104] Join condition: None (60) Project [codegen id : 17] -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] +Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] (61) ReusedExchange [Reuses operator id: 43] -Output [1]: [s_store_sk#18] +Output [1]: [s_store_sk#105] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#98] +Right keys [1]: [s_store_sk#105] Join condition: None (63) Project [codegen id : 17] -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#18] +Output [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, s_store_sk#105] (64) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#106] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#103] +Right keys [1]: [d_date_sk#106] Join condition: None (66) Project [codegen id : 17] -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] +Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [7]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#106] (67) Scan parquet default.item -Output [1]: [i_item_sk#21] +Output [1]: [i_item_sk#107] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#21] +Input [1]: [i_item_sk#107] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#21] -Condition : isnotnull(i_item_sk#21) +Input [1]: [i_item_sk#107] +Condition : isnotnull(i_item_sk#107) (70) BroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] +Input [1]: [i_item_sk#107] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#96] +Right keys [1]: [i_item_sk#107] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] +Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] +Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] -Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] +Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] (74) Exchange -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] -Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] +Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] +Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#15] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 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 04c79a5413ad9..f05f4e9f2e43e 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#61), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -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] +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] (35) Filter [codegen id : 11] -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)) +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#62] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#55] +Right keys [1]: [cd_demo_sk#62] Join condition: None (38) Project [codegen id : 11] -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] +Output [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, cd_demo_sk#62] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#63] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#61] +Right keys [1]: [d_date_sk#63] Join condition: None (41) Project [codegen id : 11] -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] +Output [6]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [8]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#63] (42) Scan parquet default.store -Output [2]: [s_store_sk#18, s_state#19] +Output [2]: [s_store_sk#64, s_state#65] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (43) ColumnarToRow [codegen id : 9] -Input [2]: [s_store_sk#18, s_state#19] +Input [2]: [s_store_sk#64, s_state#65] (44) Filter [codegen id : 9] -Input [2]: [s_store_sk#18, s_state#19] -Condition : ((isnotnull(s_state#19) AND (s_state#19 = TN)) AND isnotnull(s_store_sk#18)) +Input [2]: [s_store_sk#64, s_state#65] +Condition : ((isnotnull(s_state#65) AND (s_state#65 = TN)) AND isnotnull(s_store_sk#64)) (45) Project [codegen id : 9] -Output [1]: [s_store_sk#18] -Input [2]: [s_store_sk#18, s_state#19] +Output [1]: [s_store_sk#64] +Input [2]: [s_store_sk#64, s_state#65] (46) BroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [s_store_sk#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#66] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#56] +Right keys [1]: [s_store_sk#64] Join condition: None (48) Project [codegen id : 11] -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#18] +Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, s_store_sk#64] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#21, i_item_id#22] +Output [2]: [i_item_sk#67, i_item_id#68] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#67] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] +Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] +Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#22] +Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#68] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] +Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] (53) Exchange -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [1]: [i_item_id#22] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Keys [1]: [i_item_id#68] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] -Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] +Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] +Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] (55) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 17] -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] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] (57) Filter [codegen id : 17] -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)) +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#104] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#104] Join condition: None (60) Project [codegen id : 17] -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] +Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] (61) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#105] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#103] +Right keys [1]: [d_date_sk#105] Join condition: None (63) Project [codegen id : 17] -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] +Output [6]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#105] (64) ReusedExchange [Reuses operator id: 46] -Output [1]: [s_store_sk#18] +Output [1]: [s_store_sk#106] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#98] +Right keys [1]: [s_store_sk#106] Join condition: None (66) Project [codegen id : 17] -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#18] +Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#106] (67) Scan parquet default.item -Output [1]: [i_item_sk#21] +Output [1]: [i_item_sk#107] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#21] +Input [1]: [i_item_sk#107] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#21] -Condition : isnotnull(i_item_sk#21) +Input [1]: [i_item_sk#107] +Condition : isnotnull(i_item_sk#107) (70) BroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] +Input [1]: [i_item_sk#107] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#96] +Right keys [1]: [i_item_sk#107] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] +Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] +Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] -Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] +Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] (74) Exchange -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] -Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] +Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] +Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#15] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt index 0a861f4ce8e6d..f7fff421817c0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt @@ -145,20 +145,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#17] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#10] +Right keys [1]: [d_date_sk#17] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] -Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] +Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] (23) Exchange Input [1]: [ws_bill_customer_sk#15] -Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] @@ -170,38 +170,38 @@ Right keys [1]: [ws_bill_customer_sk#15] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Output [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#21] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [cs_sold_date_sk#20] +Right keys [1]: [d_date_sk#21] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#18] -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] +Output [1]: [cs_ship_customer_sk#19] +Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] (31) Exchange -Input [1]: [cs_ship_customer_sk#18] -Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [cs_ship_customer_sk#19] +Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#18] -Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#19] +Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#19] Join condition: None (34) Filter [codegen id : 12] @@ -214,103 +214,103 @@ Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2 (36) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#21] +Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#23] (37) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 (38) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_state#23] +Output [2]: [ca_address_sk#24, ca_state#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 14] -Input [2]: [ca_address_sk#22, ca_state#23] +Input [2]: [ca_address_sk#24, ca_state#25] (40) Filter [codegen id : 14] -Input [2]: [ca_address_sk#22, ca_state#23] -Condition : isnotnull(ca_address_sk#22) +Input [2]: [ca_address_sk#24, ca_state#25] +Condition : isnotnull(ca_address_sk#24) (41) Exchange -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: hashpartitioning(ca_address_sk#22, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] (42) Sort [codegen id : 15] -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#24] Join condition: None (44) Project [codegen id : 16] -Output [2]: [c_current_cdemo_sk#4, ca_state#23] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] +Output [2]: [c_current_cdemo_sk#4, ca_state#25] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24, ca_state#25] (45) Exchange -Input [2]: [c_current_cdemo_sk#4, ca_state#23] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [2]: [c_current_cdemo_sk#4, ca_state#25] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#27] (46) Sort [codegen id : 17] -Input [2]: [c_current_cdemo_sk#4, ca_state#23] +Input [2]: [c_current_cdemo_sk#4, ca_state#25] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 (47) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 18] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (49) Filter [codegen id : 18] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Condition : isnotnull(cd_demo_sk#26) +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Condition : isnotnull(cd_demo_sk#28) (50) Exchange -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: hashpartitioning(cd_demo_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#34] (51) Sort [codegen id : 19] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: [cd_demo_sk#26 ASC NULLS FIRST], false, 0 +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#26] +Right keys [1]: [cd_demo_sk#28] Join condition: None (53) Project [codegen id : 20] -Output [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [8]: [c_current_cdemo_sk#4, ca_state#25, cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (54) HashAggregate [codegen id : 20] -Input [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#29), partial_max(cd_dep_count#29), partial_sum(cd_dep_count#29), partial_avg(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_sum(cd_dep_employed_count#30), partial_avg(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_sum(cd_dep_college_count#31)] -Aggregate Attributes [13]: [count#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45] -Results [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] +Input [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#31), partial_max(cd_dep_count#31), partial_sum(cd_dep_count#31), partial_avg(cd_dep_employed_count#32), partial_max(cd_dep_employed_count#32), partial_sum(cd_dep_employed_count#32), partial_avg(cd_dep_college_count#33), partial_max(cd_dep_college_count#33), partial_sum(cd_dep_college_count#33)] +Aggregate Attributes [13]: [count#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47] +Results [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] (55) Exchange -Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] -Arguments: hashpartitioning(ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] +Arguments: hashpartitioning(ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#61] (56) HashAggregate [codegen id : 21] -Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] -Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [count(1), avg(cd_dep_count#29), max(cd_dep_count#29), sum(cd_dep_count#29), avg(cd_dep_employed_count#30), max(cd_dep_employed_count#30), sum(cd_dep_employed_count#30), avg(cd_dep_college_count#31), max(cd_dep_college_count#31), sum(cd_dep_college_count#31)] -Aggregate Attributes [10]: [count(1)#60, avg(cd_dep_count#29)#61, max(cd_dep_count#29)#62, sum(cd_dep_count#29)#63, avg(cd_dep_employed_count#30)#64, max(cd_dep_employed_count#30)#65, sum(cd_dep_employed_count#30)#66, avg(cd_dep_college_count#31)#67, max(cd_dep_college_count#31)#68, sum(cd_dep_college_count#31)#69] -Results [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, count(1)#60 AS cnt1#70, avg(cd_dep_count#29)#61 AS avg(cd_dep_count)#71, max(cd_dep_count#29)#62 AS max(cd_dep_count)#72, sum(cd_dep_count#29)#63 AS sum(cd_dep_count)#73, cd_dep_employed_count#30, count(1)#60 AS cnt2#74, avg(cd_dep_employed_count#30)#64 AS avg(cd_dep_employed_count)#75, max(cd_dep_employed_count#30)#65 AS max(cd_dep_employed_count)#76, sum(cd_dep_employed_count#30)#66 AS sum(cd_dep_employed_count)#77, cd_dep_college_count#31, count(1)#60 AS cnt3#78, avg(cd_dep_college_count#31)#67 AS avg(cd_dep_college_count)#79, max(cd_dep_college_count#31)#68 AS max(cd_dep_college_count)#80, sum(cd_dep_college_count#31)#69 AS sum(cd_dep_college_count)#81] +Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] +Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Functions [10]: [count(1), avg(cd_dep_count#31), max(cd_dep_count#31), sum(cd_dep_count#31), avg(cd_dep_employed_count#32), max(cd_dep_employed_count#32), sum(cd_dep_employed_count#32), avg(cd_dep_college_count#33), max(cd_dep_college_count#33), sum(cd_dep_college_count#33)] +Aggregate Attributes [10]: [count(1)#62, avg(cd_dep_count#31)#63, max(cd_dep_count#31)#64, sum(cd_dep_count#31)#65, avg(cd_dep_employed_count#32)#66, max(cd_dep_employed_count#32)#67, sum(cd_dep_employed_count#32)#68, avg(cd_dep_college_count#33)#69, max(cd_dep_college_count#33)#70, sum(cd_dep_college_count#33)#71] +Results [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, count(1)#62 AS cnt1#72, avg(cd_dep_count#31)#63 AS avg(cd_dep_count)#73, max(cd_dep_count#31)#64 AS max(cd_dep_count)#74, sum(cd_dep_count#31)#65 AS sum(cd_dep_count)#75, cd_dep_employed_count#32, count(1)#62 AS cnt2#76, avg(cd_dep_employed_count#32)#66 AS avg(cd_dep_employed_count)#77, max(cd_dep_employed_count#32)#67 AS max(cd_dep_employed_count)#78, sum(cd_dep_employed_count#32)#68 AS sum(cd_dep_employed_count)#79, cd_dep_college_count#33, count(1)#62 AS cnt3#80, avg(cd_dep_college_count#33)#69 AS avg(cd_dep_college_count)#81, max(cd_dep_college_count#33)#70 AS max(cd_dep_college_count)#82, sum(cd_dep_college_count#33)#71 AS sum(cd_dep_college_count)#83] (57) TakeOrderedAndProject -Input [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#70, avg(cd_dep_count)#71, max(cd_dep_count)#72, sum(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, avg(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, sum(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, avg(cd_dep_college_count)#79, max(cd_dep_college_count)#80, sum(cd_dep_college_count)#81] -Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#70, avg(cd_dep_count)#71, max(cd_dep_count)#72, sum(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, avg(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, sum(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, avg(cd_dep_college_count)#79, max(cd_dep_college_count)#80, sum(cd_dep_college_count)#81] +Input [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cnt1#72, avg(cd_dep_count)#73, max(cd_dep_count)#74, sum(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, avg(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, sum(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, avg(cd_dep_college_count)#81, max(cd_dep_college_count)#82, sum(cd_dep_college_count)#83] +Arguments: 100, [ca_state#25 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cnt1#72, avg(cd_dep_count)#73, max(cd_dep_count)#74, sum(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, avg(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, sum(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, avg(cd_dep_college_count)#81, max(cd_dep_college_count)#82, sum(cd_dep_college_count)#83] ===== Subqueries ===== @@ -323,6 +323,6 @@ Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index 12c8230c7e3bc..642bf989159d3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -122,20 +122,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#16] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] -Right keys [1]: [d_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] +Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#16] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] @@ -143,34 +143,34 @@ Right keys [1]: [ws_bill_customer_sk#14] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#8)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#20] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#9] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#20] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#17] -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] +Output [1]: [cs_ship_customer_sk#18] +Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#20] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [cs_ship_customer_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#17] +Right keys [1]: [cs_ship_customer_sk#18] Join condition: None (29) Filter [codegen id : 9] @@ -182,80 +182,80 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +Output [2]: [ca_address_sk#22, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_state#21] +Input [2]: [ca_address_sk#22, ca_state#23] (33) Filter [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) +Input [2]: [ca_address_sk#22, ca_state#23] +Condition : isnotnull(ca_address_sk#22) (34) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +Input [2]: [ca_address_sk#22, ca_state#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] (35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#22] Join condition: None (36) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#21] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] +Output [2]: [c_current_cdemo_sk#4, ca_state#23] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] (37) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (39) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#25) (40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#25] Join condition: None (42) Project [codegen id : 9] -Output [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Output [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (43) HashAggregate [codegen id : 9] -Input [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] -Aggregate Attributes [13]: [count#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42] -Results [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] +Input [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#28), partial_max(cd_dep_count#28), partial_sum(cd_dep_count#28), partial_avg(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_sum(cd_dep_employed_count#29), partial_avg(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_sum(cd_dep_college_count#30)] +Aggregate Attributes [13]: [count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] +Results [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] (44) Exchange -Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] -Arguments: hashpartitioning(ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] +Arguments: hashpartitioning(ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] (45) HashAggregate [codegen id : 10] -Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] -Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] -Aggregate Attributes [10]: [count(1)#57, avg(cd_dep_count#26)#58, max(cd_dep_count#26)#59, sum(cd_dep_count#26)#60, avg(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, sum(cd_dep_employed_count#27)#63, avg(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, sum(cd_dep_college_count#28)#66] -Results [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, count(1)#57 AS cnt1#67, avg(cd_dep_count#26)#58 AS avg(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, sum(cd_dep_count#26)#60 AS sum(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, avg(cd_dep_employed_count#27)#61 AS avg(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, sum(cd_dep_employed_count#27)#63 AS sum(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, avg(cd_dep_college_count#28)#64 AS avg(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, sum(cd_dep_college_count#28)#66 AS sum(cd_dep_college_count)#78] +Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] +Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Functions [10]: [count(1), avg(cd_dep_count#28), max(cd_dep_count#28), sum(cd_dep_count#28), avg(cd_dep_employed_count#29), max(cd_dep_employed_count#29), sum(cd_dep_employed_count#29), avg(cd_dep_college_count#30), max(cd_dep_college_count#30), sum(cd_dep_college_count#30)] +Aggregate Attributes [10]: [count(1)#59, avg(cd_dep_count#28)#60, max(cd_dep_count#28)#61, sum(cd_dep_count#28)#62, avg(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, sum(cd_dep_employed_count#29)#65, avg(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, sum(cd_dep_college_count#30)#68] +Results [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, count(1)#59 AS cnt1#69, avg(cd_dep_count#28)#60 AS avg(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, sum(cd_dep_count#28)#62 AS sum(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, avg(cd_dep_employed_count#29)#63 AS avg(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, sum(cd_dep_employed_count#29)#65 AS sum(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, avg(cd_dep_college_count#30)#66 AS avg(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, sum(cd_dep_college_count#30)#68 AS sum(cd_dep_college_count)#80] (46) TakeOrderedAndProject -Input [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] -Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] +Input [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] +Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] ===== Subqueries ===== @@ -268,6 +268,6 @@ Output [1]: [d_date_sk#9] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt index 1c1b8ad01abfa..df75bec72326d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt @@ -142,52 +142,52 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#15] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (22) Project [codegen id : 7] -Output [1]: [ws_bill_customer_sk#13 AS customsk#15] -Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] +Output [1]: [ws_bill_customer_sk#13 AS customsk#16] +Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] (23) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] ReadSchema: struct (24) ColumnarToRow [codegen id : 9] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (25) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#19] (26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (27) Project [codegen id : 9] -Output [1]: [cs_ship_customer_sk#16 AS customsk#18] -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] +Output [1]: [cs_ship_customer_sk#17 AS customsk#20] +Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] (28) Union (29) Exchange -Input [1]: [customsk#15] -Arguments: hashpartitioning(customsk#15, 5), ENSURE_REQUIREMENTS, [id=#19] +Input [1]: [customsk#16] +Arguments: hashpartitioning(customsk#16, 5), ENSURE_REQUIREMENTS, [id=#21] (30) Sort [codegen id : 10] -Input [1]: [customsk#15] -Arguments: [customsk#15 ASC NULLS FIRST], false, 0 +Input [1]: [customsk#16] +Arguments: [customsk#16 ASC NULLS FIRST], false, 0 (31) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#15] +Right keys [1]: [customsk#16] Join condition: None (32) Project [codegen id : 11] @@ -196,103 +196,103 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (33) Exchange Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: hashpartitioning(c_current_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#20] +Arguments: hashpartitioning(c_current_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#22] (34) Sort [codegen id : 12] Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: [c_current_addr_sk#3 ASC NULLS FIRST], false, 0 (35) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] +Output [2]: [ca_address_sk#23, ca_state#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 13] -Input [2]: [ca_address_sk#21, ca_state#22] +Input [2]: [ca_address_sk#23, ca_state#24] (37) Filter [codegen id : 13] -Input [2]: [ca_address_sk#21, ca_state#22] -Condition : isnotnull(ca_address_sk#21) +Input [2]: [ca_address_sk#23, ca_state#24] +Condition : isnotnull(ca_address_sk#23) (38) Exchange -Input [2]: [ca_address_sk#21, ca_state#22] -Arguments: hashpartitioning(ca_address_sk#21, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [2]: [ca_address_sk#23, ca_state#24] +Arguments: hashpartitioning(ca_address_sk#23, 5), ENSURE_REQUIREMENTS, [id=#25] (39) Sort [codegen id : 14] -Input [2]: [ca_address_sk#21, ca_state#22] -Arguments: [ca_address_sk#21 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#23, ca_state#24] +Arguments: [ca_address_sk#23 ASC NULLS FIRST], false, 0 (40) SortMergeJoin [codegen id : 15] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#23] Join condition: None (41) Project [codegen id : 15] -Output [2]: [c_current_cdemo_sk#2, ca_state#22] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21, ca_state#22] +Output [2]: [c_current_cdemo_sk#2, ca_state#24] +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#23, ca_state#24] (42) Exchange -Input [2]: [c_current_cdemo_sk#2, ca_state#22] -Arguments: hashpartitioning(c_current_cdemo_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [2]: [c_current_cdemo_sk#2, ca_state#24] +Arguments: hashpartitioning(c_current_cdemo_sk#2, 5), ENSURE_REQUIREMENTS, [id=#26] (43) Sort [codegen id : 16] -Input [2]: [c_current_cdemo_sk#2, ca_state#22] +Input [2]: [c_current_cdemo_sk#2, ca_state#24] Arguments: [c_current_cdemo_sk#2 ASC NULLS FIRST], false, 0 (44) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (45) ColumnarToRow [codegen id : 17] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] (46) Filter [codegen id : 17] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#25) +Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Condition : isnotnull(cd_demo_sk#27) (47) Exchange -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: hashpartitioning(cd_demo_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Arguments: hashpartitioning(cd_demo_sk#27, 5), ENSURE_REQUIREMENTS, [id=#33] (48) Sort [codegen id : 18] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#25 ASC NULLS FIRST], false, 0 +Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Arguments: [cd_demo_sk#27 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 19] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#27] Join condition: None (50) Project [codegen id : 19] -Output [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [8]: [c_current_cdemo_sk#2, ca_state#24, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] (51) HashAggregate [codegen id : 19] -Input [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#28), partial_max(cd_dep_count#28), partial_sum(cd_dep_count#28), partial_avg(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_sum(cd_dep_employed_count#29), partial_avg(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_sum(cd_dep_college_count#30)] -Aggregate Attributes [13]: [count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] -Results [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] +Input [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Keys [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#30), partial_max(cd_dep_count#30), partial_sum(cd_dep_count#30), partial_avg(cd_dep_employed_count#31), partial_max(cd_dep_employed_count#31), partial_sum(cd_dep_employed_count#31), partial_avg(cd_dep_college_count#32), partial_max(cd_dep_college_count#32), partial_sum(cd_dep_college_count#32)] +Aggregate Attributes [13]: [count#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42, sum#43, count#44, max#45, sum#46] +Results [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] (52) Exchange -Input [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] -Arguments: hashpartitioning(ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] +Arguments: hashpartitioning(ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#60] (53) HashAggregate [codegen id : 20] -Input [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] -Keys [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [10]: [count(1), avg(cd_dep_count#28), max(cd_dep_count#28), sum(cd_dep_count#28), avg(cd_dep_employed_count#29), max(cd_dep_employed_count#29), sum(cd_dep_employed_count#29), avg(cd_dep_college_count#30), max(cd_dep_college_count#30), sum(cd_dep_college_count#30)] -Aggregate Attributes [10]: [count(1)#59, avg(cd_dep_count#28)#60, max(cd_dep_count#28)#61, sum(cd_dep_count#28)#62, avg(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, sum(cd_dep_employed_count#29)#65, avg(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, sum(cd_dep_college_count#30)#68] -Results [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, count(1)#59 AS cnt1#69, avg(cd_dep_count#28)#60 AS avg(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, sum(cd_dep_count#28)#62 AS sum(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, avg(cd_dep_employed_count#29)#63 AS avg(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, sum(cd_dep_employed_count#29)#65 AS sum(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, avg(cd_dep_college_count#30)#66 AS avg(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, sum(cd_dep_college_count#30)#68 AS sum(cd_dep_college_count)#80] +Input [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] +Keys [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Functions [10]: [count(1), avg(cd_dep_count#30), max(cd_dep_count#30), sum(cd_dep_count#30), avg(cd_dep_employed_count#31), max(cd_dep_employed_count#31), sum(cd_dep_employed_count#31), avg(cd_dep_college_count#32), max(cd_dep_college_count#32), sum(cd_dep_college_count#32)] +Aggregate Attributes [10]: [count(1)#61, avg(cd_dep_count#30)#62, max(cd_dep_count#30)#63, sum(cd_dep_count#30)#64, avg(cd_dep_employed_count#31)#65, max(cd_dep_employed_count#31)#66, sum(cd_dep_employed_count#31)#67, avg(cd_dep_college_count#32)#68, max(cd_dep_college_count#32)#69, sum(cd_dep_college_count#32)#70] +Results [18]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, count(1)#61 AS cnt1#71, avg(cd_dep_count#30)#62 AS avg(cd_dep_count)#72, max(cd_dep_count#30)#63 AS max(cd_dep_count)#73, sum(cd_dep_count#30)#64 AS sum(cd_dep_count)#74, cd_dep_employed_count#31, count(1)#61 AS cnt2#75, avg(cd_dep_employed_count#31)#65 AS avg(cd_dep_employed_count)#76, max(cd_dep_employed_count#31)#66 AS max(cd_dep_employed_count)#77, sum(cd_dep_employed_count#31)#67 AS sum(cd_dep_employed_count)#78, cd_dep_college_count#32, count(1)#61 AS cnt3#79, avg(cd_dep_college_count#32)#68 AS avg(cd_dep_college_count)#80, max(cd_dep_college_count#32)#69 AS max(cd_dep_college_count)#81, sum(cd_dep_college_count#32)#70 AS sum(cd_dep_college_count)#82] (54) TakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] -Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] +Input [18]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cnt1#71, avg(cd_dep_count)#72, max(cd_dep_count)#73, sum(cd_dep_count)#74, cd_dep_employed_count#31, cnt2#75, avg(cd_dep_employed_count)#76, max(cd_dep_employed_count)#77, sum(cd_dep_employed_count)#78, cd_dep_college_count#32, cnt3#79, avg(cd_dep_college_count)#80, max(cd_dep_college_count)#81, sum(cd_dep_college_count)#82] +Arguments: 100, [ca_state#24 ASC NULLS FIRST, cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cnt1#71, avg(cd_dep_count)#72, max(cd_dep_count)#73, sum(cd_dep_count)#74, cd_dep_employed_count#31, cnt2#75, avg(cd_dep_employed_count)#76, max(cd_dep_employed_count)#77, sum(cd_dep_employed_count)#78, cd_dep_college_count#32, cnt3#79, avg(cd_dep_college_count)#80, max(cd_dep_college_count)#81, sum(cd_dep_college_count)#82] ===== Subqueries ===== @@ -305,6 +305,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index ffb07d398de71..57710f427f0f8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -120,48 +120,48 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#12 AS customsk#14] -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] +Output [1]: [ws_bill_customer_sk#12 AS customsk#15] +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] (20) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#6)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] (22) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#18] (23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#18] Join condition: None (24) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#15 AS customsk#17] -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] +Output [1]: [cs_ship_customer_sk#16 AS customsk#19] +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#18] (25) Union (26) BroadcastExchange -Input [1]: [customsk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Input [1]: [customsk#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (27) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#14] +Right keys [1]: [customsk#15] Join condition: None (28) Project [codegen id : 9] @@ -169,80 +169,80 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#19, ca_state#20] +Output [2]: [ca_address_sk#21, ca_state#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#19, ca_state#20] +Input [2]: [ca_address_sk#21, ca_state#22] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#19, ca_state#20] -Condition : isnotnull(ca_address_sk#19) +Input [2]: [ca_address_sk#21, ca_state#22] +Condition : isnotnull(ca_address_sk#21) (32) BroadcastExchange -Input [2]: [ca_address_sk#19, ca_state#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Input [2]: [ca_address_sk#21, ca_state#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] (33) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#19] +Right keys [1]: [ca_address_sk#21] Join condition: None (34) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, ca_state#20] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19, ca_state#20] +Output [2]: [c_current_cdemo_sk#2, ca_state#22] +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21, ca_state#22] (35) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Output [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] (37) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Condition : isnotnull(cd_demo_sk#22) +Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Condition : isnotnull(cd_demo_sk#24) (38) BroadcastExchange -Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] +Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (39) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#22] +Right keys [1]: [cd_demo_sk#24] Join condition: None (40) Project [codegen id : 9] -Output [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Input [8]: [c_current_cdemo_sk#2, ca_state#20, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Output [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] (41) HashAggregate [codegen id : 9] -Input [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_dep_count#25), partial_sum(cd_dep_count#25), partial_avg(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_sum(cd_dep_employed_count#26), partial_avg(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_sum(cd_dep_college_count#27)] -Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] -Results [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +Input [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Keys [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#27), partial_max(cd_dep_count#27), partial_sum(cd_dep_count#27), partial_avg(cd_dep_employed_count#28), partial_max(cd_dep_employed_count#28), partial_sum(cd_dep_employed_count#28), partial_avg(cd_dep_college_count#29), partial_max(cd_dep_college_count#29), partial_sum(cd_dep_college_count#29)] +Aggregate Attributes [13]: [count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] +Results [19]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] (42) Exchange -Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Arguments: hashpartitioning(ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [19]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] +Arguments: hashpartitioning(ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, 5), ENSURE_REQUIREMENTS, [id=#57] (43) HashAggregate [codegen id : 10] -Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_dep_count#25), avg(cd_dep_employed_count#26), max(cd_dep_employed_count#26), sum(cd_dep_employed_count#26), avg(cd_dep_college_count#27), max(cd_dep_college_count#27), sum(cd_dep_college_count#27)] -Aggregate Attributes [10]: [count(1)#56, avg(cd_dep_count#25)#57, max(cd_dep_count#25)#58, sum(cd_dep_count#25)#59, avg(cd_dep_employed_count#26)#60, max(cd_dep_employed_count#26)#61, sum(cd_dep_employed_count#26)#62, avg(cd_dep_college_count#27)#63, max(cd_dep_college_count#27)#64, sum(cd_dep_college_count#27)#65] -Results [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, count(1)#56 AS cnt1#66, avg(cd_dep_count#25)#57 AS avg(cd_dep_count)#67, max(cd_dep_count#25)#58 AS max(cd_dep_count)#68, sum(cd_dep_count#25)#59 AS sum(cd_dep_count)#69, cd_dep_employed_count#26, count(1)#56 AS cnt2#70, avg(cd_dep_employed_count#26)#60 AS avg(cd_dep_employed_count)#71, max(cd_dep_employed_count#26)#61 AS max(cd_dep_employed_count)#72, sum(cd_dep_employed_count#26)#62 AS sum(cd_dep_employed_count)#73, cd_dep_college_count#27, count(1)#56 AS cnt3#74, avg(cd_dep_college_count#27)#63 AS avg(cd_dep_college_count)#75, max(cd_dep_college_count#27)#64 AS max(cd_dep_college_count)#76, sum(cd_dep_college_count#27)#65 AS sum(cd_dep_college_count)#77] +Input [19]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] +Keys [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Functions [10]: [count(1), avg(cd_dep_count#27), max(cd_dep_count#27), sum(cd_dep_count#27), avg(cd_dep_employed_count#28), max(cd_dep_employed_count#28), sum(cd_dep_employed_count#28), avg(cd_dep_college_count#29), max(cd_dep_college_count#29), sum(cd_dep_college_count#29)] +Aggregate Attributes [10]: [count(1)#58, avg(cd_dep_count#27)#59, max(cd_dep_count#27)#60, sum(cd_dep_count#27)#61, avg(cd_dep_employed_count#28)#62, max(cd_dep_employed_count#28)#63, sum(cd_dep_employed_count#28)#64, avg(cd_dep_college_count#29)#65, max(cd_dep_college_count#29)#66, sum(cd_dep_college_count#29)#67] +Results [18]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, count(1)#58 AS cnt1#68, avg(cd_dep_count#27)#59 AS avg(cd_dep_count)#69, max(cd_dep_count#27)#60 AS max(cd_dep_count)#70, sum(cd_dep_count#27)#61 AS sum(cd_dep_count)#71, cd_dep_employed_count#28, count(1)#58 AS cnt2#72, avg(cd_dep_employed_count#28)#62 AS avg(cd_dep_employed_count)#73, max(cd_dep_employed_count#28)#63 AS max(cd_dep_employed_count)#74, sum(cd_dep_employed_count#28)#64 AS sum(cd_dep_employed_count)#75, cd_dep_college_count#29, count(1)#58 AS cnt3#76, avg(cd_dep_college_count#29)#65 AS avg(cd_dep_college_count)#77, max(cd_dep_college_count#29)#66 AS max(cd_dep_college_count)#78, sum(cd_dep_college_count#29)#67 AS sum(cd_dep_college_count)#79] (44) TakeOrderedAndProject -Input [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] -Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] +Input [18]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#28, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#29, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79] +Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_dep_count#27 ASC NULLS FIRST, cd_dep_employed_count#28 ASC NULLS FIRST, cd_dep_college_count#29 ASC NULLS FIRST], [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#28, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#29, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79] ===== Subqueries ===== @@ -255,6 +255,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#6 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 9f594df6754ba..79037befbd6b5 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 @@ -169,60 +169,60 @@ Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#22, sum(UnscaledV Results [6]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#22,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#23,17,2))), DecimalType(37,20), true) as decimal(38,20)) AS gross_margin#24, i_category#15, i_class#14, 0 AS t_category#25, 0 AS t_class#26, 0 AS lochierarchy#27] (27) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#15, i_class#14, sum#28, sum#29] +Output [4]: [i_category#28, i_class#29, sum#30, sum#31] (28) HashAggregate [codegen id : 10] -Input [4]: [i_category#15, i_class#14, sum#28, sum#29] -Keys [2]: [i_category#15, i_class#14] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#30, sum(UnscaledValue(ss_ext_sales_price#3))#31] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#30,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#31,17,2) AS ss_ext_sales_price#33, i_category#15] +Input [4]: [i_category#28, i_class#29, sum#30, sum#31] +Keys [2]: [i_category#28, i_class#29] +Functions [2]: [sum(UnscaledValue(ss_net_profit#32)), sum(UnscaledValue(ss_ext_sales_price#33))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#32))#34, sum(UnscaledValue(ss_ext_sales_price#33))#35] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#32))#34,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#33))#35,17,2) AS ss_ext_sales_price#37, i_category#28] (29) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#32, ss_ext_sales_price#33, i_category#15] -Keys [1]: [i_category#15] -Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] -Aggregate Attributes [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] -Results [5]: [i_category#15, sum#38, isEmpty#39, sum#40, isEmpty#41] +Input [3]: [ss_net_profit#36, ss_ext_sales_price#37, i_category#28] +Keys [1]: [i_category#28] +Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] +Aggregate Attributes [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] +Results [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] (30) Exchange -Input [5]: [i_category#15, sum#38, isEmpty#39, sum#40, isEmpty#41] -Arguments: hashpartitioning(i_category#15, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, [id=#46] (31) HashAggregate [codegen id : 11] -Input [5]: [i_category#15, sum#38, isEmpty#39, sum#40, isEmpty#41] -Keys [1]: [i_category#15] -Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] -Aggregate Attributes [2]: [sum(ss_net_profit#32)#43, sum(ss_ext_sales_price#33)#44] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#43) / promote_precision(sum(ss_ext_sales_price#33)#44)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#45, i_category#15, null AS i_class#46, 0 AS t_category#47, 1 AS t_class#48, 1 AS lochierarchy#49] +Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] +Keys [1]: [i_category#28] +Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] +Aggregate Attributes [2]: [sum(ss_net_profit#36)#47, sum(ss_ext_sales_price#37)#48] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#47) / promote_precision(sum(ss_ext_sales_price#37)#48)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#49, i_category#28, null AS i_class#50, 0 AS t_category#51, 1 AS t_class#52, 1 AS lochierarchy#53] (32) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#15, i_class#14, sum#50, sum#51] +Output [4]: [i_category#54, i_class#55, sum#56, sum#57] (33) HashAggregate [codegen id : 16] -Input [4]: [i_category#15, i_class#14, sum#50, sum#51] -Keys [2]: [i_category#15, i_class#14] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#52, sum(UnscaledValue(ss_ext_sales_price#3))#53] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#52,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#53,17,2) AS ss_ext_sales_price#33] +Input [4]: [i_category#54, i_class#55, sum#56, sum#57] +Keys [2]: [i_category#54, i_class#55] +Functions [2]: [sum(UnscaledValue(ss_net_profit#58)), sum(UnscaledValue(ss_ext_sales_price#59))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#58))#60, sum(UnscaledValue(ss_ext_sales_price#59))#61] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#58))#60,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#59))#61,17,2) AS ss_ext_sales_price#37] (34) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#32, ss_ext_sales_price#33] +Input [2]: [ss_net_profit#36, ss_ext_sales_price#37] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] -Aggregate Attributes [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] -Results [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] +Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] +Aggregate Attributes [4]: [sum#62, isEmpty#63, sum#64, isEmpty#65] +Results [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] (35) Exchange -Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] +Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#70] (36) HashAggregate [codegen id : 17] -Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] +Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] Keys: [] -Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] -Aggregate Attributes [2]: [sum(ss_net_profit#32)#63, sum(ss_ext_sales_price#33)#64] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#63) / promote_precision(sum(ss_ext_sales_price#33)#64)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#65, null AS i_category#66, null AS i_class#67, 1 AS t_category#68, 1 AS t_class#69, 2 AS lochierarchy#70] +Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] +Aggregate Attributes [2]: [sum(ss_net_profit#36)#71, sum(ss_ext_sales_price#37)#72] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#71) / promote_precision(sum(ss_ext_sales_price#37)#72)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#73, null AS i_category#74, null AS i_class#75, 1 AS t_category#76, 1 AS t_class#77, 2 AS lochierarchy#78] (37) Union @@ -235,34 +235,34 @@ Results [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class (39) Exchange Input [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27] -Arguments: hashpartitioning(gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#71] +Arguments: hashpartitioning(gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#79] (40) HashAggregate [codegen id : 19] Input [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27] Keys [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27] Functions: [] Aggregate Attributes: [] -Results [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#15 END AS _w0#72] +Results [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#15 END AS _w0#80] (41) Exchange -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72] -Arguments: hashpartitioning(lochierarchy#27, _w0#72, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80] +Arguments: hashpartitioning(lochierarchy#27, _w0#80, 5), ENSURE_REQUIREMENTS, [id=#81] (42) Sort [codegen id : 20] -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72] -Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80] +Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#80 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 (43) Window -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72] -Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#72, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#74], [lochierarchy#27, _w0#72], [gross_margin#24 ASC NULLS FIRST] +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80] +Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#80, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#82], [lochierarchy#27, _w0#80], [gross_margin#24 ASC NULLS FIRST] (44) Project [codegen id : 21] -Output [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#74] -Input [6]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72, rank_within_parent#74] +Output [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#82] +Input [6]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80, rank_within_parent#82] (45) TakeOrderedAndProject -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#74] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#74 ASC NULLS FIRST], [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#74] +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#82] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#82 ASC NULLS FIRST], [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#82] ===== Subqueries ===== 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 5e58913557f22..bc2931129868b 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 @@ -169,60 +169,60 @@ Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#22, sum(UnscaledV Results [6]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#22,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#23,17,2))), DecimalType(37,20), true) as decimal(38,20)) AS gross_margin#24, i_category#12, i_class#11, 0 AS t_category#25, 0 AS t_class#26, 0 AS lochierarchy#27] (27) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#12, i_class#11, sum#28, sum#29] +Output [4]: [i_category#28, i_class#29, sum#30, sum#31] (28) HashAggregate [codegen id : 10] -Input [4]: [i_category#12, i_class#11, sum#28, sum#29] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#30, sum(UnscaledValue(ss_ext_sales_price#3))#31] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#30,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#31,17,2) AS ss_ext_sales_price#33, i_category#12] +Input [4]: [i_category#28, i_class#29, sum#30, sum#31] +Keys [2]: [i_category#28, i_class#29] +Functions [2]: [sum(UnscaledValue(ss_net_profit#32)), sum(UnscaledValue(ss_ext_sales_price#33))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#32))#34, sum(UnscaledValue(ss_ext_sales_price#33))#35] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#32))#34,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#33))#35,17,2) AS ss_ext_sales_price#37, i_category#28] (29) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#32, ss_ext_sales_price#33, i_category#12] -Keys [1]: [i_category#12] -Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] -Aggregate Attributes [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] -Results [5]: [i_category#12, sum#38, isEmpty#39, sum#40, isEmpty#41] +Input [3]: [ss_net_profit#36, ss_ext_sales_price#37, i_category#28] +Keys [1]: [i_category#28] +Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] +Aggregate Attributes [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] +Results [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] (30) Exchange -Input [5]: [i_category#12, sum#38, isEmpty#39, sum#40, isEmpty#41] -Arguments: hashpartitioning(i_category#12, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, [id=#46] (31) HashAggregate [codegen id : 11] -Input [5]: [i_category#12, sum#38, isEmpty#39, sum#40, isEmpty#41] -Keys [1]: [i_category#12] -Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] -Aggregate Attributes [2]: [sum(ss_net_profit#32)#43, sum(ss_ext_sales_price#33)#44] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#43) / promote_precision(sum(ss_ext_sales_price#33)#44)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#45, i_category#12, null AS i_class#46, 0 AS t_category#47, 1 AS t_class#48, 1 AS lochierarchy#49] +Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] +Keys [1]: [i_category#28] +Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] +Aggregate Attributes [2]: [sum(ss_net_profit#36)#47, sum(ss_ext_sales_price#37)#48] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#47) / promote_precision(sum(ss_ext_sales_price#37)#48)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#49, i_category#28, null AS i_class#50, 0 AS t_category#51, 1 AS t_class#52, 1 AS lochierarchy#53] (32) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#12, i_class#11, sum#50, sum#51] +Output [4]: [i_category#54, i_class#55, sum#56, sum#57] (33) HashAggregate [codegen id : 16] -Input [4]: [i_category#12, i_class#11, sum#50, sum#51] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#52, sum(UnscaledValue(ss_ext_sales_price#3))#53] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#52,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#53,17,2) AS ss_ext_sales_price#33] +Input [4]: [i_category#54, i_class#55, sum#56, sum#57] +Keys [2]: [i_category#54, i_class#55] +Functions [2]: [sum(UnscaledValue(ss_net_profit#58)), sum(UnscaledValue(ss_ext_sales_price#59))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#58))#60, sum(UnscaledValue(ss_ext_sales_price#59))#61] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#58))#60,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#59))#61,17,2) AS ss_ext_sales_price#37] (34) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#32, ss_ext_sales_price#33] +Input [2]: [ss_net_profit#36, ss_ext_sales_price#37] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] -Aggregate Attributes [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] -Results [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] +Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] +Aggregate Attributes [4]: [sum#62, isEmpty#63, sum#64, isEmpty#65] +Results [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] (35) Exchange -Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] +Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#70] (36) HashAggregate [codegen id : 17] -Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] +Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] Keys: [] -Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] -Aggregate Attributes [2]: [sum(ss_net_profit#32)#63, sum(ss_ext_sales_price#33)#64] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#63) / promote_precision(sum(ss_ext_sales_price#33)#64)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#65, null AS i_category#66, null AS i_class#67, 1 AS t_category#68, 1 AS t_class#69, 2 AS lochierarchy#70] +Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] +Aggregate Attributes [2]: [sum(ss_net_profit#36)#71, sum(ss_ext_sales_price#37)#72] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#71) / promote_precision(sum(ss_ext_sales_price#37)#72)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#73, null AS i_category#74, null AS i_class#75, 1 AS t_category#76, 1 AS t_class#77, 2 AS lochierarchy#78] (37) Union @@ -235,34 +235,34 @@ Results [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class (39) Exchange Input [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27] -Arguments: hashpartitioning(gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#71] +Arguments: hashpartitioning(gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#79] (40) HashAggregate [codegen id : 19] Input [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27] Keys [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27] Functions: [] Aggregate Attributes: [] -Results [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#12 END AS _w0#72] +Results [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#12 END AS _w0#80] (41) Exchange -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72] -Arguments: hashpartitioning(lochierarchy#27, _w0#72, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80] +Arguments: hashpartitioning(lochierarchy#27, _w0#80, 5), ENSURE_REQUIREMENTS, [id=#81] (42) Sort [codegen id : 20] -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72] -Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80] +Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#80 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 (43) Window -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72] -Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#72, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#74], [lochierarchy#27, _w0#72], [gross_margin#24 ASC NULLS FIRST] +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80] +Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#80, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#82], [lochierarchy#27, _w0#80], [gross_margin#24 ASC NULLS FIRST] (44) Project [codegen id : 21] -Output [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#74] -Input [6]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72, rank_within_parent#74] +Output [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#82] +Input [6]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80, rank_within_parent#82] (45) TakeOrderedAndProject -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#74] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#74 ASC NULLS FIRST], [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#74] +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#82] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#82 ASC NULLS FIRST], [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#82] ===== Subqueries ===== 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 a0737e88bd760..636fa32f94212 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 @@ -229,78 +229,78 @@ Output [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_ye (39) HashAggregate [codegen id : 21] Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum#36] Keys [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35] -Functions [1]: [sum(UnscaledValue(ss_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#3))#37] -Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#3))#37,17,2) AS sum_sales#38] +Functions [1]: [sum(UnscaledValue(ss_sales_price#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#37))#38] +Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#37))#38,17,2) AS sum_sales#39] (40) Exchange -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#40] (41) Sort [codegen id : 22] -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST], false, 0 (42) Window -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] (43) Project [codegen id : 23] -Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38, rn#40] +Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] (44) Exchange -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1), 5), ENSURE_REQUIREMENTS, [id=#41] +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1), 5), ENSURE_REQUIREMENTS, [id=#42] (45) Sort [codegen id : 24] -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#40 + 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#41 + 1) ASC NULLS FIRST], false, 0 (46) SortMergeJoin [codegen id : 25] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1)] +Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1)] Join condition: None (47) Project [codegen id : 25] -Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38] -Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] +Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39] +Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] (48) ReusedExchange [Reuses operator id: 40] -Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Output [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] (49) Sort [codegen id : 34] -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] +Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST], false, 0 (50) Window -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] +Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] +Arguments: [rank(d_year#47, d_moy#48) windowspecdefinition(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#50], [i_category#43, i_brand#44, s_store_name#45, s_company_name#46], [d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST] (51) Project [codegen id : 35] -Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] +Output [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] (52) Exchange -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1), 5), ENSURE_REQUIREMENTS, [id=#50] +Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Arguments: hashpartitioning(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1), 5), ENSURE_REQUIREMENTS, [id=#51] (53) Sort [codegen id : 36] -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (rn#49 - 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, (rn#50 - 1) ASC NULLS FIRST], false, 0 (54) SortMergeJoin [codegen id : 37] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1)] +Right keys [5]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1)] Join condition: None (55) Project [codegen id : 37] -Output [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#38 AS psum#51, sum_sales#48 AS nsum#52] -Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] +Output [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#39 AS psum#52, sum_sales#49 AS nsum#53] +Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39, i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] (56) TakeOrderedAndProject -Input [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] +Input [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] ===== Subqueries ===== 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 b0a231fc01379..734eec1be3565 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 @@ -202,70 +202,70 @@ Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_ye (34) HashAggregate [codegen id : 13] 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#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#35] -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#6))#35,17,2) AS sum_sales#36] +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] (35) 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#36] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#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#37] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] (36) Sort [codegen id : 14] -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36] +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: [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 (37) 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#36] -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#38], [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#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] (38) Project [codegen id : 15] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] -Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] +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] (39) BroadcastExchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] -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=#39] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +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=#40] (40) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#38 + 1)] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] Join condition: None (41) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] -Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] +Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] +Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] (42) ReusedExchange [Reuses operator id: 35] -Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] +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] (43) Sort [codegen id : 21] -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#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, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 +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 (44) Window -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] -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#47], [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] +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] (45) Project [codegen id : 22] -Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] -Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] +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] (46) BroadcastExchange -Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] -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=#48] +Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +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=#49] (47) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -Right keys [5]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#47 - 1)] +Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] Join condition: None (48) Project [codegen id : 23] -Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#49, sum_sales#46 AS nsum#50] -Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] +Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] +Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] (49) TakeOrderedAndProject -Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, 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#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#22, psum#49, nsum#50] +Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, 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#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#22, psum#50, nsum#51] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt index 57e57951851aa..a1c7e4bfb35dc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt @@ -250,225 +250,225 @@ Output [5]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_net_profit#47, cs_sold_date_sk#48] (36) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#49] (37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_sold_date_sk#48] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#49] Join condition: None (38) Project [codegen id : 12] Output [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#8] +Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#49] (39) Exchange Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#49] +Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#50] (40) Sort [codegen id : 13] Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] Arguments: [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST], false, 0 (41) Scan parquet default.catalog_returns -Output [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Output [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (42) ColumnarToRow [codegen id : 14] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] (43) Filter [codegen id : 14] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] -Condition : (((isnotnull(cr_return_amount#53) AND (cr_return_amount#53 > 10000.00)) AND isnotnull(cr_order_number#51)) AND isnotnull(cr_item_sk#50)) +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Condition : (((isnotnull(cr_return_amount#54) AND (cr_return_amount#54 > 10000.00)) AND isnotnull(cr_order_number#52)) AND isnotnull(cr_item_sk#51)) (44) Project [codegen id : 14] -Output [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Output [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] (45) Exchange -Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Arguments: hashpartitioning(cr_order_number#52, cr_item_sk#51, 5), ENSURE_REQUIREMENTS, [id=#56] (46) Sort [codegen id : 15] -Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Arguments: [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Arguments: [cr_order_number#52 ASC NULLS FIRST, cr_item_sk#51 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 16] Left keys [2]: [cs_order_number#44, cs_item_sk#43] -Right keys [2]: [cr_order_number#51, cr_item_sk#50] +Right keys [2]: [cr_order_number#52, cr_item_sk#51] Join condition: None (48) Project [codegen id : 16] -Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] -Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] +Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] (49) HashAggregate [codegen id : 16] -Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] +Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] Keys [1]: [cs_item_sk#43] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#52, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#56, sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] -Results [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#53, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] +Results [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] (50) Exchange -Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] -Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#69] (51) HashAggregate [codegen id : 17] -Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] Keys [1]: [cs_item_sk#43] -Functions [4]: [sum(coalesce(cr_return_quantity#52, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#52, 0))#69, sum(coalesce(cs_quantity#45, 0))#70, sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72] -Results [3]: [cs_item_sk#43 AS item#73, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#52, 0))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#70 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#74, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#75] +Functions [4]: [sum(coalesce(cr_return_quantity#53, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#53, 0))#70, sum(coalesce(cs_quantity#45, 0))#71, sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73] +Results [3]: [cs_item_sk#43 AS item#74, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#53, 0))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#71 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#75, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#76] (52) Exchange -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#77] (53) Sort [codegen id : 18] -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: [return_ratio#74 ASC NULLS FIRST], false, 0 +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: [return_ratio#75 ASC NULLS FIRST], false, 0 (54) Window -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: [rank(return_ratio#74) windowspecdefinition(return_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#77], [return_ratio#74 ASC NULLS FIRST] +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: [rank(return_ratio#75) windowspecdefinition(return_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#78], [return_ratio#75 ASC NULLS FIRST] (55) Sort [codegen id : 19] -Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] -Arguments: [currency_ratio#75 ASC NULLS FIRST], false, 0 +Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] +Arguments: [currency_ratio#76 ASC NULLS FIRST], false, 0 (56) Window -Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] -Arguments: [rank(currency_ratio#75) windowspecdefinition(currency_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#78], [currency_ratio#75 ASC NULLS FIRST] +Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] +Arguments: [rank(currency_ratio#76) windowspecdefinition(currency_ratio#76 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#79], [currency_ratio#76 ASC NULLS FIRST] (57) Filter [codegen id : 20] -Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] -Condition : ((return_rank#77 <= 10) OR (currency_rank#78 <= 10)) +Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] +Condition : ((return_rank#78 <= 10) OR (currency_rank#79 <= 10)) (58) Project [codegen id : 20] -Output [5]: [catalog AS channel#79, item#73, return_ratio#74, return_rank#77, currency_rank#78] -Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] +Output [5]: [catalog AS channel#80, item#74, return_ratio#75, return_rank#78, currency_rank#79] +Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] (59) Scan parquet default.store_sales -Output [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Output [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_sold_date_sk#85 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 22] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] (61) Filter [codegen id : 22] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] -Condition : (((((((isnotnull(ss_net_profit#84) AND isnotnull(ss_net_paid#83)) AND isnotnull(ss_quantity#82)) AND (ss_net_profit#84 > 1.00)) AND (ss_net_paid#83 > 0.00)) AND (ss_quantity#82 > 0)) AND isnotnull(ss_ticket_number#81)) AND isnotnull(ss_item_sk#80)) +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Condition : (((((((isnotnull(ss_net_profit#85) AND isnotnull(ss_net_paid#84)) AND isnotnull(ss_quantity#83)) AND (ss_net_profit#85 > 1.00)) AND (ss_net_paid#84 > 0.00)) AND (ss_quantity#83 > 0)) AND isnotnull(ss_ticket_number#82)) AND isnotnull(ss_item_sk#81)) (62) Project [codegen id : 22] -Output [5]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Output [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] (63) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#87] (64) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ss_sold_date_sk#85] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ss_sold_date_sk#86] +Right keys [1]: [d_date_sk#87] Join condition: None (65) Project [codegen id : 22] -Output [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85, d_date_sk#8] +Output [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86, d_date_sk#87] (66) Exchange -Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Arguments: hashpartitioning(cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint), 5), ENSURE_REQUIREMENTS, [id=#86] +Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Arguments: hashpartitioning(cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint), 5), ENSURE_REQUIREMENTS, [id=#88] (67) Sort [codegen id : 23] -Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Arguments: [cast(ss_ticket_number#81 as bigint) ASC NULLS FIRST, cast(ss_item_sk#80 as bigint) ASC NULLS FIRST], false, 0 +Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Arguments: [cast(ss_ticket_number#82 as bigint) ASC NULLS FIRST, cast(ss_item_sk#81 as bigint) ASC NULLS FIRST], false, 0 (68) Scan parquet default.store_returns -Output [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Output [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 24] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] (70) Filter [codegen id : 24] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] -Condition : (((isnotnull(sr_return_amt#90) AND (sr_return_amt#90 > 10000.00)) AND isnotnull(sr_ticket_number#88)) AND isnotnull(sr_item_sk#87)) +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Condition : (((isnotnull(sr_return_amt#92) AND (sr_return_amt#92 > 10000.00)) AND isnotnull(sr_ticket_number#90)) AND isnotnull(sr_item_sk#89)) (71) Project [codegen id : 24] -Output [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Output [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] (72) Exchange -Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Arguments: hashpartitioning(sr_ticket_number#88, sr_item_sk#87, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Arguments: hashpartitioning(sr_ticket_number#90, sr_item_sk#89, 5), ENSURE_REQUIREMENTS, [id=#94] (73) Sort [codegen id : 25] -Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Arguments: [sr_ticket_number#88 ASC NULLS FIRST, sr_item_sk#87 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Arguments: [sr_ticket_number#90 ASC NULLS FIRST, sr_item_sk#89 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 26] -Left keys [2]: [cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint)] -Right keys [2]: [sr_ticket_number#88, sr_item_sk#87] +Left keys [2]: [cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint)] +Right keys [2]: [sr_ticket_number#90, sr_item_sk#89] Join condition: None (75) Project [codegen id : 26] -Output [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] -Input [8]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Output [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] +Input [8]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] (76) HashAggregate [codegen id : 26] -Input [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] -Keys [1]: [ss_item_sk#80] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#89, 0)), partial_sum(coalesce(ss_quantity#82, 0)), partial_sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#93, sum#94, sum#95, isEmpty#96, sum#97, isEmpty#98] -Results [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Input [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] +Keys [1]: [ss_item_sk#81] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#91, 0)), partial_sum(coalesce(ss_quantity#83, 0)), partial_sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#95, sum#96, sum#97, isEmpty#98, sum#99, isEmpty#100] +Results [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] (77) Exchange -Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Arguments: hashpartitioning(ss_item_sk#80, 5), ENSURE_REQUIREMENTS, [id=#105] +Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] +Arguments: hashpartitioning(ss_item_sk#81, 5), ENSURE_REQUIREMENTS, [id=#107] (78) HashAggregate [codegen id : 27] -Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Keys [1]: [ss_item_sk#80] -Functions [4]: [sum(coalesce(sr_return_quantity#89, 0)), sum(coalesce(ss_quantity#82, 0)), sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#89, 0))#106, sum(coalesce(ss_quantity#82, 0))#107, sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108, sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109] -Results [3]: [ss_item_sk#80 AS item#110, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#89, 0))#106 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#82, 0))#107 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#111, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#112] +Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] +Keys [1]: [ss_item_sk#81] +Functions [4]: [sum(coalesce(sr_return_quantity#91, 0)), sum(coalesce(ss_quantity#83, 0)), sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#91, 0))#108, sum(coalesce(ss_quantity#83, 0))#109, sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110, sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111] +Results [3]: [ss_item_sk#81 AS item#112, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#91, 0))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#83, 0))#109 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#113, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#114] (79) Exchange -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] (80) Sort [codegen id : 28] -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: [return_ratio#111 ASC NULLS FIRST], false, 0 +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: [return_ratio#113 ASC NULLS FIRST], false, 0 (81) Window -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: [rank(return_ratio#111) windowspecdefinition(return_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#114], [return_ratio#111 ASC NULLS FIRST] +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: [rank(return_ratio#113) windowspecdefinition(return_ratio#113 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#116], [return_ratio#113 ASC NULLS FIRST] (82) Sort [codegen id : 29] -Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] -Arguments: [currency_ratio#112 ASC NULLS FIRST], false, 0 +Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] +Arguments: [currency_ratio#114 ASC NULLS FIRST], false, 0 (83) Window -Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] -Arguments: [rank(currency_ratio#112) windowspecdefinition(currency_ratio#112 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#115], [currency_ratio#112 ASC NULLS FIRST] +Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] +Arguments: [rank(currency_ratio#114) windowspecdefinition(currency_ratio#114 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#117], [currency_ratio#114 ASC NULLS FIRST] (84) Filter [codegen id : 30] -Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] -Condition : ((return_rank#114 <= 10) OR (currency_rank#115 <= 10)) +Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] +Condition : ((return_rank#116 <= 10) OR (currency_rank#117 <= 10)) (85) Project [codegen id : 30] -Output [5]: [store AS channel#116, item#110, return_ratio#111, return_rank#114, currency_rank#115] -Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] +Output [5]: [store AS channel#118, item#112, return_ratio#113, return_rank#116, currency_rank#117] +Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] (86) Union @@ -481,7 +481,7 @@ Results [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_ran (88) Exchange Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] -Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#117] +Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#119] (89) HashAggregate [codegen id : 32] Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] @@ -505,6 +505,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#85 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index ada8c0415c672..74b789e4a7a65 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -260,170 +260,170 @@ Output [6]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, Input [9]: [cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_item_sk#49, cr_order_number#50, cr_return_quantity#51, cr_return_amount#52] (40) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#54] (41) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#47] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#54] Join condition: None (42) Project [codegen id : 10] Output [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] -Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#14] +Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#54] (43) HashAggregate [codegen id : 10] Input [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] Keys [1]: [cs_item_sk#42] Functions [4]: [partial_sum(coalesce(cr_return_quantity#51, 0)), partial_sum(coalesce(cs_quantity#44, 0)), partial_sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Results [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] +Aggregate Attributes [6]: [sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Results [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] (44) Exchange -Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] -Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#67] (45) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] +Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] Keys [1]: [cs_item_sk#42] Functions [4]: [sum(coalesce(cr_return_quantity#51, 0)), sum(coalesce(cs_quantity#44, 0)), sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#67, sum(coalesce(cs_quantity#44, 0))#68, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70] -Results [3]: [cs_item_sk#42 AS item#71, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#67 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#68 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#72, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#73] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#68, sum(coalesce(cs_quantity#44, 0))#69, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71] +Results [3]: [cs_item_sk#42 AS item#72, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#68 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#69 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#73, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#74] (46) Exchange -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#74] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#75] (47) Sort [codegen id : 12] -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: [return_ratio#72 ASC NULLS FIRST], false, 0 +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [return_ratio#73 ASC NULLS FIRST], false, 0 (48) Window -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: [rank(return_ratio#72) windowspecdefinition(return_ratio#72 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#72 ASC NULLS FIRST] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#76], [return_ratio#73 ASC NULLS FIRST] (49) Sort [codegen id : 13] -Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] -Arguments: [currency_ratio#73 ASC NULLS FIRST], false, 0 +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 (50) Window -Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] -Arguments: [rank(currency_ratio#73) windowspecdefinition(currency_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#73 ASC NULLS FIRST] +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#77], [currency_ratio#74 ASC NULLS FIRST] (51) Filter [codegen id : 14] -Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] +Condition : ((return_rank#76 <= 10) OR (currency_rank#77 <= 10)) (52) Project [codegen id : 14] -Output [5]: [catalog AS channel#77, item#71, return_ratio#72, return_rank#75, currency_rank#76] -Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] +Output [5]: [catalog AS channel#78, item#72, return_ratio#73, return_rank#76, currency_rank#77] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] (53) Scan parquet default.store_sales -Output [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Output [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#83), dynamicpruningexpression(ss_sold_date_sk#83 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#84), dynamicpruningexpression(ss_sold_date_sk#84 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 15] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] (55) Filter [codegen id : 15] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] -Condition : (((((((isnotnull(ss_net_profit#82) AND isnotnull(ss_net_paid#81)) AND isnotnull(ss_quantity#80)) AND (ss_net_profit#82 > 1.00)) AND (ss_net_paid#81 > 0.00)) AND (ss_quantity#80 > 0)) AND isnotnull(ss_ticket_number#79)) AND isnotnull(ss_item_sk#78)) +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Condition : (((((((isnotnull(ss_net_profit#83) AND isnotnull(ss_net_paid#82)) AND isnotnull(ss_quantity#81)) AND (ss_net_profit#83 > 1.00)) AND (ss_net_paid#82 > 0.00)) AND (ss_quantity#81 > 0)) AND isnotnull(ss_ticket_number#80)) AND isnotnull(ss_item_sk#79)) (56) Project [codegen id : 15] -Output [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Output [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] (57) BroadcastExchange -Input [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#84] +Input [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#85] (58) Scan parquet default.store_returns -Output [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Output [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (59) ColumnarToRow -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] (60) Filter -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] -Condition : (((isnotnull(sr_return_amt#88) AND (sr_return_amt#88 > 10000.00)) AND isnotnull(sr_ticket_number#86)) AND isnotnull(sr_item_sk#85)) +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Condition : (((isnotnull(sr_return_amt#89) AND (sr_return_amt#89 > 10000.00)) AND isnotnull(sr_ticket_number#87)) AND isnotnull(sr_item_sk#86)) (61) Project -Output [4]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Output [4]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] (62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [cast(ss_ticket_number#79 as bigint), cast(ss_item_sk#78 as bigint)] -Right keys [2]: [sr_ticket_number#86, sr_item_sk#85] +Left keys [2]: [cast(ss_ticket_number#80 as bigint), cast(ss_item_sk#79 as bigint)] +Right keys [2]: [sr_ticket_number#87, sr_item_sk#86] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88] -Input [9]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] +Output [6]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89] +Input [9]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] (64) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#91] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#83] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ss_sold_date_sk#84] +Right keys [1]: [d_date_sk#91] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] -Input [7]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88, d_date_sk#14] +Output [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] +Input [7]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89, d_date_sk#91] (67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] -Keys [1]: [ss_item_sk#78] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#87, 0)), partial_sum(coalesce(ss_quantity#80, 0)), partial_sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Results [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Input [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] +Keys [1]: [ss_item_sk#79] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#88, 0)), partial_sum(coalesce(ss_quantity#81, 0)), partial_sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] +Results [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] (68) Exchange -Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Arguments: hashpartitioning(ss_item_sk#78, 5), ENSURE_REQUIREMENTS, [id=#102] +Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Arguments: hashpartitioning(ss_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#104] (69) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Keys [1]: [ss_item_sk#78] -Functions [4]: [sum(coalesce(sr_return_quantity#87, 0)), sum(coalesce(ss_quantity#80, 0)), sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#87, 0))#103, sum(coalesce(ss_quantity#80, 0))#104, sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105, sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106] -Results [3]: [ss_item_sk#78 AS item#107, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#87, 0))#103 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#80, 0))#104 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#108, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#109] +Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Keys [1]: [ss_item_sk#79] +Functions [4]: [sum(coalesce(sr_return_quantity#88, 0)), sum(coalesce(ss_quantity#81, 0)), sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#88, 0))#105, sum(coalesce(ss_quantity#81, 0))#106, sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107, sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108] +Results [3]: [ss_item_sk#79 AS item#109, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#88, 0))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#81, 0))#106 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#110, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#111] (70) Exchange -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#112] (71) Sort [codegen id : 19] -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: [return_ratio#108 ASC NULLS FIRST], false, 0 +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: [return_ratio#110 ASC NULLS FIRST], false, 0 (72) Window -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: [rank(return_ratio#108) windowspecdefinition(return_ratio#108 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#111], [return_ratio#108 ASC NULLS FIRST] +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: [rank(return_ratio#110) windowspecdefinition(return_ratio#110 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#113], [return_ratio#110 ASC NULLS FIRST] (73) Sort [codegen id : 20] -Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] -Arguments: [currency_ratio#109 ASC NULLS FIRST], false, 0 +Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] +Arguments: [currency_ratio#111 ASC NULLS FIRST], false, 0 (74) Window -Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] -Arguments: [rank(currency_ratio#109) windowspecdefinition(currency_ratio#109 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#112], [currency_ratio#109 ASC NULLS FIRST] +Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] +Arguments: [rank(currency_ratio#111) windowspecdefinition(currency_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#114], [currency_ratio#111 ASC NULLS FIRST] (75) Filter [codegen id : 21] -Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] -Condition : ((return_rank#111 <= 10) OR (currency_rank#112 <= 10)) +Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] +Condition : ((return_rank#113 <= 10) OR (currency_rank#114 <= 10)) (76) Project [codegen id : 21] -Output [5]: [store AS channel#113, item#107, return_ratio#108, return_rank#111, currency_rank#112] -Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] +Output [5]: [store AS channel#115, item#109, return_ratio#110, return_rank#113, currency_rank#114] +Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] (77) Union @@ -436,7 +436,7 @@ Results [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_ran (79) Exchange Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] -Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#114] +Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#116] (80) HashAggregate [codegen id : 23] Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] @@ -460,6 +460,6 @@ Output [1]: [d_date_sk#14] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#83 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#84 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt index bcf47f68f2e59..0a8fdb66edb77 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 @@ -164,252 +164,252 @@ Input [4]: [item_sk#13, d_date#6, sumws#14, rk#16] Arguments: [item_sk#13 ASC NULLS FIRST], false, 0 (20) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] +Output [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] (21) Sort [codegen id : 10] -Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] -Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] +Arguments: [ws_item_sk#21 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST], false, 0 (22) Window -Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#21], [ws_item_sk#1], [d_date#19 ASC NULLS FIRST] +Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] +Arguments: [row_number() windowspecdefinition(ws_item_sk#21, d_date#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#22], [ws_item_sk#21], [d_date#19 ASC NULLS FIRST] (23) Project [codegen id : 11] -Output [3]: [item_sk#18, sumws#20, rk#21] -Input [5]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1, rk#21] +Output [3]: [item_sk#18, sumws#20, rk#22] +Input [5]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21, rk#22] (24) Exchange -Input [3]: [item_sk#18, sumws#20, rk#21] -Arguments: hashpartitioning(item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [3]: [item_sk#18, sumws#20, rk#22] +Arguments: hashpartitioning(item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#23] (25) Sort [codegen id : 12] -Input [3]: [item_sk#18, sumws#20, rk#21] +Input [3]: [item_sk#18, sumws#20, rk#22] Arguments: [item_sk#18 ASC NULLS FIRST], false, 0 (26) SortMergeJoin [codegen id : 13] Left keys [1]: [item_sk#13] Right keys [1]: [item_sk#18] -Join condition: (rk#16 >= rk#21) +Join condition: (rk#16 >= rk#22) (27) Project [codegen id : 13] Output [4]: [item_sk#13, d_date#6, sumws#14, sumws#20] -Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#18, sumws#20, rk#21] +Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#18, sumws#20, rk#22] (28) HashAggregate [codegen id : 13] Input [4]: [item_sk#13, d_date#6, sumws#14, sumws#20] Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [partial_sum(sumws#20)] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +Aggregate Attributes [2]: [sum#24, isEmpty#25] +Results [5]: [item_sk#13, d_date#6, sumws#14, sum#26, isEmpty#27] (29) HashAggregate [codegen id : 13] -Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +Input [5]: [item_sk#13, d_date#6, sumws#14, sum#26, isEmpty#27] Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [sum(sumws#20)] -Aggregate Attributes [1]: [sum(sumws#20)#27] -Results [3]: [item_sk#13, d_date#6, sum(sumws#20)#27 AS cume_sales#28] +Aggregate Attributes [1]: [sum(sumws#20)#28] +Results [3]: [item_sk#13, d_date#6, sum(sumws#20)#28 AS cume_sales#29] (30) Exchange -Input [3]: [item_sk#13, d_date#6, cume_sales#28] -Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [item_sk#13, d_date#6, cume_sales#29] +Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#30] (31) Sort [codegen id : 14] -Input [3]: [item_sk#13, d_date#6, cume_sales#28] +Input [3]: [item_sk#13, d_date#6, cume_sales#29] Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 (32) Scan parquet default.store_sales -Output [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +Output [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#33), dynamicpruningexpression(ss_sold_date_sk#33 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (33) ColumnarToRow [codegen id : 16] -Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] (34) Filter [codegen id : 16] -Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] -Condition : isnotnull(ss_item_sk#30) +Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] +Condition : isnotnull(ss_item_sk#31) (35) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#33, d_date#34] +Output [2]: [d_date_sk#34, d_date#35] (36) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ss_sold_date_sk#32] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [ss_sold_date_sk#33] +Right keys [1]: [d_date_sk#34] Join condition: None (37) Project [codegen id : 16] -Output [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] -Input [5]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32, d_date_sk#33, d_date#34] +Output [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] +Input [5]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33, d_date_sk#34, d_date#35] (38) HashAggregate [codegen id : 16] -Input [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] -Keys [2]: [ss_item_sk#30, d_date#34] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum#35] -Results [3]: [ss_item_sk#30, d_date#34, sum#36] +Input [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] +Keys [2]: [ss_item_sk#31, d_date#35] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#32))] +Aggregate Attributes [1]: [sum#36] +Results [3]: [ss_item_sk#31, d_date#35, sum#37] (39) Exchange -Input [3]: [ss_item_sk#30, d_date#34, sum#36] -Arguments: hashpartitioning(ss_item_sk#30, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [ss_item_sk#31, d_date#35, sum#37] +Arguments: hashpartitioning(ss_item_sk#31, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#38] (40) HashAggregate [codegen id : 17] -Input [3]: [ss_item_sk#30, d_date#34, sum#36] -Keys [2]: [ss_item_sk#30, d_date#34] -Functions [1]: [sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#31))#38] -Results [4]: [ss_item_sk#30 AS item_sk#39, d_date#34, MakeDecimal(sum(UnscaledValue(ss_sales_price#31))#38,17,2) AS sumss#40, ss_item_sk#30] +Input [3]: [ss_item_sk#31, d_date#35, sum#37] +Keys [2]: [ss_item_sk#31, d_date#35] +Functions [1]: [sum(UnscaledValue(ss_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#32))#39] +Results [4]: [ss_item_sk#31 AS item_sk#40, d_date#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#32))#39,17,2) AS sumss#41, ss_item_sk#31] (41) Exchange -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: hashpartitioning(ss_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#42] (42) Sort [codegen id : 18] -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 (43) Window -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#42], [ss_item_sk#30], [d_date#34 ASC NULLS FIRST] +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: [row_number() windowspecdefinition(ss_item_sk#31, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [ss_item_sk#31], [d_date#35 ASC NULLS FIRST] (44) Project [codegen id : 19] -Output [4]: [item_sk#39, d_date#34, sumss#40, rk#42] -Input [5]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30, rk#42] +Output [4]: [item_sk#40, d_date#35, sumss#41, rk#43] +Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] (45) Exchange -Input [4]: [item_sk#39, d_date#34, sumss#40, rk#42] -Arguments: hashpartitioning(item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [4]: [item_sk#40, d_date#35, sumss#41, rk#43] +Arguments: hashpartitioning(item_sk#40, 5), ENSURE_REQUIREMENTS, [id=#44] (46) Sort [codegen id : 20] -Input [4]: [item_sk#39, d_date#34, sumss#40, rk#42] -Arguments: [item_sk#39 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#40, d_date#35, sumss#41, rk#43] +Arguments: [item_sk#40 ASC NULLS FIRST], false, 0 (47) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30] +Output [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] (48) Sort [codegen id : 24] -Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30] -Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] +Arguments: [ss_item_sk#48 ASC NULLS FIRST, d_date#46 ASC NULLS FIRST], false, 0 (49) Window -Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30] -Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#47], [ss_item_sk#30], [d_date#45 ASC NULLS FIRST] +Input [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] +Arguments: [row_number() windowspecdefinition(ss_item_sk#48, d_date#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [ss_item_sk#48], [d_date#46 ASC NULLS FIRST] (50) Project [codegen id : 25] -Output [3]: [item_sk#44, sumss#46, rk#47] -Input [5]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30, rk#47] +Output [3]: [item_sk#45, sumss#47, rk#49] +Input [5]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48, rk#49] (51) Exchange -Input [3]: [item_sk#44, sumss#46, rk#47] -Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [3]: [item_sk#45, sumss#47, rk#49] +Arguments: hashpartitioning(item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#50] (52) Sort [codegen id : 26] -Input [3]: [item_sk#44, sumss#46, rk#47] -Arguments: [item_sk#44 ASC NULLS FIRST], false, 0 +Input [3]: [item_sk#45, sumss#47, rk#49] +Arguments: [item_sk#45 ASC NULLS FIRST], false, 0 (53) SortMergeJoin [codegen id : 27] -Left keys [1]: [item_sk#39] -Right keys [1]: [item_sk#44] -Join condition: (rk#42 >= rk#47) +Left keys [1]: [item_sk#40] +Right keys [1]: [item_sk#45] +Join condition: (rk#43 >= rk#49) (54) Project [codegen id : 27] -Output [4]: [item_sk#39, d_date#34, sumss#40, sumss#46] -Input [7]: [item_sk#39, d_date#34, sumss#40, rk#42, item_sk#44, sumss#46, rk#47] +Output [4]: [item_sk#40, d_date#35, sumss#41, sumss#47] +Input [7]: [item_sk#40, d_date#35, sumss#41, rk#43, item_sk#45, sumss#47, rk#49] (55) HashAggregate [codegen id : 27] -Input [4]: [item_sk#39, d_date#34, sumss#40, sumss#46] -Keys [3]: [item_sk#39, d_date#34, sumss#40] -Functions [1]: [partial_sum(sumss#46)] -Aggregate Attributes [2]: [sum#49, isEmpty#50] -Results [5]: [item_sk#39, d_date#34, sumss#40, sum#51, isEmpty#52] +Input [4]: [item_sk#40, d_date#35, sumss#41, sumss#47] +Keys [3]: [item_sk#40, d_date#35, sumss#41] +Functions [1]: [partial_sum(sumss#47)] +Aggregate Attributes [2]: [sum#51, isEmpty#52] +Results [5]: [item_sk#40, d_date#35, sumss#41, sum#53, isEmpty#54] (56) HashAggregate [codegen id : 27] -Input [5]: [item_sk#39, d_date#34, sumss#40, sum#51, isEmpty#52] -Keys [3]: [item_sk#39, d_date#34, sumss#40] -Functions [1]: [sum(sumss#46)] -Aggregate Attributes [1]: [sum(sumss#46)#53] -Results [3]: [item_sk#39, d_date#34, sum(sumss#46)#53 AS cume_sales#54] +Input [5]: [item_sk#40, d_date#35, sumss#41, sum#53, isEmpty#54] +Keys [3]: [item_sk#40, d_date#35, sumss#41] +Functions [1]: [sum(sumss#47)] +Aggregate Attributes [1]: [sum(sumss#47)#55] +Results [3]: [item_sk#40, d_date#35, sum(sumss#47)#55 AS cume_sales#56] (57) Exchange -Input [3]: [item_sk#39, d_date#34, cume_sales#54] -Arguments: hashpartitioning(item_sk#39, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [3]: [item_sk#40, d_date#35, cume_sales#56] +Arguments: hashpartitioning(item_sk#40, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#57] (58) Sort [codegen id : 28] -Input [3]: [item_sk#39, d_date#34, cume_sales#54] -Arguments: [item_sk#39 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 +Input [3]: [item_sk#40, d_date#35, cume_sales#56] +Arguments: [item_sk#40 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 (59) SortMergeJoin Left keys [2]: [item_sk#13, d_date#6] -Right keys [2]: [item_sk#39, d_date#34] +Right keys [2]: [item_sk#40, d_date#35] Join condition: None (60) Filter [codegen id : 29] -Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END) +Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END) (61) Project [codegen id : 29] -Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END AS item_sk#56, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#34 END AS d_date#57, cume_sales#28 AS web_sales#58, cume_sales#54 AS store_sales#59] -Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] +Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END AS item_sk#58, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#35 END AS d_date#59, cume_sales#29 AS web_sales#60, cume_sales#56 AS store_sales#61] +Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] (62) Exchange -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: hashpartitioning(item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: hashpartitioning(item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#62] (63) Sort [codegen id : 30] -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], false, 0 (64) Window -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: [row_number() windowspecdefinition(item_sk#56, d_date#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#61], [item_sk#56], [d_date#57 ASC NULLS FIRST] +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: [row_number() windowspecdefinition(item_sk#58, d_date#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#63], [item_sk#58], [d_date#59 ASC NULLS FIRST] (65) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#62, d_date#63, web_sales#64, store_sales#65] +Output [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] (66) 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#64, d_date#65, web_sales#66, store_sales#67] +Arguments: [item_sk#64 ASC NULLS FIRST, d_date#65 ASC NULLS FIRST], false, 0 (67) 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#64, d_date#65, web_sales#66, store_sales#67] +Arguments: [row_number() windowspecdefinition(item_sk#64, d_date#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#68], [item_sk#64], [d_date#65 ASC NULLS FIRST] (68) 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#64, web_sales#66, store_sales#67, rk#68] +Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] (69) SortMergeJoin [codegen id : 62] -Left keys [1]: [item_sk#56] -Right keys [1]: [item_sk#62] -Join condition: (rk#61 >= rk#66) +Left keys [1]: [item_sk#58] +Right keys [1]: [item_sk#64] +Join condition: (rk#63 >= rk#68) (70) 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] +Output [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] +Input [9]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63, item_sk#64, web_sales#66, store_sales#67, rk#68] (71) 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] +Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] +Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Functions [2]: [partial_max(web_sales#66), partial_max(store_sales#67)] +Aggregate Attributes [2]: [max#69, max#70] +Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#71, max#72] (72) HashAggregate [codegen id : 62] -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)#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] +Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#71, max#72] +Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Functions [2]: [max(web_sales#66), max(store_sales#67)] +Aggregate Attributes [2]: [max(web_sales#66)#73, max(store_sales#67)#74] +Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max(web_sales#66)#73 AS web_cumulative#75, max(store_sales#67)#74 AS store_cumulative#76] (73) 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#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] +Condition : ((isnotnull(web_cumulative#75) AND isnotnull(store_cumulative#76)) AND (web_cumulative#75 > store_cumulative#76)) (74) 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#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] +Arguments: 100, [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] ===== Subqueries ===== @@ -420,6 +420,6 @@ ReusedExchange (75) (75) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#5, d_date#6] -Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#33 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 8aaaf48a8162c..b7d3461ce34ea 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 @@ -153,248 +153,248 @@ Output [4]: [item_sk#13, d_date#6, sumws#14, rk#16] Input [5]: [item_sk#13, d_date#6, sumws#14, ws_item_sk#1, rk#16] (18) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] +Output [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] (19) Sort [codegen id : 8] -Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] -Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] +Arguments: [ws_item_sk#20 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 (20) Window -Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#20], [ws_item_sk#1], [d_date#18 ASC NULLS FIRST] +Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] +Arguments: [row_number() windowspecdefinition(ws_item_sk#20, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#21], [ws_item_sk#20], [d_date#18 ASC NULLS FIRST] (21) Project [codegen id : 9] -Output [3]: [item_sk#17, sumws#19, rk#20] -Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1, rk#20] +Output [3]: [item_sk#17, sumws#19, rk#21] +Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20, rk#21] (22) BroadcastExchange -Input [3]: [item_sk#17, sumws#19, rk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [3]: [item_sk#17, sumws#19, rk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (23) BroadcastHashJoin [codegen id : 10] Left keys [1]: [item_sk#13] Right keys [1]: [item_sk#17] -Join condition: (rk#16 >= rk#20) +Join condition: (rk#16 >= rk#21) (24) Project [codegen id : 10] Output [4]: [item_sk#13, d_date#6, sumws#14, sumws#19] -Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#17, sumws#19, rk#20] +Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#17, sumws#19, rk#21] (25) HashAggregate [codegen id : 10] Input [4]: [item_sk#13, d_date#6, sumws#14, sumws#19] Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [partial_sum(sumws#19)] -Aggregate Attributes [2]: [sum#22, isEmpty#23] -Results [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] +Aggregate Attributes [2]: [sum#23, isEmpty#24] +Results [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] (26) Exchange -Input [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] -Arguments: hashpartitioning(item_sk#13, d_date#6, sumws#14, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +Arguments: hashpartitioning(item_sk#13, d_date#6, sumws#14, 5), ENSURE_REQUIREMENTS, [id=#27] (27) HashAggregate [codegen id : 11] -Input [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] +Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [sum(sumws#19)] -Aggregate Attributes [1]: [sum(sumws#19)#27] -Results [3]: [item_sk#13, d_date#6, sum(sumws#19)#27 AS cume_sales#28] +Aggregate Attributes [1]: [sum(sumws#19)#28] +Results [3]: [item_sk#13, d_date#6, sum(sumws#19)#28 AS cume_sales#29] (28) Exchange -Input [3]: [item_sk#13, d_date#6, cume_sales#28] -Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [item_sk#13, d_date#6, cume_sales#29] +Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#30] (29) Sort [codegen id : 12] -Input [3]: [item_sk#13, d_date#6, cume_sales#28] +Input [3]: [item_sk#13, d_date#6, cume_sales#29] Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 (30) Scan parquet default.store_sales -Output [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +Output [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#33), dynamicpruningexpression(ss_sold_date_sk#33 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 14] -Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] (32) Filter [codegen id : 14] -Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] -Condition : isnotnull(ss_item_sk#30) +Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] +Condition : isnotnull(ss_item_sk#31) (33) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#33, d_date#34] +Output [2]: [d_date_sk#34, d_date#35] (34) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_sold_date_sk#32] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [ss_sold_date_sk#33] +Right keys [1]: [d_date_sk#34] Join condition: None (35) Project [codegen id : 14] -Output [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] -Input [5]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32, d_date_sk#33, d_date#34] +Output [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] +Input [5]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33, d_date_sk#34, d_date#35] (36) HashAggregate [codegen id : 14] -Input [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] -Keys [2]: [ss_item_sk#30, d_date#34] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum#35] -Results [3]: [ss_item_sk#30, d_date#34, sum#36] +Input [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] +Keys [2]: [ss_item_sk#31, d_date#35] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#32))] +Aggregate Attributes [1]: [sum#36] +Results [3]: [ss_item_sk#31, d_date#35, sum#37] (37) Exchange -Input [3]: [ss_item_sk#30, d_date#34, sum#36] -Arguments: hashpartitioning(ss_item_sk#30, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [ss_item_sk#31, d_date#35, sum#37] +Arguments: hashpartitioning(ss_item_sk#31, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#38] (38) HashAggregate [codegen id : 15] -Input [3]: [ss_item_sk#30, d_date#34, sum#36] -Keys [2]: [ss_item_sk#30, d_date#34] -Functions [1]: [sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#31))#38] -Results [4]: [ss_item_sk#30 AS item_sk#39, d_date#34, MakeDecimal(sum(UnscaledValue(ss_sales_price#31))#38,17,2) AS sumss#40, ss_item_sk#30] +Input [3]: [ss_item_sk#31, d_date#35, sum#37] +Keys [2]: [ss_item_sk#31, d_date#35] +Functions [1]: [sum(UnscaledValue(ss_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#32))#39] +Results [4]: [ss_item_sk#31 AS item_sk#40, d_date#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#32))#39,17,2) AS sumss#41, ss_item_sk#31] (39) Exchange -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: hashpartitioning(ss_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#42] (40) Sort [codegen id : 16] -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 (41) Window -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#42], [ss_item_sk#30], [d_date#34 ASC NULLS FIRST] +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: [row_number() windowspecdefinition(ss_item_sk#31, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [ss_item_sk#31], [d_date#35 ASC NULLS FIRST] (42) Project [codegen id : 22] -Output [4]: [item_sk#39, d_date#34, sumss#40, rk#42] -Input [5]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30, rk#42] +Output [4]: [item_sk#40, d_date#35, sumss#41, rk#43] +Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] (43) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] +Output [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] (44) Sort [codegen id : 20] -Input [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] -Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#44 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], false, 0 (45) Window -Input [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] -Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#46], [ss_item_sk#30], [d_date#44 ASC NULLS FIRST] +Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] +Arguments: [row_number() windowspecdefinition(ss_item_sk#47, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [ss_item_sk#47], [d_date#45 ASC NULLS FIRST] (46) Project [codegen id : 21] -Output [3]: [item_sk#43, sumss#45, rk#46] -Input [5]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30, rk#46] +Output [3]: [item_sk#44, sumss#46, rk#48] +Input [5]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47, rk#48] (47) BroadcastExchange -Input [3]: [item_sk#43, sumss#45, rk#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] +Input [3]: [item_sk#44, sumss#46, rk#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] (48) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [item_sk#39] -Right keys [1]: [item_sk#43] -Join condition: (rk#42 >= rk#46) +Left keys [1]: [item_sk#40] +Right keys [1]: [item_sk#44] +Join condition: (rk#43 >= rk#48) (49) Project [codegen id : 22] -Output [4]: [item_sk#39, d_date#34, sumss#40, sumss#45] -Input [7]: [item_sk#39, d_date#34, sumss#40, rk#42, item_sk#43, sumss#45, rk#46] +Output [4]: [item_sk#40, d_date#35, sumss#41, sumss#46] +Input [7]: [item_sk#40, d_date#35, sumss#41, rk#43, item_sk#44, sumss#46, rk#48] (50) HashAggregate [codegen id : 22] -Input [4]: [item_sk#39, d_date#34, sumss#40, sumss#45] -Keys [3]: [item_sk#39, d_date#34, sumss#40] -Functions [1]: [partial_sum(sumss#45)] -Aggregate Attributes [2]: [sum#48, isEmpty#49] -Results [5]: [item_sk#39, d_date#34, sumss#40, sum#50, isEmpty#51] +Input [4]: [item_sk#40, d_date#35, sumss#41, sumss#46] +Keys [3]: [item_sk#40, d_date#35, sumss#41] +Functions [1]: [partial_sum(sumss#46)] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] (51) Exchange -Input [5]: [item_sk#39, d_date#34, sumss#40, sum#50, isEmpty#51] -Arguments: hashpartitioning(item_sk#39, d_date#34, sumss#40, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] +Arguments: hashpartitioning(item_sk#40, d_date#35, sumss#41, 5), ENSURE_REQUIREMENTS, [id=#54] (52) HashAggregate [codegen id : 23] -Input [5]: [item_sk#39, d_date#34, sumss#40, sum#50, isEmpty#51] -Keys [3]: [item_sk#39, d_date#34, sumss#40] -Functions [1]: [sum(sumss#45)] -Aggregate Attributes [1]: [sum(sumss#45)#53] -Results [3]: [item_sk#39, d_date#34, sum(sumss#45)#53 AS cume_sales#54] +Input [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] +Keys [3]: [item_sk#40, d_date#35, sumss#41] +Functions [1]: [sum(sumss#46)] +Aggregate Attributes [1]: [sum(sumss#46)#55] +Results [3]: [item_sk#40, d_date#35, sum(sumss#46)#55 AS cume_sales#56] (53) Exchange -Input [3]: [item_sk#39, d_date#34, cume_sales#54] -Arguments: hashpartitioning(item_sk#39, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [3]: [item_sk#40, d_date#35, cume_sales#56] +Arguments: hashpartitioning(item_sk#40, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#57] (54) Sort [codegen id : 24] -Input [3]: [item_sk#39, d_date#34, cume_sales#54] -Arguments: [item_sk#39 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 +Input [3]: [item_sk#40, d_date#35, cume_sales#56] +Arguments: [item_sk#40 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 (55) SortMergeJoin Left keys [2]: [item_sk#13, d_date#6] -Right keys [2]: [item_sk#39, d_date#34] +Right keys [2]: [item_sk#40, d_date#35] Join condition: None (56) Filter [codegen id : 25] -Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END) +Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END) (57) Project [codegen id : 25] -Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END AS item_sk#56, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#34 END AS d_date#57, cume_sales#28 AS web_sales#58, cume_sales#54 AS store_sales#59] -Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] +Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END AS item_sk#58, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#35 END AS d_date#59, cume_sales#29 AS web_sales#60, cume_sales#56 AS store_sales#61] +Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] (58) Exchange -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: hashpartitioning(item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: hashpartitioning(item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#62] (59) Sort [codegen id : 26] -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 +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], false, 0 (60) Window -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: [row_number() windowspecdefinition(item_sk#56, d_date#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#61], [item_sk#56], [d_date#57 ASC NULLS FIRST] +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: [row_number() windowspecdefinition(item_sk#58, d_date#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#63], [item_sk#58], [d_date#59 ASC NULLS FIRST] (61) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#62, d_date#63, web_sales#64, store_sales#65] +Output [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] (62) 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#64, d_date#65, web_sales#66, store_sales#67] +Arguments: [item_sk#64 ASC NULLS FIRST, d_date#65 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#64, d_date#65, web_sales#66, store_sales#67] +Arguments: [row_number() windowspecdefinition(item_sk#64, d_date#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#68], [item_sk#64], [d_date#65 ASC NULLS FIRST] (64) 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#64, web_sales#66, store_sales#67, rk#68] +Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] (65) 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#64, web_sales#66, store_sales#67, rk#68] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] (66) BroadcastHashJoin [codegen id : 54] -Left keys [1]: [item_sk#56] -Right keys [1]: [item_sk#62] -Join condition: (rk#61 >= rk#66) +Left keys [1]: [item_sk#58] +Right keys [1]: [item_sk#64] +Join condition: (rk#63 >= rk#68) (67) 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] +Output [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] +Input [9]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63, item_sk#64, web_sales#66, store_sales#67, rk#68] (68) 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] +Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] +Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Functions [2]: [partial_max(web_sales#66), partial_max(store_sales#67)] +Aggregate Attributes [2]: [max#70, max#71] +Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#72, max#73] (69) HashAggregate [codegen id : 54] -Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#70, max#71] -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] +Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#72, max#73] +Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Functions [2]: [max(web_sales#66), max(store_sales#67)] +Aggregate Attributes [2]: [max(web_sales#66)#74, max(store_sales#67)#75] +Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max(web_sales#66)#74 AS web_cumulative#76, max(store_sales#67)#75 AS store_cumulative#77] (70) 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#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] +Condition : ((isnotnull(web_cumulative#76) AND isnotnull(store_cumulative#77)) AND (web_cumulative#76 > store_cumulative#77)) (71) 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#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] +Arguments: 100, [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] ===== Subqueries ===== @@ -405,6 +405,6 @@ ReusedExchange (72) (72) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#5, d_date#6] -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#33 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 ed0399519f579..e3de4a7a053b1 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 @@ -229,78 +229,78 @@ Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] (39) HashAggregate [codegen id : 21] Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] Keys [5]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33] -Functions [1]: [sum(UnscaledValue(cs_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#3))#35] -Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#3))#35,17,2) AS sum_sales#36] +Functions [1]: [sum(UnscaledValue(cs_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#35))#36] +Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#35))#36,17,2) AS sum_sales#37] (40) Exchange -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] (41) Sort [codegen id : 22] -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 (42) Window -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] -Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] (43) Project [codegen id : 23] -Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] +Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] (44) Exchange -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#38 + 1), 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] (45) Sort [codegen id : 24] -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#38 + 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 (46) SortMergeJoin [codegen id : 25] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#38 + 1)] +Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#39 + 1)] Join condition: None (47) Project [codegen id : 25] -Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] -Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] +Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] +Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] (48) ReusedExchange [Reuses operator id: 40] -Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Output [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] (49) Sort [codegen id : 34] -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 (50) Window -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] +Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] +Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#41, i_brand#42, cc_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#41, i_brand#42, cc_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] (51) Project [codegen id : 35] -Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] +Output [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] (52) Exchange -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (rn#46 - 1), 5), ENSURE_REQUIREMENTS, [id=#47] +Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Arguments: hashpartitioning(i_category#41, i_brand#42, cc_name#43, (rn#47 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] (53) Sort [codegen id : 36] -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, (rn#47 - 1) ASC NULLS FIRST], false, 0 (54) SortMergeJoin [codegen id : 37] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (rn#46 - 1)] +Right keys [4]: [i_category#41, i_brand#42, cc_name#43, (rn#47 - 1)] Join condition: None (55) Project [codegen id : 37] -Output [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#48, sum_sales#45 AS nsum#49] -Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] +Output [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#49, sum_sales#46 AS nsum#50] +Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] (56) TakeOrderedAndProject -Input [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 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#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] +Input [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] ===== Subqueries ===== 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 c347bbbe3b4ad..a6742cf4ab1cf 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 @@ -202,70 +202,70 @@ Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] (34) HashAggregate [codegen id : 13] 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#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#33] -Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#33,17,2) AS sum_sales#34] +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] (35) Exchange -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#35] +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] (36) Sort [codegen id : 14] -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35] 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 (37) Window -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] -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#36], [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#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] (38) Project [codegen id : 15] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] -Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34, rn#36] +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] (39) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#37] +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#38] (40) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#36 + 1)] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] Join condition: None (41) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34] -Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] +Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35] +Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] (42) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] +Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] (43) Sort [codegen id : 21] -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] -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 +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 (44) Window -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] -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#44], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] +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] (45) Project [codegen id : 22] -Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] -Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43, rn#44] +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] (46) BroadcastExchange -Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#45] +Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#46] (47) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -Right keys [4]: [i_category#38, i_brand#39, cc_name#40, (rn#44 - 1)] +Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] Join condition: None (48) Project [codegen id : 23] -Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#34 AS psum#46, sum_sales#43 AS nsum#47] -Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34, i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] +Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] +Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] (49) TakeOrderedAndProject -Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, 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#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#21, psum#46, nsum#47] +Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, 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#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#21, psum#47, nsum#48] ===== Subqueries ===== 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 5374a708a0295..ef8088e415cc5 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 @@ -278,165 +278,165 @@ Output [6]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, Input [8]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] (41) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#70] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#25] +Right keys [1]: [d_date_sk#70] Join condition: None (43) Project [codegen id : 11] Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#25] +Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#70] (44) HashAggregate [codegen id : 11] Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] -Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] +Results [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] (45) Exchange -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] +Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#79] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] -Results [5]: [catalog channel AS channel#83, concat(catalog_page, cp_catalog_page_id#68) AS id#84, MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#85, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS returns#86, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#87] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] +Results [5]: [catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#68) AS id#85, MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS returns#87, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#88] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] -Condition : isnotnull(ws_web_site_sk#88) +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : isnotnull(ws_web_site_sk#89) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] (53) Exchange -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: hashpartitioning(wr_item_sk#98, wr_order_number#99, 5), ENSURE_REQUIREMENTS, [id=#103] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: hashpartitioning(wr_item_sk#99, wr_order_number#100, 5), ENSURE_REQUIREMENTS, [id=#104] (54) Sort [codegen id : 15] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: [wr_item_sk#98 ASC NULLS FIRST, wr_order_number#99 ASC NULLS FIRST], false, 0 +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: [wr_item_sk#99 ASC NULLS FIRST, wr_order_number#100 ASC NULLS FIRST], false, 0 (55) Scan parquet default.web_sales -Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 16] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (57) Filter [codegen id : 16] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) (58) Project [codegen id : 16] -Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (59) Exchange -Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Arguments: hashpartitioning(cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint), 5), ENSURE_REQUIREMENTS, [id=#106] +Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Arguments: hashpartitioning(cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint), 5), ENSURE_REQUIREMENTS, [id=#109] (60) Sort [codegen id : 17] -Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Arguments: [cast(ws_item_sk#104 as bigint) ASC NULLS FIRST, cast(ws_order_number#105 as bigint) ASC NULLS FIRST], false, 0 +Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Arguments: [cast(ws_item_sk#105 as bigint) ASC NULLS FIRST, cast(ws_order_number#107 as bigint) ASC NULLS FIRST], false, 0 (61) SortMergeJoin [codegen id : 18] -Left keys [2]: [wr_item_sk#98, wr_order_number#99] -Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] +Left keys [2]: [wr_item_sk#99, wr_order_number#100] +Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] Join condition: None (62) Project [codegen id : 18] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#107, wr_returned_date_sk#102 AS date_sk#108, 0.00 AS sales_price#109, 0.00 AS profit#110, wr_return_amt#100 AS return_amt#111, wr_net_loss#101 AS net_loss#112] -Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#110, wr_returned_date_sk#103 AS date_sk#111, 0.00 AS sales_price#112, 0.00 AS profit#113, wr_return_amt#101 AS return_amt#114, wr_net_loss#102 AS net_loss#115] +Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] (63) Union (64) Scan parquet default.web_site -Output [2]: [web_site_sk#113, web_site_id#114] +Output [2]: [web_site_sk#116, web_site_id#117] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 19] -Input [2]: [web_site_sk#113, web_site_id#114] +Input [2]: [web_site_sk#116, web_site_id#117] (66) Filter [codegen id : 19] -Input [2]: [web_site_sk#113, web_site_id#114] -Condition : isnotnull(web_site_sk#113) +Input [2]: [web_site_sk#116, web_site_id#117] +Condition : isnotnull(web_site_sk#116) (67) BroadcastExchange -Input [2]: [web_site_sk#113, web_site_id#114] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] +Input [2]: [web_site_sk#116, web_site_id#117] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] (68) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [wsr_web_site_sk#92] -Right keys [1]: [web_site_sk#113] +Left keys [1]: [wsr_web_site_sk#93] +Right keys [1]: [web_site_sk#116] Join condition: None (69) Project [codegen id : 21] -Output [6]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Input [8]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#113, web_site_id#114] +Output [6]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [8]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] (70) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#119] (71) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [date_sk#93] -Right keys [1]: [cast(d_date_sk#25 as bigint)] +Left keys [1]: [date_sk#94] +Right keys [1]: [cast(d_date_sk#119 as bigint)] Join condition: None (72) Project [codegen id : 21] -Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Input [7]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114, d_date_sk#25] +Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [7]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117, d_date_sk#119] (73) HashAggregate [codegen id : 21] -Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Keys [1]: [web_site_id#114] -Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] -Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] +Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Keys [1]: [web_site_id#117] +Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum#120, sum#121, sum#122, sum#123] +Results [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] (74) Exchange -Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] +Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#128] (75) HashAggregate [codegen id : 22] -Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -Keys [1]: [web_site_id#114] -Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#125, sum(UnscaledValue(return_amt#96))#126, sum(UnscaledValue(profit#95))#127, sum(UnscaledValue(net_loss#97))#128] -Results [5]: [web channel AS channel#129, concat(web_site, web_site_id#114) AS id#130, MakeDecimal(sum(UnscaledValue(sales_price#94))#125,17,2) AS sales#131, MakeDecimal(sum(UnscaledValue(return_amt#96))#126,17,2) AS returns#132, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#133] +Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] +Keys [1]: [web_site_id#117] +Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#129, sum(UnscaledValue(return_amt#97))#130, sum(UnscaledValue(profit#96))#131, sum(UnscaledValue(net_loss#98))#132] +Results [5]: [web channel AS channel#133, concat(web_site, web_site_id#117) AS id#134, MakeDecimal(sum(UnscaledValue(sales_price#95))#129,17,2) AS sales#135, MakeDecimal(sum(UnscaledValue(return_amt#97))#130,17,2) AS returns#136, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#131,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#132,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#137] (76) Union @@ -444,99 +444,99 @@ Results [5]: [web channel AS channel#129, concat(web_site, web_site_id#114) AS i Input [5]: [channel#41, id#42, sales#43, returns#44, profit#45] Keys [2]: [channel#41, id#42] Functions [3]: [partial_sum(sales#43), partial_sum(returns#44), partial_sum(profit#45)] -Aggregate Attributes [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Results [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Aggregate Attributes [6]: [sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Results [8]: [channel#41, id#42, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] (78) Exchange -Input [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#146] +Input [8]: [channel#41, id#42, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] +Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#150] (79) HashAggregate [codegen id : 24] -Input [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Input [8]: [channel#41, id#42, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#45)] -Aggregate Attributes [3]: [sum(sales#43)#147, sum(returns#44)#148, sum(profit#45)#149] -Results [5]: [channel#41, id#42, cast(sum(sales#43)#147 as decimal(37,2)) AS sales#150, cast(sum(returns#44)#148 as decimal(37,2)) AS returns#151, cast(sum(profit#45)#149 as decimal(38,2)) AS profit#152] +Aggregate Attributes [3]: [sum(sales#43)#151, sum(returns#44)#152, sum(profit#45)#153] +Results [5]: [channel#41, id#42, cast(sum(sales#43)#151 as decimal(37,2)) AS sales#154, cast(sum(returns#44)#152 as decimal(37,2)) AS returns#155, cast(sum(profit#45)#153 as decimal(38,2)) AS profit#156] (80) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Output [8]: [channel#41, id#42, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] (81) HashAggregate [codegen id : 48] -Input [8]: [channel#41, id#42, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Input [8]: [channel#41, id#42, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#159)] -Aggregate Attributes [3]: [sum(sales#43)#160, sum(returns#44)#161, sum(profit#159)#162] -Results [4]: [channel#41, sum(sales#43)#160 AS sales#163, sum(returns#44)#161 AS returns#164, sum(profit#159)#162 AS profit#165] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#163)] +Aggregate Attributes [3]: [sum(sales#43)#164, sum(returns#44)#165, sum(profit#163)#166] +Results [4]: [channel#41, sum(sales#43)#164 AS sales#167, sum(returns#44)#165 AS returns#168, sum(profit#163)#166 AS profit#169] (82) HashAggregate [codegen id : 48] -Input [4]: [channel#41, sales#163, returns#164, profit#165] +Input [4]: [channel#41, sales#167, returns#168, profit#169] Keys [1]: [channel#41] -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#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] +Functions [3]: [partial_sum(sales#167), partial_sum(returns#168), partial_sum(profit#169)] +Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Results [7]: [channel#41, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] (83) Exchange -Input [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] -Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#178] +Input [7]: [channel#41, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#182] (84) HashAggregate [codegen id : 49] -Input [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] +Input [7]: [channel#41, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Keys [1]: [channel#41] -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#41, 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#167), sum(returns#168), sum(profit#169)] +Aggregate Attributes [3]: [sum(sales#167)#183, sum(returns#168)#184, sum(profit#169)#185] +Results [5]: [channel#41, null AS id#186, sum(sales#167)#183 AS sum(sales)#187, sum(returns#168)#184 AS sum(returns)#188, sum(profit#169)#185 AS sum(profit)#189] (85) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] +Output [8]: [channel#41, id#42, sum#190, isEmpty#191, sum#192, isEmpty#193, sum#194, isEmpty#195] (86) HashAggregate [codegen id : 73] -Input [8]: [channel#41, id#42, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] +Input [8]: [channel#41, id#42, sum#190, isEmpty#191, sum#192, isEmpty#193, sum#194, isEmpty#195] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#192)] -Aggregate Attributes [3]: [sum(sales#43)#193, sum(returns#44)#194, sum(profit#192)#195] -Results [3]: [sum(sales#43)#193 AS sales#163, sum(returns#44)#194 AS returns#164, sum(profit#192)#195 AS profit#165] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#196)] +Aggregate Attributes [3]: [sum(sales#43)#197, sum(returns#44)#198, sum(profit#196)#199] +Results [3]: [sum(sales#43)#197 AS sales#167, sum(returns#44)#198 AS returns#168, sum(profit#196)#199 AS profit#169] (87) HashAggregate [codegen id : 73] -Input [3]: [sales#163, returns#164, profit#165] +Input [3]: [sales#167, returns#168, profit#169] Keys: [] -Functions [3]: [partial_sum(sales#163), partial_sum(returns#164), partial_sum(profit#165)] -Aggregate Attributes [6]: [sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201] -Results [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] +Functions [3]: [partial_sum(sales#167), partial_sum(returns#168), partial_sum(profit#169)] +Aggregate Attributes [6]: [sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205] +Results [6]: [sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211] (88) Exchange -Input [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#208] +Input [6]: [sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#212] (89) HashAggregate [codegen id : 74] -Input [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] +Input [6]: [sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211] Keys: [] -Functions [3]: [sum(sales#163), sum(returns#164), sum(profit#165)] -Aggregate Attributes [3]: [sum(sales#163)#209, sum(returns#164)#210, sum(profit#165)#211] -Results [5]: [null AS channel#212, null AS id#213, sum(sales#163)#209 AS sum(sales)#214, sum(returns#164)#210 AS sum(returns)#215, sum(profit#165)#211 AS sum(profit)#216] +Functions [3]: [sum(sales#167), sum(returns#168), sum(profit#169)] +Aggregate Attributes [3]: [sum(sales#167)#213, sum(returns#168)#214, sum(profit#169)#215] +Results [5]: [null AS channel#216, null AS id#217, sum(sales#167)#213 AS sum(sales)#218, sum(returns#168)#214 AS sum(returns)#219, sum(profit#169)#215 AS sum(profit)#220] (90) Union (91) HashAggregate [codegen id : 75] -Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -Keys [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Keys [5]: [channel#41, id#42, sales#154, returns#155, profit#156] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Results [5]: [channel#41, id#42, sales#154, returns#155, profit#156] (92) Exchange -Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -Arguments: hashpartitioning(channel#41, id#42, sales#150, returns#151, profit#152, 5), ENSURE_REQUIREMENTS, [id=#217] +Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Arguments: hashpartitioning(channel#41, id#42, sales#154, returns#155, profit#156, 5), ENSURE_REQUIREMENTS, [id=#221] (93) HashAggregate [codegen id : 76] -Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -Keys [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Keys [5]: [channel#41, id#42, sales#154, returns#155, profit#156] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Results [5]: [channel#41, id#42, sales#154, returns#155, profit#156] (94) TakeOrderedAndProject -Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#150, returns#151, profit#152] +Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#154, returns#155, profit#156] ===== Subqueries ===== @@ -554,12 +554,12 @@ ReusedExchange (96) (96) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#70] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 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 89362fb4f0efe..c6b64d59abf82 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 @@ -248,180 +248,180 @@ Input [4]: [cr_catalog_page_sk#57, cr_return_amount#58, cr_net_loss#59, cr_retur (34) Union (35) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#67] (36) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#67] Join condition: None (37) Project [codegen id : 11] Output [5]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56] -Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#22] +Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#67] (38) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Output [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] (40) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] -Condition : isnotnull(cp_catalog_page_sk#67) +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Condition : isnotnull(cp_catalog_page_sk#68) (41) BroadcastExchange -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#69] +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#70] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [page_sk#51] -Right keys [1]: [cp_catalog_page_sk#67] +Right keys [1]: [cp_catalog_page_sk#68] Join condition: None (43) Project [codegen id : 11] -Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] +Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] +Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#68, cp_catalog_page_id#69] (44) HashAggregate [codegen id : 11] -Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Keys [1]: [cp_catalog_page_id#68] +Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] +Keys [1]: [cp_catalog_page_id#69] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] -Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] +Results [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] (45) Exchange -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] +Arguments: hashpartitioning(cp_catalog_page_id#69, 5), ENSURE_REQUIREMENTS, [id=#79] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Keys [1]: [cp_catalog_page_id#68] +Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] +Keys [1]: [cp_catalog_page_id#69] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] -Results [5]: [catalog channel AS channel#83, concat(catalog_page, cp_catalog_page_id#68) AS id#84, MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#85, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS returns#86, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#87] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] +Results [5]: [catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#69) AS id#85, MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS returns#87, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#88] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] -Condition : isnotnull(ws_web_site_sk#88) +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : isnotnull(ws_web_site_sk#89) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] (53) BroadcastExchange -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#103] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#104] (54) Scan parquet default.web_sales -Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (55) ColumnarToRow -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (56) Filter -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) (57) Project -Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#98, wr_order_number#99] -Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] +Left keys [2]: [wr_item_sk#99, wr_order_number#100] +Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] Join condition: None (59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#106, wr_returned_date_sk#102 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#100 AS return_amt#110, wr_net_loss#101 AS net_loss#111] -Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#109, wr_returned_date_sk#103 AS date_sk#110, 0.00 AS sales_price#111, 0.00 AS profit#112, wr_return_amt#101 AS return_amt#113, wr_net_loss#102 AS net_loss#114] +Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] (60) Union (61) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#115] (62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#93] -Right keys [1]: [cast(d_date_sk#22 as bigint)] +Left keys [1]: [date_sk#94] +Right keys [1]: [cast(d_date_sk#115 as bigint)] Join condition: None (63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97] -Input [7]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, d_date_sk#22] +Output [5]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98] +Input [7]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, d_date_sk#115] (64) Scan parquet default.web_site -Output [2]: [web_site_sk#112, web_site_id#113] +Output [2]: [web_site_sk#116, web_site_id#117] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#112, web_site_id#113] +Input [2]: [web_site_sk#116, web_site_id#117] (66) Filter [codegen id : 17] -Input [2]: [web_site_sk#112, web_site_id#113] -Condition : isnotnull(web_site_sk#112) +Input [2]: [web_site_sk#116, web_site_id#117] +Condition : isnotnull(web_site_sk#116) (67) BroadcastExchange -Input [2]: [web_site_sk#112, web_site_id#113] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#114] +Input [2]: [web_site_sk#116, web_site_id#117] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] (68) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#92] -Right keys [1]: [web_site_sk#112] +Left keys [1]: [wsr_web_site_sk#93] +Right keys [1]: [web_site_sk#116] Join condition: None (69) Project [codegen id : 18] -Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] -Input [7]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#112, web_site_id#113] +Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [7]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] (70) HashAggregate [codegen id : 18] -Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] -Keys [1]: [web_site_id#113] -Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] -Results [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] +Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Keys [1]: [web_site_id#117] +Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum#119, sum#120, sum#121, sum#122] +Results [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] (71) Exchange -Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -Arguments: hashpartitioning(web_site_id#113, 5), ENSURE_REQUIREMENTS, [id=#123] +Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] +Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#127] (72) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -Keys [1]: [web_site_id#113] -Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#124, sum(UnscaledValue(return_amt#96))#125, sum(UnscaledValue(profit#95))#126, sum(UnscaledValue(net_loss#97))#127] -Results [5]: [web channel AS channel#128, concat(web_site, web_site_id#113) AS id#129, MakeDecimal(sum(UnscaledValue(sales_price#94))#124,17,2) AS sales#130, MakeDecimal(sum(UnscaledValue(return_amt#96))#125,17,2) AS returns#131, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#126,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#127,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#132] +Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] +Keys [1]: [web_site_id#117] +Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#128, sum(UnscaledValue(return_amt#97))#129, sum(UnscaledValue(profit#96))#130, sum(UnscaledValue(net_loss#98))#131] +Results [5]: [web channel AS channel#132, concat(web_site, web_site_id#117) AS id#133, MakeDecimal(sum(UnscaledValue(sales_price#95))#128,17,2) AS sales#134, MakeDecimal(sum(UnscaledValue(return_amt#97))#129,17,2) AS returns#135, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#130,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#131,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#136] (73) Union @@ -429,99 +429,99 @@ Results [5]: [web channel AS channel#128, concat(web_site, web_site_id#113) AS i Input [5]: [channel#41, id#42, sales#43, returns#44, profit#45] Keys [2]: [channel#41, id#42] Functions [3]: [partial_sum(sales#43), partial_sum(returns#44), partial_sum(profit#45)] -Aggregate Attributes [6]: [sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -Results [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] +Results [8]: [channel#41, id#42, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] (75) Exchange -Input [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#145] +Input [8]: [channel#41, id#42, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#149] (76) HashAggregate [codegen id : 21] -Input [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Input [8]: [channel#41, id#42, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#45)] -Aggregate Attributes [3]: [sum(sales#43)#146, sum(returns#44)#147, sum(profit#45)#148] -Results [5]: [channel#41, id#42, cast(sum(sales#43)#146 as decimal(37,2)) AS sales#149, cast(sum(returns#44)#147 as decimal(37,2)) AS returns#150, cast(sum(profit#45)#148 as decimal(38,2)) AS profit#151] +Aggregate Attributes [3]: [sum(sales#43)#150, sum(returns#44)#151, sum(profit#45)#152] +Results [5]: [channel#41, id#42, cast(sum(sales#43)#150 as decimal(37,2)) AS sales#153, cast(sum(returns#44)#151 as decimal(37,2)) AS returns#154, cast(sum(profit#45)#152 as decimal(38,2)) AS profit#155] (77) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] +Output [8]: [channel#41, id#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] (78) HashAggregate [codegen id : 42] -Input [8]: [channel#41, id#42, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] +Input [8]: [channel#41, id#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#158)] -Aggregate Attributes [3]: [sum(sales#43)#159, sum(returns#44)#160, sum(profit#158)#161] -Results [4]: [channel#41, sum(sales#43)#159 AS sales#162, sum(returns#44)#160 AS returns#163, sum(profit#158)#161 AS profit#164] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#162)] +Aggregate Attributes [3]: [sum(sales#43)#163, sum(returns#44)#164, sum(profit#162)#165] +Results [4]: [channel#41, sum(sales#43)#163 AS sales#166, sum(returns#44)#164 AS returns#167, sum(profit#162)#165 AS profit#168] (79) HashAggregate [codegen id : 42] -Input [4]: [channel#41, sales#162, returns#163, profit#164] +Input [4]: [channel#41, sales#166, returns#167, profit#168] Keys [1]: [channel#41] -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#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Functions [3]: [partial_sum(sales#166), partial_sum(returns#167), partial_sum(profit#168)] +Aggregate Attributes [6]: [sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174] +Results [7]: [channel#41, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] (80) Exchange -Input [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#177] +Input [7]: [channel#41, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] +Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#181] (81) HashAggregate [codegen id : 43] -Input [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Input [7]: [channel#41, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] Keys [1]: [channel#41] -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#41, 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#166), sum(returns#167), sum(profit#168)] +Aggregate Attributes [3]: [sum(sales#166)#182, sum(returns#167)#183, sum(profit#168)#184] +Results [5]: [channel#41, null AS id#185, sum(sales#166)#182 AS sum(sales)#186, sum(returns#167)#183 AS sum(returns)#187, sum(profit#168)#184 AS sum(profit)#188] (82) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Output [8]: [channel#41, id#42, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194] (83) HashAggregate [codegen id : 64] -Input [8]: [channel#41, id#42, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Input [8]: [channel#41, id#42, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#191)] -Aggregate Attributes [3]: [sum(sales#43)#192, sum(returns#44)#193, sum(profit#191)#194] -Results [3]: [sum(sales#43)#192 AS sales#162, sum(returns#44)#193 AS returns#163, sum(profit#191)#194 AS profit#164] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#195)] +Aggregate Attributes [3]: [sum(sales#43)#196, sum(returns#44)#197, sum(profit#195)#198] +Results [3]: [sum(sales#43)#196 AS sales#166, sum(returns#44)#197 AS returns#167, sum(profit#195)#198 AS profit#168] (84) HashAggregate [codegen id : 64] -Input [3]: [sales#162, returns#163, profit#164] +Input [3]: [sales#166, returns#167, profit#168] Keys: [] -Functions [3]: [partial_sum(sales#162), partial_sum(returns#163), partial_sum(profit#164)] -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#166), partial_sum(returns#167), partial_sum(profit#168)] +Aggregate Attributes [6]: [sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204] +Results [6]: [sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210] (85) Exchange -Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#207] +Input [6]: [sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#211] (86) HashAggregate [codegen id : 65] -Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] +Input [6]: [sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210] Keys: [] -Functions [3]: [sum(sales#162), sum(returns#163), sum(profit#164)] -Aggregate Attributes [3]: [sum(sales#162)#208, sum(returns#163)#209, sum(profit#164)#210] -Results [5]: [null AS channel#211, null AS id#212, sum(sales#162)#208 AS sum(sales)#213, sum(returns#163)#209 AS sum(returns)#214, sum(profit#164)#210 AS sum(profit)#215] +Functions [3]: [sum(sales#166), sum(returns#167), sum(profit#168)] +Aggregate Attributes [3]: [sum(sales#166)#212, sum(returns#167)#213, sum(profit#168)#214] +Results [5]: [null AS channel#215, null AS id#216, sum(sales#166)#212 AS sum(sales)#217, sum(returns#167)#213 AS sum(returns)#218, sum(profit#168)#214 AS sum(profit)#219] (87) Union (88) HashAggregate [codegen id : 66] -Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -Keys [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Keys [5]: [channel#41, id#42, sales#153, returns#154, profit#155] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Results [5]: [channel#41, id#42, sales#153, returns#154, profit#155] (89) Exchange -Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -Arguments: hashpartitioning(channel#41, id#42, sales#149, returns#150, profit#151, 5), ENSURE_REQUIREMENTS, [id=#216] +Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Arguments: hashpartitioning(channel#41, id#42, sales#153, returns#154, profit#155, 5), ENSURE_REQUIREMENTS, [id=#220] (90) HashAggregate [codegen id : 67] -Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -Keys [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Keys [5]: [channel#41, id#42, sales#153, returns#154, profit#155] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Results [5]: [channel#41, id#42, sales#153, returns#154, profit#155] (91) TakeOrderedAndProject -Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#149, returns#150, profit#151] +Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#153, returns#154, profit#155] ===== Subqueries ===== @@ -539,12 +539,12 @@ ReusedExchange (93) (93) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#67] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 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 2a9e1ad7c715b..3847146fa8fa8 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 @@ -795,353 +795,353 @@ Input [17]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_str Arguments: [item_sk#112 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, store_zip#114 ASC NULLS FIRST], false, 0 (132) Scan parquet default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#129)] +PartitionFilters: [isnotnull(ss_sold_date_sk#140), dynamicpruningexpression(ss_sold_date_sk#140 IN dynamicpruning#141)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (133) ColumnarToRow [codegen id : 44] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] (134) Filter [codegen id : 44] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Condition : (((((((isnotnull(ss_item_sk#129) AND isnotnull(ss_ticket_number#136)) AND isnotnull(ss_store_sk#134)) AND isnotnull(ss_customer_sk#130)) AND isnotnull(ss_cdemo_sk#131)) AND isnotnull(ss_promo_sk#135)) AND isnotnull(ss_hdemo_sk#132)) AND isnotnull(ss_addr_sk#133)) (135) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint), 5), ENSURE_REQUIREMENTS, [id=#130] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Arguments: hashpartitioning(cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint), 5), ENSURE_REQUIREMENTS, [id=#142] (136) Sort [codegen id : 45] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [cast(ss_item_sk#1 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#8 as bigint) ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Arguments: [cast(ss_item_sk#129 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#136 as bigint) ASC NULLS FIRST], false, 0 (137) ReusedExchange [Reuses operator id: 10] -Output [2]: [sr_item_sk#15, sr_ticket_number#16] +Output [2]: [sr_item_sk#143, sr_ticket_number#144] (138) Sort [codegen id : 47] -Input [2]: [sr_item_sk#15, sr_ticket_number#16] -Arguments: [sr_item_sk#15 ASC NULLS FIRST, sr_ticket_number#16 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#143, sr_ticket_number#144] +Arguments: [sr_item_sk#143 ASC NULLS FIRST, sr_ticket_number#144 ASC NULLS FIRST], false, 0 (139) SortMergeJoin [codegen id : 56] -Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] -Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] +Left keys [2]: [cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint)] +Right keys [2]: [sr_item_sk#143, sr_ticket_number#144] Join condition: None (140) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, sr_item_sk#143, sr_ticket_number#144] (141) ReusedExchange [Reuses operator id: 33] -Output [1]: [cs_item_sk#19] +Output [1]: [cs_item_sk#145] (142) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#19] +Left keys [1]: [ss_item_sk#129] +Right keys [1]: [cs_item_sk#145] Join condition: None (143) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, cs_item_sk#145] (144) Scan parquet default.date_dim -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#146, d_year#147] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (145) ColumnarToRow [codegen id : 54] -Input [2]: [d_date_sk#43, d_year#44] +Input [2]: [d_date_sk#146, d_year#147] (146) Filter [codegen id : 54] -Input [2]: [d_date_sk#43, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#43)) +Input [2]: [d_date_sk#146, d_year#147] +Condition : ((isnotnull(d_year#147) AND (d_year#147 = 2000)) AND isnotnull(d_date_sk#146)) (147) BroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#131] +Input [2]: [d_date_sk#146, d_year#147] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#148] (148) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#43] +Left keys [1]: [ss_sold_date_sk#140] +Right keys [1]: [d_date_sk#146] Join condition: None (149) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#43, d_year#44] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147] +Input [13]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, d_date_sk#146, d_year#147] (150) ReusedExchange [Reuses operator id: 45] -Output [3]: [s_store_sk#46, s_store_name#47, s_zip#48] +Output [3]: [s_store_sk#149, s_store_name#150, s_zip#151] (151) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#46] +Left keys [1]: [ss_store_sk#134] +Right keys [1]: [s_store_sk#149] Join condition: None (152) Project [codegen id : 56] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_sk#46, s_store_name#47, s_zip#48] +Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_sk#149, s_store_name#150, s_zip#151] (153) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#132] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Arguments: hashpartitioning(ss_customer_sk#130, 5), ENSURE_REQUIREMENTS, [id=#152] (154) Sort [codegen id : 57] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Arguments: [ss_customer_sk#130 ASC NULLS FIRST], false, 0 (155) ReusedExchange [Reuses operator id: 53] -Output [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Output [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] (156) Sort [codegen id : 59] -Input [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] -Arguments: [c_customer_sk#51 ASC NULLS FIRST], false, 0 +Input [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Arguments: [c_customer_sk#153 ASC NULLS FIRST], false, 0 (157) SortMergeJoin [codegen id : 62] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#51] +Left keys [1]: [ss_customer_sk#130] +Right keys [1]: [c_customer_sk#153] Join condition: None (158) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Input [18]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] (159) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#133, d_year#134] +Output [2]: [d_date_sk#159, d_year#160] (160) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_sales_date_sk#56] -Right keys [1]: [d_date_sk#133] +Left keys [1]: [c_first_sales_date_sk#158] +Right keys [1]: [d_date_sk#159] Join condition: None (161) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56, d_date_sk#133, d_year#134] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158, d_date_sk#159, d_year#160] (162) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#135, d_year#136] +Output [2]: [d_date_sk#161, d_year#162] (163) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_shipto_date_sk#55] -Right keys [1]: [d_date_sk#135] +Left keys [1]: [c_first_shipto_date_sk#157] +Right keys [1]: [d_date_sk#161] Join condition: None (164) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134, d_date_sk#135, d_year#136] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160, d_date_sk#161, d_year#162] (165) Exchange -Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Arguments: hashpartitioning(ss_cdemo_sk#3, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Arguments: hashpartitioning(ss_cdemo_sk#131, 5), ENSURE_REQUIREMENTS, [id=#163] (166) Sort [codegen id : 63] -Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Arguments: [ss_cdemo_sk#3 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Arguments: [ss_cdemo_sk#131 ASC NULLS FIRST], false, 0 (167) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#64, cd_marital_status#65] +Output [2]: [cd_demo_sk#164, cd_marital_status#165] (168) Sort [codegen id : 65] -Input [2]: [cd_demo_sk#64, cd_marital_status#65] -Arguments: [cd_demo_sk#64 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#164, cd_marital_status#165] +Arguments: [cd_demo_sk#164 ASC NULLS FIRST], false, 0 (169) SortMergeJoin [codegen id : 66] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#64] +Left keys [1]: [ss_cdemo_sk#131] +Right keys [1]: [cd_demo_sk#164] Join condition: None (170) Project [codegen id : 66] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_demo_sk#64, cd_marital_status#65] +Output [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_demo_sk#164, cd_marital_status#165] (171) Exchange -Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Arguments: hashpartitioning(c_current_cdemo_sk#52, 5), ENSURE_REQUIREMENTS, [id=#138] +Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Arguments: hashpartitioning(c_current_cdemo_sk#154, 5), ENSURE_REQUIREMENTS, [id=#166] (172) Sort [codegen id : 67] -Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Arguments: [c_current_cdemo_sk#52 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Arguments: [c_current_cdemo_sk#154 ASC NULLS FIRST], false, 0 (173) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#139, cd_marital_status#140] +Output [2]: [cd_demo_sk#167, cd_marital_status#168] (174) Sort [codegen id : 69] -Input [2]: [cd_demo_sk#139, cd_marital_status#140] -Arguments: [cd_demo_sk#139 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#167, cd_marital_status#168] +Arguments: [cd_demo_sk#167 ASC NULLS FIRST], false, 0 (175) SortMergeJoin [codegen id : 73] -Left keys [1]: [c_current_cdemo_sk#52] -Right keys [1]: [cd_demo_sk#139] -Join condition: NOT (cd_marital_status#65 = cd_marital_status#140) +Left keys [1]: [c_current_cdemo_sk#154] +Right keys [1]: [cd_demo_sk#167] +Join condition: NOT (cd_marital_status#165 = cd_marital_status#168) (176) Project [codegen id : 73] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65, cd_demo_sk#139, cd_marital_status#140] +Output [14]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [18]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165, cd_demo_sk#167, cd_marital_status#168] (177) ReusedExchange [Reuses operator id: 84] -Output [1]: [p_promo_sk#70] +Output [1]: [p_promo_sk#169] (178) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#70] +Left keys [1]: [ss_promo_sk#135] +Right keys [1]: [p_promo_sk#169] Join condition: None (179) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, p_promo_sk#70] +Output [13]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, p_promo_sk#169] (180) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#72, hd_income_band_sk#73] +Output [2]: [hd_demo_sk#170, hd_income_band_sk#171] (181) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#72] +Left keys [1]: [ss_hdemo_sk#132] +Right keys [1]: [hd_demo_sk#170] Join condition: None (182) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_demo_sk#72, hd_income_band_sk#73] +Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171] +Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_demo_sk#170, hd_income_band_sk#171] (183) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] +Output [2]: [hd_demo_sk#172, hd_income_band_sk#173] (184) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [c_current_hdemo_sk#53] -Right keys [1]: [hd_demo_sk#141] +Left keys [1]: [c_current_hdemo_sk#155] +Right keys [1]: [hd_demo_sk#172] Join condition: None (185) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_demo_sk#141, hd_income_band_sk#142] +Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Input [15]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_demo_sk#172, hd_income_band_sk#173] (186) Exchange -Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Arguments: hashpartitioning(ss_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#143] +Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Arguments: hashpartitioning(ss_addr_sk#133, 5), ENSURE_REQUIREMENTS, [id=#174] (187) Sort [codegen id : 74] -Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Arguments: [ss_addr_sk#5 ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Arguments: [ss_addr_sk#133 ASC NULLS FIRST], false, 0 (188) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Output [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] (189) Sort [codegen id : 76] -Input [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: [ca_address_sk#78 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: [ca_address_sk#175 ASC NULLS FIRST], false, 0 (190) SortMergeJoin [codegen id : 77] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#78] +Left keys [1]: [ss_addr_sk#133] +Right keys [1]: [ca_address_sk#175] Join condition: None (191) Project [codegen id : 77] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Output [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Input [18]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] (192) Exchange -Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: hashpartitioning(c_current_addr_sk#54, 5), ENSURE_REQUIREMENTS, [id=#144] +Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: hashpartitioning(c_current_addr_sk#156, 5), ENSURE_REQUIREMENTS, [id=#180] (193) Sort [codegen id : 78] -Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: [c_current_addr_sk#54 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: [c_current_addr_sk#156 ASC NULLS FIRST], false, 0 (194) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Output [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] (195) Sort [codegen id : 80] -Input [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Arguments: [ca_address_sk#145 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Arguments: [ca_address_sk#181 ASC NULLS FIRST], false, 0 (196) SortMergeJoin [codegen id : 84] -Left keys [1]: [c_current_addr_sk#54] -Right keys [1]: [ca_address_sk#145] +Left keys [1]: [c_current_addr_sk#156] +Right keys [1]: [ca_address_sk#181] Join condition: None (197) Project [codegen id : 84] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Output [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [21]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] (198) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#90] +Output [1]: [ib_income_band_sk#186] (199) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#73] -Right keys [1]: [ib_income_band_sk#90] +Left keys [1]: [hd_income_band_sk#171] +Right keys [1]: [ib_income_band_sk#186] Join condition: None (200) Project [codegen id : 84] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#90] +Output [18]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [20]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#186] (201) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#150] +Output [1]: [ib_income_band_sk#187] (202) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#142] -Right keys [1]: [ib_income_band_sk#150] +Left keys [1]: [hd_income_band_sk#173] +Right keys [1]: [ib_income_band_sk#187] Join condition: None (203) Project [codegen id : 84] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#150] +Output [17]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#187] (204) ReusedExchange [Reuses operator id: 124] -Output [2]: [i_item_sk#93, i_product_name#96] +Output [2]: [i_item_sk#188, i_product_name#189] (205) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#93] +Left keys [1]: [ss_item_sk#129] +Right keys [1]: [i_item_sk#188] Join condition: None (206) Project [codegen id : 84] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] +Output [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] +Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] (207) HashAggregate [codegen id : 84] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] -Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#151, sum#152, sum#153, sum#154] -Results [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] +Input [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] +Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#137)), partial_sum(UnscaledValue(ss_list_price#138)), partial_sum(UnscaledValue(ss_coupon_amt#139))] +Aggregate Attributes [4]: [count#190, sum#191, sum#192, sum#193] +Results [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] (208) Exchange -Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] -Arguments: hashpartitioning(i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, 5), ENSURE_REQUIREMENTS, [id=#159] +Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] +Arguments: hashpartitioning(i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, 5), ENSURE_REQUIREMENTS, [id=#198] (209) HashAggregate [codegen id : 85] -Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] -Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#160, sum(UnscaledValue(ss_wholesale_cost#9))#161, sum(UnscaledValue(ss_list_price#10))#162, sum(UnscaledValue(ss_coupon_amt#11))#163] -Results [8]: [i_item_sk#93 AS item_sk#164, s_store_name#47 AS store_name#165, s_zip#48 AS store_zip#166, d_year#44 AS syear#167, count(1)#160 AS cnt#168, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#161,17,2) AS s1#169, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#162,17,2) AS s2#170, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#163,17,2) AS s3#171] +Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] +Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#137)), sum(UnscaledValue(ss_list_price#138)), sum(UnscaledValue(ss_coupon_amt#139))] +Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#137))#200, sum(UnscaledValue(ss_list_price#138))#201, sum(UnscaledValue(ss_coupon_amt#139))#202] +Results [8]: [i_item_sk#188 AS item_sk#203, s_store_name#150 AS store_name#204, s_zip#151 AS store_zip#205, d_year#147 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#137))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#138))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#139))#202,17,2) AS s3#210] (210) Exchange -Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] -Arguments: hashpartitioning(item_sk#164, store_name#165, store_zip#166, 5), ENSURE_REQUIREMENTS, [id=#172] +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] (211) Sort [codegen id : 86] -Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] -Arguments: [item_sk#164 ASC NULLS FIRST, store_name#165 ASC NULLS FIRST, store_zip#166 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 (212) SortMergeJoin [codegen id : 87] Left keys [3]: [item_sk#112, store_name#113, store_zip#114] -Right keys [3]: [item_sk#164, store_name#165, store_zip#166] -Join condition: (cnt#168 <= cnt#124) +Right keys [3]: [item_sk#203, store_name#204, store_zip#205] +Join condition: (cnt#207 <= cnt#124) (213) Project [codegen id : 87] -Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] +Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] (214) Exchange -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#169 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#173] +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#208 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] (215) Sort [codegen id : 88] -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#169 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#208 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1152,11 +1152,11 @@ ReusedExchange (216) (216) ReusedExchange [Reuses operator id: 39] Output [2]: [d_date_sk#43, d_year#44] -Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#129 +Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#140 IN dynamicpruning#141 ReusedExchange (217) (217) ReusedExchange [Reuses operator id: 147] -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#146, d_year#147] 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 5c2ac1abaaa7e..0115f4ab8a63a 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 @@ -701,323 +701,323 @@ Input [17]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_str Arguments: [item_sk#105 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, store_zip#107 ASC NULLS FIRST], false, 0 (115) Scan parquet default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#122)] +PartitionFilters: [isnotnull(ss_sold_date_sk#133), dynamicpruningexpression(ss_sold_date_sk#133 IN dynamicpruning#134)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (116) ColumnarToRow [codegen id : 27] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] (117) Filter [codegen id : 27] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Condition : (((((((isnotnull(ss_item_sk#122) AND isnotnull(ss_ticket_number#129)) AND isnotnull(ss_store_sk#127)) AND isnotnull(ss_customer_sk#123)) AND isnotnull(ss_cdemo_sk#124)) AND isnotnull(ss_promo_sk#128)) AND isnotnull(ss_hdemo_sk#125)) AND isnotnull(ss_addr_sk#126)) (118) BroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#123] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#135] (119) Scan parquet default.store_returns -Output [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Output [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct (120) ColumnarToRow -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] (121) Filter -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] -Condition : (isnotnull(sr_item_sk#15) AND isnotnull(sr_ticket_number#16)) +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Condition : (isnotnull(sr_item_sk#136) AND isnotnull(sr_ticket_number#137)) (122) Project -Output [2]: [sr_item_sk#15, sr_ticket_number#16] -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Output [2]: [sr_item_sk#136, sr_ticket_number#137] +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] (123) BroadcastHashJoin [codegen id : 28] -Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] -Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] +Left keys [2]: [cast(ss_item_sk#122 as bigint), cast(ss_ticket_number#129 as bigint)] +Right keys [2]: [sr_item_sk#136, sr_ticket_number#137] Join condition: None (124) Project [codegen id : 28] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, sr_item_sk#136, sr_ticket_number#137] (125) Exchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: hashpartitioning(ss_item_sk#122, 5), ENSURE_REQUIREMENTS, [id=#139] (126) Sort [codegen id : 29] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: [ss_item_sk#122 ASC NULLS FIRST], false, 0 (127) ReusedExchange [Reuses operator id: 28] -Output [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] +Output [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] (128) HashAggregate [codegen id : 35] -Input [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] -Keys [1]: [cs_item_sk#19] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#21)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#21))#128, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129] -Results [3]: [cs_item_sk#19, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#21))#128,17,2) AS sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] +Input [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] +Keys [1]: [cs_item_sk#140] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#144)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#144))#148, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149] +Results [3]: [cs_item_sk#140, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#144))#148,17,2) AS sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] (129) Filter [codegen id : 35] -Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] -Condition : (isnotnull(sum(cs_ext_list_price#21)#130) AND (cast(sum(cs_ext_list_price#21)#130 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131)), DecimalType(21,2), true))) +Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] +Condition : (isnotnull(sum(cs_ext_list_price#144)#150) AND (cast(sum(cs_ext_list_price#144)#150 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151)), DecimalType(21,2), true))) (130) Project [codegen id : 35] -Output [1]: [cs_item_sk#19] -Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] +Output [1]: [cs_item_sk#140] +Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] (131) Sort [codegen id : 35] -Input [1]: [cs_item_sk#19] -Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 +Input [1]: [cs_item_sk#140] +Arguments: [cs_item_sk#140 ASC NULLS FIRST], false, 0 (132) SortMergeJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#19] +Left keys [1]: [ss_item_sk#122] +Right keys [1]: [cs_item_sk#140] Join condition: None (133) Project [codegen id : 51] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, cs_item_sk#140] (134) Scan parquet default.date_dim -Output [2]: [d_date_sk#42, d_year#43] +Output [2]: [d_date_sk#152, d_year#153] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (135) ColumnarToRow [codegen id : 36] -Input [2]: [d_date_sk#42, d_year#43] +Input [2]: [d_date_sk#152, d_year#153] (136) Filter [codegen id : 36] -Input [2]: [d_date_sk#42, d_year#43] -Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2000)) AND isnotnull(d_date_sk#42)) +Input [2]: [d_date_sk#152, d_year#153] +Condition : ((isnotnull(d_year#153) AND (d_year#153 = 2000)) AND isnotnull(d_date_sk#152)) (137) BroadcastExchange -Input [2]: [d_date_sk#42, d_year#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#132] +Input [2]: [d_date_sk#152, d_year#153] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#154] (138) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#42] +Left keys [1]: [ss_sold_date_sk#133] +Right keys [1]: [d_date_sk#152] Join condition: None (139) Project [codegen id : 51] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#42, d_year#43] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153] +Input [13]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, d_date_sk#152, d_year#153] (140) ReusedExchange [Reuses operator id: 44] -Output [3]: [s_store_sk#45, s_store_name#46, s_zip#47] +Output [3]: [s_store_sk#155, s_store_name#156, s_zip#157] (141) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#45] +Left keys [1]: [ss_store_sk#127] +Right keys [1]: [s_store_sk#155] Join condition: None (142) Project [codegen id : 51] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_sk#45, s_store_name#46, s_zip#47] +Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157] +Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_sk#155, s_store_name#156, s_zip#157] (143) ReusedExchange [Reuses operator id: 50] -Output [6]: [c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] +Output [6]: [c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] (144) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#49] +Left keys [1]: [ss_customer_sk#123] +Right keys [1]: [c_customer_sk#158] Join condition: None (145) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] +Input [18]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] (146) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#133, d_year#134] +Output [2]: [d_date_sk#164, d_year#165] (147) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_sales_date_sk#54] -Right keys [1]: [d_date_sk#133] +Left keys [1]: [c_first_sales_date_sk#163] +Right keys [1]: [d_date_sk#164] Join condition: None (148) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54, d_date_sk#133, d_year#134] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163, d_date_sk#164, d_year#165] (149) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#135, d_year#136] +Output [2]: [d_date_sk#166, d_year#167] (150) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_shipto_date_sk#53] -Right keys [1]: [d_date_sk#135] +Left keys [1]: [c_first_shipto_date_sk#162] +Right keys [1]: [d_date_sk#166] Join condition: None (151) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134, d_date_sk#135, d_year#136] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165, d_date_sk#166, d_year#167] (152) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#61, cd_marital_status#62] +Output [2]: [cd_demo_sk#168, cd_marital_status#169] (153) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#61] +Left keys [1]: [ss_cdemo_sk#124] +Right keys [1]: [cd_demo_sk#168] Join condition: None (154) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_demo_sk#61, cd_marital_status#62] +Output [16]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_demo_sk#168, cd_marital_status#169] (155) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#137, cd_marital_status#138] +Output [2]: [cd_demo_sk#170, cd_marital_status#171] (156) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_cdemo_sk#50] -Right keys [1]: [cd_demo_sk#137] -Join condition: NOT (cd_marital_status#62 = cd_marital_status#138) +Left keys [1]: [c_current_cdemo_sk#159] +Right keys [1]: [cd_demo_sk#170] +Join condition: NOT (cd_marital_status#169 = cd_marital_status#171) (157) Project [codegen id : 51] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62, cd_demo_sk#137, cd_marital_status#138] +Output [14]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [18]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169, cd_demo_sk#170, cd_marital_status#171] (158) ReusedExchange [Reuses operator id: 74] -Output [1]: [p_promo_sk#66] +Output [1]: [p_promo_sk#172] (159) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#66] +Left keys [1]: [ss_promo_sk#128] +Right keys [1]: [p_promo_sk#172] Join condition: None (160) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, p_promo_sk#66] +Output [13]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, p_promo_sk#172] (161) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#68, hd_income_band_sk#69] +Output [2]: [hd_demo_sk#173, hd_income_band_sk#174] (162) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#68] +Left keys [1]: [ss_hdemo_sk#125] +Right keys [1]: [hd_demo_sk#173] Join condition: None (163) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_demo_sk#68, hd_income_band_sk#69] +Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174] +Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_demo_sk#173, hd_income_band_sk#174] (164) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] +Output [2]: [hd_demo_sk#175, hd_income_band_sk#176] (165) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_hdemo_sk#51] -Right keys [1]: [hd_demo_sk#139] +Left keys [1]: [c_current_hdemo_sk#160] +Right keys [1]: [hd_demo_sk#175] Join condition: None (166) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_demo_sk#139, hd_income_band_sk#140] +Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176] +Input [15]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_demo_sk#175, hd_income_band_sk#176] (167) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] +Output [5]: [ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] (168) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#73] +Left keys [1]: [ss_addr_sk#126] +Right keys [1]: [ca_address_sk#177] Join condition: None (169) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] +Output [16]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] +Input [18]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] (170) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Output [5]: [ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] (171) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_addr_sk#52] -Right keys [1]: [ca_address_sk#141] +Left keys [1]: [c_current_addr_sk#161] +Right keys [1]: [ca_address_sk#182] Join condition: None (172) Project [codegen id : 51] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Output [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [21]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] (173) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#84] +Output [1]: [ib_income_band_sk#187] (174) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#69] -Right keys [1]: [ib_income_band_sk#84] +Left keys [1]: [hd_income_band_sk#174] +Right keys [1]: [ib_income_band_sk#187] Join condition: None (175) Project [codegen id : 51] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#84] +Output [18]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [20]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#187] (176) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#146] +Output [1]: [ib_income_band_sk#188] (177) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#140] -Right keys [1]: [ib_income_band_sk#146] +Left keys [1]: [hd_income_band_sk#176] +Right keys [1]: [ib_income_band_sk#188] Join condition: None (178) Project [codegen id : 51] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#146] +Output [17]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#188] (179) ReusedExchange [Reuses operator id: 108] -Output [2]: [i_item_sk#87, i_product_name#90] +Output [2]: [i_item_sk#189, i_product_name#190] (180) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#87] +Left keys [1]: [ss_item_sk#122] +Right keys [1]: [i_item_sk#189] Join condition: None (181) Project [codegen id : 51] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] +Output [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] (182) HashAggregate [codegen id : 51] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] -Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#147, sum#148, sum#149, sum#150] -Results [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] +Input [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#130)), partial_sum(UnscaledValue(ss_list_price#131)), partial_sum(UnscaledValue(ss_coupon_amt#132))] +Aggregate Attributes [4]: [count#191, sum#192, sum#193, sum#194] +Results [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] (183) HashAggregate [codegen id : 51] -Input [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#155, sum(UnscaledValue(ss_wholesale_cost#9))#156, sum(UnscaledValue(ss_list_price#10))#157, sum(UnscaledValue(ss_coupon_amt#11))#158] -Results [8]: [i_item_sk#87 AS item_sk#159, s_store_name#46 AS store_name#160, s_zip#47 AS store_zip#161, d_year#43 AS syear#162, count(1)#155 AS cnt#163, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#156,17,2) AS s1#164, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#157,17,2) AS s2#165, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#158,17,2) AS s3#166] +Input [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] +Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#130)), sum(UnscaledValue(ss_list_price#131)), sum(UnscaledValue(ss_coupon_amt#132))] +Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#130))#200, sum(UnscaledValue(ss_list_price#131))#201, sum(UnscaledValue(ss_coupon_amt#132))#202] +Results [8]: [i_item_sk#189 AS item_sk#203, s_store_name#156 AS store_name#204, s_zip#157 AS store_zip#205, d_year#153 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#130))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#131))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#132))#202,17,2) AS s3#210] (184) Exchange -Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] -Arguments: hashpartitioning(item_sk#159, store_name#160, store_zip#161, 5), ENSURE_REQUIREMENTS, [id=#167] +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] (185) Sort [codegen id : 52] -Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] -Arguments: [item_sk#159 ASC NULLS FIRST, store_name#160 ASC NULLS FIRST, store_zip#161 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 (186) SortMergeJoin [codegen id : 53] Left keys [3]: [item_sk#105, store_name#106, store_zip#107] -Right keys [3]: [item_sk#159, store_name#160, store_zip#161] -Join condition: (cnt#163 <= cnt#117) +Right keys [3]: [item_sk#203, store_name#204, store_zip#205] +Join condition: (cnt#207 <= cnt#117) (187) Project [codegen id : 53] -Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] +Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] (188) Exchange -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#164 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#168] +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#208 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] (189) Sort [codegen id : 54] -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#164 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#208 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1028,11 +1028,11 @@ ReusedExchange (190) (190) ReusedExchange [Reuses operator id: 38] Output [2]: [d_date_sk#42, d_year#43] -Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#122 +Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#133 IN dynamicpruning#134 ReusedExchange (191) (191) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#42, d_year#43] +Output [2]: [d_date_sk#152, d_year#153] 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 413a918da6a0f..af8035e67b061 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 @@ -206,234 +206,234 @@ Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 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))#28 as decimal(38,2)) AS sumsales#29] (29) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#30, isEmpty#31] +Output [10]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, s_store_id#37, sum#38, isEmpty#39] (30) HashAggregate [codegen id : 16] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#30, isEmpty#31] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#32] -Results [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#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))#32 AS sumsales#33] +Input [10]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, s_store_id#37, sum#38, isEmpty#39] +Keys [8]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, s_store_id#37] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#40 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#41 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#40 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#41 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#42] +Results [8]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#40 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#41 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#42 AS sumsales#43] (31) HashAggregate [codegen id : 16] -Input [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, sumsales#33] -Keys [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#34, isEmpty#35] -Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, sum#36, isEmpty#37] +Input [8]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sumsales#43] +Keys [7]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#44, isEmpty#45] +Results [9]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sum#46, isEmpty#47] (32) Exchange -Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, sum#36, isEmpty#37] -Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [9]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sum#46, isEmpty#47] +Arguments: hashpartitioning(i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, 5), ENSURE_REQUIREMENTS, [id=#48] (33) HashAggregate [codegen id : 17] -Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, sum#36, isEmpty#37] -Keys [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#39] -Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, null AS s_store_id#40, sum(sumsales#33)#39 AS sumsales#41] +Input [9]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sum#46, isEmpty#47] +Keys [7]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#49] +Results [9]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, null AS s_store_id#50, sum(sumsales#43)#49 AS sumsales#51] (34) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#42, isEmpty#43] +Output [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] (35) HashAggregate [codegen id : 25] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#42, isEmpty#43] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#44] -Results [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, 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))#44 AS sumsales#33] +Input [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] +Keys [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#62 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#63 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#62 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#63 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#64] +Results [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#62 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#63 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#64 AS sumsales#43] (36) HashAggregate [codegen id : 25] -Input [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, sumsales#33] -Keys [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#45, isEmpty#46] -Results [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, sum#47, isEmpty#48] +Input [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sumsales#43] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] (37) Exchange -Input [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, sum#47, isEmpty#48] -Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Arguments: hashpartitioning(i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, 5), ENSURE_REQUIREMENTS, [id=#69] (38) HashAggregate [codegen id : 26] -Input [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, sum#47, isEmpty#48] -Keys [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#50] -Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, null AS d_moy#51, null AS s_store_id#52, sum(sumsales#33)#50 AS sumsales#53] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#70] +Results [9]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, null AS d_moy#71, null AS s_store_id#72, sum(sumsales#43)#70 AS sumsales#73] (39) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#54, isEmpty#55] +Output [10]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81, sum#82, isEmpty#83] (40) HashAggregate [codegen id : 34] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#54, isEmpty#55] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#56] -Results [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#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))#56 AS sumsales#33] +Input [10]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81, sum#82, isEmpty#83] +Keys [8]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#84 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#85 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#84 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#86] +Results [6]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#84 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#86 AS sumsales#43] (41) HashAggregate [codegen id : 34] -Input [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, sumsales#33] -Keys [5]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#57, isEmpty#58] -Results [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, sum#59, isEmpty#60] +Input [6]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sumsales#43] +Keys [5]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#87, isEmpty#88] +Results [7]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum#89, isEmpty#90] (42) Exchange -Input [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, sum#59, isEmpty#60] -Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [7]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum#89, isEmpty#90] +Arguments: hashpartitioning(i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#91] (43) HashAggregate [codegen id : 35] -Input [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, sum#59, isEmpty#60] -Keys [5]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#62] -Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, null AS d_qoy#63, null AS d_moy#64, null AS s_store_id#65, sum(sumsales#33)#62 AS sumsales#66] +Input [7]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum#89, isEmpty#90] +Keys [5]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#92] +Results [9]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, null AS d_qoy#93, null AS d_moy#94, null AS s_store_id#95, sum(sumsales#43)#92 AS sumsales#96] (44) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#67, isEmpty#68] +Output [10]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, d_year#101, d_qoy#102, d_moy#103, s_store_id#104, sum#105, isEmpty#106] (45) HashAggregate [codegen id : 43] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#67, isEmpty#68] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#69] -Results [5]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, 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))#69 AS sumsales#33] +Input [10]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, d_year#101, d_qoy#102, d_moy#103, s_store_id#104, sum#105, isEmpty#106] +Keys [8]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, d_year#101, d_qoy#102, d_moy#103, s_store_id#104] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#107 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#108 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#107 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#108 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#109] +Results [5]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#107 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#108 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#109 AS sumsales#43] (46) HashAggregate [codegen id : 43] -Input [5]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, sumsales#33] -Keys [4]: [i_category#20, i_class#19, i_brand#18, i_product_name#21] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#70, isEmpty#71] -Results [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, sum#72, isEmpty#73] +Input [5]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sumsales#43] +Keys [4]: [i_category#97, i_class#98, i_brand#99, i_product_name#100] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#110, isEmpty#111] +Results [6]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sum#112, isEmpty#113] (47) Exchange -Input [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, sum#72, isEmpty#73] -Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, i_product_name#21, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [6]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sum#112, isEmpty#113] +Arguments: hashpartitioning(i_category#97, i_class#98, i_brand#99, i_product_name#100, 5), ENSURE_REQUIREMENTS, [id=#114] (48) HashAggregate [codegen id : 44] -Input [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, sum#72, isEmpty#73] -Keys [4]: [i_category#20, i_class#19, i_brand#18, i_product_name#21] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#75] -Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, null AS d_year#76, null AS d_qoy#77, null AS d_moy#78, null AS s_store_id#79, sum(sumsales#33)#75 AS sumsales#80] +Input [6]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sum#112, isEmpty#113] +Keys [4]: [i_category#97, i_class#98, i_brand#99, i_product_name#100] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#115] +Results [9]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, null AS d_year#116, null AS d_qoy#117, null AS d_moy#118, null AS s_store_id#119, sum(sumsales#43)#115 AS sumsales#120] (49) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#81, isEmpty#82] +Output [10]: [i_category#121, i_class#122, i_brand#123, i_product_name#124, d_year#125, d_qoy#126, d_moy#127, s_store_id#128, sum#129, isEmpty#130] (50) HashAggregate [codegen id : 52] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#81, isEmpty#82] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#83] -Results [4]: [i_category#20, i_class#19, i_brand#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))#83 AS sumsales#33] +Input [10]: [i_category#121, i_class#122, i_brand#123, i_product_name#124, d_year#125, d_qoy#126, d_moy#127, s_store_id#128, sum#129, isEmpty#130] +Keys [8]: [i_category#121, i_class#122, i_brand#123, i_product_name#124, d_year#125, d_qoy#126, d_moy#127, s_store_id#128] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#131 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#132 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#131 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#132 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#133] +Results [4]: [i_category#121, i_class#122, i_brand#123, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#131 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#132 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#133 AS sumsales#43] (51) HashAggregate [codegen id : 52] -Input [4]: [i_category#20, i_class#19, i_brand#18, sumsales#33] -Keys [3]: [i_category#20, i_class#19, i_brand#18] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#84, isEmpty#85] -Results [5]: [i_category#20, i_class#19, i_brand#18, sum#86, isEmpty#87] +Input [4]: [i_category#121, i_class#122, i_brand#123, sumsales#43] +Keys [3]: [i_category#121, i_class#122, i_brand#123] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#134, isEmpty#135] +Results [5]: [i_category#121, i_class#122, i_brand#123, sum#136, isEmpty#137] (52) Exchange -Input [5]: [i_category#20, i_class#19, i_brand#18, sum#86, isEmpty#87] -Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [5]: [i_category#121, i_class#122, i_brand#123, sum#136, isEmpty#137] +Arguments: hashpartitioning(i_category#121, i_class#122, i_brand#123, 5), ENSURE_REQUIREMENTS, [id=#138] (53) HashAggregate [codegen id : 53] -Input [5]: [i_category#20, i_class#19, i_brand#18, sum#86, isEmpty#87] -Keys [3]: [i_category#20, i_class#19, i_brand#18] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#89] -Results [9]: [i_category#20, i_class#19, i_brand#18, null AS i_product_name#90, null AS d_year#91, null AS d_qoy#92, null AS d_moy#93, null AS s_store_id#94, sum(sumsales#33)#89 AS sumsales#95] +Input [5]: [i_category#121, i_class#122, i_brand#123, sum#136, isEmpty#137] +Keys [3]: [i_category#121, i_class#122, i_brand#123] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#139] +Results [9]: [i_category#121, i_class#122, i_brand#123, null AS i_product_name#140, null AS d_year#141, null AS d_qoy#142, null AS d_moy#143, null AS s_store_id#144, sum(sumsales#43)#139 AS sumsales#145] (54) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#96, isEmpty#97] +Output [10]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sum#154, isEmpty#155] (55) HashAggregate [codegen id : 61] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#96, isEmpty#97] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#98] -Results [3]: [i_category#20, i_class#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))#98 AS sumsales#33] +Input [10]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sum#154, isEmpty#155] +Keys [8]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#156 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#157 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#156 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#157 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#158] +Results [3]: [i_category#146, i_class#147, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#156 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#157 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#158 AS sumsales#43] (56) HashAggregate [codegen id : 61] -Input [3]: [i_category#20, i_class#19, sumsales#33] -Keys [2]: [i_category#20, i_class#19] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#99, isEmpty#100] -Results [4]: [i_category#20, i_class#19, sum#101, isEmpty#102] +Input [3]: [i_category#146, i_class#147, sumsales#43] +Keys [2]: [i_category#146, i_class#147] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#159, isEmpty#160] +Results [4]: [i_category#146, i_class#147, sum#161, isEmpty#162] (57) Exchange -Input [4]: [i_category#20, i_class#19, sum#101, isEmpty#102] -Arguments: hashpartitioning(i_category#20, i_class#19, 5), ENSURE_REQUIREMENTS, [id=#103] +Input [4]: [i_category#146, i_class#147, sum#161, isEmpty#162] +Arguments: hashpartitioning(i_category#146, i_class#147, 5), ENSURE_REQUIREMENTS, [id=#163] (58) HashAggregate [codegen id : 62] -Input [4]: [i_category#20, i_class#19, sum#101, isEmpty#102] -Keys [2]: [i_category#20, i_class#19] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#104] -Results [9]: [i_category#20, i_class#19, null AS i_brand#105, null AS i_product_name#106, null AS d_year#107, null AS d_qoy#108, null AS d_moy#109, null AS s_store_id#110, sum(sumsales#33)#104 AS sumsales#111] +Input [4]: [i_category#146, i_class#147, sum#161, isEmpty#162] +Keys [2]: [i_category#146, i_class#147] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#164] +Results [9]: [i_category#146, i_class#147, null AS i_brand#165, null AS i_product_name#166, null AS d_year#167, null AS d_qoy#168, null AS d_moy#169, null AS s_store_id#170, sum(sumsales#43)#164 AS sumsales#171] (59) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#112, isEmpty#113] +Output [10]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179, sum#180, isEmpty#181] (60) HashAggregate [codegen id : 70] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#112, isEmpty#113] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#114] -Results [2]: [i_category#20, 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))#114 AS sumsales#33] +Input [10]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179, sum#180, isEmpty#181] +Keys [8]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#182 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#183 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#182 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#183 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#184] +Results [2]: [i_category#172, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#182 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#183 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#184 AS sumsales#43] (61) HashAggregate [codegen id : 70] -Input [2]: [i_category#20, sumsales#33] -Keys [1]: [i_category#20] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#115, isEmpty#116] -Results [3]: [i_category#20, sum#117, isEmpty#118] +Input [2]: [i_category#172, sumsales#43] +Keys [1]: [i_category#172] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#185, isEmpty#186] +Results [3]: [i_category#172, sum#187, isEmpty#188] (62) Exchange -Input [3]: [i_category#20, sum#117, isEmpty#118] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#119] +Input [3]: [i_category#172, sum#187, isEmpty#188] +Arguments: hashpartitioning(i_category#172, 5), ENSURE_REQUIREMENTS, [id=#189] (63) HashAggregate [codegen id : 71] -Input [3]: [i_category#20, sum#117, isEmpty#118] -Keys [1]: [i_category#20] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#120] -Results [9]: [i_category#20, null AS i_class#121, null AS i_brand#122, null AS i_product_name#123, null AS d_year#124, null AS d_qoy#125, null AS d_moy#126, null AS s_store_id#127, sum(sumsales#33)#120 AS sumsales#128] +Input [3]: [i_category#172, sum#187, isEmpty#188] +Keys [1]: [i_category#172] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#190] +Results [9]: [i_category#172, null AS i_class#191, null AS i_brand#192, null AS i_product_name#193, null AS d_year#194, null AS d_qoy#195, null AS d_moy#196, null AS s_store_id#197, sum(sumsales#43)#190 AS sumsales#198] (64) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#129, isEmpty#130] +Output [10]: [i_category#199, i_class#200, i_brand#201, i_product_name#202, d_year#203, d_qoy#204, d_moy#205, s_store_id#206, sum#207, isEmpty#208] (65) HashAggregate [codegen id : 79] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#129, isEmpty#130] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#131] -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))#131 AS sumsales#33] +Input [10]: [i_category#199, i_class#200, i_brand#201, i_product_name#202, d_year#203, d_qoy#204, d_moy#205, s_store_id#206, sum#207, isEmpty#208] +Keys [8]: [i_category#199, i_class#200, i_brand#201, i_product_name#202, d_year#203, d_qoy#204, d_moy#205, s_store_id#206] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#209 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#210 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#209 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#210 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#211] +Results [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#209 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#210 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#211 AS sumsales#43] (66) HashAggregate [codegen id : 79] -Input [1]: [sumsales#33] +Input [1]: [sumsales#43] Keys: [] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#132, isEmpty#133] -Results [2]: [sum#134, isEmpty#135] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#212, isEmpty#213] +Results [2]: [sum#214, isEmpty#215] (67) Exchange -Input [2]: [sum#134, isEmpty#135] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#136] +Input [2]: [sum#214, isEmpty#215] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#216] (68) HashAggregate [codegen id : 80] -Input [2]: [sum#134, isEmpty#135] +Input [2]: [sum#214, isEmpty#215] Keys: [] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#137] -Results [9]: [null AS i_category#138, null AS i_class#139, null AS i_brand#140, null AS i_product_name#141, null AS d_year#142, null AS d_qoy#143, null AS d_moy#144, null AS s_store_id#145, sum(sumsales#33)#137 AS sumsales#146] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#217] +Results [9]: [null AS i_category#218, null AS i_class#219, null AS i_brand#220, null AS i_product_name#221, null AS d_year#222, null AS d_qoy#223, null AS d_moy#224, null AS s_store_id#225, sum(sumsales#43)#217 AS sumsales#226] (69) Union (70) Exchange Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#147] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#227] (71) Sort [codegen id : 81] Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29] @@ -441,15 +441,15 @@ Arguments: [i_category#20 ASC NULLS FIRST, sumsales#29 DESC NULLS LAST], false, (72) Window Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29] -Arguments: [rank(sumsales#29) windowspecdefinition(i_category#20, sumsales#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#148], [i_category#20], [sumsales#29 DESC NULLS LAST] +Arguments: [rank(sumsales#29) windowspecdefinition(i_category#20, sumsales#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#228], [i_category#20], [sumsales#29 DESC NULLS LAST] (73) Filter [codegen id : 82] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] -Condition : (rk#148 <= 100) +Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#228] +Condition : (rk#228 <= 100) (74) TakeOrderedAndProject -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] -Arguments: 100, [i_category#20 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#18 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#14 ASC NULLS FIRST, sumsales#29 ASC NULLS FIRST, rk#148 ASC NULLS FIRST], [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] +Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#228] +Arguments: 100, [i_category#20 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#18 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#14 ASC NULLS FIRST, sumsales#29 ASC NULLS FIRST, rk#228 ASC NULLS FIRST], [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#228] ===== Subqueries ===== 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 fa822f3ac9ed7..e10a5ea0aff0b 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 @@ -191,234 +191,234 @@ Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 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))#27 as decimal(38,2)) AS sumsales#28] (26) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#29, isEmpty#30] +Output [10]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, s_store_id#36, sum#37, isEmpty#38] (27) HashAggregate [codegen id : 10] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#29, isEmpty#30] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#31] -Results [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#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))#31 AS sumsales#32] +Input [10]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, s_store_id#36, sum#37, isEmpty#38] +Keys [8]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, s_store_id#36] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#39 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#40 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#39 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#40 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#41] +Results [8]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#39 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#40 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#41 AS sumsales#42] (28) HashAggregate [codegen id : 10] -Input [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, sumsales#32] -Keys [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, sum#35, isEmpty#36] +Input [8]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sumsales#42] +Keys [7]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#43, isEmpty#44] +Results [9]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sum#45, isEmpty#46] (29) Exchange -Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, sum#35, isEmpty#36] -Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [9]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sum#45, isEmpty#46] +Arguments: hashpartitioning(i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, 5), ENSURE_REQUIREMENTS, [id=#47] (30) HashAggregate [codegen id : 11] -Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, sum#35, isEmpty#36] -Keys [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#38] -Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, null AS s_store_id#39, sum(sumsales#32)#38 AS sumsales#40] +Input [9]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sum#45, isEmpty#46] +Keys [7]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#48] +Results [9]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, null AS s_store_id#49, sum(sumsales#42)#48 AS sumsales#50] (31) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#41, isEmpty#42] +Output [10]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, d_moy#57, s_store_id#58, sum#59, isEmpty#60] (32) HashAggregate [codegen id : 16] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#41, isEmpty#42] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#43] -Results [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, 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))#43 AS sumsales#32] +Input [10]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, d_moy#57, s_store_id#58, sum#59, isEmpty#60] +Keys [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, d_moy#57, s_store_id#58] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#61 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#62 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#61 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#62 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#63] +Results [7]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#61 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#62 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#63 AS sumsales#42] (33) HashAggregate [codegen id : 16] -Input [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, sumsales#32] -Keys [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#44, isEmpty#45] -Results [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, sum#46, isEmpty#47] +Input [7]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sumsales#42] +Keys [6]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#64, isEmpty#65] +Results [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sum#66, isEmpty#67] (34) Exchange -Input [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, sum#46, isEmpty#47] -Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sum#66, isEmpty#67] +Arguments: hashpartitioning(i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, 5), ENSURE_REQUIREMENTS, [id=#68] (35) HashAggregate [codegen id : 17] -Input [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, sum#46, isEmpty#47] -Keys [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#49] -Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, null AS d_moy#50, null AS s_store_id#51, sum(sumsales#32)#49 AS sumsales#52] +Input [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sum#66, isEmpty#67] +Keys [6]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#69] +Results [9]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, null AS d_moy#70, null AS s_store_id#71, sum(sumsales#42)#69 AS sumsales#72] (36) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#53, isEmpty#54] +Output [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] (37) HashAggregate [codegen id : 22] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#53, isEmpty#54] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#55] -Results [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#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))#55 AS sumsales#32] +Input [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] +Keys [8]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#83 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#84 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#83 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#85] +Results [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#83 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#85 AS sumsales#42] (38) HashAggregate [codegen id : 22] -Input [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, sumsales#32] -Keys [5]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#56, isEmpty#57] -Results [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, sum#58, isEmpty#59] +Input [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sumsales#42] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#86, isEmpty#87] +Results [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] (39) Exchange -Input [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, sum#58, isEmpty#59] -Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Arguments: hashpartitioning(i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, 5), ENSURE_REQUIREMENTS, [id=#90] (40) HashAggregate [codegen id : 23] -Input [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, sum#58, isEmpty#59] -Keys [5]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#61] -Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, null AS d_qoy#62, null AS d_moy#63, null AS s_store_id#64, sum(sumsales#32)#61 AS sumsales#65] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#91] +Results [9]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, null AS d_qoy#92, null AS d_moy#93, null AS s_store_id#94, sum(sumsales#42)#91 AS sumsales#95] (41) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#66, isEmpty#67] +Output [10]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, d_year#100, d_qoy#101, d_moy#102, s_store_id#103, sum#104, isEmpty#105] (42) HashAggregate [codegen id : 28] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#66, isEmpty#67] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#68] -Results [5]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, 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))#68 AS sumsales#32] +Input [10]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, d_year#100, d_qoy#101, d_moy#102, s_store_id#103, sum#104, isEmpty#105] +Keys [8]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, d_year#100, d_qoy#101, d_moy#102, s_store_id#103] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#106 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#107 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#106 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#107 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#108] +Results [5]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#106 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#107 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#108 AS sumsales#42] (43) HashAggregate [codegen id : 28] -Input [5]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, sumsales#32] -Keys [4]: [i_category#19, i_class#18, i_brand#17, i_product_name#20] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#69, isEmpty#70] -Results [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, sum#71, isEmpty#72] +Input [5]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sumsales#42] +Keys [4]: [i_category#96, i_class#97, i_brand#98, i_product_name#99] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#109, isEmpty#110] +Results [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] (44) Exchange -Input [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, sum#71, isEmpty#72] -Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, i_product_name#20, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] +Arguments: hashpartitioning(i_category#96, i_class#97, i_brand#98, i_product_name#99, 5), ENSURE_REQUIREMENTS, [id=#113] (45) HashAggregate [codegen id : 29] -Input [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, sum#71, isEmpty#72] -Keys [4]: [i_category#19, i_class#18, i_brand#17, i_product_name#20] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#74] -Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, null AS d_year#75, null AS d_qoy#76, null AS d_moy#77, null AS s_store_id#78, sum(sumsales#32)#74 AS sumsales#79] +Input [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] +Keys [4]: [i_category#96, i_class#97, i_brand#98, i_product_name#99] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#114] +Results [9]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, null AS d_year#115, null AS d_qoy#116, null AS d_moy#117, null AS s_store_id#118, sum(sumsales#42)#114 AS sumsales#119] (46) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#80, isEmpty#81] +Output [10]: [i_category#120, i_class#121, i_brand#122, i_product_name#123, d_year#124, d_qoy#125, d_moy#126, s_store_id#127, sum#128, isEmpty#129] (47) HashAggregate [codegen id : 34] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#80, isEmpty#81] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#82] -Results [4]: [i_category#19, i_class#18, i_brand#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))#82 AS sumsales#32] +Input [10]: [i_category#120, i_class#121, i_brand#122, i_product_name#123, d_year#124, d_qoy#125, d_moy#126, s_store_id#127, sum#128, isEmpty#129] +Keys [8]: [i_category#120, i_class#121, i_brand#122, i_product_name#123, d_year#124, d_qoy#125, d_moy#126, s_store_id#127] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#130 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#131 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#130 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#131 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#132] +Results [4]: [i_category#120, i_class#121, i_brand#122, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#130 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#131 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#132 AS sumsales#42] (48) HashAggregate [codegen id : 34] -Input [4]: [i_category#19, i_class#18, i_brand#17, sumsales#32] -Keys [3]: [i_category#19, i_class#18, i_brand#17] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#83, isEmpty#84] -Results [5]: [i_category#19, i_class#18, i_brand#17, sum#85, isEmpty#86] +Input [4]: [i_category#120, i_class#121, i_brand#122, sumsales#42] +Keys [3]: [i_category#120, i_class#121, i_brand#122] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#133, isEmpty#134] +Results [5]: [i_category#120, i_class#121, i_brand#122, sum#135, isEmpty#136] (49) Exchange -Input [5]: [i_category#19, i_class#18, i_brand#17, sum#85, isEmpty#86] -Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, 5), ENSURE_REQUIREMENTS, [id=#87] +Input [5]: [i_category#120, i_class#121, i_brand#122, sum#135, isEmpty#136] +Arguments: hashpartitioning(i_category#120, i_class#121, i_brand#122, 5), ENSURE_REQUIREMENTS, [id=#137] (50) HashAggregate [codegen id : 35] -Input [5]: [i_category#19, i_class#18, i_brand#17, sum#85, isEmpty#86] -Keys [3]: [i_category#19, i_class#18, i_brand#17] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#88] -Results [9]: [i_category#19, i_class#18, i_brand#17, null AS i_product_name#89, null AS d_year#90, null AS d_qoy#91, null AS d_moy#92, null AS s_store_id#93, sum(sumsales#32)#88 AS sumsales#94] +Input [5]: [i_category#120, i_class#121, i_brand#122, sum#135, isEmpty#136] +Keys [3]: [i_category#120, i_class#121, i_brand#122] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#138] +Results [9]: [i_category#120, i_class#121, i_brand#122, null AS i_product_name#139, null AS d_year#140, null AS d_qoy#141, null AS d_moy#142, null AS s_store_id#143, sum(sumsales#42)#138 AS sumsales#144] (51) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#95, isEmpty#96] +Output [10]: [i_category#145, i_class#146, i_brand#147, i_product_name#148, d_year#149, d_qoy#150, d_moy#151, s_store_id#152, sum#153, isEmpty#154] (52) HashAggregate [codegen id : 40] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#95, isEmpty#96] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#97] -Results [3]: [i_category#19, i_class#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))#97 AS sumsales#32] +Input [10]: [i_category#145, i_class#146, i_brand#147, i_product_name#148, d_year#149, d_qoy#150, d_moy#151, s_store_id#152, sum#153, isEmpty#154] +Keys [8]: [i_category#145, i_class#146, i_brand#147, i_product_name#148, d_year#149, d_qoy#150, d_moy#151, s_store_id#152] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#155 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#156 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#155 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#156 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#157] +Results [3]: [i_category#145, i_class#146, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#155 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#156 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#157 AS sumsales#42] (53) HashAggregate [codegen id : 40] -Input [3]: [i_category#19, i_class#18, sumsales#32] -Keys [2]: [i_category#19, i_class#18] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#98, isEmpty#99] -Results [4]: [i_category#19, i_class#18, sum#100, isEmpty#101] +Input [3]: [i_category#145, i_class#146, sumsales#42] +Keys [2]: [i_category#145, i_class#146] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#158, isEmpty#159] +Results [4]: [i_category#145, i_class#146, sum#160, isEmpty#161] (54) Exchange -Input [4]: [i_category#19, i_class#18, sum#100, isEmpty#101] -Arguments: hashpartitioning(i_category#19, i_class#18, 5), ENSURE_REQUIREMENTS, [id=#102] +Input [4]: [i_category#145, i_class#146, sum#160, isEmpty#161] +Arguments: hashpartitioning(i_category#145, i_class#146, 5), ENSURE_REQUIREMENTS, [id=#162] (55) HashAggregate [codegen id : 41] -Input [4]: [i_category#19, i_class#18, sum#100, isEmpty#101] -Keys [2]: [i_category#19, i_class#18] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#103] -Results [9]: [i_category#19, i_class#18, null AS i_brand#104, null AS i_product_name#105, null AS d_year#106, null AS d_qoy#107, null AS d_moy#108, null AS s_store_id#109, sum(sumsales#32)#103 AS sumsales#110] +Input [4]: [i_category#145, i_class#146, sum#160, isEmpty#161] +Keys [2]: [i_category#145, i_class#146] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#163] +Results [9]: [i_category#145, i_class#146, null AS i_brand#164, null AS i_product_name#165, null AS d_year#166, null AS d_qoy#167, null AS d_moy#168, null AS s_store_id#169, sum(sumsales#42)#163 AS sumsales#170] (56) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#111, isEmpty#112] +Output [10]: [i_category#171, i_class#172, i_brand#173, i_product_name#174, d_year#175, d_qoy#176, d_moy#177, s_store_id#178, sum#179, isEmpty#180] (57) HashAggregate [codegen id : 46] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#111, isEmpty#112] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#113] -Results [2]: [i_category#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))#113 AS sumsales#32] +Input [10]: [i_category#171, i_class#172, i_brand#173, i_product_name#174, d_year#175, d_qoy#176, d_moy#177, s_store_id#178, sum#179, isEmpty#180] +Keys [8]: [i_category#171, i_class#172, i_brand#173, i_product_name#174, d_year#175, d_qoy#176, d_moy#177, s_store_id#178] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#181 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#182 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#181 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#182 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#183] +Results [2]: [i_category#171, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#181 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#182 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#183 AS sumsales#42] (58) HashAggregate [codegen id : 46] -Input [2]: [i_category#19, sumsales#32] -Keys [1]: [i_category#19] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#114, isEmpty#115] -Results [3]: [i_category#19, sum#116, isEmpty#117] +Input [2]: [i_category#171, sumsales#42] +Keys [1]: [i_category#171] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#184, isEmpty#185] +Results [3]: [i_category#171, sum#186, isEmpty#187] (59) Exchange -Input [3]: [i_category#19, sum#116, isEmpty#117] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, [id=#118] +Input [3]: [i_category#171, sum#186, isEmpty#187] +Arguments: hashpartitioning(i_category#171, 5), ENSURE_REQUIREMENTS, [id=#188] (60) HashAggregate [codegen id : 47] -Input [3]: [i_category#19, sum#116, isEmpty#117] -Keys [1]: [i_category#19] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#119] -Results [9]: [i_category#19, null AS i_class#120, null AS i_brand#121, null AS i_product_name#122, null AS d_year#123, null AS d_qoy#124, null AS d_moy#125, null AS s_store_id#126, sum(sumsales#32)#119 AS sumsales#127] +Input [3]: [i_category#171, sum#186, isEmpty#187] +Keys [1]: [i_category#171] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#189] +Results [9]: [i_category#171, null AS i_class#190, null AS i_brand#191, null AS i_product_name#192, null AS d_year#193, null AS d_qoy#194, null AS d_moy#195, null AS s_store_id#196, sum(sumsales#42)#189 AS sumsales#197] (61) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#128, isEmpty#129] +Output [10]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205, sum#206, isEmpty#207] (62) HashAggregate [codegen id : 52] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#128, isEmpty#129] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#130] -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))#130 AS sumsales#32] +Input [10]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205, sum#206, isEmpty#207] +Keys [8]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#208 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#209 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#208 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#209 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#210] +Results [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#208 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#209 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#210 AS sumsales#42] (63) HashAggregate [codegen id : 52] -Input [1]: [sumsales#32] +Input [1]: [sumsales#42] Keys: [] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#131, isEmpty#132] -Results [2]: [sum#133, isEmpty#134] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#211, isEmpty#212] +Results [2]: [sum#213, isEmpty#214] (64) Exchange -Input [2]: [sum#133, isEmpty#134] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#135] +Input [2]: [sum#213, isEmpty#214] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#215] (65) HashAggregate [codegen id : 53] -Input [2]: [sum#133, isEmpty#134] +Input [2]: [sum#213, isEmpty#214] Keys: [] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#136] -Results [9]: [null AS i_category#137, null AS i_class#138, null AS i_brand#139, null AS i_product_name#140, null AS d_year#141, null AS d_qoy#142, null AS d_moy#143, null AS s_store_id#144, sum(sumsales#32)#136 AS sumsales#145] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#216] +Results [9]: [null AS i_category#217, null AS i_class#218, null AS i_brand#219, null AS i_product_name#220, null AS d_year#221, null AS d_qoy#222, null AS d_moy#223, null AS s_store_id#224, sum(sumsales#42)#216 AS sumsales#225] (66) Union (67) Exchange Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, [id=#146] +Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, [id=#226] (68) Sort [codegen id : 54] Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28] @@ -426,15 +426,15 @@ Arguments: [i_category#19 ASC NULLS FIRST, sumsales#28 DESC NULLS LAST], false, (69) Window Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28] -Arguments: [rank(sumsales#28) windowspecdefinition(i_category#19, sumsales#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#147], [i_category#19], [sumsales#28 DESC NULLS LAST] +Arguments: [rank(sumsales#28) windowspecdefinition(i_category#19, sumsales#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#227], [i_category#19], [sumsales#28 DESC NULLS LAST] (70) Filter [codegen id : 55] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] -Condition : (rk#147 <= 100) +Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#227] +Condition : (rk#227 <= 100) (71) TakeOrderedAndProject -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] -Arguments: 100, [i_category#19 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_product_name#20 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#14 ASC NULLS FIRST, sumsales#28 ASC NULLS FIRST, rk#147 ASC NULLS FIRST], [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] +Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#227] +Arguments: 100, [i_category#19 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_product_name#20 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#14 ASC NULLS FIRST, sumsales#28 ASC NULLS FIRST, rk#227 ASC NULLS FIRST], [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#227] ===== Subqueries ===== 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 e024d06c710a7..35ed13e84c742 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 @@ -122,109 +122,109 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] (16) Filter [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_store_sk#11) (17) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#13] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] -Output [2]: [ss_store_sk#1, ss_net_profit#2] -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] +Output [2]: [ss_store_sk#11, ss_net_profit#12] +Input [4]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13, d_date_sk#14] (20) Scan parquet default.store -Output [2]: [s_store_sk#8, s_state#10] +Output [2]: [s_store_sk#15, s_state#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#8, s_state#10] +Input [2]: [s_store_sk#15, s_state#16] (22) Filter [codegen id : 3] -Input [2]: [s_store_sk#8, s_state#10] -Condition : isnotnull(s_store_sk#8) +Input [2]: [s_store_sk#15, s_state#16] +Condition : isnotnull(s_store_sk#15) (23) BroadcastExchange -Input [2]: [s_store_sk#8, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +Input [2]: [s_store_sk#15, s_state#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#8] +Left keys [1]: [ss_store_sk#11] +Right keys [1]: [s_store_sk#15] Join condition: None (25) Project [codegen id : 4] -Output [2]: [ss_net_profit#2, s_state#10] -Input [4]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_state#10] +Output [2]: [ss_net_profit#12, s_state#16] +Input [4]: [ss_store_sk#11, ss_net_profit#12, s_store_sk#15, s_state#16] (26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#2, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#12] -Results [2]: [s_state#10, sum#13] +Input [2]: [ss_net_profit#12, s_state#16] +Keys [1]: [s_state#16] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum#18] +Results [2]: [s_state#16, sum#19] (27) Exchange -Input [2]: [s_state#10, sum#13] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [2]: [s_state#16, sum#19] +Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, [id=#20] (28) HashAggregate [codegen id : 5] -Input [2]: [s_state#10, sum#13] -Keys [1]: [s_state#10] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#15] -Results [3]: [s_state#10 AS s_state#16, s_state#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#15,17,2) AS _w2#17] +Input [2]: [s_state#16, sum#19] +Keys [1]: [s_state#16] +Functions [1]: [sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#12))#21] +Results [3]: [s_state#16 AS s_state#22, s_state#16, MakeDecimal(sum(UnscaledValue(ss_net_profit#12))#21,17,2) AS _w2#23] (29) Exchange -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, [id=#24] (30) Sort [codegen id : 6] -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [s_state#10 ASC NULLS FIRST, _w2#17 DESC NULLS LAST], false, 0 +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: [s_state#16 ASC NULLS FIRST, _w2#23 DESC NULLS LAST], false, 0 (31) Window -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#10], [_w2#17 DESC NULLS LAST] +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: [rank(_w2#23) windowspecdefinition(s_state#16, _w2#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#25], [s_state#16], [_w2#23 DESC NULLS LAST] (32) Filter [codegen id : 7] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (ranking#19 <= 5) +Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] +Condition : (ranking#25 <= 5) (33) Project [codegen id : 7] -Output [1]: [s_state#16] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] +Output [1]: [s_state#22] +Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] (34) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#20] +Input [1]: [s_state#22] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#26] (35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#10] -Right keys [1]: [s_state#16] +Right keys [1]: [s_state#22] Join condition: None (36) BroadcastExchange Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] @@ -239,115 +239,115 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_county#9, s_state#10 Input [3]: [ss_net_profit#2, s_county#9, s_state#10] Keys [2]: [s_state#10, s_county#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#10, s_county#9, sum#23] +Aggregate Attributes [1]: [sum#28] +Results [3]: [s_state#10, s_county#9, sum#29] (40) Exchange -Input [3]: [s_state#10, s_county#9, sum#23] -Arguments: hashpartitioning(s_state#10, s_county#9, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [s_state#10, s_county#9, sum#29] +Arguments: hashpartitioning(s_state#10, s_county#9, 5), ENSURE_REQUIREMENTS, [id=#30] (41) HashAggregate [codegen id : 10] -Input [3]: [s_state#10, s_county#9, sum#23] +Input [3]: [s_state#10, s_county#9, sum#29] Keys [2]: [s_state#10, s_county#9] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) as decimal(27,2)) AS total_sum#26, s_state#10, s_county#9, 0 AS g_state#27, 0 AS g_county#28, 0 AS lochierarchy#29] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) as decimal(27,2)) AS total_sum#32, s_state#10, s_county#9, 0 AS g_state#33, 0 AS g_county#34, 0 AS lochierarchy#35] (42) ReusedExchange [Reuses operator id: unknown] -Output [3]: [s_state#10, s_county#9, sum#30] +Output [3]: [s_state#36, s_county#37, sum#38] (43) HashAggregate [codegen id : 20] -Input [3]: [s_state#10, s_county#9, sum#30] -Keys [2]: [s_state#10, s_county#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS total_sum#32, s_state#10] +Input [3]: [s_state#36, s_county#37, sum#38] +Keys [2]: [s_state#36, s_county#37] +Functions [1]: [sum(UnscaledValue(ss_net_profit#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#39))#40] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#39))#40,17,2) AS total_sum#41, s_state#36] (44) HashAggregate [codegen id : 20] -Input [2]: [total_sum#32, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(total_sum#32)] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [3]: [s_state#10, sum#35, isEmpty#36] +Input [2]: [total_sum#41, s_state#36] +Keys [1]: [s_state#36] +Functions [1]: [partial_sum(total_sum#41)] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [s_state#36, sum#44, isEmpty#45] (45) Exchange -Input [3]: [s_state#10, sum#35, isEmpty#36] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [s_state#36, sum#44, isEmpty#45] +Arguments: hashpartitioning(s_state#36, 5), ENSURE_REQUIREMENTS, [id=#46] (46) HashAggregate [codegen id : 21] -Input [3]: [s_state#10, sum#35, isEmpty#36] -Keys [1]: [s_state#10] -Functions [1]: [sum(total_sum#32)] -Aggregate Attributes [1]: [sum(total_sum#32)#38] -Results [6]: [sum(total_sum#32)#38 AS total_sum#39, s_state#10, null AS s_county#40, 0 AS g_state#41, 1 AS g_county#42, 1 AS lochierarchy#43] +Input [3]: [s_state#36, sum#44, isEmpty#45] +Keys [1]: [s_state#36] +Functions [1]: [sum(total_sum#41)] +Aggregate Attributes [1]: [sum(total_sum#41)#47] +Results [6]: [sum(total_sum#41)#47 AS total_sum#48, s_state#36, null AS s_county#49, 0 AS g_state#50, 1 AS g_county#51, 1 AS lochierarchy#52] (47) ReusedExchange [Reuses operator id: unknown] -Output [3]: [s_state#10, s_county#9, sum#44] +Output [3]: [s_state#53, s_county#54, sum#55] (48) HashAggregate [codegen id : 31] -Input [3]: [s_state#10, s_county#9, sum#44] -Keys [2]: [s_state#10, s_county#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#45] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#45,17,2) AS total_sum#32] +Input [3]: [s_state#53, s_county#54, sum#55] +Keys [2]: [s_state#53, s_county#54] +Functions [1]: [sum(UnscaledValue(ss_net_profit#56))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#56))#57] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#56))#57,17,2) AS total_sum#41] (49) HashAggregate [codegen id : 31] -Input [1]: [total_sum#32] +Input [1]: [total_sum#41] Keys: [] -Functions [1]: [partial_sum(total_sum#32)] -Aggregate Attributes [2]: [sum#46, isEmpty#47] -Results [2]: [sum#48, isEmpty#49] +Functions [1]: [partial_sum(total_sum#41)] +Aggregate Attributes [2]: [sum#58, isEmpty#59] +Results [2]: [sum#60, isEmpty#61] (50) Exchange -Input [2]: [sum#48, isEmpty#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [sum#60, isEmpty#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] (51) HashAggregate [codegen id : 32] -Input [2]: [sum#48, isEmpty#49] +Input [2]: [sum#60, isEmpty#61] Keys: [] -Functions [1]: [sum(total_sum#32)] -Aggregate Attributes [1]: [sum(total_sum#32)#51] -Results [6]: [sum(total_sum#32)#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] +Functions [1]: [sum(total_sum#41)] +Aggregate Attributes [1]: [sum(total_sum#41)#63] +Results [6]: [sum(total_sum#41)#63 AS total_sum#64, null AS s_state#65, null AS s_county#66, 1 AS g_state#67, 1 AS g_county#68, 2 AS lochierarchy#69] (52) Union (53) HashAggregate [codegen id : 33] -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Keys [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Keys [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Results [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] (54) Exchange -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Arguments: hashpartitioning(total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Arguments: hashpartitioning(total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35, 5), ENSURE_REQUIREMENTS, [id=#70] (55) HashAggregate [codegen id : 34] -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Keys [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Keys [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, CASE WHEN (g_county#28 = 0) THEN s_state#10 END AS _w0#59] +Results [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, CASE WHEN (g_county#34 = 0) THEN s_state#10 END AS _w0#71] (56) Exchange -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: hashpartitioning(lochierarchy#29, _w0#59, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: hashpartitioning(lochierarchy#35, _w0#71, 5), ENSURE_REQUIREMENTS, [id=#72] (57) Sort [codegen id : 35] -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: [lochierarchy#29 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: [lochierarchy#35 ASC NULLS FIRST, _w0#71 ASC NULLS FIRST, total_sum#32 DESC NULLS LAST], false, 0 (58) Window -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#29, _w0#59, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#29, _w0#59], [total_sum#26 DESC NULLS LAST] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: [rank(total_sum#32) windowspecdefinition(lochierarchy#35, _w0#71, total_sum#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#73], [lochierarchy#35, _w0#71], [total_sum#32 DESC NULLS LAST] (59) Project [codegen id : 36] -Output [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] -Input [6]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59, rank_within_parent#61] +Output [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] +Input [6]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71, rank_within_parent#73] (60) TakeOrderedAndProject -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] -Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#10 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] +Arguments: 100, [lochierarchy#35 DESC NULLS LAST, CASE WHEN (lochierarchy#35 = 0) THEN s_state#10 END ASC NULLS FIRST, rank_within_parent#73 ASC NULLS FIRST], [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] ===== Subqueries ===== @@ -358,6 +358,6 @@ ReusedExchange (61) (61) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#5] -Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#13 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 5ae5fd82839cc..df0ee68806062 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 @@ -122,109 +122,109 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] (16) Filter [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_store_sk#11) (17) Scan parquet default.store -Output [2]: [s_store_sk#8, s_state#10] +Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#8, s_state#10] +Input [2]: [s_store_sk#14, s_state#15] (19) Filter [codegen id : 2] -Input [2]: [s_store_sk#8, s_state#10] -Condition : isnotnull(s_store_sk#8) +Input [2]: [s_store_sk#14, s_state#15] +Condition : isnotnull(s_store_sk#14) (20) BroadcastExchange -Input [2]: [s_store_sk#8, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +Input [2]: [s_store_sk#14, s_state#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#16] (21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#8] +Left keys [1]: [ss_store_sk#11] +Right keys [1]: [s_store_sk#14] Join condition: None (22) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, ss_sold_date_sk#3, s_state#10] -Input [5]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, s_store_sk#8, s_state#10] +Output [3]: [ss_net_profit#12, ss_sold_date_sk#13, s_state#15] +Input [5]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13, s_store_sk#14, s_state#15] (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#17] (24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#13] +Right keys [1]: [d_date_sk#17] Join condition: None (25) Project [codegen id : 4] -Output [2]: [ss_net_profit#2, s_state#10] -Input [4]: [ss_net_profit#2, ss_sold_date_sk#3, s_state#10, d_date_sk#5] +Output [2]: [ss_net_profit#12, s_state#15] +Input [4]: [ss_net_profit#12, ss_sold_date_sk#13, s_state#15, d_date_sk#17] (26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#2, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#12] -Results [2]: [s_state#10, sum#13] +Input [2]: [ss_net_profit#12, s_state#15] +Keys [1]: [s_state#15] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum#18] +Results [2]: [s_state#15, sum#19] (27) Exchange -Input [2]: [s_state#10, sum#13] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [2]: [s_state#15, sum#19] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [id=#20] (28) HashAggregate [codegen id : 5] -Input [2]: [s_state#10, sum#13] -Keys [1]: [s_state#10] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#15] -Results [3]: [s_state#10 AS s_state#16, s_state#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#15,17,2) AS _w2#17] +Input [2]: [s_state#15, sum#19] +Keys [1]: [s_state#15] +Functions [1]: [sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#12))#21] +Results [3]: [s_state#15 AS s_state#22, s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#12))#21,17,2) AS _w2#23] (29) Exchange -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [id=#24] (30) Sort [codegen id : 6] -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [s_state#10 ASC NULLS FIRST, _w2#17 DESC NULLS LAST], false, 0 +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: [s_state#15 ASC NULLS FIRST, _w2#23 DESC NULLS LAST], false, 0 (31) Window -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#10], [_w2#17 DESC NULLS LAST] +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: [rank(_w2#23) windowspecdefinition(s_state#15, _w2#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#25], [s_state#15], [_w2#23 DESC NULLS LAST] (32) Filter [codegen id : 7] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (ranking#19 <= 5) +Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] +Condition : (ranking#25 <= 5) (33) Project [codegen id : 7] -Output [1]: [s_state#16] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] +Output [1]: [s_state#22] +Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] (34) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#20] +Input [1]: [s_state#22] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#26] (35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#10] -Right keys [1]: [s_state#16] +Right keys [1]: [s_state#22] Join condition: None (36) BroadcastExchange Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] @@ -239,115 +239,115 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_county#9, s_state#10 Input [3]: [ss_net_profit#2, s_county#9, s_state#10] Keys [2]: [s_state#10, s_county#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#10, s_county#9, sum#23] +Aggregate Attributes [1]: [sum#28] +Results [3]: [s_state#10, s_county#9, sum#29] (40) Exchange -Input [3]: [s_state#10, s_county#9, sum#23] -Arguments: hashpartitioning(s_state#10, s_county#9, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [s_state#10, s_county#9, sum#29] +Arguments: hashpartitioning(s_state#10, s_county#9, 5), ENSURE_REQUIREMENTS, [id=#30] (41) HashAggregate [codegen id : 10] -Input [3]: [s_state#10, s_county#9, sum#23] +Input [3]: [s_state#10, s_county#9, sum#29] Keys [2]: [s_state#10, s_county#9] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) as decimal(27,2)) AS total_sum#26, s_state#10, s_county#9, 0 AS g_state#27, 0 AS g_county#28, 0 AS lochierarchy#29] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) as decimal(27,2)) AS total_sum#32, s_state#10, s_county#9, 0 AS g_state#33, 0 AS g_county#34, 0 AS lochierarchy#35] (42) ReusedExchange [Reuses operator id: unknown] -Output [3]: [s_state#10, s_county#9, sum#30] +Output [3]: [s_state#36, s_county#37, sum#38] (43) HashAggregate [codegen id : 20] -Input [3]: [s_state#10, s_county#9, sum#30] -Keys [2]: [s_state#10, s_county#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS total_sum#32, s_state#10] +Input [3]: [s_state#36, s_county#37, sum#38] +Keys [2]: [s_state#36, s_county#37] +Functions [1]: [sum(UnscaledValue(ss_net_profit#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#39))#40] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#39))#40,17,2) AS total_sum#41, s_state#36] (44) HashAggregate [codegen id : 20] -Input [2]: [total_sum#32, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(total_sum#32)] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [3]: [s_state#10, sum#35, isEmpty#36] +Input [2]: [total_sum#41, s_state#36] +Keys [1]: [s_state#36] +Functions [1]: [partial_sum(total_sum#41)] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [s_state#36, sum#44, isEmpty#45] (45) Exchange -Input [3]: [s_state#10, sum#35, isEmpty#36] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [s_state#36, sum#44, isEmpty#45] +Arguments: hashpartitioning(s_state#36, 5), ENSURE_REQUIREMENTS, [id=#46] (46) HashAggregate [codegen id : 21] -Input [3]: [s_state#10, sum#35, isEmpty#36] -Keys [1]: [s_state#10] -Functions [1]: [sum(total_sum#32)] -Aggregate Attributes [1]: [sum(total_sum#32)#38] -Results [6]: [sum(total_sum#32)#38 AS total_sum#39, s_state#10, null AS s_county#40, 0 AS g_state#41, 1 AS g_county#42, 1 AS lochierarchy#43] +Input [3]: [s_state#36, sum#44, isEmpty#45] +Keys [1]: [s_state#36] +Functions [1]: [sum(total_sum#41)] +Aggregate Attributes [1]: [sum(total_sum#41)#47] +Results [6]: [sum(total_sum#41)#47 AS total_sum#48, s_state#36, null AS s_county#49, 0 AS g_state#50, 1 AS g_county#51, 1 AS lochierarchy#52] (47) ReusedExchange [Reuses operator id: unknown] -Output [3]: [s_state#10, s_county#9, sum#44] +Output [3]: [s_state#53, s_county#54, sum#55] (48) HashAggregate [codegen id : 31] -Input [3]: [s_state#10, s_county#9, sum#44] -Keys [2]: [s_state#10, s_county#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#45] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#45,17,2) AS total_sum#32] +Input [3]: [s_state#53, s_county#54, sum#55] +Keys [2]: [s_state#53, s_county#54] +Functions [1]: [sum(UnscaledValue(ss_net_profit#56))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#56))#57] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#56))#57,17,2) AS total_sum#41] (49) HashAggregate [codegen id : 31] -Input [1]: [total_sum#32] +Input [1]: [total_sum#41] Keys: [] -Functions [1]: [partial_sum(total_sum#32)] -Aggregate Attributes [2]: [sum#46, isEmpty#47] -Results [2]: [sum#48, isEmpty#49] +Functions [1]: [partial_sum(total_sum#41)] +Aggregate Attributes [2]: [sum#58, isEmpty#59] +Results [2]: [sum#60, isEmpty#61] (50) Exchange -Input [2]: [sum#48, isEmpty#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [sum#60, isEmpty#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] (51) HashAggregate [codegen id : 32] -Input [2]: [sum#48, isEmpty#49] +Input [2]: [sum#60, isEmpty#61] Keys: [] -Functions [1]: [sum(total_sum#32)] -Aggregate Attributes [1]: [sum(total_sum#32)#51] -Results [6]: [sum(total_sum#32)#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] +Functions [1]: [sum(total_sum#41)] +Aggregate Attributes [1]: [sum(total_sum#41)#63] +Results [6]: [sum(total_sum#41)#63 AS total_sum#64, null AS s_state#65, null AS s_county#66, 1 AS g_state#67, 1 AS g_county#68, 2 AS lochierarchy#69] (52) Union (53) HashAggregate [codegen id : 33] -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Keys [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Keys [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Results [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] (54) Exchange -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Arguments: hashpartitioning(total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Arguments: hashpartitioning(total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35, 5), ENSURE_REQUIREMENTS, [id=#70] (55) HashAggregate [codegen id : 34] -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Keys [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Keys [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, CASE WHEN (g_county#28 = 0) THEN s_state#10 END AS _w0#59] +Results [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, CASE WHEN (g_county#34 = 0) THEN s_state#10 END AS _w0#71] (56) Exchange -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: hashpartitioning(lochierarchy#29, _w0#59, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: hashpartitioning(lochierarchy#35, _w0#71, 5), ENSURE_REQUIREMENTS, [id=#72] (57) Sort [codegen id : 35] -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: [lochierarchy#29 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: [lochierarchy#35 ASC NULLS FIRST, _w0#71 ASC NULLS FIRST, total_sum#32 DESC NULLS LAST], false, 0 (58) Window -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#29, _w0#59, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#29, _w0#59], [total_sum#26 DESC NULLS LAST] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: [rank(total_sum#32) windowspecdefinition(lochierarchy#35, _w0#71, total_sum#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#73], [lochierarchy#35, _w0#71], [total_sum#32 DESC NULLS LAST] (59) Project [codegen id : 36] -Output [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] -Input [6]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59, rank_within_parent#61] +Output [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] +Input [6]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71, rank_within_parent#73] (60) TakeOrderedAndProject -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] -Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#10 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] +Arguments: 100, [lochierarchy#35 DESC NULLS LAST, CASE WHEN (lochierarchy#35 = 0) THEN s_state#10 END ASC NULLS FIRST, rank_within_parent#73 ASC NULLS FIRST], [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] ===== Subqueries ===== @@ -358,6 +358,6 @@ ReusedExchange (61) (61) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#5] -Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#13 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 035f89d3a04f7..7eea8040043d6 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 @@ -199,285 +199,285 @@ Input [2]: [customer_id#18, year_total#19] Arguments: [customer_id#18 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] +Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] +Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] (27) Filter [codegen id : 10] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_customer_sk#1) +Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Condition : isnotnull(ss_customer_sk#21) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#25, d_year#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#5, d_year#6] +Input [2]: [d_date_sk#25, d_year#26] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#5, d_year#6] -Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2002)) AND d_year#6 IN (2001,2002)) AND isnotnull(d_date_sk#5)) +Input [2]: [d_date_sk#25, d_year#26] +Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) (31) BroadcastExchange -Input [2]: [d_date_sk#5, d_year#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +Input [2]: [d_date_sk#25, d_year#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#23] +Right keys [1]: [d_date_sk#25] Join condition: None (33) Project [codegen id : 10] -Output [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Input [5]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6] +Output [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Input [5]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] (34) Exchange -Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Arguments: hashpartitioning(ss_customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#28] (35) Sort [codegen id : 11] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Arguments: [ss_customer_sk#21 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] (37) Sort [codegen id : 13] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ss_customer_sk#21] +Right keys [1]: [c_customer_sk#29] Join condition: None (39) Project [codegen id : 14] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] -Input [7]: [ss_customer_sk#1, ss_net_paid#2, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] +Input [7]: [ss_customer_sk#21, ss_net_paid#22, d_year#26, c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] (40) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#2))] -Aggregate Attributes [1]: [sum#24] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] +Aggregate Attributes [1]: [sum#33] +Results [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] (41) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] +Arguments: hashpartitioning(c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, 5), ENSURE_REQUIREMENTS, [id=#35] (42) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ss_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#2))#27] -Results [4]: [c_customer_id#10 AS customer_id#28, c_first_name#11 AS customer_first_name#29, c_last_name#12 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#2))#27,17,2) AS year_total#31] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] +Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#22))#36] +Results [4]: [c_customer_id#30 AS customer_id#37, c_first_name#31 AS customer_first_name#38, c_last_name#32 AS customer_last_name#39, MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#36,17,2) AS year_total#40] (43) Exchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: hashpartitioning(customer_id#28, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] +Arguments: hashpartitioning(customer_id#37, 5), ENSURE_REQUIREMENTS, [id=#41] (44) Sort [codegen id : 16] -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#28 ASC NULLS FIRST], false, 0 +Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] +Arguments: [customer_id#37 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#28] +Right keys [1]: [customer_id#37] Join condition: None (46) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] (48) Filter [codegen id : 19] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_bill_customer_sk#33) +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_bill_customer_sk#42) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#45, d_year#46] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (51) Project [codegen id : 19] -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] +Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Input [5]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] (52) Exchange -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#47] (53) Sort [codegen id : 20] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Arguments: [ws_bill_customer_sk#42 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] (55) Sort [codegen id : 22] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Arguments: [c_customer_sk#48 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#33] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ws_bill_customer_sk#42] +Right keys [1]: [c_customer_sk#48] Join condition: None (57) Project [codegen id : 23] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] +Input [7]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46, c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] (58) HashAggregate [codegen id : 23] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum#37] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] +Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] +Aggregate Attributes [1]: [sum#52] +Results [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] (59) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] +Arguments: hashpartitioning(c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, 5), ENSURE_REQUIREMENTS, [id=#54] (60) HashAggregate [codegen id : 24] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#40] -Results [2]: [c_customer_id#10 AS customer_id#41, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#40,17,2) AS year_total#42] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] +Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] +Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#55] +Results [2]: [c_customer_id#49 AS customer_id#56, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#55,17,2) AS year_total#57] (61) Filter [codegen id : 24] -Input [2]: [customer_id#41, year_total#42] -Condition : (isnotnull(year_total#42) AND (year_total#42 > 0.00)) +Input [2]: [customer_id#56, year_total#57] +Condition : (isnotnull(year_total#57) AND (year_total#57 > 0.00)) (62) Project [codegen id : 24] -Output [2]: [customer_id#41 AS customer_id#43, year_total#42 AS year_total#44] -Input [2]: [customer_id#41, year_total#42] +Output [2]: [customer_id#56 AS customer_id#58, year_total#57 AS year_total#59] +Input [2]: [customer_id#56, year_total#57] (63) Exchange -Input [2]: [customer_id#43, year_total#44] -Arguments: hashpartitioning(customer_id#43, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [2]: [customer_id#58, year_total#59] +Arguments: hashpartitioning(customer_id#58, 5), ENSURE_REQUIREMENTS, [id=#60] (64) Sort [codegen id : 25] -Input [2]: [customer_id#43, year_total#44] -Arguments: [customer_id#43 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#58, year_total#59] +Arguments: [customer_id#58 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#43] +Right keys [1]: [customer_id#58] Join condition: None (66) Project [codegen id : 26] -Output [7]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44] -Input [8]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#43, year_total#44] +Output [7]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59] +Input [8]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, customer_id#58, year_total#59] (67) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] (69) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_bill_customer_sk#33) +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_bill_customer_sk#61) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#64, d_year#65] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#63] +Right keys [1]: [d_date_sk#64] Join condition: None (72) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] +Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Input [5]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65] (73) Exchange -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Arguments: hashpartitioning(ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#66] (74) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Arguments: [ws_bill_customer_sk#61 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] (76) Sort [codegen id : 31] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] +Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#33] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ws_bill_customer_sk#61] +Right keys [1]: [c_customer_sk#67] Join condition: None (78) Project [codegen id : 32] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] +Input [7]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65, c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] (79) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum#47] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] +Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#62))] +Aggregate Attributes [1]: [sum#71] +Results [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] (80) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] +Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#73] (81) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#50] -Results [2]: [c_customer_id#10 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#50,17,2) AS year_total#52] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] +Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] +Functions [1]: [sum(UnscaledValue(ws_net_paid#62))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#62))#74] +Results [2]: [c_customer_id#68 AS customer_id#75, MakeDecimal(sum(UnscaledValue(ws_net_paid#62))#74,17,2) AS year_total#76] (82) Exchange -Input [2]: [customer_id#51, year_total#52] -Arguments: hashpartitioning(customer_id#51, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [2]: [customer_id#75, year_total#76] +Arguments: hashpartitioning(customer_id#75, 5), ENSURE_REQUIREMENTS, [id=#77] (83) Sort [codegen id : 34] -Input [2]: [customer_id#51, year_total#52] -Arguments: [customer_id#51 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#75, year_total#76] +Arguments: [customer_id#75 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#51] -Join condition: (CASE WHEN (year_total#44 > 0.00) THEN CheckOverflow((promote_precision(year_total#52) / promote_precision(year_total#44)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#31) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#75] +Join condition: (CASE WHEN (year_total#59 > 0.00) THEN CheckOverflow((promote_precision(year_total#76) / promote_precision(year_total#59)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#40) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) (85) Project [codegen id : 35] -Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Input [9]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44, customer_id#51, year_total#52] +Output [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] +Input [9]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59, customer_id#75, year_total#76] (86) TakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: 100, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] +Input [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] +Arguments: 100, [customer_first_name#38 ASC NULLS FIRST, customer_id#37 ASC NULLS FIRST, customer_last_name#39 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39] ===== Subqueries ===== @@ -488,15 +488,15 @@ ReusedExchange (87) (87) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#5, d_year#6] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#21 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#25, d_year#26] -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#21 +Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#24 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 44a902cf6da07..3ee6b56189888 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 @@ -171,282 +171,282 @@ Input [2]: [customer_id#17, year_total#18] Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) (20) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] (22) Filter [codegen id : 6] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Condition : (isnotnull(c_customer_sk#19) AND isnotnull(c_customer_id#20)) (23) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] (25) Filter [codegen id : 4] -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_customer_sk#5) +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_customer_sk#23) (26) BroadcastExchange -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#5] +Left keys [1]: [c_customer_sk#19] +Right keys [1]: [ss_customer_sk#23] Join condition: None (28) Project [codegen id : 6] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25] +Input [7]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#28, d_year#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#10, d_year#11] +Input [2]: [d_date_sk#28, d_year#29] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#10, d_year#11] -Condition : (((isnotnull(d_year#11) AND (d_year#11 = 2002)) AND d_year#11 IN (2001,2002)) AND isnotnull(d_date_sk#10)) +Input [2]: [d_date_sk#28, d_year#29] +Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) (32) BroadcastExchange -Input [2]: [d_date_sk#10, d_year#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Input [2]: [d_date_sk#28, d_year#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#25] +Right keys [1]: [d_date_sk#28] Join condition: None (34) Project [codegen id : 6] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] +Input [7]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#28, d_year#29] (35) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum#22] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] +Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] +Aggregate Attributes [1]: [sum#31] +Results [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] (36) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] +Arguments: hashpartitioning(c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, 5), ENSURE_REQUIREMENTS, [id=#33] (37) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#25] -Results [4]: [c_customer_id#2 AS customer_id#26, c_first_name#3 AS customer_first_name#27, c_last_name#4 AS customer_last_name#28, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#25,17,2) AS year_total#29] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] +Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] +Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#34] +Results [4]: [c_customer_id#20 AS customer_id#35, c_first_name#21 AS customer_first_name#36, c_last_name#22 AS customer_last_name#37, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#34,17,2) AS year_total#38] (38) BroadcastExchange -Input [4]: [customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] +Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#39] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#26] +Right keys [1]: [customer_id#35] Join condition: None (40) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] (42) Filter [codegen id : 10] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] +Condition : (isnotnull(c_customer_sk#40) AND isnotnull(c_customer_id#41)) (43) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] (45) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_bill_customer_sk#31) +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Condition : isnotnull(ws_bill_customer_sk#44) (46) BroadcastExchange -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#31] +Left keys [1]: [c_customer_sk#40] +Right keys [1]: [ws_bill_customer_sk#44] Join condition: None (48) Project [codegen id : 10] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46] +Input [7]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#48, d_year#49] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#46] +Right keys [1]: [d_date_sk#48] Join condition: None (51) Project [codegen id : 10] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] +Input [7]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46, d_date_sk#48, d_year#49] (52) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum#35] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] +Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#45))] +Aggregate Attributes [1]: [sum#50] +Results [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] (53) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] +Arguments: hashpartitioning(c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, 5), ENSURE_REQUIREMENTS, [id=#52] (54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#38] -Results [2]: [c_customer_id#2 AS customer_id#39, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#38,17,2) AS year_total#40] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] +Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] +Functions [1]: [sum(UnscaledValue(ws_net_paid#45))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#45))#53] +Results [2]: [c_customer_id#41 AS customer_id#54, MakeDecimal(sum(UnscaledValue(ws_net_paid#45))#53,17,2) AS year_total#55] (55) Filter [codegen id : 11] -Input [2]: [customer_id#39, year_total#40] -Condition : (isnotnull(year_total#40) AND (year_total#40 > 0.00)) +Input [2]: [customer_id#54, year_total#55] +Condition : (isnotnull(year_total#55) AND (year_total#55 > 0.00)) (56) Project [codegen id : 11] -Output [2]: [customer_id#39 AS customer_id#41, year_total#40 AS year_total#42] -Input [2]: [customer_id#39, year_total#40] +Output [2]: [customer_id#54 AS customer_id#56, year_total#55 AS year_total#57] +Input [2]: [customer_id#54, year_total#55] (57) BroadcastExchange -Input [2]: [customer_id#41, year_total#42] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#43] +Input [2]: [customer_id#56, year_total#57] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#58] (58) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#41] +Right keys [1]: [customer_id#56] Join condition: None (59) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42] -Input [8]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, customer_id#41, year_total#42] +Output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57] +Input [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, customer_id#56, year_total#57] (60) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] (62) Filter [codegen id : 14] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] +Condition : (isnotnull(c_customer_sk#59) AND isnotnull(c_customer_id#60)) (63) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] (65) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_bill_customer_sk#31) +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Condition : isnotnull(ws_bill_customer_sk#63) (66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#31] +Left keys [1]: [c_customer_sk#59] +Right keys [1]: [ws_bill_customer_sk#63] Join condition: None (68) Project [codegen id : 14] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65] +Input [7]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#67, d_year#68] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#65] +Right keys [1]: [d_date_sk#67] Join condition: None (71) Project [codegen id : 14] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] +Input [7]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65, d_date_sk#67, d_year#68] (72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum#45] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] +Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#64))] +Aggregate Attributes [1]: [sum#69] +Results [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] (73) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] +Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, 5), ENSURE_REQUIREMENTS, [id=#71] (74) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#48] -Results [2]: [c_customer_id#2 AS customer_id#49, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#48,17,2) AS year_total#50] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] +Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] +Functions [1]: [sum(UnscaledValue(ws_net_paid#64))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#64))#72] +Results [2]: [c_customer_id#60 AS customer_id#73, MakeDecimal(sum(UnscaledValue(ws_net_paid#64))#72,17,2) AS year_total#74] (75) BroadcastExchange -Input [2]: [customer_id#49, year_total#50] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#51] +Input [2]: [customer_id#73, year_total#74] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#75] (76) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#49] -Join condition: (CASE WHEN (year_total#42 > 0.00) THEN CheckOverflow((promote_precision(year_total#50) / promote_precision(year_total#42)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#29) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#73] +Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#57)), 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) (77) Project [codegen id : 16] -Output [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] -Input [9]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42, customer_id#49, year_total#50] +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#57, customer_id#73, year_total#74] (78) TakeOrderedAndProject -Input [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] -Arguments: 100, [customer_first_name#27 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST, customer_last_name#28 ASC NULLS FIRST], [customer_id#26, customer_first_name#27, customer_last_name#28] +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] ===== Subqueries ===== @@ -457,15 +457,15 @@ ReusedExchange (79) (79) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#10, d_year#11] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#19 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#28, d_year#29] -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#19 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#26 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 7ecbe2e51c9b4..56998d9197835 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 @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#31] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#36, d_year#37] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#36] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] -Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (42) Exchange -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_item_sk#47) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ws_item_sk#47] +Right keys [1]: [i_item_sk#52] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#57, d_year#58] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#57] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] (55) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] -Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (61) Exchange -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] +Right keys [2]: [wr_order_number#61, wr_item_sk#60] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] +Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#55, sum#56] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Aggregate Attributes [2]: [sum#69, sum#70] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] +Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Condition : isnotnull(cs_item_sk#79) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [cs_item_sk#79] +Right keys [1]: [i_item_sk#85] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] +Input [2]: [d_date_sk#90, d_year#91] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] -Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) +Input [2]: [d_date_sk#90, d_year#91] +Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) (83) BroadcastExchange -Input [2]: [d_date_sk#71, d_year#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [d_date_sk#90, d_year#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [cs_sold_date_sk#83] +Right keys [1]: [d_date_sk#90] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] (86) Exchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] -Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#2, cs_item_sk#1] -Right keys [2]: [cr_order_number#19, cr_item_sk#18] +Left keys [2]: [cs_order_number#80, cs_item_sk#79] +Right keys [2]: [cr_order_number#95, cr_item_sk#94] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Condition : isnotnull(ss_item_sk#26) +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Condition : isnotnull(ss_item_sk#98) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#98] +Right keys [1]: [i_item_sk#103] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#108, d_year#109] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ss_sold_date_sk#102] +Right keys [1]: [d_date_sk#108] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] (101) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] +Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] +Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Condition : isnotnull(ws_item_sk#116) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ws_item_sk#116] +Right keys [1]: [i_item_sk#121] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#126, d_year#127] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ws_sold_date_sk#120] +Right keys [1]: [d_date_sk#126] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] (116) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] +Right keys [2]: [wr_order_number#130, wr_item_sk#129] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] +Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Aggregate Attributes [2]: [sum#135, sum#136] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] (127) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] +Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] (129) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] (130) Sort [codegen id : 50] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 (131) 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#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] (133) TakeOrderedAndProject -Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] -Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST, sales_amt_diff#97 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST, sales_amt_diff#150 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 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 7ecbe2e51c9b4..56998d9197835 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 @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#31] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#36, d_year#37] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#36] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] -Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (42) Exchange -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_item_sk#47) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ws_item_sk#47] +Right keys [1]: [i_item_sk#52] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#57, d_year#58] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#57] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] (55) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] -Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (61) Exchange -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] +Right keys [2]: [wr_order_number#61, wr_item_sk#60] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] +Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#55, sum#56] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Aggregate Attributes [2]: [sum#69, sum#70] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] +Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Condition : isnotnull(cs_item_sk#79) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [cs_item_sk#79] +Right keys [1]: [i_item_sk#85] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] +Input [2]: [d_date_sk#90, d_year#91] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] -Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) +Input [2]: [d_date_sk#90, d_year#91] +Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) (83) BroadcastExchange -Input [2]: [d_date_sk#71, d_year#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [d_date_sk#90, d_year#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [cs_sold_date_sk#83] +Right keys [1]: [d_date_sk#90] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] (86) Exchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] -Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#2, cs_item_sk#1] -Right keys [2]: [cr_order_number#19, cr_item_sk#18] +Left keys [2]: [cs_order_number#80, cs_item_sk#79] +Right keys [2]: [cr_order_number#95, cr_item_sk#94] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Condition : isnotnull(ss_item_sk#26) +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Condition : isnotnull(ss_item_sk#98) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#98] +Right keys [1]: [i_item_sk#103] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#108, d_year#109] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ss_sold_date_sk#102] +Right keys [1]: [d_date_sk#108] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] (101) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] +Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] +Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Condition : isnotnull(ws_item_sk#116) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ws_item_sk#116] +Right keys [1]: [i_item_sk#121] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#126, d_year#127] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ws_sold_date_sk#120] +Right keys [1]: [d_date_sk#126] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] (116) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] +Right keys [2]: [wr_order_number#130, wr_item_sk#129] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] +Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Aggregate Attributes [2]: [sum#135, sum#136] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] (127) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] +Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] (129) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] (130) Sort [codegen id : 50] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 (131) 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#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] (133) TakeOrderedAndProject -Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] -Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST, sales_amt_diff#97 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST, sales_amt_diff#150 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 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 e6ade3880e2d4..422443509b417 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 @@ -222,38 +222,38 @@ Output [4]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_s Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25] (26) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#26] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Right keys [1]: [cast(d_date_sk#26 as bigint)] Join condition: None (28) Project [codegen id : 6] Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#6] +Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#26] (29) HashAggregate [codegen id : 6] Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] Keys [1]: [s_store_sk#25] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#26, sum#27] -Results [3]: [s_store_sk#25, sum#28, sum#29] +Aggregate Attributes [2]: [sum#27, sum#28] +Results [3]: [s_store_sk#25, sum#29, sum#30] (30) Exchange -Input [3]: [s_store_sk#25, sum#28, sum#29] -Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [s_store_sk#25, sum#29, sum#30] +Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#25, sum#28, sum#29] +Input [3]: [s_store_sk#25, sum#29, sum#30] Keys [1]: [s_store_sk#25] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] -Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] +Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] (32) BroadcastExchange -Input [3]: [s_store_sk#25, returns#33, profit_loss#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [3]: [s_store_sk#25, returns#34, profit_loss#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] @@ -261,341 +261,341 @@ Right keys [1]: [s_store_sk#25] Join condition: None (34) Project [codegen id : 8] -Output [5]: [store channel AS channel#36, s_store_sk#9 AS id#37, sales#18, coalesce(returns#33, 0.00) AS returns#38, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#39] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] +Output [5]: [store channel AS channel#37, s_store_sk#9 AS id#38, sales#18, coalesce(returns#34, 0.00) AS returns#39, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#40] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#34, profit_loss#35] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#45] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#43] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cs_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] +Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum#44, sum#45] -Results [3]: [cs_call_center_sk#40, sum#46, sum#47] +Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum#46, sum#47] +Results [3]: [cs_call_center_sk#41, sum#48, sum#49] (41) Exchange -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] (42) HashAggregate [codegen id : 14] -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] -Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] +Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] (43) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +Output [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#55), dynamicpruningexpression(cr_returned_date_sk#55 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#5)] ReadSchema: struct (44) ColumnarToRow [codegen id : 12] -Input [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +Input [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] (45) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#58] (46) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cr_returned_date_sk#55] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#57] +Right keys [1]: [d_date_sk#58] Join condition: None (47) Project [codegen id : 12] -Output [2]: [cr_return_amount#53, cr_net_loss#54] -Input [4]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55, d_date_sk#6] +Output [2]: [cr_return_amount#55, cr_net_loss#56] +Input [4]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57, d_date_sk#58] (48) HashAggregate [codegen id : 12] -Input [2]: [cr_return_amount#53, cr_net_loss#54] +Input [2]: [cr_return_amount#55, cr_net_loss#56] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#53)), partial_sum(UnscaledValue(cr_net_loss#54))] -Aggregate Attributes [2]: [sum#56, sum#57] -Results [2]: [sum#58, sum#59] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#55)), partial_sum(UnscaledValue(cr_net_loss#56))] +Aggregate Attributes [2]: [sum#59, sum#60] +Results [2]: [sum#61, sum#62] (49) Exchange -Input [2]: [sum#58, sum#59] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#60] +Input [2]: [sum#61, sum#62] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] (50) HashAggregate [codegen id : 13] -Input [2]: [sum#58, sum#59] +Input [2]: [sum#61, sum#62] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#53)), sum(UnscaledValue(cr_net_loss#54))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#53))#61, sum(UnscaledValue(cr_net_loss#54))#62] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#53))#61,17,2) AS returns#63, MakeDecimal(sum(UnscaledValue(cr_net_loss#54))#62,17,2) AS profit_loss#64] +Functions [2]: [sum(UnscaledValue(cr_return_amount#55)), sum(UnscaledValue(cr_net_loss#56))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#55))#64, sum(UnscaledValue(cr_net_loss#56))#65] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#55))#64,17,2) AS returns#66, MakeDecimal(sum(UnscaledValue(cr_net_loss#56))#65,17,2) AS profit_loss#67] (51) BroadcastExchange -Input [2]: [returns#63, profit_loss#64] -Arguments: IdentityBroadcastMode, [id=#65] +Input [2]: [returns#66, profit_loss#67] +Arguments: IdentityBroadcastMode, [id=#68] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#66, cs_call_center_sk#40 AS id#67, sales#51, returns#63, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#64 as decimal(18,2)))), DecimalType(18,2), true) AS profit#68] -Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#63, profit_loss#64] +Output [5]: [catalog channel AS channel#69, cs_call_center_sk#41 AS id#70, sales#53, returns#66, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#67 as decimal(18,2)))), DecimalType(18,2), true) AS profit#71] +Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#66, profit_loss#67] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] -Condition : isnotnull(ws_web_page_sk#69) +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Condition : isnotnull(ws_web_page_sk#72) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#76] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#72] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#75] +Right keys [1]: [d_date_sk#76] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] -Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] +Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] +Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#73] +Output [1]: [wp_web_page_sk#77] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#73] +Input [1]: [wp_web_page_sk#77] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#73] -Condition : isnotnull(wp_web_page_sk#73) +Input [1]: [wp_web_page_sk#77] +Condition : isnotnull(wp_web_page_sk#77) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [1]: [wp_web_page_sk#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#69] -Right keys [1]: [wp_web_page_sk#73] +Left keys [1]: [ws_web_page_sk#72] +Right keys [1]: [wp_web_page_sk#77] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum#75, sum#76] -Results [3]: [wp_web_page_sk#73, sum#77, sum#78] +Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum#79, sum#80] +Results [3]: [wp_web_page_sk#77, sum#81, sum#82] (67) Exchange -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] -Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] +Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : isnotnull(wr_web_page_sk#84) +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Condition : isnotnull(wr_web_page_sk#88) (72) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#88] +Output [1]: [wp_web_page_sk#92] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#84] -Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] +Left keys [1]: [wr_web_page_sk#88] +Right keys [1]: [cast(wp_web_page_sk#92 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [4]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] -Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] +Output [4]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] +Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] (75) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#93] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#87] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#91] +Right keys [1]: [cast(d_date_sk#93 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Input [5]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88, d_date_sk#6] +Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] +Input [5]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92, d_date_sk#93] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum#89, sum#90] -Results [3]: [wp_web_page_sk#88, sum#91, sum#92] +Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] +Keys [1]: [wp_web_page_sk#92] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum#94, sum#95] +Results [3]: [wp_web_page_sk#92, sum#96, sum#97] (79) Exchange -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [3]: [wp_web_page_sk#92, sum#96, sum#97] +Arguments: hashpartitioning(wp_web_page_sk#92, 5), ENSURE_REQUIREMENTS, [id=#98] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] -Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] +Input [3]: [wp_web_page_sk#92, sum#96, sum#97] +Keys [1]: [wp_web_page_sk#92] +Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] +Results [3]: [wp_web_page_sk#92, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] +Input [3]: [wp_web_page_sk#92, returns#101, profit_loss#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#73] -Right keys [1]: [wp_web_page_sk#88] +Left keys [1]: [wp_web_page_sk#77] +Right keys [1]: [wp_web_page_sk#92] Join condition: None (83) Project [codegen id : 22] -Output [5]: [web channel AS channel#99, wp_web_page_sk#73 AS id#100, sales#82, coalesce(returns#96, 0.00) AS returns#101, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#102] -Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] +Output [5]: [web channel AS channel#104, wp_web_page_sk#77 AS id#105, sales#86, coalesce(returns#101, 0.00) AS returns#106, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#107] +Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#92, returns#101, profit_loss#102] (84) Union (85) HashAggregate [codegen id : 23] -Input [5]: [channel#36, id#37, sales#18, returns#38, profit#39] -Keys [2]: [channel#36, id#37] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#38), partial_sum(profit#39)] -Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Results [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Input [5]: [channel#37, id#38, sales#18, returns#39, profit#40] +Keys [2]: [channel#37, id#38] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#39), partial_sum(profit#40)] +Aggregate Attributes [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +Results [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] (86) Exchange -Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#36, id#37, 5), ENSURE_REQUIREMENTS, [id=#115] +Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [id=#120] (87) HashAggregate [codegen id : 24] -Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#18)#116, sum(returns#38)#117, sum(profit#39)#118] -Results [5]: [channel#36, id#37, cast(sum(sales#18)#116 as decimal(37,2)) AS sales#119, cast(sum(returns#38)#117 as decimal(37,2)) AS returns#120, cast(sum(profit#39)#118 as decimal(38,2)) AS profit#121] +Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#40)] +Aggregate Attributes [3]: [sum(sales#18)#121, sum(returns#39)#122, sum(profit#40)#123] +Results [5]: [channel#37, id#38, cast(sum(sales#18)#121 as decimal(37,2)) AS sales#124, cast(sum(returns#39)#122 as decimal(37,2)) AS returns#125, cast(sum(profit#40)#123 as decimal(38,2)) AS profit#126] (88) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] +Output [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] (89) HashAggregate [codegen id : 48] -Input [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#128)] -Aggregate Attributes [3]: [sum(sales#18)#129, sum(returns#38)#130, sum(profit#128)#131] -Results [4]: [channel#36, sum(sales#18)#129 AS sales#132, sum(returns#38)#130 AS returns#133, sum(profit#128)#131 AS profit#134] +Input [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#133)] +Aggregate Attributes [3]: [sum(sales#18)#134, sum(returns#39)#135, sum(profit#133)#136] +Results [4]: [channel#37, sum(sales#18)#134 AS sales#137, sum(returns#39)#135 AS returns#138, sum(profit#133)#136 AS profit#139] (90) HashAggregate [codegen id : 48] -Input [4]: [channel#36, sales#132, returns#133, profit#134] -Keys [1]: [channel#36] -Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] -Aggregate Attributes [6]: [sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Results [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Input [4]: [channel#37, sales#137, returns#138, profit#139] +Keys [1]: [channel#37] +Functions [3]: [partial_sum(sales#137), partial_sum(returns#138), partial_sum(profit#139)] +Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Results [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] (91) Exchange -Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Arguments: hashpartitioning(channel#36, 5), ENSURE_REQUIREMENTS, [id=#147] +Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [id=#152] (92) HashAggregate [codegen id : 49] -Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Keys [1]: [channel#36] -Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] -Aggregate Attributes [3]: [sum(sales#132)#148, sum(returns#133)#149, sum(profit#134)#150] -Results [5]: [channel#36, null AS id#151, sum(sales#132)#148 AS sales#152, sum(returns#133)#149 AS returns#153, sum(profit#134)#150 AS profit#154] +Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Keys [1]: [channel#37] +Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] +Aggregate Attributes [3]: [sum(sales#137)#153, sum(returns#138)#154, sum(profit#139)#155] +Results [5]: [channel#37, null AS id#156, sum(sales#137)#153 AS sales#157, sum(returns#138)#154 AS returns#158, sum(profit#139)#155 AS profit#159] (93) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] +Output [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] (94) HashAggregate [codegen id : 73] -Input [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#161)] -Aggregate Attributes [3]: [sum(sales#18)#162, sum(returns#38)#163, sum(profit#161)#164] -Results [3]: [sum(sales#18)#162 AS sales#132, sum(returns#38)#163 AS returns#133, sum(profit#161)#164 AS profit#134] +Input [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#166)] +Aggregate Attributes [3]: [sum(sales#18)#167, sum(returns#39)#168, sum(profit#166)#169] +Results [3]: [sum(sales#18)#167 AS sales#137, sum(returns#39)#168 AS returns#138, sum(profit#166)#169 AS profit#139] (95) HashAggregate [codegen id : 73] -Input [3]: [sales#132, returns#133, profit#134] +Input [3]: [sales#137, returns#138, profit#139] Keys: [] -Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] -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#137), partial_sum(returns#138), partial_sum(profit#139)] +Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Results [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] (96) Exchange -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#177] +Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#182] (97) HashAggregate [codegen id : 74] -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Keys: [] -Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] -Aggregate Attributes [3]: [sum(sales#132)#178, sum(returns#133)#179, sum(profit#134)#180] -Results [5]: [null AS channel#181, null AS id#182, sum(sales#132)#178 AS sales#183, sum(returns#133)#179 AS returns#184, sum(profit#134)#180 AS profit#185] +Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] +Aggregate Attributes [3]: [sum(sales#137)#183, sum(returns#138)#184, sum(profit#139)#185] +Results [5]: [null AS channel#186, null AS id#187, sum(sales#137)#183 AS sales#188, sum(returns#138)#184 AS returns#189, sum(profit#139)#185 AS profit#190] (98) Union (99) HashAggregate [codegen id : 75] -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] (100) Exchange -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Arguments: hashpartitioning(channel#36, id#37, sales#119, returns#120, profit#121, 5), ENSURE_REQUIREMENTS, [id=#186] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Arguments: hashpartitioning(channel#37, id#38, sales#124, returns#125, profit#126, 5), ENSURE_REQUIREMENTS, [id=#191] (101) HashAggregate [codegen id : 76] -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] (102) TakeOrderedAndProject -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Arguments: 100, [channel#36 ASC NULLS FIRST, id#37 ASC NULLS FIRST], [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#124, returns#125, profit#126] ===== Subqueries ===== @@ -611,14 +611,14 @@ ReusedExchange (104) (104) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#26] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#55 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 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 0ec2ed7cca356..705277b65ff0a 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 @@ -210,392 +210,392 @@ Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_s Condition : isnotnull(sr_store_sk#20) (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#25] (24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None (25) Project [codegen id : 6] Output [3]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22] -Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#6] +Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#25] (26) ReusedExchange [Reuses operator id: 14] -Output [1]: [s_store_sk#25] +Output [1]: [s_store_sk#26] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_store_sk#20] -Right keys [1]: [cast(s_store_sk#25 as bigint)] +Right keys [1]: [cast(s_store_sk#26 as bigint)] Join condition: None (28) Project [codegen id : 6] -Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#25] +Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] +Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#26] (29) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Keys [1]: [s_store_sk#25] +Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] +Keys [1]: [s_store_sk#26] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#26, sum#27] -Results [3]: [s_store_sk#25, sum#28, sum#29] +Aggregate Attributes [2]: [sum#27, sum#28] +Results [3]: [s_store_sk#26, sum#29, sum#30] (30) Exchange -Input [3]: [s_store_sk#25, sum#28, sum#29] -Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [s_store_sk#26, sum#29, sum#30] +Arguments: hashpartitioning(s_store_sk#26, 5), ENSURE_REQUIREMENTS, [id=#31] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#25, sum#28, sum#29] -Keys [1]: [s_store_sk#25] +Input [3]: [s_store_sk#26, sum#29, sum#30] +Keys [1]: [s_store_sk#26] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] -Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] +Results [3]: [s_store_sk#26, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] (32) BroadcastExchange -Input [3]: [s_store_sk#25, returns#33, profit_loss#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [3]: [s_store_sk#26, returns#34, profit_loss#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] -Right keys [1]: [s_store_sk#25] +Right keys [1]: [s_store_sk#26] Join condition: None (34) Project [codegen id : 8] -Output [5]: [store channel AS channel#36, s_store_sk#9 AS id#37, sales#18, coalesce(returns#33, 0.00) AS returns#38, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#39] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] +Output [5]: [store channel AS channel#37, s_store_sk#9 AS id#38, sales#18, coalesce(returns#34, 0.00) AS returns#39, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#40] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#26, returns#34, profit_loss#35] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#45] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#43] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cs_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] +Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum#44, sum#45] -Results [3]: [cs_call_center_sk#40, sum#46, sum#47] +Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum#46, sum#47] +Results [3]: [cs_call_center_sk#41, sum#48, sum#49] (41) Exchange -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] -Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] +Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] (43) BroadcastExchange -Input [3]: [cs_call_center_sk#40, sales#51, profit#52] -Arguments: IdentityBroadcastMode, [id=#53] +Input [3]: [cs_call_center_sk#41, sales#53, profit#54] +Arguments: IdentityBroadcastMode, [id=#55] (44) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Output [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#58), dynamicpruningexpression(cr_returned_date_sk#58 IN dynamicpruning#5)] ReadSchema: struct (45) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Input [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] (46) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#59] (47) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#56] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#58] +Right keys [1]: [d_date_sk#59] Join condition: None (48) Project [codegen id : 13] -Output [2]: [cr_return_amount#54, cr_net_loss#55] -Input [4]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56, d_date_sk#6] +Output [2]: [cr_return_amount#56, cr_net_loss#57] +Input [4]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58, d_date_sk#59] (49) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#54, cr_net_loss#55] +Input [2]: [cr_return_amount#56, cr_net_loss#57] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#54)), partial_sum(UnscaledValue(cr_net_loss#55))] -Aggregate Attributes [2]: [sum#57, sum#58] -Results [2]: [sum#59, sum#60] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#56)), partial_sum(UnscaledValue(cr_net_loss#57))] +Aggregate Attributes [2]: [sum#60, sum#61] +Results [2]: [sum#62, sum#63] (50) Exchange -Input [2]: [sum#59, sum#60] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] +Input [2]: [sum#62, sum#63] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#64] (51) HashAggregate -Input [2]: [sum#59, sum#60] +Input [2]: [sum#62, sum#63] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#54)), sum(UnscaledValue(cr_net_loss#55))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#54))#62, sum(UnscaledValue(cr_net_loss#55))#63] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#54))#62,17,2) AS returns#64, MakeDecimal(sum(UnscaledValue(cr_net_loss#55))#63,17,2) AS profit_loss#65] +Functions [2]: [sum(UnscaledValue(cr_return_amount#56)), sum(UnscaledValue(cr_net_loss#57))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#56))#65, sum(UnscaledValue(cr_net_loss#57))#66] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#56))#65,17,2) AS returns#67, MakeDecimal(sum(UnscaledValue(cr_net_loss#57))#66,17,2) AS profit_loss#68] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#66, cs_call_center_sk#40 AS id#67, sales#51, returns#64, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#65 as decimal(18,2)))), DecimalType(18,2), true) AS profit#68] -Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#64, profit_loss#65] +Output [5]: [catalog channel AS channel#69, cs_call_center_sk#41 AS id#70, sales#53, returns#67, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#68 as decimal(18,2)))), DecimalType(18,2), true) AS profit#71] +Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#67, profit_loss#68] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] -Condition : isnotnull(ws_web_page_sk#69) +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Condition : isnotnull(ws_web_page_sk#72) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#76] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#72] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#75] +Right keys [1]: [d_date_sk#76] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] -Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] +Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] +Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#73] +Output [1]: [wp_web_page_sk#77] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#73] +Input [1]: [wp_web_page_sk#77] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#73] -Condition : isnotnull(wp_web_page_sk#73) +Input [1]: [wp_web_page_sk#77] +Condition : isnotnull(wp_web_page_sk#77) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [1]: [wp_web_page_sk#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#69] -Right keys [1]: [wp_web_page_sk#73] +Left keys [1]: [ws_web_page_sk#72] +Right keys [1]: [wp_web_page_sk#77] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum#75, sum#76] -Results [3]: [wp_web_page_sk#73, sum#77, sum#78] +Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum#79, sum#80] +Results [3]: [wp_web_page_sk#77, sum#81, sum#82] (67) Exchange -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] -Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] +Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : isnotnull(wr_web_page_sk#84) +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Condition : isnotnull(wr_web_page_sk#88) (72) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#92] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#87] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#91] +Right keys [1]: [cast(d_date_sk#92 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86] -Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, d_date_sk#6] +Output [3]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90] +Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, d_date_sk#92] (75) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#88] +Output [1]: [wp_web_page_sk#93] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#84] -Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] +Left keys [1]: [wr_web_page_sk#88] +Right keys [1]: [cast(wp_web_page_sk#93 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum#89, sum#90] -Results [3]: [wp_web_page_sk#88, sum#91, sum#92] +Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] +Keys [1]: [wp_web_page_sk#93] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum#94, sum#95] +Results [3]: [wp_web_page_sk#93, sum#96, sum#97] (79) Exchange -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [3]: [wp_web_page_sk#93, sum#96, sum#97] +Arguments: hashpartitioning(wp_web_page_sk#93, 5), ENSURE_REQUIREMENTS, [id=#98] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] -Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] +Input [3]: [wp_web_page_sk#93, sum#96, sum#97] +Keys [1]: [wp_web_page_sk#93] +Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] +Results [3]: [wp_web_page_sk#93, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] +Input [3]: [wp_web_page_sk#93, returns#101, profit_loss#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#73] -Right keys [1]: [wp_web_page_sk#88] +Left keys [1]: [wp_web_page_sk#77] +Right keys [1]: [wp_web_page_sk#93] Join condition: None (83) Project [codegen id : 22] -Output [5]: [web channel AS channel#99, wp_web_page_sk#73 AS id#100, sales#82, coalesce(returns#96, 0.00) AS returns#101, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#102] -Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] +Output [5]: [web channel AS channel#104, wp_web_page_sk#77 AS id#105, sales#86, coalesce(returns#101, 0.00) AS returns#106, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#107] +Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#93, returns#101, profit_loss#102] (84) Union (85) HashAggregate [codegen id : 23] -Input [5]: [channel#36, id#37, sales#18, returns#38, profit#39] -Keys [2]: [channel#36, id#37] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#38), partial_sum(profit#39)] -Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Results [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Input [5]: [channel#37, id#38, sales#18, returns#39, profit#40] +Keys [2]: [channel#37, id#38] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#39), partial_sum(profit#40)] +Aggregate Attributes [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +Results [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] (86) Exchange -Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#36, id#37, 5), ENSURE_REQUIREMENTS, [id=#115] +Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [id=#120] (87) HashAggregate [codegen id : 24] -Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#18)#116, sum(returns#38)#117, sum(profit#39)#118] -Results [5]: [channel#36, id#37, cast(sum(sales#18)#116 as decimal(37,2)) AS sales#119, cast(sum(returns#38)#117 as decimal(37,2)) AS returns#120, cast(sum(profit#39)#118 as decimal(38,2)) AS profit#121] +Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#40)] +Aggregate Attributes [3]: [sum(sales#18)#121, sum(returns#39)#122, sum(profit#40)#123] +Results [5]: [channel#37, id#38, cast(sum(sales#18)#121 as decimal(37,2)) AS sales#124, cast(sum(returns#39)#122 as decimal(37,2)) AS returns#125, cast(sum(profit#40)#123 as decimal(38,2)) AS profit#126] (88) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] +Output [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] (89) HashAggregate [codegen id : 48] -Input [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#128)] -Aggregate Attributes [3]: [sum(sales#18)#129, sum(returns#38)#130, sum(profit#128)#131] -Results [4]: [channel#36, sum(sales#18)#129 AS sales#132, sum(returns#38)#130 AS returns#133, sum(profit#128)#131 AS profit#134] +Input [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#133)] +Aggregate Attributes [3]: [sum(sales#18)#134, sum(returns#39)#135, sum(profit#133)#136] +Results [4]: [channel#37, sum(sales#18)#134 AS sales#137, sum(returns#39)#135 AS returns#138, sum(profit#133)#136 AS profit#139] (90) HashAggregate [codegen id : 48] -Input [4]: [channel#36, sales#132, returns#133, profit#134] -Keys [1]: [channel#36] -Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] -Aggregate Attributes [6]: [sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Results [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Input [4]: [channel#37, sales#137, returns#138, profit#139] +Keys [1]: [channel#37] +Functions [3]: [partial_sum(sales#137), partial_sum(returns#138), partial_sum(profit#139)] +Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Results [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] (91) Exchange -Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Arguments: hashpartitioning(channel#36, 5), ENSURE_REQUIREMENTS, [id=#147] +Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [id=#152] (92) HashAggregate [codegen id : 49] -Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Keys [1]: [channel#36] -Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] -Aggregate Attributes [3]: [sum(sales#132)#148, sum(returns#133)#149, sum(profit#134)#150] -Results [5]: [channel#36, null AS id#151, sum(sales#132)#148 AS sales#152, sum(returns#133)#149 AS returns#153, sum(profit#134)#150 AS profit#154] +Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Keys [1]: [channel#37] +Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] +Aggregate Attributes [3]: [sum(sales#137)#153, sum(returns#138)#154, sum(profit#139)#155] +Results [5]: [channel#37, null AS id#156, sum(sales#137)#153 AS sales#157, sum(returns#138)#154 AS returns#158, sum(profit#139)#155 AS profit#159] (93) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] +Output [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] (94) HashAggregate [codegen id : 73] -Input [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#161)] -Aggregate Attributes [3]: [sum(sales#18)#162, sum(returns#38)#163, sum(profit#161)#164] -Results [3]: [sum(sales#18)#162 AS sales#132, sum(returns#38)#163 AS returns#133, sum(profit#161)#164 AS profit#134] +Input [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#166)] +Aggregate Attributes [3]: [sum(sales#18)#167, sum(returns#39)#168, sum(profit#166)#169] +Results [3]: [sum(sales#18)#167 AS sales#137, sum(returns#39)#168 AS returns#138, sum(profit#166)#169 AS profit#139] (95) HashAggregate [codegen id : 73] -Input [3]: [sales#132, returns#133, profit#134] +Input [3]: [sales#137, returns#138, profit#139] Keys: [] -Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] -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#137), partial_sum(returns#138), partial_sum(profit#139)] +Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Results [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] (96) Exchange -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#177] +Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#182] (97) HashAggregate [codegen id : 74] -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Keys: [] -Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] -Aggregate Attributes [3]: [sum(sales#132)#178, sum(returns#133)#179, sum(profit#134)#180] -Results [5]: [null AS channel#181, null AS id#182, sum(sales#132)#178 AS sales#183, sum(returns#133)#179 AS returns#184, sum(profit#134)#180 AS profit#185] +Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] +Aggregate Attributes [3]: [sum(sales#137)#183, sum(returns#138)#184, sum(profit#139)#185] +Results [5]: [null AS channel#186, null AS id#187, sum(sales#137)#183 AS sales#188, sum(returns#138)#184 AS returns#189, sum(profit#139)#185 AS profit#190] (98) Union (99) HashAggregate [codegen id : 75] -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] (100) Exchange -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Arguments: hashpartitioning(channel#36, id#37, sales#119, returns#120, profit#121, 5), ENSURE_REQUIREMENTS, [id=#186] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Arguments: hashpartitioning(channel#37, id#38, sales#124, returns#125, profit#126, 5), ENSURE_REQUIREMENTS, [id=#191] (101) HashAggregate [codegen id : 76] -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] (102) TakeOrderedAndProject -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Arguments: 100, [channel#36 ASC NULLS FIRST, id#37 ASC NULLS FIRST], [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#124, returns#125, profit#126] ===== Subqueries ===== @@ -611,14 +611,14 @@ ReusedExchange (104) (104) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#25] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#58 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt index 0bd6422fffa5a..a12ab50a89f43 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt @@ -248,164 +248,164 @@ Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale Input [9]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#43, d_year#44] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#43] Join condition: None (41) Project [codegen id : 13] -Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] -Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#14, d_year#15] +Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] +Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#43, d_year#44] (42) HashAggregate [codegen id : 13] -Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] -Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] +Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [partial_sum(cs_quantity#34), partial_sum(UnscaledValue(cs_wholesale_cost#35)), partial_sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum#43, sum#44, sum#45] -Results [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] +Aggregate Attributes [3]: [sum#45, sum#46, sum#47] +Results [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] (43) Exchange -Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] -Arguments: hashpartitioning(d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] +Arguments: hashpartitioning(d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#51] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] -Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] +Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [sum(cs_quantity#34), sum(UnscaledValue(cs_wholesale_cost#35)), sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum(cs_quantity#34)#50, sum(UnscaledValue(cs_wholesale_cost#35))#51, sum(UnscaledValue(cs_sales_price#36))#52] -Results [6]: [d_year#15 AS cs_sold_year#53, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#54, sum(cs_quantity#34)#50 AS cs_qty#55, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#51,17,2) AS cs_wc#56, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#52,17,2) AS cs_sp#57] +Aggregate Attributes [3]: [sum(cs_quantity#34)#52, sum(UnscaledValue(cs_wholesale_cost#35))#53, sum(UnscaledValue(cs_sales_price#36))#54] +Results [6]: [d_year#44 AS cs_sold_year#55, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#56, sum(cs_quantity#34)#52 AS cs_qty#57, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#53,17,2) AS cs_wc#58, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#54,17,2) AS cs_sp#59] (45) Filter [codegen id : 14] -Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] -Condition : (coalesce(cs_qty#55, 0) > 0) +Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] +Condition : (coalesce(cs_qty#57, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] -Arguments: [cs_sold_year#53 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] +Arguments: [cs_sold_year#55 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#56 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54] +Right keys [3]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] (49) Scan parquet default.web_sales -Output [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Output [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] (51) Filter [codegen id : 16] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Condition : (isnotnull(ws_item_sk#58) AND isnotnull(ws_bill_customer_sk#59)) +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Condition : (isnotnull(ws_item_sk#60) AND isnotnull(ws_bill_customer_sk#61)) (52) Exchange -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Arguments: hashpartitioning(cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint), 5), ENSURE_REQUIREMENTS, [id=#65] +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Arguments: hashpartitioning(cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint), 5), ENSURE_REQUIREMENTS, [id=#67] (53) Sort [codegen id : 17] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Arguments: [cast(ws_order_number#60 as bigint) ASC NULLS FIRST, cast(ws_item_sk#58 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Arguments: [cast(ws_order_number#62 as bigint) ASC NULLS FIRST, cast(ws_item_sk#60 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.web_returns -Output [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Output [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] (56) Filter [codegen id : 18] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] -Condition : (isnotnull(wr_order_number#67) AND isnotnull(wr_item_sk#66)) +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Condition : (isnotnull(wr_order_number#69) AND isnotnull(wr_item_sk#68)) (57) Project [codegen id : 18] -Output [2]: [wr_item_sk#66, wr_order_number#67] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Output [2]: [wr_item_sk#68, wr_order_number#69] +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] (58) Exchange -Input [2]: [wr_item_sk#66, wr_order_number#67] -Arguments: hashpartitioning(wr_order_number#67, wr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [wr_item_sk#68, wr_order_number#69] +Arguments: hashpartitioning(wr_order_number#69, wr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] (59) Sort [codegen id : 19] -Input [2]: [wr_item_sk#66, wr_order_number#67] -Arguments: [wr_order_number#67 ASC NULLS FIRST, wr_item_sk#66 ASC NULLS FIRST], false, 0 +Input [2]: [wr_item_sk#68, wr_order_number#69] +Arguments: [wr_order_number#69 ASC NULLS FIRST, wr_item_sk#68 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint)] -Right keys [2]: [wr_order_number#67, wr_item_sk#66] +Left keys [2]: [cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint)] +Right keys [2]: [wr_order_number#69, wr_item_sk#68] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] -Condition : isnull(wr_order_number#67) +Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] +Condition : isnull(wr_order_number#69) (62) Project [codegen id : 21] -Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] +Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#72, d_year#73] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [ws_sold_date_sk#64] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#66] +Right keys [1]: [d_date_sk#72] Join condition: None (65) Project [codegen id : 21] -Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] -Input [8]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, d_date_sk#14, d_year#15] +Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] +Input [8]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, d_date_sk#72, d_year#73] (66) HashAggregate [codegen id : 21] -Input [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] -Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] -Functions [3]: [partial_sum(ws_quantity#61), partial_sum(UnscaledValue(ws_wholesale_cost#62)), partial_sum(UnscaledValue(ws_sales_price#63))] -Aggregate Attributes [3]: [sum#70, sum#71, sum#72] -Results [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] +Input [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] +Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] +Functions [3]: [partial_sum(ws_quantity#63), partial_sum(UnscaledValue(ws_wholesale_cost#64)), partial_sum(UnscaledValue(ws_sales_price#65))] +Aggregate Attributes [3]: [sum#74, sum#75, sum#76] +Results [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] (67) Exchange -Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] +Arguments: hashpartitioning(d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#80] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] -Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] -Functions [3]: [sum(ws_quantity#61), sum(UnscaledValue(ws_wholesale_cost#62)), sum(UnscaledValue(ws_sales_price#63))] -Aggregate Attributes [3]: [sum(ws_quantity#61)#77, sum(UnscaledValue(ws_wholesale_cost#62))#78, sum(UnscaledValue(ws_sales_price#63))#79] -Results [6]: [d_year#15 AS ws_sold_year#80, ws_item_sk#58, ws_bill_customer_sk#59 AS ws_customer_sk#81, sum(ws_quantity#61)#77 AS ws_qty#82, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#62))#78,17,2) AS ws_wc#83, MakeDecimal(sum(UnscaledValue(ws_sales_price#63))#79,17,2) AS ws_sp#84] +Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] +Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] +Functions [3]: [sum(ws_quantity#63), sum(UnscaledValue(ws_wholesale_cost#64)), sum(UnscaledValue(ws_sales_price#65))] +Aggregate Attributes [3]: [sum(ws_quantity#63)#81, sum(UnscaledValue(ws_wholesale_cost#64))#82, sum(UnscaledValue(ws_sales_price#65))#83] +Results [6]: [d_year#73 AS ws_sold_year#84, ws_item_sk#60, ws_bill_customer_sk#61 AS ws_customer_sk#85, sum(ws_quantity#63)#81 AS ws_qty#86, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#64))#82,17,2) AS ws_wc#87, MakeDecimal(sum(UnscaledValue(ws_sales_price#65))#83,17,2) AS ws_sp#88] (69) Filter [codegen id : 22] -Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] -Condition : (coalesce(ws_qty#82, 0) > 0) +Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] +Condition : (coalesce(ws_qty#86, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] -Arguments: [ws_sold_year#80 ASC NULLS FIRST, ws_item_sk#58 ASC NULLS FIRST, ws_customer_sk#81 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] +Arguments: [ws_sold_year#84 ASC NULLS FIRST, ws_item_sk#60 ASC NULLS FIRST, ws_customer_sk#85 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81] +Right keys [3]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85] Join condition: None (72) Project [codegen id : 23] -Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#82 + cs_qty#55), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#82, 0) + coalesce(cs_qty#55, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#83, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#56, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#84, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#57, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57, ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] +Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#86 + cs_qty#57), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#86, 0) + coalesce(cs_qty#57, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#87, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#58, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#88, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#59, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59, ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] (73) TakeOrderedAndProject -Input [13]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, ratio#85 ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] +Input [13]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, ratio#89 ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt index 2d84cacd5d09c..b14ab47d28afe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt @@ -248,164 +248,164 @@ Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale Input [9]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_order_number#33, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, wr_item_sk#39, wr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#43, d_year#44] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#37] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#43] Join condition: None (41) Project [codegen id : 13] -Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] -Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#14, d_year#15] +Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] +Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#43, d_year#44] (42) HashAggregate [codegen id : 13] -Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] -Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] +Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [partial_sum(ws_quantity#34), partial_sum(UnscaledValue(ws_wholesale_cost#35)), partial_sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum#43, sum#44, sum#45] -Results [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] +Aggregate Attributes [3]: [sum#45, sum#46, sum#47] +Results [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] (43) Exchange -Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] -Arguments: hashpartitioning(d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] +Arguments: hashpartitioning(d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#51] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] -Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] +Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [sum(ws_quantity#34), sum(UnscaledValue(ws_wholesale_cost#35)), sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum(ws_quantity#34)#50, sum(UnscaledValue(ws_wholesale_cost#35))#51, sum(UnscaledValue(ws_sales_price#36))#52] -Results [6]: [d_year#15 AS ws_sold_year#53, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#54, sum(ws_quantity#34)#50 AS ws_qty#55, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#51,17,2) AS ws_wc#56, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#52,17,2) AS ws_sp#57] +Aggregate Attributes [3]: [sum(ws_quantity#34)#52, sum(UnscaledValue(ws_wholesale_cost#35))#53, sum(UnscaledValue(ws_sales_price#36))#54] +Results [6]: [d_year#44 AS ws_sold_year#55, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#56, sum(ws_quantity#34)#52 AS ws_qty#57, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#53,17,2) AS ws_wc#58, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#54,17,2) AS ws_sp#59] (45) Filter [codegen id : 14] -Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] -Condition : (coalesce(ws_qty#55, 0) > 0) +Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] +Condition : (coalesce(ws_qty#57, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] -Arguments: [ws_sold_year#53 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] +Arguments: [ws_sold_year#55 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#56 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54] +Right keys [3]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] (49) Scan parquet default.catalog_sales -Output [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Output [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#66), dynamicpruningexpression(cs_sold_date_sk#66 IN dynamicpruning#8)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] (51) Filter [codegen id : 16] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Condition : (isnotnull(cs_item_sk#59) AND isnotnull(cs_bill_customer_sk#58)) +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Condition : (isnotnull(cs_item_sk#61) AND isnotnull(cs_bill_customer_sk#60)) (52) Exchange -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Arguments: hashpartitioning(cs_order_number#60, cs_item_sk#59, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Arguments: hashpartitioning(cs_order_number#62, cs_item_sk#61, 5), ENSURE_REQUIREMENTS, [id=#67] (53) Sort [codegen id : 17] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Arguments: [cs_order_number#60 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST], false, 0 +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Arguments: [cs_order_number#62 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST], false, 0 (54) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Output [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] (56) Filter [codegen id : 18] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] -Condition : (isnotnull(cr_order_number#67) AND isnotnull(cr_item_sk#66)) +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Condition : (isnotnull(cr_order_number#69) AND isnotnull(cr_item_sk#68)) (57) Project [codegen id : 18] -Output [2]: [cr_item_sk#66, cr_order_number#67] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Output [2]: [cr_item_sk#68, cr_order_number#69] +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] (58) Exchange -Input [2]: [cr_item_sk#66, cr_order_number#67] -Arguments: hashpartitioning(cr_order_number#67, cr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [cr_item_sk#68, cr_order_number#69] +Arguments: hashpartitioning(cr_order_number#69, cr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] (59) Sort [codegen id : 19] -Input [2]: [cr_item_sk#66, cr_order_number#67] -Arguments: [cr_order_number#67 ASC NULLS FIRST, cr_item_sk#66 ASC NULLS FIRST], false, 0 +Input [2]: [cr_item_sk#68, cr_order_number#69] +Arguments: [cr_order_number#69 ASC NULLS FIRST, cr_item_sk#68 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cs_order_number#60, cs_item_sk#59] -Right keys [2]: [cr_order_number#67, cr_item_sk#66] +Left keys [2]: [cs_order_number#62, cs_item_sk#61] +Right keys [2]: [cr_order_number#69, cr_item_sk#68] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] -Condition : isnull(cr_order_number#67) +Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] +Condition : isnull(cr_order_number#69) (62) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] +Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#72, d_year#73] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [cs_sold_date_sk#64] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [cs_sold_date_sk#66] +Right keys [1]: [d_date_sk#72] Join condition: None (65) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] -Input [8]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, d_date_sk#14, d_year#15] +Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] +Input [8]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, d_date_sk#72, d_year#73] (66) HashAggregate [codegen id : 21] -Input [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] -Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] -Functions [3]: [partial_sum(cs_quantity#61), partial_sum(UnscaledValue(cs_wholesale_cost#62)), partial_sum(UnscaledValue(cs_sales_price#63))] -Aggregate Attributes [3]: [sum#70, sum#71, sum#72] -Results [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] +Input [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] +Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] +Functions [3]: [partial_sum(cs_quantity#63), partial_sum(UnscaledValue(cs_wholesale_cost#64)), partial_sum(UnscaledValue(cs_sales_price#65))] +Aggregate Attributes [3]: [sum#74, sum#75, sum#76] +Results [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] (67) Exchange -Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] +Arguments: hashpartitioning(d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#80] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] -Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] -Functions [3]: [sum(cs_quantity#61), sum(UnscaledValue(cs_wholesale_cost#62)), sum(UnscaledValue(cs_sales_price#63))] -Aggregate Attributes [3]: [sum(cs_quantity#61)#77, sum(UnscaledValue(cs_wholesale_cost#62))#78, sum(UnscaledValue(cs_sales_price#63))#79] -Results [6]: [d_year#15 AS cs_sold_year#80, cs_item_sk#59, cs_bill_customer_sk#58 AS cs_customer_sk#81, sum(cs_quantity#61)#77 AS cs_qty#82, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#62))#78,17,2) AS cs_wc#83, MakeDecimal(sum(UnscaledValue(cs_sales_price#63))#79,17,2) AS cs_sp#84] +Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] +Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] +Functions [3]: [sum(cs_quantity#63), sum(UnscaledValue(cs_wholesale_cost#64)), sum(UnscaledValue(cs_sales_price#65))] +Aggregate Attributes [3]: [sum(cs_quantity#63)#81, sum(UnscaledValue(cs_wholesale_cost#64))#82, sum(UnscaledValue(cs_sales_price#65))#83] +Results [6]: [d_year#73 AS cs_sold_year#84, cs_item_sk#61, cs_bill_customer_sk#60 AS cs_customer_sk#85, sum(cs_quantity#63)#81 AS cs_qty#86, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#64))#82,17,2) AS cs_wc#87, MakeDecimal(sum(UnscaledValue(cs_sales_price#65))#83,17,2) AS cs_sp#88] (69) Filter [codegen id : 22] -Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] -Condition : (coalesce(cs_qty#82, 0) > 0) +Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] +Condition : (coalesce(cs_qty#86, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] -Arguments: [cs_sold_year#80 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST, cs_customer_sk#81 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] +Arguments: [cs_sold_year#84 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST, cs_customer_sk#85 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81] +Right keys [3]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85] Join condition: None (72) Project [codegen id : 23] -Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#55 + cs_qty#82), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#55, 0) + coalesce(cs_qty#82, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#56, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#83, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#57, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#84, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57, cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] +Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#57 + cs_qty#86), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#57, 0) + coalesce(cs_qty#86, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#58, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#87, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#59, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#88, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59, cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] (73) TakeOrderedAndProject -Input [13]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, ratio#85 ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] +Input [13]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, ratio#89 ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#8 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 8a384ec4b1795..a5d47ede475ca 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 @@ -380,224 +380,224 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#16] +Output [1]: [i_item_sk#61] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#16] +Right keys [1]: [i_item_sk#61] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#16] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#61] (60) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#19] +Output [1]: [p_promo_sk#62] (61) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#19] +Right keys [1]: [p_promo_sk#62] Join condition: None (62) Project [codegen id : 19] Output [6]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#19] +Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#62] (63) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#63] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#63] Join condition: None (65) Project [codegen id : 19] Output [5]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#22] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#63] (66) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 18] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] (68) Filter [codegen id : 18] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Condition : isnotnull(cp_catalog_page_sk#61) +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Condition : isnotnull(cp_catalog_page_sk#64) (69) BroadcastExchange -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#61] +Right keys [1]: [cp_catalog_page_sk#64] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#64, cp_catalog_page_id#65] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Keys [1]: [cp_catalog_page_id#62] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] +Keys [1]: [cp_catalog_page_id#65] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Results [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] (73) Exchange -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [id=#77] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [cp_catalog_page_id#62] +Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Keys [1]: [cp_catalog_page_id#65] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] -Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_page_id#62) AS id#79, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#80, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#81, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#82] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] +Results [5]: [catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#65) AS id#82, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#83, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#84, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#85] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) (78) Exchange -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] -Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (84) Exchange -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] -Right keys [2]: [wr_item_sk#91, wr_order_number#92] +Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] +Right keys [2]: [wr_item_sk#94, wr_order_number#95] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#16] +Output [1]: [i_item_sk#100] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#83] -Right keys [1]: [i_item_sk#16] +Left keys [1]: [ws_item_sk#86] +Right keys [1]: [i_item_sk#100] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, i_item_sk#16] +Output [7]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, i_item_sk#100] (91) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#19] +Output [1]: [p_promo_sk#101] (92) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#85] -Right keys [1]: [p_promo_sk#19] +Left keys [1]: [ws_promo_sk#88] +Right keys [1]: [p_promo_sk#101] Join condition: None (93) Project [codegen id : 29] -Output [6]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [8]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, p_promo_sk#19] +Output [6]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [8]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, p_promo_sk#101] (94) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#102] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#89] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [ws_sold_date_sk#92] +Right keys [1]: [d_date_sk#102] Join condition: None (96) Project [codegen id : 29] -Output [5]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] -Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#22] +Output [5]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] +Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#102] (97) Scan parquet default.web_site -Output [2]: [web_site_sk#97, web_site_id#98] +Output [2]: [web_site_sk#103, web_site_id#104] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (98) ColumnarToRow [codegen id : 28] -Input [2]: [web_site_sk#97, web_site_id#98] +Input [2]: [web_site_sk#103, web_site_id#104] (99) Filter [codegen id : 28] -Input [2]: [web_site_sk#97, web_site_id#98] -Condition : isnotnull(web_site_sk#97) +Input [2]: [web_site_sk#103, web_site_id#104] +Condition : isnotnull(web_site_sk#103) (100) BroadcastExchange -Input [2]: [web_site_sk#97, web_site_id#98] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Input [2]: [web_site_sk#103, web_site_id#104] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#105] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#84] -Right keys [1]: [web_site_sk#97] +Left keys [1]: [ws_web_site_sk#87] +Right keys [1]: [web_site_sk#103] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] +Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] +Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#103, web_site_id#104] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Keys [1]: [web_site_id#98] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] +Keys [1]: [web_site_id#104] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Results [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] (104) Exchange -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] +Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, [id=#116] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Keys [1]: [web_site_id#98] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] -Results [5]: [web channel AS channel#114, concat(web_site, web_site_id#98) AS id#115, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#116, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#117, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#118] +Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Keys [1]: [web_site_id#104] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] +Results [5]: [web channel AS channel#120, concat(web_site, web_site_id#104) AS id#121, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#122, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#123, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#124] (106) Union @@ -605,99 +605,99 @@ Results [5]: [web channel AS channel#114, concat(web_site, web_site_id#98) AS id Input [5]: [channel#42, id#43, sales#44, returns#45, profit#46] Keys [2]: [channel#42, id#43] Functions [3]: [partial_sum(sales#44), partial_sum(returns#45), partial_sum(profit#46)] -Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Results [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Aggregate Attributes [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Results [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] (108) Exchange -Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#131] +Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#137] (109) HashAggregate [codegen id : 32] -Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#132, sum(returns#45)#133, sum(profit#46)#134] -Results [5]: [channel#42, id#43, cast(sum(sales#44)#132 as decimal(37,2)) AS sales#135, cast(sum(returns#45)#133 as decimal(38,2)) AS returns#136, cast(sum(profit#46)#134 as decimal(38,2)) AS profit#137] +Aggregate Attributes [3]: [sum(sales#44)#138, sum(returns#45)#139, sum(profit#46)#140] +Results [5]: [channel#42, id#43, cast(sum(sales#44)#138 as decimal(37,2)) AS sales#141, cast(sum(returns#45)#139 as decimal(38,2)) AS returns#142, cast(sum(profit#46)#140 as decimal(38,2)) AS profit#143] (110) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Output [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] (111) HashAggregate [codegen id : 64] -Input [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Input [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#144, sum(returns#45)#145, sum(profit#46)#146] -Results [4]: [channel#42, sum(sales#44)#144 AS sales#147, sum(returns#45)#145 AS returns#148, sum(profit#46)#146 AS profit#149] +Aggregate Attributes [3]: [sum(sales#44)#150, sum(returns#45)#151, sum(profit#46)#152] +Results [4]: [channel#42, sum(sales#44)#150 AS sales#153, sum(returns#45)#151 AS returns#154, sum(profit#46)#152 AS profit#155] (112) HashAggregate [codegen id : 64] -Input [4]: [channel#42, sales#147, returns#148, profit#149] +Input [4]: [channel#42, sales#153, returns#154, profit#155] Keys [1]: [channel#42] -Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] -Aggregate Attributes [6]: [sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] -Results [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +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#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] (113) Exchange -Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] -Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#162] +Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#168] (114) HashAggregate [codegen id : 65] -Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] Keys [1]: [channel#42] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#163, sum(returns#148)#164, sum(profit#149)#165] -Results [5]: [channel#42, null AS id#166, sum(sales#147)#163 AS sales#167, sum(returns#148)#164 AS returns#168, sum(profit#149)#165 AS profit#169] +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#42, null AS id#172, sum(sales#153)#169 AS sales#173, sum(returns#154)#170 AS returns#174, sum(profit#155)#171 AS profit#175] (115) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Output [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] (116) HashAggregate [codegen id : 97] -Input [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Input [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#176, sum(returns#45)#177, sum(profit#46)#178] -Results [3]: [sum(sales#44)#176 AS sales#147, sum(returns#45)#177 AS returns#148, sum(profit#46)#178 AS profit#149] +Aggregate Attributes [3]: [sum(sales#44)#182, sum(returns#45)#183, sum(profit#46)#184] +Results [3]: [sum(sales#44)#182 AS sales#153, sum(returns#45)#183 AS returns#154, sum(profit#46)#184 AS profit#155] (117) HashAggregate [codegen id : 97] -Input [3]: [sales#147, returns#148, profit#149] +Input [3]: [sales#153, returns#154, profit#155] Keys: [] -Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] -Aggregate Attributes [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] -Results [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Functions [3]: [partial_sum(sales#153), partial_sum(returns#154), partial_sum(profit#155)] +Aggregate Attributes [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Results [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] (118) Exchange -Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#191] +Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#197] (119) HashAggregate [codegen id : 98] -Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] Keys: [] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#192, sum(returns#148)#193, sum(profit#149)#194] -Results [5]: [null AS channel#195, null AS id#196, sum(sales#147)#192 AS sales#197, sum(returns#148)#193 AS returns#198, sum(profit#149)#194 AS profit#199] +Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] +Aggregate Attributes [3]: [sum(sales#153)#198, sum(returns#154)#199, sum(profit#155)#200] +Results [5]: [null AS channel#201, null AS id#202, sum(sales#153)#198 AS sales#203, sum(returns#154)#199 AS returns#204, sum(profit#155)#200 AS profit#205] (120) Union (121) HashAggregate [codegen id : 99] -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] (122) Exchange -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Arguments: hashpartitioning(channel#42, id#43, sales#135, returns#136, profit#137, 5), ENSURE_REQUIREMENTS, [id=#200] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Arguments: hashpartitioning(channel#42, id#43, sales#141, returns#142, profit#143, 5), ENSURE_REQUIREMENTS, [id=#206] (123) HashAggregate [codegen id : 100] -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] (124) TakeOrderedAndProject -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#141, returns#142, profit#143] ===== Subqueries ===== @@ -710,6 +710,6 @@ Output [1]: [d_date_sk#22] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 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 596f9497b5905..9e5bd3aeaa1d1 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 @@ -380,224 +380,224 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#61] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#61] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#16] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#61] (60) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] (62) Filter [codegen id : 16] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Condition : isnotnull(cp_catalog_page_sk#61) +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Condition : isnotnull(cp_catalog_page_sk#62) (63) BroadcastExchange -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#61] +Right keys [1]: [cp_catalog_page_sk#62] Join condition: None (65) Project [codegen id : 19] -Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#62, cp_catalog_page_id#63] (66) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#22] +Output [1]: [i_item_sk#65] (67) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#22] +Right keys [1]: [i_item_sk#65] Join condition: None (68) Project [codegen id : 19] -Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, i_item_sk#22] +Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, i_item_sk#65] (69) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#25] +Output [1]: [p_promo_sk#66] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#25] +Right keys [1]: [p_promo_sk#66] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, p_promo_sk#25] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, p_promo_sk#66] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Keys [1]: [cp_catalog_page_id#62] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Keys [1]: [cp_catalog_page_id#63] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Results [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] (73) Exchange -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Arguments: hashpartitioning(cp_catalog_page_id#63, 5), ENSURE_REQUIREMENTS, [id=#77] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [cp_catalog_page_id#62] +Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Keys [1]: [cp_catalog_page_id#63] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] -Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_page_id#62) AS id#79, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#80, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#81, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#82] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] +Results [5]: [catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#63) AS id#82, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#83, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#84, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#85] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) (78) Exchange -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] -Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (84) Exchange -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] -Right keys [2]: [wr_item_sk#91, wr_order_number#92] +Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] +Right keys [2]: [wr_item_sk#94, wr_order_number#95] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#100] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#89] -Right keys [1]: [d_date_sk#16] +Left keys [1]: [ws_sold_date_sk#92] +Right keys [1]: [d_date_sk#100] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#16] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#100] (91) Scan parquet default.web_site -Output [2]: [web_site_sk#97, web_site_id#98] +Output [2]: [web_site_sk#101, web_site_id#102] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 26] -Input [2]: [web_site_sk#97, web_site_id#98] +Input [2]: [web_site_sk#101, web_site_id#102] (93) Filter [codegen id : 26] -Input [2]: [web_site_sk#97, web_site_id#98] -Condition : isnotnull(web_site_sk#97) +Input [2]: [web_site_sk#101, web_site_id#102] +Condition : isnotnull(web_site_sk#101) (94) BroadcastExchange -Input [2]: [web_site_sk#97, web_site_id#98] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Input [2]: [web_site_sk#101, web_site_id#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#84] -Right keys [1]: [web_site_sk#97] +Left keys [1]: [ws_web_site_sk#87] +Right keys [1]: [web_site_sk#101] Join condition: None (96) Project [codegen id : 29] -Output [7]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] +Output [7]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#101, web_site_id#102] (97) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#22] +Output [1]: [i_item_sk#104] (98) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#83] -Right keys [1]: [i_item_sk#22] +Left keys [1]: [ws_item_sk#86] +Right keys [1]: [i_item_sk#104] Join condition: None (99) Project [codegen id : 29] -Output [6]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [8]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, i_item_sk#22] +Output [6]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [8]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, i_item_sk#104] (100) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#25] +Output [1]: [p_promo_sk#105] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#85] -Right keys [1]: [p_promo_sk#25] +Left keys [1]: [ws_promo_sk#88] +Right keys [1]: [p_promo_sk#105] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [7]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, p_promo_sk#25] +Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [7]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, p_promo_sk#105] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Keys [1]: [web_site_id#98] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Keys [1]: [web_site_id#102] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Results [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] (104) Exchange -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] +Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Arguments: hashpartitioning(web_site_id#102, 5), ENSURE_REQUIREMENTS, [id=#116] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Keys [1]: [web_site_id#98] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] -Results [5]: [web channel AS channel#114, concat(web_site, web_site_id#98) AS id#115, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#116, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#117, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#118] +Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Keys [1]: [web_site_id#102] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] +Results [5]: [web channel AS channel#120, concat(web_site, web_site_id#102) AS id#121, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#122, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#123, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#124] (106) Union @@ -605,99 +605,99 @@ Results [5]: [web channel AS channel#114, concat(web_site, web_site_id#98) AS id Input [5]: [channel#42, id#43, sales#44, returns#45, profit#46] Keys [2]: [channel#42, id#43] Functions [3]: [partial_sum(sales#44), partial_sum(returns#45), partial_sum(profit#46)] -Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Results [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Aggregate Attributes [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Results [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] (108) Exchange -Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#131] +Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#137] (109) HashAggregate [codegen id : 32] -Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#132, sum(returns#45)#133, sum(profit#46)#134] -Results [5]: [channel#42, id#43, cast(sum(sales#44)#132 as decimal(37,2)) AS sales#135, cast(sum(returns#45)#133 as decimal(38,2)) AS returns#136, cast(sum(profit#46)#134 as decimal(38,2)) AS profit#137] +Aggregate Attributes [3]: [sum(sales#44)#138, sum(returns#45)#139, sum(profit#46)#140] +Results [5]: [channel#42, id#43, cast(sum(sales#44)#138 as decimal(37,2)) AS sales#141, cast(sum(returns#45)#139 as decimal(38,2)) AS returns#142, cast(sum(profit#46)#140 as decimal(38,2)) AS profit#143] (110) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Output [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] (111) HashAggregate [codegen id : 64] -Input [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Input [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#144, sum(returns#45)#145, sum(profit#46)#146] -Results [4]: [channel#42, sum(sales#44)#144 AS sales#147, sum(returns#45)#145 AS returns#148, sum(profit#46)#146 AS profit#149] +Aggregate Attributes [3]: [sum(sales#44)#150, sum(returns#45)#151, sum(profit#46)#152] +Results [4]: [channel#42, sum(sales#44)#150 AS sales#153, sum(returns#45)#151 AS returns#154, sum(profit#46)#152 AS profit#155] (112) HashAggregate [codegen id : 64] -Input [4]: [channel#42, sales#147, returns#148, profit#149] +Input [4]: [channel#42, sales#153, returns#154, profit#155] Keys [1]: [channel#42] -Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] -Aggregate Attributes [6]: [sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] -Results [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +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#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] (113) Exchange -Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] -Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#162] +Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#168] (114) HashAggregate [codegen id : 65] -Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] Keys [1]: [channel#42] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#163, sum(returns#148)#164, sum(profit#149)#165] -Results [5]: [channel#42, null AS id#166, sum(sales#147)#163 AS sales#167, sum(returns#148)#164 AS returns#168, sum(profit#149)#165 AS profit#169] +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#42, null AS id#172, sum(sales#153)#169 AS sales#173, sum(returns#154)#170 AS returns#174, sum(profit#155)#171 AS profit#175] (115) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Output [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] (116) HashAggregate [codegen id : 97] -Input [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Input [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#176, sum(returns#45)#177, sum(profit#46)#178] -Results [3]: [sum(sales#44)#176 AS sales#147, sum(returns#45)#177 AS returns#148, sum(profit#46)#178 AS profit#149] +Aggregate Attributes [3]: [sum(sales#44)#182, sum(returns#45)#183, sum(profit#46)#184] +Results [3]: [sum(sales#44)#182 AS sales#153, sum(returns#45)#183 AS returns#154, sum(profit#46)#184 AS profit#155] (117) HashAggregate [codegen id : 97] -Input [3]: [sales#147, returns#148, profit#149] +Input [3]: [sales#153, returns#154, profit#155] Keys: [] -Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] -Aggregate Attributes [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] -Results [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Functions [3]: [partial_sum(sales#153), partial_sum(returns#154), partial_sum(profit#155)] +Aggregate Attributes [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Results [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] (118) Exchange -Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#191] +Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#197] (119) HashAggregate [codegen id : 98] -Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] Keys: [] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#192, sum(returns#148)#193, sum(profit#149)#194] -Results [5]: [null AS channel#195, null AS id#196, sum(sales#147)#192 AS sales#197, sum(returns#148)#193 AS returns#198, sum(profit#149)#194 AS profit#199] +Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] +Aggregate Attributes [3]: [sum(sales#153)#198, sum(returns#154)#199, sum(profit#155)#200] +Results [5]: [null AS channel#201, null AS id#202, sum(sales#153)#198 AS sales#203, sum(returns#154)#199 AS returns#204, sum(profit#155)#200 AS profit#205] (120) Union (121) HashAggregate [codegen id : 99] -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] (122) Exchange -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Arguments: hashpartitioning(channel#42, id#43, sales#135, returns#136, profit#137, 5), ENSURE_REQUIREMENTS, [id=#200] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Arguments: hashpartitioning(channel#42, id#43, sales#141, returns#142, profit#143, 5), ENSURE_REQUIREMENTS, [id=#206] (123) HashAggregate [codegen id : 100] -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] (124) TakeOrderedAndProject -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#141, returns#142, profit#143] ===== Subqueries ===== @@ -710,6 +710,6 @@ Output [1]: [d_date_sk#16] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 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 3333d5d1aed52..64f6270e81183 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 @@ -131,60 +131,60 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#15] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#15,17,2) as decimal(27,2)) AS total_sum#16, i_category#10, i_class#9, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] (20) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#10, i_class#9, sum#20] +Output [3]: [i_category#20, i_class#21, sum#22] (21) HashAggregate [codegen id : 8] -Input [3]: [i_category#10, i_class#9, sum#20] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#21] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#21,17,2) AS total_sum#22, i_category#10] +Input [3]: [i_category#20, i_class#21, sum#22] +Keys [2]: [i_category#20, i_class#21] +Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#23))#24] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#23))#24,17,2) AS total_sum#25, i_category#20] (22) HashAggregate [codegen id : 8] -Input [2]: [total_sum#22, i_category#10] -Keys [1]: [i_category#10] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [3]: [i_category#10, sum#25, isEmpty#26] +Input [2]: [total_sum#25, i_category#20] +Keys [1]: [i_category#20] +Functions [1]: [partial_sum(total_sum#25)] +Aggregate Attributes [2]: [sum#26, isEmpty#27] +Results [3]: [i_category#20, sum#28, isEmpty#29] (23) Exchange -Input [3]: [i_category#10, sum#25, isEmpty#26] -Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [i_category#20, sum#28, isEmpty#29] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#30] (24) HashAggregate [codegen id : 9] -Input [3]: [i_category#10, sum#25, isEmpty#26] -Keys [1]: [i_category#10] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#28] -Results [6]: [sum(total_sum#22)#28 AS total_sum#29, i_category#10, null AS i_class#30, 0 AS g_category#31, 1 AS g_class#32, 1 AS lochierarchy#33] +Input [3]: [i_category#20, sum#28, isEmpty#29] +Keys [1]: [i_category#20] +Functions [1]: [sum(total_sum#25)] +Aggregate Attributes [1]: [sum(total_sum#25)#31] +Results [6]: [sum(total_sum#25)#31 AS total_sum#32, i_category#20, null AS i_class#33, 0 AS g_category#34, 1 AS g_class#35, 1 AS lochierarchy#36] (25) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#10, i_class#9, sum#34] +Output [3]: [i_category#37, i_class#38, sum#39] (26) HashAggregate [codegen id : 13] -Input [3]: [i_category#10, i_class#9, sum#34] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#35] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#35,17,2) AS total_sum#22] +Input [3]: [i_category#37, i_class#38, sum#39] +Keys [2]: [i_category#37, i_class#38] +Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#40))#41] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#40))#41,17,2) AS total_sum#25] (27) HashAggregate [codegen id : 13] -Input [1]: [total_sum#22] +Input [1]: [total_sum#25] Keys: [] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [2]: [sum#38, isEmpty#39] +Functions [1]: [partial_sum(total_sum#25)] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [2]: [sum#44, isEmpty#45] (28) Exchange -Input [2]: [sum#38, isEmpty#39] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] +Input [2]: [sum#44, isEmpty#45] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#46] (29) HashAggregate [codegen id : 14] -Input [2]: [sum#38, isEmpty#39] +Input [2]: [sum#44, isEmpty#45] Keys: [] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#41] -Results [6]: [sum(total_sum#22)#41 AS total_sum#42, null AS i_category#43, null AS i_class#44, 1 AS g_category#45, 1 AS g_class#46, 2 AS lochierarchy#47] +Functions [1]: [sum(total_sum#25)] +Aggregate Attributes [1]: [sum(total_sum#25)#47] +Results [6]: [sum(total_sum#25)#47 AS total_sum#48, null AS i_category#49, null AS i_class#50, 1 AS g_category#51, 1 AS g_class#52, 2 AS lochierarchy#53] (30) Union @@ -197,34 +197,34 @@ Results [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, (32) Exchange Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] -Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#48] +Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#54] (33) HashAggregate [codegen id : 16] Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Keys [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#49] +Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#55] (34) Exchange -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: hashpartitioning(lochierarchy#19, _w0#49, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: hashpartitioning(lochierarchy#19, _w0#55, 5), ENSURE_REQUIREMENTS, [id=#56] (35) Sort [codegen id : 17] -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#49 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 (36) Window -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#49, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#51], [lochierarchy#19, _w0#49], [total_sum#16 DESC NULLS LAST] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#55, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#57], [lochierarchy#19, _w0#55], [total_sum#16 DESC NULLS LAST] (37) Project [codegen id : 18] -Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] -Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49, rank_within_parent#51] +Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] +Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55, rank_within_parent#57] (38) TakeOrderedAndProject -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#51 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] +Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#57 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] ===== Subqueries ===== 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 3333d5d1aed52..64f6270e81183 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 @@ -131,60 +131,60 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#15] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#15,17,2) as decimal(27,2)) AS total_sum#16, i_category#10, i_class#9, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] (20) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#10, i_class#9, sum#20] +Output [3]: [i_category#20, i_class#21, sum#22] (21) HashAggregate [codegen id : 8] -Input [3]: [i_category#10, i_class#9, sum#20] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#21] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#21,17,2) AS total_sum#22, i_category#10] +Input [3]: [i_category#20, i_class#21, sum#22] +Keys [2]: [i_category#20, i_class#21] +Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#23))#24] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#23))#24,17,2) AS total_sum#25, i_category#20] (22) HashAggregate [codegen id : 8] -Input [2]: [total_sum#22, i_category#10] -Keys [1]: [i_category#10] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [3]: [i_category#10, sum#25, isEmpty#26] +Input [2]: [total_sum#25, i_category#20] +Keys [1]: [i_category#20] +Functions [1]: [partial_sum(total_sum#25)] +Aggregate Attributes [2]: [sum#26, isEmpty#27] +Results [3]: [i_category#20, sum#28, isEmpty#29] (23) Exchange -Input [3]: [i_category#10, sum#25, isEmpty#26] -Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [i_category#20, sum#28, isEmpty#29] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#30] (24) HashAggregate [codegen id : 9] -Input [3]: [i_category#10, sum#25, isEmpty#26] -Keys [1]: [i_category#10] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#28] -Results [6]: [sum(total_sum#22)#28 AS total_sum#29, i_category#10, null AS i_class#30, 0 AS g_category#31, 1 AS g_class#32, 1 AS lochierarchy#33] +Input [3]: [i_category#20, sum#28, isEmpty#29] +Keys [1]: [i_category#20] +Functions [1]: [sum(total_sum#25)] +Aggregate Attributes [1]: [sum(total_sum#25)#31] +Results [6]: [sum(total_sum#25)#31 AS total_sum#32, i_category#20, null AS i_class#33, 0 AS g_category#34, 1 AS g_class#35, 1 AS lochierarchy#36] (25) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#10, i_class#9, sum#34] +Output [3]: [i_category#37, i_class#38, sum#39] (26) HashAggregate [codegen id : 13] -Input [3]: [i_category#10, i_class#9, sum#34] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#35] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#35,17,2) AS total_sum#22] +Input [3]: [i_category#37, i_class#38, sum#39] +Keys [2]: [i_category#37, i_class#38] +Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#40))#41] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#40))#41,17,2) AS total_sum#25] (27) HashAggregate [codegen id : 13] -Input [1]: [total_sum#22] +Input [1]: [total_sum#25] Keys: [] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [2]: [sum#38, isEmpty#39] +Functions [1]: [partial_sum(total_sum#25)] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [2]: [sum#44, isEmpty#45] (28) Exchange -Input [2]: [sum#38, isEmpty#39] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] +Input [2]: [sum#44, isEmpty#45] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#46] (29) HashAggregate [codegen id : 14] -Input [2]: [sum#38, isEmpty#39] +Input [2]: [sum#44, isEmpty#45] Keys: [] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#41] -Results [6]: [sum(total_sum#22)#41 AS total_sum#42, null AS i_category#43, null AS i_class#44, 1 AS g_category#45, 1 AS g_class#46, 2 AS lochierarchy#47] +Functions [1]: [sum(total_sum#25)] +Aggregate Attributes [1]: [sum(total_sum#25)#47] +Results [6]: [sum(total_sum#25)#47 AS total_sum#48, null AS i_category#49, null AS i_class#50, 1 AS g_category#51, 1 AS g_class#52, 2 AS lochierarchy#53] (30) Union @@ -197,34 +197,34 @@ Results [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, (32) Exchange Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] -Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#48] +Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#54] (33) HashAggregate [codegen id : 16] Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Keys [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#49] +Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#55] (34) Exchange -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: hashpartitioning(lochierarchy#19, _w0#49, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: hashpartitioning(lochierarchy#19, _w0#55, 5), ENSURE_REQUIREMENTS, [id=#56] (35) Sort [codegen id : 17] -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#49 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 (36) Window -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#49, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#51], [lochierarchy#19, _w0#49], [total_sum#16 DESC NULLS LAST] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#55, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#57], [lochierarchy#19, _w0#55], [total_sum#16 DESC NULLS LAST] (37) Project [codegen id : 18] -Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] -Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49, rank_within_parent#51] +Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] +Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55, rank_within_parent#57] (38) TakeOrderedAndProject -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#51 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] +Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#57 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] ===== Subqueries ===== From 7e46f7261d242825c31f321116fa4a3e5e86af06 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 25 Mar 2021 15:24:40 +0800 Subject: [PATCH 28/38] Revert "gen golden" This reverts commit 92111708 --- .../q10.sf100/explain.txt | 102 ++-- .../approved-plans-modified/q10/explain.txt | 100 ++-- .../q27.sf100/explain.txt | 162 +++--- .../approved-plans-modified/q27/explain.txt | 162 +++--- .../q46.sf100/explain.txt | 18 +- .../approved-plans-modified/q46/explain.txt | 14 +- .../q59.sf100/explain.txt | 102 ++-- .../approved-plans-modified/q59/explain.txt | 102 ++-- .../q65.sf100/explain.txt | 86 +-- .../approved-plans-modified/q65/explain.txt | 48 +- .../q68.sf100/explain.txt | 20 +- .../approved-plans-modified/q68/explain.txt | 14 +- .../approved-plans-v1_4/q1.sf100/explain.txt | 122 ++-- .../approved-plans-v1_4/q1/explain.txt | 116 ++-- .../approved-plans-v1_4/q10.sf100/explain.txt | 104 ++-- .../approved-plans-v1_4/q10/explain.txt | 94 ++-- .../approved-plans-v1_4/q11.sf100/explain.txt | 272 ++++----- .../approved-plans-v1_4/q11/explain.txt | 254 ++++----- .../approved-plans-v1_4/q16.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q16/explain.txt | 2 +- .../approved-plans-v1_4/q17.sf100/explain.txt | 8 +- .../approved-plans-v1_4/q17/explain.txt | 8 +- .../approved-plans-v1_4/q2.sf100/explain.txt | 52 +- .../approved-plans-v1_4/q2/explain.txt | 52 +- .../q23a.sf100/explain.txt | 362 ++++++------ .../approved-plans-v1_4/q23a/explain.txt | 274 ++++----- .../q23b.sf100/explain.txt | 528 +++++++++--------- .../approved-plans-v1_4/q23b/explain.txt | 404 +++++++------- .../q24a.sf100/explain.txt | 206 +++---- .../approved-plans-v1_4/q24a/explain.txt | 170 +++--- .../q24b.sf100/explain.txt | 206 +++---- .../approved-plans-v1_4/q24b/explain.txt | 170 +++--- .../approved-plans-v1_4/q28.sf100/explain.txt | 300 +++++----- .../approved-plans-v1_4/q28/explain.txt | 300 +++++----- .../approved-plans-v1_4/q30.sf100/explain.txt | 142 ++--- .../approved-plans-v1_4/q30/explain.txt | 130 ++--- .../approved-plans-v1_4/q31.sf100/explain.txt | 430 +++++++------- .../approved-plans-v1_4/q31/explain.txt | 362 ++++++------ .../approved-plans-v1_4/q32/explain.txt | 89 +-- .../approved-plans-v1_4/q33.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q33/explain.txt | 4 +- .../approved-plans-v1_4/q35.sf100/explain.txt | 116 ++-- .../approved-plans-v1_4/q35/explain.txt | 96 ++-- .../q39a.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q39a/explain.txt | 4 +- .../q39b.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q39b/explain.txt | 4 +- .../approved-plans-v1_4/q4.sf100/explain.txt | 442 +++++++-------- .../approved-plans-v1_4/q4/explain.txt | 412 +++++++------- .../approved-plans-v1_4/q41.sf100/explain.txt | 51 +- .../approved-plans-v1_4/q41/explain.txt | 51 +- .../approved-plans-v1_4/q44.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q44/explain.txt | 4 +- .../approved-plans-v1_4/q45.sf100/explain.txt | 36 +- .../approved-plans-v1_4/q45/explain.txt | 36 +- .../approved-plans-v1_4/q46.sf100/explain.txt | 44 +- .../approved-plans-v1_4/q46/explain.txt | 14 +- .../approved-plans-v1_4/q47.sf100/explain.txt | 66 +-- .../approved-plans-v1_4/q47/explain.txt | 58 +- .../approved-plans-v1_4/q49.sf100/explain.txt | 198 +++---- .../approved-plans-v1_4/q49/explain.txt | 152 ++--- .../approved-plans-v1_4/q5.sf100/explain.txt | 162 +++--- .../approved-plans-v1_4/q5/explain.txt | 174 +++--- .../approved-plans-v1_4/q54.sf100/explain.txt | 118 ++-- .../approved-plans-v1_4/q54/explain.txt | 120 ++-- .../approved-plans-v1_4/q56.sf100/explain.txt | 140 ++--- .../approved-plans-v1_4/q56/explain.txt | 140 ++--- .../approved-plans-v1_4/q57.sf100/explain.txt | 66 +-- .../approved-plans-v1_4/q57/explain.txt | 58 +- .../approved-plans-v1_4/q58.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q58/explain.txt | 2 +- .../approved-plans-v1_4/q59.sf100/explain.txt | 70 +-- .../approved-plans-v1_4/q59/explain.txt | 70 +-- .../approved-plans-v1_4/q60.sf100/explain.txt | 140 ++--- .../approved-plans-v1_4/q60/explain.txt | 140 ++--- .../approved-plans-v1_4/q61.sf100/explain.txt | 84 +-- .../approved-plans-v1_4/q61/explain.txt | 94 ++-- .../approved-plans-v1_4/q64.sf100/explain.txt | 316 +++++------ .../approved-plans-v1_4/q64/explain.txt | 284 +++++----- .../approved-plans-v1_4/q65.sf100/explain.txt | 98 ++-- .../approved-plans-v1_4/q65/explain.txt | 48 +- .../approved-plans-v1_4/q66.sf100/explain.txt | 70 +-- .../approved-plans-v1_4/q66/explain.txt | 74 +-- .../approved-plans-v1_4/q68.sf100/explain.txt | 44 +- .../approved-plans-v1_4/q68/explain.txt | 14 +- .../approved-plans-v1_4/q69.sf100/explain.txt | 96 ++-- .../approved-plans-v1_4/q69/explain.txt | 94 ++-- .../approved-plans-v1_4/q70.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q70/explain.txt | 2 +- .../approved-plans-v1_4/q71.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q71/explain.txt | 4 +- .../approved-plans-v1_4/q74.sf100/explain.txt | 268 ++++----- .../approved-plans-v1_4/q74/explain.txt | 250 ++++----- .../approved-plans-v1_4/q75.sf100/explain.txt | 368 ++++++------ .../approved-plans-v1_4/q75/explain.txt | 368 ++++++------ .../approved-plans-v1_4/q76.sf100/explain.txt | 80 +-- .../approved-plans-v1_4/q76/explain.txt | 64 +-- .../approved-plans-v1_4/q77.sf100/explain.txt | 284 +++++----- .../approved-plans-v1_4/q77/explain.txt | 300 +++++----- .../approved-plans-v1_4/q78.sf100/explain.txt | 146 ++--- .../approved-plans-v1_4/q78/explain.txt | 146 ++--- .../approved-plans-v1_4/q8.sf100/explain.txt | 120 ++-- .../approved-plans-v1_4/q8/explain.txt | 106 ++-- .../approved-plans-v1_4/q80.sf100/explain.txt | 204 +++---- .../approved-plans-v1_4/q80/explain.txt | 208 +++---- .../approved-plans-v1_4/q81.sf100/explain.txt | 144 ++--- .../approved-plans-v1_4/q81/explain.txt | 126 ++--- .../approved-plans-v1_4/q83.sf100/explain.txt | 210 +++---- .../approved-plans-v1_4/q83/explain.txt | 186 +++--- .../approved-plans-v1_4/q88.sf100/explain.txt | 504 ++++++++--------- .../approved-plans-v1_4/q88/explain.txt | 504 ++++++++--------- .../approved-plans-v1_4/q9.sf100/explain.txt | 416 +++++++------- .../approved-plans-v1_4/q9/explain.txt | 416 +++++++------- .../approved-plans-v1_4/q90.sf100/explain.txt | 80 +-- .../approved-plans-v1_4/q90/explain.txt | 80 +-- .../approved-plans-v1_4/q92/explain.txt | 103 ++-- .../approved-plans-v1_4/q94.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q94/explain.txt | 6 +- .../approved-plans-v1_4/q95.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q95/explain.txt | 6 +- .../approved-plans-v1_4/q97.sf100/explain.txt | 40 +- .../approved-plans-v1_4/q97/explain.txt | 40 +- .../q10a.sf100/explain.txt | 96 ++-- .../approved-plans-v2_7/q10a/explain.txt | 94 ++-- .../approved-plans-v2_7/q11.sf100/explain.txt | 268 ++++----- .../approved-plans-v2_7/q11/explain.txt | 250 ++++----- .../q18a.sf100/explain.txt | 402 ++++++------- .../approved-plans-v2_7/q18a/explain.txt | 410 +++++++------- .../q22a.sf100/explain.txt | 122 ++-- .../approved-plans-v2_7/q22a/explain.txt | 122 ++-- .../approved-plans-v2_7/q24.sf100/explain.txt | 182 +++--- .../approved-plans-v2_7/q24/explain.txt | 170 +++--- .../q27a.sf100/explain.txt | 162 +++--- .../approved-plans-v2_7/q27a/explain.txt | 162 +++--- .../approved-plans-v2_7/q35.sf100/explain.txt | 116 ++-- .../approved-plans-v2_7/q35/explain.txt | 96 ++-- .../q35a.sf100/explain.txt | 116 ++-- .../approved-plans-v2_7/q35a/explain.txt | 96 ++-- .../q36a.sf100/explain.txt | 92 +-- .../approved-plans-v2_7/q36a/explain.txt | 92 +-- .../approved-plans-v2_7/q47.sf100/explain.txt | 66 +-- .../approved-plans-v2_7/q47/explain.txt | 58 +- .../approved-plans-v2_7/q49.sf100/explain.txt | 198 +++---- .../approved-plans-v2_7/q49/explain.txt | 152 ++--- .../q51a.sf100/explain.txt | 244 ++++---- .../approved-plans-v2_7/q51a/explain.txt | 242 ++++---- .../approved-plans-v2_7/q57.sf100/explain.txt | 66 +-- .../approved-plans-v2_7/q57/explain.txt | 58 +- .../approved-plans-v2_7/q5a.sf100/explain.txt | 230 ++++---- .../approved-plans-v2_7/q5a/explain.txt | 242 ++++---- .../approved-plans-v2_7/q64.sf100/explain.txt | 316 +++++------ .../approved-plans-v2_7/q64/explain.txt | 284 +++++----- .../q67a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q67a/explain.txt | 2 +- .../q70a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q70a/explain.txt | 2 +- .../approved-plans-v2_7/q74.sf100/explain.txt | 268 ++++----- .../approved-plans-v2_7/q74/explain.txt | 250 ++++----- .../approved-plans-v2_7/q75.sf100/explain.txt | 368 ++++++------ .../approved-plans-v2_7/q75/explain.txt | 368 ++++++------ .../q77a.sf100/explain.txt | 364 ++++++------ .../approved-plans-v2_7/q77a/explain.txt | 380 ++++++------- .../approved-plans-v2_7/q78.sf100/explain.txt | 146 ++--- .../approved-plans-v2_7/q78/explain.txt | 146 ++--- .../q80a.sf100/explain.txt | 268 ++++----- .../approved-plans-v2_7/q80a/explain.txt | 272 ++++----- .../q86a.sf100/explain.txt | 92 +-- .../approved-plans-v2_7/q86a/explain.txt | 92 +-- 168 files changed, 12328 insertions(+), 12308 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt index 1fdcba1ed4de4..8a4b341ac925b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt @@ -137,22 +137,22 @@ Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] Condition : isnotnull(cs_ship_customer_sk#13) (19) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#8] (20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] +Right keys [1]: [d_date_sk#8] Join condition: None (21) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13 AS customer_sk#16] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] +Output [1]: [cs_ship_customer_sk#13 AS customer_sk#15] +Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#8] (22) Union (23) Exchange Input [1]: [customer_sk#12] -Arguments: hashpartitioning(customer_sk#12, 5), ENSURE_REQUIREMENTS, [id=#17] +Arguments: hashpartitioning(customer_sk#12, 5), ENSURE_REQUIREMENTS, [id=#16] (24) Sort [codegen id : 7] Input [1]: [customer_sk#12] @@ -164,43 +164,43 @@ Right keys [1]: [customer_sk#12] Join condition: None (26) Scan parquet default.store_sales -Output [2]: [ss_customer_sk#18, ss_sold_date_sk#19] +Output [2]: [ss_customer_sk#17, ss_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#19), dynamicpruningexpression(ss_sold_date_sk#19 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 9] -Input [2]: [ss_customer_sk#18, ss_sold_date_sk#19] +Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] (28) Filter [codegen id : 9] -Input [2]: [ss_customer_sk#18, ss_sold_date_sk#19] -Condition : isnotnull(ss_customer_sk#18) +Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] +Condition : isnotnull(ss_customer_sk#17) (29) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#20] +Output [1]: [d_date_sk#8] (30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#19] -Right keys [1]: [d_date_sk#20] +Left keys [1]: [ss_sold_date_sk#18] +Right keys [1]: [d_date_sk#8] Join condition: None (31) Project [codegen id : 9] -Output [1]: [ss_customer_sk#18 AS customer_sk#21] -Input [3]: [ss_customer_sk#18, ss_sold_date_sk#19, d_date_sk#20] +Output [1]: [ss_customer_sk#17 AS customer_sk#19] +Input [3]: [ss_customer_sk#17, ss_sold_date_sk#18, d_date_sk#8] (32) Exchange -Input [1]: [customer_sk#21] -Arguments: hashpartitioning(customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [customer_sk#19] +Arguments: hashpartitioning(customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#20] (33) Sort [codegen id : 10] -Input [1]: [customer_sk#21] -Arguments: [customer_sk#21 ASC NULLS FIRST], false, 0 +Input [1]: [customer_sk#19] +Arguments: [customer_sk#19 ASC NULLS FIRST], false, 0 (34) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#21] +Right keys [1]: [customer_sk#19] Join condition: None (35) Project [codegen id : 12] @@ -208,84 +208,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (36) Scan parquet default.customer_address -Output [2]: [ca_address_sk#23, ca_county#24] +Output [2]: [ca_address_sk#21, ca_county#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 11] -Input [2]: [ca_address_sk#23, ca_county#24] +Input [2]: [ca_address_sk#21, ca_county#22] (38) Filter [codegen id : 11] -Input [2]: [ca_address_sk#23, ca_county#24] -Condition : (ca_county#24 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#23)) +Input [2]: [ca_address_sk#21, ca_county#22] +Condition : (ca_county#22 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#21)) (39) Project [codegen id : 11] -Output [1]: [ca_address_sk#23] -Input [2]: [ca_address_sk#23, ca_county#24] +Output [1]: [ca_address_sk#21] +Input [2]: [ca_address_sk#21, ca_county#22] (40) BroadcastExchange -Input [1]: [ca_address_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] +Input [1]: [ca_address_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#23] +Right keys [1]: [ca_address_sk#21] Join condition: None (42) Project [codegen id : 12] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#23] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21] (43) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (44) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (45) ColumnarToRow -Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (46) Filter -Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -Condition : isnotnull(cd_demo_sk#27) +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Condition : isnotnull(cd_demo_sk#25) (47) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#27] +Right keys [1]: [cd_demo_sk#25] Join condition: None (48) Project [codegen id : 13] -Output [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (49) HashAggregate [codegen id : 13] -Input [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#36] -Results [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] +Aggregate Attributes [1]: [count#34] +Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#35] (50) Exchange -Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] -Arguments: hashpartitioning(cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#35] +Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#36] (51) HashAggregate [codegen id : 14] -Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] -Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#35] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#39] -Results [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, count(1)#39 AS cnt1#40, cd_purchase_estimate#31, count(1)#39 AS cnt2#41, cd_credit_rating#32, count(1)#39 AS cnt3#42, cd_dep_count#33, count(1)#39 AS cnt4#43, cd_dep_employed_count#34, count(1)#39 AS cnt5#44, cd_dep_college_count#35, count(1)#39 AS cnt6#45] +Aggregate Attributes [1]: [count(1)#37] +Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#37 AS cnt1#38, cd_purchase_estimate#29, count(1)#37 AS cnt2#39, cd_credit_rating#30, count(1)#37 AS cnt3#40, cd_dep_count#31, count(1)#37 AS cnt4#41, cd_dep_employed_count#32, count(1)#37 AS cnt5#42, cd_dep_college_count#33, count(1)#37 AS cnt6#43] (52) TakeOrderedAndProject -Input [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#40, cd_purchase_estimate#31, cnt2#41, cd_credit_rating#32, cnt3#42, cd_dep_count#33, cnt4#43, cd_dep_employed_count#34, cnt5#44, cd_dep_college_count#35, cnt6#45] -Arguments: 100, [cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_education_status#30 ASC NULLS FIRST, cd_purchase_estimate#31 ASC NULLS FIRST, cd_credit_rating#32 ASC NULLS FIRST, cd_dep_count#33 ASC NULLS FIRST, cd_dep_employed_count#34 ASC NULLS FIRST, cd_dep_college_count#35 ASC NULLS FIRST], [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#40, cd_purchase_estimate#31, cnt2#41, cd_credit_rating#32, cnt3#42, cd_dep_count#33, cnt4#43, cd_dep_employed_count#34, cnt5#44, cd_dep_college_count#35, cnt6#45] +Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#38, cd_purchase_estimate#29, cnt2#39, cd_credit_rating#30, cnt3#40, cd_dep_count#31, cnt4#41, cd_dep_employed_count#32, cnt5#42, cd_dep_college_count#33, cnt6#43] +Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#38, cd_purchase_estimate#29, cnt2#39, cd_credit_rating#30, cnt3#40, cd_dep_count#31, cnt4#41, cd_dep_employed_count#32, cnt5#42, cd_dep_college_count#33, cnt6#43] ===== Subqueries ===== @@ -298,6 +298,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#19 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt index ff55b1c511456..6f64579d09c78 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt @@ -125,22 +125,22 @@ Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] Condition : isnotnull(cs_ship_customer_sk#12) (17) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#7] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#7] Join condition: None (19) Project [codegen id : 4] -Output [1]: [cs_ship_customer_sk#12 AS customer_sk#15] -Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] +Output [1]: [cs_ship_customer_sk#12 AS customer_sk#14] +Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#7] (20) Union (21) BroadcastExchange Input [1]: [customer_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (22) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] @@ -148,39 +148,39 @@ Right keys [1]: [customer_sk#11] Join condition: None (23) Scan parquet default.store_sales -Output [2]: [ss_customer_sk#17, ss_sold_date_sk#18] +Output [2]: [ss_customer_sk#16, ss_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ss_sold_date_sk#17), dynamicpruningexpression(ss_sold_date_sk#17 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] +Input [2]: [ss_customer_sk#16, ss_sold_date_sk#17] (25) Filter [codegen id : 6] -Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#17) +Input [2]: [ss_customer_sk#16, ss_sold_date_sk#17] +Condition : isnotnull(ss_customer_sk#16) (26) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#19] +Output [1]: [d_date_sk#7] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#18] -Right keys [1]: [d_date_sk#19] +Left keys [1]: [ss_sold_date_sk#17] +Right keys [1]: [d_date_sk#7] Join condition: None (28) Project [codegen id : 6] -Output [1]: [ss_customer_sk#17 AS customer_sk#20] -Input [3]: [ss_customer_sk#17, ss_sold_date_sk#18, d_date_sk#19] +Output [1]: [ss_customer_sk#16 AS customer_sk#18] +Input [3]: [ss_customer_sk#16, ss_sold_date_sk#17, d_date_sk#7] (29) BroadcastExchange -Input [1]: [customer_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [customer_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (30) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#20] +Right keys [1]: [customer_sk#18] Join condition: None (31) Project [codegen id : 9] @@ -188,84 +188,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (32) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_county#23] +Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (33) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#22, ca_county#23] +Input [2]: [ca_address_sk#20, ca_county#21] (34) Filter [codegen id : 7] -Input [2]: [ca_address_sk#22, ca_county#23] -Condition : (ca_county#23 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#22)) +Input [2]: [ca_address_sk#20, ca_county#21] +Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) (35) Project [codegen id : 7] -Output [1]: [ca_address_sk#22] -Input [2]: [ca_address_sk#22, ca_county#23] +Output [1]: [ca_address_sk#20] +Input [2]: [ca_address_sk#20, ca_county#21] (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#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#20] Join condition: None (38) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#22] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] (39) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] (41) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Condition : isnotnull(cd_demo_sk#25) +Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Condition : isnotnull(cd_demo_sk#23) (42) BroadcastExchange -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] (43) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#23] Join condition: None (44) Project [codegen id : 9] -Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] (45) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#35] -Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Aggregate Attributes [1]: [count#33] +Results [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] (46) Exchange -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] +Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#35] (47) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] +Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] +Aggregate Attributes [1]: [count(1)#36] +Results [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#36 AS cnt1#37, cd_purchase_estimate#27, count(1)#36 AS cnt2#38, cd_credit_rating#28, count(1)#36 AS cnt3#39, cd_dep_count#29, count(1)#36 AS cnt4#40, cd_dep_employed_count#30, count(1)#36 AS cnt5#41, cd_dep_college_count#31, count(1)#36 AS cnt6#42] (48) TakeOrderedAndProject -Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] +Input [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] +Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] ===== Subqueries ===== @@ -278,6 +278,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 14 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#17 IN dynamicpruning#6 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 8f7cbc7d0ffbc..baed833e42a47 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +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#61), (ss_sold_date_sk#61 >= 2451545), (ss_sold_date_sk#61 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#61 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 (34) ColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +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] (35) Filter [codegen id : 11] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] -Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) +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)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#62] +Output [1]: [d_date_sk#10] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#61] -Right keys [1]: [d_date_sk#62] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] Join condition: None (38) Project [codegen id : 11] -Output [7]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] -Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#62] +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] (39) Scan parquet default.store -Output [2]: [s_store_sk#63, s_state#64] +Output [2]: [s_store_sk#18, s_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_state, [TN,AL,SD]), IsNotNull(s_store_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 8] -Input [2]: [s_store_sk#63, s_state#64] +Input [2]: [s_store_sk#18, s_state#19] (41) Filter [codegen id : 8] -Input [2]: [s_store_sk#63, s_state#64] -Condition : (s_state#64 IN (TN,AL,SD) AND isnotnull(s_store_sk#63)) +Input [2]: [s_store_sk#18, s_state#19] +Condition : (s_state#19 IN (TN,AL,SD) AND isnotnull(s_store_sk#18)) (42) Project [codegen id : 8] -Output [1]: [s_store_sk#63] -Input [2]: [s_store_sk#63, s_state#64] +Output [1]: [s_store_sk#18] +Input [2]: [s_store_sk#18, s_state#19] (43) BroadcastExchange -Input [1]: [s_store_sk#63] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] +Input [1]: [s_store_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] (44) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#56] -Right keys [1]: [s_store_sk#63] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None (45) Project [codegen id : 11] -Output [6]: [ss_item_sk#54, ss_cdemo_sk#55, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, s_store_sk#63] +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#18] (46) ReusedExchange [Reuses operator id: 15] -Output [1]: [cd_demo_sk#66] +Output [1]: [cd_demo_sk#13] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#55] -Right keys [1]: [cd_demo_sk#66] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#13] Join condition: None (48) Project [codegen id : 11] -Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] -Input [7]: [ss_item_sk#54, ss_cdemo_sk#55, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, cd_demo_sk#66] +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#13] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#67, i_item_id#68] +Output [2]: [i_item_sk#21, i_item_id#22] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#67] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#21] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] -Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] +Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] +Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#68] +Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#22] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] -Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] +Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] (53) Exchange -Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Keys [1]: [i_item_id#68] +Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Keys [1]: [i_item_id#22] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] -Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] +Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] +Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] (55) Scan parquet default.store_sales -Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +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#103), (ss_sold_date_sk#103 >= 2451545), (ss_sold_date_sk#103 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#103 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 (56) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +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] (57) Filter [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) +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)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#104] +Output [1]: [d_date_sk#10] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#103] -Right keys [1]: [d_date_sk#104] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] Join condition: None (60) Project [codegen id : 17] -Output [7]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#104] +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] (61) ReusedExchange [Reuses operator id: 43] -Output [1]: [s_store_sk#105] +Output [1]: [s_store_sk#18] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#98] -Right keys [1]: [s_store_sk#105] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#96, ss_cdemo_sk#97, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#105] +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#18] (64) ReusedExchange [Reuses operator id: 15] -Output [1]: [cd_demo_sk#106] +Output [1]: [cd_demo_sk#13] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#97] -Right keys [1]: [cd_demo_sk#106] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#13] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [7]: [ss_item_sk#96, ss_cdemo_sk#97, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, cd_demo_sk#106] +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#13] (67) Scan parquet default.item -Output [1]: [i_item_sk#107] +Output [1]: [i_item_sk#21] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#107] +Input [1]: [i_item_sk#21] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#107] -Condition : isnotnull(i_item_sk#107) +Input [1]: [i_item_sk#21] +Condition : isnotnull(i_item_sk#21) (70) BroadcastExchange -Input [1]: [i_item_sk#107] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] +Input [1]: [i_item_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#96] -Right keys [1]: [i_item_sk#107] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#21] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] -Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] +Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] +Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] -Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] +Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] (74) Exchange -Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] +Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] -Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] +Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] +Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 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 bfb9e8384817d..735f763ae5d20 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +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#61), (ss_sold_date_sk#61 >= 2451545), (ss_sold_date_sk#61 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#61 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 (34) ColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +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] (35) Filter [codegen id : 11] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] -Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) +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)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#62] +Output [1]: [cd_demo_sk#10] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#55] -Right keys [1]: [cd_demo_sk#62] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (38) Project [codegen id : 11] -Output [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] -Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, cd_demo_sk#62] +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] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#63] +Output [1]: [d_date_sk#15] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#61] -Right keys [1]: [d_date_sk#63] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (41) Project [codegen id : 11] -Output [6]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] -Input [8]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#63] +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] (42) Scan parquet default.store -Output [2]: [s_store_sk#64, s_state#65] +Output [2]: [s_store_sk#18, s_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_state, [TN,AL,SD]), IsNotNull(s_store_sk)] ReadSchema: struct (43) ColumnarToRow [codegen id : 9] -Input [2]: [s_store_sk#64, s_state#65] +Input [2]: [s_store_sk#18, s_state#19] (44) Filter [codegen id : 9] -Input [2]: [s_store_sk#64, s_state#65] -Condition : (s_state#65 IN (TN,AL,SD) AND isnotnull(s_store_sk#64)) +Input [2]: [s_store_sk#18, s_state#19] +Condition : (s_state#19 IN (TN,AL,SD) AND isnotnull(s_store_sk#18)) (45) Project [codegen id : 9] -Output [1]: [s_store_sk#64] -Input [2]: [s_store_sk#64, s_state#65] +Output [1]: [s_store_sk#18] +Input [2]: [s_store_sk#18, s_state#19] (46) BroadcastExchange -Input [1]: [s_store_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#66] +Input [1]: [s_store_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#56] -Right keys [1]: [s_store_sk#64] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None (48) Project [codegen id : 11] -Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] -Input [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, s_store_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_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#18] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#67, i_item_id#68] +Output [2]: [i_item_sk#21, i_item_id#22] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#67] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#21] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] -Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] +Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] +Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#68] +Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#22] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] -Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] +Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] (53) Exchange -Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Keys [1]: [i_item_id#68] +Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Keys [1]: [i_item_id#22] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] -Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] +Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] +Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] (55) Scan parquet default.store_sales -Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +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#103), (ss_sold_date_sk#103 >= 2451545), (ss_sold_date_sk#103 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#103 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 (56) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +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] (57) Filter [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) +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)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#104] +Output [1]: [cd_demo_sk#10] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#97] -Right keys [1]: [cd_demo_sk#104] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (60) Project [codegen id : 17] -Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] +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] (61) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#105] +Output [1]: [d_date_sk#15] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#103] -Right keys [1]: [d_date_sk#105] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#105] +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] (64) ReusedExchange [Reuses operator id: 46] -Output [1]: [s_store_sk#106] +Output [1]: [s_store_sk#18] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#98] -Right keys [1]: [s_store_sk#106] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#106] +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#18] (67) Scan parquet default.item -Output [1]: [i_item_sk#107] +Output [1]: [i_item_sk#21] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#107] +Input [1]: [i_item_sk#21] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#107] -Condition : isnotnull(i_item_sk#107) +Input [1]: [i_item_sk#21] +Condition : isnotnull(i_item_sk#21) (70) BroadcastExchange -Input [1]: [i_item_sk#107] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] +Input [1]: [i_item_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#96] -Right keys [1]: [i_item_sk#107] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#21] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] -Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] +Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] +Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] -Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] +Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] (74) Exchange -Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] +Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] -Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] +Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] +Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#15] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt index e62003f82faec..4eb212c336669 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt @@ -261,24 +261,24 @@ Input [7]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_add Arguments: [c_current_addr_sk#36 ASC NULLS FIRST], false, 0 (47) ReusedExchange [Reuses operator id: 30] -Output [2]: [ca_address_sk#41, ca_city#42] +Output [2]: [ca_address_sk#22, ca_city#23] (48) Sort [codegen id : 15] -Input [2]: [ca_address_sk#41, ca_city#42] -Arguments: [ca_address_sk#41 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#22, ca_city#23] +Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#36] -Right keys [1]: [ca_address_sk#41] -Join condition: NOT (ca_city#42 = bought_city#31) +Right keys [1]: [ca_address_sk#22] +Join condition: NOT (ca_city#23 = bought_city#31) (50) Project [codegen id : 16] -Output [7]: [c_last_name#38, c_first_name#37, ca_city#42, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#36, c_first_name#37, c_last_name#38, ca_address_sk#41, ca_city#42] +Output [7]: [c_last_name#38, c_first_name#37, ca_city#23, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#36, c_first_name#37, c_last_name#38, ca_address_sk#22, ca_city#23] (51) TakeOrderedAndProject -Input [7]: [c_last_name#38, c_first_name#37, ca_city#42, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Arguments: 100, [c_last_name#38 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, ca_city#42 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#38, c_first_name#37, ca_city#42, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [7]: [c_last_name#38, c_first_name#37, ca_city#23, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Arguments: 100, [c_last_name#38 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, ca_city#23 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#38, c_first_name#37, ca_city#23, bought_city#31, ss_ticket_number#5, amt#32, profit#33] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt index a00e2eeac49b7..951558ca1b130 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt @@ -225,20 +225,20 @@ Output [7]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_ad Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#31, amt#32, profit#33, c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#39, ca_city#40] +Output [2]: [ca_address_sk#21, ca_city#22] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#39] -Join condition: NOT (ca_city#40 = bought_city#31) +Right keys [1]: [ca_address_sk#21] +Join condition: NOT (ca_city#22 = bought_city#31) (42) Project [codegen id : 8] -Output [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#39, ca_city#40] +Output [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#21, ca_city#22] (43) TakeOrderedAndProject -Input [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#40 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#22 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] ===== Subqueries ===== 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..4cdcd9685acbe 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))#67, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72] +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))#67,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72,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#37, s_store_id#38] 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#37, s_store_id#38] (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#37, s_store_id#38] +Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) (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#37, s_store_id#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] (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#37] 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#38] +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#37, s_store_id#38] (41) Scan parquet default.date_dim -Output [2]: [d_month_seq#82, d_week_seq#83] +Output [2]: [d_month_seq#74, d_week_seq#75] 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#74, d_week_seq#75] (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#74, d_week_seq#75] +Condition : (((isnotnull(d_month_seq#74) AND (d_month_seq#74 >= 1197)) AND (d_month_seq#74 <= 1208)) AND isnotnull(d_week_seq#75)) (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#75] +Input [2]: [d_month_seq#74, d_week_seq#75] (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#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] (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#75] 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#77, s_store_id#38 AS s_store_id2#78, sun_sales#30 AS sun_sales2#79, mon_sales#31 AS mon_sales2#80, wed_sales#33 AS wed_sales2#81, thu_sales#34 AS thu_sales2#82, fri_sales#35 AS fri_sales2#83, sat_sales#36 AS sat_sales2#84] +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#38, d_week_seq#75] (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#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#85] (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#78, (d_week_seq2#77 - 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#79)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#86, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#80)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#87, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#88, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#81)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#89, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#82)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#90, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#83)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#91, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#84)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#92] +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#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] (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)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] +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)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] 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..4cdcd9685acbe 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))#67, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72] +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))#67,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72,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#37, s_store_id#38] 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#37, s_store_id#38] (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#37, s_store_id#38] +Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) (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#37, s_store_id#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] (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#37] 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#38] +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#37, s_store_id#38] (41) Scan parquet default.date_dim -Output [2]: [d_month_seq#82, d_week_seq#83] +Output [2]: [d_month_seq#74, d_week_seq#75] 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#74, d_week_seq#75] (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#74, d_week_seq#75] +Condition : (((isnotnull(d_month_seq#74) AND (d_month_seq#74 >= 1197)) AND (d_month_seq#74 <= 1208)) AND isnotnull(d_week_seq#75)) (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#75] +Input [2]: [d_month_seq#74, d_week_seq#75] (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#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] (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#75] 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#77, s_store_id#38 AS s_store_id2#78, sun_sales#30 AS sun_sales2#79, mon_sales#31 AS mon_sales2#80, wed_sales#33 AS wed_sales2#81, thu_sales#34 AS thu_sales2#82, fri_sales#35 AS fri_sales2#83, sat_sales#36 AS sat_sales2#84] +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#38, d_week_seq#75] (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#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#85] (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#78, (d_week_seq2#77 - 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#79)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#86, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#80)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#87, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#88, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#81)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#89, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#82)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#90, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#83)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#91, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#84)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#92] +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#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] (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)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] +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)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt index 240f579025038..741a644c4fdbc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt @@ -127,123 +127,123 @@ Input [4]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17 Condition : isnotnull(ss_store_sk#15) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#6] (19) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#17] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#6] Join condition: None (20) Project [codegen id : 4] Output [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] -Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#18] +Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#6] (21) HashAggregate [codegen id : 4] Input [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] +Aggregate Attributes [1]: [sum#18] +Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] (22) Exchange -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] -Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] +Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#20] (23) HashAggregate [codegen id : 5] -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#22] -Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#22,17,2) AS revenue#23] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#21] +Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#21,17,2) AS revenue#22] (24) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#15, revenue#23] +Input [2]: [ss_store_sk#15, revenue#22] Keys [1]: [ss_store_sk#15] -Functions [1]: [partial_avg(revenue#23)] -Aggregate Attributes [2]: [sum#24, count#25] -Results [3]: [ss_store_sk#15, sum#26, count#27] +Functions [1]: [partial_avg(revenue#22)] +Aggregate Attributes [2]: [sum#23, count#24] +Results [3]: [ss_store_sk#15, sum#25, count#26] (25) Exchange -Input [3]: [ss_store_sk#15, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [ss_store_sk#15, sum#25, count#26] +Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#27] (26) HashAggregate [codegen id : 6] -Input [3]: [ss_store_sk#15, sum#26, count#27] +Input [3]: [ss_store_sk#15, sum#25, count#26] Keys [1]: [ss_store_sk#15] -Functions [1]: [avg(revenue#23)] -Aggregate Attributes [1]: [avg(revenue#23)#29] -Results [2]: [ss_store_sk#15, avg(revenue#23)#29 AS ave#30] +Functions [1]: [avg(revenue#22)] +Aggregate Attributes [1]: [avg(revenue#22)#28] +Results [2]: [ss_store_sk#15, avg(revenue#22)#28 AS ave#29] (27) BroadcastExchange -Input [2]: [ss_store_sk#15, ave#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [2]: [ss_store_sk#15, ave#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] (28) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_store_sk#2] Right keys [1]: [ss_store_sk#15] -Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#30)), DecimalType(23,7), true)) +Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#29)), DecimalType(23,7), true)) (29) Project [codegen id : 7] Output [3]: [ss_store_sk#2, ss_item_sk#1, revenue#13] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#30] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#29] (30) BroadcastExchange Input [3]: [ss_store_sk#2, ss_item_sk#1, revenue#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (31) Scan parquet default.store -Output [2]: [s_store_sk#33, s_store_name#34] +Output [2]: [s_store_sk#32, s_store_name#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (32) ColumnarToRow -Input [2]: [s_store_sk#33, s_store_name#34] +Input [2]: [s_store_sk#32, s_store_name#33] (33) Filter -Input [2]: [s_store_sk#33, s_store_name#34] -Condition : isnotnull(s_store_sk#33) +Input [2]: [s_store_sk#32, s_store_name#33] +Condition : isnotnull(s_store_sk#32) (34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#33] +Right keys [1]: [s_store_sk#32] Join condition: None (35) Project [codegen id : 8] -Output [3]: [ss_item_sk#1, revenue#13, s_store_name#34] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#33, s_store_name#34] +Output [3]: [ss_item_sk#1, revenue#13, s_store_name#33] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#32, s_store_name#33] (36) BroadcastExchange -Input [3]: [ss_item_sk#1, revenue#13, s_store_name#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [3]: [ss_item_sk#1, revenue#13, s_store_name#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] (37) Scan parquet default.item -Output [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Output [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (38) ColumnarToRow -Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] (39) Filter -Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] -Condition : isnotnull(i_item_sk#36) +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Condition : isnotnull(i_item_sk#35) (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#36] +Right keys [1]: [i_item_sk#35] Join condition: None (41) Project [codegen id : 9] -Output [6]: [s_store_name#34, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] -Input [8]: [ss_item_sk#1, revenue#13, s_store_name#34, i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Output [6]: [s_store_name#33, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [8]: [ss_item_sk#1, revenue#13, s_store_name#33, i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] (42) TakeOrderedAndProject -Input [6]: [s_store_name#34, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] -Arguments: 100, [s_store_name#34 ASC NULLS FIRST, i_item_desc#37 ASC NULLS FIRST], [s_store_name#34, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Input [6]: [s_store_name#33, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Arguments: 100, [s_store_name#33 ASC NULLS FIRST, i_item_desc#36 ASC NULLS FIRST], [s_store_name#33, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt index 43ef320ed235f..0851190e54fe0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt @@ -181,65 +181,65 @@ Input [4]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26 Condition : isnotnull(ss_store_sk#24) (30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#8] (31) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#8] Join condition: None (32) Project [codegen id : 6] Output [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] -Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#27] +Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#8] (33) HashAggregate [codegen id : 6] Input [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#28] -Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] +Aggregate Attributes [1]: [sum#27] +Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] (34) Exchange -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] -Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] +Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#29] (35) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] -Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS revenue#32] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#30] +Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#30,17,2) AS revenue#31] (36) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#24, revenue#32] +Input [2]: [ss_store_sk#24, revenue#31] Keys [1]: [ss_store_sk#24] -Functions [1]: [partial_avg(revenue#32)] -Aggregate Attributes [2]: [sum#33, count#34] -Results [3]: [ss_store_sk#24, sum#35, count#36] +Functions [1]: [partial_avg(revenue#31)] +Aggregate Attributes [2]: [sum#32, count#33] +Results [3]: [ss_store_sk#24, sum#34, count#35] (37) Exchange -Input [3]: [ss_store_sk#24, sum#35, count#36] -Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [ss_store_sk#24, sum#34, count#35] +Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#36] (38) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#24, sum#35, count#36] +Input [3]: [ss_store_sk#24, sum#34, count#35] Keys [1]: [ss_store_sk#24] -Functions [1]: [avg(revenue#32)] -Aggregate Attributes [1]: [avg(revenue#32)#38] -Results [2]: [ss_store_sk#24, avg(revenue#32)#38 AS ave#39] +Functions [1]: [avg(revenue#31)] +Aggregate Attributes [1]: [avg(revenue#31)#37] +Results [2]: [ss_store_sk#24, avg(revenue#31)#37 AS ave#38] (39) BroadcastExchange -Input [2]: [ss_store_sk#24, ave#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] +Input [2]: [ss_store_sk#24, ave#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#24] -Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#39)), DecimalType(23,7), true)) +Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#38)), DecimalType(23,7), true)) (41) Project [codegen id : 9] Output [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#39] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#38] (42) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt index 0b96b76a0ffc4..10ac72a6b44de 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt @@ -232,31 +232,31 @@ Input [8]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37 Arguments: HashedRelationBroadcastMode(List(cast(input[5, int, true] as bigint)),false), [id=#44] (41) Scan parquet default.customer_address -Output [2]: [ca_address_sk#45, ca_city#46] +Output [2]: [ca_address_sk#23, ca_city#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] ReadSchema: struct (42) ColumnarToRow -Input [2]: [ca_address_sk#45, ca_city#46] +Input [2]: [ca_address_sk#23, ca_city#24] (43) Filter -Input [2]: [ca_address_sk#45, ca_city#46] -Condition : (isnotnull(ca_address_sk#45) AND isnotnull(ca_city#46)) +Input [2]: [ca_address_sk#23, ca_city#24] +Condition : (isnotnull(ca_address_sk#23) AND isnotnull(ca_city#24)) (44) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#41] -Right keys [1]: [ca_address_sk#45] -Join condition: NOT (ca_city#46 = bought_city#35) +Right keys [1]: [ca_address_sk#23] +Join condition: NOT (ca_city#24 = bought_city#35) (45) Project [codegen id : 8] -Output [8]: [c_last_name#43, c_first_name#42, ca_city#46, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#41, c_first_name#42, c_last_name#43, ca_address_sk#45, ca_city#46] +Output [8]: [c_last_name#43, c_first_name#42, ca_city#24, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#41, c_first_name#42, c_last_name#43, ca_address_sk#23, ca_city#24] (46) TakeOrderedAndProject -Input [8]: [c_last_name#43, c_first_name#42, ca_city#46, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Arguments: 100, [c_last_name#43 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#43, c_first_name#42, ca_city#46, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [8]: [c_last_name#43, c_first_name#42, ca_city#24, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Arguments: 100, [c_last_name#43 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#43, c_first_name#42, ca_city#24, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt index f50329db7e05e..81faa98299f82 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt @@ -225,20 +225,20 @@ Output [8]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#3 Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_customer_sk#39, c_current_addr_sk#40, c_first_name#41, c_last_name#42] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#44, ca_city#45] +Output [2]: [ca_address_sk#22, ca_city#23] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#40] -Right keys [1]: [ca_address_sk#44] -Join condition: NOT (ca_city#45 = bought_city#35) +Right keys [1]: [ca_address_sk#22] +Join condition: NOT (ca_city#23 = bought_city#35) (42) Project [codegen id : 8] -Output [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#44, ca_city#45] +Output [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#22, ca_city#23] (43) TakeOrderedAndProject -Input [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] ===== Subqueries ===== 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 a950e7b9e8639..9c68e55238aee 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 @@ -117,158 +117,158 @@ Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) (15) Scan parquet default.store_returns -Output [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] +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#19), dynamicpruningexpression(sr_returned_date_sk#19 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 (16) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] +Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] (17) Filter [codegen id : 4] -Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#17) +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) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#20] +Output [1]: [d_date_sk#6] (19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [cast(d_date_sk#20 as bigint)] +Left keys [1]: [sr_returned_date_sk#4] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (20) Project [codegen id : 4] -Output [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] -Input [5]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19, d_date_sk#20] +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] (21) HashAggregate [codegen id : 4] -Input [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] -Keys [2]: [sr_customer_sk#16, sr_store_sk#17] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#18))] -Aggregate Attributes [1]: [sum#21] -Results [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] +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#16] +Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] (22) Exchange -Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] -Arguments: hashpartitioning(sr_customer_sk#16, sr_store_sk#17, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] +Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#18] (23) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] -Keys [2]: [sr_customer_sk#16, sr_store_sk#17] -Functions [1]: [sum(UnscaledValue(sr_return_amt#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#18))#24] -Results [2]: [sr_store_sk#17 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#18))#24,17,2) AS ctr_total_return#15] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] +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))#19] +Results [2]: [sr_store_sk#2 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#19,17,2) AS ctr_total_return#15] (24) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#14, ctr_total_return#15] Keys [1]: [ctr_store_sk#14] Functions [1]: [partial_avg(ctr_total_return#15)] -Aggregate Attributes [2]: [sum#25, count#26] -Results [3]: [ctr_store_sk#14, sum#27, count#28] +Aggregate Attributes [2]: [sum#20, count#21] +Results [3]: [ctr_store_sk#14, sum#22, count#23] (25) Exchange -Input [3]: [ctr_store_sk#14, sum#27, count#28] -Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ctr_store_sk#14, sum#22, count#23] +Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#24] (26) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#14, sum#27, count#28] +Input [3]: [ctr_store_sk#14, sum#22, count#23] Keys [1]: [ctr_store_sk#14] Functions [1]: [avg(ctr_total_return#15)] -Aggregate Attributes [1]: [avg(ctr_total_return#15)#30] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14 AS ctr_store_sk#14#32] +Aggregate Attributes [1]: [avg(ctr_total_return#15)#25] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#25) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14 AS ctr_store_sk#14#27] (27) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) +Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#26) (28) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#33] +Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] +Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#28] (29) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [ctr_store_sk#14#32] -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) +Right keys [1]: [ctr_store_sk#14#27] +Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#26) (30) Project [codegen id : 8] Output [2]: [ctr_customer_sk#13, ctr_store_sk#14] -Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] +Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] (31) Scan parquet default.store -Output [2]: [s_store_sk#34, s_state#35] +Output [2]: [s_store_sk#29, s_state#30] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#34, s_state#35] +Input [2]: [s_store_sk#29, s_state#30] (33) Filter [codegen id : 7] -Input [2]: [s_store_sk#34, s_state#35] -Condition : ((isnotnull(s_state#35) AND (s_state#35 = TN)) AND isnotnull(s_store_sk#34)) +Input [2]: [s_store_sk#29, s_state#30] +Condition : ((isnotnull(s_state#30) AND (s_state#30 = TN)) AND isnotnull(s_store_sk#29)) (34) Project [codegen id : 7] -Output [1]: [s_store_sk#34] -Input [2]: [s_store_sk#34, s_state#35] +Output [1]: [s_store_sk#29] +Input [2]: [s_store_sk#29, s_state#30] (35) BroadcastExchange -Input [1]: [s_store_sk#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [1]: [s_store_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (36) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [cast(s_store_sk#34 as bigint)] +Right keys [1]: [cast(s_store_sk#29 as bigint)] Join condition: None (37) Project [codegen id : 8] Output [1]: [ctr_customer_sk#13] -Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#34] +Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#29] (38) Exchange Input [1]: [ctr_customer_sk#13] -Arguments: hashpartitioning(ctr_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#37] +Arguments: hashpartitioning(ctr_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#32] (39) Sort [codegen id : 9] Input [1]: [ctr_customer_sk#13] Arguments: [ctr_customer_sk#13 ASC NULLS FIRST], false, 0 (40) Scan parquet default.customer -Output [2]: [c_customer_sk#38, c_customer_id#39] +Output [2]: [c_customer_sk#33, c_customer_id#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [2]: [c_customer_sk#38, c_customer_id#39] +Input [2]: [c_customer_sk#33, c_customer_id#34] (42) Filter [codegen id : 10] -Input [2]: [c_customer_sk#38, c_customer_id#39] -Condition : isnotnull(c_customer_sk#38) +Input [2]: [c_customer_sk#33, c_customer_id#34] +Condition : isnotnull(c_customer_sk#33) (43) Exchange -Input [2]: [c_customer_sk#38, c_customer_id#39] -Arguments: hashpartitioning(cast(c_customer_sk#38 as bigint), 5), ENSURE_REQUIREMENTS, [id=#40] +Input [2]: [c_customer_sk#33, c_customer_id#34] +Arguments: hashpartitioning(cast(c_customer_sk#33 as bigint), 5), ENSURE_REQUIREMENTS, [id=#35] (44) Sort [codegen id : 11] -Input [2]: [c_customer_sk#38, c_customer_id#39] -Arguments: [cast(c_customer_sk#38 as bigint) ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#33, c_customer_id#34] +Arguments: [cast(c_customer_sk#33 as bigint) ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 12] Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [cast(c_customer_sk#38 as bigint)] +Right keys [1]: [cast(c_customer_sk#33 as bigint)] Join condition: None (46) Project [codegen id : 12] -Output [1]: [c_customer_id#39] -Input [3]: [ctr_customer_sk#13, c_customer_sk#38, c_customer_id#39] +Output [1]: [c_customer_id#34] +Input [3]: [ctr_customer_sk#13, c_customer_sk#33, c_customer_id#34] (47) TakeOrderedAndProject -Input [1]: [c_customer_id#39] -Arguments: 100, [c_customer_id#39 ASC NULLS FIRST], [c_customer_id#39] +Input [1]: [c_customer_id#34] +Arguments: 100, [c_customer_id#34 ASC NULLS FIRST], [c_customer_id#34] ===== Subqueries ===== @@ -279,6 +279,6 @@ ReusedExchange (48) (48) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 15 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 15 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 015228686d39b..4487f04c8293d 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 @@ -114,146 +114,146 @@ Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) (15) Scan parquet default.store_returns -Output [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] +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#19), dynamicpruningexpression(sr_returned_date_sk#19 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 (16) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] +Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] (17) Filter [codegen id : 4] -Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] -Condition : isnotnull(sr_store_sk#17) +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) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#20] +Output [1]: [d_date_sk#6] (19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#19] -Right keys [1]: [cast(d_date_sk#20 as bigint)] +Left keys [1]: [sr_returned_date_sk#4] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (20) Project [codegen id : 4] -Output [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] -Input [5]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19, d_date_sk#20] +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] (21) HashAggregate [codegen id : 4] -Input [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] -Keys [2]: [sr_customer_sk#16, sr_store_sk#17] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#18))] -Aggregate Attributes [1]: [sum#21] -Results [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] +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#16] +Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] (22) Exchange -Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] -Arguments: hashpartitioning(sr_customer_sk#16, sr_store_sk#17, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] +Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#18] (23) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] -Keys [2]: [sr_customer_sk#16, sr_store_sk#17] -Functions [1]: [sum(UnscaledValue(sr_return_amt#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#18))#24] -Results [2]: [sr_store_sk#17 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#18))#24,17,2) AS ctr_total_return#15] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] +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))#19] +Results [2]: [sr_store_sk#2 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#19,17,2) AS ctr_total_return#15] (24) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#14, ctr_total_return#15] Keys [1]: [ctr_store_sk#14] Functions [1]: [partial_avg(ctr_total_return#15)] -Aggregate Attributes [2]: [sum#25, count#26] -Results [3]: [ctr_store_sk#14, sum#27, count#28] +Aggregate Attributes [2]: [sum#20, count#21] +Results [3]: [ctr_store_sk#14, sum#22, count#23] (25) Exchange -Input [3]: [ctr_store_sk#14, sum#27, count#28] -Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ctr_store_sk#14, sum#22, count#23] +Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#24] (26) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#14, sum#27, count#28] +Input [3]: [ctr_store_sk#14, sum#22, count#23] Keys [1]: [ctr_store_sk#14] Functions [1]: [avg(ctr_total_return#15)] -Aggregate Attributes [1]: [avg(ctr_total_return#15)#30] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14 AS ctr_store_sk#14#32] +Aggregate Attributes [1]: [avg(ctr_total_return#15)#25] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#25) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14 AS ctr_store_sk#14#27] (27) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) +Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#26) (28) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#33] +Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] +Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#28] (29) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [ctr_store_sk#14#32] -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) +Right keys [1]: [ctr_store_sk#14#27] +Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#26) (30) Project [codegen id : 9] Output [2]: [ctr_customer_sk#13, ctr_store_sk#14] -Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] +Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] (31) Scan parquet default.store -Output [2]: [s_store_sk#34, s_state#35] +Output [2]: [s_store_sk#29, s_state#30] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#34, s_state#35] +Input [2]: [s_store_sk#29, s_state#30] (33) Filter [codegen id : 7] -Input [2]: [s_store_sk#34, s_state#35] -Condition : ((isnotnull(s_state#35) AND (s_state#35 = TN)) AND isnotnull(s_store_sk#34)) +Input [2]: [s_store_sk#29, s_state#30] +Condition : ((isnotnull(s_state#30) AND (s_state#30 = TN)) AND isnotnull(s_store_sk#29)) (34) Project [codegen id : 7] -Output [1]: [s_store_sk#34] -Input [2]: [s_store_sk#34, s_state#35] +Output [1]: [s_store_sk#29] +Input [2]: [s_store_sk#29, s_state#30] (35) BroadcastExchange -Input [1]: [s_store_sk#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [1]: [s_store_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [cast(s_store_sk#34 as bigint)] +Right keys [1]: [cast(s_store_sk#29 as bigint)] Join condition: None (37) Project [codegen id : 9] Output [1]: [ctr_customer_sk#13] -Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#34] +Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#29] (38) Scan parquet default.customer -Output [2]: [c_customer_sk#37, c_customer_id#38] +Output [2]: [c_customer_sk#32, c_customer_id#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 8] -Input [2]: [c_customer_sk#37, c_customer_id#38] +Input [2]: [c_customer_sk#32, c_customer_id#33] (40) Filter [codegen id : 8] -Input [2]: [c_customer_sk#37, c_customer_id#38] -Condition : isnotnull(c_customer_sk#37) +Input [2]: [c_customer_sk#32, c_customer_id#33] +Condition : isnotnull(c_customer_sk#32) (41) BroadcastExchange -Input [2]: [c_customer_sk#37, c_customer_id#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#39] +Input [2]: [c_customer_sk#32, c_customer_id#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [cast(c_customer_sk#37 as bigint)] +Right keys [1]: [cast(c_customer_sk#32 as bigint)] Join condition: None (43) Project [codegen id : 9] -Output [1]: [c_customer_id#38] -Input [3]: [ctr_customer_sk#13, c_customer_sk#37, c_customer_id#38] +Output [1]: [c_customer_id#33] +Input [3]: [ctr_customer_sk#13, c_customer_sk#32, c_customer_id#33] (44) TakeOrderedAndProject -Input [1]: [c_customer_id#38] -Arguments: 100, [c_customer_id#38 ASC NULLS FIRST], [c_customer_id#38] +Input [1]: [c_customer_id#33] +Arguments: 100, [c_customer_id#33 ASC NULLS FIRST], [c_customer_id#33] ===== Subqueries ===== @@ -264,6 +264,6 @@ ReusedExchange (45) (45) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 15 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 15 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/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt index 2dd0a13e74a74..c25973f527384 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt @@ -143,20 +143,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#17] +Output [1]: [d_date_sk#10] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#17] +Right keys [1]: [d_date_sk#10] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] -Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] +Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] (23) Exchange Input [1]: [ws_bill_customer_sk#15] -Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] +Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] @@ -168,38 +168,38 @@ Right keys [1]: [ws_bill_customer_sk#15] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] +Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] +Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#21] +Output [1]: [d_date_sk#10] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#21] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#10] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#19] -Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] +Output [1]: [cs_ship_customer_sk#18] +Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] (31) Exchange -Input [1]: [cs_ship_customer_sk#19] -Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [cs_ship_customer_sk#18] +Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#19] -Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#18] +Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#19] +Right keys [1]: [cs_ship_customer_sk#18] Join condition: None (34) Filter [codegen id : 13] @@ -211,96 +211,96 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (36) Scan parquet default.customer_address -Output [2]: [ca_address_sk#23, ca_county#24] +Output [2]: [ca_address_sk#21, ca_county#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 12] -Input [2]: [ca_address_sk#23, ca_county#24] +Input [2]: [ca_address_sk#21, ca_county#22] (38) Filter [codegen id : 12] -Input [2]: [ca_address_sk#23, ca_county#24] -Condition : (ca_county#24 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#23)) +Input [2]: [ca_address_sk#21, ca_county#22] +Condition : (ca_county#22 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#21)) (39) Project [codegen id : 12] -Output [1]: [ca_address_sk#23] -Input [2]: [ca_address_sk#23, ca_county#24] +Output [1]: [ca_address_sk#21] +Input [2]: [ca_address_sk#21, ca_county#22] (40) BroadcastExchange -Input [1]: [ca_address_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] +Input [1]: [ca_address_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (41) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#23] +Right keys [1]: [ca_address_sk#21] Join condition: None (42) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#23] +Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#21] (43) Exchange Input [1]: [c_current_cdemo_sk#4] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#26] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#24] (44) Sort [codegen id : 14] Input [1]: [c_current_cdemo_sk#4] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 (45) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (46) ColumnarToRow [codegen id : 15] -Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (47) Filter [codegen id : 15] -Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -Condition : isnotnull(cd_demo_sk#27) +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Condition : isnotnull(cd_demo_sk#25) (48) Exchange -Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -Arguments: hashpartitioning(cd_demo_sk#27, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: hashpartitioning(cd_demo_sk#25, 5), ENSURE_REQUIREMENTS, [id=#34] (49) Sort [codegen id : 16] -Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -Arguments: [cd_demo_sk#27 ASC NULLS FIRST], false, 0 +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: [cd_demo_sk#25 ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 17] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#27] +Right keys [1]: [cd_demo_sk#25] Join condition: None (51) Project [codegen id : 17] -Output [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (52) HashAggregate [codegen id : 17] -Input [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] -Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#37] -Results [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] +Aggregate Attributes [1]: [count#35] +Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] (53) Exchange -Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] -Arguments: hashpartitioning(cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] (54) HashAggregate [codegen id : 18] -Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] -Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#40] -Results [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, count(1)#40 AS cnt1#41, cd_purchase_estimate#31, count(1)#40 AS cnt2#42, cd_credit_rating#32, count(1)#40 AS cnt3#43, cd_dep_count#33, count(1)#40 AS cnt4#44, cd_dep_employed_count#34, count(1)#40 AS cnt5#45, cd_dep_college_count#35, count(1)#40 AS cnt6#46] +Aggregate Attributes [1]: [count(1)#38] +Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] (55) TakeOrderedAndProject -Input [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#41, cd_purchase_estimate#31, cnt2#42, cd_credit_rating#32, cnt3#43, cd_dep_count#33, cnt4#44, cd_dep_employed_count#34, cnt5#45, cd_dep_college_count#35, cnt6#46] -Arguments: 100, [cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_education_status#30 ASC NULLS FIRST, cd_purchase_estimate#31 ASC NULLS FIRST, cd_credit_rating#32 ASC NULLS FIRST, cd_dep_count#33 ASC NULLS FIRST, cd_dep_employed_count#34 ASC NULLS FIRST, cd_dep_college_count#35 ASC NULLS FIRST], [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#41, cd_purchase_estimate#31, cnt2#42, cd_credit_rating#32, cnt3#43, cd_dep_count#33, cnt4#44, cd_dep_employed_count#34, cnt5#45, cd_dep_college_count#35, cnt6#46] +Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] +Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] ===== Subqueries ===== @@ -313,6 +313,6 @@ Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index 370551a2de652..07bcd309e9ce6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -123,20 +123,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#9] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#9] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#16] +Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] @@ -144,34 +144,34 @@ Right keys [1]: [ws_bill_customer_sk#14] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#20] +Output [1]: [d_date_sk#9] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#20] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#9] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#18] -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#20] +Output [1]: [cs_ship_customer_sk#17] +Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [cs_ship_customer_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#17] Join condition: None (29) Filter [codegen id : 9] @@ -183,84 +183,84 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_county#23] +Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#22, ca_county#23] +Input [2]: [ca_address_sk#20, ca_county#21] (33) Filter [codegen id : 7] -Input [2]: [ca_address_sk#22, ca_county#23] -Condition : (ca_county#23 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#22)) +Input [2]: [ca_address_sk#20, ca_county#21] +Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#20)) (34) Project [codegen id : 7] -Output [1]: [ca_address_sk#22] -Input [2]: [ca_address_sk#22, ca_county#23] +Output [1]: [ca_address_sk#20] +Input [2]: [ca_address_sk#20, ca_county#21] (35) 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#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#20] Join condition: None (37) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22] +Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] (38) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] (40) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Condition : isnotnull(cd_demo_sk#25) +Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Condition : isnotnull(cd_demo_sk#23) (41) BroadcastExchange -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] (42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#23] Join condition: None (43) Project [codegen id : 9] -Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] (44) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#35] -Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Aggregate Attributes [1]: [count#33] +Results [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] (45) Exchange -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] +Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#35] (46) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] +Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] +Aggregate Attributes [1]: [count(1)#36] +Results [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#36 AS cnt1#37, cd_purchase_estimate#27, count(1)#36 AS cnt2#38, cd_credit_rating#28, count(1)#36 AS cnt3#39, cd_dep_count#29, count(1)#36 AS cnt4#40, cd_dep_employed_count#30, count(1)#36 AS cnt5#41, cd_dep_college_count#31, count(1)#36 AS cnt6#42] (47) TakeOrderedAndProject -Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] +Input [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] +Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] ===== Subqueries ===== @@ -273,6 +273,6 @@ Output [1]: [d_date_sk#9] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 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 92e8e38634ea6..da2143dc20a90 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 @@ -200,289 +200,289 @@ Input [2]: [customer_id#23, year_total#24] Arguments: [customer_id#23 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] +Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] (27) Filter [codegen id : 10] -Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] -Condition : isnotnull(ss_customer_sk#26) +Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] +Condition : isnotnull(ss_customer_sk#1) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#31, d_year#32] +Output [2]: [d_date_sk#6, d_year#7] 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 (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#31, d_year#32] +Input [2]: [d_date_sk#6, d_year#7] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#31, d_year#32] -Condition : ((isnotnull(d_year#32) AND (d_year#32 = 2002)) AND isnotnull(d_date_sk#31)) +Input [2]: [d_date_sk#6, d_year#7] +Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) (31) BroadcastExchange -Input [2]: [d_date_sk#31, d_year#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +Input [2]: [d_date_sk#6, d_year#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#29] -Right keys [1]: [d_date_sk#31] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#6] Join condition: None (33) Project [codegen id : 10] -Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Input [6]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29, d_date_sk#31, d_year#32] +Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4, d_date_sk#6, d_year#7] (34) Exchange -Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#28] (35) Sort [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (37) Sort [codegen id : 13] -Input [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] -Arguments: [c_customer_sk#35 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#35] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#10] Join condition: None (39) Project [codegen id : 14] -Output [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Input [12]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32, c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Input [12]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (40) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#43] -Results [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] +Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#29] +Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] (41) Exchange -Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] +Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, 5), ENSURE_REQUIREMENTS, [id=#31] (42) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] -Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46] -Results [3]: [c_customer_id#36 AS customer_id#47, c_preferred_cust_flag#39 AS customer_preferred_cust_flag#48, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46,18,2) AS year_total#49] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32] +Results [3]: [c_customer_id#11 AS customer_id#33, c_preferred_cust_flag#14 AS customer_preferred_cust_flag#34, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32,18,2) AS year_total#35] (43) Exchange -Input [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] -Arguments: hashpartitioning(customer_id#47, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [3]: [customer_id#33, customer_preferred_cust_flag#34, year_total#35] +Arguments: hashpartitioning(customer_id#33, 5), ENSURE_REQUIREMENTS, [id=#36] (44) Sort [codegen id : 16] -Input [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] -Arguments: [customer_id#47 ASC NULLS FIRST], false, 0 +Input [3]: [customer_id#33, customer_preferred_cust_flag#34, year_total#35] +Arguments: [customer_id#33 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#47] +Right keys [1]: [customer_id#33] Join condition: None (46) Project [codegen id : 17] -Output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49] -Input [5]: [customer_id#23, year_total#24, customer_id#47, customer_preferred_cust_flag#48, year_total#49] +Output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35] +Input [5]: [customer_id#23, year_total#24, customer_id#33, customer_preferred_cust_flag#34, year_total#35] (47) 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#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] 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#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (48) 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#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] (49) 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#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Condition : isnotnull(ws_bill_customer_sk#37) (50) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#55, d_year#56] +Output [2]: [d_date_sk#6, d_year#7] (51) 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#40] +Right keys [1]: [d_date_sk#6] Join condition: None (52) 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#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Input [6]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#6, d_year#7] (53) 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#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Arguments: hashpartitioning(ws_bill_customer_sk#37, 5), ENSURE_REQUIREMENTS, [id=#41] (54) 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#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Arguments: [ws_bill_customer_sk#37 ASC NULLS FIRST], false, 0 (55) ReusedExchange [Reuses operator id: 15] -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#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (56) 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#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 (57) 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#37] +Right keys [1]: [c_customer_sk#10] Join condition: None (58) 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#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Input [12]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (59) 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#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#42] +Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#43] (60) 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#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#43] +Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#44] (61) 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#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#43] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45] +Results [2]: [c_customer_id#11 AS customer_id#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45,18,2) AS year_total#47] (62) 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#46, year_total#47] +Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) (63) Project [codegen id : 24] -Output [2]: [customer_id#70 AS customer_id#72, year_total#71 AS year_total#73] -Input [2]: [customer_id#70, year_total#71] +Output [2]: [customer_id#46 AS customer_id#48, year_total#47 AS year_total#49] +Input [2]: [customer_id#46, year_total#47] (64) Exchange -Input [2]: [customer_id#72, year_total#73] -Arguments: hashpartitioning(customer_id#72, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [2]: [customer_id#48, year_total#49] +Arguments: hashpartitioning(customer_id#48, 5), ENSURE_REQUIREMENTS, [id=#50] (65) Sort [codegen id : 25] -Input [2]: [customer_id#72, year_total#73] -Arguments: [customer_id#72 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#48, year_total#49] +Arguments: [customer_id#48 ASC NULLS FIRST], false, 0 (66) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#72] +Right keys [1]: [customer_id#48] Join condition: None (67) Project [codegen id : 26] -Output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73] -Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, customer_id#72, year_total#73] +Output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35, year_total#49] +Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35, customer_id#48, year_total#49] (68) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] +Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#78), dynamicpruningexpression(ws_sold_date_sk#78 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] (70) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] -Condition : isnotnull(ws_bill_customer_sk#75) +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Condition : isnotnull(ws_bill_customer_sk#37) (71) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#79, d_year#80] +Output [2]: [d_date_sk#6, d_year#7] (72) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#78] -Right keys [1]: [d_date_sk#79] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#6] Join condition: None (73) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Input [6]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78, d_date_sk#79, d_year#80] +Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Input [6]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#6, d_year#7] (74) Exchange -Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Arguments: hashpartitioning(ws_bill_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Arguments: hashpartitioning(ws_bill_customer_sk#37, 5), ENSURE_REQUIREMENTS, [id=#51] (75) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Arguments: [ws_bill_customer_sk#75 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Arguments: [ws_bill_customer_sk#37 ASC NULLS FIRST], false, 0 (76) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] +Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (77) Sort [codegen id : 31] -Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] -Arguments: [c_customer_sk#82 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#75] -Right keys [1]: [c_customer_sk#82] +Left keys [1]: [ws_bill_customer_sk#37] +Right keys [1]: [c_customer_sk#10] Join condition: None (79) Project [codegen id : 32] -Output [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Input [12]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80, c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] +Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Input [12]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (80) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] -Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#90] -Results [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] +Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#52] +Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#53] (81) Exchange -Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] -Arguments: hashpartitioning(c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#53] +Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#54] (82) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] -Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93] -Results [2]: [c_customer_id#83 AS customer_id#94, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93,18,2) AS year_total#95] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#53] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55] +Results [2]: [c_customer_id#11 AS customer_id#56, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55,18,2) AS year_total#57] (83) Exchange -Input [2]: [customer_id#94, year_total#95] -Arguments: hashpartitioning(customer_id#94, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [2]: [customer_id#56, year_total#57] +Arguments: hashpartitioning(customer_id#56, 5), ENSURE_REQUIREMENTS, [id=#58] (84) Sort [codegen id : 34] -Input [2]: [customer_id#94, year_total#95] -Arguments: [customer_id#94 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#56, year_total#57] +Arguments: [customer_id#56 ASC NULLS FIRST], false, 0 (85) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#94] -Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#95) / promote_precision(year_total#73)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#49) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE null END) +Right keys [1]: [customer_id#56] +Join condition: (CASE WHEN (year_total#49 > 0.00) THEN CheckOverflow((promote_precision(year_total#57) / promote_precision(year_total#49)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#35) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE null END) (86) Project [codegen id : 35] -Output [1]: [customer_preferred_cust_flag#48] -Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73, customer_id#94, year_total#95] +Output [1]: [customer_preferred_cust_flag#34] +Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35, year_total#49, customer_id#56, year_total#57] (87) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#48] -Arguments: 100, [customer_preferred_cust_flag#48 ASC NULLS FIRST], [customer_preferred_cust_flag#48] +Input [1]: [customer_preferred_cust_flag#34] +Arguments: 100, [customer_preferred_cust_flag#34 ASC NULLS FIRST], [customer_preferred_cust_flag#34] ===== Subqueries ===== @@ -493,15 +493,15 @@ ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#6, d_year#7] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#30 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#26 ReusedExchange (89) (89) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#31, d_year#32] +Output [2]: [d_date_sk#6, d_year#7] -Subquery:3 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#78 IN dynamicpruning#30 +Subquery:4 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#26 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 4081f69732168..5a5249a3aa81b 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 @@ -172,286 +172,286 @@ Input [2]: [customer_id#22, year_total#23] Condition : (isnotnull(year_total#23) AND (year_total#23 > 0.00)) (20) Scan parquet default.customer -Output [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] -Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_customer_id#25)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (23) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] 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#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] (25) Filter [codegen id : 4] -Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] -Condition : isnotnull(ss_customer_sk#32) +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_customer_sk#9) (26) BroadcastExchange -Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#24] -Right keys [1]: [ss_customer_sk#32] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#9] Join condition: None (28) Project [codegen id : 6] -Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] -Input [12]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#38, d_year#39] +Output [2]: [d_date_sk#15, d_year#16] 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 (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#38, d_year#39] +Input [2]: [d_date_sk#15, d_year#16] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) +Input [2]: [d_date_sk#15, d_year#16] +Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2002)) AND isnotnull(d_date_sk#15)) (32) BroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] +Input [2]: [d_date_sk#15, d_year#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#35] -Right keys [1]: [d_date_sk#38] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#15] Join condition: None (34) Project [codegen id : 6] -Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] -Input [12]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#38, d_year#39] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#15, d_year#16] (35) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] -Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#41] -Results [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#27] +Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] (36) Exchange -Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] -Arguments: hashpartitioning(c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [id=#29] (37) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] -Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44] -Results [3]: [c_customer_id#25 AS customer_id#45, c_preferred_cust_flag#28 AS customer_preferred_cust_flag#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44,18,2) AS year_total#47] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30] +Results [3]: [c_customer_id#2 AS customer_id#31, c_preferred_cust_flag#5 AS customer_preferred_cust_flag#32, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30,18,2) AS year_total#33] (38) BroadcastExchange -Input [3]: [customer_id#45, customer_preferred_cust_flag#46, year_total#47] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] +Input [3]: [customer_id#31, customer_preferred_cust_flag#32, year_total#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#45] +Right keys [1]: [customer_id#31] Join condition: None (40) Project [codegen id : 16] -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#32, year_total#33] +Input [5]: [customer_id#22, year_total#23, customer_id#31, customer_preferred_cust_flag#32, year_total#33] (41) 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#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (42) 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#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (43) 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#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (44) 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#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] 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#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (45) 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#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] (46) 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#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_bill_customer_sk#35) (47) 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#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#39] (48) 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#1] +Right keys [1]: [ws_bill_customer_sk#35] Join condition: None (49) 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#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] (50) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#62, d_year#63] +Output [2]: [d_date_sk#15, d_year#16] (51) 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#38] +Right keys [1]: [d_date_sk#15] Join condition: None (52) 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#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38, d_date_sk#15, d_year#16] (53) 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#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#40] +Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#41] (54) 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#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#41] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#42] (55) 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#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#41] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#43] +Results [2]: [c_customer_id#2 AS customer_id#44, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#43,18,2) AS year_total#45] (56) 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#44, year_total#45] +Condition : (isnotnull(year_total#45) AND (year_total#45 > 0.00)) (57) Project [codegen id : 11] -Output [2]: [customer_id#68 AS customer_id#70, year_total#69 AS year_total#71] -Input [2]: [customer_id#68, year_total#69] +Output [2]: [customer_id#44 AS customer_id#46, year_total#45 AS year_total#47] +Input [2]: [customer_id#44, year_total#45] (58) BroadcastExchange -Input [2]: [customer_id#70, year_total#71] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#72] +Input [2]: [customer_id#46, year_total#47] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] (59) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#70] +Right keys [1]: [customer_id#46] Join condition: None (60) Project [codegen id : 16] -Output [5]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, year_total#71] -Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, customer_id#70, year_total#71] +Output [5]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33, year_total#47] +Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33, customer_id#46, year_total#47] (61) Scan parquet default.customer -Output [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (62) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (63) Filter [codegen id : 14] -Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] -Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_customer_id#74)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (64) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Output [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#84), dynamicpruningexpression(ws_sold_date_sk#84 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] (66) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] -Condition : isnotnull(ws_bill_customer_sk#81) +Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_bill_customer_sk#35) (67) BroadcastExchange -Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#85] +Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] (68) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#73] -Right keys [1]: [ws_bill_customer_sk#81] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#35] Join condition: None (69) Project [codegen id : 14] -Output [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] -Input [12]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] (70) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#86, d_year#87] +Output [2]: [d_date_sk#15, d_year#16] (71) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#84] -Right keys [1]: [d_date_sk#86] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#15] Join condition: None (72) Project [codegen id : 14] -Output [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, d_year#87] -Input [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84, d_date_sk#86, d_year#87] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38, d_date_sk#15, d_year#16] (73) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, d_year#87] -Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#88] -Results [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#50] +Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#51] (74) Exchange -Input [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] -Arguments: hashpartitioning(c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#51] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#52] (75) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] -Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))#91] -Results [2]: [c_customer_id#74 AS customer_id#92, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))#91,18,2) AS year_total#93] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#51] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#53] +Results [2]: [c_customer_id#2 AS customer_id#54, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#53,18,2) AS year_total#55] (76) BroadcastExchange -Input [2]: [customer_id#92, year_total#93] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#94] +Input [2]: [customer_id#54, year_total#55] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] (77) BroadcastHashJoin [codegen id : 16] 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 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#54] +Join condition: (CASE WHEN (year_total#47 > 0.00) THEN CheckOverflow((promote_precision(year_total#55) / promote_precision(year_total#47)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#33) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE null END) (78) Project [codegen id : 16] -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#71, customer_id#92, year_total#93] +Output [1]: [customer_preferred_cust_flag#32] +Input [7]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33, year_total#47, customer_id#54, year_total#55] (79) 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#32] +Arguments: 100, [customer_preferred_cust_flag#32 ASC NULLS FIRST], [customer_preferred_cust_flag#32] ===== Subqueries ===== @@ -462,15 +462,15 @@ ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#15, d_year#16] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#24 ReusedExchange (81) (81) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#38, d_year#39] +Output [2]: [d_date_sk#15, d_year#16] -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#84 IN dynamicpruning#36 +Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index adfa3e44f996d..9c56438870f64 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -203,7 +203,7 @@ Input [2]: [d_date_sk#22, d_date#23] (36) Filter [codegen id : 10] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11719)) AND (d_date#23 <= 11779)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2002-02-01)) AND (d_date#23 <= 2002-04-02)) AND isnotnull(d_date_sk#22)) (37) Project [codegen id : 10] Output [1]: [d_date_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index 5640564564396..dde6a9f564859 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -141,7 +141,7 @@ Input [2]: [d_date_sk#16, d_date#17] (22) Filter [codegen id : 8] Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 11719)) AND (d_date#17 <= 11779)) AND isnotnull(d_date_sk#16)) +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2002-02-01)) AND (d_date#17 <= 2002-04-02)) AND isnotnull(d_date_sk#16)) (23) Project [codegen id : 8] Output [1]: [d_date_sk#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index 97467fe1dc1db..8918c2a36e2ec 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -77,7 +77,7 @@ Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isno Output [2]: [d_date_sk#8, d_quarter_name#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -85,7 +85,7 @@ Input [2]: [d_date_sk#8, d_quarter_name#9] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#8, d_quarter_name#9] -Condition : ((isnotnull(d_quarter_name#9) AND (d_quarter_name#9 = 2001Q1)) AND isnotnull(d_date_sk#8)) +Condition : ((isnotnull(d_quarter_name#9) AND (rpad(d_quarter_name#9, 6, ) = 2001Q1)) AND isnotnull(d_date_sk#8)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#8] @@ -197,7 +197,7 @@ Condition : ((isnotnull(sr_customer_sk#21) AND isnotnull(sr_item_sk#20)) AND isn Output [2]: [d_date_sk#26, d_quarter_name#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 9] @@ -205,7 +205,7 @@ Input [2]: [d_date_sk#26, d_quarter_name#27] (33) Filter [codegen id : 9] Input [2]: [d_date_sk#26, d_quarter_name#27] -Condition : (d_quarter_name#27 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#26)) +Condition : (rpad(d_quarter_name#27, 6, ) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#26)) (34) Project [codegen id : 9] Output [1]: [d_date_sk#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index 32c8de9a1d16e..dafa24943be82 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -124,7 +124,7 @@ Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_i Output [2]: [d_date_sk#21, d_quarter_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 3] @@ -132,7 +132,7 @@ Input [2]: [d_date_sk#21, d_quarter_name#22] (18) Filter [codegen id : 3] Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : ((isnotnull(d_quarter_name#22) AND (d_quarter_name#22 = 2001Q1)) AND isnotnull(d_date_sk#21)) +Condition : ((isnotnull(d_quarter_name#22) AND (rpad(d_quarter_name#22, 6, ) = 2001Q1)) AND isnotnull(d_date_sk#21)) (19) Project [codegen id : 3] Output [1]: [d_date_sk#21] @@ -155,7 +155,7 @@ Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_re Output [2]: [d_date_sk#24, d_quarter_name#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] @@ -163,7 +163,7 @@ Input [2]: [d_date_sk#24, d_quarter_name#25] (25) Filter [codegen id : 4] Input [2]: [d_date_sk#24, d_quarter_name#25] -Condition : (d_quarter_name#25 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) +Condition : (rpad(d_quarter_name#25, 6, ) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) (26) Project [codegen id : 4] Output [1]: [d_date_sk#24] 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..f540e067bbc92 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))#60, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66] +Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#60,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66,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#67, d_year#68] 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#67, d_year#68] (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#67, d_year#68] +Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) (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#67] +Input [2]: [d_week_seq#67, d_year#68] (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#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] (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#67] 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#70, sun_sales#35 AS sun_sales2#71, mon_sales#36 AS mon_sales2#72, tue_sales#37 AS tue_sales2#73, wed_sales#38 AS wed_sales2#74, thu_sales#39 AS thu_sales2#75, fri_sales#40 AS fri_sales2#76, sat_sales#41 AS sat_sales2#77] +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#67] (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#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] +Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#78] (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#70 - 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#71)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#79, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#72)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#80, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#73)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#81, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#74)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#82, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#75)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#83, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#76)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#84, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#77)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#85] +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#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] (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)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] +Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#86] (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)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] 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..f540e067bbc92 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))#60, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66] +Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#60,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66,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#67, d_year#68] 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#67, d_year#68] (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#67, d_year#68] +Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) (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#67] +Input [2]: [d_week_seq#67, d_year#68] (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#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] (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#67] 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#70, sun_sales#35 AS sun_sales2#71, mon_sales#36 AS mon_sales2#72, tue_sales#37 AS tue_sales2#73, wed_sales#38 AS wed_sales2#74, thu_sales#39 AS thu_sales2#75, fri_sales#40 AS fri_sales2#76, sat_sales#41 AS sat_sales2#77] +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#67] (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#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] +Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#78] (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#70 - 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#71)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#79, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#72)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#80, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#73)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#81, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#74)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#82, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#75)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#83, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#76)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#84, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#77)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#85] +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#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] (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)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] +Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#86] (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)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] 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 f42a7615324c2..aa96edd4c7fba 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 @@ -246,91 +246,91 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (33) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] (35) Filter [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Condition : isnotnull(ss_customer_sk#26) (36) Project [codegen id : 11] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] (37) Exchange Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#30] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#29] (38) Sort [codegen id : 12] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (39) Scan parquet default.customer -Output [1]: [c_customer_sk#31] +Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 13] -Input [1]: [c_customer_sk#31] +Input [1]: [c_customer_sk#30] (41) Filter [codegen id : 13] -Input [1]: [c_customer_sk#31] -Condition : isnotnull(c_customer_sk#31) +Input [1]: [c_customer_sk#30] +Condition : isnotnull(c_customer_sk#30) (42) Exchange -Input [1]: [c_customer_sk#31] -Arguments: hashpartitioning(c_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [1]: [c_customer_sk#30] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#31] (43) Sort [codegen id : 14] -Input [1]: [c_customer_sk#31] -Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (44) SortMergeJoin [codegen id : 15] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#31] +Right keys [1]: [c_customer_sk#30] Join condition: None (45) Project [codegen id : 15] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (46) HashAggregate [codegen id : 15] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] -Keys [1]: [c_customer_sk#31] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [3]: [c_customer_sk#31, sum#35, isEmpty#36] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] (47) HashAggregate [codegen id : 15] -Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] -Keys [1]: [c_customer_sk#31] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (48) Filter [codegen id : 15] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (49) Project [codegen id : 15] -Output [1]: [c_customer_sk#31] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (50) Sort [codegen id : 15] -Input [1]: [c_customer_sk#31] -Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (51) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#31] +Right keys [1]: [c_customer_sk#30] Join condition: None (52) Project [codegen id : 17] @@ -338,209 +338,209 @@ Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (53) Scan parquet default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [3]: [d_date_sk#11, d_year#13, d_moy#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 16] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Input [3]: [d_date_sk#11, d_year#13, d_moy#40] (55) Filter [codegen id : 16] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) +Input [3]: [d_date_sk#11, d_year#13, d_moy#40] +Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#40)) AND (d_year#13 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#11)) (56) Project [codegen id : 16] -Output [1]: [d_date_sk#43] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [1]: [d_date_sk#11] +Input [3]: [d_date_sk#11, d_year#13, d_moy#40] (57) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] (58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#11] Join condition: None (59) Project [codegen id : 17] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#47] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] +Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#42] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#11] (60) Scan parquet default.web_sales -Output [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] ReadSchema: struct (61) ColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] (62) Exchange -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: hashpartitioning(ws_item_sk#48, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: hashpartitioning(ws_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#48] (63) Sort [codegen id : 19] -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: [ws_item_sk#48 ASC NULLS FIRST], false, 0 +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: [ws_item_sk#43 ASC NULLS FIRST], false, 0 (64) ReusedExchange [Reuses operator id: unknown] -Output [2]: [ss_item_sk#54, d_date#55] +Output [2]: [ss_item_sk#8, d_date#12] (65) Sort [codegen id : 22] -Input [2]: [ss_item_sk#54, d_date#55] -Arguments: [ss_item_sk#54 ASC NULLS FIRST], false, 0 +Input [2]: [ss_item_sk#8, d_date#12] +Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 (66) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#56, i_item_desc#57] +Output [2]: [i_item_sk#16, i_item_desc#17] (67) Sort [codegen id : 24] -Input [2]: [i_item_sk#56, i_item_desc#57] -Arguments: [i_item_sk#56 ASC NULLS FIRST], false, 0 +Input [2]: [i_item_sk#16, i_item_desc#17] +Arguments: [i_item_sk#16 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#56] +Left keys [1]: [ss_item_sk#8] +Right keys [1]: [i_item_sk#16] Join condition: None (69) Project [codegen id : 25] -Output [3]: [d_date#55, i_item_sk#56, i_item_desc#57] -Input [4]: [ss_item_sk#54, d_date#55, i_item_sk#56, i_item_desc#57] +Output [3]: [d_date#12, i_item_sk#16, i_item_desc#17] +Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#16, i_item_desc#17] (70) HashAggregate [codegen id : 25] -Input [3]: [d_date#55, i_item_sk#56, i_item_desc#57] -Keys [3]: [substr(i_item_desc#57, 1, 30) AS substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55] +Input [3]: [d_date#12, i_item_sk#16, i_item_desc#17] +Keys [3]: [substr(i_item_desc#17, 1, 30) AS substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#59] -Results [4]: [substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55, count#60] +Aggregate Attributes [1]: [count#50] +Results [4]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12, count#51] (71) HashAggregate [codegen id : 25] -Input [4]: [substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55, count#60] -Keys [3]: [substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55] +Input [4]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12, count#51] +Keys [3]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#61] -Results [2]: [i_item_sk#56 AS item_sk#23, count(1)#61 AS count(1)#62] +Aggregate Attributes [1]: [count(1)#52] +Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#52 AS count(1)#53] (72) Filter [codegen id : 25] -Input [2]: [item_sk#23, count(1)#62] -Condition : (count(1)#62 > 4) +Input [2]: [item_sk#23, count(1)#53] +Condition : (count(1)#53 > 4) (73) Project [codegen id : 25] Output [1]: [item_sk#23] -Input [2]: [item_sk#23, count(1)#62] +Input [2]: [item_sk#23, count(1)#53] (74) Sort [codegen id : 25] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 (75) SortMergeJoin -Left keys [1]: [ws_item_sk#48] +Left keys [1]: [ws_item_sk#43] Right keys [1]: [item_sk#23] Join condition: None (76) Project [codegen id : 26] -Output [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] (77) Exchange -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: hashpartitioning(ws_bill_customer_sk#49, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [id=#54] (78) Sort [codegen id : 27] -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: [ws_bill_customer_sk#49 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 (79) ReusedExchange [Reuses operator id: 37] -Output [3]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66] +Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] (80) Sort [codegen id : 29] -Input [3]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66] -Arguments: [ss_customer_sk#64 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (81) ReusedExchange [Reuses operator id: 42] -Output [1]: [c_customer_sk#67] +Output [1]: [c_customer_sk#30] (82) Sort [codegen id : 31] -Input [1]: [c_customer_sk#67] -Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (83) SortMergeJoin [codegen id : 32] -Left keys [1]: [ss_customer_sk#64] -Right keys [1]: [c_customer_sk#67] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#30] Join condition: None (84) Project [codegen id : 32] -Output [3]: [ss_quantity#65, ss_sales_price#66, c_customer_sk#67] -Input [4]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66, c_customer_sk#67] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (85) HashAggregate [codegen id : 32] -Input [3]: [ss_quantity#65, ss_sales_price#66, c_customer_sk#67] -Keys [1]: [c_customer_sk#67] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#68, isEmpty#69] -Results [3]: [c_customer_sk#67, sum#70, isEmpty#71] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#55, isEmpty#56] +Results [3]: [c_customer_sk#30, sum#57, isEmpty#58] (86) HashAggregate [codegen id : 32] -Input [3]: [c_customer_sk#67, sum#70, isEmpty#71] -Keys [1]: [c_customer_sk#67] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))#72] -Results [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))#72 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] +Input [3]: [c_customer_sk#30, sum#57, isEmpty#58] +Keys [1]: [c_customer_sk#30] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#59] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#59 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] (87) Filter [codegen id : 32] -Input [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (88) Project [codegen id : 32] -Output [1]: [c_customer_sk#67] -Input [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] (89) Sort [codegen id : 32] -Input [1]: [c_customer_sk#67] -Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#49] -Right keys [1]: [c_customer_sk#67] +Left keys [1]: [ws_bill_customer_sk#44] +Right keys [1]: [c_customer_sk#30] Join condition: None (91) Project [codegen id : 34] -Output [3]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [3]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] (92) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#74] +Output [1]: [d_date_sk#11] (93) BroadcastHashJoin [codegen id : 34] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#74] +Left keys [1]: [ws_sold_date_sk#47] +Right keys [1]: [d_date_sk#11] Join condition: None (94) Project [codegen id : 34] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#51 as decimal(12,2)))), DecimalType(18,2), true) AS sales#75] -Input [4]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52, d_date_sk#74] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2), true) AS sales#61] +Input [4]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#11] (95) Union (96) HashAggregate [codegen id : 35] -Input [1]: [sales#47] +Input [1]: [sales#42] Keys: [] -Functions [1]: [partial_sum(sales#47)] -Aggregate Attributes [2]: [sum#76, isEmpty#77] -Results [2]: [sum#78, isEmpty#79] +Functions [1]: [partial_sum(sales#42)] +Aggregate Attributes [2]: [sum#62, isEmpty#63] +Results [2]: [sum#64, isEmpty#65] (97) Exchange -Input [2]: [sum#78, isEmpty#79] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#80] +Input [2]: [sum#64, isEmpty#65] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] (98) HashAggregate [codegen id : 36] -Input [2]: [sum#78, isEmpty#79] +Input [2]: [sum#64, isEmpty#65] Keys: [] -Functions [1]: [sum(sales#47)] -Aggregate Attributes [1]: [sum(sales#47)#81] -Results [1]: [sum(sales#47)#81 AS sum(sales)#82] +Functions [1]: [sum(sales#42)] +Aggregate Attributes [1]: [sum(sales#42)#67] +Results [1]: [sum(sales#42)#67 AS sum(sales)#68] ===== Subqueries ===== @@ -549,7 +549,7 @@ ReusedExchange (99) (99) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#43] +Output [1]: [d_date_sk#11] Subquery:2 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 ReusedExchange (100) @@ -558,7 +558,7 @@ ReusedExchange (100) (100) ReusedExchange [Reuses operator id: 12] Output [2]: [d_date_sk#11, d_date#12] -Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#38, [id=#39] * HashAggregate (124) +- Exchange (123) +- * HashAggregate (122) @@ -586,131 +586,131 @@ Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#87)] +PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#69)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] (103) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] -Condition : isnotnull(ss_customer_sk#83) +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Condition : isnotnull(ss_customer_sk#26) (104) Scan parquet default.date_dim -Output [2]: [d_date_sk#88, d_year#89] +Output [2]: [d_date_sk#11, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (105) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#88, d_year#89] +Input [2]: [d_date_sk#11, d_year#13] (106) Filter [codegen id : 1] -Input [2]: [d_date_sk#88, d_year#89] -Condition : (d_year#89 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#88)) +Input [2]: [d_date_sk#11, d_year#13] +Condition : (d_year#13 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) (107) Project [codegen id : 1] -Output [1]: [d_date_sk#88] -Input [2]: [d_date_sk#88, d_year#89] +Output [1]: [d_date_sk#11] +Input [2]: [d_date_sk#11, d_year#13] (108) BroadcastExchange -Input [1]: [d_date_sk#88] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#90] +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#70] (109) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#86] -Right keys [1]: [d_date_sk#88] +Left keys [1]: [ss_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] Join condition: None (110) Project [codegen id : 2] -Output [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] -Input [5]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86, d_date_sk#88] +Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9, d_date_sk#11] (111) Exchange -Input [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] -Arguments: hashpartitioning(ss_customer_sk#83, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#71] (112) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] -Arguments: [ss_customer_sk#83 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (113) Scan parquet default.customer -Output [1]: [c_customer_sk#92] +Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (114) ColumnarToRow [codegen id : 4] -Input [1]: [c_customer_sk#92] +Input [1]: [c_customer_sk#30] (115) Filter [codegen id : 4] -Input [1]: [c_customer_sk#92] -Condition : isnotnull(c_customer_sk#92) +Input [1]: [c_customer_sk#30] +Condition : isnotnull(c_customer_sk#30) (116) Exchange -Input [1]: [c_customer_sk#92] -Arguments: hashpartitioning(c_customer_sk#92, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [1]: [c_customer_sk#30] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#72] (117) Sort [codegen id : 5] -Input [1]: [c_customer_sk#92] -Arguments: [c_customer_sk#92 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (118) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#83] -Right keys [1]: [c_customer_sk#92] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#30] Join condition: None (119) Project [codegen id : 6] -Output [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#92] -Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, c_customer_sk#92] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (120) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#92] -Keys [1]: [c_customer_sk#92] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#94, isEmpty#95] -Results [3]: [c_customer_sk#92, sum#96, isEmpty#97] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#73, isEmpty#74] +Results [3]: [c_customer_sk#30, sum#75, isEmpty#76] (121) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#92, sum#96, isEmpty#97] -Keys [1]: [c_customer_sk#92] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98 AS csales#99] +Input [3]: [c_customer_sk#30, sum#75, isEmpty#76] +Keys [1]: [c_customer_sk#30] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#77] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#77 AS csales#78] (122) HashAggregate [codegen id : 6] -Input [1]: [csales#99] +Input [1]: [csales#78] Keys: [] -Functions [1]: [partial_max(csales#99)] -Aggregate Attributes [1]: [max#100] -Results [1]: [max#101] +Functions [1]: [partial_max(csales#78)] +Aggregate Attributes [1]: [max#79] +Results [1]: [max#80] (123) Exchange -Input [1]: [max#101] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#102] +Input [1]: [max#80] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] (124) HashAggregate [codegen id : 7] -Input [1]: [max#101] +Input [1]: [max#80] Keys: [] -Functions [1]: [max(csales#99)] -Aggregate Attributes [1]: [max(csales#99)#103] -Results [1]: [max(csales#99)#103 AS tpcds_cmax#104] +Functions [1]: [max(csales#78)] +Aggregate Attributes [1]: [max(csales#78)#82] +Results [1]: [max(csales#78)#82 AS tpcds_cmax#83] -Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#87 +Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#69 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 108] -Output [1]: [d_date_sk#88] +Output [1]: [d_date_sk#11] -Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:6 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] 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 8c2aed03ce0cc..692671c9f287d 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 @@ -206,83 +206,83 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (29) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] (31) Filter [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Condition : isnotnull(ss_customer_sk#26) (32) Project [codegen id : 8] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] (33) Scan parquet default.customer -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 7] -Input [1]: [c_customer_sk#30] +Input [1]: [c_customer_sk#29] (35) Filter [codegen id : 7] -Input [1]: [c_customer_sk#30] -Condition : isnotnull(c_customer_sk#30) +Input [1]: [c_customer_sk#29] +Condition : isnotnull(c_customer_sk#29) (36) BroadcastExchange -Input [1]: [c_customer_sk#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [1]: [c_customer_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (37) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#29] Join condition: None (38) Project [codegen id : 8] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#29] (39) HashAggregate [codegen id : 8] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Keys [1]: [c_customer_sk#29] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Aggregate Attributes [2]: [sum#31, isEmpty#32] +Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] (40) Exchange -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#35] (41) HashAggregate [codegen id : 9] -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Keys [1]: [c_customer_sk#30] +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#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] +Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (42) Filter [codegen id : 9] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (43) Project [codegen id : 9] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Output [1]: [c_customer_sk#29] +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (44) Sort [codegen id : 9] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (45) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#29] Join condition: None (46) Project [codegen id : 11] @@ -290,128 +290,128 @@ Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (47) Scan parquet default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [3]: [d_date_sk#10, d_year#12, d_moy#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 10] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Input [3]: [d_date_sk#10, d_year#12, d_moy#40] (49) Filter [codegen id : 10] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) +Input [3]: [d_date_sk#10, d_year#12, d_moy#40] +Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#40)) AND (d_year#12 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#10)) (50) Project [codegen id : 10] -Output [1]: [d_date_sk#43] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [1]: [d_date_sk#10] +Input [3]: [d_date_sk#10, d_year#12, d_moy#40] (51) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Input [1]: [d_date_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] (52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#10] Join condition: None (53) Project [codegen id : 11] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#47] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] +Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#42] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#10] (54) Scan parquet default.web_sales -Output [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] ReadSchema: struct (55) ColumnarToRow [codegen id : 16] -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] (56) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] (57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#48] +Left keys [1]: [ws_item_sk#43] Right keys [1]: [item_sk#22] Join condition: None (58) Project [codegen id : 16] -Output [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] (59) Exchange -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: hashpartitioning(ws_bill_customer_sk#49, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [id=#48] (60) Sort [codegen id : 17] -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Arguments: [ws_bill_customer_sk#49 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 (61) ReusedExchange [Reuses operator id: 40] -Output [3]: [c_customer_sk#54, sum#55, isEmpty#56] +Output [3]: [c_customer_sk#29, sum#49, isEmpty#50] (62) HashAggregate [codegen id : 20] -Input [3]: [c_customer_sk#54, sum#55, isEmpty#56] -Keys [1]: [c_customer_sk#54] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))#59] -Results [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))#59 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] +Input [3]: [c_customer_sk#29, sum#49, isEmpty#50] +Keys [1]: [c_customer_sk#29] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#51] +Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#51 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] (63) Filter [codegen id : 20] -Input [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (64) Project [codegen id : 20] -Output [1]: [c_customer_sk#54] -Input [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] +Output [1]: [c_customer_sk#29] +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] (65) Sort [codegen id : 20] -Input [1]: [c_customer_sk#54] -Arguments: [c_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (66) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#49] -Right keys [1]: [c_customer_sk#54] +Left keys [1]: [ws_bill_customer_sk#44] +Right keys [1]: [c_customer_sk#29] Join condition: None (67) Project [codegen id : 22] -Output [3]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] -Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Output [3]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] (68) ReusedExchange [Reuses operator id: 51] -Output [1]: [d_date_sk#61] +Output [1]: [d_date_sk#10] (69) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#52] -Right keys [1]: [d_date_sk#61] +Left keys [1]: [ws_sold_date_sk#47] +Right keys [1]: [d_date_sk#10] Join condition: None (70) Project [codegen id : 22] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#51 as decimal(12,2)))), DecimalType(18,2), true) AS sales#62] -Input [4]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52, d_date_sk#61] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2), true) AS sales#53] +Input [4]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#10] (71) Union (72) HashAggregate [codegen id : 23] -Input [1]: [sales#47] +Input [1]: [sales#42] Keys: [] -Functions [1]: [partial_sum(sales#47)] -Aggregate Attributes [2]: [sum#63, isEmpty#64] -Results [2]: [sum#65, isEmpty#66] +Functions [1]: [partial_sum(sales#42)] +Aggregate Attributes [2]: [sum#54, isEmpty#55] +Results [2]: [sum#56, isEmpty#57] (73) Exchange -Input [2]: [sum#65, isEmpty#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] +Input [2]: [sum#56, isEmpty#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#58] (74) HashAggregate [codegen id : 24] -Input [2]: [sum#65, isEmpty#66] +Input [2]: [sum#56, isEmpty#57] Keys: [] -Functions [1]: [sum(sales#47)] -Aggregate Attributes [1]: [sum(sales#47)#68] -Results [1]: [sum(sales#47)#68 AS sum(sales)#69] +Functions [1]: [sum(sales#42)] +Aggregate Attributes [1]: [sum(sales#42)#59] +Results [1]: [sum(sales#42)#59 AS sum(sales)#60] ===== Subqueries ===== @@ -420,7 +420,7 @@ ReusedExchange (75) (75) ReusedExchange [Reuses operator id: 51] -Output [1]: [d_date_sk#43] +Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 ReusedExchange (76) @@ -429,7 +429,7 @@ ReusedExchange (76) (76) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#10, d_date#11] -Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#38, [id=#39] * HashAggregate (98) +- Exchange (97) +- * HashAggregate (96) @@ -455,123 +455,123 @@ Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquer (77) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#61)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (78) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] (79) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] -Condition : isnotnull(ss_customer_sk#70) +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Condition : isnotnull(ss_customer_sk#26) (80) Scan parquet default.customer -Output [1]: [c_customer_sk#75] +Output [1]: [c_customer_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (81) ColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#75] +Input [1]: [c_customer_sk#29] (82) Filter [codegen id : 1] -Input [1]: [c_customer_sk#75] -Condition : isnotnull(c_customer_sk#75) +Input [1]: [c_customer_sk#29] +Condition : isnotnull(c_customer_sk#29) (83) BroadcastExchange -Input [1]: [c_customer_sk#75] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] +Input [1]: [c_customer_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#62] (84) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#70] -Right keys [1]: [c_customer_sk#75] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#29] Join condition: None (85) Project [codegen id : 3] -Output [4]: [ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75] -Input [5]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75] +Output [4]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] +Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#77, d_year#78] +Output [2]: [d_date_sk#10, d_year#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#77, d_year#78] +Input [2]: [d_date_sk#10, d_year#12] (88) Filter [codegen id : 2] -Input [2]: [d_date_sk#77, d_year#78] -Condition : (d_year#78 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#77)) +Input [2]: [d_date_sk#10, d_year#12] +Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) (89) Project [codegen id : 2] -Output [1]: [d_date_sk#77] -Input [2]: [d_date_sk#77, d_year#78] +Output [1]: [d_date_sk#10] +Input [2]: [d_date_sk#10, d_year#12] (90) BroadcastExchange -Input [1]: [d_date_sk#77] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#79] +Input [1]: [d_date_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] (91) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#73] -Right keys [1]: [d_date_sk#77] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] Join condition: None (92) Project [codegen id : 3] -Output [3]: [ss_quantity#71, ss_sales_price#72, c_customer_sk#75] -Input [5]: [ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75, d_date_sk#77] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Input [5]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29, d_date_sk#10] (93) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#71, ss_sales_price#72, c_customer_sk#75] -Keys [1]: [c_customer_sk#75] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#80, isEmpty#81] -Results [3]: [c_customer_sk#75, sum#82, isEmpty#83] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Keys [1]: [c_customer_sk#29] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#64, isEmpty#65] +Results [3]: [c_customer_sk#29, sum#66, isEmpty#67] (94) Exchange -Input [3]: [c_customer_sk#75, sum#82, isEmpty#83] -Arguments: hashpartitioning(c_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#84] +Input [3]: [c_customer_sk#29, sum#66, isEmpty#67] +Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#68] (95) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#75, sum#82, isEmpty#83] -Keys [1]: [c_customer_sk#75] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))#85] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))#85 AS csales#86] +Input [3]: [c_customer_sk#29, sum#66, isEmpty#67] +Keys [1]: [c_customer_sk#29] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS csales#70] (96) HashAggregate [codegen id : 4] -Input [1]: [csales#86] +Input [1]: [csales#70] Keys: [] -Functions [1]: [partial_max(csales#86)] -Aggregate Attributes [1]: [max#87] -Results [1]: [max#88] +Functions [1]: [partial_max(csales#70)] +Aggregate Attributes [1]: [max#71] +Results [1]: [max#72] (97) Exchange -Input [1]: [max#88] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#89] +Input [1]: [max#72] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#73] (98) HashAggregate [codegen id : 5] -Input [1]: [max#88] +Input [1]: [max#72] Keys: [] -Functions [1]: [max(csales#86)] -Aggregate Attributes [1]: [max(csales#86)#90] -Results [1]: [max(csales#86)#90 AS tpcds_cmax#91] +Functions [1]: [max(csales#70)] +Aggregate Attributes [1]: [max(csales#70)#74] +Results [1]: [max(csales#70)#74 AS tpcds_cmax#75] -Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 +Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#61 ReusedExchange (99) (99) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#77] +Output [1]: [d_date_sk#10] -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:6 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] 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 e75b223ea599a..35a42fa256325 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 @@ -290,145 +290,145 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (34) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] (36) Filter [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Condition : isnotnull(ss_customer_sk#26) (37) Project [codegen id : 11] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] (38) Exchange Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#30] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#29] (39) Sort [codegen id : 12] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (40) Scan parquet default.customer -Output [1]: [c_customer_sk#31] +Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 13] -Input [1]: [c_customer_sk#31] +Input [1]: [c_customer_sk#30] (42) Filter [codegen id : 13] -Input [1]: [c_customer_sk#31] -Condition : isnotnull(c_customer_sk#31) +Input [1]: [c_customer_sk#30] +Condition : isnotnull(c_customer_sk#30) (43) Exchange -Input [1]: [c_customer_sk#31] -Arguments: hashpartitioning(c_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [1]: [c_customer_sk#30] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#31] (44) Sort [codegen id : 14] -Input [1]: [c_customer_sk#31] -Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 15] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#31] +Right keys [1]: [c_customer_sk#30] Join condition: None (46) Project [codegen id : 15] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (47) HashAggregate [codegen id : 15] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] -Keys [1]: [c_customer_sk#31] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [3]: [c_customer_sk#31, sum#35, isEmpty#36] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] (48) HashAggregate [codegen id : 15] -Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] -Keys [1]: [c_customer_sk#31] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (49) Filter [codegen id : 15] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (50) Project [codegen id : 15] -Output [1]: [c_customer_sk#31] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (51) Sort [codegen id : 15] -Input [1]: [c_customer_sk#31] -Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (52) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#31] +Right keys [1]: [c_customer_sk#30] Join condition: None (53) Scan parquet default.date_dim -Output [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [3]: [d_date_sk#11, d_year#13, d_moy#40] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 16] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Input [3]: [d_date_sk#11, d_year#13, d_moy#40] (55) Filter [codegen id : 16] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] -Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) +Input [3]: [d_date_sk#11, d_year#13, d_moy#40] +Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#40)) AND (d_year#13 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#11)) (56) Project [codegen id : 16] -Output [1]: [d_date_sk#43] -Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Output [1]: [d_date_sk#11] +Input [3]: [d_date_sk#11, d_year#13, d_moy#40] (57) BroadcastExchange -Input [1]: [d_date_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] (58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#11] Join condition: None (59) Project [codegen id : 17] Output [3]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4] -Input [5]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] +Input [5]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#11] (60) Scan parquet default.customer -Output [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] +Output [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 18] -Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] +Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] (62) Filter [codegen id : 18] -Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] -Condition : isnotnull(c_customer_sk#47) +Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Condition : isnotnull(c_customer_sk#30) (63) Exchange -Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] -Arguments: hashpartitioning(c_customer_sk#47, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#44] (64) Sort [codegen id : 19] -Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] -Arguments: [c_customer_sk#47 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (65) ReusedExchange [Reuses operator id: 38] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] @@ -438,327 +438,327 @@ Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (67) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#31] +Output [1]: [c_customer_sk#30] (68) Sort [codegen id : 23] -Input [1]: [c_customer_sk#31] -Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (69) SortMergeJoin [codegen id : 24] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#31] +Right keys [1]: [c_customer_sk#30] Join condition: None (70) Project [codegen id : 24] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (71) HashAggregate [codegen id : 24] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] -Keys [1]: [c_customer_sk#31] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [3]: [c_customer_sk#31, sum#35, isEmpty#36] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] (72) HashAggregate [codegen id : 24] -Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] -Keys [1]: [c_customer_sk#31] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (73) Filter [codegen id : 24] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (74) Project [codegen id : 24] -Output [1]: [c_customer_sk#31] -Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Output [1]: [c_customer_sk#30 AS c_customer_sk#30#45] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (75) Sort [codegen id : 24] -Input [1]: [c_customer_sk#31] -Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30#45] +Arguments: [c_customer_sk#30#45 ASC NULLS FIRST], false, 0 (76) SortMergeJoin -Left keys [1]: [c_customer_sk#47] -Right keys [1]: [c_customer_sk#31] +Left keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#30#45] Join condition: None (77) SortMergeJoin [codegen id : 25] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#47] +Right keys [1]: [c_customer_sk#30] Join condition: None (78) Project [codegen id : 25] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#48, c_last_name#49] -Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#47, c_first_name#48, c_last_name#49] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#42, c_last_name#43] +Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#30, c_first_name#42, c_last_name#43] (79) HashAggregate [codegen id : 25] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#48, c_last_name#49] -Keys [2]: [c_last_name#49, c_first_name#48] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#42, c_last_name#43] +Keys [2]: [c_last_name#43, c_first_name#42] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#51, isEmpty#52] -Results [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] +Aggregate Attributes [2]: [sum#46, isEmpty#47] +Results [4]: [c_last_name#43, c_first_name#42, sum#48, isEmpty#49] (80) Exchange -Input [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] -Arguments: hashpartitioning(c_last_name#49, c_first_name#48, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [c_last_name#43, c_first_name#42, sum#48, isEmpty#49] +Arguments: hashpartitioning(c_last_name#43, c_first_name#42, 5), ENSURE_REQUIREMENTS, [id=#50] (81) HashAggregate [codegen id : 26] -Input [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] -Keys [2]: [c_last_name#49, c_first_name#48] +Input [4]: [c_last_name#43, c_first_name#42, sum#48, isEmpty#49] +Keys [2]: [c_last_name#43, c_first_name#42] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#56] -Results [3]: [c_last_name#49, c_first_name#48, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sales#57] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#51] +Results [3]: [c_last_name#43, c_first_name#42, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#51 AS sales#52] (82) Scan parquet default.web_sales -Output [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Output [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#57), dynamicpruningexpression(ws_sold_date_sk#57 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (83) ColumnarToRow [codegen id : 27] -Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] (84) Filter [codegen id : 27] -Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Condition : isnotnull(ws_bill_customer_sk#59) +Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Condition : isnotnull(ws_bill_customer_sk#54) (85) Exchange -Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Arguments: hashpartitioning(ws_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Arguments: hashpartitioning(ws_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#58] (86) Sort [codegen id : 28] -Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Arguments: [ws_item_sk#58 ASC NULLS FIRST], false, 0 +Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Arguments: [ws_item_sk#53 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: unknown] -Output [2]: [ss_item_sk#64, d_date#65] +Output [2]: [ss_item_sk#8, d_date#12] (88) Sort [codegen id : 31] -Input [2]: [ss_item_sk#64, d_date#65] -Arguments: [ss_item_sk#64 ASC NULLS FIRST], false, 0 +Input [2]: [ss_item_sk#8, d_date#12] +Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 (89) ReusedExchange [Reuses operator id: 21] -Output [2]: [i_item_sk#66, i_item_desc#67] +Output [2]: [i_item_sk#16, i_item_desc#17] (90) Sort [codegen id : 33] -Input [2]: [i_item_sk#66, i_item_desc#67] -Arguments: [i_item_sk#66 ASC NULLS FIRST], false, 0 +Input [2]: [i_item_sk#16, i_item_desc#17] +Arguments: [i_item_sk#16 ASC NULLS FIRST], false, 0 (91) SortMergeJoin [codegen id : 34] -Left keys [1]: [ss_item_sk#64] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#8] +Right keys [1]: [i_item_sk#16] Join condition: None (92) Project [codegen id : 34] -Output [3]: [d_date#65, i_item_sk#66, i_item_desc#67] -Input [4]: [ss_item_sk#64, d_date#65, i_item_sk#66, i_item_desc#67] +Output [3]: [d_date#12, i_item_sk#16, i_item_desc#17] +Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#16, i_item_desc#17] (93) HashAggregate [codegen id : 34] -Input [3]: [d_date#65, i_item_sk#66, i_item_desc#67] -Keys [3]: [substr(i_item_desc#67, 1, 30) AS substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65] +Input [3]: [d_date#12, i_item_sk#16, i_item_desc#17] +Keys [3]: [substr(i_item_desc#17, 1, 30) AS substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#69] -Results [4]: [substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65, count#70] +Aggregate Attributes [1]: [count#60] +Results [4]: [substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12, count#61] (94) HashAggregate [codegen id : 34] -Input [4]: [substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65, count#70] -Keys [3]: [substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65] +Input [4]: [substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12, count#61] +Keys [3]: [substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#71] -Results [2]: [i_item_sk#66 AS item_sk#23, count(1)#71 AS count(1)#72] +Aggregate Attributes [1]: [count(1)#62] +Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#62 AS count(1)#63] (95) Filter [codegen id : 34] -Input [2]: [item_sk#23, count(1)#72] -Condition : (count(1)#72 > 4) +Input [2]: [item_sk#23, count(1)#63] +Condition : (count(1)#63 > 4) (96) Project [codegen id : 34] Output [1]: [item_sk#23] -Input [2]: [item_sk#23, count(1)#72] +Input [2]: [item_sk#23, count(1)#63] (97) Sort [codegen id : 34] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ws_item_sk#58] +Left keys [1]: [ws_item_sk#53] Right keys [1]: [item_sk#23] Join condition: None (99) Project [codegen id : 35] -Output [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Output [4]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] (100) Exchange -Input [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Arguments: hashpartitioning(ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [4]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Arguments: hashpartitioning(ws_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [id=#64] (101) Sort [codegen id : 36] -Input [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] -Arguments: [ws_bill_customer_sk#59 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Arguments: [ws_bill_customer_sk#54 ASC NULLS FIRST], false, 0 (102) ReusedExchange [Reuses operator id: 38] -Output [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] +Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] (103) Sort [codegen id : 38] -Input [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] -Arguments: [ss_customer_sk#74 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (104) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#77] +Output [1]: [c_customer_sk#30] (105) Sort [codegen id : 40] -Input [1]: [c_customer_sk#77] -Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (106) SortMergeJoin [codegen id : 41] -Left keys [1]: [ss_customer_sk#74] -Right keys [1]: [c_customer_sk#77] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#30] Join condition: None (107) Project [codegen id : 41] -Output [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] -Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (108) HashAggregate [codegen id : 41] -Input [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] -Keys [1]: [c_customer_sk#77] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#78, isEmpty#79] -Results [3]: [c_customer_sk#77, sum#80, isEmpty#81] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [3]: [c_customer_sk#30, sum#67, isEmpty#68] (109) HashAggregate [codegen id : 41] -Input [3]: [c_customer_sk#77, sum#80, isEmpty#81] -Keys [1]: [c_customer_sk#77] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82] -Results [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Input [3]: [c_customer_sk#30, sum#67, isEmpty#68] +Keys [1]: [c_customer_sk#30] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] (110) Filter [codegen id : 41] -Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (111) Project [codegen id : 41] -Output [1]: [c_customer_sk#77] -Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] (112) Sort [codegen id : 41] -Input [1]: [c_customer_sk#77] -Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (113) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#59] -Right keys [1]: [c_customer_sk#77] +Left keys [1]: [ws_bill_customer_sk#54] +Right keys [1]: [c_customer_sk#30] Join condition: None (114) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#84] +Output [1]: [d_date_sk#11] (115) BroadcastHashJoin [codegen id : 43] -Left keys [1]: [ws_sold_date_sk#62] -Right keys [1]: [d_date_sk#84] +Left keys [1]: [ws_sold_date_sk#57] +Right keys [1]: [d_date_sk#11] Join condition: None (116) Project [codegen id : 43] -Output [3]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61] -Input [5]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62, d_date_sk#84] +Output [3]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56] +Input [5]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57, d_date_sk#11] (117) ReusedExchange [Reuses operator id: 63] -Output [3]: [c_customer_sk#85, c_first_name#86, c_last_name#87] +Output [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] (118) Sort [codegen id : 45] -Input [3]: [c_customer_sk#85, c_first_name#86, c_last_name#87] -Arguments: [c_customer_sk#85 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (119) ReusedExchange [Reuses operator id: 38] -Output [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] +Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] (120) Sort [codegen id : 47] -Input [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] -Arguments: [ss_customer_sk#74 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (121) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#77] +Output [1]: [c_customer_sk#30] (122) Sort [codegen id : 49] -Input [1]: [c_customer_sk#77] -Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (123) SortMergeJoin [codegen id : 50] -Left keys [1]: [ss_customer_sk#74] -Right keys [1]: [c_customer_sk#77] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#30] Join condition: None (124) Project [codegen id : 50] -Output [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] -Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (125) HashAggregate [codegen id : 50] -Input [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] -Keys [1]: [c_customer_sk#77] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#78, isEmpty#79] -Results [3]: [c_customer_sk#77, sum#80, isEmpty#81] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [3]: [c_customer_sk#30, sum#67, isEmpty#68] (126) HashAggregate [codegen id : 50] -Input [3]: [c_customer_sk#77, sum#80, isEmpty#81] -Keys [1]: [c_customer_sk#77] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82] -Results [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Input [3]: [c_customer_sk#30, sum#67, isEmpty#68] +Keys [1]: [c_customer_sk#30] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] (127) Filter [codegen id : 50] -Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (128) Project [codegen id : 50] -Output [1]: [c_customer_sk#77] -Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Output [1]: [c_customer_sk#30 AS c_customer_sk#30#71] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] (129) Sort [codegen id : 50] -Input [1]: [c_customer_sk#77] -Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30#71] +Arguments: [c_customer_sk#30#71 ASC NULLS FIRST], false, 0 (130) SortMergeJoin -Left keys [1]: [c_customer_sk#85] -Right keys [1]: [c_customer_sk#77] +Left keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#30#71] Join condition: None (131) SortMergeJoin [codegen id : 51] -Left keys [1]: [ws_bill_customer_sk#59] -Right keys [1]: [c_customer_sk#85] +Left keys [1]: [ws_bill_customer_sk#54] +Right keys [1]: [c_customer_sk#30] Join condition: None (132) Project [codegen id : 51] -Output [4]: [ws_quantity#60, ws_list_price#61, c_first_name#86, c_last_name#87] -Input [6]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, c_customer_sk#85, c_first_name#86, c_last_name#87] +Output [4]: [ws_quantity#55, ws_list_price#56, c_first_name#42, c_last_name#43] +Input [6]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, c_customer_sk#30, c_first_name#42, c_last_name#43] (133) HashAggregate [codegen id : 51] -Input [4]: [ws_quantity#60, ws_list_price#61, c_first_name#86, c_last_name#87] -Keys [2]: [c_last_name#87, c_first_name#86] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#88, isEmpty#89] -Results [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] +Input [4]: [ws_quantity#55, ws_list_price#56, c_first_name#42, c_last_name#43] +Keys [2]: [c_last_name#43, c_first_name#42] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#72, isEmpty#73] +Results [4]: [c_last_name#43, c_first_name#42, sum#74, isEmpty#75] (134) Exchange -Input [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] -Arguments: hashpartitioning(c_last_name#87, c_first_name#86, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [4]: [c_last_name#43, c_first_name#42, sum#74, isEmpty#75] +Arguments: hashpartitioning(c_last_name#43, c_first_name#42, 5), ENSURE_REQUIREMENTS, [id=#76] (135) HashAggregate [codegen id : 52] -Input [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] -Keys [2]: [c_last_name#87, c_first_name#86] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#93] -Results [3]: [c_last_name#87, c_first_name#86, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#93 AS sales#94] +Input [4]: [c_last_name#43, c_first_name#42, sum#74, isEmpty#75] +Keys [2]: [c_last_name#43, c_first_name#42] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))#77] +Results [3]: [c_last_name#43, c_first_name#42, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))#77 AS sales#78] (136) Union (137) TakeOrderedAndProject -Input [3]: [c_last_name#49, c_first_name#48, sales#57] -Arguments: 100, [c_last_name#49 ASC NULLS FIRST, c_first_name#48 ASC NULLS FIRST, sales#57 ASC NULLS FIRST], [c_last_name#49, c_first_name#48, sales#57] +Input [3]: [c_last_name#43, c_first_name#42, sales#52] +Arguments: 100, [c_last_name#43 ASC NULLS FIRST, c_first_name#42 ASC NULLS FIRST, sales#52 ASC NULLS FIRST], [c_last_name#43, c_first_name#42, sales#52] ===== Subqueries ===== @@ -767,7 +767,7 @@ ReusedExchange (138) (138) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#43] +Output [1]: [d_date_sk#11] Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 ReusedExchange (139) @@ -776,7 +776,7 @@ ReusedExchange (139) (139) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#11, d_date#12] -Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#38, [id=#39] * HashAggregate (163) +- Exchange (162) +- * HashAggregate (161) @@ -804,135 +804,135 @@ Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (140) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#98), dynamicpruningexpression(ss_sold_date_sk#98 IN dynamicpruning#99)] +PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#79)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (141) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] (142) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] -Condition : isnotnull(ss_customer_sk#95) +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Condition : isnotnull(ss_customer_sk#26) (143) Scan parquet default.date_dim -Output [2]: [d_date_sk#100, d_year#101] +Output [2]: [d_date_sk#11, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#100, d_year#101] +Input [2]: [d_date_sk#11, d_year#13] (145) Filter [codegen id : 1] -Input [2]: [d_date_sk#100, d_year#101] -Condition : (d_year#101 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#100)) +Input [2]: [d_date_sk#11, d_year#13] +Condition : (d_year#13 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) (146) Project [codegen id : 1] -Output [1]: [d_date_sk#100] -Input [2]: [d_date_sk#100, d_year#101] +Output [1]: [d_date_sk#11] +Input [2]: [d_date_sk#11, d_year#13] (147) BroadcastExchange -Input [1]: [d_date_sk#100] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#102] +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#80] (148) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#98] -Right keys [1]: [d_date_sk#100] +Left keys [1]: [ss_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] Join condition: None (149) Project [codegen id : 2] -Output [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] -Input [5]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98, d_date_sk#100] +Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9, d_date_sk#11] (150) Exchange -Input [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] -Arguments: hashpartitioning(ss_customer_sk#95, 5), ENSURE_REQUIREMENTS, [id=#103] +Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#81] (151) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] -Arguments: [ss_customer_sk#95 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (152) Scan parquet default.customer -Output [1]: [c_customer_sk#104] +Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (153) ColumnarToRow [codegen id : 4] -Input [1]: [c_customer_sk#104] +Input [1]: [c_customer_sk#30] (154) Filter [codegen id : 4] -Input [1]: [c_customer_sk#104] -Condition : isnotnull(c_customer_sk#104) +Input [1]: [c_customer_sk#30] +Condition : isnotnull(c_customer_sk#30) (155) Exchange -Input [1]: [c_customer_sk#104] -Arguments: hashpartitioning(c_customer_sk#104, 5), ENSURE_REQUIREMENTS, [id=#105] +Input [1]: [c_customer_sk#30] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#82] (156) Sort [codegen id : 5] -Input [1]: [c_customer_sk#104] -Arguments: [c_customer_sk#104 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (157) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#95] -Right keys [1]: [c_customer_sk#104] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#30] Join condition: None (158) Project [codegen id : 6] -Output [3]: [ss_quantity#96, ss_sales_price#97, c_customer_sk#104] -Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, c_customer_sk#104] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (159) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#96, ss_sales_price#97, c_customer_sk#104] -Keys [1]: [c_customer_sk#104] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#106, isEmpty#107] -Results [3]: [c_customer_sk#104, sum#108, isEmpty#109] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#83, isEmpty#84] +Results [3]: [c_customer_sk#30, sum#85, isEmpty#86] (160) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#104, sum#108, isEmpty#109] -Keys [1]: [c_customer_sk#104] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))#110] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))#110 AS csales#111] +Input [3]: [c_customer_sk#30, sum#85, isEmpty#86] +Keys [1]: [c_customer_sk#30] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#87] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#87 AS csales#88] (161) HashAggregate [codegen id : 6] -Input [1]: [csales#111] +Input [1]: [csales#88] Keys: [] -Functions [1]: [partial_max(csales#111)] -Aggregate Attributes [1]: [max#112] -Results [1]: [max#113] +Functions [1]: [partial_max(csales#88)] +Aggregate Attributes [1]: [max#89] +Results [1]: [max#90] (162) Exchange -Input [1]: [max#113] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] +Input [1]: [max#90] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#91] (163) HashAggregate [codegen id : 7] -Input [1]: [max#113] +Input [1]: [max#90] Keys: [] -Functions [1]: [max(csales#111)] -Aggregate Attributes [1]: [max(csales#111)#115] -Results [1]: [max(csales#111)#115 AS tpcds_cmax#116] +Functions [1]: [max(csales#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#98 IN dynamicpruning#99 +Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#79 ReusedExchange (164) (164) ReusedExchange [Reuses operator id: 147] -Output [1]: [d_date_sk#100] +Output [1]: [d_date_sk#11] -Subquery:5 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:5 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] -Subquery:6 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#57 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 110 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:7 Hosting operator id = 110 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] -Subquery:8 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:8 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] 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 f5c64908c8fb9..385cf017e58c3 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 @@ -240,340 +240,340 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (30) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] (32) Filter [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Condition : isnotnull(ss_customer_sk#26) (33) Project [codegen id : 8] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] (34) Scan parquet default.customer -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 7] -Input [1]: [c_customer_sk#30] +Input [1]: [c_customer_sk#29] (36) Filter [codegen id : 7] -Input [1]: [c_customer_sk#30] -Condition : isnotnull(c_customer_sk#30) +Input [1]: [c_customer_sk#29] +Condition : isnotnull(c_customer_sk#29) (37) BroadcastExchange -Input [1]: [c_customer_sk#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [1]: [c_customer_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (38) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#29] Join condition: None (39) Project [codegen id : 8] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#29] (40) HashAggregate [codegen id : 8] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Keys [1]: [c_customer_sk#29] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Aggregate Attributes [2]: [sum#31, isEmpty#32] +Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] (41) Exchange -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#35] (42) HashAggregate [codegen id : 9] -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Keys [1]: [c_customer_sk#30] +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#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] +Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (43) Filter [codegen id : 9] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (44) Project [codegen id : 9] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Output [1]: [c_customer_sk#29] +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (45) Sort [codegen id : 9] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (46) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#29] Join condition: None (47) Scan parquet default.customer -Output [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Output [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 10] -Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] (49) Filter [codegen id : 10] -Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] -Condition : isnotnull(c_customer_sk#43) +Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Condition : isnotnull(c_customer_sk#29) (50) Exchange -Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] -Arguments: hashpartitioning(c_customer_sk#43, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#42] (51) Sort [codegen id : 11] -Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] -Arguments: [c_customer_sk#43 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (52) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Output [3]: [c_customer_sk#29, sum#33, isEmpty#34] (53) HashAggregate [codegen id : 14] -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Keys [1]: [c_customer_sk#30] +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#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] +Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (54) Filter [codegen id : 14] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (55) Project [codegen id : 14] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Output [1]: [c_customer_sk#29 AS c_customer_sk#29#43] +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] (56) Sort [codegen id : 14] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29#43] +Arguments: [c_customer_sk#29#43 ASC NULLS FIRST], false, 0 (57) SortMergeJoin -Left keys [1]: [c_customer_sk#43] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#29#43] Join condition: None (58) BroadcastExchange -Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] +Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] (59) BroadcastHashJoin [codegen id : 16] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#43] +Right keys [1]: [c_customer_sk#29] Join condition: None (60) Project [codegen id : 16] -Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#44, c_last_name#45] -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#43, c_first_name#44, c_last_name#45] +Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#40, c_last_name#41] +Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#29, c_first_name#40, c_last_name#41] (61) Scan parquet default.date_dim -Output [3]: [d_date_sk#48, d_year#49, d_moy#50] +Output [3]: [d_date_sk#10, d_year#12, d_moy#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 15] -Input [3]: [d_date_sk#48, d_year#49, d_moy#50] +Input [3]: [d_date_sk#10, d_year#12, d_moy#45] (63) Filter [codegen id : 15] -Input [3]: [d_date_sk#48, d_year#49, d_moy#50] -Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2000)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#48)) +Input [3]: [d_date_sk#10, d_year#12, d_moy#45] +Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#45)) AND (d_year#12 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#10)) (64) Project [codegen id : 15] -Output [1]: [d_date_sk#48] -Input [3]: [d_date_sk#48, d_year#49, d_moy#50] +Output [1]: [d_date_sk#10] +Input [3]: [d_date_sk#10, d_year#12, d_moy#45] (65) BroadcastExchange -Input [1]: [d_date_sk#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] +Input [1]: [d_date_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] (66) BroadcastHashJoin [codegen id : 16] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#48] +Right keys [1]: [d_date_sk#10] Join condition: None (67) Project [codegen id : 16] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#44, c_last_name#45] -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#44, c_last_name#45, d_date_sk#48] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#40, c_last_name#41] +Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#40, c_last_name#41, d_date_sk#10] (68) HashAggregate [codegen id : 16] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#44, c_last_name#45] -Keys [2]: [c_last_name#45, c_first_name#44] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#40, c_last_name#41] +Keys [2]: [c_last_name#41, c_first_name#40] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] +Aggregate Attributes [2]: [sum#47, isEmpty#48] +Results [4]: [c_last_name#41, c_first_name#40, sum#49, isEmpty#50] (69) Exchange -Input [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] -Arguments: hashpartitioning(c_last_name#45, c_first_name#44, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [4]: [c_last_name#41, c_first_name#40, sum#49, isEmpty#50] +Arguments: hashpartitioning(c_last_name#41, c_first_name#40, 5), ENSURE_REQUIREMENTS, [id=#51] (70) HashAggregate [codegen id : 17] -Input [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] -Keys [2]: [c_last_name#45, c_first_name#44] +Input [4]: [c_last_name#41, c_first_name#40, sum#49, isEmpty#50] +Keys [2]: [c_last_name#41, c_first_name#40] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#57] -Results [3]: [c_last_name#45, c_first_name#44, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#57 AS sales#58] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#52] +Results [3]: [c_last_name#41, c_first_name#40, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#52 AS sales#53] (71) Scan parquet default.web_sales -Output [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Output [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 22] -Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] (73) Filter [codegen id : 22] -Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_customer_sk#60) +Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Condition : isnotnull(ws_bill_customer_sk#55) (74) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] (75) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_item_sk#59] +Left keys [1]: [ws_item_sk#54] Right keys [1]: [item_sk#22] Join condition: None (76) Project [codegen id : 22] -Output [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] -Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Output [4]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] (77) Exchange -Input [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] -Arguments: hashpartitioning(ws_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [4]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Arguments: hashpartitioning(ws_bill_customer_sk#55, 5), ENSURE_REQUIREMENTS, [id=#59] (78) Sort [codegen id : 23] -Input [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] -Arguments: [ws_bill_customer_sk#60 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Arguments: [ws_bill_customer_sk#55 ASC NULLS FIRST], false, 0 (79) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#65, sum#66, isEmpty#67] +Output [3]: [c_customer_sk#29, sum#60, isEmpty#61] (80) HashAggregate [codegen id : 26] -Input [3]: [c_customer_sk#65, sum#66, isEmpty#67] -Keys [1]: [c_customer_sk#65] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70] -Results [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] +Input [3]: [c_customer_sk#29, sum#60, isEmpty#61] +Keys [1]: [c_customer_sk#29] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62] +Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] (81) Filter [codegen id : 26] -Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (82) Project [codegen id : 26] -Output [1]: [c_customer_sk#65] -Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] +Output [1]: [c_customer_sk#29] +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] (83) Sort [codegen id : 26] -Input [1]: [c_customer_sk#65] -Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (84) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#60] -Right keys [1]: [c_customer_sk#65] +Left keys [1]: [ws_bill_customer_sk#55] +Right keys [1]: [c_customer_sk#29] Join condition: None (85) ReusedExchange [Reuses operator id: 50] -Output [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] +Output [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] (86) Sort [codegen id : 28] -Input [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] -Arguments: [c_customer_sk#72 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#65, sum#66, isEmpty#67] +Output [3]: [c_customer_sk#29, sum#60, isEmpty#61] (88) HashAggregate [codegen id : 31] -Input [3]: [c_customer_sk#65, sum#66, isEmpty#67] -Keys [1]: [c_customer_sk#65] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70] -Results [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] +Input [3]: [c_customer_sk#29, sum#60, isEmpty#61] +Keys [1]: [c_customer_sk#29] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62] +Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] (89) Filter [codegen id : 31] -Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) (90) Project [codegen id : 31] -Output [1]: [c_customer_sk#65] -Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] +Output [1]: [c_customer_sk#29 AS c_customer_sk#29#64] +Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] (91) Sort [codegen id : 31] -Input [1]: [c_customer_sk#65] -Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#29#64] +Arguments: [c_customer_sk#29#64 ASC NULLS FIRST], false, 0 (92) SortMergeJoin -Left keys [1]: [c_customer_sk#72] -Right keys [1]: [c_customer_sk#65] +Left keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#29#64] Join condition: None (93) BroadcastExchange -Input [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] +Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] (94) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ws_bill_customer_sk#60] -Right keys [1]: [c_customer_sk#72] +Left keys [1]: [ws_bill_customer_sk#55] +Right keys [1]: [c_customer_sk#29] Join condition: None (95) Project [codegen id : 33] -Output [5]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74] -Input [7]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_customer_sk#72, c_first_name#73, c_last_name#74] +Output [5]: [ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58, c_first_name#40, c_last_name#41] +Input [7]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58, c_customer_sk#29, c_first_name#40, c_last_name#41] (96) ReusedExchange [Reuses operator id: 65] -Output [1]: [d_date_sk#76] +Output [1]: [d_date_sk#10] (97) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ws_sold_date_sk#63] -Right keys [1]: [d_date_sk#76] +Left keys [1]: [ws_sold_date_sk#58] +Right keys [1]: [d_date_sk#10] Join condition: None (98) Project [codegen id : 33] -Output [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74] -Input [6]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74, d_date_sk#76] +Output [4]: [ws_quantity#56, ws_list_price#57, c_first_name#40, c_last_name#41] +Input [6]: [ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58, c_first_name#40, c_last_name#41, d_date_sk#10] (99) HashAggregate [codegen id : 33] -Input [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74] -Keys [2]: [c_last_name#74, c_first_name#73] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#77, isEmpty#78] -Results [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] +Input [4]: [ws_quantity#56, ws_list_price#57, c_first_name#40, c_last_name#41] +Keys [2]: [c_last_name#41, c_first_name#40] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#66, isEmpty#67] +Results [4]: [c_last_name#41, c_first_name#40, sum#68, isEmpty#69] (100) Exchange -Input [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] -Arguments: hashpartitioning(c_last_name#74, c_first_name#73, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [4]: [c_last_name#41, c_first_name#40, sum#68, isEmpty#69] +Arguments: hashpartitioning(c_last_name#41, c_first_name#40, 5), ENSURE_REQUIREMENTS, [id=#70] (101) HashAggregate [codegen id : 34] -Input [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] -Keys [2]: [c_last_name#74, c_first_name#73] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#82] -Results [3]: [c_last_name#74, c_first_name#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sales#83] +Input [4]: [c_last_name#41, c_first_name#40, sum#68, isEmpty#69] +Keys [2]: [c_last_name#41, c_first_name#40] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))#71] +Results [3]: [c_last_name#41, c_first_name#40, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sales#72] (102) Union (103) TakeOrderedAndProject -Input [3]: [c_last_name#45, c_first_name#44, sales#58] -Arguments: 100, [c_last_name#45 ASC NULLS FIRST, c_first_name#44 ASC NULLS FIRST, sales#58 ASC NULLS FIRST], [c_last_name#45, c_first_name#44, sales#58] +Input [3]: [c_last_name#41, c_first_name#40, sales#53] +Arguments: 100, [c_last_name#41 ASC NULLS FIRST, c_first_name#40 ASC NULLS FIRST, sales#53 ASC NULLS FIRST], [c_last_name#41, c_first_name#40, sales#53] ===== Subqueries ===== @@ -582,7 +582,7 @@ ReusedExchange (104) (104) ReusedExchange [Reuses operator id: 65] -Output [1]: [d_date_sk#48] +Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 ReusedExchange (105) @@ -591,7 +591,7 @@ ReusedExchange (105) (105) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#10, d_date#11] -Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#38, [id=#39] * HashAggregate (127) +- Exchange (126) +- * HashAggregate (125) @@ -617,127 +617,127 @@ Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquer (106) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#73)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (107) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] (108) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] -Condition : isnotnull(ss_customer_sk#84) +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Condition : isnotnull(ss_customer_sk#26) (109) Scan parquet default.customer -Output [1]: [c_customer_sk#89] +Output [1]: [c_customer_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (110) ColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#89] +Input [1]: [c_customer_sk#29] (111) Filter [codegen id : 1] -Input [1]: [c_customer_sk#89] -Condition : isnotnull(c_customer_sk#89) +Input [1]: [c_customer_sk#29] +Condition : isnotnull(c_customer_sk#29) (112) BroadcastExchange -Input [1]: [c_customer_sk#89] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#90] +Input [1]: [c_customer_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] (113) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#84] -Right keys [1]: [c_customer_sk#89] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#29] Join condition: None (114) Project [codegen id : 3] -Output [4]: [ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89] -Input [5]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89] +Output [4]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] +Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] (115) Scan parquet default.date_dim -Output [2]: [d_date_sk#91, d_year#92] +Output [2]: [d_date_sk#10, d_year#12] 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 (116) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#91, d_year#92] +Input [2]: [d_date_sk#10, d_year#12] (117) Filter [codegen id : 2] -Input [2]: [d_date_sk#91, d_year#92] -Condition : (d_year#92 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#91)) +Input [2]: [d_date_sk#10, d_year#12] +Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) (118) Project [codegen id : 2] -Output [1]: [d_date_sk#91] -Input [2]: [d_date_sk#91, d_year#92] +Output [1]: [d_date_sk#10] +Input [2]: [d_date_sk#10, d_year#12] (119) BroadcastExchange -Input [1]: [d_date_sk#91] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] +Input [1]: [d_date_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#75] (120) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#87] -Right keys [1]: [d_date_sk#91] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] Join condition: None (121) Project [codegen id : 3] -Output [3]: [ss_quantity#85, ss_sales_price#86, c_customer_sk#89] -Input [5]: [ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89, d_date_sk#91] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Input [5]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29, d_date_sk#10] (122) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#85, ss_sales_price#86, c_customer_sk#89] -Keys [1]: [c_customer_sk#89] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#94, isEmpty#95] -Results [3]: [c_customer_sk#89, sum#96, isEmpty#97] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Keys [1]: [c_customer_sk#29] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#76, isEmpty#77] +Results [3]: [c_customer_sk#29, sum#78, isEmpty#79] (123) Exchange -Input [3]: [c_customer_sk#89, sum#96, isEmpty#97] -Arguments: hashpartitioning(c_customer_sk#89, 5), ENSURE_REQUIREMENTS, [id=#98] +Input [3]: [c_customer_sk#29, sum#78, isEmpty#79] +Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#80] (124) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#89, sum#96, isEmpty#97] -Keys [1]: [c_customer_sk#89] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))#99] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))#99 AS csales#100] +Input [3]: [c_customer_sk#29, sum#78, isEmpty#79] +Keys [1]: [c_customer_sk#29] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#81] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#81 AS csales#82] (125) HashAggregate [codegen id : 4] -Input [1]: [csales#100] +Input [1]: [csales#82] Keys: [] -Functions [1]: [partial_max(csales#100)] -Aggregate Attributes [1]: [max#101] -Results [1]: [max#102] +Functions [1]: [partial_max(csales#82)] +Aggregate Attributes [1]: [max#83] +Results [1]: [max#84] (126) Exchange -Input [1]: [max#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103] +Input [1]: [max#84] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#85] (127) HashAggregate [codegen id : 5] -Input [1]: [max#102] +Input [1]: [max#84] Keys: [] -Functions [1]: [max(csales#100)] -Aggregate Attributes [1]: [max(csales#100)#104] -Results [1]: [max(csales#100)#104 AS tpcds_cmax#105] +Functions [1]: [max(csales#82)] +Aggregate Attributes [1]: [max(csales#82)#86] +Results [1]: [max(csales#82)#86 AS tpcds_cmax#87] -Subquery:4 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#88 +Subquery:4 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#73 ReusedExchange (128) (128) ReusedExchange [Reuses operator id: 119] -Output [1]: [d_date_sk#91] +Output [1]: [d_date_sk#10] -Subquery:5 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:5 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] -Subquery:6 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:7 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] -Subquery:8 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] +Subquery:8 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] 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 3fd978920dc47..e9a0ada3801b4 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 @@ -340,248 +340,248 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (51) Scan parquet default.store_sales -Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +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 (52) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +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] (53) Filter [codegen id : 2] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] -Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) +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)) (54) Project [codegen id : 2] -Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +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] (55) Scan parquet default.store -Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Output [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (56) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] (57) Filter [codegen id : 1] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Condition : (((isnotnull(s_market_id#22) AND (s_market_id#22 = 8)) AND isnotnull(s_store_sk#20)) AND isnotnull(s_zip#24)) (58) Project [codegen id : 1] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Output [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] (59) BroadcastExchange -Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] +Input [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] (60) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#52] -Right keys [1]: [s_store_sk#56] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#20] Join condition: None (61) Project [codegen id : 2] -Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] +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#20, s_store_name#21, s_state#23, s_zip#24] (62) Exchange -Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#51] (63) Sort [codegen id : 3] -Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: [ss_customer_sk#51 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#21, s_state#23, s_zip#24] +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (64) Scan parquet default.customer -Output [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (65) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] (66) Filter [codegen id : 4] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_birth_country#66)) +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_birth_country#18)) (67) Exchange -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, [id=#67] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#52] (68) Sort [codegen id : 5] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: [c_customer_sk#63 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 (69) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#51] -Right keys [1]: [c_customer_sk#63] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#15] Join condition: None (70) Project [codegen id : 6] -Output [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Input [11]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] (71) Exchange -Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#53] (72) Sort [codegen id : 7] -Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (73) Scan parquet default.item -Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +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 (74) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] (75) Filter [codegen id : 8] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Condition : isnotnull(i_item_sk#69) +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) (76) Exchange -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: hashpartitioning(i_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] +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=#54] (77) Sort [codegen id : 9] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: [i_item_sk#69 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 (78) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#50] -Right keys [1]: [i_item_sk#69] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#7] Join condition: None (79) Project [codegen id : 10] -Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] (80) Exchange -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: hashpartitioning(c_birth_country#66, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: hashpartitioning(c_birth_country#18, s_zip#24, 5), ENSURE_REQUIREMENTS, [id=#55] (81) Sort [codegen id : 11] -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: [c_birth_country#66 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: [c_birth_country#18 ASC NULLS FIRST, s_zip#24 ASC NULLS FIRST], false, 0 (82) Scan parquet default.customer_address -Output [3]: [ca_state#77, ca_zip#78, ca_country#79] +Output [3]: [ca_state#26, ca_zip#27, ca_country#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (83) ColumnarToRow [codegen id : 12] -Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] (84) Filter [codegen id : 12] -Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Condition : (isnotnull(ca_country#28) AND isnotnull(ca_zip#27)) (85) Exchange -Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -Arguments: hashpartitioning(upper(ca_country#79), ca_zip#78, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Arguments: hashpartitioning(upper(ca_country#28), ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#56] (86) Sort [codegen id : 13] -Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -Arguments: [upper(ca_country#79) ASC NULLS FIRST, ca_zip#78 ASC NULLS FIRST], false, 0 +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Arguments: [upper(ca_country#28) ASC NULLS FIRST, ca_zip#27 ASC NULLS FIRST], false, 0 (87) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#66, s_zip#60] -Right keys [2]: [upper(ca_country#79), ca_zip#78] +Left keys [2]: [c_birth_country#18, s_zip#24] +Right keys [2]: [upper(ca_country#28), ca_zip#27] Join condition: None (88) Project [codegen id : 14] -Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] -Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, ca_zip#78, ca_country#79] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] +Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, ca_zip#27, ca_country#28] (89) Exchange -Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] -Arguments: hashpartitioning(cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint), 5), ENSURE_REQUIREMENTS, [id=#81] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#57] (90) Sort [codegen id : 15] -Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] -Arguments: [cast(ss_ticket_number#53 as bigint) ASC NULLS FIRST, cast(ss_item_sk#50 as bigint) ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] +Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (91) Scan parquet default.store_returns -Output [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Output [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 16] -Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] (93) Filter [codegen id : 16] -Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] -Condition : (isnotnull(sr_ticket_number#83) AND isnotnull(sr_item_sk#82)) +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) (94) Project [codegen id : 16] -Output [2]: [sr_item_sk#82, sr_ticket_number#83] -Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Output [2]: [sr_item_sk#31, sr_ticket_number#32] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] (95) Exchange -Input [2]: [sr_item_sk#82, sr_ticket_number#83] -Arguments: hashpartitioning(sr_ticket_number#83, sr_item_sk#82, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#58] (96) Sort [codegen id : 17] -Input [2]: [sr_item_sk#82, sr_ticket_number#83] -Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 (97) SortMergeJoin [codegen id : 18] -Left keys [2]: [cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint)] -Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] +Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] +Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] Join condition: None (98) Project [codegen id : 18] -Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] -Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, sr_item_sk#82, sr_ticket_number#83] +Output [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, sr_item_sk#31, sr_ticket_number#32] (99) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] -Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] -Aggregate Attributes [1]: [sum#86] -Results [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Input [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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#59] +Results [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] (100) Exchange -Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] -Arguments: hashpartitioning(c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#61] (101) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] -Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] -Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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))#62] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#62,17,2) AS netpaid#39] (102) HashAggregate [codegen id : 19] Input [1]: [netpaid#39] Keys: [] Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#90, count#91] -Results [2]: [sum#92, count#93] +Aggregate Attributes [2]: [sum#63, count#64] +Results [2]: [sum#65, count#66] (103) Exchange -Input [2]: [sum#92, count#93] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] +Input [2]: [sum#65, count#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] (104) HashAggregate [codegen id : 20] -Input [2]: [sum#92, count#93] +Input [2]: [sum#65, count#66] Keys: [] Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#95] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] +Aggregate Attributes [1]: [avg(netpaid#39)#68] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#68)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#69] 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 a298a4e537788..3b304ebe3f649 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 @@ -316,212 +316,212 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (48) Scan parquet default.store_sales -Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +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 (49) ColumnarToRow [codegen id : 1] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +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] (50) Filter [codegen id : 1] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] -Condition : (((isnotnull(ss_ticket_number#52) AND isnotnull(ss_item_sk#49)) AND isnotnull(ss_store_sk#51)) AND isnotnull(ss_customer_sk#50)) +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)) (51) Project [codegen id : 1] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +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] (52) Exchange -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: hashpartitioning(cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#49] (53) Sort [codegen id : 2] -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [cast(ss_ticket_number#52 as bigint) ASC NULLS FIRST, cast(ss_item_sk#49 as bigint) 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: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.store_returns -Output [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Output [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] (56) Filter [codegen id : 3] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] -Condition : (isnotnull(sr_ticket_number#57) AND isnotnull(sr_item_sk#56)) +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Condition : (isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#8)) (57) Project [codegen id : 3] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Output [2]: [sr_item_sk#8, sr_ticket_number#9] +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] (58) Exchange -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: hashpartitioning(sr_ticket_number#57, sr_item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [2]: [sr_item_sk#8, sr_ticket_number#9] +Arguments: hashpartitioning(sr_ticket_number#9, sr_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#50] (59) Sort [codegen id : 4] -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST], false, 0 +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 (60) SortMergeJoin [codegen id : 9] -Left keys [2]: [cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint)] -Right keys [2]: [sr_ticket_number#57, sr_item_sk#56] +Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] +Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] Join condition: None (61) Project [codegen id : 9] -Output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#56, sr_ticket_number#57] +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] (62) Scan parquet default.store -Output [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Output [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (63) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] (64) Filter [codegen id : 5] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] -Condition : (((isnotnull(s_market_id#62) AND (s_market_id#62 = 8)) AND isnotnull(s_store_sk#60)) AND isnotnull(s_zip#64)) +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Condition : (((isnotnull(s_market_id#14) AND (s_market_id#14 = 8)) AND isnotnull(s_store_sk#12)) AND isnotnull(s_zip#16)) (65) Project [codegen id : 5] -Output [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] (66) BroadcastExchange -Input [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] +Input [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] (67) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#51] -Right keys [1]: [s_store_sk#60] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#12] Join condition: None (68) Project [codegen id : 9] -Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64] -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +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] (69) Scan parquet default.item -Output [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +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 (70) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] (71) Filter [codegen id : 6] -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Condition : isnotnull(i_item_sk#66) +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) (72) BroadcastExchange -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#72] +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=#52] (73) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#49] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#18] Join condition: None (74) Project [codegen id : 9] -Output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +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] (75) Scan parquet default.customer -Output [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Output [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (76) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] (77) Filter [codegen id : 7] -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] -Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_birth_country#76)) +Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_birth_country#28)) (78) BroadcastExchange -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] +Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] (79) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#50] -Right keys [1]: [c_customer_sk#73] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#25] Join condition: None (80) Project [codegen id : 9] -Output [12]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76] -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +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] (81) Scan parquet default.customer_address -Output [3]: [ca_state#78, ca_zip#79, ca_country#80] +Output [3]: [ca_state#30, ca_zip#31, ca_country#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (82) ColumnarToRow [codegen id : 8] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Input [3]: [ca_state#30, ca_zip#31, ca_country#32] (83) Filter [codegen id : 8] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) +Input [3]: [ca_state#30, ca_zip#31, ca_country#32] +Condition : (isnotnull(ca_country#32) AND isnotnull(ca_zip#31)) (84) BroadcastExchange -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#81] +Input [3]: [ca_state#30, ca_zip#31, ca_country#32] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#54] (85) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#76, s_zip#64] -Right keys [2]: [upper(ca_country#80), ca_zip#79] +Left keys [2]: [c_birth_country#28, s_zip#16] +Right keys [2]: [upper(ca_country#32), ca_zip#31] Join condition: None (86) Project [codegen id : 9] -Output [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] -Input [15]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76, ca_state#78, ca_zip#79, ca_country#80] +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] (87) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] -Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#82] -Results [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +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#55] +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#56] (88) Exchange -Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] -Arguments: hashpartitioning(c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, [id=#84] +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#56] +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=#57] (89) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] -Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#85] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#38] +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#56] +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))#58] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#58,17,2) AS netpaid#38] (90) HashAggregate [codegen id : 10] Input [1]: [netpaid#38] Keys: [] Functions [1]: [partial_avg(netpaid#38)] -Aggregate Attributes [2]: [sum#86, count#87] -Results [2]: [sum#88, count#89] +Aggregate Attributes [2]: [sum#59, count#60] +Results [2]: [sum#61, count#62] (91) 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] (92) HashAggregate [codegen id : 11] -Input [2]: [sum#88, count#89] +Input [2]: [sum#61, count#62] Keys: [] Functions [1]: [avg(netpaid#38)] -Aggregate Attributes [1]: [avg(netpaid#38)#91] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] +Aggregate Attributes [1]: [avg(netpaid#38)#64] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#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.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt index 9cb54bfce5475..695f6efe560e4 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 @@ -340,248 +340,248 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (51) Scan parquet default.store_sales -Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +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 (52) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +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] (53) Filter [codegen id : 2] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] -Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) +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)) (54) Project [codegen id : 2] -Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] -Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +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] (55) Scan parquet default.store -Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Output [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (56) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] (57) Filter [codegen id : 1] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Condition : (((isnotnull(s_market_id#22) AND (s_market_id#22 = 8)) AND isnotnull(s_store_sk#20)) AND isnotnull(s_zip#24)) (58) Project [codegen id : 1] -Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Output [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] (59) BroadcastExchange -Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] +Input [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] (60) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#52] -Right keys [1]: [s_store_sk#56] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#20] Join condition: None (61) Project [codegen id : 2] -Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] +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#20, s_store_name#21, s_state#23, s_zip#24] (62) Exchange -Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#51] (63) Sort [codegen id : 3] -Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] -Arguments: [ss_customer_sk#51 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#21, s_state#23, s_zip#24] +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (64) Scan parquet default.customer -Output [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (65) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] (66) Filter [codegen id : 4] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_birth_country#66)) +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_birth_country#18)) (67) Exchange -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, [id=#67] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#52] (68) Sort [codegen id : 5] -Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: [c_customer_sk#63 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 (69) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#51] -Right keys [1]: [c_customer_sk#63] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#15] Join condition: None (70) Project [codegen id : 6] -Output [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Input [11]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] (71) Exchange -Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#53] (72) Sort [codegen id : 7] -Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] -Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (73) Scan parquet default.item -Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +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 (74) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] (75) Filter [codegen id : 8] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Condition : isnotnull(i_item_sk#69) +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) (76) Exchange -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: hashpartitioning(i_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] +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=#54] (77) Sort [codegen id : 9] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: [i_item_sk#69 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 (78) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#50] -Right keys [1]: [i_item_sk#69] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#7] Join condition: None (79) Project [codegen id : 10] -Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] (80) Exchange -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: hashpartitioning(c_birth_country#66, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: hashpartitioning(c_birth_country#18, s_zip#24, 5), ENSURE_REQUIREMENTS, [id=#55] (81) Sort [codegen id : 11] -Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: [c_birth_country#66 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: [c_birth_country#18 ASC NULLS FIRST, s_zip#24 ASC NULLS FIRST], false, 0 (82) Scan parquet default.customer_address -Output [3]: [ca_state#77, ca_zip#78, ca_country#79] +Output [3]: [ca_state#26, ca_zip#27, ca_country#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (83) ColumnarToRow [codegen id : 12] -Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] (84) Filter [codegen id : 12] -Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Condition : (isnotnull(ca_country#28) AND isnotnull(ca_zip#27)) (85) Exchange -Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -Arguments: hashpartitioning(upper(ca_country#79), ca_zip#78, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Arguments: hashpartitioning(upper(ca_country#28), ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#56] (86) Sort [codegen id : 13] -Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -Arguments: [upper(ca_country#79) ASC NULLS FIRST, ca_zip#78 ASC NULLS FIRST], false, 0 +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Arguments: [upper(ca_country#28) ASC NULLS FIRST, ca_zip#27 ASC NULLS FIRST], false, 0 (87) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#66, s_zip#60] -Right keys [2]: [upper(ca_country#79), ca_zip#78] +Left keys [2]: [c_birth_country#18, s_zip#24] +Right keys [2]: [upper(ca_country#28), ca_zip#27] Join condition: None (88) Project [codegen id : 14] -Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] -Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, ca_zip#78, ca_country#79] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] +Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, ca_zip#27, ca_country#28] (89) Exchange -Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] -Arguments: hashpartitioning(cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint), 5), ENSURE_REQUIREMENTS, [id=#81] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#57] (90) Sort [codegen id : 15] -Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] -Arguments: [cast(ss_ticket_number#53 as bigint) ASC NULLS FIRST, cast(ss_item_sk#50 as bigint) ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] +Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (91) Scan parquet default.store_returns -Output [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Output [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 16] -Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] (93) Filter [codegen id : 16] -Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] -Condition : (isnotnull(sr_ticket_number#83) AND isnotnull(sr_item_sk#82)) +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) (94) Project [codegen id : 16] -Output [2]: [sr_item_sk#82, sr_ticket_number#83] -Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Output [2]: [sr_item_sk#31, sr_ticket_number#32] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] (95) Exchange -Input [2]: [sr_item_sk#82, sr_ticket_number#83] -Arguments: hashpartitioning(sr_ticket_number#83, sr_item_sk#82, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#58] (96) Sort [codegen id : 17] -Input [2]: [sr_item_sk#82, sr_ticket_number#83] -Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 (97) SortMergeJoin [codegen id : 18] -Left keys [2]: [cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint)] -Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] +Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] +Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] Join condition: None (98) Project [codegen id : 18] -Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] -Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, sr_item_sk#82, sr_ticket_number#83] +Output [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, sr_item_sk#31, sr_ticket_number#32] (99) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] -Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] -Aggregate Attributes [1]: [sum#86] -Results [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Input [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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#59] +Results [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] (100) Exchange -Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] -Arguments: hashpartitioning(c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#61] (101) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] -Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] -Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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))#62] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#62,17,2) AS netpaid#39] (102) HashAggregate [codegen id : 19] Input [1]: [netpaid#39] Keys: [] Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#90, count#91] -Results [2]: [sum#92, count#93] +Aggregate Attributes [2]: [sum#63, count#64] +Results [2]: [sum#65, count#66] (103) Exchange -Input [2]: [sum#92, count#93] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] +Input [2]: [sum#65, count#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] (104) HashAggregate [codegen id : 20] -Input [2]: [sum#92, count#93] +Input [2]: [sum#65, count#66] Keys: [] Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#95] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] +Aggregate Attributes [1]: [avg(netpaid#39)#68] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#68)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#69] 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 ff0ca40ea8836..49f76525f88d8 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 @@ -316,212 +316,212 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (48) Scan parquet default.store_sales -Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +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 (49) ColumnarToRow [codegen id : 1] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +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] (50) Filter [codegen id : 1] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] -Condition : (((isnotnull(ss_ticket_number#52) AND isnotnull(ss_item_sk#49)) AND isnotnull(ss_store_sk#51)) AND isnotnull(ss_customer_sk#50)) +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)) (51) Project [codegen id : 1] -Output [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] +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] (52) Exchange -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: hashpartitioning(cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#49] (53) Sort [codegen id : 2] -Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] -Arguments: [cast(ss_ticket_number#52 as bigint) ASC NULLS FIRST, cast(ss_item_sk#49 as bigint) 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: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.store_returns -Output [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Output [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] (56) Filter [codegen id : 3] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] -Condition : (isnotnull(sr_ticket_number#57) AND isnotnull(sr_item_sk#56)) +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Condition : (isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#8)) (57) Project [codegen id : 3] -Output [2]: [sr_item_sk#56, sr_ticket_number#57] -Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Output [2]: [sr_item_sk#8, sr_ticket_number#9] +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] (58) Exchange -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: hashpartitioning(sr_ticket_number#57, sr_item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [2]: [sr_item_sk#8, sr_ticket_number#9] +Arguments: hashpartitioning(sr_ticket_number#9, sr_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#50] (59) Sort [codegen id : 4] -Input [2]: [sr_item_sk#56, sr_ticket_number#57] -Arguments: [sr_ticket_number#57 ASC NULLS FIRST, sr_item_sk#56 ASC NULLS FIRST], false, 0 +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 (60) SortMergeJoin [codegen id : 9] -Left keys [2]: [cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint)] -Right keys [2]: [sr_ticket_number#57, sr_item_sk#56] +Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] +Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] Join condition: None (61) Project [codegen id : 9] -Output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] -Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#56, sr_ticket_number#57] +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] (62) Scan parquet default.store -Output [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Output [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (63) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] (64) Filter [codegen id : 5] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] -Condition : (((isnotnull(s_market_id#62) AND (s_market_id#62 = 8)) AND isnotnull(s_store_sk#60)) AND isnotnull(s_zip#64)) +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Condition : (((isnotnull(s_market_id#14) AND (s_market_id#14 = 8)) AND isnotnull(s_store_sk#12)) AND isnotnull(s_zip#16)) (65) Project [codegen id : 5] -Output [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] -Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] (66) BroadcastExchange -Input [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] +Input [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] (67) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#51] -Right keys [1]: [s_store_sk#60] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#12] Join condition: None (68) Project [codegen id : 9] -Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64] -Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +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] (69) Scan parquet default.item -Output [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +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 (70) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] (71) Filter [codegen id : 6] -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Condition : isnotnull(i_item_sk#66) +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) (72) BroadcastExchange -Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#72] +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=#52] (73) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#49] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#18] Join condition: None (74) Project [codegen id : 9] -Output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] -Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +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] (75) Scan parquet default.customer -Output [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Output [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (76) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] (77) Filter [codegen id : 7] -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] -Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_birth_country#76)) +Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_birth_country#28)) (78) BroadcastExchange -Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] +Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] (79) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#50] -Right keys [1]: [c_customer_sk#73] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#25] Join condition: None (80) Project [codegen id : 9] -Output [12]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76] -Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +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] (81) Scan parquet default.customer_address -Output [3]: [ca_state#78, ca_zip#79, ca_country#80] +Output [3]: [ca_state#30, ca_zip#31, ca_country#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (82) ColumnarToRow [codegen id : 8] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Input [3]: [ca_state#30, ca_zip#31, ca_country#32] (83) Filter [codegen id : 8] -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) +Input [3]: [ca_state#30, ca_zip#31, ca_country#32] +Condition : (isnotnull(ca_country#32) AND isnotnull(ca_zip#31)) (84) BroadcastExchange -Input [3]: [ca_state#78, ca_zip#79, ca_country#80] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#81] +Input [3]: [ca_state#30, ca_zip#31, ca_country#32] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#54] (85) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#76, s_zip#64] -Right keys [2]: [upper(ca_country#80), ca_zip#79] +Left keys [2]: [c_birth_country#28, s_zip#16] +Right keys [2]: [upper(ca_country#32), ca_zip#31] Join condition: None (86) Project [codegen id : 9] -Output [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] -Input [15]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76, ca_state#78, ca_zip#79, ca_country#80] +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] (87) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] -Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum#82] -Results [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +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#55] +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#56] (88) Exchange -Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] -Arguments: hashpartitioning(c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, [id=#84] +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#56] +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=#57] (89) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] -Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] -Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#85] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#38] +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#56] +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))#58] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#58,17,2) AS netpaid#38] (90) HashAggregate [codegen id : 10] Input [1]: [netpaid#38] Keys: [] Functions [1]: [partial_avg(netpaid#38)] -Aggregate Attributes [2]: [sum#86, count#87] -Results [2]: [sum#88, count#89] +Aggregate Attributes [2]: [sum#59, count#60] +Results [2]: [sum#61, count#62] (91) 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] (92) HashAggregate [codegen id : 11] -Input [2]: [sum#88, count#89] +Input [2]: [sum#61, count#62] Keys: [] Functions [1]: [avg(netpaid#38)] -Aggregate Attributes [1]: [avg(netpaid#38)#91] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] +Aggregate Attributes [1]: [avg(netpaid#38)#64] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#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/q28.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt index eec45ea549531..9f15199d8c8b7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt @@ -126,306 +126,306 @@ Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#6 / 100.0) as decimal(11,6)) AS B1_LP#15, count(ss_list_price#3)#7 AS B1_CNT#16, count(ss_list_price#3)#12 AS B1_CNTD#17] (11) Scan parquet default.store_sales -Output [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (13) Filter [codegen id : 3] -Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] -Condition : (((isnotnull(ss_quantity#18) AND (ss_quantity#18 >= 6)) AND (ss_quantity#18 <= 10)) AND ((((ss_list_price#20 >= 90.00) AND (ss_list_price#20 <= 100.00)) OR ((ss_coupon_amt#21 >= 2323.00) AND (ss_coupon_amt#21 <= 3323.00))) OR ((ss_wholesale_cost#19 >= 31.00) AND (ss_wholesale_cost#19 <= 51.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 6)) AND (ss_quantity#1 <= 10)) AND ((((ss_list_price#3 >= 90.00) AND (ss_list_price#3 <= 100.00)) OR ((ss_coupon_amt#4 >= 2323.00) AND (ss_coupon_amt#4 <= 3323.00))) OR ((ss_wholesale_cost#2 >= 31.00) AND (ss_wholesale_cost#2 <= 51.00)))) (14) Project [codegen id : 3] -Output [1]: [ss_list_price#20] -Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (15) HashAggregate [codegen id : 3] -Input [1]: [ss_list_price#20] -Keys [1]: [ss_list_price#20] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#20)), partial_count(ss_list_price#20)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] -Results [4]: [ss_list_price#20, sum#25, count#26, count#27] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] +Results [4]: [ss_list_price#3, sum#20, count#21, count#22] (16) Exchange -Input [4]: [ss_list_price#20, sum#25, count#26, count#27] -Arguments: hashpartitioning(ss_list_price#20, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [4]: [ss_list_price#3, sum#20, count#21, count#22] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#23] (17) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#20, sum#25, count#26, count#27] -Keys [1]: [ss_list_price#20] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] -Results [4]: [ss_list_price#20, sum#25, count#26, count#27] +Input [4]: [ss_list_price#3, sum#20, count#21, count#22] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] +Results [4]: [ss_list_price#3, sum#20, count#21, count#22] (18) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#20, sum#25, count#26, count#27] +Input [4]: [ss_list_price#3, sum#20, count#21, count#22] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20), partial_count(distinct ss_list_price#20)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] -Results [4]: [sum#25, count#26, count#27, count#30] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] +Results [4]: [sum#20, count#21, count#22, count#25] (19) Exchange -Input [4]: [sum#25, count#26, count#27, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] +Input [4]: [sum#20, count#21, count#22, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] (20) HashAggregate [codegen id : 5] -Input [4]: [sum#25, count#26, count#27, count#30] +Input [4]: [sum#20, count#21, count#22, count#25] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#20)), count(ss_list_price#20), count(distinct ss_list_price#20)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#20))#23 / 100.0) as decimal(11,6)) AS B2_LP#32, count(ss_list_price#20)#24 AS B2_CNT#33, count(ss_list_price#20)#29 AS B2_CNTD#34] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#18 / 100.0) as decimal(11,6)) AS B2_LP#27, count(ss_list_price#3)#19 AS B2_CNT#28, count(ss_list_price#3)#24 AS B2_CNTD#29] (21) BroadcastExchange -Input [3]: [B2_LP#32, B2_CNT#33, B2_CNTD#34] -Arguments: IdentityBroadcastMode, [id=#35] +Input [3]: [B2_LP#27, B2_CNT#28, B2_CNTD#29] +Arguments: IdentityBroadcastMode, [id=#30] (22) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (23) Scan parquet default.store_sales -Output [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (25) Filter [codegen id : 6] -Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] -Condition : (((isnotnull(ss_quantity#36) AND (ss_quantity#36 >= 11)) AND (ss_quantity#36 <= 15)) AND ((((ss_list_price#38 >= 142.00) AND (ss_list_price#38 <= 152.00)) OR ((ss_coupon_amt#39 >= 12214.00) AND (ss_coupon_amt#39 <= 13214.00))) OR ((ss_wholesale_cost#37 >= 79.00) AND (ss_wholesale_cost#37 <= 99.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 11)) AND (ss_quantity#1 <= 15)) AND ((((ss_list_price#3 >= 142.00) AND (ss_list_price#3 <= 152.00)) OR ((ss_coupon_amt#4 >= 12214.00) AND (ss_coupon_amt#4 <= 13214.00))) OR ((ss_wholesale_cost#2 >= 79.00) AND (ss_wholesale_cost#2 <= 99.00)))) (26) Project [codegen id : 6] -Output [1]: [ss_list_price#38] -Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (27) HashAggregate [codegen id : 6] -Input [1]: [ss_list_price#38] -Keys [1]: [ss_list_price#38] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#38)), partial_count(ss_list_price#38)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] -Results [4]: [ss_list_price#38, sum#43, count#44, count#45] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] +Results [4]: [ss_list_price#3, sum#33, count#34, count#35] (28) Exchange -Input [4]: [ss_list_price#38, sum#43, count#44, count#45] -Arguments: hashpartitioning(ss_list_price#38, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [4]: [ss_list_price#3, sum#33, count#34, count#35] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#36] (29) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#38, sum#43, count#44, count#45] -Keys [1]: [ss_list_price#38] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] -Results [4]: [ss_list_price#38, sum#43, count#44, count#45] +Input [4]: [ss_list_price#3, sum#33, count#34, count#35] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] +Results [4]: [ss_list_price#3, sum#33, count#34, count#35] (30) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#38, sum#43, count#44, count#45] +Input [4]: [ss_list_price#3, sum#33, count#34, count#35] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38), partial_count(distinct ss_list_price#38)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] -Results [4]: [sum#43, count#44, count#45, count#48] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] +Results [4]: [sum#33, count#34, count#35, count#38] (31) Exchange -Input [4]: [sum#43, count#44, count#45, count#48] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [sum#33, count#34, count#35, count#38] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#39] (32) HashAggregate [codegen id : 8] -Input [4]: [sum#43, count#44, count#45, count#48] +Input [4]: [sum#33, count#34, count#35, count#38] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#38)), count(ss_list_price#38), count(distinct ss_list_price#38)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#38))#41 / 100.0) as decimal(11,6)) AS B3_LP#50, count(ss_list_price#38)#42 AS B3_CNT#51, count(ss_list_price#38)#47 AS B3_CNTD#52] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#31 / 100.0) as decimal(11,6)) AS B3_LP#40, count(ss_list_price#3)#32 AS B3_CNT#41, count(ss_list_price#3)#37 AS B3_CNTD#42] (33) BroadcastExchange -Input [3]: [B3_LP#50, B3_CNT#51, B3_CNTD#52] -Arguments: IdentityBroadcastMode, [id=#53] +Input [3]: [B3_LP#40, B3_CNT#41, B3_CNTD#42] +Arguments: IdentityBroadcastMode, [id=#43] (34) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (35) Scan parquet default.store_sales -Output [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] ReadSchema: struct (36) ColumnarToRow [codegen id : 9] -Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (37) Filter [codegen id : 9] -Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] -Condition : (((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 16)) AND (ss_quantity#54 <= 20)) AND ((((ss_list_price#56 >= 135.00) AND (ss_list_price#56 <= 145.00)) OR ((ss_coupon_amt#57 >= 6071.00) AND (ss_coupon_amt#57 <= 7071.00))) OR ((ss_wholesale_cost#55 >= 38.00) AND (ss_wholesale_cost#55 <= 58.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 16)) AND (ss_quantity#1 <= 20)) AND ((((ss_list_price#3 >= 135.00) AND (ss_list_price#3 <= 145.00)) OR ((ss_coupon_amt#4 >= 6071.00) AND (ss_coupon_amt#4 <= 7071.00))) OR ((ss_wholesale_cost#2 >= 38.00) AND (ss_wholesale_cost#2 <= 58.00)))) (38) Project [codegen id : 9] -Output [1]: [ss_list_price#56] -Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (39) HashAggregate [codegen id : 9] -Input [1]: [ss_list_price#56] -Keys [1]: [ss_list_price#56] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#56)), partial_count(ss_list_price#56)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] -Results [4]: [ss_list_price#56, sum#61, count#62, count#63] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] +Results [4]: [ss_list_price#3, sum#46, count#47, count#48] (40) Exchange -Input [4]: [ss_list_price#56, sum#61, count#62, count#63] -Arguments: hashpartitioning(ss_list_price#56, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [4]: [ss_list_price#3, sum#46, count#47, count#48] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#49] (41) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#56, sum#61, count#62, count#63] -Keys [1]: [ss_list_price#56] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] -Results [4]: [ss_list_price#56, sum#61, count#62, count#63] +Input [4]: [ss_list_price#3, sum#46, count#47, count#48] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] +Results [4]: [ss_list_price#3, sum#46, count#47, count#48] (42) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#56, sum#61, count#62, count#63] +Input [4]: [ss_list_price#3, sum#46, count#47, count#48] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56), partial_count(distinct ss_list_price#56)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] -Results [4]: [sum#61, count#62, count#63, count#66] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] +Results [4]: [sum#46, count#47, count#48, count#51] (43) Exchange -Input [4]: [sum#61, count#62, count#63, count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] +Input [4]: [sum#46, count#47, count#48, count#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] (44) HashAggregate [codegen id : 11] -Input [4]: [sum#61, count#62, count#63, count#66] +Input [4]: [sum#46, count#47, count#48, count#51] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#56)), count(ss_list_price#56), count(distinct ss_list_price#56)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#56))#59 / 100.0) as decimal(11,6)) AS B4_LP#68, count(ss_list_price#56)#60 AS B4_CNT#69, count(ss_list_price#56)#65 AS B4_CNTD#70] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#44 / 100.0) as decimal(11,6)) AS B4_LP#53, count(ss_list_price#3)#45 AS B4_CNT#54, count(ss_list_price#3)#50 AS B4_CNTD#55] (45) BroadcastExchange -Input [3]: [B4_LP#68, B4_CNT#69, B4_CNTD#70] -Arguments: IdentityBroadcastMode, [id=#71] +Input [3]: [B4_LP#53, B4_CNT#54, B4_CNTD#55] +Arguments: IdentityBroadcastMode, [id=#56] (46) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (47) Scan parquet default.store_sales -Output [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] ReadSchema: struct (48) ColumnarToRow [codegen id : 12] -Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (49) Filter [codegen id : 12] -Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] -Condition : (((isnotnull(ss_quantity#72) AND (ss_quantity#72 >= 21)) AND (ss_quantity#72 <= 25)) AND ((((ss_list_price#74 >= 122.00) AND (ss_list_price#74 <= 132.00)) OR ((ss_coupon_amt#75 >= 836.00) AND (ss_coupon_amt#75 <= 1836.00))) OR ((ss_wholesale_cost#73 >= 17.00) AND (ss_wholesale_cost#73 <= 37.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 21)) AND (ss_quantity#1 <= 25)) AND ((((ss_list_price#3 >= 122.00) AND (ss_list_price#3 <= 132.00)) OR ((ss_coupon_amt#4 >= 836.00) AND (ss_coupon_amt#4 <= 1836.00))) OR ((ss_wholesale_cost#2 >= 17.00) AND (ss_wholesale_cost#2 <= 37.00)))) (50) Project [codegen id : 12] -Output [1]: [ss_list_price#74] -Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (51) HashAggregate [codegen id : 12] -Input [1]: [ss_list_price#74] -Keys [1]: [ss_list_price#74] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#74)), partial_count(ss_list_price#74)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] -Results [4]: [ss_list_price#74, sum#79, count#80, count#81] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] +Results [4]: [ss_list_price#3, sum#59, count#60, count#61] (52) Exchange -Input [4]: [ss_list_price#74, sum#79, count#80, count#81] -Arguments: hashpartitioning(ss_list_price#74, 5), ENSURE_REQUIREMENTS, [id=#82] +Input [4]: [ss_list_price#3, sum#59, count#60, count#61] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#62] (53) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#74, sum#79, count#80, count#81] -Keys [1]: [ss_list_price#74] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] -Results [4]: [ss_list_price#74, sum#79, count#80, count#81] +Input [4]: [ss_list_price#3, sum#59, count#60, count#61] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] +Results [4]: [ss_list_price#3, sum#59, count#60, count#61] (54) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#74, sum#79, count#80, count#81] +Input [4]: [ss_list_price#3, sum#59, count#60, count#61] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74), partial_count(distinct ss_list_price#74)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] -Results [4]: [sum#79, count#80, count#81, count#84] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] +Results [4]: [sum#59, count#60, count#61, count#64] (55) Exchange -Input [4]: [sum#79, count#80, count#81, count#84] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#85] +Input [4]: [sum#59, count#60, count#61, count#64] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] (56) HashAggregate [codegen id : 14] -Input [4]: [sum#79, count#80, count#81, count#84] +Input [4]: [sum#59, count#60, count#61, count#64] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#74)), count(ss_list_price#74), count(distinct ss_list_price#74)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#74))#77 / 100.0) as decimal(11,6)) AS B5_LP#86, count(ss_list_price#74)#78 AS B5_CNT#87, count(ss_list_price#74)#83 AS B5_CNTD#88] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#57 / 100.0) as decimal(11,6)) AS B5_LP#66, count(ss_list_price#3)#58 AS B5_CNT#67, count(ss_list_price#3)#63 AS B5_CNTD#68] (57) BroadcastExchange -Input [3]: [B5_LP#86, B5_CNT#87, B5_CNTD#88] -Arguments: IdentityBroadcastMode, [id=#89] +Input [3]: [B5_LP#66, B5_CNT#67, B5_CNTD#68] +Arguments: IdentityBroadcastMode, [id=#69] (58) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (59) Scan parquet default.store_sales -Output [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] ReadSchema: struct (60) ColumnarToRow [codegen id : 15] -Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (61) Filter [codegen id : 15] -Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] -Condition : (((isnotnull(ss_quantity#90) AND (ss_quantity#90 >= 26)) AND (ss_quantity#90 <= 30)) AND ((((ss_list_price#92 >= 154.00) AND (ss_list_price#92 <= 164.00)) OR ((ss_coupon_amt#93 >= 7326.00) AND (ss_coupon_amt#93 <= 8326.00))) OR ((ss_wholesale_cost#91 >= 7.00) AND (ss_wholesale_cost#91 <= 27.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 26)) AND (ss_quantity#1 <= 30)) AND ((((ss_list_price#3 >= 154.00) AND (ss_list_price#3 <= 164.00)) OR ((ss_coupon_amt#4 >= 7326.00) AND (ss_coupon_amt#4 <= 8326.00))) OR ((ss_wholesale_cost#2 >= 7.00) AND (ss_wholesale_cost#2 <= 27.00)))) (62) Project [codegen id : 15] -Output [1]: [ss_list_price#92] -Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (63) HashAggregate [codegen id : 15] -Input [1]: [ss_list_price#92] -Keys [1]: [ss_list_price#92] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#92)), partial_count(ss_list_price#92)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] -Results [4]: [ss_list_price#92, sum#97, count#98, count#99] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] +Results [4]: [ss_list_price#3, sum#72, count#73, count#74] (64) Exchange -Input [4]: [ss_list_price#92, sum#97, count#98, count#99] -Arguments: hashpartitioning(ss_list_price#92, 5), ENSURE_REQUIREMENTS, [id=#100] +Input [4]: [ss_list_price#3, sum#72, count#73, count#74] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#75] (65) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#92, sum#97, count#98, count#99] -Keys [1]: [ss_list_price#92] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] -Results [4]: [ss_list_price#92, sum#97, count#98, count#99] +Input [4]: [ss_list_price#3, sum#72, count#73, count#74] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] +Results [4]: [ss_list_price#3, sum#72, count#73, count#74] (66) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#92, sum#97, count#98, count#99] +Input [4]: [ss_list_price#3, sum#72, count#73, count#74] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92), partial_count(distinct ss_list_price#92)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] -Results [4]: [sum#97, count#98, count#99, count#102] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] +Results [4]: [sum#72, count#73, count#74, count#77] (67) Exchange -Input [4]: [sum#97, count#98, count#99, count#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103] +Input [4]: [sum#72, count#73, count#74, count#77] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] (68) HashAggregate [codegen id : 17] -Input [4]: [sum#97, count#98, count#99, count#102] +Input [4]: [sum#72, count#73, count#74, count#77] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#92)), count(ss_list_price#92), count(distinct ss_list_price#92)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#92))#95 / 100.0) as decimal(11,6)) AS B6_LP#104, count(ss_list_price#92)#96 AS B6_CNT#105, count(ss_list_price#92)#101 AS B6_CNTD#106] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#70 / 100.0) as decimal(11,6)) AS B6_LP#79, count(ss_list_price#3)#71 AS B6_CNT#80, count(ss_list_price#3)#76 AS B6_CNTD#81] (69) BroadcastExchange -Input [3]: [B6_LP#104, B6_CNT#105, B6_CNTD#106] -Arguments: IdentityBroadcastMode, [id=#107] +Input [3]: [B6_LP#79, B6_CNT#80, B6_CNTD#81] +Arguments: IdentityBroadcastMode, [id=#82] (70) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt index eec45ea549531..9f15199d8c8b7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt @@ -126,306 +126,306 @@ Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#6 / 100.0) as decimal(11,6)) AS B1_LP#15, count(ss_list_price#3)#7 AS B1_CNT#16, count(ss_list_price#3)#12 AS B1_CNTD#17] (11) Scan parquet default.store_sales -Output [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (13) Filter [codegen id : 3] -Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] -Condition : (((isnotnull(ss_quantity#18) AND (ss_quantity#18 >= 6)) AND (ss_quantity#18 <= 10)) AND ((((ss_list_price#20 >= 90.00) AND (ss_list_price#20 <= 100.00)) OR ((ss_coupon_amt#21 >= 2323.00) AND (ss_coupon_amt#21 <= 3323.00))) OR ((ss_wholesale_cost#19 >= 31.00) AND (ss_wholesale_cost#19 <= 51.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 6)) AND (ss_quantity#1 <= 10)) AND ((((ss_list_price#3 >= 90.00) AND (ss_list_price#3 <= 100.00)) OR ((ss_coupon_amt#4 >= 2323.00) AND (ss_coupon_amt#4 <= 3323.00))) OR ((ss_wholesale_cost#2 >= 31.00) AND (ss_wholesale_cost#2 <= 51.00)))) (14) Project [codegen id : 3] -Output [1]: [ss_list_price#20] -Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (15) HashAggregate [codegen id : 3] -Input [1]: [ss_list_price#20] -Keys [1]: [ss_list_price#20] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#20)), partial_count(ss_list_price#20)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] -Results [4]: [ss_list_price#20, sum#25, count#26, count#27] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] +Results [4]: [ss_list_price#3, sum#20, count#21, count#22] (16) Exchange -Input [4]: [ss_list_price#20, sum#25, count#26, count#27] -Arguments: hashpartitioning(ss_list_price#20, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [4]: [ss_list_price#3, sum#20, count#21, count#22] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#23] (17) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#20, sum#25, count#26, count#27] -Keys [1]: [ss_list_price#20] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] -Results [4]: [ss_list_price#20, sum#25, count#26, count#27] +Input [4]: [ss_list_price#3, sum#20, count#21, count#22] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] +Results [4]: [ss_list_price#3, sum#20, count#21, count#22] (18) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#20, sum#25, count#26, count#27] +Input [4]: [ss_list_price#3, sum#20, count#21, count#22] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20), partial_count(distinct ss_list_price#20)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] -Results [4]: [sum#25, count#26, count#27, count#30] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] +Results [4]: [sum#20, count#21, count#22, count#25] (19) Exchange -Input [4]: [sum#25, count#26, count#27, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] +Input [4]: [sum#20, count#21, count#22, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] (20) HashAggregate [codegen id : 5] -Input [4]: [sum#25, count#26, count#27, count#30] +Input [4]: [sum#20, count#21, count#22, count#25] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#20)), count(ss_list_price#20), count(distinct ss_list_price#20)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#20))#23 / 100.0) as decimal(11,6)) AS B2_LP#32, count(ss_list_price#20)#24 AS B2_CNT#33, count(ss_list_price#20)#29 AS B2_CNTD#34] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#18 / 100.0) as decimal(11,6)) AS B2_LP#27, count(ss_list_price#3)#19 AS B2_CNT#28, count(ss_list_price#3)#24 AS B2_CNTD#29] (21) BroadcastExchange -Input [3]: [B2_LP#32, B2_CNT#33, B2_CNTD#34] -Arguments: IdentityBroadcastMode, [id=#35] +Input [3]: [B2_LP#27, B2_CNT#28, B2_CNTD#29] +Arguments: IdentityBroadcastMode, [id=#30] (22) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (23) Scan parquet default.store_sales -Output [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (25) Filter [codegen id : 6] -Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] -Condition : (((isnotnull(ss_quantity#36) AND (ss_quantity#36 >= 11)) AND (ss_quantity#36 <= 15)) AND ((((ss_list_price#38 >= 142.00) AND (ss_list_price#38 <= 152.00)) OR ((ss_coupon_amt#39 >= 12214.00) AND (ss_coupon_amt#39 <= 13214.00))) OR ((ss_wholesale_cost#37 >= 79.00) AND (ss_wholesale_cost#37 <= 99.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 11)) AND (ss_quantity#1 <= 15)) AND ((((ss_list_price#3 >= 142.00) AND (ss_list_price#3 <= 152.00)) OR ((ss_coupon_amt#4 >= 12214.00) AND (ss_coupon_amt#4 <= 13214.00))) OR ((ss_wholesale_cost#2 >= 79.00) AND (ss_wholesale_cost#2 <= 99.00)))) (26) Project [codegen id : 6] -Output [1]: [ss_list_price#38] -Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (27) HashAggregate [codegen id : 6] -Input [1]: [ss_list_price#38] -Keys [1]: [ss_list_price#38] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#38)), partial_count(ss_list_price#38)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] -Results [4]: [ss_list_price#38, sum#43, count#44, count#45] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] +Results [4]: [ss_list_price#3, sum#33, count#34, count#35] (28) Exchange -Input [4]: [ss_list_price#38, sum#43, count#44, count#45] -Arguments: hashpartitioning(ss_list_price#38, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [4]: [ss_list_price#3, sum#33, count#34, count#35] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#36] (29) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#38, sum#43, count#44, count#45] -Keys [1]: [ss_list_price#38] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] -Results [4]: [ss_list_price#38, sum#43, count#44, count#45] +Input [4]: [ss_list_price#3, sum#33, count#34, count#35] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] +Results [4]: [ss_list_price#3, sum#33, count#34, count#35] (30) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#38, sum#43, count#44, count#45] +Input [4]: [ss_list_price#3, sum#33, count#34, count#35] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38), partial_count(distinct ss_list_price#38)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] -Results [4]: [sum#43, count#44, count#45, count#48] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] +Results [4]: [sum#33, count#34, count#35, count#38] (31) Exchange -Input [4]: [sum#43, count#44, count#45, count#48] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [sum#33, count#34, count#35, count#38] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#39] (32) HashAggregate [codegen id : 8] -Input [4]: [sum#43, count#44, count#45, count#48] +Input [4]: [sum#33, count#34, count#35, count#38] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#38)), count(ss_list_price#38), count(distinct ss_list_price#38)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#38))#41 / 100.0) as decimal(11,6)) AS B3_LP#50, count(ss_list_price#38)#42 AS B3_CNT#51, count(ss_list_price#38)#47 AS B3_CNTD#52] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#31 / 100.0) as decimal(11,6)) AS B3_LP#40, count(ss_list_price#3)#32 AS B3_CNT#41, count(ss_list_price#3)#37 AS B3_CNTD#42] (33) BroadcastExchange -Input [3]: [B3_LP#50, B3_CNT#51, B3_CNTD#52] -Arguments: IdentityBroadcastMode, [id=#53] +Input [3]: [B3_LP#40, B3_CNT#41, B3_CNTD#42] +Arguments: IdentityBroadcastMode, [id=#43] (34) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (35) Scan parquet default.store_sales -Output [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] ReadSchema: struct (36) ColumnarToRow [codegen id : 9] -Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (37) Filter [codegen id : 9] -Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] -Condition : (((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 16)) AND (ss_quantity#54 <= 20)) AND ((((ss_list_price#56 >= 135.00) AND (ss_list_price#56 <= 145.00)) OR ((ss_coupon_amt#57 >= 6071.00) AND (ss_coupon_amt#57 <= 7071.00))) OR ((ss_wholesale_cost#55 >= 38.00) AND (ss_wholesale_cost#55 <= 58.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 16)) AND (ss_quantity#1 <= 20)) AND ((((ss_list_price#3 >= 135.00) AND (ss_list_price#3 <= 145.00)) OR ((ss_coupon_amt#4 >= 6071.00) AND (ss_coupon_amt#4 <= 7071.00))) OR ((ss_wholesale_cost#2 >= 38.00) AND (ss_wholesale_cost#2 <= 58.00)))) (38) Project [codegen id : 9] -Output [1]: [ss_list_price#56] -Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (39) HashAggregate [codegen id : 9] -Input [1]: [ss_list_price#56] -Keys [1]: [ss_list_price#56] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#56)), partial_count(ss_list_price#56)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] -Results [4]: [ss_list_price#56, sum#61, count#62, count#63] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] +Results [4]: [ss_list_price#3, sum#46, count#47, count#48] (40) Exchange -Input [4]: [ss_list_price#56, sum#61, count#62, count#63] -Arguments: hashpartitioning(ss_list_price#56, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [4]: [ss_list_price#3, sum#46, count#47, count#48] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#49] (41) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#56, sum#61, count#62, count#63] -Keys [1]: [ss_list_price#56] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] -Results [4]: [ss_list_price#56, sum#61, count#62, count#63] +Input [4]: [ss_list_price#3, sum#46, count#47, count#48] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] +Results [4]: [ss_list_price#3, sum#46, count#47, count#48] (42) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#56, sum#61, count#62, count#63] +Input [4]: [ss_list_price#3, sum#46, count#47, count#48] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56), partial_count(distinct ss_list_price#56)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] -Results [4]: [sum#61, count#62, count#63, count#66] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] +Results [4]: [sum#46, count#47, count#48, count#51] (43) Exchange -Input [4]: [sum#61, count#62, count#63, count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] +Input [4]: [sum#46, count#47, count#48, count#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] (44) HashAggregate [codegen id : 11] -Input [4]: [sum#61, count#62, count#63, count#66] +Input [4]: [sum#46, count#47, count#48, count#51] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#56)), count(ss_list_price#56), count(distinct ss_list_price#56)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#56))#59 / 100.0) as decimal(11,6)) AS B4_LP#68, count(ss_list_price#56)#60 AS B4_CNT#69, count(ss_list_price#56)#65 AS B4_CNTD#70] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#44 / 100.0) as decimal(11,6)) AS B4_LP#53, count(ss_list_price#3)#45 AS B4_CNT#54, count(ss_list_price#3)#50 AS B4_CNTD#55] (45) BroadcastExchange -Input [3]: [B4_LP#68, B4_CNT#69, B4_CNTD#70] -Arguments: IdentityBroadcastMode, [id=#71] +Input [3]: [B4_LP#53, B4_CNT#54, B4_CNTD#55] +Arguments: IdentityBroadcastMode, [id=#56] (46) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (47) Scan parquet default.store_sales -Output [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] ReadSchema: struct (48) ColumnarToRow [codegen id : 12] -Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (49) Filter [codegen id : 12] -Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] -Condition : (((isnotnull(ss_quantity#72) AND (ss_quantity#72 >= 21)) AND (ss_quantity#72 <= 25)) AND ((((ss_list_price#74 >= 122.00) AND (ss_list_price#74 <= 132.00)) OR ((ss_coupon_amt#75 >= 836.00) AND (ss_coupon_amt#75 <= 1836.00))) OR ((ss_wholesale_cost#73 >= 17.00) AND (ss_wholesale_cost#73 <= 37.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 21)) AND (ss_quantity#1 <= 25)) AND ((((ss_list_price#3 >= 122.00) AND (ss_list_price#3 <= 132.00)) OR ((ss_coupon_amt#4 >= 836.00) AND (ss_coupon_amt#4 <= 1836.00))) OR ((ss_wholesale_cost#2 >= 17.00) AND (ss_wholesale_cost#2 <= 37.00)))) (50) Project [codegen id : 12] -Output [1]: [ss_list_price#74] -Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (51) HashAggregate [codegen id : 12] -Input [1]: [ss_list_price#74] -Keys [1]: [ss_list_price#74] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#74)), partial_count(ss_list_price#74)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] -Results [4]: [ss_list_price#74, sum#79, count#80, count#81] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] +Results [4]: [ss_list_price#3, sum#59, count#60, count#61] (52) Exchange -Input [4]: [ss_list_price#74, sum#79, count#80, count#81] -Arguments: hashpartitioning(ss_list_price#74, 5), ENSURE_REQUIREMENTS, [id=#82] +Input [4]: [ss_list_price#3, sum#59, count#60, count#61] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#62] (53) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#74, sum#79, count#80, count#81] -Keys [1]: [ss_list_price#74] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] -Results [4]: [ss_list_price#74, sum#79, count#80, count#81] +Input [4]: [ss_list_price#3, sum#59, count#60, count#61] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] +Results [4]: [ss_list_price#3, sum#59, count#60, count#61] (54) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#74, sum#79, count#80, count#81] +Input [4]: [ss_list_price#3, sum#59, count#60, count#61] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74), partial_count(distinct ss_list_price#74)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] -Results [4]: [sum#79, count#80, count#81, count#84] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] +Results [4]: [sum#59, count#60, count#61, count#64] (55) Exchange -Input [4]: [sum#79, count#80, count#81, count#84] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#85] +Input [4]: [sum#59, count#60, count#61, count#64] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] (56) HashAggregate [codegen id : 14] -Input [4]: [sum#79, count#80, count#81, count#84] +Input [4]: [sum#59, count#60, count#61, count#64] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#74)), count(ss_list_price#74), count(distinct ss_list_price#74)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#74))#77 / 100.0) as decimal(11,6)) AS B5_LP#86, count(ss_list_price#74)#78 AS B5_CNT#87, count(ss_list_price#74)#83 AS B5_CNTD#88] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#57 / 100.0) as decimal(11,6)) AS B5_LP#66, count(ss_list_price#3)#58 AS B5_CNT#67, count(ss_list_price#3)#63 AS B5_CNTD#68] (57) BroadcastExchange -Input [3]: [B5_LP#86, B5_CNT#87, B5_CNTD#88] -Arguments: IdentityBroadcastMode, [id=#89] +Input [3]: [B5_LP#66, B5_CNT#67, B5_CNTD#68] +Arguments: IdentityBroadcastMode, [id=#69] (58) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (59) Scan parquet default.store_sales -Output [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] ReadSchema: struct (60) ColumnarToRow [codegen id : 15] -Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (61) Filter [codegen id : 15] -Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] -Condition : (((isnotnull(ss_quantity#90) AND (ss_quantity#90 >= 26)) AND (ss_quantity#90 <= 30)) AND ((((ss_list_price#92 >= 154.00) AND (ss_list_price#92 <= 164.00)) OR ((ss_coupon_amt#93 >= 7326.00) AND (ss_coupon_amt#93 <= 8326.00))) OR ((ss_wholesale_cost#91 >= 7.00) AND (ss_wholesale_cost#91 <= 27.00)))) +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 26)) AND (ss_quantity#1 <= 30)) AND ((((ss_list_price#3 >= 154.00) AND (ss_list_price#3 <= 164.00)) OR ((ss_coupon_amt#4 >= 7326.00) AND (ss_coupon_amt#4 <= 8326.00))) OR ((ss_wholesale_cost#2 >= 7.00) AND (ss_wholesale_cost#2 <= 27.00)))) (62) Project [codegen id : 15] -Output [1]: [ss_list_price#92] -Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Output [1]: [ss_list_price#3] +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] (63) HashAggregate [codegen id : 15] -Input [1]: [ss_list_price#92] -Keys [1]: [ss_list_price#92] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#92)), partial_count(ss_list_price#92)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] -Results [4]: [ss_list_price#92, sum#97, count#98, count#99] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] +Results [4]: [ss_list_price#3, sum#72, count#73, count#74] (64) Exchange -Input [4]: [ss_list_price#92, sum#97, count#98, count#99] -Arguments: hashpartitioning(ss_list_price#92, 5), ENSURE_REQUIREMENTS, [id=#100] +Input [4]: [ss_list_price#3, sum#72, count#73, count#74] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#75] (65) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#92, sum#97, count#98, count#99] -Keys [1]: [ss_list_price#92] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] -Results [4]: [ss_list_price#92, sum#97, count#98, count#99] +Input [4]: [ss_list_price#3, sum#72, count#73, count#74] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] +Results [4]: [ss_list_price#3, sum#72, count#73, count#74] (66) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#92, sum#97, count#98, count#99] +Input [4]: [ss_list_price#3, sum#72, count#73, count#74] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92), partial_count(distinct ss_list_price#92)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] -Results [4]: [sum#97, count#98, count#99, count#102] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] +Results [4]: [sum#72, count#73, count#74, count#77] (67) Exchange -Input [4]: [sum#97, count#98, count#99, count#102] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103] +Input [4]: [sum#72, count#73, count#74, count#77] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] (68) HashAggregate [codegen id : 17] -Input [4]: [sum#97, count#98, count#99, count#102] +Input [4]: [sum#72, count#73, count#74, count#77] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#92)), count(ss_list_price#92), count(distinct ss_list_price#92)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#92))#95 / 100.0) as decimal(11,6)) AS B6_LP#104, count(ss_list_price#92)#96 AS B6_CNT#105, count(ss_list_price#92)#101 AS B6_CNTD#106] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#70 / 100.0) as decimal(11,6)) AS B6_LP#79, count(ss_list_price#3)#71 AS B6_CNT#80, count(ss_list_price#3)#76 AS B6_CNTD#81] (69) BroadcastExchange -Input [3]: [B6_LP#104, B6_CNT#105, B6_CNTD#106] -Arguments: IdentityBroadcastMode, [id=#107] +Input [3]: [B6_LP#79, B6_CNT#80, B6_CNTD#81] +Arguments: IdentityBroadcastMode, [id=#82] (70) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None 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 9e84e385be302..ca09ec6ea1a42 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 @@ -168,170 +168,170 @@ Input [3]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19] Condition : isnotnull(ctr_total_return#19) (24) Scan parquet default.web_returns -Output [4]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23] +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#23), dynamicpruningexpression(wr_returned_date_sk#23 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 (25) ColumnarToRow [codegen id : 8] -Input [4]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23] +Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] (26) Filter [codegen id : 8] -Input [4]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23] -Condition : isnotnull(wr_returning_addr_sk#21) +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) (27) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#24] +Output [1]: [d_date_sk#6] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#24 as bigint)] +Left keys [1]: [wr_returned_date_sk#4] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (29) Project [codegen id : 8] -Output [3]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22] -Input [5]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23, d_date_sk#24] +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] (30) Exchange -Input [3]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22] -Arguments: hashpartitioning(wr_returning_addr_sk#21, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] +Arguments: hashpartitioning(wr_returning_addr_sk#2, 5), ENSURE_REQUIREMENTS, [id=#20] (31) Sort [codegen id : 9] -Input [3]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22] -Arguments: [wr_returning_addr_sk#21 ASC NULLS FIRST], false, 0 +Input [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] +Arguments: [wr_returning_addr_sk#2 ASC NULLS FIRST], false, 0 (32) ReusedExchange [Reuses operator id: 16] -Output [2]: [ca_address_sk#26, ca_state#27] +Output [2]: [ca_address_sk#10, ca_state#11] (33) Sort [codegen id : 11] -Input [2]: [ca_address_sk#26, ca_state#27] -Arguments: [cast(ca_address_sk#26 as bigint) ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#10, ca_state#11] +Arguments: [cast(ca_address_sk#10 as bigint) ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 12] -Left keys [1]: [wr_returning_addr_sk#21] -Right keys [1]: [cast(ca_address_sk#26 as bigint)] +Left keys [1]: [wr_returning_addr_sk#2] +Right keys [1]: [cast(ca_address_sk#10 as bigint)] Join condition: None (35) Project [codegen id : 12] -Output [3]: [wr_returning_customer_sk#20, wr_return_amt#22, ca_state#27] -Input [5]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, ca_address_sk#26, ca_state#27] +Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#11] +Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#10, ca_state#11] (36) HashAggregate [codegen id : 12] -Input [3]: [wr_returning_customer_sk#20, wr_return_amt#22, ca_state#27] -Keys [2]: [wr_returning_customer_sk#20, ca_state#27] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#22))] -Aggregate Attributes [1]: [sum#28] -Results [3]: [wr_returning_customer_sk#20, ca_state#27, sum#29] +Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#11] +Keys [2]: [wr_returning_customer_sk#1, ca_state#11] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] +Aggregate Attributes [1]: [sum#21] +Results [3]: [wr_returning_customer_sk#1, ca_state#11, sum#22] (37) Exchange -Input [3]: [wr_returning_customer_sk#20, ca_state#27, sum#29] -Arguments: hashpartitioning(wr_returning_customer_sk#20, ca_state#27, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [wr_returning_customer_sk#1, ca_state#11, sum#22] +Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#11, 5), ENSURE_REQUIREMENTS, [id=#23] (38) HashAggregate [codegen id : 13] -Input [3]: [wr_returning_customer_sk#20, ca_state#27, sum#29] -Keys [2]: [wr_returning_customer_sk#20, ca_state#27] -Functions [1]: [sum(UnscaledValue(wr_return_amt#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#22))#31] -Results [2]: [ca_state#27 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(wr_return_amt#22))#31,17,2) AS ctr_total_return#19] +Input [3]: [wr_returning_customer_sk#1, ca_state#11, sum#22] +Keys [2]: [wr_returning_customer_sk#1, ca_state#11] +Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#24] +Results [2]: [ca_state#11 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#24,17,2) AS ctr_total_return#19] (39) HashAggregate [codegen id : 13] Input [2]: [ctr_state#18, ctr_total_return#19] Keys [1]: [ctr_state#18] Functions [1]: [partial_avg(ctr_total_return#19)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [3]: [ctr_state#18, sum#34, count#35] +Aggregate Attributes [2]: [sum#25, count#26] +Results [3]: [ctr_state#18, sum#27, count#28] (40) Exchange -Input [3]: [ctr_state#18, sum#34, count#35] -Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ctr_state#18, sum#27, count#28] +Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#29] (41) HashAggregate [codegen id : 14] -Input [3]: [ctr_state#18, sum#34, count#35] +Input [3]: [ctr_state#18, sum#27, count#28] Keys [1]: [ctr_state#18] Functions [1]: [avg(ctr_total_return#19)] -Aggregate Attributes [1]: [avg(ctr_total_return#19)#37] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#37) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#38, ctr_state#18 AS ctr_state#18#39] +Aggregate Attributes [1]: [avg(ctr_total_return#19)#30] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#18 AS ctr_state#18#32] (42) Filter [codegen id : 14] -Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#38) +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) (43) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#40] +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#33] (44) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ctr_state#18] -Right keys [1]: [ctr_state#18#39] -Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#38) +Right keys [1]: [ctr_state#18#32] +Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) (45) Project [codegen id : 17] Output [2]: [ctr_customer_sk#17, ctr_total_return#19] -Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] +Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] (46) Scan parquet default.customer -Output [14]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] +Output [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 16] -Input [14]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] +Input [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] (48) Filter [codegen id : 16] -Input [14]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] -Condition : (isnotnull(c_customer_sk#41) AND isnotnull(c_current_addr_sk#43)) +Input [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Condition : (isnotnull(c_customer_sk#34) AND isnotnull(c_current_addr_sk#36)) (49) Scan parquet default.customer_address -Output [2]: [ca_address_sk#55, ca_state#56] +Output [2]: [ca_address_sk#10, ca_state#11] 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 (50) ColumnarToRow [codegen id : 15] -Input [2]: [ca_address_sk#55, ca_state#56] +Input [2]: [ca_address_sk#10, ca_state#11] (51) Filter [codegen id : 15] -Input [2]: [ca_address_sk#55, ca_state#56] -Condition : ((isnotnull(ca_state#56) AND (ca_state#56 = GA)) AND isnotnull(ca_address_sk#55)) +Input [2]: [ca_address_sk#10, ca_state#11] +Condition : ((isnotnull(ca_state#11) AND (ca_state#11 = GA)) AND isnotnull(ca_address_sk#10)) (52) Project [codegen id : 15] -Output [1]: [ca_address_sk#55] -Input [2]: [ca_address_sk#55, ca_state#56] +Output [1]: [ca_address_sk#10] +Input [2]: [ca_address_sk#10, ca_state#11] (53) BroadcastExchange -Input [1]: [ca_address_sk#55] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#57] +Input [1]: [ca_address_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] (54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_addr_sk#43] -Right keys [1]: [ca_address_sk#55] +Left keys [1]: [c_current_addr_sk#36] +Right keys [1]: [ca_address_sk#10] Join condition: None (55) Project [codegen id : 16] -Output [13]: [c_customer_sk#41, c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] -Input [15]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ca_address_sk#55] +Output [13]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Input [15]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ca_address_sk#10] (56) BroadcastExchange -Input [13]: [c_customer_sk#41, c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] +Input [13]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] (57) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ctr_customer_sk#17] -Right keys [1]: [cast(c_customer_sk#41 as bigint)] +Right keys [1]: [cast(c_customer_sk#34 as bigint)] Join condition: None (58) Project [codegen id : 17] -Output [13]: [c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ctr_total_return#19] -Input [15]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#41, c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] +Output [13]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#19] +Input [15]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] (59) TakeOrderedAndProject -Input [13]: [c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ctr_total_return#19] -Arguments: 100, [c_customer_id#42 ASC NULLS FIRST, c_salutation#44 ASC NULLS FIRST, c_first_name#45 ASC NULLS FIRST, c_last_name#46 ASC NULLS FIRST, c_preferred_cust_flag#47 ASC NULLS FIRST, c_birth_day#48 ASC NULLS FIRST, c_birth_month#49 ASC NULLS FIRST, c_birth_year#50 ASC NULLS FIRST, c_birth_country#51 ASC NULLS FIRST, c_login#52 ASC NULLS FIRST, c_email_address#53 ASC NULLS FIRST, c_last_review_date#54 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ctr_total_return#19] +Input [13]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#19] +Arguments: 100, [c_customer_id#35 ASC NULLS FIRST, c_salutation#37 ASC NULLS FIRST, c_first_name#38 ASC NULLS FIRST, c_last_name#39 ASC NULLS FIRST, c_preferred_cust_flag#40 ASC NULLS FIRST, c_birth_day#41 ASC NULLS FIRST, c_birth_month#42 ASC NULLS FIRST, c_birth_year#43 ASC NULLS FIRST, c_birth_country#44 ASC NULLS FIRST, c_login#45 ASC NULLS FIRST, c_email_address#46 ASC NULLS FIRST, c_last_review_date#47 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#19] ===== Subqueries ===== @@ -342,6 +342,6 @@ ReusedExchange (60) (60) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 24 Hosting Expression = wr_returned_date_sk#23 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 24 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/q30/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt index 1aaebad9f1920..57b85bc9db340 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 @@ -150,158 +150,158 @@ Input [3]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18] Condition : isnotnull(ctr_total_return#18) (21) Scan parquet default.web_returns -Output [4]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, wr_returned_date_sk#22] +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#22), dynamicpruningexpression(wr_returned_date_sk#22 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 (22) ColumnarToRow [codegen id : 6] -Input [4]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, wr_returned_date_sk#22] +Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] (23) Filter [codegen id : 6] -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) +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) (24) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#23] +Output [1]: [d_date_sk#6] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returned_date_sk#22] -Right keys [1]: [cast(d_date_sk#23 as bigint)] +Left keys [1]: [wr_returned_date_sk#4] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (26) Project [codegen id : 6] -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#23] +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] (27) ReusedExchange [Reuses operator id: 14] -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#9, ca_state#10] (28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returning_addr_sk#20] -Right keys [1]: [cast(ca_address_sk#24 as bigint)] +Left keys [1]: [wr_returning_addr_sk#2] +Right keys [1]: [cast(ca_address_sk#9 as bigint)] Join condition: None (29) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#19, wr_return_amt#21, ca_state#25] -Input [5]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, ca_address_sk#24, ca_state#25] +Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] +Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#9, ca_state#10] (30) HashAggregate [codegen id : 6] -Input [3]: [wr_returning_customer_sk#19, wr_return_amt#21, ca_state#25] -Keys [2]: [wr_returning_customer_sk#19, ca_state#25] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#21))] -Aggregate Attributes [1]: [sum#26] -Results [3]: [wr_returning_customer_sk#19, ca_state#25, sum#27] +Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] +Keys [2]: [wr_returning_customer_sk#1, ca_state#10] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] +Aggregate Attributes [1]: [sum#19] +Results [3]: [wr_returning_customer_sk#1, ca_state#10, sum#20] (31) Exchange -Input [3]: [wr_returning_customer_sk#19, ca_state#25, sum#27] -Arguments: hashpartitioning(wr_returning_customer_sk#19, ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#20] +Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#21] (32) HashAggregate [codegen id : 7] -Input [3]: [wr_returning_customer_sk#19, ca_state#25, sum#27] -Keys [2]: [wr_returning_customer_sk#19, ca_state#25] -Functions [1]: [sum(UnscaledValue(wr_return_amt#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#21))#29] -Results [2]: [ca_state#25 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(wr_return_amt#21))#29,17,2) AS ctr_total_return#18] +Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#20] +Keys [2]: [wr_returning_customer_sk#1, ca_state#10] +Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#22] +Results [2]: [ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#22,17,2) AS ctr_total_return#18] (33) HashAggregate [codegen id : 7] Input [2]: [ctr_state#17, ctr_total_return#18] Keys [1]: [ctr_state#17] Functions [1]: [partial_avg(ctr_total_return#18)] -Aggregate Attributes [2]: [sum#30, count#31] -Results [3]: [ctr_state#17, sum#32, count#33] +Aggregate Attributes [2]: [sum#23, count#24] +Results [3]: [ctr_state#17, sum#25, count#26] (34) Exchange -Input [3]: [ctr_state#17, sum#32, count#33] -Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [ctr_state#17, sum#25, count#26] +Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#27] (35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#17, sum#32, count#33] +Input [3]: [ctr_state#17, sum#25, count#26] Keys [1]: [ctr_state#17] Functions [1]: [avg(ctr_total_return#18)] -Aggregate Attributes [1]: [avg(ctr_total_return#18)#35] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#35) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#36, ctr_state#17 AS ctr_state#17#37] +Aggregate Attributes [1]: [avg(ctr_total_return#18)#28] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#28) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#29, ctr_state#17 AS ctr_state#17#30] (36) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#36) +Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) (37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#38] +Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#31] (38) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#17] -Right keys [1]: [ctr_state#17#37] -Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#36) +Right keys [1]: [ctr_state#17#30] +Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) (39) Project [codegen id : 11] Output [2]: [ctr_customer_sk#16, ctr_total_return#18] -Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] +Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] (40) Scan parquet default.customer -Output [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] +Output [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] +Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] (42) Filter [codegen id : 9] -Input [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] -Condition : (isnotnull(c_customer_sk#39) AND isnotnull(c_current_addr_sk#41)) +Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34)) (43) BroadcastExchange -Input [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] +Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#46] (44) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#16] -Right keys [1]: [cast(c_customer_sk#39 as bigint)] +Right keys [1]: [cast(c_customer_sk#32 as bigint)] Join condition: None (45) Project [codegen id : 11] -Output [14]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] -Input [16]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] +Output [14]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +Input [16]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] (46) Scan parquet default.customer_address -Output [2]: [ca_address_sk#54, ca_state#55] +Output [2]: [ca_address_sk#9, ca_state#10] 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 (47) ColumnarToRow [codegen id : 10] -Input [2]: [ca_address_sk#54, ca_state#55] +Input [2]: [ca_address_sk#9, ca_state#10] (48) Filter [codegen id : 10] -Input [2]: [ca_address_sk#54, ca_state#55] -Condition : ((isnotnull(ca_state#55) AND (ca_state#55 = GA)) AND isnotnull(ca_address_sk#54)) +Input [2]: [ca_address_sk#9, ca_state#10] +Condition : ((isnotnull(ca_state#10) AND (ca_state#10 = GA)) AND isnotnull(ca_address_sk#9)) (49) Project [codegen id : 10] -Output [1]: [ca_address_sk#54] -Input [2]: [ca_address_sk#54, ca_state#55] +Output [1]: [ca_address_sk#9] +Input [2]: [ca_address_sk#9, ca_state#10] (50) BroadcastExchange -Input [1]: [ca_address_sk#54] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +Input [1]: [ca_address_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] (51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#41] -Right keys [1]: [ca_address_sk#54] +Left keys [1]: [c_current_addr_sk#34] +Right keys [1]: [ca_address_sk#9] Join condition: None (52) Project [codegen id : 11] -Output [13]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ctr_total_return#18] -Input [15]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ca_address_sk#54] +Output [13]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#18] +Input [15]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ca_address_sk#9] (53) TakeOrderedAndProject -Input [13]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ctr_total_return#18] -Arguments: 100, [c_customer_id#40 ASC NULLS FIRST, c_salutation#42 ASC NULLS FIRST, c_first_name#43 ASC NULLS FIRST, c_last_name#44 ASC NULLS FIRST, c_preferred_cust_flag#45 ASC NULLS FIRST, c_birth_day#46 ASC NULLS FIRST, c_birth_month#47 ASC NULLS FIRST, c_birth_year#48 ASC NULLS FIRST, c_birth_country#49 ASC NULLS FIRST, c_login#50 ASC NULLS FIRST, c_email_address#51 ASC NULLS FIRST, c_last_review_date#52 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ctr_total_return#18] +Input [13]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#18] +Arguments: 100, [c_customer_id#33 ASC NULLS FIRST, c_salutation#35 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, c_last_name#37 ASC NULLS FIRST, c_preferred_cust_flag#38 ASC NULLS FIRST, c_birth_day#39 ASC NULLS FIRST, c_birth_month#40 ASC NULLS FIRST, c_birth_year#41 ASC NULLS FIRST, c_birth_country#42 ASC NULLS FIRST, c_login#43 ASC NULLS FIRST, c_email_address#44 ASC NULLS FIRST, c_last_review_date#45 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#18] ===== Subqueries ===== @@ -312,6 +312,6 @@ ReusedExchange (54) (54) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 21 Hosting Expression = wr_returned_date_sk#22 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 21 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 1703da84e7914..61aa5ea5e7e5c 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 @@ -220,452 +220,452 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#16] Results [2]: [ca_county#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#16,17,2) AS store_sales#17] (22) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20] +Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#18)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 8] -Input [3]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20] +Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] (24) Filter [codegen id : 8] -Input [3]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20] -Condition : isnotnull(ss_addr_sk#18) +Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_addr_sk#1) (25) Scan parquet default.date_dim -Output [3]: [d_date_sk#22, d_year#23, d_qoy#24] +Output [3]: [d_date_sk#19, d_year#20, d_qoy#21] 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 (26) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#22, d_year#23, d_qoy#24] +Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] (27) Filter [codegen id : 7] -Input [3]: [d_date_sk#22, d_year#23, d_qoy#24] -Condition : ((((isnotnull(d_qoy#24) AND isnotnull(d_year#23)) AND (d_qoy#24 = 3)) AND (d_year#23 = 2000)) AND isnotnull(d_date_sk#22)) +Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] +Condition : ((((isnotnull(d_qoy#21) AND isnotnull(d_year#20)) AND (d_qoy#21 = 3)) AND (d_year#20 = 2000)) AND isnotnull(d_date_sk#19)) (28) BroadcastExchange -Input [3]: [d_date_sk#22, d_year#23, d_qoy#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] +Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] (29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#20] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#19] Join condition: None (30) Project [codegen id : 8] -Output [4]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24] -Input [6]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23, d_qoy#24] +Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21] +Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#19, d_year#20, d_qoy#21] (31) Exchange -Input [4]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24] -Arguments: hashpartitioning(ss_addr_sk#18, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21] +Arguments: hashpartitioning(ss_addr_sk#1, 5), ENSURE_REQUIREMENTS, [id=#23] (32) Sort [codegen id : 9] -Input [4]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24] -Arguments: [ss_addr_sk#18 ASC NULLS FIRST], false, 0 +Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21] +Arguments: [ss_addr_sk#1 ASC NULLS FIRST], false, 0 (33) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#27, ca_county#28] +Output [2]: [ca_address_sk#24, ca_county#25] (34) Sort [codegen id : 11] -Input [2]: [ca_address_sk#27, ca_county#28] -Arguments: [ca_address_sk#27 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#24, ca_county#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 (35) SortMergeJoin [codegen id : 12] -Left keys [1]: [ss_addr_sk#18] -Right keys [1]: [ca_address_sk#27] +Left keys [1]: [ss_addr_sk#1] +Right keys [1]: [ca_address_sk#24] Join condition: None (36) Project [codegen id : 12] -Output [4]: [ss_ext_sales_price#19, d_year#23, d_qoy#24, ca_county#28] -Input [6]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24, ca_address_sk#27, ca_county#28] +Output [4]: [ss_ext_sales_price#2, d_year#20, d_qoy#21, ca_county#25] +Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21, ca_address_sk#24, ca_county#25] (37) HashAggregate [codegen id : 12] -Input [4]: [ss_ext_sales_price#19, d_year#23, d_qoy#24, ca_county#28] -Keys [3]: [ca_county#28, d_qoy#24, d_year#23] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#19))] -Aggregate Attributes [1]: [sum#29] -Results [4]: [ca_county#28, d_qoy#24, d_year#23, sum#30] +Input [4]: [ss_ext_sales_price#2, d_year#20, d_qoy#21, ca_county#25] +Keys [3]: [ca_county#25, d_qoy#21, d_year#20] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#26] +Results [4]: [ca_county#25, d_qoy#21, d_year#20, sum#27] (38) Exchange -Input [4]: [ca_county#28, d_qoy#24, d_year#23, sum#30] -Arguments: hashpartitioning(ca_county#28, d_qoy#24, d_year#23, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [4]: [ca_county#25, d_qoy#21, d_year#20, sum#27] +Arguments: hashpartitioning(ca_county#25, d_qoy#21, d_year#20, 5), ENSURE_REQUIREMENTS, [id=#28] (39) HashAggregate [codegen id : 13] -Input [4]: [ca_county#28, d_qoy#24, d_year#23, sum#30] -Keys [3]: [ca_county#28, d_qoy#24, d_year#23] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#19))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#19))#32] -Results [2]: [ca_county#28, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#19))#32,17,2) AS store_sales#33] +Input [4]: [ca_county#25, d_qoy#21, d_year#20, sum#27] +Keys [3]: [ca_county#25, d_qoy#21, d_year#20] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#29] +Results [2]: [ca_county#25, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#29,17,2) AS store_sales#30] (40) BroadcastExchange -Input [2]: [ca_county#28, store_sales#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] +Input [2]: [ca_county#25, store_sales#30] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#31] (41) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ca_county#11] -Right keys [1]: [ca_county#28] +Right keys [1]: [ca_county#25] Join condition: None (42) Project [codegen id : 42] -Output [3]: [ca_county#11, store_sales#17, store_sales#33] -Input [4]: [ca_county#11, store_sales#17, ca_county#28, store_sales#33] +Output [3]: [ca_county#11, store_sales#17, store_sales#30] +Input [4]: [ca_county#11, store_sales#17, ca_county#25, store_sales#30] (43) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37] +Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#37), dynamicpruningexpression(ss_sold_date_sk#37 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 15] -Input [3]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37] +Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] (45) Filter [codegen id : 15] -Input [3]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37] -Condition : isnotnull(ss_addr_sk#35) +Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_addr_sk#1) (46) Scan parquet default.date_dim -Output [3]: [d_date_sk#39, d_year#40, d_qoy#41] +Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] 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 (47) ColumnarToRow [codegen id : 14] -Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] +Input [3]: [d_date_sk#33, d_year#34, d_qoy#35] (48) Filter [codegen id : 14] -Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] -Condition : ((((isnotnull(d_qoy#41) AND isnotnull(d_year#40)) AND (d_qoy#41 = 1)) AND (d_year#40 = 2000)) AND isnotnull(d_date_sk#39)) +Input [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Condition : ((((isnotnull(d_qoy#35) AND isnotnull(d_year#34)) AND (d_qoy#35 = 1)) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) (49) BroadcastExchange -Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#42] +Input [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] (50) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_sold_date_sk#37] -Right keys [1]: [d_date_sk#39] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#33] Join condition: None (51) Project [codegen id : 15] -Output [4]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41] -Input [6]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37, d_date_sk#39, d_year#40, d_qoy#41] +Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35] +Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#33, d_year#34, d_qoy#35] (52) Exchange -Input [4]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41] -Arguments: hashpartitioning(ss_addr_sk#35, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35] +Arguments: hashpartitioning(ss_addr_sk#1, 5), ENSURE_REQUIREMENTS, [id=#37] (53) Sort [codegen id : 16] -Input [4]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41] -Arguments: [ss_addr_sk#35 ASC NULLS FIRST], false, 0 +Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35] +Arguments: [ss_addr_sk#1 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#44, ca_county#45] +Output [2]: [ca_address_sk#38, ca_county#39] (55) Sort [codegen id : 18] -Input [2]: [ca_address_sk#44, ca_county#45] -Arguments: [ca_address_sk#44 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#38, ca_county#39] +Arguments: [ca_address_sk#38 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 19] -Left keys [1]: [ss_addr_sk#35] -Right keys [1]: [ca_address_sk#44] +Left keys [1]: [ss_addr_sk#1] +Right keys [1]: [ca_address_sk#38] Join condition: None (57) Project [codegen id : 19] -Output [4]: [ss_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#45] -Input [6]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41, ca_address_sk#44, ca_county#45] +Output [4]: [ss_ext_sales_price#2, d_year#34, d_qoy#35, ca_county#39] +Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35, ca_address_sk#38, ca_county#39] (58) HashAggregate [codegen id : 19] -Input [4]: [ss_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#45] -Keys [3]: [ca_county#45, d_qoy#41, d_year#40] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#36))] -Aggregate Attributes [1]: [sum#46] -Results [4]: [ca_county#45, d_qoy#41, d_year#40, sum#47] +Input [4]: [ss_ext_sales_price#2, d_year#34, d_qoy#35, ca_county#39] +Keys [3]: [ca_county#39, d_qoy#35, d_year#34] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#40] +Results [4]: [ca_county#39, d_qoy#35, d_year#34, sum#41] (59) Exchange -Input [4]: [ca_county#45, d_qoy#41, d_year#40, sum#47] -Arguments: hashpartitioning(ca_county#45, d_qoy#41, d_year#40, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [4]: [ca_county#39, d_qoy#35, d_year#34, sum#41] +Arguments: hashpartitioning(ca_county#39, d_qoy#35, d_year#34, 5), ENSURE_REQUIREMENTS, [id=#42] (60) HashAggregate [codegen id : 20] -Input [4]: [ca_county#45, d_qoy#41, d_year#40, sum#47] -Keys [3]: [ca_county#45, d_qoy#41, d_year#40] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#36))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#36))#49] -Results [3]: [ca_county#45, d_year#40, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#36))#49,17,2) AS store_sales#50] +Input [4]: [ca_county#39, d_qoy#35, d_year#34, sum#41] +Keys [3]: [ca_county#39, d_qoy#35, d_year#34] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#43] +Results [3]: [ca_county#39, d_year#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#43,17,2) AS store_sales#44] (61) BroadcastExchange -Input [3]: [ca_county#45, d_year#40, store_sales#50] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#51] +Input [3]: [ca_county#39, d_year#34, store_sales#44] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#45] (62) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ca_county#11] -Right keys [1]: [ca_county#45] +Right keys [1]: [ca_county#39] Join condition: None (63) Project [codegen id : 42] -Output [5]: [store_sales#17, store_sales#33, ca_county#45, d_year#40, store_sales#50] -Input [6]: [ca_county#11, store_sales#17, store_sales#33, ca_county#45, d_year#40, store_sales#50] +Output [5]: [store_sales#17, store_sales#30, ca_county#39, d_year#34, store_sales#44] +Input [6]: [ca_county#11, store_sales#17, store_sales#30, ca_county#39, d_year#34, store_sales#44] (64) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, ws_sold_date_sk#54] +Output [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, ws_sold_date_sk#54] +Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] (66) Filter [codegen id : 22] -Input [3]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, ws_sold_date_sk#54] -Condition : isnotnull(ws_bill_addr_sk#52) +Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Condition : isnotnull(ws_bill_addr_sk#46) (67) ReusedExchange [Reuses operator id: 49] -Output [3]: [d_date_sk#55, d_year#56, d_qoy#57] +Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] (68) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#54] -Right keys [1]: [d_date_sk#55] +Left keys [1]: [ws_sold_date_sk#48] +Right keys [1]: [d_date_sk#49] Join condition: None (69) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57] -Input [6]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, ws_sold_date_sk#54, d_date_sk#55, d_year#56, d_qoy#57] +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] (70) Exchange -Input [4]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57] -Arguments: hashpartitioning(ws_bill_addr_sk#52, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51] +Arguments: hashpartitioning(ws_bill_addr_sk#46, 5), ENSURE_REQUIREMENTS, [id=#52] (71) Sort [codegen id : 23] -Input [4]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57] -Arguments: [ws_bill_addr_sk#52 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51] +Arguments: [ws_bill_addr_sk#46 ASC NULLS FIRST], false, 0 (72) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#59, ca_county#60] +Output [2]: [ca_address_sk#53, ca_county#54] (73) Sort [codegen id : 25] -Input [2]: [ca_address_sk#59, ca_county#60] -Arguments: [ca_address_sk#59 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#53, ca_county#54] +Arguments: [ca_address_sk#53 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 26] -Left keys [1]: [ws_bill_addr_sk#52] -Right keys [1]: [ca_address_sk#59] +Left keys [1]: [ws_bill_addr_sk#46] +Right keys [1]: [ca_address_sk#53] Join condition: None (75) Project [codegen id : 26] -Output [4]: [ws_ext_sales_price#53, d_year#56, d_qoy#57, ca_county#60] -Input [6]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57, ca_address_sk#59, ca_county#60] +Output [4]: [ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_county#54] +Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_address_sk#53, ca_county#54] (76) HashAggregate [codegen id : 26] -Input [4]: [ws_ext_sales_price#53, d_year#56, d_qoy#57, ca_county#60] -Keys [3]: [ca_county#60, d_qoy#57, d_year#56] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#53))] -Aggregate Attributes [1]: [sum#61] -Results [4]: [ca_county#60, d_qoy#57, d_year#56, sum#62] +Input [4]: [ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_county#54] +Keys [3]: [ca_county#54, d_qoy#51, d_year#50] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#47))] +Aggregate Attributes [1]: [sum#55] +Results [4]: [ca_county#54, d_qoy#51, d_year#50, sum#56] (77) Exchange -Input [4]: [ca_county#60, d_qoy#57, d_year#56, sum#62] -Arguments: hashpartitioning(ca_county#60, d_qoy#57, d_year#56, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [4]: [ca_county#54, d_qoy#51, d_year#50, sum#56] +Arguments: hashpartitioning(ca_county#54, d_qoy#51, d_year#50, 5), ENSURE_REQUIREMENTS, [id=#57] (78) HashAggregate [codegen id : 41] -Input [4]: [ca_county#60, d_qoy#57, d_year#56, sum#62] -Keys [3]: [ca_county#60, d_qoy#57, d_year#56] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#53))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#53))#64] -Results [2]: [ca_county#60, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#53))#64,17,2) AS web_sales#65] +Input [4]: [ca_county#54, d_qoy#51, d_year#50, sum#56] +Keys [3]: [ca_county#54, 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))#58] +Results [2]: [ca_county#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#47))#58,17,2) AS web_sales#59] (79) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, ws_sold_date_sk#68] +Output [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (80) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, ws_sold_date_sk#68] +Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] (81) Filter [codegen id : 28] -Input [3]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, ws_sold_date_sk#68] -Condition : isnotnull(ws_bill_addr_sk#66) +Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Condition : isnotnull(ws_bill_addr_sk#46) (82) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#69, d_year#70, d_qoy#71] +Output [3]: [d_date_sk#60, d_year#61, d_qoy#62] (83) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#68] -Right keys [1]: [d_date_sk#69] +Left keys [1]: [ws_sold_date_sk#48] +Right keys [1]: [d_date_sk#60] Join condition: None (84) Project [codegen id : 28] -Output [4]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71] -Input [6]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, ws_sold_date_sk#68, d_date_sk#69, d_year#70, d_qoy#71] +Output [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62] +Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48, d_date_sk#60, d_year#61, d_qoy#62] (85) Exchange -Input [4]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71] -Arguments: hashpartitioning(ws_bill_addr_sk#66, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62] +Arguments: hashpartitioning(ws_bill_addr_sk#46, 5), ENSURE_REQUIREMENTS, [id=#63] (86) Sort [codegen id : 29] -Input [4]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71] -Arguments: [ws_bill_addr_sk#66 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62] +Arguments: [ws_bill_addr_sk#46 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#73, ca_county#74] +Output [2]: [ca_address_sk#64, ca_county#65] (88) Sort [codegen id : 31] -Input [2]: [ca_address_sk#73, ca_county#74] -Arguments: [ca_address_sk#73 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#64, ca_county#65] +Arguments: [ca_address_sk#64 ASC NULLS FIRST], false, 0 (89) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_addr_sk#66] -Right keys [1]: [ca_address_sk#73] +Left keys [1]: [ws_bill_addr_sk#46] +Right keys [1]: [ca_address_sk#64] Join condition: None (90) Project [codegen id : 32] -Output [4]: [ws_ext_sales_price#67, d_year#70, d_qoy#71, ca_county#74] -Input [6]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71, ca_address_sk#73, ca_county#74] +Output [4]: [ws_ext_sales_price#47, d_year#61, d_qoy#62, ca_county#65] +Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62, ca_address_sk#64, ca_county#65] (91) HashAggregate [codegen id : 32] -Input [4]: [ws_ext_sales_price#67, d_year#70, d_qoy#71, ca_county#74] -Keys [3]: [ca_county#74, d_qoy#71, d_year#70] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#67))] -Aggregate Attributes [1]: [sum#75] -Results [4]: [ca_county#74, d_qoy#71, d_year#70, sum#76] +Input [4]: [ws_ext_sales_price#47, d_year#61, d_qoy#62, ca_county#65] +Keys [3]: [ca_county#65, d_qoy#62, d_year#61] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#47))] +Aggregate Attributes [1]: [sum#66] +Results [4]: [ca_county#65, d_qoy#62, d_year#61, sum#67] (92) Exchange -Input [4]: [ca_county#74, d_qoy#71, d_year#70, sum#76] -Arguments: hashpartitioning(ca_county#74, d_qoy#71, d_year#70, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [4]: [ca_county#65, d_qoy#62, d_year#61, sum#67] +Arguments: hashpartitioning(ca_county#65, d_qoy#62, d_year#61, 5), ENSURE_REQUIREMENTS, [id=#68] (93) HashAggregate [codegen id : 33] -Input [4]: [ca_county#74, d_qoy#71, d_year#70, sum#76] -Keys [3]: [ca_county#74, d_qoy#71, d_year#70] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#67))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#67))#78] -Results [2]: [ca_county#74, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#67))#78,17,2) AS web_sales#79] +Input [4]: [ca_county#65, d_qoy#62, d_year#61, sum#67] +Keys [3]: [ca_county#65, d_qoy#62, d_year#61] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#47))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#47))#69] +Results [2]: [ca_county#65, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#47))#69,17,2) AS web_sales#70] (94) BroadcastExchange -Input [2]: [ca_county#74, web_sales#79] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#80] +Input [2]: [ca_county#65, web_sales#70] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#71] (95) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#60] -Right keys [1]: [ca_county#74] +Left keys [1]: [ca_county#54] +Right keys [1]: [ca_county#65] Join condition: None (96) Project [codegen id : 41] -Output [3]: [ca_county#60, web_sales#65, web_sales#79] -Input [4]: [ca_county#60, web_sales#65, ca_county#74, web_sales#79] +Output [3]: [ca_county#54, web_sales#59, web_sales#70] +Input [4]: [ca_county#54, web_sales#59, ca_county#65, web_sales#70] (97) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, ws_sold_date_sk#83] +Output [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#18)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (98) ColumnarToRow [codegen id : 35] -Input [3]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, ws_sold_date_sk#83] +Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] (99) Filter [codegen id : 35] -Input [3]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, ws_sold_date_sk#83] -Condition : isnotnull(ws_bill_addr_sk#81) +Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Condition : isnotnull(ws_bill_addr_sk#46) (100) ReusedExchange [Reuses operator id: 28] -Output [3]: [d_date_sk#84, d_year#85, d_qoy#86] +Output [3]: [d_date_sk#72, d_year#73, d_qoy#74] (101) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_sold_date_sk#83] -Right keys [1]: [d_date_sk#84] +Left keys [1]: [ws_sold_date_sk#48] +Right keys [1]: [d_date_sk#72] Join condition: None (102) Project [codegen id : 35] -Output [4]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86] -Input [6]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, ws_sold_date_sk#83, d_date_sk#84, d_year#85, d_qoy#86] +Output [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74] +Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48, d_date_sk#72, d_year#73, d_qoy#74] (103) Exchange -Input [4]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86] -Arguments: hashpartitioning(ws_bill_addr_sk#81, 5), ENSURE_REQUIREMENTS, [id=#87] +Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74] +Arguments: hashpartitioning(ws_bill_addr_sk#46, 5), ENSURE_REQUIREMENTS, [id=#75] (104) Sort [codegen id : 36] -Input [4]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86] -Arguments: [ws_bill_addr_sk#81 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74] +Arguments: [ws_bill_addr_sk#46 ASC NULLS FIRST], false, 0 (105) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#88, ca_county#89] +Output [2]: [ca_address_sk#76, ca_county#77] (106) Sort [codegen id : 38] -Input [2]: [ca_address_sk#88, ca_county#89] -Arguments: [ca_address_sk#88 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#76, ca_county#77] +Arguments: [ca_address_sk#76 ASC NULLS FIRST], false, 0 (107) SortMergeJoin [codegen id : 39] -Left keys [1]: [ws_bill_addr_sk#81] -Right keys [1]: [ca_address_sk#88] +Left keys [1]: [ws_bill_addr_sk#46] +Right keys [1]: [ca_address_sk#76] Join condition: None (108) Project [codegen id : 39] -Output [4]: [ws_ext_sales_price#82, d_year#85, d_qoy#86, ca_county#89] -Input [6]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86, ca_address_sk#88, ca_county#89] +Output [4]: [ws_ext_sales_price#47, d_year#73, d_qoy#74, ca_county#77] +Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74, ca_address_sk#76, ca_county#77] (109) HashAggregate [codegen id : 39] -Input [4]: [ws_ext_sales_price#82, d_year#85, d_qoy#86, ca_county#89] -Keys [3]: [ca_county#89, d_qoy#86, d_year#85] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#82))] -Aggregate Attributes [1]: [sum#90] -Results [4]: [ca_county#89, d_qoy#86, d_year#85, sum#91] +Input [4]: [ws_ext_sales_price#47, d_year#73, d_qoy#74, ca_county#77] +Keys [3]: [ca_county#77, d_qoy#74, d_year#73] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#47))] +Aggregate Attributes [1]: [sum#78] +Results [4]: [ca_county#77, d_qoy#74, d_year#73, sum#79] (110) Exchange -Input [4]: [ca_county#89, d_qoy#86, d_year#85, sum#91] -Arguments: hashpartitioning(ca_county#89, d_qoy#86, d_year#85, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [4]: [ca_county#77, d_qoy#74, d_year#73, sum#79] +Arguments: hashpartitioning(ca_county#77, d_qoy#74, d_year#73, 5), ENSURE_REQUIREMENTS, [id=#80] (111) HashAggregate [codegen id : 40] -Input [4]: [ca_county#89, d_qoy#86, d_year#85, sum#91] -Keys [3]: [ca_county#89, d_qoy#86, d_year#85] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#82))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#82))#93] -Results [2]: [ca_county#89, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#82))#93,17,2) AS web_sales#94] +Input [4]: [ca_county#77, d_qoy#74, d_year#73, sum#79] +Keys [3]: [ca_county#77, d_qoy#74, d_year#73] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#47))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#47))#81] +Results [2]: [ca_county#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#47))#81,17,2) AS web_sales#82] (112) BroadcastExchange -Input [2]: [ca_county#89, web_sales#94] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#95] +Input [2]: [ca_county#77, web_sales#82] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#83] (113) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#60] -Right keys [1]: [ca_county#89] +Left keys [1]: [ca_county#54] +Right keys [1]: [ca_county#77] Join condition: None (114) Project [codegen id : 41] -Output [4]: [ca_county#60, web_sales#65, web_sales#79, web_sales#94] -Input [5]: [ca_county#60, web_sales#65, web_sales#79, ca_county#89, web_sales#94] +Output [4]: [ca_county#54, web_sales#59, web_sales#70, web_sales#82] +Input [5]: [ca_county#54, web_sales#59, web_sales#70, ca_county#77, web_sales#82] (115) BroadcastExchange -Input [4]: [ca_county#60, web_sales#65, web_sales#79, web_sales#94] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#96] +Input [4]: [ca_county#54, web_sales#59, web_sales#70, web_sales#82] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#84] (116) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ca_county#45] -Right keys [1]: [ca_county#60] -Join condition: ((CASE WHEN (web_sales#65 > 0.00) THEN CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#65)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#50 > 0.00) THEN CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#50)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#79 > 0.00) THEN CheckOverflow((promote_precision(web_sales#94) / promote_precision(web_sales#79)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#17 > 0.00) THEN CheckOverflow((promote_precision(store_sales#33) / promote_precision(store_sales#17)), DecimalType(37,20), true) ELSE null END)) +Left keys [1]: [ca_county#39] +Right keys [1]: [ca_county#54] +Join condition: ((CASE WHEN (web_sales#59 > 0.00) THEN CheckOverflow((promote_precision(web_sales#70) / promote_precision(web_sales#59)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#44 > 0.00) THEN CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#44)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#70 > 0.00) THEN CheckOverflow((promote_precision(web_sales#82) / promote_precision(web_sales#70)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#17 > 0.00) THEN CheckOverflow((promote_precision(store_sales#30) / promote_precision(store_sales#17)), DecimalType(37,20), true) ELSE null END)) (117) Project [codegen id : 42] -Output [6]: [ca_county#45, d_year#40, CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#65)), DecimalType(37,20), true) AS web_q1_q2_increase#97, CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#50)), DecimalType(37,20), true) AS store_q1_q2_increase#98, CheckOverflow((promote_precision(web_sales#94) / promote_precision(web_sales#79)), DecimalType(37,20), true) AS web_q2_q3_increase#99, CheckOverflow((promote_precision(store_sales#33) / promote_precision(store_sales#17)), DecimalType(37,20), true) AS store_q2_q3_increase#100] -Input [9]: [store_sales#17, store_sales#33, ca_county#45, d_year#40, store_sales#50, ca_county#60, web_sales#65, web_sales#79, web_sales#94] +Output [6]: [ca_county#39, d_year#34, CheckOverflow((promote_precision(web_sales#70) / promote_precision(web_sales#59)), DecimalType(37,20), true) AS web_q1_q2_increase#85, CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#44)), DecimalType(37,20), true) AS store_q1_q2_increase#86, CheckOverflow((promote_precision(web_sales#82) / promote_precision(web_sales#70)), DecimalType(37,20), true) AS web_q2_q3_increase#87, CheckOverflow((promote_precision(store_sales#30) / promote_precision(store_sales#17)), DecimalType(37,20), true) AS store_q2_q3_increase#88] +Input [9]: [store_sales#17, store_sales#30, ca_county#39, d_year#34, store_sales#44, ca_county#54, web_sales#59, web_sales#70, web_sales#82] (118) Exchange -Input [6]: [ca_county#45, d_year#40, web_q1_q2_increase#97, store_q1_q2_increase#98, web_q2_q3_increase#99, store_q2_q3_increase#100] -Arguments: rangepartitioning(ca_county#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#101] +Input [6]: [ca_county#39, d_year#34, web_q1_q2_increase#85, store_q1_q2_increase#86, web_q2_q3_increase#87, store_q2_q3_increase#88] +Arguments: rangepartitioning(ca_county#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#89] (119) Sort [codegen id : 43] -Input [6]: [ca_county#45, d_year#40, web_q1_q2_increase#97, store_q1_q2_increase#98, web_q2_q3_increase#99, store_q2_q3_increase#100] -Arguments: [ca_county#45 ASC NULLS FIRST], true, 0 +Input [6]: [ca_county#39, d_year#34, web_q1_q2_increase#85, store_q1_q2_increase#86, web_q2_q3_increase#87, store_q2_q3_increase#88] +Arguments: [ca_county#39 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -676,24 +676,24 @@ ReusedExchange (120) (120) ReusedExchange [Reuses operator id: 7] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#18 ReusedExchange (121) (121) ReusedExchange [Reuses operator id: 28] -Output [3]: [d_date_sk#22, d_year#23, d_qoy#24] +Output [3]: [d_date_sk#19, d_year#20, d_qoy#21] -Subquery:3 Hosting operator id = 43 Hosting Expression = ss_sold_date_sk#37 IN dynamicpruning#38 +Subquery:3 Hosting operator id = 43 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#32 ReusedExchange (122) (122) ReusedExchange [Reuses operator id: 49] -Output [3]: [d_date_sk#39, d_year#40, d_qoy#41] +Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] -Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#38 +Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#32 -Subquery:5 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#4 +Subquery:5 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#4 -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#21 +Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#18 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 f47a7af5a15e3..81a4a7fed99b2 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 @@ -188,383 +188,383 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#15] Results [3]: [ca_county#10, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#15,17,2) AS store_sales#16] (19) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19] +Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#19), dynamicpruningexpression(ss_sold_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#17)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (20) ColumnarToRow [codegen id : 6] -Input [3]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19] +Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] (21) Filter [codegen id : 6] -Input [3]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19] -Condition : isnotnull(ss_addr_sk#17) +Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_addr_sk#1) (22) Scan parquet default.date_dim -Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] +Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 4] -Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] +Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] (24) Filter [codegen id : 4] -Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] -Condition : ((((isnotnull(d_qoy#23) AND isnotnull(d_year#22)) AND (d_qoy#23 = 2)) AND (d_year#22 = 2000)) AND isnotnull(d_date_sk#21)) +Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) (25) 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=#24] +Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#19] -Right keys [1]: [d_date_sk#21] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#18] Join condition: None (27) Project [codegen id : 6] -Output [4]: [ss_addr_sk#17, ss_ext_sales_price#18, d_year#22, d_qoy#23] -Input [6]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19, d_date_sk#21, d_year#22, d_qoy#23] +Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#19, d_qoy#20] +Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#18, d_year#19, d_qoy#20] (28) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#25, ca_county#26] +Output [2]: [ca_address_sk#22, ca_county#23] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#17] -Right keys [1]: [ca_address_sk#25] +Left keys [1]: [ss_addr_sk#1] +Right keys [1]: [ca_address_sk#22] Join condition: None (30) Project [codegen id : 6] -Output [4]: [ss_ext_sales_price#18, d_year#22, d_qoy#23, ca_county#26] -Input [6]: [ss_addr_sk#17, ss_ext_sales_price#18, d_year#22, d_qoy#23, ca_address_sk#25, ca_county#26] +Output [4]: [ss_ext_sales_price#2, d_year#19, d_qoy#20, ca_county#23] +Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#19, d_qoy#20, ca_address_sk#22, ca_county#23] (31) HashAggregate [codegen id : 6] -Input [4]: [ss_ext_sales_price#18, d_year#22, d_qoy#23, ca_county#26] -Keys [3]: [ca_county#26, d_qoy#23, d_year#22] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#18))] -Aggregate Attributes [1]: [sum#27] -Results [4]: [ca_county#26, d_qoy#23, d_year#22, sum#28] +Input [4]: [ss_ext_sales_price#2, d_year#19, d_qoy#20, ca_county#23] +Keys [3]: [ca_county#23, d_qoy#20, d_year#19] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#24] +Results [4]: [ca_county#23, d_qoy#20, d_year#19, sum#25] (32) Exchange -Input [4]: [ca_county#26, d_qoy#23, d_year#22, sum#28] -Arguments: hashpartitioning(ca_county#26, d_qoy#23, d_year#22, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [4]: [ca_county#23, d_qoy#20, d_year#19, sum#25] +Arguments: hashpartitioning(ca_county#23, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, [id=#26] (33) HashAggregate [codegen id : 7] -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] +Input [4]: [ca_county#23, d_qoy#20, d_year#19, sum#25] +Keys [3]: [ca_county#23, d_qoy#20, d_year#19] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#27] +Results [2]: [ca_county#23, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#27,17,2) AS store_sales#28] (34) BroadcastExchange -Input [2]: [ca_county#26, store_sales#31] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] +Input [2]: [ca_county#23, store_sales#28] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#29] (35) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#10] -Right keys [1]: [ca_county#26] +Right keys [1]: [ca_county#23] Join condition: None (36) 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#1, ss_ext_sales_price#2, ss_sold_date_sk#3] 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#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 10] -Input [3]: [ss_addr_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] +Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] (38) Filter [codegen id : 10] -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#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_addr_sk#1) (39) Scan parquet default.date_dim -Output [3]: [d_date_sk#37, d_year#38, d_qoy#39] +Output [3]: [d_date_sk#31, d_year#32, d_qoy#33] 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 (40) ColumnarToRow [codegen id : 8] -Input [3]: [d_date_sk#37, d_year#38, d_qoy#39] +Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] (41) Filter [codegen id : 8] -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 = 3)) AND (d_year#38 = 2000)) AND isnotnull(d_date_sk#37)) +Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] +Condition : ((((isnotnull(d_qoy#33) AND isnotnull(d_year#32)) AND (d_qoy#33 = 3)) AND (d_year#32 = 2000)) AND isnotnull(d_date_sk#31)) (42) 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=#40] +Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (43) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#35] -Right keys [1]: [d_date_sk#37] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#31] Join condition: None (44) Project [codegen id : 10] -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#1, ss_ext_sales_price#2, d_year#32, d_qoy#33] +Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#31, d_year#32, d_qoy#33] (45) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#41, ca_county#42] +Output [2]: [ca_address_sk#35, ca_county#36] (46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_addr_sk#33] -Right keys [1]: [ca_address_sk#41] +Left keys [1]: [ss_addr_sk#1] +Right keys [1]: [ca_address_sk#35] Join condition: None (47) Project [codegen id : 10] -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#2, d_year#32, d_qoy#33, ca_county#36] +Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#32, d_qoy#33, ca_address_sk#35, ca_county#36] (48) HashAggregate [codegen id : 10] -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#2, d_year#32, d_qoy#33, ca_county#36] +Keys [3]: [ca_county#36, d_qoy#33, d_year#32] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#37] +Results [4]: [ca_county#36, d_qoy#33, d_year#32, sum#38] (49) 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#36, d_qoy#33, d_year#32, sum#38] +Arguments: hashpartitioning(ca_county#36, d_qoy#33, d_year#32, 5), ENSURE_REQUIREMENTS, [id=#39] (50) HashAggregate [codegen id : 11] -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 [2]: [ca_county#42, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#46,17,2) AS store_sales#47] +Input [4]: [ca_county#36, d_qoy#33, d_year#32, sum#38] +Keys [3]: [ca_county#36, d_qoy#33, d_year#32] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#40] +Results [2]: [ca_county#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#40,17,2) AS store_sales#41] (51) BroadcastExchange -Input [2]: [ca_county#42, store_sales#47] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] +Input [2]: [ca_county#36, store_sales#41] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#42] (52) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#26] -Right keys [1]: [ca_county#42] +Left keys [1]: [ca_county#23] +Right keys [1]: [ca_county#36] Join condition: None (53) Project [codegen id : 24] -Output [5]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47] -Input [7]: [ca_county#10, d_year#6, store_sales#16, ca_county#26, store_sales#31, ca_county#42, store_sales#47] +Output [5]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41] +Input [7]: [ca_county#10, d_year#6, store_sales#16, ca_county#23, store_sales#28, ca_county#36, store_sales#41] (54) 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#43, ws_ext_sales_price#44, ws_sold_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 14] -Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] (56) Filter [codegen id : 14] -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#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Condition : isnotnull(ws_bill_addr_sk#43) (57) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#52, d_year#53, d_qoy#54] +Output [3]: [d_date_sk#46, d_year#47, d_qoy#48] (58) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#52] +Left keys [1]: [ws_sold_date_sk#45] +Right keys [1]: [d_date_sk#46] Join condition: None (59) Project [codegen id : 14] -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#43, ws_ext_sales_price#44, d_year#47, d_qoy#48] +Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45, d_date_sk#46, d_year#47, d_qoy#48] (60) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#55, ca_county#56] +Output [2]: [ca_address_sk#49, ca_county#50] (61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#49] -Right keys [1]: [ca_address_sk#55] +Left keys [1]: [ws_bill_addr_sk#43] +Right keys [1]: [ca_address_sk#49] Join condition: None (62) Project [codegen id : 14] -Output [4]: [ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_county#56] -Input [6]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_address_sk#55, ca_county#56] +Output [4]: [ws_ext_sales_price#44, d_year#47, d_qoy#48, ca_county#50] +Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#47, d_qoy#48, ca_address_sk#49, ca_county#50] (63) HashAggregate [codegen id : 14] -Input [4]: [ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_county#56] -Keys [3]: [ca_county#56, d_qoy#54, d_year#53] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#50))] -Aggregate Attributes [1]: [sum#57] -Results [4]: [ca_county#56, d_qoy#54, d_year#53, sum#58] +Input [4]: [ws_ext_sales_price#44, d_year#47, d_qoy#48, ca_county#50] +Keys [3]: [ca_county#50, d_qoy#48, d_year#47] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#44))] +Aggregate Attributes [1]: [sum#51] +Results [4]: [ca_county#50, d_qoy#48, d_year#47, sum#52] (64) Exchange -Input [4]: [ca_county#56, d_qoy#54, d_year#53, sum#58] -Arguments: hashpartitioning(ca_county#56, d_qoy#54, d_year#53, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [4]: [ca_county#50, d_qoy#48, d_year#47, sum#52] +Arguments: hashpartitioning(ca_county#50, d_qoy#48, d_year#47, 5), ENSURE_REQUIREMENTS, [id=#53] (65) HashAggregate [codegen id : 15] -Input [4]: [ca_county#56, d_qoy#54, d_year#53, sum#58] -Keys [3]: [ca_county#56, 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))#60] -Results [2]: [ca_county#56, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#50))#60,17,2) AS web_sales#61] +Input [4]: [ca_county#50, d_qoy#48, d_year#47, sum#52] +Keys [3]: [ca_county#50, d_qoy#48, d_year#47] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#44))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#44))#54] +Results [2]: [ca_county#50, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#44))#54,17,2) AS web_sales#55] (66) BroadcastExchange -Input [2]: [ca_county#56, web_sales#61] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#62] +Input [2]: [ca_county#50, web_sales#55] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] (67) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#10] -Right keys [1]: [ca_county#56] +Right keys [1]: [ca_county#50] Join condition: None (68) 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#43, ws_ext_sales_price#44, ws_sold_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#17)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 18] -Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] (70) Filter [codegen id : 18] -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#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Condition : isnotnull(ws_bill_addr_sk#43) (71) ReusedExchange [Reuses operator id: 25] -Output [3]: [d_date_sk#66, d_year#67, d_qoy#68] +Output [3]: [d_date_sk#57, d_year#58, d_qoy#59] (72) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#66] +Left keys [1]: [ws_sold_date_sk#45] +Right keys [1]: [d_date_sk#57] Join condition: None (73) Project [codegen id : 18] -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#43, ws_ext_sales_price#44, d_year#58, d_qoy#59] +Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45, d_date_sk#57, d_year#58, d_qoy#59] (74) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#69, ca_county#70] +Output [2]: [ca_address_sk#60, ca_county#61] (75) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_addr_sk#63] -Right keys [1]: [ca_address_sk#69] +Left keys [1]: [ws_bill_addr_sk#43] +Right keys [1]: [ca_address_sk#60] Join condition: None (76) Project [codegen id : 18] -Output [4]: [ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_county#70] -Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_address_sk#69, ca_county#70] +Output [4]: [ws_ext_sales_price#44, d_year#58, d_qoy#59, ca_county#61] +Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#58, d_qoy#59, ca_address_sk#60, ca_county#61] (77) HashAggregate [codegen id : 18] -Input [4]: [ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_county#70] -Keys [3]: [ca_county#70, d_qoy#68, d_year#67] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#64))] -Aggregate Attributes [1]: [sum#71] -Results [4]: [ca_county#70, d_qoy#68, d_year#67, sum#72] +Input [4]: [ws_ext_sales_price#44, d_year#58, d_qoy#59, ca_county#61] +Keys [3]: [ca_county#61, d_qoy#59, d_year#58] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#44))] +Aggregate Attributes [1]: [sum#62] +Results [4]: [ca_county#61, d_qoy#59, d_year#58, sum#63] (78) Exchange -Input [4]: [ca_county#70, d_qoy#68, d_year#67, sum#72] -Arguments: hashpartitioning(ca_county#70, d_qoy#68, d_year#67, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [4]: [ca_county#61, d_qoy#59, d_year#58, sum#63] +Arguments: hashpartitioning(ca_county#61, d_qoy#59, d_year#58, 5), ENSURE_REQUIREMENTS, [id=#64] (79) HashAggregate [codegen id : 19] -Input [4]: [ca_county#70, d_qoy#68, d_year#67, sum#72] -Keys [3]: [ca_county#70, 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))#74] -Results [2]: [ca_county#70, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#64))#74,17,2) AS web_sales#75] +Input [4]: [ca_county#61, d_qoy#59, d_year#58, sum#63] +Keys [3]: [ca_county#61, d_qoy#59, d_year#58] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#44))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#44))#65] +Results [2]: [ca_county#61, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#44))#65,17,2) AS web_sales#66] (80) BroadcastExchange -Input [2]: [ca_county#70, web_sales#75] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#76] +Input [2]: [ca_county#61, web_sales#66] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#67] (81) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#56] -Right keys [1]: [ca_county#70] -Join condition: (CASE WHEN (web_sales#61 > 0.00) THEN CheckOverflow((promote_precision(web_sales#75) / promote_precision(web_sales#61)), 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#50] +Right keys [1]: [ca_county#61] +Join condition: (CASE WHEN (web_sales#55 > 0.00) THEN CheckOverflow((promote_precision(web_sales#66) / promote_precision(web_sales#55)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#16 > 0.00) THEN CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#16)), DecimalType(37,20), true) ELSE null END) (82) Project [codegen id : 24] -Output [8]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47, ca_county#56, web_sales#61, web_sales#75] -Input [9]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47, ca_county#56, web_sales#61, ca_county#70, web_sales#75] +Output [8]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41, ca_county#50, web_sales#55, web_sales#66] +Input [9]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41, ca_county#50, web_sales#55, ca_county#61, web_sales#66] (83) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79] +Output [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (84) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79] +Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] (85) Filter [codegen id : 22] -Input [3]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79] -Condition : isnotnull(ws_bill_addr_sk#77) +Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Condition : isnotnull(ws_bill_addr_sk#43) (86) ReusedExchange [Reuses operator id: 42] -Output [3]: [d_date_sk#80, d_year#81, d_qoy#82] +Output [3]: [d_date_sk#68, d_year#69, d_qoy#70] (87) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#79] -Right keys [1]: [d_date_sk#80] +Left keys [1]: [ws_sold_date_sk#45] +Right keys [1]: [d_date_sk#68] Join condition: None (88) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, d_year#81, d_qoy#82] -Input [6]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79, d_date_sk#80, d_year#81, d_qoy#82] +Output [4]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#69, d_qoy#70] +Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45, d_date_sk#68, d_year#69, d_qoy#70] (89) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#83, ca_county#84] +Output [2]: [ca_address_sk#71, ca_county#72] (90) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_bill_addr_sk#77] -Right keys [1]: [ca_address_sk#83] +Left keys [1]: [ws_bill_addr_sk#43] +Right keys [1]: [ca_address_sk#71] Join condition: None (91) Project [codegen id : 22] -Output [4]: [ws_ext_sales_price#78, d_year#81, d_qoy#82, ca_county#84] -Input [6]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, d_year#81, d_qoy#82, ca_address_sk#83, ca_county#84] +Output [4]: [ws_ext_sales_price#44, d_year#69, d_qoy#70, ca_county#72] +Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#69, d_qoy#70, ca_address_sk#71, ca_county#72] (92) HashAggregate [codegen id : 22] -Input [4]: [ws_ext_sales_price#78, d_year#81, d_qoy#82, ca_county#84] -Keys [3]: [ca_county#84, d_qoy#82, d_year#81] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#78))] -Aggregate Attributes [1]: [sum#85] -Results [4]: [ca_county#84, d_qoy#82, d_year#81, sum#86] +Input [4]: [ws_ext_sales_price#44, d_year#69, d_qoy#70, ca_county#72] +Keys [3]: [ca_county#72, d_qoy#70, d_year#69] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#44))] +Aggregate Attributes [1]: [sum#73] +Results [4]: [ca_county#72, d_qoy#70, d_year#69, sum#74] (93) Exchange -Input [4]: [ca_county#84, d_qoy#82, d_year#81, sum#86] -Arguments: hashpartitioning(ca_county#84, d_qoy#82, d_year#81, 5), ENSURE_REQUIREMENTS, [id=#87] +Input [4]: [ca_county#72, d_qoy#70, d_year#69, sum#74] +Arguments: hashpartitioning(ca_county#72, d_qoy#70, d_year#69, 5), ENSURE_REQUIREMENTS, [id=#75] (94) HashAggregate [codegen id : 23] -Input [4]: [ca_county#84, d_qoy#82, d_year#81, sum#86] -Keys [3]: [ca_county#84, d_qoy#82, d_year#81] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#78))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#78))#88] -Results [2]: [ca_county#84, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#78))#88,17,2) AS web_sales#89] +Input [4]: [ca_county#72, d_qoy#70, d_year#69, sum#74] +Keys [3]: [ca_county#72, d_qoy#70, d_year#69] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#44))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#44))#76] +Results [2]: [ca_county#72, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#44))#76,17,2) AS web_sales#77] (95) BroadcastExchange -Input [2]: [ca_county#84, web_sales#89] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#90] +Input [2]: [ca_county#72, web_sales#77] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#78] (96) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#56] -Right keys [1]: [ca_county#84] -Join condition: (CASE WHEN (web_sales#75 > 0.00) THEN CheckOverflow((promote_precision(web_sales#89) / promote_precision(web_sales#75)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#31 > 0.00) THEN CheckOverflow((promote_precision(store_sales#47) / promote_precision(store_sales#31)), DecimalType(37,20), true) ELSE null END) +Left keys [1]: [ca_county#50] +Right keys [1]: [ca_county#72] +Join condition: (CASE WHEN (web_sales#66 > 0.00) THEN CheckOverflow((promote_precision(web_sales#77) / promote_precision(web_sales#66)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#28 > 0.00) THEN CheckOverflow((promote_precision(store_sales#41) / promote_precision(store_sales#28)), DecimalType(37,20), true) ELSE null END) (97) Project [codegen id : 24] -Output [6]: [ca_county#10, d_year#6, CheckOverflow((promote_precision(web_sales#75) / promote_precision(web_sales#61)), DecimalType(37,20), true) AS web_q1_q2_increase#91, CheckOverflow((promote_precision(store_sales#31) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q1_q2_increase#92, CheckOverflow((promote_precision(web_sales#89) / promote_precision(web_sales#75)), DecimalType(37,20), true) AS web_q2_q3_increase#93, CheckOverflow((promote_precision(store_sales#47) / promote_precision(store_sales#31)), DecimalType(37,20), true) AS store_q2_q3_increase#94] -Input [10]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47, ca_county#56, web_sales#61, web_sales#75, ca_county#84, web_sales#89] +Output [6]: [ca_county#10, d_year#6, CheckOverflow((promote_precision(web_sales#66) / promote_precision(web_sales#55)), DecimalType(37,20), true) AS web_q1_q2_increase#79, CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q1_q2_increase#80, CheckOverflow((promote_precision(web_sales#77) / promote_precision(web_sales#66)), DecimalType(37,20), true) AS web_q2_q3_increase#81, CheckOverflow((promote_precision(store_sales#41) / promote_precision(store_sales#28)), DecimalType(37,20), true) AS store_q2_q3_increase#82] +Input [10]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41, ca_county#50, web_sales#55, web_sales#66, ca_county#72, web_sales#77] (98) Exchange -Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#91, store_q1_q2_increase#92, web_q2_q3_increase#93, store_q2_q3_increase#94] -Arguments: rangepartitioning(ca_county#10 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#95] +Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#79, store_q1_q2_increase#80, web_q2_q3_increase#81, store_q2_q3_increase#82] +Arguments: rangepartitioning(ca_county#10 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#83] (99) Sort [codegen id : 25] -Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#91, store_q1_q2_increase#92, web_q2_q3_increase#93, store_q2_q3_increase#94] +Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#79, store_q1_q2_increase#80, web_q2_q3_increase#81, store_q2_q3_increase#82] Arguments: [ca_county#10 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -576,24 +576,24 @@ ReusedExchange (100) (100) ReusedExchange [Reuses operator id: 7] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#19 IN dynamicpruning#20 +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#17 ReusedExchange (101) (101) ReusedExchange [Reuses operator id: 25] -Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] +Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Subquery:3 Hosting operator id = 36 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 +Subquery:3 Hosting operator id = 36 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#30 ReusedExchange (102) (102) ReusedExchange [Reuses operator id: 42] -Output [3]: [d_date_sk#37, d_year#38, d_qoy#39] +Output [3]: [d_date_sk#31, d_year#32, d_qoy#33] -Subquery:4 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#20 +Subquery:5 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#17 -Subquery:6 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#36 +Subquery:6 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#30 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index 9537689459170..0f35745fc2144 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -79,100 +79,105 @@ Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] (11) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] +Output [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#4)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] +Input [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] (13) Filter [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) +Input [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] +Condition : isnotnull(cs_item_sk#8) (14) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#9] +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#8, d_date#9] +Input [2]: [d_date_sk#11, d_date#12] (16) Filter [codegen id : 2] +<<<<<<< HEAD +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) +======= Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) +>>>>>>> abfd9b23cd7c21e9525df85a16e0611ef0f35908 (17) Project [codegen id : 2] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#9] +Output [1]: [d_date_sk#11] +Input [2]: [d_date_sk#11, d_date#12] (18) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] (19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#10] +Right keys [1]: [d_date_sk#11] Join condition: None (20) Project [codegen id : 3] -Output [2]: [cs_item_sk#1, cs_ext_discount_amt#2] -Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#8] +Output [2]: [cs_item_sk#8, cs_ext_discount_amt#9] +Input [4]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10, d_date_sk#11] (21) HashAggregate [codegen id : 3] -Input [2]: [cs_item_sk#1, cs_ext_discount_amt#2] -Keys [1]: [cs_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [2]: [sum#11, count#12] -Results [3]: [cs_item_sk#1, sum#13, count#14] +Input [2]: [cs_item_sk#8, cs_ext_discount_amt#9] +Keys [1]: [cs_item_sk#8] +Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#9))] +Aggregate Attributes [2]: [sum#14, count#15] +Results [3]: [cs_item_sk#8, sum#16, count#17] (22) Exchange -Input [3]: [cs_item_sk#1, sum#13, count#14] -Arguments: hashpartitioning(cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [3]: [cs_item_sk#8, sum#16, count#17] +Arguments: hashpartitioning(cs_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#18] (23) HashAggregate [codegen id : 4] -Input [3]: [cs_item_sk#1, sum#13, count#14] -Keys [1]: [cs_item_sk#1] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#2))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#2))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1 AS cs_item_sk#1#18] +Input [3]: [cs_item_sk#8, sum#16, count#17] +Keys [1]: [cs_item_sk#8] +Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#9))] +Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#9))#19] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#9))#19 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] (24) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] +Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#20) (25) BroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#19] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#21] (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] -Right keys [1]: [cs_item_sk#1#18] -Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) +Right keys [1]: [cs_item_sk#8] +Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#20) (27) Project [codegen id : 6] Output [1]: [cs_sold_date_sk#3] -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] +Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] (28) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 6] -Output [1]: [1 AS excess discount amount #20] -Input [2]: [cs_sold_date_sk#3, d_date_sk#8] +Output [1]: [1 AS excess discount amount #23] +Input [2]: [cs_sold_date_sk#3, d_date_sk#22] (31) CollectLimit -Input [1]: [excess discount amount #20] +Input [1]: [excess discount amount #23] Arguments: 100 ===== Subqueries ===== @@ -182,8 +187,8 @@ ReusedExchange (32) (32) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] -Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index cba099c2bb5b0..1349aa54f3cc2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -163,7 +163,7 @@ Condition : isnotnull(i_item_sk#13) Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics)) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index cba099c2bb5b0..1349aa54f3cc2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -163,7 +163,7 @@ Condition : isnotnull(i_item_sk#13) Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics)) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt index 9da5f36eb0553..7098389df6776 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt @@ -145,20 +145,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#17] +Output [1]: [d_date_sk#10] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#17] +Right keys [1]: [d_date_sk#10] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] -Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] +Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] (23) Exchange Input [1]: [ws_bill_customer_sk#15] -Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] +Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] @@ -170,38 +170,38 @@ Right keys [1]: [ws_bill_customer_sk#15] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] +Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] +Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#21] +Output [1]: [d_date_sk#10] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#21] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#10] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#19] -Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] +Output [1]: [cs_ship_customer_sk#18] +Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] (31) Exchange -Input [1]: [cs_ship_customer_sk#19] -Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [cs_ship_customer_sk#18] +Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#19] -Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#18] +Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#19] +Right keys [1]: [cs_ship_customer_sk#18] Join condition: None (34) Filter [codegen id : 12] @@ -214,103 +214,103 @@ Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2 (36) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#23] +Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#21] (37) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 (38) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#22, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 14] -Input [2]: [ca_address_sk#24, ca_state#25] +Input [2]: [ca_address_sk#22, ca_state#23] (40) Filter [codegen id : 14] -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : isnotnull(ca_address_sk#24) +Input [2]: [ca_address_sk#22, ca_state#23] +Condition : isnotnull(ca_address_sk#22) (41) Exchange -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [ca_address_sk#22, ca_state#23] +Arguments: hashpartitioning(ca_address_sk#22, 5), ENSURE_REQUIREMENTS, [id=#24] (42) Sort [codegen id : 15] -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#22, ca_state#23] +Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#24] +Right keys [1]: [ca_address_sk#22] Join condition: None (44) Project [codegen id : 16] -Output [2]: [c_current_cdemo_sk#4, ca_state#25] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24, ca_state#25] +Output [2]: [c_current_cdemo_sk#4, ca_state#23] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] (45) Exchange -Input [2]: [c_current_cdemo_sk#4, ca_state#25] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [2]: [c_current_cdemo_sk#4, ca_state#23] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#25] (46) Sort [codegen id : 17] -Input [2]: [c_current_cdemo_sk#4, ca_state#25] +Input [2]: [c_current_cdemo_sk#4, ca_state#23] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 (47) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 18] -Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] (49) Filter [codegen id : 18] -Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Condition : isnotnull(cd_demo_sk#28) +Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Condition : isnotnull(cd_demo_sk#26) (50) Exchange -Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Arguments: hashpartitioning(cd_demo_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] (51) Sort [codegen id : 19] -Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 +Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Arguments: [cd_demo_sk#26 ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#28] +Right keys [1]: [cd_demo_sk#26] Join condition: None (53) Project [codegen id : 20] -Output [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Input [8]: [c_current_cdemo_sk#4, ca_state#25, cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] (54) HashAggregate [codegen id : 20] -Input [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#31), partial_max(cd_dep_count#31), partial_avg(cd_dep_count#31), partial_min(cd_dep_employed_count#32), partial_max(cd_dep_employed_count#32), partial_avg(cd_dep_employed_count#32), partial_min(cd_dep_college_count#33), partial_max(cd_dep_college_count#33), partial_avg(cd_dep_college_count#33)] -Aggregate Attributes [13]: [count#35, min#36, max#37, sum#38, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47] -Results [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] +Input [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Functions [10]: [partial_count(1), partial_min(cd_dep_count#29), partial_max(cd_dep_count#29), partial_avg(cd_dep_count#29), partial_min(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_avg(cd_dep_employed_count#30), partial_min(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_avg(cd_dep_college_count#31)] +Aggregate Attributes [13]: [count#33, min#34, max#35, sum#36, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45] +Results [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] (55) Exchange -Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] -Arguments: hashpartitioning(ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] +Arguments: hashpartitioning(ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#59] (56) HashAggregate [codegen id : 21] -Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] -Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Functions [10]: [count(1), min(cd_dep_count#31), max(cd_dep_count#31), avg(cd_dep_count#31), min(cd_dep_employed_count#32), max(cd_dep_employed_count#32), avg(cd_dep_employed_count#32), min(cd_dep_college_count#33), max(cd_dep_college_count#33), avg(cd_dep_college_count#33)] -Aggregate Attributes [10]: [count(1)#62, min(cd_dep_count#31)#63, max(cd_dep_count#31)#64, avg(cd_dep_count#31)#65, min(cd_dep_employed_count#32)#66, max(cd_dep_employed_count#32)#67, avg(cd_dep_employed_count#32)#68, min(cd_dep_college_count#33)#69, max(cd_dep_college_count#33)#70, avg(cd_dep_college_count#33)#71] -Results [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, count(1)#62 AS cnt1#72, min(cd_dep_count#31)#63 AS min(cd_dep_count)#73, max(cd_dep_count#31)#64 AS max(cd_dep_count)#74, avg(cd_dep_count#31)#65 AS avg(cd_dep_count)#75, cd_dep_employed_count#32, count(1)#62 AS cnt2#76, min(cd_dep_employed_count#32)#66 AS min(cd_dep_employed_count)#77, max(cd_dep_employed_count#32)#67 AS max(cd_dep_employed_count)#78, avg(cd_dep_employed_count#32)#68 AS avg(cd_dep_employed_count)#79, cd_dep_college_count#33, count(1)#62 AS cnt3#80, min(cd_dep_college_count#33)#69 AS min(cd_dep_college_count)#81, max(cd_dep_college_count#33)#70 AS max(cd_dep_college_count)#82, avg(cd_dep_college_count#33)#71 AS avg(cd_dep_college_count)#83, cd_dep_count#31 AS aggOrder#84] +Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] +Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Functions [10]: [count(1), min(cd_dep_count#29), max(cd_dep_count#29), avg(cd_dep_count#29), min(cd_dep_employed_count#30), max(cd_dep_employed_count#30), avg(cd_dep_employed_count#30), min(cd_dep_college_count#31), max(cd_dep_college_count#31), avg(cd_dep_college_count#31)] +Aggregate Attributes [10]: [count(1)#60, min(cd_dep_count#29)#61, max(cd_dep_count#29)#62, avg(cd_dep_count#29)#63, min(cd_dep_employed_count#30)#64, max(cd_dep_employed_count#30)#65, avg(cd_dep_employed_count#30)#66, min(cd_dep_college_count#31)#67, max(cd_dep_college_count#31)#68, avg(cd_dep_college_count#31)#69] +Results [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, count(1)#60 AS cnt1#70, min(cd_dep_count#29)#61 AS min(cd_dep_count)#71, max(cd_dep_count#29)#62 AS max(cd_dep_count)#72, avg(cd_dep_count#29)#63 AS avg(cd_dep_count)#73, cd_dep_employed_count#30, count(1)#60 AS cnt2#74, min(cd_dep_employed_count#30)#64 AS min(cd_dep_employed_count)#75, max(cd_dep_employed_count#30)#65 AS max(cd_dep_employed_count)#76, avg(cd_dep_employed_count#30)#66 AS avg(cd_dep_employed_count)#77, cd_dep_college_count#31, count(1)#60 AS cnt3#78, min(cd_dep_college_count#31)#67 AS min(cd_dep_college_count)#79, max(cd_dep_college_count#31)#68 AS max(cd_dep_college_count)#80, avg(cd_dep_college_count#31)#69 AS avg(cd_dep_college_count)#81, cd_dep_count#29 AS aggOrder#82] (57) TakeOrderedAndProject -Input [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cnt1#72, min(cd_dep_count)#73, max(cd_dep_count)#74, avg(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, min(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, avg(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, min(cd_dep_college_count)#81, max(cd_dep_college_count)#82, avg(cd_dep_college_count)#83, aggOrder#84] -Arguments: 100, [ca_state#25 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, aggOrder#84 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [ca_state#25, cd_gender#29, cd_marital_status#30, cnt1#72, min(cd_dep_count)#73, max(cd_dep_count)#74, avg(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, min(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, avg(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, min(cd_dep_college_count)#81, max(cd_dep_college_count)#82, avg(cd_dep_college_count)#83] +Input [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cnt1#70, min(cd_dep_count)#71, max(cd_dep_count)#72, avg(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, min(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, avg(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, min(cd_dep_college_count)#79, max(cd_dep_college_count)#80, avg(cd_dep_college_count)#81, aggOrder#82] +Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, aggOrder#82 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#23, cd_gender#27, cd_marital_status#28, cnt1#70, min(cd_dep_count)#71, max(cd_dep_count)#72, avg(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, min(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, avg(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, min(cd_dep_college_count)#79, max(cd_dep_college_count)#80, avg(cd_dep_college_count)#81] ===== Subqueries ===== @@ -323,6 +323,6 @@ Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index 48ae824834450..4dab13491784a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -122,20 +122,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#9] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#9] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#16] +Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] @@ -143,34 +143,34 @@ Right keys [1]: [ws_bill_customer_sk#14] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#20] +Output [1]: [d_date_sk#9] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#20] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#9] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#18] -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#20] +Output [1]: [cs_ship_customer_sk#17] +Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [cs_ship_customer_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#17] Join condition: None (29) Filter [codegen id : 9] @@ -182,80 +182,80 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_state#23] +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#22, ca_state#23] +Input [2]: [ca_address_sk#20, ca_state#21] (33) Filter [codegen id : 7] -Input [2]: [ca_address_sk#22, ca_state#23] -Condition : isnotnull(ca_address_sk#22) +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) (34) BroadcastExchange -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] +Input [2]: [ca_address_sk#20, ca_state#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] (35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#20] Join condition: None (36) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#23] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] +Output [2]: [c_current_cdemo_sk#4, ca_state#21] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] (37) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (39) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#25) +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Condition : isnotnull(cd_demo_sk#23) (40) BroadcastExchange -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#23] Join condition: None (42) Project [codegen id : 9] -Output [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (43) HashAggregate [codegen id : 9] -Input [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#28), partial_max(cd_dep_count#28), partial_avg(cd_dep_count#28), partial_min(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_avg(cd_dep_employed_count#29), partial_min(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_avg(cd_dep_college_count#30)] -Aggregate Attributes [13]: [count#32, min#33, max#34, sum#35, count#36, min#37, max#38, sum#39, count#40, min#41, max#42, sum#43, count#44] -Results [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] +Input [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [10]: [partial_count(1), partial_min(cd_dep_count#26), partial_max(cd_dep_count#26), partial_avg(cd_dep_count#26), partial_min(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_avg(cd_dep_employed_count#27), partial_min(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_avg(cd_dep_college_count#28)] +Aggregate Attributes [13]: [count#30, min#31, max#32, sum#33, count#34, min#35, max#36, sum#37, count#38, min#39, max#40, sum#41, count#42] +Results [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] (44) Exchange -Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] -Arguments: hashpartitioning(ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] +Arguments: hashpartitioning(ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [id=#56] (45) HashAggregate [codegen id : 10] -Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] -Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [10]: [count(1), min(cd_dep_count#28), max(cd_dep_count#28), avg(cd_dep_count#28), min(cd_dep_employed_count#29), max(cd_dep_employed_count#29), avg(cd_dep_employed_count#29), min(cd_dep_college_count#30), max(cd_dep_college_count#30), avg(cd_dep_college_count#30)] -Aggregate Attributes [10]: [count(1)#59, min(cd_dep_count#28)#60, max(cd_dep_count#28)#61, avg(cd_dep_count#28)#62, min(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, avg(cd_dep_employed_count#29)#65, min(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, avg(cd_dep_college_count#30)#68] -Results [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, count(1)#59 AS cnt1#69, min(cd_dep_count#28)#60 AS min(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, avg(cd_dep_count#28)#62 AS avg(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, min(cd_dep_employed_count#29)#63 AS min(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, avg(cd_dep_employed_count#29)#65 AS avg(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, min(cd_dep_college_count#30)#66 AS min(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, avg(cd_dep_college_count#30)#68 AS avg(cd_dep_college_count)#80, cd_dep_count#28 AS aggOrder#81] +Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] +Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [10]: [count(1), min(cd_dep_count#26), max(cd_dep_count#26), avg(cd_dep_count#26), min(cd_dep_employed_count#27), max(cd_dep_employed_count#27), avg(cd_dep_employed_count#27), min(cd_dep_college_count#28), max(cd_dep_college_count#28), avg(cd_dep_college_count#28)] +Aggregate Attributes [10]: [count(1)#57, min(cd_dep_count#26)#58, max(cd_dep_count#26)#59, avg(cd_dep_count#26)#60, min(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, avg(cd_dep_employed_count#27)#63, min(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, avg(cd_dep_college_count#28)#66] +Results [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, count(1)#57 AS cnt1#67, min(cd_dep_count#26)#58 AS min(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, avg(cd_dep_count#26)#60 AS avg(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, min(cd_dep_employed_count#27)#61 AS min(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, avg(cd_dep_employed_count#27)#63 AS avg(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, min(cd_dep_college_count#28)#64 AS min(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, avg(cd_dep_college_count#28)#66 AS avg(cd_dep_college_count)#78, cd_dep_count#26 AS aggOrder#79] (46) TakeOrderedAndProject -Input [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80, aggOrder#81] -Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, aggOrder#81 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#23, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80] +Input [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78, aggOrder#79] +Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, aggOrder#79 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#21, cd_gender#24, cd_marital_status#25, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78] ===== Subqueries ===== @@ -268,6 +268,6 @@ Output [1]: [d_date_sk#9] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 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 a53bec582637d..68ab7a8bbecab 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 @@ -176,7 +176,7 @@ Results [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stddev_samp(cast(inv_qua (26) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stdev#28, mean#29] -Condition : (((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) +Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND (NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0))) (27) Project [codegen id : 5] Output [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -280,7 +280,7 @@ Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stddev_samp(cast(inv_qu (49) Filter [codegen id : 11] Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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)) +Condition : ((isnotnull(mean#54) AND isnotnull(stdev#53)) AND (NOT (mean#54 = 0.0) AND ((stdev#53 / mean#54) > 1.0))) (50) Project [codegen id : 11] Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, CASE WHEN (mean#54 = 0.0) THEN null ELSE (stdev#53 / mean#54) END AS cov#55] 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 c61480a05a4cf..7abd3bb1acb11 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 @@ -173,7 +173,7 @@ Results [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stddev_samp(cast(inv_quan (26) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] -Condition : (((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) +Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND (NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0))) (27) Project [codegen id : 10] Output [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -269,7 +269,7 @@ Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_qu (47) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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)) +Condition : ((isnotnull(mean#53) AND isnotnull(stdev#52)) AND (NOT (mean#53 = 0.0) AND ((stdev#52 / mean#53) > 1.0))) (48) Project [codegen id : 9] Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] 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 6bc5980231252..f0dfbac4f3a0f 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 @@ -176,7 +176,7 @@ Results [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stddev_samp(cast(inv_qua (26) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stdev#28, mean#29] -Condition : (((((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.5)) +Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND ((NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0)) AND ((stdev#28 / mean#29) > 1.5))) (27) Project [codegen id : 5] Output [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -280,7 +280,7 @@ Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stddev_samp(cast(inv_qu (49) Filter [codegen id : 11] Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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)) +Condition : ((isnotnull(mean#54) AND isnotnull(stdev#53)) AND (NOT (mean#54 = 0.0) AND ((stdev#53 / mean#54) > 1.0))) (50) Project [codegen id : 11] Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, CASE WHEN (mean#54 = 0.0) THEN null ELSE (stdev#53 / mean#54) END AS cov#55] 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 0ffe40240be2b..d1a5ecaa6a4d8 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 @@ -173,7 +173,7 @@ Results [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stddev_samp(cast(inv_quan (26) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] -Condition : (((((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.5)) +Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND ((NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0)) AND ((stdev#28 / mean#29) > 1.5))) (27) Project [codegen id : 10] Output [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -269,7 +269,7 @@ Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_qu (47) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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)) +Condition : ((isnotnull(mean#53) AND isnotnull(stdev#52)) AND (NOT (mean#53 = 0.0) AND ((stdev#52 / mean#53) > 1.0))) (48) Project [codegen id : 9] Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] 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 ce8b8bed5a26e..7b6bd35bfe180 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 @@ -239,465 +239,465 @@ Input [2]: [customer_id#27, year_total#28] Arguments: [customer_id#27 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Output [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6] 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#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6] (27) Filter [codegen id : 10] -Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] -Condition : isnotnull(ss_customer_sk#30) +Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6] +Condition : isnotnull(ss_customer_sk#1) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#37, d_year#38] +Output [2]: [d_date_sk#8, d_year#9] 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 (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#37, d_year#38] +Input [2]: [d_date_sk#8, d_year#9] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#37, d_year#38] -Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_sk#37)) +Input [2]: [d_date_sk#8, d_year#9] +Condition : ((isnotnull(d_year#9) AND (d_year#9 = 2002)) AND isnotnull(d_date_sk#8)) (31) BroadcastExchange -Input [2]: [d_date_sk#37, d_year#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#39] +Input [2]: [d_date_sk#8, d_year#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#35] -Right keys [1]: [d_date_sk#37] +Left keys [1]: [ss_sold_date_sk#6] +Right keys [1]: [d_date_sk#8] Join condition: None (33) Project [codegen id : 10] -Output [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] -Input [8]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#37, d_year#38] +Output [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] +Input [8]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6, d_date_sk#8, d_year#9] (34) Exchange -Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] -Arguments: hashpartitioning(ss_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] +Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#32] (35) Sort [codegen id : 11] -Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] -Arguments: [ss_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] +Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (37) Sort [codegen id : 13] -Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] -Arguments: [c_customer_sk#41 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#30] -Right keys [1]: [c_customer_sk#41] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#12] Join condition: None (39) Project [codegen id : 14] -Output [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] -Input [14]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38, c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] +Input [14]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (40) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] -Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#49, isEmpty#50] -Results [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, sum#51, isEmpty#52] +Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#33, isEmpty#34] +Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#35, isEmpty#36] (41) Exchange -Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, sum#51, isEmpty#52] -Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#35, isEmpty#36] +Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#37] (42) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, sum#51, isEmpty#52] -Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 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#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#54] -Results [8]: [c_customer_id#42 AS customer_id#55, c_first_name#43 AS customer_first_name#56, c_last_name#44 AS customer_last_name#57, c_preferred_cust_flag#45 AS customer_preferred_cust_flag#58, c_birth_country#46 AS customer_birth_country#59, c_login#47 AS customer_login#60, c_email_address#48 AS customer_email_address#61, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#54 AS year_total#62] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#35, isEmpty#36] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 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#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#38] +Results [8]: [c_customer_id#13 AS customer_id#39, c_first_name#14 AS customer_first_name#40, c_last_name#15 AS customer_last_name#41, c_preferred_cust_flag#16 AS customer_preferred_cust_flag#42, c_birth_country#17 AS customer_birth_country#43, c_login#18 AS customer_login#44, c_email_address#19 AS customer_email_address#45, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#38 AS year_total#46] (43) Exchange -Input [8]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62] -Arguments: hashpartitioning(customer_id#55, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [8]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46] +Arguments: hashpartitioning(customer_id#39, 5), ENSURE_REQUIREMENTS, [id=#47] (44) Sort [codegen id : 16] -Input [8]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62] -Arguments: [customer_id#55 ASC NULLS FIRST], false, 0 +Input [8]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46] +Arguments: [customer_id#39 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#55] +Right keys [1]: [customer_id#39] Join condition: None (46) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69] +Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#69), dynamicpruningexpression(cs_sold_date_sk#69 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#53), dynamicpruningexpression(cs_sold_date_sk#53 IN dynamicpruning#7)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69] +Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] (48) Filter [codegen id : 19] -Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69] -Condition : isnotnull(cs_bill_customer_sk#64) +Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] +Condition : isnotnull(cs_bill_customer_sk#48) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#70, d_year#71] +Output [2]: [d_date_sk#8, d_year#9] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_sold_date_sk#69] -Right keys [1]: [d_date_sk#70] +Left keys [1]: [cs_sold_date_sk#53] +Right keys [1]: [d_date_sk#8] Join condition: None (51) Project [codegen id : 19] -Output [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] -Input [8]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69, d_date_sk#70, d_year#71] +Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Input [8]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53, d_date_sk#8, d_year#9] (52) Exchange -Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] -Arguments: hashpartitioning(cs_bill_customer_sk#64, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Arguments: hashpartitioning(cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, [id=#54] (53) Sort [codegen id : 20] -Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] -Arguments: [cs_bill_customer_sk#64 ASC NULLS FIRST], false, 0 +Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Arguments: [cs_bill_customer_sk#48 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (55) Sort [codegen id : 22] -Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] -Arguments: [c_customer_sk#73 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#64] -Right keys [1]: [c_customer_sk#73] +Left keys [1]: [cs_bill_customer_sk#48] +Right keys [1]: [c_customer_sk#12] Join condition: None (57) Project [codegen id : 23] -Output [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] -Input [14]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71, c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Input [14]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (58) HashAggregate [codegen id : 23] -Input [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] -Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#81, isEmpty#82] -Results [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, sum#83, isEmpty#84] +Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#55, isEmpty#56] +Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#57, isEmpty#58] (59) Exchange -Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, sum#83, isEmpty#84] -Arguments: hashpartitioning(c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#57, isEmpty#58] +Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#59] (60) HashAggregate [codegen id : 24] -Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, sum#83, isEmpty#84] -Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 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#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86] -Results [2]: [c_customer_id#74 AS customer_id#87, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86 AS year_total#88] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#57, isEmpty#58] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 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#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#60] +Results [2]: [c_customer_id#13 AS customer_id#61, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#60 AS year_total#62] (61) Filter [codegen id : 24] -Input [2]: [customer_id#87, year_total#88] -Condition : (isnotnull(year_total#88) AND (year_total#88 > 0.000000)) +Input [2]: [customer_id#61, year_total#62] +Condition : (isnotnull(year_total#62) AND (year_total#62 > 0.000000)) (62) Project [codegen id : 24] -Output [2]: [customer_id#87 AS customer_id#89, year_total#88 AS year_total#90] -Input [2]: [customer_id#87, year_total#88] +Output [2]: [customer_id#61 AS customer_id#63, year_total#62 AS year_total#64] +Input [2]: [customer_id#61, year_total#62] (63) Exchange -Input [2]: [customer_id#89, year_total#90] -Arguments: hashpartitioning(customer_id#89, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [2]: [customer_id#63, year_total#64] +Arguments: hashpartitioning(customer_id#63, 5), ENSURE_REQUIREMENTS, [id=#65] (64) Sort [codegen id : 25] -Input [2]: [customer_id#89, year_total#90] -Arguments: [customer_id#89 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#63, year_total#64] +Arguments: [customer_id#63 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#89] +Right keys [1]: [customer_id#63] Join condition: None (66) Project [codegen id : 26] -Output [11]: [customer_id#27, year_total#28, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62, year_total#90] -Input [12]: [customer_id#27, year_total#28, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62, customer_id#89, year_total#90] +Output [11]: [customer_id#27, year_total#28, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46, year_total#64] +Input [12]: [customer_id#27, year_total#28, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46, customer_id#63, year_total#64] (67) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97] +Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#97), dynamicpruningexpression(cs_sold_date_sk#97 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(cs_sold_date_sk#53), dynamicpruningexpression(cs_sold_date_sk#53 IN dynamicpruning#30)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97] +Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] (69) Filter [codegen id : 28] -Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97] -Condition : isnotnull(cs_bill_customer_sk#92) +Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] +Condition : isnotnull(cs_bill_customer_sk#48) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#98, d_year#99] +Output [2]: [d_date_sk#8, d_year#9] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#97] -Right keys [1]: [d_date_sk#98] +Left keys [1]: [cs_sold_date_sk#53] +Right keys [1]: [d_date_sk#8] Join condition: None (72) Project [codegen id : 28] -Output [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] -Input [8]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97, d_date_sk#98, d_year#99] +Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Input [8]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53, d_date_sk#8, d_year#9] (73) Exchange -Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] -Arguments: hashpartitioning(cs_bill_customer_sk#92, 5), ENSURE_REQUIREMENTS, [id=#100] +Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Arguments: hashpartitioning(cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, [id=#66] (74) Sort [codegen id : 29] -Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] -Arguments: [cs_bill_customer_sk#92 ASC NULLS FIRST], false, 0 +Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Arguments: [cs_bill_customer_sk#48 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] +Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (76) Sort [codegen id : 31] -Input [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] -Arguments: [c_customer_sk#101 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [cs_bill_customer_sk#92] -Right keys [1]: [c_customer_sk#101] +Left keys [1]: [cs_bill_customer_sk#48] +Right keys [1]: [c_customer_sk#12] Join condition: None (78) Project [codegen id : 32] -Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] -Input [14]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99, c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] +Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Input [14]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (79) HashAggregate [codegen id : 32] -Input [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] -Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#109, isEmpty#110] -Results [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, sum#111, isEmpty#112] +Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#67, isEmpty#68] +Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#69, isEmpty#70] (80) Exchange -Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, sum#111, isEmpty#112] -Arguments: hashpartitioning(c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, 5), ENSURE_REQUIREMENTS, [id=#113] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#69, isEmpty#70] +Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#71] (81) HashAggregate [codegen id : 33] -Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, sum#111, isEmpty#112] -Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 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#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#114] -Results [2]: [c_customer_id#102 AS customer_id#115, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#114 AS year_total#116] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#69, isEmpty#70] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 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#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#72] +Results [2]: [c_customer_id#13 AS customer_id#73, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#72 AS year_total#74] (82) Exchange -Input [2]: [customer_id#115, year_total#116] -Arguments: hashpartitioning(customer_id#115, 5), ENSURE_REQUIREMENTS, [id=#117] +Input [2]: [customer_id#73, year_total#74] +Arguments: hashpartitioning(customer_id#73, 5), ENSURE_REQUIREMENTS, [id=#75] (83) Sort [codegen id : 34] -Input [2]: [customer_id#115, year_total#116] -Arguments: [customer_id#115 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#73, year_total#74] +Arguments: [customer_id#73 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#115] -Join condition: (CASE WHEN (year_total#90 > 0.000000) THEN CheckOverflow((promote_precision(year_total#116) / promote_precision(year_total#90)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#28 > 0.000000) THEN CheckOverflow((promote_precision(year_total#62) / promote_precision(year_total#28)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#73] +Join condition: (CASE WHEN (year_total#64 > 0.000000) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#64)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#28 > 0.000000) THEN CheckOverflow((promote_precision(year_total#46) / promote_precision(year_total#28)), DecimalType(38,14), true) ELSE null END) (85) Project [codegen id : 35] -Output [10]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116] -Input [13]: [customer_id#27, year_total#28, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62, year_total#90, customer_id#115, year_total#116] +Output [10]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74] +Input [13]: [customer_id#27, year_total#28, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46, year_total#64, customer_id#73, year_total#74] (86) Scan parquet default.web_sales -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] +Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#123), dynamicpruningexpression(ws_sold_date_sk#123 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 37] -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] +Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] (88) Filter [codegen id : 37] -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) +Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Condition : isnotnull(ws_bill_customer_sk#76) (89) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#124, d_year#125] +Output [2]: [d_date_sk#8, d_year#9] (90) BroadcastHashJoin [codegen id : 37] -Left keys [1]: [ws_sold_date_sk#123] -Right keys [1]: [d_date_sk#124] +Left keys [1]: [ws_sold_date_sk#81] +Right keys [1]: [d_date_sk#8] Join condition: None (91) Project [codegen id : 37] -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, d_year#125] -Input [8]: [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, d_date_sk#124, d_year#125] +Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Input [8]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#8, d_year#9] (92) Exchange -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, d_year#125] -Arguments: hashpartitioning(ws_bill_customer_sk#118, 5), ENSURE_REQUIREMENTS, [id=#126] +Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Arguments: hashpartitioning(ws_bill_customer_sk#76, 5), ENSURE_REQUIREMENTS, [id=#82] (93) Sort [codegen id : 38] -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, d_year#125] -Arguments: [ws_bill_customer_sk#118 ASC NULLS FIRST], false, 0 +Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Arguments: [ws_bill_customer_sk#76 ASC NULLS FIRST], false, 0 (94) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#127, c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134] +Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (95) Sort [codegen id : 40] -Input [8]: [c_customer_sk#127, c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134] -Arguments: [c_customer_sk#127 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 (96) SortMergeJoin [codegen id : 41] -Left keys [1]: [ws_bill_customer_sk#118] -Right keys [1]: [c_customer_sk#127] +Left keys [1]: [ws_bill_customer_sk#76] +Right keys [1]: [c_customer_sk#12] Join condition: None (97) Project [codegen id : 41] -Output [12]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, d_year#125] -Input [14]: [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, d_year#125, c_customer_sk#127, c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134] +Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Input [14]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (98) HashAggregate [codegen id : 41] -Input [12]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, d_year#125] -Keys [8]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125] -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#135, isEmpty#136] -Results [10]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, sum#137, isEmpty#138] +Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#83, isEmpty#84] +Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#85, isEmpty#86] (99) Exchange -Input [10]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, sum#137, isEmpty#138] -Arguments: hashpartitioning(c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, 5), ENSURE_REQUIREMENTS, [id=#139] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#85, isEmpty#86] +Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#87] (100) HashAggregate [codegen id : 42] -Input [10]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, sum#137, isEmpty#138] -Keys [8]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125] -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))#140] -Results [2]: [c_customer_id#128 AS customer_id#141, 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))#140 AS year_total#142] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#85, isEmpty#86] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 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#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#88] +Results [2]: [c_customer_id#13 AS customer_id#89, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#88 AS year_total#90] (101) Filter [codegen id : 42] -Input [2]: [customer_id#141, year_total#142] -Condition : (isnotnull(year_total#142) AND (year_total#142 > 0.000000)) +Input [2]: [customer_id#89, year_total#90] +Condition : (isnotnull(year_total#90) AND (year_total#90 > 0.000000)) (102) Project [codegen id : 42] -Output [2]: [customer_id#141 AS customer_id#143, year_total#142 AS year_total#144] -Input [2]: [customer_id#141, year_total#142] +Output [2]: [customer_id#89 AS customer_id#91, year_total#90 AS year_total#92] +Input [2]: [customer_id#89, year_total#90] (103) Exchange -Input [2]: [customer_id#143, year_total#144] -Arguments: hashpartitioning(customer_id#143, 5), ENSURE_REQUIREMENTS, [id=#145] +Input [2]: [customer_id#91, year_total#92] +Arguments: hashpartitioning(customer_id#91, 5), ENSURE_REQUIREMENTS, [id=#93] (104) Sort [codegen id : 43] -Input [2]: [customer_id#143, year_total#144] -Arguments: [customer_id#143 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#91, year_total#92] +Arguments: [customer_id#91 ASC NULLS FIRST], false, 0 (105) SortMergeJoin [codegen id : 44] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#143] +Right keys [1]: [customer_id#91] Join condition: None (106) Project [codegen id : 44] -Output [11]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116, year_total#144] -Input [12]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116, customer_id#143, year_total#144] +Output [11]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74, year_total#92] +Input [12]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74, customer_id#91, year_total#92] (107) 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#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#151), dynamicpruningexpression(ws_sold_date_sk#151 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 46] -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#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] (109) Filter [codegen id : 46] -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#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] +Condition : isnotnull(ws_bill_customer_sk#76) (110) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#152, d_year#153] +Output [2]: [d_date_sk#8, d_year#9] (111) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [ws_sold_date_sk#151] -Right keys [1]: [d_date_sk#152] +Left keys [1]: [ws_sold_date_sk#81] +Right keys [1]: [d_date_sk#8] Join condition: None (112) Project [codegen id : 46] -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, d_year#153] -Input [8]: [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, d_date_sk#152, d_year#153] +Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Input [8]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#8, d_year#9] (113) Exchange -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, d_year#153] -Arguments: hashpartitioning(ws_bill_customer_sk#146, 5), ENSURE_REQUIREMENTS, [id=#154] +Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Arguments: hashpartitioning(ws_bill_customer_sk#76, 5), ENSURE_REQUIREMENTS, [id=#94] (114) Sort [codegen id : 47] -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, d_year#153] -Arguments: [ws_bill_customer_sk#146 ASC NULLS FIRST], false, 0 +Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Arguments: [ws_bill_customer_sk#76 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] +Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (116) Sort [codegen id : 49] -Input [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] -Arguments: [c_customer_sk#155 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 (117) SortMergeJoin [codegen id : 50] -Left keys [1]: [ws_bill_customer_sk#146] -Right keys [1]: [c_customer_sk#155] +Left keys [1]: [ws_bill_customer_sk#76] +Right keys [1]: [c_customer_sk#12] Join condition: None (118) Project [codegen id : 50] -Output [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] -Input [14]: [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, d_year#153, c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] +Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Input [14]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] (119) HashAggregate [codegen id : 50] -Input [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] -Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] -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#163, isEmpty#164] -Results [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] +Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#95, isEmpty#96] +Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#97, isEmpty#98] (120) Exchange -Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] -Arguments: hashpartitioning(c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, 5), ENSURE_REQUIREMENTS, [id=#167] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#97, isEmpty#98] +Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#99] (121) HashAggregate [codegen id : 51] -Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] -Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] -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))#168] -Results [2]: [c_customer_id#156 AS customer_id#169, 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))#168 AS year_total#170] +Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#97, isEmpty#98] +Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 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#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#100] +Results [2]: [c_customer_id#13 AS customer_id#101, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#100 AS year_total#102] (122) Exchange -Input [2]: [customer_id#169, year_total#170] -Arguments: hashpartitioning(customer_id#169, 5), ENSURE_REQUIREMENTS, [id=#171] +Input [2]: [customer_id#101, year_total#102] +Arguments: hashpartitioning(customer_id#101, 5), ENSURE_REQUIREMENTS, [id=#103] (123) Sort [codegen id : 52] -Input [2]: [customer_id#169, year_total#170] -Arguments: [customer_id#169 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#101, year_total#102] +Arguments: [customer_id#101 ASC NULLS FIRST], false, 0 (124) SortMergeJoin [codegen id : 53] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#169] -Join condition: (CASE WHEN (year_total#90 > 0.000000) THEN CheckOverflow((promote_precision(year_total#116) / promote_precision(year_total#90)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#144 > 0.000000) THEN CheckOverflow((promote_precision(year_total#170) / promote_precision(year_total#144)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#101] +Join condition: (CASE WHEN (year_total#64 > 0.000000) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#64)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#92 > 0.000000) THEN CheckOverflow((promote_precision(year_total#102) / promote_precision(year_total#92)), DecimalType(38,14), true) ELSE null END) (125) Project [codegen id : 53] -Output [7]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61] -Input [13]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116, year_total#144, customer_id#169, year_total#170] +Output [7]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45] +Input [13]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74, year_total#92, customer_id#101, year_total#102] (126) TakeOrderedAndProject -Input [7]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61] -Arguments: 100, [customer_id#55 ASC NULLS FIRST, customer_first_name#56 ASC NULLS FIRST, customer_last_name#57 ASC NULLS FIRST, customer_preferred_cust_flag#58 ASC NULLS FIRST, customer_birth_country#59 ASC NULLS FIRST, customer_login#60 ASC NULLS FIRST, customer_email_address#61 ASC NULLS FIRST], [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61] +Input [7]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45] +Arguments: 100, [customer_id#39 ASC NULLS FIRST, customer_first_name#40 ASC NULLS FIRST, customer_last_name#41 ASC NULLS FIRST, customer_preferred_cust_flag#42 ASC NULLS FIRST, customer_birth_country#43 ASC NULLS FIRST, customer_login#44 ASC NULLS FIRST, customer_email_address#45 ASC NULLS FIRST], [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45] ===== Subqueries ===== @@ -708,19 +708,19 @@ ReusedExchange (127) (127) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#8, d_year#9] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#30 ReusedExchange (128) (128) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#37, d_year#38] +Output [2]: [d_date_sk#8, d_year#9] -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#69 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#7 -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#97 IN dynamicpruning#36 +Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#30 -Subquery:5 Hosting operator id = 86 Hosting Expression = ws_sold_date_sk#123 IN dynamicpruning#7 +Subquery:5 Hosting operator id = 86 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#7 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#151 IN dynamicpruning#36 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#30 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 1cf27630d2994..80dee3d164511 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 @@ -209,460 +209,460 @@ Input [2]: [customer_id#26, year_total#27] Condition : (isnotnull(year_total#27) AND (year_total#27 > 0.000000)) (20) Scan parquet default.customer -Output [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] -Condition : (isnotnull(c_customer_sk#28) AND isnotnull(c_customer_id#29)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (23) Scan parquet default.store_sales -Output [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] +Output [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#41), dynamicpruningexpression(ss_sold_date_sk#41 IN dynamicpruning#42)] +PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] +Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] (25) Filter [codegen id : 4] -Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Condition : isnotnull(ss_customer_sk#36) +Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Condition : isnotnull(ss_customer_sk#9) (26) BroadcastExchange -Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] +Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#28] -Right keys [1]: [ss_customer_sk#36] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#9] Join condition: None (28) Project [codegen id : 6] -Output [12]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] -Input [14]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#44, d_year#45] +Output [2]: [d_date_sk#17, d_year#18] 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 (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#44, d_year#45] +Input [2]: [d_date_sk#17, d_year#18] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#44, d_year#45] -Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) +Input [2]: [d_date_sk#17, d_year#18] +Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2002)) AND isnotnull(d_date_sk#17)) (32) BroadcastExchange -Input [2]: [d_date_sk#44, d_year#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#46] +Input [2]: [d_date_sk#17, d_year#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#41] -Right keys [1]: [d_date_sk#44] +Left keys [1]: [ss_sold_date_sk#14] +Right keys [1]: [d_date_sk#17] Join condition: None (34) Project [codegen id : 6] -Output [12]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#45] -Input [14]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41, d_date_sk#44, d_year#45] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#18] +Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14, d_date_sk#17, d_year#18] (35) HashAggregate [codegen id : 6] -Input [12]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#45] -Keys [8]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#47, isEmpty#48] -Results [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, sum#49, isEmpty#50] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#18] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#31, isEmpty#32] +Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#33, isEmpty#34] (36) Exchange -Input [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, sum#49, isEmpty#50] -Arguments: hashpartitioning(c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#33, isEmpty#34] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#35] (37) HashAggregate [codegen id : 7] -Input [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, sum#49, isEmpty#50] -Keys [8]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 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#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#52] -Results [8]: [c_customer_id#29 AS customer_id#53, c_first_name#30 AS customer_first_name#54, c_last_name#31 AS customer_last_name#55, c_preferred_cust_flag#32 AS customer_preferred_cust_flag#56, c_birth_country#33 AS customer_birth_country#57, c_login#34 AS customer_login#58, c_email_address#35 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#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#52 AS year_total#60] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#33, isEmpty#34] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 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#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#36] +Results [8]: [c_customer_id#2 AS customer_id#37, c_first_name#3 AS customer_first_name#38, c_last_name#4 AS customer_last_name#39, c_preferred_cust_flag#5 AS customer_preferred_cust_flag#40, c_birth_country#6 AS customer_birth_country#41, c_login#7 AS customer_login#42, c_email_address#8 AS customer_email_address#43, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#36 AS year_total#44] (38) BroadcastExchange -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: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#61] +Input [8]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#45] (39) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#53] +Right keys [1]: [customer_id#37] Join condition: None (40) Scan parquet default.customer -Output [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (42) Filter [codegen id : 10] -Input [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] -Condition : (isnotnull(c_customer_sk#62) AND isnotnull(c_customer_id#63)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (43) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] +Output [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#75), dynamicpruningexpression(cs_sold_date_sk#75 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cs_sold_date_sk#51), dynamicpruningexpression(cs_sold_date_sk#51 IN dynamicpruning#15)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] +Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] (45) Filter [codegen id : 8] -Input [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] -Condition : isnotnull(cs_bill_customer_sk#70) +Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Condition : isnotnull(cs_bill_customer_sk#46) (46) BroadcastExchange -Input [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] +Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#62] -Right keys [1]: [cs_bill_customer_sk#70] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [cs_bill_customer_sk#46] Join condition: None (48) Project [codegen id : 10] -Output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] -Input [14]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#77, d_year#78] +Output [2]: [d_date_sk#17, d_year#18] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#75] -Right keys [1]: [d_date_sk#77] +Left keys [1]: [cs_sold_date_sk#51] +Right keys [1]: [d_date_sk#17] Join condition: None (51) Project [codegen id : 10] -Output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, d_year#78] -Input [14]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75, d_date_sk#77, d_year#78] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] +Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51, d_date_sk#17, d_year#18] (52) HashAggregate [codegen id : 10] -Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, d_year#78] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 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#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, sum#81, isEmpty#82] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#53, isEmpty#54] +Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#55, isEmpty#56] (53) Exchange -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, sum#81, isEmpty#82] -Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#55, isEmpty#56] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#57] (54) HashAggregate [codegen id : 11] -Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, sum#81, isEmpty#82] -Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 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#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#84] -Results [2]: [c_customer_id#63 AS customer_id#85, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#84 AS year_total#86] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#55, isEmpty#56] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 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#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#58] +Results [2]: [c_customer_id#2 AS customer_id#59, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#58 AS year_total#60] (55) Filter [codegen id : 11] -Input [2]: [customer_id#85, year_total#86] -Condition : (isnotnull(year_total#86) AND (year_total#86 > 0.000000)) +Input [2]: [customer_id#59, year_total#60] +Condition : (isnotnull(year_total#60) AND (year_total#60 > 0.000000)) (56) Project [codegen id : 11] -Output [2]: [customer_id#85 AS customer_id#87, year_total#86 AS year_total#88] -Input [2]: [customer_id#85, year_total#86] +Output [2]: [customer_id#59 AS customer_id#61, year_total#60 AS year_total#62] +Input [2]: [customer_id#59, year_total#60] (57) BroadcastExchange -Input [2]: [customer_id#87, year_total#88] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#89] +Input [2]: [customer_id#61, year_total#62] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#63] (58) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#87] +Right keys [1]: [customer_id#61] Join condition: None (59) Project [codegen id : 24] -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#88] -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#87, year_total#88] +Output [11]: [customer_id#26, year_total#27, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44, year_total#62] +Input [12]: [customer_id#26, year_total#27, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44, customer_id#61, year_total#62] (60) Scan parquet default.customer -Output [8]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (62) Filter [codegen id : 14] -Input [8]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97] -Condition : (isnotnull(c_customer_sk#90) AND isnotnull(c_customer_id#91)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (63) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] +Output [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#103), dynamicpruningexpression(cs_sold_date_sk#103 IN dynamicpruning#42)] +PartitionFilters: [isnotnull(cs_sold_date_sk#51), dynamicpruningexpression(cs_sold_date_sk#51 IN dynamicpruning#28)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] +Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] (65) Filter [codegen id : 12] -Input [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] -Condition : isnotnull(cs_bill_customer_sk#98) +Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Condition : isnotnull(cs_bill_customer_sk#46) (66) BroadcastExchange -Input [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#104] +Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#90] -Right keys [1]: [cs_bill_customer_sk#98] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [cs_bill_customer_sk#46] Join condition: None (68) Project [codegen id : 14] -Output [12]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] -Input [14]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#105, d_year#106] +Output [2]: [d_date_sk#17, d_year#18] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#103] -Right keys [1]: [d_date_sk#105] +Left keys [1]: [cs_sold_date_sk#51] +Right keys [1]: [d_date_sk#17] Join condition: None (71) Project [codegen id : 14] -Output [12]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, d_year#106] -Input [14]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103, d_date_sk#105, d_year#106] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] +Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51, d_date_sk#17, d_year#18] (72) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, d_year#106] -Keys [8]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#107, isEmpty#108] -Results [10]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, sum#109, isEmpty#110] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#67, isEmpty#68] (73) Exchange -Input [10]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, sum#109, isEmpty#110] -Arguments: hashpartitioning(c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, 5), ENSURE_REQUIREMENTS, [id=#111] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#67, isEmpty#68] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#69] (74) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, sum#109, isEmpty#110] -Keys [8]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 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#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112] -Results [2]: [c_customer_id#91 AS customer_id#113, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112 AS year_total#114] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#67, isEmpty#68] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 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#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#70] +Results [2]: [c_customer_id#2 AS customer_id#71, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#70 AS year_total#72] (75) BroadcastExchange -Input [2]: [customer_id#113, year_total#114] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#115] +Input [2]: [customer_id#71, year_total#72] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#73] (76) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#113] -Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), 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#71] +Join condition: (CASE WHEN (year_total#62 > 0.000000) THEN CheckOverflow((promote_precision(year_total#72) / promote_precision(year_total#62)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#27 > 0.000000) THEN CheckOverflow((promote_precision(year_total#44) / promote_precision(year_total#27)), DecimalType(38,14), true) ELSE null END) (77) Project [codegen id : 24] -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#88, year_total#114] -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#88, customer_id#113, year_total#114] +Output [10]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72] +Input [13]: [customer_id#26, year_total#27, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44, year_total#62, customer_id#71, year_total#72] (78) Scan parquet default.customer -Output [8]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (79) ColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (80) Filter [codegen id : 18] -Input [8]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123] -Condition : (isnotnull(c_customer_sk#116) AND isnotnull(c_customer_id#117)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (81) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] +Output [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#129), dynamicpruningexpression(ws_sold_date_sk#129 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#15)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (82) ColumnarToRow [codegen id : 16] -Input [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] +Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] (83) Filter [codegen id : 16] -Input [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] -Condition : isnotnull(ws_bill_customer_sk#124) +Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Condition : isnotnull(ws_bill_customer_sk#74) (84) BroadcastExchange -Input [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#130] +Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#80] (85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#116] -Right keys [1]: [ws_bill_customer_sk#124] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#74] Join condition: None (86) Project [codegen id : 18] -Output [12]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] -Input [14]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] (87) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#131, d_year#132] +Output [2]: [d_date_sk#17, d_year#18] (88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#129] -Right keys [1]: [d_date_sk#131] +Left keys [1]: [ws_sold_date_sk#79] +Right keys [1]: [d_date_sk#17] Join condition: None (89) Project [codegen id : 18] -Output [12]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, d_year#132] -Input [14]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129, d_date_sk#131, d_year#132] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] +Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#17, d_year#18] (90) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, d_year#132] -Keys [8]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#133, isEmpty#134] -Results [10]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, sum#135, isEmpty#136] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#81, isEmpty#82] +Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#83, isEmpty#84] (91) Exchange -Input [10]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, sum#135, isEmpty#136] -Arguments: hashpartitioning(c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#83, isEmpty#84] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#85] (92) HashAggregate [codegen id : 19] -Input [10]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, sum#135, isEmpty#136] -Keys [8]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 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#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138] -Results [2]: [c_customer_id#117 AS customer_id#139, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138 AS year_total#140] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#83, isEmpty#84] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 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#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86] +Results [2]: [c_customer_id#2 AS customer_id#87, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86 AS year_total#88] (93) Filter [codegen id : 19] -Input [2]: [customer_id#139, year_total#140] -Condition : (isnotnull(year_total#140) AND (year_total#140 > 0.000000)) +Input [2]: [customer_id#87, year_total#88] +Condition : (isnotnull(year_total#88) AND (year_total#88 > 0.000000)) (94) Project [codegen id : 19] -Output [2]: [customer_id#139 AS customer_id#141, year_total#140 AS year_total#142] -Input [2]: [customer_id#139, year_total#140] +Output [2]: [customer_id#87 AS customer_id#89, year_total#88 AS year_total#90] +Input [2]: [customer_id#87, year_total#88] (95) BroadcastExchange -Input [2]: [customer_id#141, year_total#142] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#143] +Input [2]: [customer_id#89, year_total#90] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#91] (96) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#141] +Right keys [1]: [customer_id#89] Join condition: None (97) Project [codegen id : 24] -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#88, year_total#114, year_total#142] -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#88, year_total#114, customer_id#141, year_total#142] +Output [11]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72, year_total#90] +Input [12]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72, customer_id#89, year_total#90] (98) Scan parquet default.customer -Output [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (99) ColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (100) Filter [codegen id : 22] -Input [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] -Condition : (isnotnull(c_customer_sk#144) AND isnotnull(c_customer_id#145)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (101) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Output [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#157), dynamicpruningexpression(ws_sold_date_sk#157 IN dynamicpruning#42)] +PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 20] -Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] (103) Filter [codegen id : 20] -Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] -Condition : isnotnull(ws_bill_customer_sk#152) +Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Condition : isnotnull(ws_bill_customer_sk#74) (104) BroadcastExchange -Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#158] +Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] (105) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#144] -Right keys [1]: [ws_bill_customer_sk#152] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#74] Join condition: None (106) Project [codegen id : 22] -Output [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] -Input [14]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] (107) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#159, d_year#160] +Output [2]: [d_date_sk#17, d_year#18] (108) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#157] -Right keys [1]: [d_date_sk#159] +Left keys [1]: [ws_sold_date_sk#79] +Right keys [1]: [d_date_sk#17] Join condition: None (109) Project [codegen id : 22] -Output [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, d_year#160] -Input [14]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157, d_date_sk#159, d_year#160] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] +Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#17, d_year#18] (110) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, d_year#160] -Keys [8]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#161, isEmpty#162] -Results [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#93, isEmpty#94] +Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#95, isEmpty#96] (111) Exchange -Input [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] -Arguments: hashpartitioning(c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, 5), ENSURE_REQUIREMENTS, [id=#165] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#95, isEmpty#96] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#97] (112) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] -Keys [8]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 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#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166] -Results [2]: [c_customer_id#145 AS customer_id#167, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166 AS year_total#168] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#95, isEmpty#96] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 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#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#98] +Results [2]: [c_customer_id#2 AS customer_id#99, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#98 AS year_total#100] (113) BroadcastExchange -Input [2]: [customer_id#167, year_total#168] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#169] +Input [2]: [customer_id#99, year_total#100] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#101] (114) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#167] -Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#142 > 0.000000) THEN CheckOverflow((promote_precision(year_total#168) / promote_precision(year_total#142)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#99] +Join condition: (CASE WHEN (year_total#62 > 0.000000) THEN CheckOverflow((promote_precision(year_total#72) / promote_precision(year_total#62)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#90 > 0.000000) THEN CheckOverflow((promote_precision(year_total#100) / promote_precision(year_total#90)), DecimalType(38,14), true) ELSE null END) (115) Project [codegen id : 24] -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#88, year_total#114, year_total#142, customer_id#167, year_total#168] +Output [7]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43] +Input [13]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72, year_total#90, customer_id#99, year_total#100] (116) 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#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43] +Arguments: 100, [customer_id#37 ASC NULLS FIRST, customer_first_name#38 ASC NULLS FIRST, customer_last_name#39 ASC NULLS FIRST, customer_preferred_cust_flag#40 ASC NULLS FIRST, customer_birth_country#41 ASC NULLS FIRST, customer_login#42 ASC NULLS FIRST, customer_email_address#43 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43] ===== Subqueries ===== @@ -673,19 +673,19 @@ ReusedExchange (117) (117) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#17, d_year#18] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#41 IN dynamicpruning#42 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#28 ReusedExchange (118) (118) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#44, d_year#45] +Output [2]: [d_date_sk#17, d_year#18] -Subquery:3 Hosting operator id = 43 Hosting Expression = cs_sold_date_sk#75 IN dynamicpruning#15 +Subquery:3 Hosting operator id = 43 Hosting Expression = cs_sold_date_sk#51 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 63 Hosting Expression = cs_sold_date_sk#103 IN dynamicpruning#42 +Subquery:4 Hosting operator id = 63 Hosting Expression = cs_sold_date_sk#51 IN dynamicpruning#28 -Subquery:5 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#129 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#157 IN dynamicpruning#42 +Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#28 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt index 13d73e61e1443..0c227d75c67c4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt @@ -40,61 +40,66 @@ Output [2]: [i_manufact#2, i_product_name#3] Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] (5) Scan parquet default.item -Output [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [Or(Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,powder),EqualTo(i_color,khaki)),Or(EqualTo(i_units,Ounce),EqualTo(i_units,Oz))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,brown),EqualTo(i_color,honeydew)),Or(EqualTo(i_units,Bunch),EqualTo(i_units,Ton))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,floral),EqualTo(i_color,deep)),Or(EqualTo(i_units,N/A),EqualTo(i_units,Dozen))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,light),EqualTo(i_color,cornflower)),Or(EqualTo(i_units,Box),EqualTo(i_units,Pound))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large)))))),Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,midnight),EqualTo(i_color,snow)),Or(EqualTo(i_units,Pallet),EqualTo(i_units,Gross))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,cyan),EqualTo(i_color,papaya)),Or(EqualTo(i_units,Cup),EqualTo(i_units,Dram))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,orange),EqualTo(i_color,frosted)),Or(EqualTo(i_units,Each),EqualTo(i_units,Tbl))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,forest),EqualTo(i_color,ghost)),Or(EqualTo(i_units,Lb),EqualTo(i_units,Bundle))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))))))), IsNotNull(i_manufact)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] (7) Filter [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Condition : (((((i_category#4 = Women) AND (((((i_color#6 = powder) OR (i_color#6 = khaki)) AND ((i_units#7 = Ounce) OR (i_units#7 = Oz))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = brown) OR (i_color#6 = honeydew)) AND ((i_units#7 = Bunch) OR (i_units#7 = Ton))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = floral) OR (i_color#6 = deep)) AND ((i_units#7 = N/A) OR (i_units#7 = Dozen))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = light) OR (i_color#6 = cornflower)) AND ((i_units#7 = Box) OR (i_units#7 = Pound))) AND ((i_size#5 = medium) OR (i_size#5 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#6 = midnight) OR (i_color#6 = snow)) AND ((i_units#7 = Pallet) OR (i_units#7 = Gross))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = cyan) OR (i_color#6 = papaya)) AND ((i_units#7 = Cup) OR (i_units#7 = Dram))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = orange) OR (i_color#6 = frosted)) AND ((i_units#7 = Each) OR (i_units#7 = Tbl))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = forest) OR (i_color#6 = ghost)) AND ((i_units#7 = Lb) OR (i_units#7 = Bundle))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))))))) AND isnotnull(i_manufact#2)) +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Condition : (((((i_category#4 = Women) AND (((((i_color#7 = powder) OR (i_color#7 = khaki)) AND ((i_units#8 = Ounce) OR (i_units#8 = Oz))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = brown) OR (i_color#7 = honeydew)) AND ((i_units#8 = Bunch) OR (i_units#8 = Ton))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = floral) OR (i_color#7 = deep)) AND ((i_units#8 = N/A) OR (i_units#8 = Dozen))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = light) OR (i_color#7 = cornflower)) AND ((i_units#8 = Box) OR (i_units#8 = Pound))) AND ((i_size#6 = medium) OR (i_size#6 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#7 = midnight) OR (i_color#7 = snow)) AND ((i_units#8 = Pallet) OR (i_units#8 = Gross))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = cyan) OR (i_color#7 = papaya)) AND ((i_units#8 = Cup) OR (i_units#8 = Dram))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = orange) OR (i_color#7 = frosted)) AND ((i_units#8 = Each) OR (i_units#8 = Tbl))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = forest) OR (i_color#7 = ghost)) AND ((i_units#8 = Lb) OR (i_units#8 = Bundle))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))))))) AND isnotnull(i_manufact#5)) (8) Project [codegen id : 1] -Output [1]: [i_manufact#2] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Output [1]: [i_manufact#5] +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] (9) HashAggregate [codegen id : 1] -Input [1]: [i_manufact#2] -Keys [1]: [i_manufact#2] +Input [1]: [i_manufact#5] +Keys [1]: [i_manufact#5] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#8] -Results [2]: [i_manufact#2, count#9] +Aggregate Attributes [1]: [count#9] +Results [2]: [i_manufact#5, count#10] (10) Exchange +<<<<<<< HEAD +Input [2]: [i_manufact#5, count#10] +Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [id=#11] +======= Input [2]: [i_manufact#2, count#9] -Arguments: hashpartitioning(i_manufact#2, 5), true, [id=#10] +Arguments: hashpartitioning(i_manufact#2, 5), ENSURE_REQUIREMENTS, [id=#10] +>>>>>>> abfd9b23cd7c21e9525df85a16e0611ef0f35908 (11) HashAggregate [codegen id : 2] -Input [2]: [i_manufact#2, count#9] -Keys [1]: [i_manufact#2] +Input [2]: [i_manufact#5, count#10] +Keys [1]: [i_manufact#5] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#11] -Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13] +Aggregate Attributes [1]: [count(1)#12] +Results [2]: [count(1)#12 AS item_cnt#13, i_manufact#5] (12) Filter [codegen id : 2] -Input [2]: [item_cnt#12, i_manufact#2#13] -Condition : (item_cnt#12 > 0) +Input [2]: [item_cnt#13, i_manufact#5] +Condition : (item_cnt#13 > 0) (13) Project [codegen id : 2] -Output [1]: [i_manufact#2#13] -Input [2]: [item_cnt#12, i_manufact#2#13] +Output [1]: [i_manufact#5] +Input [2]: [item_cnt#13, i_manufact#5] (14) BroadcastExchange -Input [1]: [i_manufact#2#13] +Input [1]: [i_manufact#5] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14] (15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_manufact#2] -Right keys [1]: [i_manufact#2#13] +Right keys [1]: [i_manufact#5] Join condition: None (16) Project [codegen id : 3] Output [1]: [i_product_name#3] -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#2#13] +Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] (17) HashAggregate [codegen id : 3] Input [1]: [i_product_name#3] @@ -105,7 +110,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] +Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index 13d73e61e1443..0c227d75c67c4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -40,61 +40,66 @@ Output [2]: [i_manufact#2, i_product_name#3] Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] (5) Scan parquet default.item -Output [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [Or(Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,powder),EqualTo(i_color,khaki)),Or(EqualTo(i_units,Ounce),EqualTo(i_units,Oz))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,brown),EqualTo(i_color,honeydew)),Or(EqualTo(i_units,Bunch),EqualTo(i_units,Ton))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,floral),EqualTo(i_color,deep)),Or(EqualTo(i_units,N/A),EqualTo(i_units,Dozen))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,light),EqualTo(i_color,cornflower)),Or(EqualTo(i_units,Box),EqualTo(i_units,Pound))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large)))))),Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,midnight),EqualTo(i_color,snow)),Or(EqualTo(i_units,Pallet),EqualTo(i_units,Gross))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,cyan),EqualTo(i_color,papaya)),Or(EqualTo(i_units,Cup),EqualTo(i_units,Dram))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,orange),EqualTo(i_color,frosted)),Or(EqualTo(i_units,Each),EqualTo(i_units,Tbl))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,forest),EqualTo(i_color,ghost)),Or(EqualTo(i_units,Lb),EqualTo(i_units,Bundle))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))))))), IsNotNull(i_manufact)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] (7) Filter [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Condition : (((((i_category#4 = Women) AND (((((i_color#6 = powder) OR (i_color#6 = khaki)) AND ((i_units#7 = Ounce) OR (i_units#7 = Oz))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = brown) OR (i_color#6 = honeydew)) AND ((i_units#7 = Bunch) OR (i_units#7 = Ton))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = floral) OR (i_color#6 = deep)) AND ((i_units#7 = N/A) OR (i_units#7 = Dozen))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = light) OR (i_color#6 = cornflower)) AND ((i_units#7 = Box) OR (i_units#7 = Pound))) AND ((i_size#5 = medium) OR (i_size#5 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#6 = midnight) OR (i_color#6 = snow)) AND ((i_units#7 = Pallet) OR (i_units#7 = Gross))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = cyan) OR (i_color#6 = papaya)) AND ((i_units#7 = Cup) OR (i_units#7 = Dram))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = orange) OR (i_color#6 = frosted)) AND ((i_units#7 = Each) OR (i_units#7 = Tbl))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = forest) OR (i_color#6 = ghost)) AND ((i_units#7 = Lb) OR (i_units#7 = Bundle))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))))))) AND isnotnull(i_manufact#2)) +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Condition : (((((i_category#4 = Women) AND (((((i_color#7 = powder) OR (i_color#7 = khaki)) AND ((i_units#8 = Ounce) OR (i_units#8 = Oz))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = brown) OR (i_color#7 = honeydew)) AND ((i_units#8 = Bunch) OR (i_units#8 = Ton))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = floral) OR (i_color#7 = deep)) AND ((i_units#8 = N/A) OR (i_units#8 = Dozen))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = light) OR (i_color#7 = cornflower)) AND ((i_units#8 = Box) OR (i_units#8 = Pound))) AND ((i_size#6 = medium) OR (i_size#6 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#7 = midnight) OR (i_color#7 = snow)) AND ((i_units#8 = Pallet) OR (i_units#8 = Gross))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = cyan) OR (i_color#7 = papaya)) AND ((i_units#8 = Cup) OR (i_units#8 = Dram))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = orange) OR (i_color#7 = frosted)) AND ((i_units#8 = Each) OR (i_units#8 = Tbl))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = forest) OR (i_color#7 = ghost)) AND ((i_units#8 = Lb) OR (i_units#8 = Bundle))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))))))) AND isnotnull(i_manufact#5)) (8) Project [codegen id : 1] -Output [1]: [i_manufact#2] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Output [1]: [i_manufact#5] +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] (9) HashAggregate [codegen id : 1] -Input [1]: [i_manufact#2] -Keys [1]: [i_manufact#2] +Input [1]: [i_manufact#5] +Keys [1]: [i_manufact#5] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#8] -Results [2]: [i_manufact#2, count#9] +Aggregate Attributes [1]: [count#9] +Results [2]: [i_manufact#5, count#10] (10) Exchange +<<<<<<< HEAD +Input [2]: [i_manufact#5, count#10] +Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [id=#11] +======= Input [2]: [i_manufact#2, count#9] -Arguments: hashpartitioning(i_manufact#2, 5), true, [id=#10] +Arguments: hashpartitioning(i_manufact#2, 5), ENSURE_REQUIREMENTS, [id=#10] +>>>>>>> abfd9b23cd7c21e9525df85a16e0611ef0f35908 (11) HashAggregate [codegen id : 2] -Input [2]: [i_manufact#2, count#9] -Keys [1]: [i_manufact#2] +Input [2]: [i_manufact#5, count#10] +Keys [1]: [i_manufact#5] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#11] -Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13] +Aggregate Attributes [1]: [count(1)#12] +Results [2]: [count(1)#12 AS item_cnt#13, i_manufact#5] (12) Filter [codegen id : 2] -Input [2]: [item_cnt#12, i_manufact#2#13] -Condition : (item_cnt#12 > 0) +Input [2]: [item_cnt#13, i_manufact#5] +Condition : (item_cnt#13 > 0) (13) Project [codegen id : 2] -Output [1]: [i_manufact#2#13] -Input [2]: [item_cnt#12, i_manufact#2#13] +Output [1]: [i_manufact#5] +Input [2]: [item_cnt#13, i_manufact#5] (14) BroadcastExchange -Input [1]: [i_manufact#2#13] +Input [1]: [i_manufact#5] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14] (15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_manufact#2] -Right keys [1]: [i_manufact#2#13] +Right keys [1]: [i_manufact#5] Join condition: None (16) Project [codegen id : 3] Output [1]: [i_product_name#3] -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#2#13] +Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] (17) HashAggregate [codegen id : 3] Input [1]: [i_product_name#3] @@ -105,7 +110,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] +Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt index 35924d2ffd9a9..fdb95bd5de9d3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt @@ -95,7 +95,7 @@ Arguments: [rank(rank_col#12) windowspecdefinition(rank_col#12 ASC NULLS FIRST, (13) Filter [codegen id : 10] Input [3]: [item_sk#11, rank_col#12, rnk#17] -Condition : ((isnotnull(rnk#17) AND (rnk#17 < 11)) AND isnotnull(item_sk#11)) +Condition : ((rnk#17 < 11) AND isnotnull(item_sk#11)) (14) Project [codegen id : 10] Output [2]: [item_sk#11, rnk#17] @@ -133,7 +133,7 @@ Arguments: [rank(rank_col#22) windowspecdefinition(rank_col#22 DESC NULLS LAST, (22) Filter [codegen id : 7] Input [3]: [item_sk#21, rank_col#22, rnk#25] -Condition : ((isnotnull(rnk#25) AND (rnk#25 < 11)) AND isnotnull(item_sk#21)) +Condition : ((rnk#25 < 11) AND isnotnull(item_sk#21)) (23) Project [codegen id : 7] Output [2]: [item_sk#21, rnk#25] @@ -141,7 +141,7 @@ Input [3]: [item_sk#21, rank_col#22, rnk#25] (24) BroadcastExchange Input [2]: [item_sk#21, rnk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#26] (25) BroadcastHashJoin [codegen id : 10] Left keys [1]: [rnk#17] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index 4ca133ffd838d..f52ad2e63762e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -96,7 +96,7 @@ Arguments: [rank(rank_col#12) windowspecdefinition(rank_col#12 ASC NULLS FIRST, (13) Filter [codegen id : 4] Input [3]: [item_sk#11, rank_col#12, rnk#17] -Condition : ((isnotnull(rnk#17) AND (rnk#17 < 11)) AND isnotnull(item_sk#11)) +Condition : ((rnk#17 < 11) AND isnotnull(item_sk#11)) (14) Project [codegen id : 4] Output [2]: [item_sk#11, rnk#17] @@ -138,7 +138,7 @@ Arguments: [rank(rank_col#22) windowspecdefinition(rank_col#22 DESC NULLS LAST, (23) Filter [codegen id : 8] Input [3]: [item_sk#21, rank_col#22, rnk#25] -Condition : ((isnotnull(rnk#25) AND (rnk#25 < 11)) AND isnotnull(item_sk#21)) +Condition : ((rnk#25 < 11) AND isnotnull(item_sk#21)) (24) Project [codegen id : 8] Output [2]: [item_sk#21, rnk#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt index 701414b22eb80..22f746eb1d91a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt @@ -199,30 +199,30 @@ Output [4]: [ws_sales_price#4, ca_city#19, ca_zip#20, i_item_id#12] Input [6]: [ws_bill_customer_sk#3, ws_sales_price#4, i_item_id#12, c_customer_sk#15, ca_city#19, ca_zip#20] (35) Scan parquet default.item -Output [2]: [i_item_sk#23, i_item_id#24] +Output [2]: [i_item_sk#11, i_item_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_item_sk, [2,3,5,7,11,13,17,19,23,29])] ReadSchema: struct (36) ColumnarToRow [codegen id : 11] -Input [2]: [i_item_sk#23, i_item_id#24] +Input [2]: [i_item_sk#11, i_item_id#12] (37) Filter [codegen id : 11] -Input [2]: [i_item_sk#23, i_item_id#24] -Condition : i_item_sk#23 IN (2,3,5,7,11,13,17,19,23,29) +Input [2]: [i_item_sk#11, i_item_id#12] +Condition : i_item_sk#11 IN (2,3,5,7,11,13,17,19,23,29) (38) Project [codegen id : 11] -Output [1]: [i_item_id#24] -Input [2]: [i_item_sk#23, i_item_id#24] +Output [1]: [i_item_id#12 AS i_item_id#12#23] +Input [2]: [i_item_sk#11, i_item_id#12] (39) BroadcastExchange -Input [1]: [i_item_id#24] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#25] +Input [1]: [i_item_id#12#23] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#24] (40) BroadcastHashJoin [codegen id : 12] Left keys [1]: [i_item_id#12] -Right keys [1]: [i_item_id#24] +Right keys [1]: [i_item_id#12#23] Join condition: None (41) Filter [codegen id : 12] @@ -237,23 +237,23 @@ Input [5]: [ws_sales_price#4, ca_city#19, ca_zip#20, i_item_id#12, exists#1] Input [3]: [ws_sales_price#4, ca_city#19, ca_zip#20] Keys [2]: [ca_zip#20, ca_city#19] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#26] -Results [3]: [ca_zip#20, ca_city#19, sum#27] +Aggregate Attributes [1]: [sum#25] +Results [3]: [ca_zip#20, ca_city#19, sum#26] (44) Exchange -Input [3]: [ca_zip#20, ca_city#19, sum#27] -Arguments: hashpartitioning(ca_zip#20, ca_city#19, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [ca_zip#20, ca_city#19, sum#26] +Arguments: hashpartitioning(ca_zip#20, ca_city#19, 5), ENSURE_REQUIREMENTS, [id=#27] (45) HashAggregate [codegen id : 13] -Input [3]: [ca_zip#20, ca_city#19, sum#27] +Input [3]: [ca_zip#20, ca_city#19, sum#26] Keys [2]: [ca_zip#20, ca_city#19] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#29] -Results [3]: [ca_zip#20, ca_city#19, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#29,17,2) AS sum(ws_sales_price)#30] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#28] +Results [3]: [ca_zip#20, ca_city#19, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#28,17,2) AS sum(ws_sales_price)#29] (46) TakeOrderedAndProject -Input [3]: [ca_zip#20, ca_city#19, sum(ws_sales_price)#30] -Arguments: 100, [ca_zip#20 ASC NULLS FIRST, ca_city#19 ASC NULLS FIRST], [ca_zip#20, ca_city#19, sum(ws_sales_price)#30] +Input [3]: [ca_zip#20, ca_city#19, sum(ws_sales_price)#29] +Arguments: 100, [ca_zip#20 ASC NULLS FIRST, ca_city#19 ASC NULLS FIRST], [ca_zip#20, ca_city#19, sum(ws_sales_price)#29] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt index f3a37f9e8767e..b81498ddcea34 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt @@ -169,30 +169,30 @@ Output [4]: [ws_sales_price#4, ca_city#11, ca_zip#12, i_item_id#19] Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#11, ca_zip#12, i_item_sk#18, i_item_id#19] (29) Scan parquet default.item -Output [2]: [i_item_sk#21, i_item_id#22] +Output [2]: [i_item_sk#18, i_item_id#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_item_sk, [2,3,5,7,11,13,17,19,23,29])] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] -Input [2]: [i_item_sk#21, i_item_id#22] +Input [2]: [i_item_sk#18, i_item_id#19] (31) Filter [codegen id : 5] -Input [2]: [i_item_sk#21, i_item_id#22] -Condition : i_item_sk#21 IN (2,3,5,7,11,13,17,19,23,29) +Input [2]: [i_item_sk#18, i_item_id#19] +Condition : i_item_sk#18 IN (2,3,5,7,11,13,17,19,23,29) (32) Project [codegen id : 5] -Output [1]: [i_item_id#22] -Input [2]: [i_item_sk#21, i_item_id#22] +Output [1]: [i_item_id#19 AS i_item_id#19#21] +Input [2]: [i_item_sk#18, i_item_id#19] (33) BroadcastExchange -Input [1]: [i_item_id#22] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#23] +Input [1]: [i_item_id#19#21] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#22] (34) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_id#19] -Right keys [1]: [i_item_id#22] +Right keys [1]: [i_item_id#19#21] Join condition: None (35) Filter [codegen id : 6] @@ -207,23 +207,23 @@ Input [5]: [ws_sales_price#4, ca_city#11, ca_zip#12, i_item_id#19, exists#1] Input [3]: [ws_sales_price#4, ca_city#11, ca_zip#12] Keys [2]: [ca_zip#12, ca_city#11] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#24] -Results [3]: [ca_zip#12, ca_city#11, sum#25] +Aggregate Attributes [1]: [sum#23] +Results [3]: [ca_zip#12, ca_city#11, sum#24] (38) Exchange -Input [3]: [ca_zip#12, ca_city#11, sum#25] -Arguments: hashpartitioning(ca_zip#12, ca_city#11, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [3]: [ca_zip#12, ca_city#11, sum#24] +Arguments: hashpartitioning(ca_zip#12, ca_city#11, 5), ENSURE_REQUIREMENTS, [id=#25] (39) HashAggregate [codegen id : 7] -Input [3]: [ca_zip#12, ca_city#11, sum#25] +Input [3]: [ca_zip#12, ca_city#11, sum#24] Keys [2]: [ca_zip#12, ca_city#11] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#27] -Results [3]: [ca_zip#12, ca_city#11, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#27,17,2) AS sum(ws_sales_price)#28] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#26] +Results [3]: [ca_zip#12, ca_city#11, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#26,17,2) AS sum(ws_sales_price)#27] (40) TakeOrderedAndProject -Input [3]: [ca_zip#12, ca_city#11, sum(ws_sales_price)#28] -Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#11 ASC NULLS FIRST], [ca_zip#12, ca_city#11, sum(ws_sales_price)#28] +Input [3]: [ca_zip#12, ca_city#11, sum(ws_sales_price)#27] +Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#11 ASC NULLS FIRST], [ca_zip#12, ca_city#11, sum(ws_sales_price)#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt index ac35640c7b004..78ade994364a7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt @@ -230,55 +230,55 @@ Input [5]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt Arguments: [ss_addr_sk#12 ASC NULLS FIRST], false, 0 (41) ReusedExchange [Reuses operator id: 9] -Output [2]: [ca_address_sk#31, ca_city#32] +Output [2]: [ca_address_sk#6, ca_city#7] (42) Sort [codegen id : 13] -Input [2]: [ca_address_sk#31, ca_city#32] -Arguments: [ca_address_sk#31 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#6, ca_city#7] +Arguments: [ca_address_sk#6 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 14] Left keys [1]: [ss_addr_sk#12] -Right keys [1]: [ca_address_sk#31] +Right keys [1]: [ca_address_sk#6] Join condition: None (44) Project [codegen id : 14] -Output [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#32] -Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_address_sk#31, ca_city#32] +Output [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#7] +Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_address_sk#6, ca_city#7] (45) HashAggregate [codegen id : 14] -Input [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#32] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32] +Input [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#7] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#15)), partial_sum(UnscaledValue(ss_net_profit#16))] -Aggregate Attributes [2]: [sum#33, sum#34] -Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32, sum#35, sum#36] +Aggregate Attributes [2]: [sum#31, sum#32] +Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#33, sum#34] (46) HashAggregate [codegen id : 14] -Input [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32, sum#35, sum#36] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32] +Input [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#33, sum#34] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#15)), sum(UnscaledValue(ss_net_profit#16))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#15))#37, sum(UnscaledValue(ss_net_profit#16))#38] -Results [5]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#32 AS bought_city#39, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#15))#37,17,2) AS amt#40, MakeDecimal(sum(UnscaledValue(ss_net_profit#16))#38,17,2) AS profit#41] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#15))#35, sum(UnscaledValue(ss_net_profit#16))#36] +Results [5]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#7 AS bought_city#37, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#15))#35,17,2) AS amt#38, MakeDecimal(sum(UnscaledValue(ss_net_profit#16))#36,17,2) AS profit#39] (47) Exchange -Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#39, amt#40, profit#41] -Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#37, amt#38, profit#39] +Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#40] (48) Sort [codegen id : 15] -Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#39, amt#40, profit#41] +Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#37, amt#38, profit#39] Arguments: [ss_customer_sk#10 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 16] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#10] -Join condition: NOT (ca_city#7 = bought_city#39) +Join condition: NOT (ca_city#7 = bought_city#37) (50) Project [codegen id : 16] -Output [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#39, ss_ticket_number#14, amt#40, profit#41] -Input [9]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#39, amt#40, profit#41] +Output [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#37, ss_ticket_number#14, amt#38, profit#39] +Input [9]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#37, amt#38, profit#39] (51) TakeOrderedAndProject -Input [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#39, ss_ticket_number#14, amt#40, profit#41] -Arguments: 100, [c_last_name#4 ASC NULLS FIRST, c_first_name#3 ASC NULLS FIRST, ca_city#7 ASC NULLS FIRST, bought_city#39 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#39, ss_ticket_number#14, amt#40, profit#41] +Input [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#37, ss_ticket_number#14, amt#38, profit#39] +Arguments: 100, [c_last_name#4 ASC NULLS FIRST, c_first_name#3 ASC NULLS FIRST, ca_city#7 ASC NULLS FIRST, bought_city#37 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#37, ss_ticket_number#14, amt#38, profit#39] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt index 2532a14d7907f..e9bc22389d8a6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt @@ -225,20 +225,20 @@ Output [7]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_ad Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#31, amt#32, profit#33, c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#39, ca_city#40] +Output [2]: [ca_address_sk#21, ca_city#22] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#39] -Join condition: NOT (ca_city#40 = bought_city#31) +Right keys [1]: [ca_address_sk#21] +Join condition: NOT (ca_city#22 = bought_city#31) (42) Project [codegen id : 8] -Output [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#39, ca_city#40] +Output [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#21, ca_city#22] (43) TakeOrderedAndProject -Input [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#40 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#22 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] ===== Subqueries ===== 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 b7ffba01f92c6..dc19996f9957b 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 @@ -229,78 +229,78 @@ Output [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_ye (39) HashAggregate [codegen id : 21] Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum#36] Keys [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35] -Functions [1]: [sum(UnscaledValue(ss_sales_price#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#37))#38] -Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#37))#38,17,2) AS sum_sales#39] +Functions [1]: [sum(UnscaledValue(ss_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#3))#37] +Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#3))#37,17,2) AS sum_sales#38] (40) Exchange -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#39] (41) Sort [codegen id : 22] -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST], false, 0 (42) Window -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] (43) Project [codegen id : 23] -Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] -Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] +Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] +Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38, rn#40] (44) Exchange -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1), 5), ENSURE_REQUIREMENTS, [id=#42] +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1), 5), ENSURE_REQUIREMENTS, [id=#41] (45) Sort [codegen id : 24] -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#41 + 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] +Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#40 + 1) ASC NULLS FIRST], false, 0 (46) SortMergeJoin [codegen id : 25] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1)] +Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1)] Join condition: None (47) Project [codegen id : 25] -Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39] -Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38] +Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] (48) ReusedExchange [Reuses operator id: 40] -Output [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] +Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] (49) Sort [codegen id : 34] -Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] -Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 (50) Window -Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] -Arguments: [rank(d_year#47, d_moy#48) windowspecdefinition(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#50], [i_category#43, i_brand#44, s_store_name#45, s_company_name#46], [d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST] +Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] (51) Project [codegen id : 35] -Output [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] -Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] +Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] +Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] (52) Exchange -Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] -Arguments: hashpartitioning(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1), 5), ENSURE_REQUIREMENTS, [id=#51] +Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] +Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1), 5), ENSURE_REQUIREMENTS, [id=#50] (53) Sort [codegen id : 36] -Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] -Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, (rn#50 - 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] +Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (rn#49 - 1) ASC NULLS FIRST], false, 0 (54) SortMergeJoin [codegen id : 37] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1)] +Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1)] Join condition: None (55) Project [codegen id : 37] -Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#39 AS psum#52, sum_sales#49 AS nsum#53] -Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39, i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#38 AS psum#51, sum_sales#48 AS nsum#52] +Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] (56) TakeOrderedAndProject -Input [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#11 ASC NULLS FIRST], [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] +Input [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#11 ASC NULLS FIRST], [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] ===== Subqueries ===== 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 917b21c247449..3d58d26e6a3a4 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 @@ -202,70 +202,70 @@ Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_ye (34) HashAggregate [codegen id : 13] 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] +Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#35] +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#6))#35,17,2) AS sum_sales#36] (35) 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#36] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#37] (36) Sort [codegen id : 14] -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#36] 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 (37) 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#36] +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#38], [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] (38) Project [codegen id : 15] -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#36, rn#38] +Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] (39) BroadcastExchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -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=#40] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] +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=#39] (40) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -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#38 + 1)] Join condition: None (41) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] -Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] +Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] (42) ReusedExchange [Reuses operator id: 35] -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#46] (43) Sort [codegen id : 21] -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#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, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 (44) 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#46] +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#47], [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] (45) Project [codegen id : 22] -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#46, rn#47] +Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] (46) BroadcastExchange -Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -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=#49] +Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] +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=#48] (47) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -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#47 - 1)] Join condition: None (48) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] -Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, 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#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#49, sum_sales#46 AS nsum#50] +Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] (49) TakeOrderedAndProject -Input [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, psum#50, nsum#51] +Input [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt index 3f30183af3a94..e93d0e9503493 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt @@ -250,225 +250,225 @@ Output [5]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_net_profit#47, cs_sold_date_sk#48] (36) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#49] +Output [1]: [d_date_sk#8] (37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_sold_date_sk#48] -Right keys [1]: [d_date_sk#49] +Right keys [1]: [d_date_sk#8] Join condition: None (38) Project [codegen id : 12] Output [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#49] +Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#8] (39) Exchange Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#50] +Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#49] (40) Sort [codegen id : 13] Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] Arguments: [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST], false, 0 (41) Scan parquet default.catalog_returns -Output [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Output [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (42) ColumnarToRow [codegen id : 14] -Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] (43) Filter [codegen id : 14] -Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] -Condition : (((isnotnull(cr_return_amount#54) AND (cr_return_amount#54 > 10000.00)) AND isnotnull(cr_order_number#52)) AND isnotnull(cr_item_sk#51)) +Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Condition : (((isnotnull(cr_return_amount#53) AND (cr_return_amount#53 > 10000.00)) AND isnotnull(cr_order_number#51)) AND isnotnull(cr_item_sk#50)) (44) Project [codegen id : 14] -Output [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] -Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Output [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] (45) Exchange -Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] -Arguments: hashpartitioning(cr_order_number#52, cr_item_sk#51, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#55] (46) Sort [codegen id : 15] -Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] -Arguments: [cr_order_number#52 ASC NULLS FIRST, cr_item_sk#51 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Arguments: [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 16] Left keys [2]: [cs_order_number#44, cs_item_sk#43] -Right keys [2]: [cr_order_number#52, cr_item_sk#51] +Right keys [2]: [cr_order_number#51, cr_item_sk#50] Join condition: None (48) Project [codegen id : 16] -Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] -Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] +Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] (49) HashAggregate [codegen id : 16] -Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] +Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] Keys [1]: [cs_item_sk#43] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#53, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] -Results [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#52, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#56, sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] +Results [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] (50) Exchange -Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#68] (51) HashAggregate [codegen id : 17] -Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] Keys [1]: [cs_item_sk#43] -Functions [4]: [sum(coalesce(cr_return_quantity#53, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#53, 0))#70, sum(coalesce(cs_quantity#45, 0))#71, sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73] -Results [3]: [cs_item_sk#43 AS item#74, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#53, 0))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#71 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#75, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#76] +Functions [4]: [sum(coalesce(cr_return_quantity#52, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#52, 0))#69, sum(coalesce(cs_quantity#45, 0))#70, sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72] +Results [3]: [cs_item_sk#43 AS item#73, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#52, 0))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#70 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#74, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#75] (52) Exchange -Input [3]: [item#74, return_ratio#75, currency_ratio#76] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#77] +Input [3]: [item#73, return_ratio#74, currency_ratio#75] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] (53) Sort [codegen id : 18] -Input [3]: [item#74, return_ratio#75, currency_ratio#76] -Arguments: [return_ratio#75 ASC NULLS FIRST], false, 0 +Input [3]: [item#73, return_ratio#74, currency_ratio#75] +Arguments: [return_ratio#74 ASC NULLS FIRST], false, 0 (54) Window -Input [3]: [item#74, return_ratio#75, currency_ratio#76] -Arguments: [rank(return_ratio#75) windowspecdefinition(return_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#78], [return_ratio#75 ASC NULLS FIRST] +Input [3]: [item#73, return_ratio#74, currency_ratio#75] +Arguments: [rank(return_ratio#74) windowspecdefinition(return_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#77], [return_ratio#74 ASC NULLS FIRST] (55) Sort [codegen id : 19] -Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] -Arguments: [currency_ratio#76 ASC NULLS FIRST], false, 0 +Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] +Arguments: [currency_ratio#75 ASC NULLS FIRST], false, 0 (56) Window -Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] -Arguments: [rank(currency_ratio#76) windowspecdefinition(currency_ratio#76 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#79], [currency_ratio#76 ASC NULLS FIRST] +Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] +Arguments: [rank(currency_ratio#75) windowspecdefinition(currency_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#78], [currency_ratio#75 ASC NULLS FIRST] (57) Filter [codegen id : 20] -Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] -Condition : ((return_rank#78 <= 10) OR (currency_rank#79 <= 10)) +Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] +Condition : ((return_rank#77 <= 10) OR (currency_rank#78 <= 10)) (58) Project [codegen id : 20] -Output [5]: [catalog AS channel#80, item#74, return_ratio#75, return_rank#78, currency_rank#79] -Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] +Output [5]: [catalog AS channel#79, item#73, return_ratio#74, return_rank#77, currency_rank#78] +Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] (59) Scan parquet default.store_sales -Output [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Output [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_sold_date_sk#85 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 22] -Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] (61) Filter [codegen id : 22] -Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] -Condition : (((((((isnotnull(ss_net_profit#85) AND isnotnull(ss_net_paid#84)) AND isnotnull(ss_quantity#83)) AND (ss_net_profit#85 > 1.00)) AND (ss_net_paid#84 > 0.00)) AND (ss_quantity#83 > 0)) AND isnotnull(ss_ticket_number#82)) AND isnotnull(ss_item_sk#81)) +Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Condition : (((((((isnotnull(ss_net_profit#84) AND isnotnull(ss_net_paid#83)) AND isnotnull(ss_quantity#82)) AND (ss_net_profit#84 > 1.00)) AND (ss_net_paid#83 > 0.00)) AND (ss_quantity#82 > 0)) AND isnotnull(ss_ticket_number#81)) AND isnotnull(ss_item_sk#80)) (62) Project [codegen id : 22] -Output [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86] -Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Output [5]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85] +Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] (63) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#87] +Output [1]: [d_date_sk#8] (64) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ss_sold_date_sk#86] -Right keys [1]: [d_date_sk#87] +Left keys [1]: [ss_sold_date_sk#85] +Right keys [1]: [d_date_sk#8] Join condition: None (65) Project [codegen id : 22] -Output [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] -Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86, d_date_sk#87] +Output [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] +Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85, d_date_sk#8] (66) Exchange -Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] -Arguments: hashpartitioning(cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint), 5), ENSURE_REQUIREMENTS, [id=#88] +Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] +Arguments: hashpartitioning(cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint), 5), ENSURE_REQUIREMENTS, [id=#86] (67) Sort [codegen id : 23] -Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] -Arguments: [cast(ss_ticket_number#82 as bigint) ASC NULLS FIRST, cast(ss_item_sk#81 as bigint) ASC NULLS FIRST], false, 0 +Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] +Arguments: [cast(ss_ticket_number#81 as bigint) ASC NULLS FIRST, cast(ss_item_sk#80 as bigint) ASC NULLS FIRST], false, 0 (68) Scan parquet default.store_returns -Output [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Output [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 24] -Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] (70) Filter [codegen id : 24] -Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] -Condition : (((isnotnull(sr_return_amt#92) AND (sr_return_amt#92 > 10000.00)) AND isnotnull(sr_ticket_number#90)) AND isnotnull(sr_item_sk#89)) +Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Condition : (((isnotnull(sr_return_amt#90) AND (sr_return_amt#90 > 10000.00)) AND isnotnull(sr_ticket_number#88)) AND isnotnull(sr_item_sk#87)) (71) Project [codegen id : 24] -Output [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] -Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Output [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] (72) Exchange -Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] -Arguments: hashpartitioning(sr_ticket_number#90, sr_item_sk#89, 5), ENSURE_REQUIREMENTS, [id=#94] +Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Arguments: hashpartitioning(sr_ticket_number#88, sr_item_sk#87, 5), ENSURE_REQUIREMENTS, [id=#92] (73) Sort [codegen id : 25] -Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] -Arguments: [sr_ticket_number#90 ASC NULLS FIRST, sr_item_sk#89 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Arguments: [sr_ticket_number#88 ASC NULLS FIRST, sr_item_sk#87 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 26] -Left keys [2]: [cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint)] -Right keys [2]: [sr_ticket_number#90, sr_item_sk#89] +Left keys [2]: [cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint)] +Right keys [2]: [sr_ticket_number#88, sr_item_sk#87] Join condition: None (75) Project [codegen id : 26] -Output [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] -Input [8]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Output [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] +Input [8]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] (76) HashAggregate [codegen id : 26] -Input [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] -Keys [1]: [ss_item_sk#81] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#91, 0)), partial_sum(coalesce(ss_quantity#83, 0)), partial_sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#95, sum#96, sum#97, isEmpty#98, sum#99, isEmpty#100] -Results [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] +Input [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] +Keys [1]: [ss_item_sk#80] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#89, 0)), partial_sum(coalesce(ss_quantity#82, 0)), partial_sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#93, sum#94, sum#95, isEmpty#96, sum#97, isEmpty#98] +Results [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] (77) Exchange -Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] -Arguments: hashpartitioning(ss_item_sk#81, 5), ENSURE_REQUIREMENTS, [id=#107] +Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Arguments: hashpartitioning(ss_item_sk#80, 5), ENSURE_REQUIREMENTS, [id=#105] (78) HashAggregate [codegen id : 27] -Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] -Keys [1]: [ss_item_sk#81] -Functions [4]: [sum(coalesce(sr_return_quantity#91, 0)), sum(coalesce(ss_quantity#83, 0)), sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#91, 0))#108, sum(coalesce(ss_quantity#83, 0))#109, sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110, sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111] -Results [3]: [ss_item_sk#81 AS item#112, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#91, 0))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#83, 0))#109 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#113, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#114] +Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Keys [1]: [ss_item_sk#80] +Functions [4]: [sum(coalesce(sr_return_quantity#89, 0)), sum(coalesce(ss_quantity#82, 0)), sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#89, 0))#106, sum(coalesce(ss_quantity#82, 0))#107, sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108, sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109] +Results [3]: [ss_item_sk#80 AS item#110, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#89, 0))#106 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#82, 0))#107 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#111, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#112] (79) Exchange -Input [3]: [item#112, return_ratio#113, currency_ratio#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] +Input [3]: [item#110, return_ratio#111, currency_ratio#112] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] (80) Sort [codegen id : 28] -Input [3]: [item#112, return_ratio#113, currency_ratio#114] -Arguments: [return_ratio#113 ASC NULLS FIRST], false, 0 +Input [3]: [item#110, return_ratio#111, currency_ratio#112] +Arguments: [return_ratio#111 ASC NULLS FIRST], false, 0 (81) Window -Input [3]: [item#112, return_ratio#113, currency_ratio#114] -Arguments: [rank(return_ratio#113) windowspecdefinition(return_ratio#113 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#116], [return_ratio#113 ASC NULLS FIRST] +Input [3]: [item#110, return_ratio#111, currency_ratio#112] +Arguments: [rank(return_ratio#111) windowspecdefinition(return_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#114], [return_ratio#111 ASC NULLS FIRST] (82) Sort [codegen id : 29] -Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] -Arguments: [currency_ratio#114 ASC NULLS FIRST], false, 0 +Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] +Arguments: [currency_ratio#112 ASC NULLS FIRST], false, 0 (83) Window -Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] -Arguments: [rank(currency_ratio#114) windowspecdefinition(currency_ratio#114 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#117], [currency_ratio#114 ASC NULLS FIRST] +Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] +Arguments: [rank(currency_ratio#112) windowspecdefinition(currency_ratio#112 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#115], [currency_ratio#112 ASC NULLS FIRST] (84) Filter [codegen id : 30] -Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] -Condition : ((return_rank#116 <= 10) OR (currency_rank#117 <= 10)) +Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] +Condition : ((return_rank#114 <= 10) OR (currency_rank#115 <= 10)) (85) Project [codegen id : 30] -Output [5]: [store AS channel#118, item#112, return_ratio#113, return_rank#116, currency_rank#117] -Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] +Output [5]: [store AS channel#116, item#110, return_ratio#111, return_rank#114, currency_rank#115] +Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] (86) Union @@ -481,7 +481,7 @@ Results [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_ran (88) Exchange Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] -Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#119] +Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#117] (89) HashAggregate [codegen id : 32] Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] @@ -505,6 +505,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#85 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index a98f59f84be7d..aeaca4deaae0c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -260,170 +260,170 @@ Output [6]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, Input [9]: [cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_item_sk#49, cr_order_number#50, cr_return_quantity#51, cr_return_amount#52] (40) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#54] +Output [1]: [d_date_sk#14] (41) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#47] -Right keys [1]: [d_date_sk#54] +Right keys [1]: [d_date_sk#14] Join condition: None (42) Project [codegen id : 10] Output [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] -Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#54] +Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#14] (43) HashAggregate [codegen id : 10] Input [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] Keys [1]: [cs_item_sk#42] Functions [4]: [partial_sum(coalesce(cr_return_quantity#51, 0)), partial_sum(coalesce(cs_quantity#44, 0)), partial_sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Results [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Aggregate Attributes [6]: [sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] +Results [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] (44) Exchange -Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#67] +Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] +Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#66] (45) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] Keys [1]: [cs_item_sk#42] Functions [4]: [sum(coalesce(cr_return_quantity#51, 0)), sum(coalesce(cs_quantity#44, 0)), sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#68, sum(coalesce(cs_quantity#44, 0))#69, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71] -Results [3]: [cs_item_sk#42 AS item#72, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#68 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#69 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#73, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#74] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#67, sum(coalesce(cs_quantity#44, 0))#68, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70] +Results [3]: [cs_item_sk#42 AS item#71, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#67 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#68 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#72, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#73] (46) Exchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#75] +Input [3]: [item#71, return_ratio#72, currency_ratio#73] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#74] (47) Sort [codegen id : 12] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [return_ratio#73 ASC NULLS FIRST], false, 0 +Input [3]: [item#71, return_ratio#72, currency_ratio#73] +Arguments: [return_ratio#72 ASC NULLS FIRST], false, 0 (48) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#76], [return_ratio#73 ASC NULLS FIRST] +Input [3]: [item#71, return_ratio#72, currency_ratio#73] +Arguments: [rank(return_ratio#72) windowspecdefinition(return_ratio#72 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#72 ASC NULLS FIRST] (49) Sort [codegen id : 13] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 +Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] +Arguments: [currency_ratio#73 ASC NULLS FIRST], false, 0 (50) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#77], [currency_ratio#74 ASC NULLS FIRST] +Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] +Arguments: [rank(currency_ratio#73) windowspecdefinition(currency_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#73 ASC NULLS FIRST] (51) Filter [codegen id : 14] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] -Condition : ((return_rank#76 <= 10) OR (currency_rank#77 <= 10)) +Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] +Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) (52) Project [codegen id : 14] -Output [5]: [catalog AS channel#78, item#72, return_ratio#73, return_rank#76, currency_rank#77] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] +Output [5]: [catalog AS channel#77, item#71, return_ratio#72, return_rank#75, currency_rank#76] +Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] (53) Scan parquet default.store_sales -Output [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Output [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#84), dynamicpruningexpression(ss_sold_date_sk#84 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#83), dynamicpruningexpression(ss_sold_date_sk#83 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 15] -Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] (55) Filter [codegen id : 15] -Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] -Condition : (((((((isnotnull(ss_net_profit#83) AND isnotnull(ss_net_paid#82)) AND isnotnull(ss_quantity#81)) AND (ss_net_profit#83 > 1.00)) AND (ss_net_paid#82 > 0.00)) AND (ss_quantity#81 > 0)) AND isnotnull(ss_ticket_number#80)) AND isnotnull(ss_item_sk#79)) +Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Condition : (((((((isnotnull(ss_net_profit#82) AND isnotnull(ss_net_paid#81)) AND isnotnull(ss_quantity#80)) AND (ss_net_profit#82 > 1.00)) AND (ss_net_paid#81 > 0.00)) AND (ss_quantity#80 > 0)) AND isnotnull(ss_ticket_number#79)) AND isnotnull(ss_item_sk#78)) (56) Project [codegen id : 15] -Output [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] -Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Output [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] +Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] (57) BroadcastExchange -Input [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#85] +Input [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#84] (58) Scan parquet default.store_returns -Output [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Output [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (59) ColumnarToRow -Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] (60) Filter -Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] -Condition : (((isnotnull(sr_return_amt#89) AND (sr_return_amt#89 > 10000.00)) AND isnotnull(sr_ticket_number#87)) AND isnotnull(sr_item_sk#86)) +Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Condition : (((isnotnull(sr_return_amt#88) AND (sr_return_amt#88 > 10000.00)) AND isnotnull(sr_ticket_number#86)) AND isnotnull(sr_item_sk#85)) (61) Project -Output [4]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] -Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Output [4]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] +Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] (62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [cast(ss_ticket_number#80 as bigint), cast(ss_item_sk#79 as bigint)] -Right keys [2]: [sr_ticket_number#87, sr_item_sk#86] +Left keys [2]: [cast(ss_ticket_number#79 as bigint), cast(ss_item_sk#78 as bigint)] +Right keys [2]: [sr_ticket_number#86, sr_item_sk#85] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89] -Input [9]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] +Output [6]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88] +Input [9]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] (64) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#91] +Output [1]: [d_date_sk#14] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#84] -Right keys [1]: [d_date_sk#91] +Left keys [1]: [ss_sold_date_sk#83] +Right keys [1]: [d_date_sk#14] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] -Input [7]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89, d_date_sk#91] +Output [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] +Input [7]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88, d_date_sk#14] (67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] -Keys [1]: [ss_item_sk#79] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#88, 0)), partial_sum(coalesce(ss_quantity#81, 0)), partial_sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Results [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Input [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] +Keys [1]: [ss_item_sk#78] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#87, 0)), partial_sum(coalesce(ss_quantity#80, 0)), partial_sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Results [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] (68) Exchange -Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] -Arguments: hashpartitioning(ss_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#104] +Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Arguments: hashpartitioning(ss_item_sk#78, 5), ENSURE_REQUIREMENTS, [id=#102] (69) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] -Keys [1]: [ss_item_sk#79] -Functions [4]: [sum(coalesce(sr_return_quantity#88, 0)), sum(coalesce(ss_quantity#81, 0)), sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#88, 0))#105, sum(coalesce(ss_quantity#81, 0))#106, sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107, sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108] -Results [3]: [ss_item_sk#79 AS item#109, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#88, 0))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#81, 0))#106 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#110, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#111] +Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Keys [1]: [ss_item_sk#78] +Functions [4]: [sum(coalesce(sr_return_quantity#87, 0)), sum(coalesce(ss_quantity#80, 0)), sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#87, 0))#103, sum(coalesce(ss_quantity#80, 0))#104, sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105, sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106] +Results [3]: [ss_item_sk#78 AS item#107, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#87, 0))#103 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#80, 0))#104 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#108, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#109] (70) Exchange -Input [3]: [item#109, return_ratio#110, currency_ratio#111] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#112] +Input [3]: [item#107, return_ratio#108, currency_ratio#109] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] (71) Sort [codegen id : 19] -Input [3]: [item#109, return_ratio#110, currency_ratio#111] -Arguments: [return_ratio#110 ASC NULLS FIRST], false, 0 +Input [3]: [item#107, return_ratio#108, currency_ratio#109] +Arguments: [return_ratio#108 ASC NULLS FIRST], false, 0 (72) Window -Input [3]: [item#109, return_ratio#110, currency_ratio#111] -Arguments: [rank(return_ratio#110) windowspecdefinition(return_ratio#110 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#113], [return_ratio#110 ASC NULLS FIRST] +Input [3]: [item#107, return_ratio#108, currency_ratio#109] +Arguments: [rank(return_ratio#108) windowspecdefinition(return_ratio#108 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#111], [return_ratio#108 ASC NULLS FIRST] (73) Sort [codegen id : 20] -Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] -Arguments: [currency_ratio#111 ASC NULLS FIRST], false, 0 +Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] +Arguments: [currency_ratio#109 ASC NULLS FIRST], false, 0 (74) Window -Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] -Arguments: [rank(currency_ratio#111) windowspecdefinition(currency_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#114], [currency_ratio#111 ASC NULLS FIRST] +Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] +Arguments: [rank(currency_ratio#109) windowspecdefinition(currency_ratio#109 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#112], [currency_ratio#109 ASC NULLS FIRST] (75) Filter [codegen id : 21] -Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] -Condition : ((return_rank#113 <= 10) OR (currency_rank#114 <= 10)) +Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] +Condition : ((return_rank#111 <= 10) OR (currency_rank#112 <= 10)) (76) Project [codegen id : 21] -Output [5]: [store AS channel#115, item#109, return_ratio#110, return_rank#113, currency_rank#114] -Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] +Output [5]: [store AS channel#113, item#107, return_ratio#108, return_rank#111, currency_rank#112] +Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] (77) Union @@ -436,7 +436,7 @@ Results [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_ran (79) Exchange Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] -Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#116] +Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#114] (80) HashAggregate [codegen id : 23] Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] @@ -460,6 +460,6 @@ Output [1]: [d_date_sk#14] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#84 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#83 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index 87682d1a814ae..2d3dea5d84565 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -265,193 +265,193 @@ Output [6]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, Input [8]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] (41) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#70] +Output [1]: [d_date_sk#25] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#70] +Right keys [1]: [d_date_sk#25] Join condition: None (43) Project [codegen id : 11] Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#70] +Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#25] (44) HashAggregate [codegen id : 11] Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] -Results [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] +Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] +Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] (45) Exchange -Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] -Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] +Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#84, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS RETURNS#85, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#86, catalog channel AS channel#87, concat(catalog_page, cp_catalog_page_id#68) AS id#88] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#83, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS RETURNS#84, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#85, catalog channel AS channel#86, concat(catalog_page, cp_catalog_page_id#68) AS id#87] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Condition : isnotnull(ws_web_site_sk#89) +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Condition : isnotnull(ws_web_site_sk#88) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] (53) Exchange -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] -Arguments: hashpartitioning(wr_item_sk#99, wr_order_number#100, 5), ENSURE_REQUIREMENTS, [id=#104] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Arguments: hashpartitioning(wr_item_sk#98, wr_order_number#99, 5), ENSURE_REQUIREMENTS, [id=#103] (54) Sort [codegen id : 15] -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] -Arguments: [wr_item_sk#99 ASC NULLS FIRST, wr_order_number#100 ASC NULLS FIRST], false, 0 +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Arguments: [wr_item_sk#98 ASC NULLS FIRST, wr_order_number#99 ASC NULLS FIRST], false, 0 (55) Scan parquet default.web_sales -Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 16] -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] (57) Filter [codegen id : 16] -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] -Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) (58) Project [codegen id : 16] -Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] (59) Exchange -Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] -Arguments: hashpartitioning(cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint), 5), ENSURE_REQUIREMENTS, [id=#109] +Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Arguments: hashpartitioning(cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint), 5), ENSURE_REQUIREMENTS, [id=#106] (60) Sort [codegen id : 17] -Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] -Arguments: [cast(ws_item_sk#105 as bigint) ASC NULLS FIRST, cast(ws_order_number#107 as bigint) ASC NULLS FIRST], false, 0 +Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Arguments: [cast(ws_item_sk#104 as bigint) ASC NULLS FIRST, cast(ws_order_number#105 as bigint) ASC NULLS FIRST], false, 0 (61) SortMergeJoin [codegen id : 18] -Left keys [2]: [wr_item_sk#99, wr_order_number#100] -Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] +Left keys [2]: [wr_item_sk#98, wr_order_number#99] +Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None (62) Project [codegen id : 18] -Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#110, wr_returned_date_sk#103 AS date_sk#111, 0.00 AS sales_price#112, 0.00 AS profit#113, wr_return_amt#101 AS return_amt#114, wr_net_loss#102 AS net_loss#115] -Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#107, wr_returned_date_sk#102 AS date_sk#108, 0.00 AS sales_price#109, 0.00 AS profit#110, wr_return_amt#100 AS return_amt#111, wr_net_loss#101 AS net_loss#112] +Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] (63) Union (64) Scan parquet default.web_site -Output [2]: [web_site_sk#116, web_site_id#117] +Output [2]: [web_site_sk#113, web_site_id#114] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 19] -Input [2]: [web_site_sk#116, web_site_id#117] +Input [2]: [web_site_sk#113, web_site_id#114] (66) Filter [codegen id : 19] -Input [2]: [web_site_sk#116, web_site_id#117] -Condition : isnotnull(web_site_sk#116) +Input [2]: [web_site_sk#113, web_site_id#114] +Condition : isnotnull(web_site_sk#113) (67) BroadcastExchange -Input [2]: [web_site_sk#116, web_site_id#117] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] +Input [2]: [web_site_sk#113, web_site_id#114] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] (68) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [wsr_web_site_sk#93] -Right keys [1]: [web_site_sk#116] +Left keys [1]: [wsr_web_site_sk#92] +Right keys [1]: [web_site_sk#113] Join condition: None (69) Project [codegen id : 21] -Output [6]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Input [8]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] +Output [6]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] +Input [8]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#113, web_site_id#114] (70) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#119] +Output [1]: [d_date_sk#25] (71) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [date_sk#94] -Right keys [1]: [cast(d_date_sk#119 as bigint)] +Left keys [1]: [date_sk#93] +Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None (72) Project [codegen id : 21] -Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Input [7]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117, d_date_sk#119] +Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] +Input [7]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114, d_date_sk#25] (73) HashAggregate [codegen id : 21] -Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Keys [1]: [web_site_id#117] -Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] -Aggregate Attributes [4]: [sum#120, sum#121, sum#122, sum#123] -Results [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] +Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] +Keys [1]: [web_site_id#114] +Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] +Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] +Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] (74) Exchange -Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] -Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#128] +Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] +Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] (75) HashAggregate [codegen id : 22] -Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] -Keys [1]: [web_site_id#117] -Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#129, sum(UnscaledValue(return_amt#97))#130, sum(UnscaledValue(profit#96))#131, sum(UnscaledValue(net_loss#98))#132] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#95))#129,17,2) AS sales#133, MakeDecimal(sum(UnscaledValue(return_amt#97))#130,17,2) AS RETURNS#134, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#131,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#132,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#135, web channel AS channel#136, concat(web_site, web_site_id#117) AS id#137] +Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] +Keys [1]: [web_site_id#114] +Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#125, sum(UnscaledValue(return_amt#96))#126, sum(UnscaledValue(profit#95))#127, sum(UnscaledValue(net_loss#97))#128] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#94))#125,17,2) AS sales#129, MakeDecimal(sum(UnscaledValue(return_amt#96))#126,17,2) AS RETURNS#130, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#131, web channel AS channel#132, concat(web_site, web_site_id#114) AS id#133] (76) Union (77) Expand [codegen id : 23] Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45] -Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#138, id#139, spark_grouping_id#140] +Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#134, id#135, spark_grouping_id#136] (78) HashAggregate [codegen id : 23] -Input [6]: [sales#41, returns#42, profit#43, channel#138, id#139, spark_grouping_id#140] -Keys [3]: [channel#138, id#139, spark_grouping_id#140] +Input [6]: [sales#41, returns#42, profit#43, channel#134, id#135, spark_grouping_id#136] +Keys [3]: [channel#134, id#135, spark_grouping_id#136] Functions [3]: [partial_sum(sales#41), partial_sum(returns#42), partial_sum(profit#43)] -Aggregate Attributes [6]: [sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Results [9]: [channel#138, id#139, spark_grouping_id#140, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] +Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] +Results [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] (79) Exchange -Input [9]: [channel#138, id#139, spark_grouping_id#140, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -Arguments: hashpartitioning(channel#138, id#139, spark_grouping_id#140, 5), ENSURE_REQUIREMENTS, [id=#153] +Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Arguments: hashpartitioning(channel#134, id#135, spark_grouping_id#136, 5), ENSURE_REQUIREMENTS, [id=#149] (80) HashAggregate [codegen id : 24] -Input [9]: [channel#138, id#139, spark_grouping_id#140, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -Keys [3]: [channel#138, id#139, spark_grouping_id#140] +Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Keys [3]: [channel#134, id#135, spark_grouping_id#136] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] -Aggregate Attributes [3]: [sum(sales#41)#154, sum(returns#42)#155, sum(profit#43)#156] -Results [5]: [channel#138, id#139, sum(sales#41)#154 AS sales#157, sum(returns#42)#155 AS returns#158, sum(profit#43)#156 AS profit#159] +Aggregate Attributes [3]: [sum(sales#41)#150, sum(returns#42)#151, sum(profit#43)#152] +Results [5]: [channel#134, id#135, sum(sales#41)#150 AS sales#153, sum(returns#42)#151 AS returns#154, sum(profit#43)#152 AS profit#155] (81) TakeOrderedAndProject -Input [5]: [channel#138, id#139, sales#157, returns#158, profit#159] -Arguments: 100, [channel#138 ASC NULLS FIRST, id#139 ASC NULLS FIRST], [channel#138, id#139, sales#157, returns#158, profit#159] +Input [5]: [channel#134, id#135, sales#153, returns#154, profit#155] +Arguments: 100, [channel#134 ASC NULLS FIRST, id#135 ASC NULLS FIRST], [channel#134, id#135, sales#153, returns#154, profit#155] ===== Subqueries ===== @@ -469,12 +469,12 @@ ReusedExchange (83) (83) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#70] +Output [1]: [d_date_sk#25] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index 1282cc64052dd..f6fe8bee89369 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -235,208 +235,208 @@ Input [4]: [cr_catalog_page_sk#57, cr_return_amount#58, cr_net_loss#59, cr_retur (34) Union (35) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#67] +Output [1]: [d_date_sk#22] (36) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#67] +Right keys [1]: [d_date_sk#22] Join condition: None (37) Project [codegen id : 11] Output [5]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56] -Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#67] +Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#22] (38) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Output [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] (40) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] -Condition : isnotnull(cp_catalog_page_sk#68) +Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Condition : isnotnull(cp_catalog_page_sk#67) (41) BroadcastExchange -Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#70] +Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#69] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [page_sk#51] -Right keys [1]: [cp_catalog_page_sk#68] +Right keys [1]: [cp_catalog_page_sk#67] Join condition: None (43) Project [codegen id : 11] -Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] -Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#68, cp_catalog_page_id#69] +Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] +Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] (44) HashAggregate [codegen id : 11] -Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] -Keys [1]: [cp_catalog_page_id#69] +Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] +Keys [1]: [cp_catalog_page_id#68] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] -Results [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] +Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] +Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] (45) Exchange -Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] -Arguments: hashpartitioning(cp_catalog_page_id#69, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] -Keys [1]: [cp_catalog_page_id#69] +Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Keys [1]: [cp_catalog_page_id#68] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#84, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS RETURNS#85, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#86, catalog channel AS channel#87, concat(catalog_page, cp_catalog_page_id#69) AS id#88] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#83, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS RETURNS#84, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#85, catalog channel AS channel#86, concat(catalog_page, cp_catalog_page_id#68) AS id#87] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Condition : isnotnull(ws_web_site_sk#89) +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Condition : isnotnull(ws_web_site_sk#88) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] (53) BroadcastExchange -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#104] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#103] (54) Scan parquet default.web_sales -Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (55) ColumnarToRow -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] (56) Filter -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] -Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) (57) Project -Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] (58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#99, wr_order_number#100] -Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] +Left keys [2]: [wr_item_sk#98, wr_order_number#99] +Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None (59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#109, wr_returned_date_sk#103 AS date_sk#110, 0.00 AS sales_price#111, 0.00 AS profit#112, wr_return_amt#101 AS return_amt#113, wr_net_loss#102 AS net_loss#114] -Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#106, wr_returned_date_sk#102 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#100 AS return_amt#110, wr_net_loss#101 AS net_loss#111] +Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] (60) Union (61) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#115] +Output [1]: [d_date_sk#22] (62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#94] -Right keys [1]: [cast(d_date_sk#115 as bigint)] +Left keys [1]: [date_sk#93] +Right keys [1]: [cast(d_date_sk#22 as bigint)] Join condition: None (63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98] -Input [7]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, d_date_sk#115] +Output [5]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97] +Input [7]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, d_date_sk#22] (64) Scan parquet default.web_site -Output [2]: [web_site_sk#116, web_site_id#117] +Output [2]: [web_site_sk#112, web_site_id#113] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#116, web_site_id#117] +Input [2]: [web_site_sk#112, web_site_id#113] (66) Filter [codegen id : 17] -Input [2]: [web_site_sk#116, web_site_id#117] -Condition : isnotnull(web_site_sk#116) +Input [2]: [web_site_sk#112, web_site_id#113] +Condition : isnotnull(web_site_sk#112) (67) BroadcastExchange -Input [2]: [web_site_sk#116, web_site_id#117] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] +Input [2]: [web_site_sk#112, web_site_id#113] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#114] (68) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#93] -Right keys [1]: [web_site_sk#116] +Left keys [1]: [wsr_web_site_sk#92] +Right keys [1]: [web_site_sk#112] Join condition: None (69) Project [codegen id : 18] -Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Input [7]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] +Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] +Input [7]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#112, web_site_id#113] (70) HashAggregate [codegen id : 18] -Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Keys [1]: [web_site_id#117] -Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] -Aggregate Attributes [4]: [sum#119, sum#120, sum#121, sum#122] -Results [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] +Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] +Keys [1]: [web_site_id#113] +Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] +Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] +Results [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] (71) Exchange -Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] -Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#127] +Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] +Arguments: hashpartitioning(web_site_id#113, 5), ENSURE_REQUIREMENTS, [id=#123] (72) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] -Keys [1]: [web_site_id#117] -Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#128, sum(UnscaledValue(return_amt#97))#129, sum(UnscaledValue(profit#96))#130, sum(UnscaledValue(net_loss#98))#131] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#95))#128,17,2) AS sales#132, MakeDecimal(sum(UnscaledValue(return_amt#97))#129,17,2) AS RETURNS#133, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#130,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#131,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#134, web channel AS channel#135, concat(web_site, web_site_id#117) AS id#136] +Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] +Keys [1]: [web_site_id#113] +Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#124, sum(UnscaledValue(return_amt#96))#125, sum(UnscaledValue(profit#95))#126, sum(UnscaledValue(net_loss#97))#127] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#94))#124,17,2) AS sales#128, MakeDecimal(sum(UnscaledValue(return_amt#96))#125,17,2) AS RETURNS#129, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#126,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#127,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#130, web channel AS channel#131, concat(web_site, web_site_id#113) AS id#132] (73) Union (74) Expand [codegen id : 20] Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45] -Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#137, id#138, spark_grouping_id#139] +Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#133, id#134, spark_grouping_id#135] (75) HashAggregate [codegen id : 20] -Input [6]: [sales#41, returns#42, profit#43, channel#137, id#138, spark_grouping_id#139] -Keys [3]: [channel#137, id#138, spark_grouping_id#139] +Input [6]: [sales#41, returns#42, profit#43, channel#133, id#134, spark_grouping_id#135] +Keys [3]: [channel#133, id#134, spark_grouping_id#135] Functions [3]: [partial_sum(sales#41), partial_sum(returns#42), partial_sum(profit#43)] -Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -Results [9]: [channel#137, id#138, spark_grouping_id#139, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Aggregate Attributes [6]: [sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] +Results [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] (76) Exchange -Input [9]: [channel#137, id#138, spark_grouping_id#139, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -Arguments: hashpartitioning(channel#137, id#138, spark_grouping_id#139, 5), ENSURE_REQUIREMENTS, [id=#152] +Input [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Arguments: hashpartitioning(channel#133, id#134, spark_grouping_id#135, 5), ENSURE_REQUIREMENTS, [id=#148] (77) HashAggregate [codegen id : 21] -Input [9]: [channel#137, id#138, spark_grouping_id#139, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -Keys [3]: [channel#137, id#138, spark_grouping_id#139] +Input [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Keys [3]: [channel#133, id#134, spark_grouping_id#135] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] -Aggregate Attributes [3]: [sum(sales#41)#153, sum(returns#42)#154, sum(profit#43)#155] -Results [5]: [channel#137, id#138, sum(sales#41)#153 AS sales#156, sum(returns#42)#154 AS returns#157, sum(profit#43)#155 AS profit#158] +Aggregate Attributes [3]: [sum(sales#41)#149, sum(returns#42)#150, sum(profit#43)#151] +Results [5]: [channel#133, id#134, sum(sales#41)#149 AS sales#152, sum(returns#42)#150 AS returns#153, sum(profit#43)#151 AS profit#154] (78) TakeOrderedAndProject -Input [5]: [channel#137, id#138, sales#156, returns#157, profit#158] -Arguments: 100, [channel#137 ASC NULLS FIRST, id#138 ASC NULLS FIRST], [channel#137, id#138, sales#156, returns#157, profit#158] +Input [5]: [channel#133, id#134, sales#152, returns#153, profit#154] +Arguments: 100, [channel#133 ASC NULLS FIRST, id#134 ASC NULLS FIRST], [channel#133, id#134, sales#152, returns#153, profit#154] ===== Subqueries ===== @@ -454,12 +454,12 @@ ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#67] +Output [1]: [d_date_sk#22] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt index 4306fee77b356..c8058eeee7ca1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt @@ -297,39 +297,39 @@ Input [3]: [ss_customer_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] Condition : isnotnull(ss_customer_sk#33) (51) Scan parquet default.date_dim -Output [2]: [d_date_sk#37, d_month_seq#38] +Output [2]: [d_date_sk#21, d_month_seq#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#37, d_month_seq#38] +Input [2]: [d_date_sk#21, d_month_seq#37] (53) Filter [codegen id : 12] -Input [2]: [d_date_sk#37, d_month_seq#38] -Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= Subquery scalar-subquery#39, [id=#40])) AND (d_month_seq#38 <= Subquery scalar-subquery#41, [id=#42])) AND isnotnull(d_date_sk#37)) +Input [2]: [d_date_sk#21, d_month_seq#37] +Condition : (((isnotnull(d_month_seq#37) AND (d_month_seq#37 >= Subquery scalar-subquery#38, [id=#39])) AND (d_month_seq#37 <= Subquery scalar-subquery#40, [id=#41])) AND isnotnull(d_date_sk#21)) (54) Project [codegen id : 12] -Output [1]: [d_date_sk#37] -Input [2]: [d_date_sk#37, d_month_seq#38] +Output [1]: [d_date_sk#21] +Input [2]: [d_date_sk#21, d_month_seq#37] (55) BroadcastExchange -Input [1]: [d_date_sk#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#43] +Input [1]: [d_date_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (56) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ss_sold_date_sk#35] -Right keys [1]: [d_date_sk#37] +Right keys [1]: [d_date_sk#21] Join condition: None (57) Project [codegen id : 13] Output [2]: [ss_customer_sk#33, ss_ext_sales_price#34] -Input [4]: [ss_customer_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35, d_date_sk#37] +Input [4]: [ss_customer_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35, d_date_sk#21] (58) Exchange Input [2]: [ss_customer_sk#33, ss_ext_sales_price#34] -Arguments: hashpartitioning(ss_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#44] +Arguments: hashpartitioning(ss_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#43] (59) Sort [codegen id : 14] Input [2]: [ss_customer_sk#33, ss_ext_sales_price#34] @@ -348,37 +348,37 @@ Input [3]: [c_customer_sk#30, ss_customer_sk#33, ss_ext_sales_price#34] Input [2]: [c_customer_sk#30, ss_ext_sales_price#34] Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#34))] -Aggregate Attributes [1]: [sum#45] -Results [2]: [c_customer_sk#30, sum#46] +Aggregate Attributes [1]: [sum#44] +Results [2]: [c_customer_sk#30, sum#45] (63) HashAggregate [codegen id : 15] -Input [2]: [c_customer_sk#30, sum#46] +Input [2]: [c_customer_sk#30, sum#45] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#34))#47] -Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#47,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#48] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#34))#46] +Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#46,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#47] (64) HashAggregate [codegen id : 15] -Input [1]: [segment#48] -Keys [1]: [segment#48] +Input [1]: [segment#47] +Keys [1]: [segment#47] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#49] -Results [2]: [segment#48, count#50] +Aggregate Attributes [1]: [count#48] +Results [2]: [segment#47, count#49] (65) Exchange -Input [2]: [segment#48, count#50] -Arguments: hashpartitioning(segment#48, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [2]: [segment#47, count#49] +Arguments: hashpartitioning(segment#47, 5), ENSURE_REQUIREMENTS, [id=#50] (66) HashAggregate [codegen id : 16] -Input [2]: [segment#48, count#50] -Keys [1]: [segment#48] +Input [2]: [segment#47, count#49] +Keys [1]: [segment#47] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#52] -Results [3]: [segment#48, count(1)#52 AS num_customers#53, (segment#48 * 50) AS segment_base#54] +Aggregate Attributes [1]: [count(1)#51] +Results [3]: [segment#47, count(1)#51 AS num_customers#52, (segment#47 * 50) AS segment_base#53] (67) TakeOrderedAndProject -Input [3]: [segment#48, num_customers#53, segment_base#54] -Arguments: 100, [segment#48 ASC NULLS FIRST, num_customers#53 ASC NULLS FIRST], [segment#48, num_customers#53, segment_base#54] +Input [3]: [segment#47, num_customers#52, segment_base#53] +Arguments: 100, [segment#47 ASC NULLS FIRST, num_customers#52 ASC NULLS FIRST], [segment#47, num_customers#52, segment_base#53] ===== Subqueries ===== @@ -396,9 +396,9 @@ ReusedExchange (69) (69) ReusedExchange [Reuses operator id: 55] -Output [1]: [d_date_sk#37] +Output [1]: [d_date_sk#21] -Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#39, [id=#40] +Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#38, [id=#39] * HashAggregate (76) +- Exchange (75) +- * HashAggregate (74) @@ -409,42 +409,42 @@ Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (70) Scan parquet default.date_dim -Output [3]: [d_month_seq#55, d_year#56, d_moy#57] +Output [3]: [d_month_seq#37, d_year#22, d_moy#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (71) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#55, d_year#56, d_moy#57] +Input [3]: [d_month_seq#37, d_year#22, d_moy#23] (72) Filter [codegen id : 1] -Input [3]: [d_month_seq#55, d_year#56, d_moy#57] -Condition : (((isnotnull(d_year#56) AND isnotnull(d_moy#57)) AND (d_year#56 = 1998)) AND (d_moy#57 = 12)) +Input [3]: [d_month_seq#37, d_year#22, d_moy#23] +Condition : (((isnotnull(d_year#22) AND isnotnull(d_moy#23)) AND (d_year#22 = 1998)) AND (d_moy#23 = 12)) (73) Project [codegen id : 1] -Output [1]: [(d_month_seq#55 + 1) AS (d_month_seq + 1)#58] -Input [3]: [d_month_seq#55, d_year#56, d_moy#57] +Output [1]: [(d_month_seq#37 + 1) AS (d_month_seq + 1)#54] +Input [3]: [d_month_seq#37, d_year#22, d_moy#23] (74) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 1)#58] -Keys [1]: [(d_month_seq + 1)#58] +Input [1]: [(d_month_seq + 1)#54] +Keys [1]: [(d_month_seq + 1)#54] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#58] +Results [1]: [(d_month_seq + 1)#54] (75) Exchange -Input [1]: [(d_month_seq + 1)#58] -Arguments: hashpartitioning((d_month_seq + 1)#58, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [1]: [(d_month_seq + 1)#54] +Arguments: hashpartitioning((d_month_seq + 1)#54, 5), ENSURE_REQUIREMENTS, [id=#55] (76) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#58] -Keys [1]: [(d_month_seq + 1)#58] +Input [1]: [(d_month_seq + 1)#54] +Keys [1]: [(d_month_seq + 1)#54] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#58] +Results [1]: [(d_month_seq + 1)#54] -Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#40, [id=#41] * HashAggregate (83) +- Exchange (82) +- * HashAggregate (81) @@ -455,39 +455,39 @@ Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (77) Scan parquet default.date_dim -Output [3]: [d_month_seq#60, d_year#61, d_moy#62] +Output [3]: [d_month_seq#37, d_year#22, d_moy#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (78) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#60, d_year#61, d_moy#62] +Input [3]: [d_month_seq#37, d_year#22, d_moy#23] (79) Filter [codegen id : 1] -Input [3]: [d_month_seq#60, d_year#61, d_moy#62] -Condition : (((isnotnull(d_year#61) AND isnotnull(d_moy#62)) AND (d_year#61 = 1998)) AND (d_moy#62 = 12)) +Input [3]: [d_month_seq#37, d_year#22, d_moy#23] +Condition : (((isnotnull(d_year#22) AND isnotnull(d_moy#23)) AND (d_year#22 = 1998)) AND (d_moy#23 = 12)) (80) Project [codegen id : 1] -Output [1]: [(d_month_seq#60 + 3) AS (d_month_seq + 3)#63] -Input [3]: [d_month_seq#60, d_year#61, d_moy#62] +Output [1]: [(d_month_seq#37 + 3) AS (d_month_seq + 3)#56] +Input [3]: [d_month_seq#37, d_year#22, d_moy#23] (81) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 3)#63] -Keys [1]: [(d_month_seq + 3)#63] +Input [1]: [(d_month_seq + 3)#56] +Keys [1]: [(d_month_seq + 3)#56] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#63] +Results [1]: [(d_month_seq + 3)#56] (82) Exchange -Input [1]: [(d_month_seq + 3)#63] -Arguments: hashpartitioning((d_month_seq + 3)#63, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [1]: [(d_month_seq + 3)#56] +Arguments: hashpartitioning((d_month_seq + 3)#56, 5), ENSURE_REQUIREMENTS, [id=#57] (83) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#63] -Keys [1]: [(d_month_seq + 3)#63] +Input [1]: [(d_month_seq + 3)#56] +Keys [1]: [(d_month_seq + 3)#56] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#63] +Results [1]: [(d_month_seq + 3)#56] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index 928f16637bef8..b14ae55d75093 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -295,75 +295,75 @@ Output [3]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28] Input [7]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28, ca_county#32, ca_state#33, s_county#35, s_state#36] (51) Scan parquet default.date_dim -Output [2]: [d_date_sk#38, d_month_seq#39] +Output [2]: [d_date_sk#18, d_month_seq#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 10] -Input [2]: [d_date_sk#38, d_month_seq#39] +Input [2]: [d_date_sk#18, d_month_seq#38] (53) Filter [codegen id : 10] -Input [2]: [d_date_sk#38, d_month_seq#39] -Condition : (((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= Subquery scalar-subquery#40, [id=#41])) AND (d_month_seq#39 <= Subquery scalar-subquery#42, [id=#43])) AND isnotnull(d_date_sk#38)) +Input [2]: [d_date_sk#18, d_month_seq#38] +Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= Subquery scalar-subquery#39, [id=#40])) AND (d_month_seq#38 <= Subquery scalar-subquery#41, [id=#42])) AND isnotnull(d_date_sk#18)) (54) Project [codegen id : 10] -Output [1]: [d_date_sk#38] -Input [2]: [d_date_sk#38, d_month_seq#39] +Output [1]: [d_date_sk#18] +Input [2]: [d_date_sk#18, d_month_seq#38] (55) BroadcastExchange -Input [1]: [d_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] +Input [1]: [d_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#43] (56) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#38] +Right keys [1]: [d_date_sk#18] Join condition: None (57) Project [codegen id : 11] Output [2]: [c_customer_sk#22, ss_ext_sales_price#27] -Input [4]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#38] +Input [4]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#18] (58) HashAggregate [codegen id : 11] Input [2]: [c_customer_sk#22, ss_ext_sales_price#27] Keys [1]: [c_customer_sk#22] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum#45] -Results [2]: [c_customer_sk#22, sum#46] +Aggregate Attributes [1]: [sum#44] +Results [2]: [c_customer_sk#22, sum#45] (59) Exchange -Input [2]: [c_customer_sk#22, sum#46] -Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [2]: [c_customer_sk#22, sum#45] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#46] (60) HashAggregate [codegen id : 12] -Input [2]: [c_customer_sk#22, sum#46] +Input [2]: [c_customer_sk#22, sum#45] Keys [1]: [c_customer_sk#22] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#48] -Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#48,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#49] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#47] +Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#47,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#48] (61) HashAggregate [codegen id : 12] -Input [1]: [segment#49] -Keys [1]: [segment#49] +Input [1]: [segment#48] +Keys [1]: [segment#48] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#50] -Results [2]: [segment#49, count#51] +Aggregate Attributes [1]: [count#49] +Results [2]: [segment#48, count#50] (62) Exchange -Input [2]: [segment#49, count#51] -Arguments: hashpartitioning(segment#49, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [2]: [segment#48, count#50] +Arguments: hashpartitioning(segment#48, 5), ENSURE_REQUIREMENTS, [id=#51] (63) HashAggregate [codegen id : 13] -Input [2]: [segment#49, count#51] -Keys [1]: [segment#49] +Input [2]: [segment#48, count#50] +Keys [1]: [segment#48] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#53] -Results [3]: [segment#49, count(1)#53 AS num_customers#54, (segment#49 * 50) AS segment_base#55] +Aggregate Attributes [1]: [count(1)#52] +Results [3]: [segment#48, count(1)#52 AS num_customers#53, (segment#48 * 50) AS segment_base#54] (64) TakeOrderedAndProject -Input [3]: [segment#49, num_customers#54, segment_base#55] -Arguments: 100, [segment#49 ASC NULLS FIRST, num_customers#54 ASC NULLS FIRST], [segment#49, num_customers#54, segment_base#55] +Input [3]: [segment#48, num_customers#53, segment_base#54] +Arguments: 100, [segment#48 ASC NULLS FIRST, num_customers#53 ASC NULLS FIRST], [segment#48, num_customers#53, segment_base#54] ===== Subqueries ===== @@ -381,9 +381,9 @@ ReusedExchange (66) (66) ReusedExchange [Reuses operator id: 55] -Output [1]: [d_date_sk#38] +Output [1]: [d_date_sk#18] -Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#40, [id=#41] +Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#39, [id=#40] * HashAggregate (73) +- Exchange (72) +- * HashAggregate (71) @@ -394,42 +394,42 @@ Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (67) Scan parquet default.date_dim -Output [3]: [d_month_seq#56, d_year#57, d_moy#58] +Output [3]: [d_month_seq#38, d_year#19, d_moy#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (68) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#56, d_year#57, d_moy#58] +Input [3]: [d_month_seq#38, d_year#19, d_moy#20] (69) Filter [codegen id : 1] -Input [3]: [d_month_seq#56, d_year#57, d_moy#58] -Condition : (((isnotnull(d_year#57) AND isnotnull(d_moy#58)) AND (d_year#57 = 1998)) AND (d_moy#58 = 12)) +Input [3]: [d_month_seq#38, d_year#19, d_moy#20] +Condition : (((isnotnull(d_year#19) AND isnotnull(d_moy#20)) AND (d_year#19 = 1998)) AND (d_moy#20 = 12)) (70) Project [codegen id : 1] -Output [1]: [(d_month_seq#56 + 1) AS (d_month_seq + 1)#59] -Input [3]: [d_month_seq#56, d_year#57, d_moy#58] +Output [1]: [(d_month_seq#38 + 1) AS (d_month_seq + 1)#55] +Input [3]: [d_month_seq#38, d_year#19, d_moy#20] (71) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 1)#59] -Keys [1]: [(d_month_seq + 1)#59] +Input [1]: [(d_month_seq + 1)#55] +Keys [1]: [(d_month_seq + 1)#55] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#59] +Results [1]: [(d_month_seq + 1)#55] (72) Exchange -Input [1]: [(d_month_seq + 1)#59] -Arguments: hashpartitioning((d_month_seq + 1)#59, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [1]: [(d_month_seq + 1)#55] +Arguments: hashpartitioning((d_month_seq + 1)#55, 5), ENSURE_REQUIREMENTS, [id=#56] (73) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#59] -Keys [1]: [(d_month_seq + 1)#59] +Input [1]: [(d_month_seq + 1)#55] +Keys [1]: [(d_month_seq + 1)#55] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#59] +Results [1]: [(d_month_seq + 1)#55] -Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#42, [id=#43] +Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (80) +- Exchange (79) +- * HashAggregate (78) @@ -440,39 +440,39 @@ Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (74) Scan parquet default.date_dim -Output [3]: [d_month_seq#61, d_year#62, d_moy#63] +Output [3]: [d_month_seq#38, d_year#19, d_moy#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (75) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#61, d_year#62, d_moy#63] +Input [3]: [d_month_seq#38, d_year#19, d_moy#20] (76) Filter [codegen id : 1] -Input [3]: [d_month_seq#61, d_year#62, d_moy#63] -Condition : (((isnotnull(d_year#62) AND isnotnull(d_moy#63)) AND (d_year#62 = 1998)) AND (d_moy#63 = 12)) +Input [3]: [d_month_seq#38, d_year#19, d_moy#20] +Condition : (((isnotnull(d_year#19) AND isnotnull(d_moy#20)) AND (d_year#19 = 1998)) AND (d_moy#20 = 12)) (77) Project [codegen id : 1] -Output [1]: [(d_month_seq#61 + 3) AS (d_month_seq + 3)#64] -Input [3]: [d_month_seq#61, d_year#62, d_moy#63] +Output [1]: [(d_month_seq#38 + 3) AS (d_month_seq + 3)#57] +Input [3]: [d_month_seq#38, d_year#19, d_moy#20] (78) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 3)#64] -Keys [1]: [(d_month_seq + 3)#64] +Input [1]: [(d_month_seq + 3)#57] +Keys [1]: [(d_month_seq + 3)#57] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#64] +Results [1]: [(d_month_seq + 3)#57] (79) Exchange -Input [1]: [(d_month_seq + 3)#64] -Arguments: hashpartitioning((d_month_seq + 3)#64, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [1]: [(d_month_seq + 3)#57] +Arguments: hashpartitioning((d_month_seq + 3)#57, 5), ENSURE_REQUIREMENTS, [id=#58] (80) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#64] -Keys [1]: [(d_month_seq + 3)#64] +Input [1]: [(d_month_seq + 3)#57] +Keys [1]: [(d_month_seq + 3)#57] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#64] +Results [1]: [(d_month_seq + 3)#57] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt index 47eda1483a06c..8e3bf5bf57bcf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt @@ -160,30 +160,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#15, i_color#16] +Output [2]: [i_item_id#14, i_color#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#15, i_color#16] +Input [2]: [i_item_id#14, i_color#15] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#15, i_color#16] -Condition : i_color#16 IN (slate ,blanched ,burnished ) +Input [2]: [i_item_id#14, i_color#15] +Condition : i_color#15 IN (slate ,blanched ,burnished ) (24) Project [codegen id : 3] -Output [1]: [i_item_id#15] -Input [2]: [i_item_id#15, i_color#16] +Output [1]: [i_item_id#14 AS i_item_id#14#16] +Input [2]: [i_item_id#14, i_color#15] (25) BroadcastExchange -Input [1]: [i_item_id#15] +Input [1]: [i_item_id#14#16] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#15] +Right keys [1]: [i_item_id#14#16] Join condition: None (27) BroadcastExchange @@ -233,127 +233,127 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#28] +Output [1]: [d_date_sk#6] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#28] +Right keys [1]: [d_date_sk#6] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#29] +Output [1]: [ca_address_sk#10] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#29] +Right keys [1]: [ca_address_sk#10] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#30, i_item_id#31] +Output [2]: [i_item_sk#13, i_item_id#14] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#30] +Right keys [1]: [i_item_sk#13] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#31] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] +Output [2]: [cs_ext_sales_price#26, i_item_id#14] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#31] -Keys [1]: [i_item_id#31] +Input [2]: [cs_ext_sales_price#26, i_item_id#14] +Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#31, sum#33] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_item_id#14, sum#29] (46) Exchange -Input [2]: [i_item_id#31, sum#33] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [2]: [i_item_id#14, sum#29] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] (47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#31, sum#33] -Keys [1]: [i_item_id#31] +Input [2]: [i_item_id#14, sum#29] +Keys [1]: [i_item_id#14] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] -Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] +Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] (48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] (50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#41] +Output [1]: [d_date_sk#6] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#41] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#6] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] -Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] +Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#42] +Output [1]: [ca_address_sk#10] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#42] +Left keys [1]: [ws_bill_addr_sk#34] +Right keys [1]: [ca_address_sk#10] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] +Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#43, i_item_id#44] +Output [2]: [i_item_sk#13, i_item_id#14] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#43] +Left keys [1]: [ws_item_sk#33] +Right keys [1]: [i_item_sk#13] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#39, i_item_id#44] -Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] +Output [2]: [ws_ext_sales_price#35, i_item_id#14] +Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#39, i_item_id#44] -Keys [1]: [i_item_id#44] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum#45] -Results [2]: [i_item_id#44, sum#46] +Input [2]: [ws_ext_sales_price#35, i_item_id#14] +Keys [1]: [i_item_id#14] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#37] +Results [2]: [i_item_id#14, sum#38] (61) Exchange -Input [2]: [i_item_id#44, sum#46] -Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [2]: [i_item_id#14, sum#38] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] (62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#44, sum#46] -Keys [1]: [i_item_id#44] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] -Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] +Input [2]: [i_item_id#14, sum#38] +Keys [1]: [i_item_id#14] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] +Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] (63) Union @@ -361,23 +361,23 @@ Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39) Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [3]: [i_item_id#14, sum#52, isEmpty#53] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [i_item_id#14, sum#44, isEmpty#45] (65) Exchange -Input [3]: [i_item_id#14, sum#52, isEmpty#53] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [3]: [i_item_id#14, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] (66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#52, isEmpty#53] +Input [3]: [i_item_id#14, sum#44, isEmpty#45] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#55] -Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] +Aggregate Attributes [1]: [sum(total_sales#23)#47] +Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] (67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#56] -Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] +Input [2]: [i_item_id#14, total_sales#48] +Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] ===== Subqueries ===== @@ -390,6 +390,6 @@ Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 47eda1483a06c..8e3bf5bf57bcf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -160,30 +160,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#15, i_color#16] +Output [2]: [i_item_id#14, i_color#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#15, i_color#16] +Input [2]: [i_item_id#14, i_color#15] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#15, i_color#16] -Condition : i_color#16 IN (slate ,blanched ,burnished ) +Input [2]: [i_item_id#14, i_color#15] +Condition : i_color#15 IN (slate ,blanched ,burnished ) (24) Project [codegen id : 3] -Output [1]: [i_item_id#15] -Input [2]: [i_item_id#15, i_color#16] +Output [1]: [i_item_id#14 AS i_item_id#14#16] +Input [2]: [i_item_id#14, i_color#15] (25) BroadcastExchange -Input [1]: [i_item_id#15] +Input [1]: [i_item_id#14#16] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#15] +Right keys [1]: [i_item_id#14#16] Join condition: None (27) BroadcastExchange @@ -233,127 +233,127 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#28] +Output [1]: [d_date_sk#6] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#28] +Right keys [1]: [d_date_sk#6] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#29] +Output [1]: [ca_address_sk#10] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#29] +Right keys [1]: [ca_address_sk#10] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#30, i_item_id#31] +Output [2]: [i_item_sk#13, i_item_id#14] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#30] +Right keys [1]: [i_item_sk#13] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#31] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] +Output [2]: [cs_ext_sales_price#26, i_item_id#14] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#31] -Keys [1]: [i_item_id#31] +Input [2]: [cs_ext_sales_price#26, i_item_id#14] +Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#31, sum#33] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_item_id#14, sum#29] (46) Exchange -Input [2]: [i_item_id#31, sum#33] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [2]: [i_item_id#14, sum#29] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] (47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#31, sum#33] -Keys [1]: [i_item_id#31] +Input [2]: [i_item_id#14, sum#29] +Keys [1]: [i_item_id#14] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] -Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] +Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] (48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] (50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#41] +Output [1]: [d_date_sk#6] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#41] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#6] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] -Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] +Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#42] +Output [1]: [ca_address_sk#10] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#42] +Left keys [1]: [ws_bill_addr_sk#34] +Right keys [1]: [ca_address_sk#10] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] +Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#43, i_item_id#44] +Output [2]: [i_item_sk#13, i_item_id#14] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#43] +Left keys [1]: [ws_item_sk#33] +Right keys [1]: [i_item_sk#13] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#39, i_item_id#44] -Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] +Output [2]: [ws_ext_sales_price#35, i_item_id#14] +Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#39, i_item_id#44] -Keys [1]: [i_item_id#44] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum#45] -Results [2]: [i_item_id#44, sum#46] +Input [2]: [ws_ext_sales_price#35, i_item_id#14] +Keys [1]: [i_item_id#14] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#37] +Results [2]: [i_item_id#14, sum#38] (61) Exchange -Input [2]: [i_item_id#44, sum#46] -Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [2]: [i_item_id#14, sum#38] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] (62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#44, sum#46] -Keys [1]: [i_item_id#44] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] -Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] +Input [2]: [i_item_id#14, sum#38] +Keys [1]: [i_item_id#14] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] +Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] (63) Union @@ -361,23 +361,23 @@ Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39) Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [3]: [i_item_id#14, sum#52, isEmpty#53] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [i_item_id#14, sum#44, isEmpty#45] (65) Exchange -Input [3]: [i_item_id#14, sum#52, isEmpty#53] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [3]: [i_item_id#14, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] (66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#52, isEmpty#53] +Input [3]: [i_item_id#14, sum#44, isEmpty#45] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#55] -Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] +Aggregate Attributes [1]: [sum(total_sales#23)#47] +Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] (67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#56] -Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] +Input [2]: [i_item_id#14, total_sales#48] +Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] ===== Subqueries ===== @@ -390,6 +390,6 @@ Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 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 6e39745703215..f0be099379418 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 @@ -229,78 +229,78 @@ Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] (39) HashAggregate [codegen id : 21] Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] Keys [5]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33] -Functions [1]: [sum(UnscaledValue(cs_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#35))#36] -Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#35))#36,17,2) AS sum_sales#37] +Functions [1]: [sum(UnscaledValue(cs_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#3))#35] +Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#3))#35,17,2) AS sum_sales#36] (40) Exchange -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#37] (41) Sort [codegen id : 22] -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 (42) Window -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] -Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] (43) Project [codegen id : 23] -Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] -Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] +Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] (44) Exchange -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#38 + 1), 5), ENSURE_REQUIREMENTS, [id=#39] (45) Sort [codegen id : 24] -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] -Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] +Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#38 + 1) ASC NULLS FIRST], false, 0 (46) SortMergeJoin [codegen id : 25] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#39 + 1)] +Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#38 + 1)] Join condition: None (47) Project [codegen id : 25] -Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] -Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] +Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] (48) ReusedExchange [Reuses operator id: 40] -Output [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] +Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] (49) Sort [codegen id : 34] -Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 (50) Window -Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] -Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#41, i_brand#42, cc_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#41, i_brand#42, cc_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] +Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] (51) Project [codegen id : 35] -Output [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] -Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] +Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] +Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] (52) Exchange -Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] -Arguments: hashpartitioning(i_category#41, i_brand#42, cc_name#43, (rn#47 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] +Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] +Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (rn#46 - 1), 5), ENSURE_REQUIREMENTS, [id=#47] (53) Sort [codegen id : 36] -Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, (rn#47 - 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 (54) SortMergeJoin [codegen id : 37] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#41, i_brand#42, cc_name#43, (rn#47 - 1)] +Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (rn#46 - 1)] Join condition: None (55) Project [codegen id : 37] -Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#49, sum_sales#46 AS nsum#50] -Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#48, sum_sales#45 AS nsum#49] +Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] (56) TakeOrderedAndProject -Input [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] +Input [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] ===== Subqueries ===== 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 fa24f5ede5453..1eb7dbe89d6e1 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 @@ -202,70 +202,70 @@ Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] (34) HashAggregate [codegen id : 13] 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] +Functions [1]: [sum(UnscaledValue(cs_sales_price#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#33] +Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#33,17,2) AS sum_sales#34] (35) 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#34] +Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#35] (36) Sort [codegen id : 14] -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#34] 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 (37) 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#34] +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#36], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (38) Project [codegen id : 15] -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#34, rn#36] +Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34, rn#36] (39) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#38] +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#37] (40) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -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#36 + 1)] Join condition: None (41) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35] -Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34] +Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] (42) ReusedExchange [Reuses operator id: 35] -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#43] (43) Sort [codegen id : 21] -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#43] +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 (44) 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#43] +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#44], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] (45) Project [codegen id : 22] -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#43, rn#44] +Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43, rn#44] (46) BroadcastExchange -Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#46] +Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#45] (47) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -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#44 - 1)] Join condition: None (48) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] -Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#34 AS psum#46, sum_sales#43 AS nsum#47] +Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34, i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] (49) TakeOrderedAndProject -Input [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, 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#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, psum#47, nsum#48] +Input [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, 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#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, psum#46, nsum#47] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index ee3abb39ed053..f6dd5a59de4d4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -467,7 +467,7 @@ Input [2]: [d_date#5, d_week_seq#6] (82) Filter [codegen id : 1] Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_date#5) AND (d_date#5 = 10959)) +Condition : (isnotnull(d_date#5) AND (d_date#5 = 2000-01-03)) (83) Project [codegen id : 1] Output [1]: [d_week_seq#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index dd37b8801e957..ab4c4ad4ae65b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -467,7 +467,7 @@ Input [2]: [d_date#8, d_week_seq#9] (82) Filter [codegen id : 1] Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_date#8) AND (d_date#8 = 10959)) +Condition : (isnotnull(d_date#8) AND (d_date#8 = 2000-01-03)) (83) Project [codegen id : 1] Output [1]: [d_week_seq#9] 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..fb57c1b00ecf3 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))#61, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67] +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))#61,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63,17,2) AS tue_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67,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#37, s_store_id#38] 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#37, s_store_id#38] (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#37, s_store_id#38] +Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) (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#37, s_store_id#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#68] (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#37] 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#38] +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#37, s_store_id#38] (34) Scan parquet default.date_dim -Output [2]: [d_month_seq#75, d_week_seq#76] +Output [2]: [d_month_seq#69, d_week_seq#70] 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#69, d_week_seq#70] (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#69, d_week_seq#70] +Condition : (((isnotnull(d_month_seq#69) AND (d_month_seq#69 >= 1224)) AND (d_month_seq#69 <= 1235)) AND isnotnull(d_week_seq#70)) (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#70] +Input [2]: [d_month_seq#69, d_week_seq#70] (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#70] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] (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#70] 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#72, s_store_id#38 AS s_store_id2#73, sun_sales#30 AS sun_sales2#74, mon_sales#31 AS mon_sales2#75, tue_sales#32 AS tue_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 [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#38, d_week_seq#70] (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#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_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] (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#73, (d_week_seq2#72 - 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#74)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#82, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#75)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#83, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#76)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#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 [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#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] (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)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales2)#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_sales2)#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/q59/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt index cf22f178dd33f..fb57c1b00ecf3 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))#61, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67] +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))#61,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63,17,2) AS tue_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67,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#37, s_store_id#38] 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#37, s_store_id#38] (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#37, s_store_id#38] +Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) (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#37, s_store_id#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#68] (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#37] 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#38] +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#37, s_store_id#38] (34) Scan parquet default.date_dim -Output [2]: [d_month_seq#75, d_week_seq#76] +Output [2]: [d_month_seq#69, d_week_seq#70] 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#69, d_week_seq#70] (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#69, d_week_seq#70] +Condition : (((isnotnull(d_month_seq#69) AND (d_month_seq#69 >= 1224)) AND (d_month_seq#69 <= 1235)) AND isnotnull(d_week_seq#70)) (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#70] +Input [2]: [d_month_seq#69, d_week_seq#70] (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#70] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] (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#70] 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#72, s_store_id#38 AS s_store_id2#73, sun_sales#30 AS sun_sales2#74, mon_sales#31 AS mon_sales2#75, tue_sales#32 AS tue_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 [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#38, d_week_seq#70] (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#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_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] (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#73, (d_week_seq2#72 - 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#74)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#82, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#75)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#83, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#76)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#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 [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#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] (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)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales2)#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_sales2)#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/q60.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt index 49cfd232239aa..10b8d0d9e7f05 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt @@ -160,30 +160,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#15, i_category#16] +Output [2]: [i_item_id#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#15, i_category#16] +Input [2]: [i_item_id#14, i_category#15] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#15, i_category#16] -Condition : (isnotnull(i_category#16) AND (i_category#16 = Music )) +Input [2]: [i_item_id#14, i_category#15] +Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) (24) Project [codegen id : 3] -Output [1]: [i_item_id#15] -Input [2]: [i_item_id#15, i_category#16] +Output [1]: [i_item_id#14 AS i_item_id#14#16] +Input [2]: [i_item_id#14, i_category#15] (25) BroadcastExchange -Input [1]: [i_item_id#15] +Input [1]: [i_item_id#14#16] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#15] +Right keys [1]: [i_item_id#14#16] Join condition: None (27) BroadcastExchange @@ -233,127 +233,127 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#28] +Output [1]: [d_date_sk#6] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#28] +Right keys [1]: [d_date_sk#6] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#29] +Output [1]: [ca_address_sk#10] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#29] +Right keys [1]: [ca_address_sk#10] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#30, i_item_id#31] +Output [2]: [i_item_sk#13, i_item_id#14] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#30] +Right keys [1]: [i_item_sk#13] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#31] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] +Output [2]: [cs_ext_sales_price#26, i_item_id#14] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#31] -Keys [1]: [i_item_id#31] +Input [2]: [cs_ext_sales_price#26, i_item_id#14] +Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#31, sum#33] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_item_id#14, sum#29] (46) Exchange -Input [2]: [i_item_id#31, sum#33] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [2]: [i_item_id#14, sum#29] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] (47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#31, sum#33] -Keys [1]: [i_item_id#31] +Input [2]: [i_item_id#14, sum#29] +Keys [1]: [i_item_id#14] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] -Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] +Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] (48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] (50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#41] +Output [1]: [d_date_sk#6] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#41] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#6] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] -Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] +Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#42] +Output [1]: [ca_address_sk#10] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#42] +Left keys [1]: [ws_bill_addr_sk#34] +Right keys [1]: [ca_address_sk#10] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] +Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#43, i_item_id#44] +Output [2]: [i_item_sk#13, i_item_id#14] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#43] +Left keys [1]: [ws_item_sk#33] +Right keys [1]: [i_item_sk#13] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#39, i_item_id#44] -Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] +Output [2]: [ws_ext_sales_price#35, i_item_id#14] +Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#39, i_item_id#44] -Keys [1]: [i_item_id#44] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum#45] -Results [2]: [i_item_id#44, sum#46] +Input [2]: [ws_ext_sales_price#35, i_item_id#14] +Keys [1]: [i_item_id#14] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#37] +Results [2]: [i_item_id#14, sum#38] (61) Exchange -Input [2]: [i_item_id#44, sum#46] -Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [2]: [i_item_id#14, sum#38] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] (62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#44, sum#46] -Keys [1]: [i_item_id#44] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] -Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] +Input [2]: [i_item_id#14, sum#38] +Keys [1]: [i_item_id#14] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] +Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] (63) Union @@ -361,23 +361,23 @@ Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39) Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [3]: [i_item_id#14, sum#52, isEmpty#53] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [i_item_id#14, sum#44, isEmpty#45] (65) Exchange -Input [3]: [i_item_id#14, sum#52, isEmpty#53] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [3]: [i_item_id#14, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] (66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#52, isEmpty#53] +Input [3]: [i_item_id#14, sum#44, isEmpty#45] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#55] -Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] +Aggregate Attributes [1]: [sum(total_sales#23)#47] +Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] (67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#56] -Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] +Input [2]: [i_item_id#14, total_sales#48] +Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] ===== Subqueries ===== @@ -390,6 +390,6 @@ Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 49cfd232239aa..10b8d0d9e7f05 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -160,30 +160,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#15, i_category#16] +Output [2]: [i_item_id#14, i_category#15] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#15, i_category#16] +Input [2]: [i_item_id#14, i_category#15] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#15, i_category#16] -Condition : (isnotnull(i_category#16) AND (i_category#16 = Music )) +Input [2]: [i_item_id#14, i_category#15] +Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) (24) Project [codegen id : 3] -Output [1]: [i_item_id#15] -Input [2]: [i_item_id#15, i_category#16] +Output [1]: [i_item_id#14 AS i_item_id#14#16] +Input [2]: [i_item_id#14, i_category#15] (25) BroadcastExchange -Input [1]: [i_item_id#15] +Input [1]: [i_item_id#14#16] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#15] +Right keys [1]: [i_item_id#14#16] Join condition: None (27) BroadcastExchange @@ -233,127 +233,127 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#28] +Output [1]: [d_date_sk#6] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#28] +Right keys [1]: [d_date_sk#6] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#29] +Output [1]: [ca_address_sk#10] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#29] +Right keys [1]: [ca_address_sk#10] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#30, i_item_id#31] +Output [2]: [i_item_sk#13, i_item_id#14] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#30] +Right keys [1]: [i_item_sk#13] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#31] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] +Output [2]: [cs_ext_sales_price#26, i_item_id#14] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#31] -Keys [1]: [i_item_id#31] +Input [2]: [cs_ext_sales_price#26, i_item_id#14] +Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#32] -Results [2]: [i_item_id#31, sum#33] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_item_id#14, sum#29] (46) Exchange -Input [2]: [i_item_id#31, sum#33] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [2]: [i_item_id#14, sum#29] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] (47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#31, sum#33] -Keys [1]: [i_item_id#31] +Input [2]: [i_item_id#14, sum#29] +Keys [1]: [i_item_id#14] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] -Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] +Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] (48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] (50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] -Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#41] +Output [1]: [d_date_sk#6] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#41] +Left keys [1]: [ws_sold_date_sk#36] +Right keys [1]: [d_date_sk#6] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] -Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] +Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#42] +Output [1]: [ca_address_sk#10] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#38] -Right keys [1]: [ca_address_sk#42] +Left keys [1]: [ws_bill_addr_sk#34] +Right keys [1]: [ca_address_sk#10] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] -Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] +Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#43, i_item_id#44] +Output [2]: [i_item_sk#13, i_item_id#14] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#43] +Left keys [1]: [ws_item_sk#33] +Right keys [1]: [i_item_sk#13] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#39, i_item_id#44] -Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] +Output [2]: [ws_ext_sales_price#35, i_item_id#14] +Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#39, i_item_id#44] -Keys [1]: [i_item_id#44] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum#45] -Results [2]: [i_item_id#44, sum#46] +Input [2]: [ws_ext_sales_price#35, i_item_id#14] +Keys [1]: [i_item_id#14] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum#37] +Results [2]: [i_item_id#14, sum#38] (61) Exchange -Input [2]: [i_item_id#44, sum#46] -Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [2]: [i_item_id#14, sum#38] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] (62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#44, sum#46] -Keys [1]: [i_item_id#44] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] -Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] +Input [2]: [i_item_id#14, sum#38] +Keys [1]: [i_item_id#14] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] +Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] (63) Union @@ -361,23 +361,23 @@ Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39) Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [3]: [i_item_id#14, sum#52, isEmpty#53] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [i_item_id#14, sum#44, isEmpty#45] (65) Exchange -Input [3]: [i_item_id#14, sum#52, isEmpty#53] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [3]: [i_item_id#14, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] (66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#52, isEmpty#53] +Input [3]: [i_item_id#14, sum#44, isEmpty#45] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#55] -Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] +Aggregate Attributes [1]: [sum(total_sales#23)#47] +Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] (67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#56] -Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] +Input [2]: [i_item_id#14, total_sales#48] +Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] ===== Subqueries ===== @@ -390,6 +390,6 @@ Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt index 410fd9bc3d4e7..23807f0561551 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt @@ -286,100 +286,100 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#32] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#32,17,2) AS promotions#33] (48) Scan parquet default.store_sales -Output [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] (50) Filter [codegen id : 13] -Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_store_sk#36) AND isnotnull(ss_customer_sk#35)) AND isnotnull(ss_item_sk#34)) +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] +Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#39] +Output [1]: [d_date_sk#8] (52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#38] -Right keys [1]: [d_date_sk#39] +Left keys [1]: [ss_sold_date_sk#6] +Right keys [1]: [d_date_sk#8] Join condition: None (53) Project [codegen id : 13] -Output [4]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37] -Input [6]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38, d_date_sk#39] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [i_item_sk#40] +Output [1]: [i_item_sk#12] (55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#34] -Right keys [1]: [i_item_sk#40] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#12] Join condition: None (56) Project [codegen id : 13] -Output [3]: [ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37] -Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, i_item_sk#40] +Output [3]: [ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, i_item_sk#12] (57) ReusedExchange [Reuses operator id: 29] -Output [1]: [s_store_sk#41] +Output [1]: [s_store_sk#20] (58) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#36] -Right keys [1]: [s_store_sk#41] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#20] Join condition: None (59) Project [codegen id : 13] -Output [2]: [ss_customer_sk#35, ss_ext_sales_price#37] -Input [4]: [ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, s_store_sk#41] +Output [2]: [ss_customer_sk#2, ss_ext_sales_price#5] +Input [4]: [ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, s_store_sk#20] (60) ReusedExchange [Reuses operator id: 42] -Output [1]: [c_customer_sk#42] +Output [1]: [c_customer_sk#23] (61) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#35] -Right keys [1]: [c_customer_sk#42] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#23] Join condition: None (62) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#37] -Input [3]: [ss_customer_sk#35, ss_ext_sales_price#37, c_customer_sk#42] +Output [1]: [ss_ext_sales_price#5] +Input [3]: [ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#23] (63) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#37] +Input [1]: [ss_ext_sales_price#5] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#37))] -Aggregate Attributes [1]: [sum#43] -Results [1]: [sum#44] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum#34] +Results [1]: [sum#35] (64) Exchange -Input [1]: [sum#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45] +Input [1]: [sum#35] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#36] (65) HashAggregate [codegen id : 14] -Input [1]: [sum#44] +Input [1]: [sum#35] Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#37))#46] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#37))#46,17,2) AS total#47] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#37] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#37,17,2) AS total#38] (66) BroadcastExchange -Input [1]: [total#47] -Arguments: IdentityBroadcastMode, [id=#48] +Input [1]: [total#38] +Arguments: IdentityBroadcastMode, [id=#39] (67) BroadcastNestedLoopJoin [codegen id : 15] Join condition: None (68) Project [codegen id : 15] -Output [3]: [promotions#33, total#47, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#47 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#49] -Input [2]: [promotions#33, total#47] +Output [3]: [promotions#33, total#38, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#38 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] +Input [2]: [promotions#33, total#38] (69) Sort [codegen id : 15] -Input [3]: [promotions#33, total#47, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#49] -Arguments: [promotions#33 ASC NULLS FIRST, total#47 ASC NULLS FIRST], true, 0 +Input [3]: [promotions#33, total#38, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] +Arguments: [promotions#33 ASC NULLS FIRST, total#38 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -390,6 +390,6 @@ ReusedExchange (70) (70) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt index e0bee37e047cb..c0fad04ce4026 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt @@ -289,112 +289,112 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#32] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#32,17,2) AS promotions#33] (48) Scan parquet default.store_sales -Output [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] (50) Filter [codegen id : 13] -Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_store_sk#36) AND isnotnull(ss_customer_sk#35)) AND isnotnull(ss_item_sk#34)) +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] +Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [s_store_sk#39] +Output [1]: [s_store_sk#8] (52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#36] -Right keys [1]: [s_store_sk#39] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#8] Join condition: None (53) Project [codegen id : 13] -Output [4]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37, ss_sold_date_sk#38] -Input [6]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38, s_store_sk#39] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] (54) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#40] +Output [1]: [d_date_sk#16] (55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#38] -Right keys [1]: [d_date_sk#40] +Left keys [1]: [ss_sold_date_sk#6] +Right keys [1]: [d_date_sk#16] Join condition: None (56) Project [codegen id : 13] -Output [3]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37] -Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37, ss_sold_date_sk#38, d_date_sk#40] +Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#16] (57) ReusedExchange [Reuses operator id: 28] -Output [2]: [c_customer_sk#41, c_current_addr_sk#42] +Output [2]: [c_customer_sk#20, c_current_addr_sk#21] (58) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#35] -Right keys [1]: [c_customer_sk#41] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#20] Join condition: None (59) Project [codegen id : 13] -Output [3]: [ss_item_sk#34, ss_ext_sales_price#37, c_current_addr_sk#42] -Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37, c_customer_sk#41, c_current_addr_sk#42] +Output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#21] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#20, c_current_addr_sk#21] (60) ReusedExchange [Reuses operator id: 35] -Output [1]: [ca_address_sk#43] +Output [1]: [ca_address_sk#23] (61) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [c_current_addr_sk#42] -Right keys [1]: [ca_address_sk#43] +Left keys [1]: [c_current_addr_sk#21] +Right keys [1]: [ca_address_sk#23] Join condition: None (62) Project [codegen id : 13] -Output [2]: [ss_item_sk#34, ss_ext_sales_price#37] -Input [4]: [ss_item_sk#34, ss_ext_sales_price#37, c_current_addr_sk#42, ca_address_sk#43] +Output [2]: [ss_item_sk#1, ss_ext_sales_price#5] +Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#21, ca_address_sk#23] (63) ReusedExchange [Reuses operator id: 42] -Output [1]: [i_item_sk#44] +Output [1]: [i_item_sk#26] (64) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#34] -Right keys [1]: [i_item_sk#44] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#26] Join condition: None (65) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#37] -Input [3]: [ss_item_sk#34, ss_ext_sales_price#37, i_item_sk#44] +Output [1]: [ss_ext_sales_price#5] +Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#26] (66) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#37] +Input [1]: [ss_ext_sales_price#5] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#37))] -Aggregate Attributes [1]: [sum#45] -Results [1]: [sum#46] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum#34] +Results [1]: [sum#35] (67) Exchange -Input [1]: [sum#46] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#47] +Input [1]: [sum#35] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#36] (68) HashAggregate [codegen id : 14] -Input [1]: [sum#46] +Input [1]: [sum#35] Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#37))#48] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#37))#48,17,2) AS total#49] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#37] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#37,17,2) AS total#38] (69) BroadcastExchange -Input [1]: [total#49] -Arguments: IdentityBroadcastMode, [id=#50] +Input [1]: [total#38] +Arguments: IdentityBroadcastMode, [id=#39] (70) BroadcastNestedLoopJoin [codegen id : 15] Join condition: None (71) Project [codegen id : 15] -Output [3]: [promotions#33, total#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#49 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#51] -Input [2]: [promotions#33, total#49] +Output [3]: [promotions#33, total#38, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#38 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] +Input [2]: [promotions#33, total#38] (72) Sort [codegen id : 15] -Input [3]: [promotions#33, total#49, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#51] -Arguments: [promotions#33 ASC NULLS FIRST, total#49 ASC NULLS FIRST], true, 0 +Input [3]: [promotions#33, total#38, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] +Arguments: [promotions#33 ASC NULLS FIRST, total#38 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -405,6 +405,6 @@ ReusedExchange (73) (73) ReusedExchange [Reuses operator id: 22] Output [1]: [d_date_sk#16] -Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt index e8259ee31d909..5d74065179f44 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt @@ -795,353 +795,353 @@ Input [17]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_str Arguments: [item_sk#112 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, store_zip#114 ASC NULLS FIRST], false, 0 (132) Scan parquet default.store_sales -Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#140), dynamicpruningexpression(ss_sold_date_sk#140 IN dynamicpruning#141)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#129)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (133) ColumnarToRow [codegen id : 44] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] (134) Filter [codegen id : 44] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Condition : (((((((isnotnull(ss_item_sk#129) AND isnotnull(ss_ticket_number#136)) AND isnotnull(ss_store_sk#134)) AND isnotnull(ss_customer_sk#130)) AND isnotnull(ss_cdemo_sk#131)) AND isnotnull(ss_promo_sk#135)) AND isnotnull(ss_hdemo_sk#132)) AND isnotnull(ss_addr_sk#133)) +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) (135) Exchange -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Arguments: hashpartitioning(cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint), 5), ENSURE_REQUIREMENTS, [id=#142] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: hashpartitioning(cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint), 5), ENSURE_REQUIREMENTS, [id=#130] (136) Sort [codegen id : 45] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Arguments: [cast(ss_item_sk#129 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#136 as bigint) ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: [cast(ss_item_sk#1 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#8 as bigint) ASC NULLS FIRST], false, 0 (137) ReusedExchange [Reuses operator id: 10] -Output [2]: [sr_item_sk#143, sr_ticket_number#144] +Output [2]: [sr_item_sk#15, sr_ticket_number#16] (138) Sort [codegen id : 47] -Input [2]: [sr_item_sk#143, sr_ticket_number#144] -Arguments: [sr_item_sk#143 ASC NULLS FIRST, sr_ticket_number#144 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#15, sr_ticket_number#16] +Arguments: [sr_item_sk#15 ASC NULLS FIRST, sr_ticket_number#16 ASC NULLS FIRST], false, 0 (139) SortMergeJoin [codegen id : 56] -Left keys [2]: [cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint)] -Right keys [2]: [sr_item_sk#143, sr_ticket_number#144] +Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] +Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] Join condition: None (140) Project [codegen id : 56] -Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, sr_item_sk#143, sr_ticket_number#144] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] (141) ReusedExchange [Reuses operator id: 33] -Output [1]: [cs_item_sk#145] +Output [1]: [cs_item_sk#19] (142) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_item_sk#129] -Right keys [1]: [cs_item_sk#145] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [cs_item_sk#19] Join condition: None (143) Project [codegen id : 56] -Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, cs_item_sk#145] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] (144) Scan parquet default.date_dim -Output [2]: [d_date_sk#146, d_year#147] +Output [2]: [d_date_sk#43, d_year#44] 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 (145) ColumnarToRow [codegen id : 54] -Input [2]: [d_date_sk#146, d_year#147] +Input [2]: [d_date_sk#43, d_year#44] (146) Filter [codegen id : 54] -Input [2]: [d_date_sk#146, d_year#147] -Condition : ((isnotnull(d_year#147) AND (d_year#147 = 2000)) AND isnotnull(d_date_sk#146)) +Input [2]: [d_date_sk#43, d_year#44] +Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#43)) (147) BroadcastExchange -Input [2]: [d_date_sk#146, d_year#147] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#148] +Input [2]: [d_date_sk#43, d_year#44] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#131] (148) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_sold_date_sk#140] -Right keys [1]: [d_date_sk#146] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#43] Join condition: None (149) Project [codegen id : 56] -Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147] -Input [13]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, d_date_sk#146, d_year#147] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#43, d_year#44] (150) ReusedExchange [Reuses operator id: 45] -Output [3]: [s_store_sk#149, s_store_name#150, s_zip#151] +Output [3]: [s_store_sk#46, s_store_name#47, s_zip#48] (151) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_store_sk#134] -Right keys [1]: [s_store_sk#149] +Left keys [1]: [ss_store_sk#6] +Right keys [1]: [s_store_sk#46] Join condition: None (152) Project [codegen id : 56] -Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] -Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_sk#149, s_store_name#150, s_zip#151] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_sk#46, s_store_name#47, s_zip#48] (153) Exchange -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] -Arguments: hashpartitioning(ss_customer_sk#130, 5), ENSURE_REQUIREMENTS, [id=#152] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#132] (154) Sort [codegen id : 57] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] -Arguments: [ss_customer_sk#130 ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (155) ReusedExchange [Reuses operator id: 53] -Output [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Output [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] (156) Sort [codegen id : 59] -Input [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] -Arguments: [c_customer_sk#153 ASC NULLS FIRST], false, 0 +Input [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Arguments: [c_customer_sk#51 ASC NULLS FIRST], false, 0 (157) SortMergeJoin [codegen id : 62] -Left keys [1]: [ss_customer_sk#130] -Right keys [1]: [c_customer_sk#153] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#51] Join condition: None (158) Project [codegen id : 62] -Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] -Input [18]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] (159) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#159, d_year#160] +Output [2]: [d_date_sk#133, d_year#134] (160) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_sales_date_sk#158] -Right keys [1]: [d_date_sk#159] +Left keys [1]: [c_first_sales_date_sk#56] +Right keys [1]: [d_date_sk#133] Join condition: None (161) Project [codegen id : 62] -Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160] -Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158, d_date_sk#159, d_year#160] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56, d_date_sk#133, d_year#134] (162) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#161, d_year#162] +Output [2]: [d_date_sk#135, d_year#136] (163) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_shipto_date_sk#157] -Right keys [1]: [d_date_sk#161] +Left keys [1]: [c_first_shipto_date_sk#55] +Right keys [1]: [d_date_sk#135] Join condition: None (164) Project [codegen id : 62] -Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160, d_date_sk#161, d_year#162] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134, d_date_sk#135, d_year#136] (165) Exchange -Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Arguments: hashpartitioning(ss_cdemo_sk#131, 5), ENSURE_REQUIREMENTS, [id=#163] +Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Arguments: hashpartitioning(ss_cdemo_sk#3, 5), ENSURE_REQUIREMENTS, [id=#137] (166) Sort [codegen id : 63] -Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Arguments: [ss_cdemo_sk#131 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Arguments: [ss_cdemo_sk#3 ASC NULLS FIRST], false, 0 (167) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#164, cd_marital_status#165] +Output [2]: [cd_demo_sk#64, cd_marital_status#65] (168) Sort [codegen id : 65] -Input [2]: [cd_demo_sk#164, cd_marital_status#165] -Arguments: [cd_demo_sk#164 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#64, cd_marital_status#65] +Arguments: [cd_demo_sk#64 ASC NULLS FIRST], false, 0 (169) SortMergeJoin [codegen id : 66] -Left keys [1]: [ss_cdemo_sk#131] -Right keys [1]: [cd_demo_sk#164] +Left keys [1]: [ss_cdemo_sk#3] +Right keys [1]: [cd_demo_sk#64] Join condition: None (170) Project [codegen id : 66] -Output [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] -Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_demo_sk#164, cd_marital_status#165] +Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_demo_sk#64, cd_marital_status#65] (171) Exchange -Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] -Arguments: hashpartitioning(c_current_cdemo_sk#154, 5), ENSURE_REQUIREMENTS, [id=#166] +Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] +Arguments: hashpartitioning(c_current_cdemo_sk#52, 5), ENSURE_REQUIREMENTS, [id=#138] (172) Sort [codegen id : 67] -Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] -Arguments: [c_current_cdemo_sk#154 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] +Arguments: [c_current_cdemo_sk#52 ASC NULLS FIRST], false, 0 (173) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#167, cd_marital_status#168] +Output [2]: [cd_demo_sk#139, cd_marital_status#140] (174) Sort [codegen id : 69] -Input [2]: [cd_demo_sk#167, cd_marital_status#168] -Arguments: [cd_demo_sk#167 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#139, cd_marital_status#140] +Arguments: [cd_demo_sk#139 ASC NULLS FIRST], false, 0 (175) SortMergeJoin [codegen id : 73] -Left keys [1]: [c_current_cdemo_sk#154] -Right keys [1]: [cd_demo_sk#167] -Join condition: NOT (cd_marital_status#165 = cd_marital_status#168) +Left keys [1]: [c_current_cdemo_sk#52] +Right keys [1]: [cd_demo_sk#139] +Join condition: NOT (cd_marital_status#65 = cd_marital_status#140) (176) Project [codegen id : 73] -Output [14]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Input [18]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165, cd_demo_sk#167, cd_marital_status#168] +Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65, cd_demo_sk#139, cd_marital_status#140] (177) ReusedExchange [Reuses operator id: 84] -Output [1]: [p_promo_sk#169] +Output [1]: [p_promo_sk#70] (178) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_promo_sk#135] -Right keys [1]: [p_promo_sk#169] +Left keys [1]: [ss_promo_sk#7] +Right keys [1]: [p_promo_sk#70] Join condition: None (179) Project [codegen id : 73] -Output [13]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, p_promo_sk#169] +Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, p_promo_sk#70] (180) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#170, hd_income_band_sk#171] +Output [2]: [hd_demo_sk#72, hd_income_band_sk#73] (181) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_hdemo_sk#132] -Right keys [1]: [hd_demo_sk#170] +Left keys [1]: [ss_hdemo_sk#4] +Right keys [1]: [hd_demo_sk#72] Join condition: None (182) Project [codegen id : 73] -Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171] -Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_demo_sk#170, hd_income_band_sk#171] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_demo_sk#72, hd_income_band_sk#73] (183) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#172, hd_income_band_sk#173] +Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] (184) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [c_current_hdemo_sk#155] -Right keys [1]: [hd_demo_sk#172] +Left keys [1]: [c_current_hdemo_sk#53] +Right keys [1]: [hd_demo_sk#141] Join condition: None (185) Project [codegen id : 73] -Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] -Input [15]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_demo_sk#172, hd_income_band_sk#173] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_demo_sk#141, hd_income_band_sk#142] (186) Exchange -Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] -Arguments: hashpartitioning(ss_addr_sk#133, 5), ENSURE_REQUIREMENTS, [id=#174] +Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] +Arguments: hashpartitioning(ss_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#143] (187) Sort [codegen id : 74] -Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] -Arguments: [ss_addr_sk#133 ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] +Arguments: [ss_addr_sk#5 ASC NULLS FIRST], false, 0 (188) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Output [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] (189) Sort [codegen id : 76] -Input [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -Arguments: [ca_address_sk#175 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Arguments: [ca_address_sk#78 ASC NULLS FIRST], false, 0 (190) SortMergeJoin [codegen id : 77] -Left keys [1]: [ss_addr_sk#133] -Right keys [1]: [ca_address_sk#175] +Left keys [1]: [ss_addr_sk#5] +Right keys [1]: [ca_address_sk#78] Join condition: None (191) Project [codegen id : 77] -Output [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -Input [18]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] (192) Exchange -Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -Arguments: hashpartitioning(c_current_addr_sk#156, 5), ENSURE_REQUIREMENTS, [id=#180] +Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Arguments: hashpartitioning(c_current_addr_sk#54, 5), ENSURE_REQUIREMENTS, [id=#144] (193) Sort [codegen id : 78] -Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -Arguments: [c_current_addr_sk#156 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Arguments: [c_current_addr_sk#54 ASC NULLS FIRST], false, 0 (194) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Output [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] (195) Sort [codegen id : 80] -Input [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] -Arguments: [ca_address_sk#181 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Arguments: [ca_address_sk#145 ASC NULLS FIRST], false, 0 (196) SortMergeJoin [codegen id : 84] -Left keys [1]: [c_current_addr_sk#156] -Right keys [1]: [ca_address_sk#181] +Left keys [1]: [c_current_addr_sk#54] +Right keys [1]: [ca_address_sk#145] Join condition: None (197) Project [codegen id : 84] -Output [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] -Input [21]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] (198) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#186] +Output [1]: [ib_income_band_sk#90] (199) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#171] -Right keys [1]: [ib_income_band_sk#186] +Left keys [1]: [hd_income_band_sk#73] +Right keys [1]: [ib_income_band_sk#90] Join condition: None (200) Project [codegen id : 84] -Output [18]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] -Input [20]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#186] +Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#90] (201) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#187] +Output [1]: [ib_income_band_sk#150] (202) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#173] -Right keys [1]: [ib_income_band_sk#187] +Left keys [1]: [hd_income_band_sk#142] +Right keys [1]: [ib_income_band_sk#150] Join condition: None (203) Project [codegen id : 84] -Output [17]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] -Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#187] +Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#150] (204) ReusedExchange [Reuses operator id: 124] -Output [2]: [i_item_sk#188, i_product_name#189] +Output [2]: [i_item_sk#93, i_product_name#96] (205) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#129] -Right keys [1]: [i_item_sk#188] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#93] Join condition: None (206) Project [codegen id : 84] -Output [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] -Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] +Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] (207) HashAggregate [codegen id : 84] -Input [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] -Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#137)), partial_sum(UnscaledValue(ss_list_price#138)), partial_sum(UnscaledValue(ss_coupon_amt#139))] -Aggregate Attributes [4]: [count#190, sum#191, sum#192, sum#193] -Results [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] +Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count#151, sum#152, sum#153, sum#154] +Results [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] (208) Exchange -Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] -Arguments: hashpartitioning(i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, 5), ENSURE_REQUIREMENTS, [id=#198] +Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] +Arguments: hashpartitioning(i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, 5), ENSURE_REQUIREMENTS, [id=#159] (209) HashAggregate [codegen id : 85] -Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] -Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#137)), sum(UnscaledValue(ss_list_price#138)), sum(UnscaledValue(ss_coupon_amt#139))] -Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#137))#200, sum(UnscaledValue(ss_list_price#138))#201, sum(UnscaledValue(ss_coupon_amt#139))#202] -Results [8]: [i_item_sk#188 AS item_sk#203, s_store_name#150 AS store_name#204, s_zip#151 AS store_zip#205, d_year#147 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#137))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#138))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#139))#202,17,2) AS s3#210] +Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] +Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count(1)#160, sum(UnscaledValue(ss_wholesale_cost#9))#161, sum(UnscaledValue(ss_list_price#10))#162, sum(UnscaledValue(ss_coupon_amt#11))#163] +Results [8]: [i_item_sk#93 AS item_sk#164, s_store_name#47 AS store_name#165, s_zip#48 AS store_zip#166, d_year#44 AS syear#167, count(1)#160 AS cnt#168, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#161,17,2) AS s1#169, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#162,17,2) AS s2#170, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#163,17,2) AS s3#171] (210) Exchange -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] +Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] +Arguments: hashpartitioning(item_sk#164, store_name#165, store_zip#166, 5), ENSURE_REQUIREMENTS, [id=#172] (211) Sort [codegen id : 86] -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] +Arguments: [item_sk#164 ASC NULLS FIRST, store_name#165 ASC NULLS FIRST, store_zip#166 ASC NULLS FIRST], false, 0 (212) SortMergeJoin [codegen id : 87] Left keys [3]: [item_sk#112, store_name#113, store_zip#114] -Right keys [3]: [item_sk#203, store_name#204, store_zip#205] -Join condition: (cnt#207 <= cnt#124) +Right keys [3]: [item_sk#164, store_name#165, store_zip#166] +Join condition: (cnt#168 <= cnt#124) (213) Project [codegen id : 87] -Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] -Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] +Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] (214) Exchange -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] +Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#173] (215) Sort [codegen id : 88] -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] +Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1152,11 +1152,11 @@ ReusedExchange (216) (216) ReusedExchange [Reuses operator id: 39] Output [2]: [d_date_sk#43, d_year#44] -Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#140 IN dynamicpruning#141 +Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#129 ReusedExchange (217) (217) ReusedExchange [Reuses operator id: 147] -Output [2]: [d_date_sk#146, d_year#147] +Output [2]: [d_date_sk#43, d_year#44] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index fe9a3ca6d8cb2..9cc3415e6e146 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt @@ -701,323 +701,323 @@ Input [17]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_str Arguments: [item_sk#105 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, store_zip#107 ASC NULLS FIRST], false, 0 (115) Scan parquet default.store_sales -Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#133), dynamicpruningexpression(ss_sold_date_sk#133 IN dynamicpruning#134)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#122)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (116) ColumnarToRow [codegen id : 27] -Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] (117) Filter [codegen id : 27] -Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Condition : (((((((isnotnull(ss_item_sk#122) AND isnotnull(ss_ticket_number#129)) AND isnotnull(ss_store_sk#127)) AND isnotnull(ss_customer_sk#123)) AND isnotnull(ss_cdemo_sk#124)) AND isnotnull(ss_promo_sk#128)) AND isnotnull(ss_hdemo_sk#125)) AND isnotnull(ss_addr_sk#126)) +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) (118) BroadcastExchange -Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#135] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#123] (119) Scan parquet default.store_returns -Output [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Output [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct (120) ColumnarToRow -Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] (121) Filter -Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] -Condition : (isnotnull(sr_item_sk#136) AND isnotnull(sr_ticket_number#137)) +Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Condition : (isnotnull(sr_item_sk#15) AND isnotnull(sr_ticket_number#16)) (122) Project -Output [2]: [sr_item_sk#136, sr_ticket_number#137] -Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Output [2]: [sr_item_sk#15, sr_ticket_number#16] +Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] (123) BroadcastHashJoin [codegen id : 28] -Left keys [2]: [cast(ss_item_sk#122 as bigint), cast(ss_ticket_number#129 as bigint)] -Right keys [2]: [sr_item_sk#136, sr_ticket_number#137] +Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] +Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] Join condition: None (124) Project [codegen id : 28] -Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, sr_item_sk#136, sr_ticket_number#137] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] (125) Exchange -Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Arguments: hashpartitioning(ss_item_sk#122, 5), ENSURE_REQUIREMENTS, [id=#139] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#124] (126) Sort [codegen id : 29] -Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Arguments: [ss_item_sk#122 ASC NULLS FIRST], false, 0 +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (127) ReusedExchange [Reuses operator id: 28] -Output [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] +Output [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] (128) HashAggregate [codegen id : 35] -Input [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] -Keys [1]: [cs_item_sk#140] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#144)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#144))#148, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149] -Results [3]: [cs_item_sk#140, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#144))#148,17,2) AS sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] +Input [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] +Keys [1]: [cs_item_sk#19] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#21)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#21))#128, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129] +Results [3]: [cs_item_sk#19, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#21))#128,17,2) AS sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] (129) Filter [codegen id : 35] -Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] -Condition : (isnotnull(sum(cs_ext_list_price#144)#150) AND (cast(sum(cs_ext_list_price#144)#150 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151)), DecimalType(21,2), true))) +Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] +Condition : (isnotnull(sum(cs_ext_list_price#21)#130) AND (cast(sum(cs_ext_list_price#21)#130 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131)), DecimalType(21,2), true))) (130) Project [codegen id : 35] -Output [1]: [cs_item_sk#140] -Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] +Output [1]: [cs_item_sk#19] +Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] (131) Sort [codegen id : 35] -Input [1]: [cs_item_sk#140] -Arguments: [cs_item_sk#140 ASC NULLS FIRST], false, 0 +Input [1]: [cs_item_sk#19] +Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 (132) SortMergeJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#122] -Right keys [1]: [cs_item_sk#140] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [cs_item_sk#19] Join condition: None (133) Project [codegen id : 51] -Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, cs_item_sk#140] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] (134) Scan parquet default.date_dim -Output [2]: [d_date_sk#152, d_year#153] +Output [2]: [d_date_sk#42, d_year#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (135) ColumnarToRow [codegen id : 36] -Input [2]: [d_date_sk#152, d_year#153] +Input [2]: [d_date_sk#42, d_year#43] (136) Filter [codegen id : 36] -Input [2]: [d_date_sk#152, d_year#153] -Condition : ((isnotnull(d_year#153) AND (d_year#153 = 2000)) AND isnotnull(d_date_sk#152)) +Input [2]: [d_date_sk#42, d_year#43] +Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2000)) AND isnotnull(d_date_sk#42)) (137) BroadcastExchange -Input [2]: [d_date_sk#152, d_year#153] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#154] +Input [2]: [d_date_sk#42, d_year#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#132] (138) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_sold_date_sk#133] -Right keys [1]: [d_date_sk#152] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#42] Join condition: None (139) Project [codegen id : 51] -Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153] -Input [13]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, d_date_sk#152, d_year#153] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#42, d_year#43] (140) ReusedExchange [Reuses operator id: 44] -Output [3]: [s_store_sk#155, s_store_name#156, s_zip#157] +Output [3]: [s_store_sk#45, s_store_name#46, s_zip#47] (141) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_store_sk#127] -Right keys [1]: [s_store_sk#155] +Left keys [1]: [ss_store_sk#6] +Right keys [1]: [s_store_sk#45] Join condition: None (142) Project [codegen id : 51] -Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157] -Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_sk#155, s_store_name#156, s_zip#157] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_sk#45, s_store_name#46, s_zip#47] (143) ReusedExchange [Reuses operator id: 50] -Output [6]: [c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] +Output [6]: [c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] (144) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_customer_sk#123] -Right keys [1]: [c_customer_sk#158] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#49] Join condition: None (145) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] -Input [18]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] (146) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#164, d_year#165] +Output [2]: [d_date_sk#133, d_year#134] (147) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_sales_date_sk#163] -Right keys [1]: [d_date_sk#164] +Left keys [1]: [c_first_sales_date_sk#54] +Right keys [1]: [d_date_sk#133] Join condition: None (148) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165] -Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163, d_date_sk#164, d_year#165] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54, d_date_sk#133, d_year#134] (149) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#166, d_year#167] +Output [2]: [d_date_sk#135, d_year#136] (150) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_shipto_date_sk#162] -Right keys [1]: [d_date_sk#166] +Left keys [1]: [c_first_shipto_date_sk#53] +Right keys [1]: [d_date_sk#135] Join condition: None (151) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] -Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165, d_date_sk#166, d_year#167] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134, d_date_sk#135, d_year#136] (152) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#168, cd_marital_status#169] +Output [2]: [cd_demo_sk#61, cd_marital_status#62] (153) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_cdemo_sk#124] -Right keys [1]: [cd_demo_sk#168] +Left keys [1]: [ss_cdemo_sk#3] +Right keys [1]: [cd_demo_sk#61] Join condition: None (154) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169] -Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_demo_sk#168, cd_marital_status#169] +Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_demo_sk#61, cd_marital_status#62] (155) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#170, cd_marital_status#171] +Output [2]: [cd_demo_sk#137, cd_marital_status#138] (156) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_cdemo_sk#159] -Right keys [1]: [cd_demo_sk#170] -Join condition: NOT (cd_marital_status#169 = cd_marital_status#171) +Left keys [1]: [c_current_cdemo_sk#50] +Right keys [1]: [cd_demo_sk#137] +Join condition: NOT (cd_marital_status#62 = cd_marital_status#138) (157) Project [codegen id : 51] -Output [14]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] -Input [18]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169, cd_demo_sk#170, cd_marital_status#171] +Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62, cd_demo_sk#137, cd_marital_status#138] (158) ReusedExchange [Reuses operator id: 74] -Output [1]: [p_promo_sk#172] +Output [1]: [p_promo_sk#66] (159) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_promo_sk#128] -Right keys [1]: [p_promo_sk#172] +Left keys [1]: [ss_promo_sk#7] +Right keys [1]: [p_promo_sk#66] Join condition: None (160) Project [codegen id : 51] -Output [13]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] -Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, p_promo_sk#172] +Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, p_promo_sk#66] (161) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#173, hd_income_band_sk#174] +Output [2]: [hd_demo_sk#68, hd_income_band_sk#69] (162) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_hdemo_sk#125] -Right keys [1]: [hd_demo_sk#173] +Left keys [1]: [ss_hdemo_sk#4] +Right keys [1]: [hd_demo_sk#68] Join condition: None (163) Project [codegen id : 51] -Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174] -Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_demo_sk#173, hd_income_band_sk#174] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_demo_sk#68, hd_income_band_sk#69] (164) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#175, hd_income_band_sk#176] +Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] (165) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_hdemo_sk#160] -Right keys [1]: [hd_demo_sk#175] +Left keys [1]: [c_current_hdemo_sk#51] +Right keys [1]: [hd_demo_sk#139] Join condition: None (166) Project [codegen id : 51] -Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176] -Input [15]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_demo_sk#175, hd_income_band_sk#176] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140] +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_demo_sk#139, hd_income_band_sk#140] (167) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] +Output [5]: [ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] (168) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_addr_sk#126] -Right keys [1]: [ca_address_sk#177] +Left keys [1]: [ss_addr_sk#5] +Right keys [1]: [ca_address_sk#73] Join condition: None (169) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] -Input [18]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] +Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] (170) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] (171) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_addr_sk#161] -Right keys [1]: [ca_address_sk#182] +Left keys [1]: [c_current_addr_sk#52] +Right keys [1]: [ca_address_sk#141] Join condition: None (172) Project [codegen id : 51] -Output [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] -Input [21]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] (173) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#187] +Output [1]: [ib_income_band_sk#84] (174) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#174] -Right keys [1]: [ib_income_band_sk#187] +Left keys [1]: [hd_income_band_sk#69] +Right keys [1]: [ib_income_band_sk#84] Join condition: None (175) Project [codegen id : 51] -Output [18]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] -Input [20]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#187] +Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#84] (176) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#188] +Output [1]: [ib_income_band_sk#146] (177) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#176] -Right keys [1]: [ib_income_band_sk#188] +Left keys [1]: [hd_income_band_sk#140] +Right keys [1]: [ib_income_band_sk#146] Join condition: None (178) Project [codegen id : 51] -Output [17]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] -Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#188] +Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#146] (179) ReusedExchange [Reuses operator id: 108] -Output [2]: [i_item_sk#189, i_product_name#190] +Output [2]: [i_item_sk#87, i_product_name#90] (180) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#122] -Right keys [1]: [i_item_sk#189] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#87] Join condition: None (181) Project [codegen id : 51] -Output [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] -Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] (182) HashAggregate [codegen id : 51] -Input [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] -Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#130)), partial_sum(UnscaledValue(ss_list_price#131)), partial_sum(UnscaledValue(ss_coupon_amt#132))] -Aggregate Attributes [4]: [count#191, sum#192, sum#193, sum#194] -Results [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] +Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count#147, sum#148, sum#149, sum#150] +Results [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] (183) HashAggregate [codegen id : 51] -Input [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] -Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#130)), sum(UnscaledValue(ss_list_price#131)), sum(UnscaledValue(ss_coupon_amt#132))] -Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#130))#200, sum(UnscaledValue(ss_list_price#131))#201, sum(UnscaledValue(ss_coupon_amt#132))#202] -Results [8]: [i_item_sk#189 AS item_sk#203, s_store_name#156 AS store_name#204, s_zip#157 AS store_zip#205, d_year#153 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#130))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#131))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#132))#202,17,2) AS s3#210] +Input [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] +Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count(1)#155, sum(UnscaledValue(ss_wholesale_cost#9))#156, sum(UnscaledValue(ss_list_price#10))#157, sum(UnscaledValue(ss_coupon_amt#11))#158] +Results [8]: [i_item_sk#87 AS item_sk#159, s_store_name#46 AS store_name#160, s_zip#47 AS store_zip#161, d_year#43 AS syear#162, count(1)#155 AS cnt#163, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#156,17,2) AS s1#164, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#157,17,2) AS s2#165, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#158,17,2) AS s3#166] (184) Exchange -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] +Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] +Arguments: hashpartitioning(item_sk#159, store_name#160, store_zip#161, 5), ENSURE_REQUIREMENTS, [id=#167] (185) Sort [codegen id : 52] -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] +Arguments: [item_sk#159 ASC NULLS FIRST, store_name#160 ASC NULLS FIRST, store_zip#161 ASC NULLS FIRST], false, 0 (186) SortMergeJoin [codegen id : 53] Left keys [3]: [item_sk#105, store_name#106, store_zip#107] -Right keys [3]: [item_sk#203, store_name#204, store_zip#205] -Join condition: (cnt#207 <= cnt#117) +Right keys [3]: [item_sk#159, store_name#160, store_zip#161] +Join condition: (cnt#163 <= cnt#117) (187) Project [codegen id : 53] -Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] -Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] +Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] (188) Exchange -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] +Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#168] (189) Sort [codegen id : 54] -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] +Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1028,11 +1028,11 @@ ReusedExchange (190) (190) ReusedExchange [Reuses operator id: 38] Output [2]: [d_date_sk#42, d_year#43] -Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#133 IN dynamicpruning#134 +Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#122 ReusedExchange (191) (191) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#152, d_year#153] +Output [2]: [d_date_sk#42, d_year#43] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt index 84e49ab9373e4..41c2ffcc75e7a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt @@ -130,135 +130,135 @@ Input [4]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17 Condition : isnotnull(ss_store_sk#15) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#6] (19) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#17] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#6] Join condition: None (20) Project [codegen id : 4] Output [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] -Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#18] +Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#6] (21) HashAggregate [codegen id : 4] Input [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] +Aggregate Attributes [1]: [sum#18] +Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] (22) Exchange -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] -Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] +Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#20] (23) HashAggregate [codegen id : 5] -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#22] -Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#22,17,2) AS revenue#23] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#21] +Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#21,17,2) AS revenue#22] (24) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#15, revenue#23] +Input [2]: [ss_store_sk#15, revenue#22] Keys [1]: [ss_store_sk#15] -Functions [1]: [partial_avg(revenue#23)] -Aggregate Attributes [2]: [sum#24, count#25] -Results [3]: [ss_store_sk#15, sum#26, count#27] +Functions [1]: [partial_avg(revenue#22)] +Aggregate Attributes [2]: [sum#23, count#24] +Results [3]: [ss_store_sk#15, sum#25, count#26] (25) Exchange -Input [3]: [ss_store_sk#15, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [ss_store_sk#15, sum#25, count#26] +Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#27] (26) HashAggregate [codegen id : 6] -Input [3]: [ss_store_sk#15, sum#26, count#27] +Input [3]: [ss_store_sk#15, sum#25, count#26] Keys [1]: [ss_store_sk#15] -Functions [1]: [avg(revenue#23)] -Aggregate Attributes [1]: [avg(revenue#23)#29] -Results [2]: [ss_store_sk#15, avg(revenue#23)#29 AS ave#30] +Functions [1]: [avg(revenue#22)] +Aggregate Attributes [1]: [avg(revenue#22)#28] +Results [2]: [ss_store_sk#15, avg(revenue#22)#28 AS ave#29] (27) BroadcastExchange -Input [2]: [ss_store_sk#15, ave#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [2]: [ss_store_sk#15, ave#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] (28) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] Right keys [1]: [ss_store_sk#15] -Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#30)), DecimalType(23,7), true)) +Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#29)), DecimalType(23,7), true)) (29) Project [codegen id : 8] Output [3]: [ss_store_sk#2, ss_item_sk#1, revenue#13] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#30] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#29] (30) Scan parquet default.store -Output [2]: [s_store_sk#32, s_store_name#33] +Output [2]: [s_store_sk#31, s_store_name#32] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#32, s_store_name#33] +Input [2]: [s_store_sk#31, s_store_name#32] (32) Filter [codegen id : 7] -Input [2]: [s_store_sk#32, s_store_name#33] -Condition : isnotnull(s_store_sk#32) +Input [2]: [s_store_sk#31, s_store_name#32] +Condition : isnotnull(s_store_sk#31) (33) BroadcastExchange -Input [2]: [s_store_sk#32, s_store_name#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [2]: [s_store_sk#31, s_store_name#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] (34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#32] +Right keys [1]: [s_store_sk#31] Join condition: None (35) Project [codegen id : 8] -Output [3]: [ss_item_sk#1, revenue#13, s_store_name#33] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#32, s_store_name#33] +Output [3]: [ss_item_sk#1, revenue#13, s_store_name#32] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#31, s_store_name#32] (36) Exchange -Input [3]: [ss_item_sk#1, revenue#13, s_store_name#33] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [ss_item_sk#1, revenue#13, s_store_name#32] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#34] (37) Sort [codegen id : 9] -Input [3]: [ss_item_sk#1, revenue#13, s_store_name#33] +Input [3]: [ss_item_sk#1, revenue#13, s_store_name#32] Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (38) Scan parquet default.item -Output [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Output [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 10] -Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] (40) Filter [codegen id : 10] -Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] -Condition : isnotnull(i_item_sk#36) +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Condition : isnotnull(i_item_sk#35) (41) Exchange -Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] -Arguments: hashpartitioning(i_item_sk#36, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Arguments: hashpartitioning(i_item_sk#35, 5), ENSURE_REQUIREMENTS, [id=#40] (42) Sort [codegen id : 11] -Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] -Arguments: [i_item_sk#36 ASC NULLS FIRST], false, 0 +Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Arguments: [i_item_sk#35 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 12] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#36] +Right keys [1]: [i_item_sk#35] Join condition: None (44) Project [codegen id : 12] -Output [6]: [s_store_name#33, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] -Input [8]: [ss_item_sk#1, revenue#13, s_store_name#33, i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Output [6]: [s_store_name#32, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [8]: [ss_item_sk#1, revenue#13, s_store_name#32, i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] (45) TakeOrderedAndProject -Input [6]: [s_store_name#33, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] -Arguments: 100, [s_store_name#33 ASC NULLS FIRST, i_item_desc#37 ASC NULLS FIRST], [s_store_name#33, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Input [6]: [s_store_name#32, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Arguments: 100, [s_store_name#32 ASC NULLS FIRST, i_item_desc#36 ASC NULLS FIRST], [s_store_name#32, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt index 45c7c051601c5..52de9873db590 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt @@ -181,65 +181,65 @@ Input [4]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26 Condition : isnotnull(ss_store_sk#24) (30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#8] (31) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#8] Join condition: None (32) Project [codegen id : 6] Output [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] -Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#27] +Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#8] (33) HashAggregate [codegen id : 6] Input [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#28] -Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] +Aggregate Attributes [1]: [sum#27] +Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] (34) Exchange -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] -Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] +Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#29] (35) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] -Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS revenue#32] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#30] +Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#30,17,2) AS revenue#31] (36) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#24, revenue#32] +Input [2]: [ss_store_sk#24, revenue#31] Keys [1]: [ss_store_sk#24] -Functions [1]: [partial_avg(revenue#32)] -Aggregate Attributes [2]: [sum#33, count#34] -Results [3]: [ss_store_sk#24, sum#35, count#36] +Functions [1]: [partial_avg(revenue#31)] +Aggregate Attributes [2]: [sum#32, count#33] +Results [3]: [ss_store_sk#24, sum#34, count#35] (37) Exchange -Input [3]: [ss_store_sk#24, sum#35, count#36] -Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [ss_store_sk#24, sum#34, count#35] +Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#36] (38) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#24, sum#35, count#36] +Input [3]: [ss_store_sk#24, sum#34, count#35] Keys [1]: [ss_store_sk#24] -Functions [1]: [avg(revenue#32)] -Aggregate Attributes [1]: [avg(revenue#32)#38] -Results [2]: [ss_store_sk#24, avg(revenue#32)#38 AS ave#39] +Functions [1]: [avg(revenue#31)] +Aggregate Attributes [1]: [avg(revenue#31)#37] +Results [2]: [ss_store_sk#24, avg(revenue#31)#37 AS ave#38] (39) BroadcastExchange -Input [2]: [ss_store_sk#24, ave#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] +Input [2]: [ss_store_sk#24, ave#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#24] -Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#39)), DecimalType(23,7), true)) +Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#38)), DecimalType(23,7), true)) (41) Project [codegen id : 9] Output [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#39] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#38] (42) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt index 8c342961cf970..51298b80bbbbe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt @@ -221,70 +221,70 @@ Input [7]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_qu Condition : ((isnotnull(cs_warehouse_sk#176) AND isnotnull(cs_sold_time_sk#174)) AND isnotnull(cs_ship_mode_sk#175)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [sm_ship_mode_sk#181] +Output [1]: [sm_ship_mode_sk#9] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_mode_sk#175] -Right keys [1]: [sm_ship_mode_sk#181] +Right keys [1]: [sm_ship_mode_sk#9] Join condition: None (38) Project [codegen id : 11] Output [6]: [cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180] -Input [8]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, sm_ship_mode_sk#181] +Input [8]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, sm_ship_mode_sk#9] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [t_time_sk#182] +Output [1]: [t_time_sk#12] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_time_sk#174] -Right keys [1]: [t_time_sk#182] +Right keys [1]: [t_time_sk#12] Join condition: None (41) Project [codegen id : 11] Output [5]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180] -Input [7]: [cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, t_time_sk#182] +Input [7]: [cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, t_time_sk#12] (42) ReusedExchange [Reuses operator id: 21] -Output [3]: [d_date_sk#183, d_year#184, d_moy#185] +Output [3]: [d_date_sk#15, d_year#16, d_moy#17] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#180] -Right keys [1]: [d_date_sk#183] +Right keys [1]: [d_date_sk#15] Join condition: None (44) Project [codegen id : 11] -Output [6]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#184, d_moy#185] -Input [8]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, d_date_sk#183, d_year#184, d_moy#185] +Output [6]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#16, d_moy#17] +Input [8]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, d_date_sk#15, d_year#16, d_moy#17] (45) ReusedExchange [Reuses operator id: 27] -Output [7]: [w_warehouse_sk#186, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192] +Output [7]: [w_warehouse_sk#19, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25] (46) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_warehouse_sk#176] -Right keys [1]: [w_warehouse_sk#186] +Right keys [1]: [w_warehouse_sk#19] Join condition: None (47) Project [codegen id : 11] -Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, d_moy#185] -Input [13]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#184, d_moy#185, w_warehouse_sk#186, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192] +Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, d_moy#17] +Input [13]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#16, d_moy#17, w_warehouse_sk#19, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25] (48) HashAggregate [codegen id : 11] -Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, d_moy#185] -Keys [7]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184] -Functions [24]: [partial_sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240] -Results [55]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] +Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, d_moy#17] +Keys [7]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16] +Functions [24]: [partial_sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [48]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228] +Results [55]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] (49) Exchange -Input [55]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] -Arguments: hashpartitioning(w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, 5), ENSURE_REQUIREMENTS, [id=#289] +Input [55]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] +Arguments: hashpartitioning(w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#277] (50) HashAggregate [codegen id : 12] -Input [55]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] -Keys [7]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184] -Functions [24]: [sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313] -Results [32]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, DHL,BARIAN AS ship_carriers#314, d_year#184 AS year#315, sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_sales#316, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_sales#317, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_sales#318, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_sales#319, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_sales#320, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_sales#321, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_sales#322, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_sales#323, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_sales#324, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_sales#325, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_sales#326, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_sales#327, sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302 AS jan_net#328, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303 AS feb_net#329, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304 AS mar_net#330, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305 AS apr_net#331, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306 AS may_net#332, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307 AS jun_net#333, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308 AS jul_net#334, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309 AS aug_net#335, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310 AS sep_net#336, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311 AS oct_net#337, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312 AS nov_net#338, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313 AS dec_net#339] +Input [55]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] +Keys [7]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16] +Functions [24]: [sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301] +Results [32]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, DHL,BARIAN AS ship_carriers#302, d_year#16 AS year#303, sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278 AS jan_sales#304, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279 AS feb_sales#305, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280 AS mar_sales#306, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281 AS apr_sales#307, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282 AS may_sales#308, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283 AS jun_sales#309, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284 AS jul_sales#310, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285 AS aug_sales#311, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286 AS sep_sales#312, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287 AS oct_sales#313, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288 AS nov_sales#314, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289 AS dec_sales#315, sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_net#316, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_net#317, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_net#318, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_net#319, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_net#320, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_net#321, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_net#322, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_net#323, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_net#324, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_net#325, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_net#326, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_net#327] (51) Union @@ -292,23 +292,23 @@ Results [32]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county Input [32]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#150, feb_sales#151, mar_sales#152, apr_sales#153, may_sales#154, jun_sales#155, jul_sales#156, aug_sales#157, sep_sales#158, oct_sales#159, nov_sales#160, dec_sales#161, jan_net#162, feb_net#163, mar_net#164, apr_net#165, may_net#166, jun_net#167, jul_net#168, aug_net#169, sep_net#170, oct_net#171, nov_net#172, dec_net#173] Keys [8]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149] Functions [36]: [partial_sum(jan_sales#150), partial_sum(feb_sales#151), partial_sum(mar_sales#152), partial_sum(apr_sales#153), partial_sum(may_sales#154), partial_sum(jun_sales#155), partial_sum(jul_sales#156), partial_sum(aug_sales#157), partial_sum(sep_sales#158), partial_sum(oct_sales#159), partial_sum(nov_sales#160), partial_sum(dec_sales#161), partial_sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(jan_net#162), partial_sum(feb_net#163), partial_sum(mar_net#164), partial_sum(apr_net#165), partial_sum(may_net#166), partial_sum(jun_net#167), partial_sum(jul_net#168), partial_sum(aug_net#169), partial_sum(sep_net#170), partial_sum(oct_net#171), partial_sum(nov_net#172), partial_sum(dec_net#173)] -Aggregate Attributes [72]: [sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411] -Results [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] +Aggregate Attributes [72]: [sum#328, isEmpty#329, sum#330, isEmpty#331, sum#332, isEmpty#333, sum#334, isEmpty#335, sum#336, isEmpty#337, sum#338, isEmpty#339, sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399] +Results [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] (53) Exchange -Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] -Arguments: hashpartitioning(w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#484] +Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] +Arguments: hashpartitioning(w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#472] (54) HashAggregate [codegen id : 14] -Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] +Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] Keys [8]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149] Functions [36]: [sum(jan_sales#150), sum(feb_sales#151), sum(mar_sales#152), sum(apr_sales#153), sum(may_sales#154), sum(jun_sales#155), sum(jul_sales#156), sum(aug_sales#157), sum(sep_sales#158), sum(oct_sales#159), sum(nov_sales#160), sum(dec_sales#161), sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(jan_net#162), sum(feb_net#163), sum(mar_net#164), sum(apr_net#165), sum(may_net#166), sum(jun_net#167), sum(jul_net#168), sum(aug_net#169), sum(sep_net#170), sum(oct_net#171), sum(nov_net#172), sum(dec_net#173)] -Aggregate Attributes [36]: [sum(jan_sales#150)#485, sum(feb_sales#151)#486, sum(mar_sales#152)#487, sum(apr_sales#153)#488, sum(may_sales#154)#489, sum(jun_sales#155)#490, sum(jul_sales#156)#491, sum(aug_sales#157)#492, sum(sep_sales#158)#493, sum(oct_sales#159)#494, sum(nov_sales#160)#495, sum(dec_sales#161)#496, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508, sum(jan_net#162)#509, sum(feb_net#163)#510, sum(mar_net#164)#511, sum(apr_net#165)#512, sum(may_net#166)#513, sum(jun_net#167)#514, sum(jul_net#168)#515, sum(aug_net#169)#516, sum(sep_net#170)#517, sum(oct_net#171)#518, sum(nov_net#172)#519, sum(dec_net#173)#520] -Results [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum(jan_sales#150)#485 AS jan_sales#521, sum(feb_sales#151)#486 AS feb_sales#522, sum(mar_sales#152)#487 AS mar_sales#523, sum(apr_sales#153)#488 AS apr_sales#524, sum(may_sales#154)#489 AS may_sales#525, sum(jun_sales#155)#490 AS jun_sales#526, sum(jul_sales#156)#491 AS jul_sales#527, sum(aug_sales#157)#492 AS aug_sales#528, sum(sep_sales#158)#493 AS sep_sales#529, sum(oct_sales#159)#494 AS oct_sales#530, sum(nov_sales#160)#495 AS nov_sales#531, sum(dec_sales#161)#496 AS dec_sales#532, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497 AS jan_sales_per_sq_foot#533, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498 AS feb_sales_per_sq_foot#534, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499 AS mar_sales_per_sq_foot#535, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500 AS apr_sales_per_sq_foot#536, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501 AS may_sales_per_sq_foot#537, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502 AS jun_sales_per_sq_foot#538, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503 AS jul_sales_per_sq_foot#539, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504 AS aug_sales_per_sq_foot#540, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505 AS sep_sales_per_sq_foot#541, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506 AS oct_sales_per_sq_foot#542, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507 AS nov_sales_per_sq_foot#543, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508 AS dec_sales_per_sq_foot#544, sum(jan_net#162)#509 AS jan_net#545, sum(feb_net#163)#510 AS feb_net#546, sum(mar_net#164)#511 AS mar_net#547, sum(apr_net#165)#512 AS apr_net#548, sum(may_net#166)#513 AS may_net#549, sum(jun_net#167)#514 AS jun_net#550, sum(jul_net#168)#515 AS jul_net#551, sum(aug_net#169)#516 AS aug_net#552, sum(sep_net#170)#517 AS sep_net#553, sum(oct_net#171)#518 AS oct_net#554, sum(nov_net#172)#519 AS nov_net#555, sum(dec_net#173)#520 AS dec_net#556] +Aggregate Attributes [36]: [sum(jan_sales#150)#473, sum(feb_sales#151)#474, sum(mar_sales#152)#475, sum(apr_sales#153)#476, sum(may_sales#154)#477, sum(jun_sales#155)#478, sum(jul_sales#156)#479, sum(aug_sales#157)#480, sum(sep_sales#158)#481, sum(oct_sales#159)#482, sum(nov_sales#160)#483, sum(dec_sales#161)#484, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496, sum(jan_net#162)#497, sum(feb_net#163)#498, sum(mar_net#164)#499, sum(apr_net#165)#500, sum(may_net#166)#501, sum(jun_net#167)#502, sum(jul_net#168)#503, sum(aug_net#169)#504, sum(sep_net#170)#505, sum(oct_net#171)#506, sum(nov_net#172)#507, sum(dec_net#173)#508] +Results [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum(jan_sales#150)#473 AS jan_sales#509, sum(feb_sales#151)#474 AS feb_sales#510, sum(mar_sales#152)#475 AS mar_sales#511, sum(apr_sales#153)#476 AS apr_sales#512, sum(may_sales#154)#477 AS may_sales#513, sum(jun_sales#155)#478 AS jun_sales#514, sum(jul_sales#156)#479 AS jul_sales#515, sum(aug_sales#157)#480 AS aug_sales#516, sum(sep_sales#158)#481 AS sep_sales#517, sum(oct_sales#159)#482 AS oct_sales#518, sum(nov_sales#160)#483 AS nov_sales#519, sum(dec_sales#161)#484 AS dec_sales#520, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485 AS jan_sales_per_sq_foot#521, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486 AS feb_sales_per_sq_foot#522, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487 AS mar_sales_per_sq_foot#523, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488 AS apr_sales_per_sq_foot#524, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489 AS may_sales_per_sq_foot#525, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490 AS jun_sales_per_sq_foot#526, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491 AS jul_sales_per_sq_foot#527, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492 AS aug_sales_per_sq_foot#528, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493 AS sep_sales_per_sq_foot#529, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494 AS oct_sales_per_sq_foot#530, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495 AS nov_sales_per_sq_foot#531, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496 AS dec_sales_per_sq_foot#532, sum(jan_net#162)#497 AS jan_net#533, sum(feb_net#163)#498 AS feb_net#534, sum(mar_net#164)#499 AS mar_net#535, sum(apr_net#165)#500 AS apr_net#536, sum(may_net#166)#501 AS may_net#537, sum(jun_net#167)#502 AS jun_net#538, sum(jul_net#168)#503 AS jul_net#539, sum(aug_net#169)#504 AS aug_net#540, sum(sep_net#170)#505 AS sep_net#541, sum(oct_net#171)#506 AS oct_net#542, sum(nov_net#172)#507 AS nov_net#543, sum(dec_net#173)#508 AS dec_net#544] (55) TakeOrderedAndProject -Input [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, may_sales_per_sq_foot#537, jun_sales_per_sq_foot#538, jul_sales_per_sq_foot#539, aug_sales_per_sq_foot#540, sep_sales_per_sq_foot#541, oct_sales_per_sq_foot#542, nov_sales_per_sq_foot#543, dec_sales_per_sq_foot#544, jan_net#545, feb_net#546, mar_net#547, apr_net#548, may_net#549, jun_net#550, jul_net#551, aug_net#552, sep_net#553, oct_net#554, nov_net#555, dec_net#556] -Arguments: 100, [w_warehouse_name#20 ASC NULLS FIRST], [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, ... 20 more fields] +Input [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, may_sales_per_sq_foot#525, jun_sales_per_sq_foot#526, jul_sales_per_sq_foot#527, aug_sales_per_sq_foot#528, sep_sales_per_sq_foot#529, oct_sales_per_sq_foot#530, nov_sales_per_sq_foot#531, dec_sales_per_sq_foot#532, jan_net#533, feb_net#534, mar_net#535, apr_net#536, may_net#537, jun_net#538, jul_net#539, aug_net#540, sep_net#541, oct_net#542, nov_net#543, dec_net#544] +Arguments: 100, [w_warehouse_name#20 ASC NULLS FIRST], [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, ... 20 more fields] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt index 832965c1aaa31..3d44b22396486 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt @@ -221,70 +221,70 @@ Input [7]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_qu Condition : ((isnotnull(cs_warehouse_sk#176) AND isnotnull(cs_sold_time_sk#174)) AND isnotnull(cs_ship_mode_sk#175)) (36) ReusedExchange [Reuses operator id: 7] -Output [7]: [w_warehouse_sk#181, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187] +Output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_warehouse_sk#176] -Right keys [1]: [w_warehouse_sk#181] +Right keys [1]: [w_warehouse_sk#9] Join condition: None (38) Project [codegen id : 11] -Output [12]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187] -Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_sk#181, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187] +Output [12]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] (39) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#188, d_year#189, d_moy#190] +Output [3]: [d_date_sk#17, d_year#18, d_moy#19] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#180] -Right keys [1]: [d_date_sk#188] +Right keys [1]: [d_date_sk#17] Join condition: None (41) Project [codegen id : 11] -Output [13]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] -Input [15]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_date_sk#188, d_year#189, d_moy#190] +Output [13]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] +Input [15]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_date_sk#17, d_year#18, d_moy#19] (42) ReusedExchange [Reuses operator id: 20] -Output [1]: [t_time_sk#191] +Output [1]: [t_time_sk#21] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_time_sk#174] -Right keys [1]: [t_time_sk#191] +Right keys [1]: [t_time_sk#21] Join condition: None (44) Project [codegen id : 11] -Output [12]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] -Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190, t_time_sk#191] +Output [12]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] +Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19, t_time_sk#21] (45) ReusedExchange [Reuses operator id: 27] -Output [1]: [sm_ship_mode_sk#192] +Output [1]: [sm_ship_mode_sk#24] (46) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_mode_sk#175] -Right keys [1]: [sm_ship_mode_sk#192] +Right keys [1]: [sm_ship_mode_sk#24] Join condition: None (47) Project [codegen id : 11] -Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] -Input [13]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190, sm_ship_mode_sk#192] +Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] +Input [13]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#24] (48) HashAggregate [codegen id : 11] -Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] -Keys [7]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189] -Functions [24]: [partial_sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240] -Results [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] +Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] +Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18] +Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [48]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228] +Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] (49) Exchange -Input [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] -Arguments: hashpartitioning(w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, 5), ENSURE_REQUIREMENTS, [id=#289] +Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#277] (50) HashAggregate [codegen id : 12] -Input [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] -Keys [7]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189] -Functions [24]: [sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313] -Results [32]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, DHL,BARIAN AS ship_carriers#314, d_year#189 AS year#315, sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_sales#316, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_sales#317, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_sales#318, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_sales#319, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_sales#320, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_sales#321, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_sales#322, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_sales#323, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_sales#324, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_sales#325, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_sales#326, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_sales#327, sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302 AS jan_net#328, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303 AS feb_net#329, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304 AS mar_net#330, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305 AS apr_net#331, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306 AS may_net#332, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307 AS jun_net#333, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308 AS jul_net#334, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309 AS aug_net#335, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310 AS sep_net#336, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311 AS oct_net#337, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312 AS nov_net#338, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313 AS dec_net#339] +Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] +Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18] +Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301] +Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, DHL,BARIAN AS ship_carriers#302, d_year#18 AS year#303, sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278 AS jan_sales#304, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279 AS feb_sales#305, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280 AS mar_sales#306, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281 AS apr_sales#307, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282 AS may_sales#308, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283 AS jun_sales#309, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284 AS jul_sales#310, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285 AS aug_sales#311, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286 AS sep_sales#312, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287 AS oct_sales#313, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288 AS nov_sales#314, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289 AS dec_sales#315, sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_net#316, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_net#317, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_net#318, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_net#319, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_net#320, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_net#321, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_net#322, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_net#323, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_net#324, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_net#325, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_net#326, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_net#327] (51) Union @@ -292,23 +292,23 @@ Results [32]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#150, feb_sales#151, mar_sales#152, apr_sales#153, may_sales#154, jun_sales#155, jul_sales#156, aug_sales#157, sep_sales#158, oct_sales#159, nov_sales#160, dec_sales#161, jan_net#162, feb_net#163, mar_net#164, apr_net#165, may_net#166, jun_net#167, jul_net#168, aug_net#169, sep_net#170, oct_net#171, nov_net#172, dec_net#173] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149] Functions [36]: [partial_sum(jan_sales#150), partial_sum(feb_sales#151), partial_sum(mar_sales#152), partial_sum(apr_sales#153), partial_sum(may_sales#154), partial_sum(jun_sales#155), partial_sum(jul_sales#156), partial_sum(aug_sales#157), partial_sum(sep_sales#158), partial_sum(oct_sales#159), partial_sum(nov_sales#160), partial_sum(dec_sales#161), partial_sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(jan_net#162), partial_sum(feb_net#163), partial_sum(mar_net#164), partial_sum(apr_net#165), partial_sum(may_net#166), partial_sum(jun_net#167), partial_sum(jul_net#168), partial_sum(aug_net#169), partial_sum(sep_net#170), partial_sum(oct_net#171), partial_sum(nov_net#172), partial_sum(dec_net#173)] -Aggregate Attributes [72]: [sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411] -Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] +Aggregate Attributes [72]: [sum#328, isEmpty#329, sum#330, isEmpty#331, sum#332, isEmpty#333, sum#334, isEmpty#335, sum#336, isEmpty#337, sum#338, isEmpty#339, sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399] +Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] (53) Exchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#484] +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#472] (54) HashAggregate [codegen id : 14] -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149] Functions [36]: [sum(jan_sales#150), sum(feb_sales#151), sum(mar_sales#152), sum(apr_sales#153), sum(may_sales#154), sum(jun_sales#155), sum(jul_sales#156), sum(aug_sales#157), sum(sep_sales#158), sum(oct_sales#159), sum(nov_sales#160), sum(dec_sales#161), sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(jan_net#162), sum(feb_net#163), sum(mar_net#164), sum(apr_net#165), sum(may_net#166), sum(jun_net#167), sum(jul_net#168), sum(aug_net#169), sum(sep_net#170), sum(oct_net#171), sum(nov_net#172), sum(dec_net#173)] -Aggregate Attributes [36]: [sum(jan_sales#150)#485, sum(feb_sales#151)#486, sum(mar_sales#152)#487, sum(apr_sales#153)#488, sum(may_sales#154)#489, sum(jun_sales#155)#490, sum(jul_sales#156)#491, sum(aug_sales#157)#492, sum(sep_sales#158)#493, sum(oct_sales#159)#494, sum(nov_sales#160)#495, sum(dec_sales#161)#496, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508, sum(jan_net#162)#509, sum(feb_net#163)#510, sum(mar_net#164)#511, sum(apr_net#165)#512, sum(may_net#166)#513, sum(jun_net#167)#514, sum(jul_net#168)#515, sum(aug_net#169)#516, sum(sep_net#170)#517, sum(oct_net#171)#518, sum(nov_net#172)#519, sum(dec_net#173)#520] -Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum(jan_sales#150)#485 AS jan_sales#521, sum(feb_sales#151)#486 AS feb_sales#522, sum(mar_sales#152)#487 AS mar_sales#523, sum(apr_sales#153)#488 AS apr_sales#524, sum(may_sales#154)#489 AS may_sales#525, sum(jun_sales#155)#490 AS jun_sales#526, sum(jul_sales#156)#491 AS jul_sales#527, sum(aug_sales#157)#492 AS aug_sales#528, sum(sep_sales#158)#493 AS sep_sales#529, sum(oct_sales#159)#494 AS oct_sales#530, sum(nov_sales#160)#495 AS nov_sales#531, sum(dec_sales#161)#496 AS dec_sales#532, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497 AS jan_sales_per_sq_foot#533, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498 AS feb_sales_per_sq_foot#534, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499 AS mar_sales_per_sq_foot#535, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500 AS apr_sales_per_sq_foot#536, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501 AS may_sales_per_sq_foot#537, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502 AS jun_sales_per_sq_foot#538, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503 AS jul_sales_per_sq_foot#539, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504 AS aug_sales_per_sq_foot#540, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505 AS sep_sales_per_sq_foot#541, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506 AS oct_sales_per_sq_foot#542, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507 AS nov_sales_per_sq_foot#543, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508 AS dec_sales_per_sq_foot#544, sum(jan_net#162)#509 AS jan_net#545, sum(feb_net#163)#510 AS feb_net#546, sum(mar_net#164)#511 AS mar_net#547, sum(apr_net#165)#512 AS apr_net#548, sum(may_net#166)#513 AS may_net#549, sum(jun_net#167)#514 AS jun_net#550, sum(jul_net#168)#515 AS jul_net#551, sum(aug_net#169)#516 AS aug_net#552, sum(sep_net#170)#517 AS sep_net#553, sum(oct_net#171)#518 AS oct_net#554, sum(nov_net#172)#519 AS nov_net#555, sum(dec_net#173)#520 AS dec_net#556] +Aggregate Attributes [36]: [sum(jan_sales#150)#473, sum(feb_sales#151)#474, sum(mar_sales#152)#475, sum(apr_sales#153)#476, sum(may_sales#154)#477, sum(jun_sales#155)#478, sum(jul_sales#156)#479, sum(aug_sales#157)#480, sum(sep_sales#158)#481, sum(oct_sales#159)#482, sum(nov_sales#160)#483, sum(dec_sales#161)#484, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496, sum(jan_net#162)#497, sum(feb_net#163)#498, sum(mar_net#164)#499, sum(apr_net#165)#500, sum(may_net#166)#501, sum(jun_net#167)#502, sum(jul_net#168)#503, sum(aug_net#169)#504, sum(sep_net#170)#505, sum(oct_net#171)#506, sum(nov_net#172)#507, sum(dec_net#173)#508] +Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum(jan_sales#150)#473 AS jan_sales#509, sum(feb_sales#151)#474 AS feb_sales#510, sum(mar_sales#152)#475 AS mar_sales#511, sum(apr_sales#153)#476 AS apr_sales#512, sum(may_sales#154)#477 AS may_sales#513, sum(jun_sales#155)#478 AS jun_sales#514, sum(jul_sales#156)#479 AS jul_sales#515, sum(aug_sales#157)#480 AS aug_sales#516, sum(sep_sales#158)#481 AS sep_sales#517, sum(oct_sales#159)#482 AS oct_sales#518, sum(nov_sales#160)#483 AS nov_sales#519, sum(dec_sales#161)#484 AS dec_sales#520, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485 AS jan_sales_per_sq_foot#521, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486 AS feb_sales_per_sq_foot#522, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487 AS mar_sales_per_sq_foot#523, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488 AS apr_sales_per_sq_foot#524, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489 AS may_sales_per_sq_foot#525, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490 AS jun_sales_per_sq_foot#526, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491 AS jul_sales_per_sq_foot#527, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492 AS aug_sales_per_sq_foot#528, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493 AS sep_sales_per_sq_foot#529, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494 AS oct_sales_per_sq_foot#530, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495 AS nov_sales_per_sq_foot#531, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496 AS dec_sales_per_sq_foot#532, sum(jan_net#162)#497 AS jan_net#533, sum(feb_net#163)#498 AS feb_net#534, sum(mar_net#164)#499 AS mar_net#535, sum(apr_net#165)#500 AS apr_net#536, sum(may_net#166)#501 AS may_net#537, sum(jun_net#167)#502 AS jun_net#538, sum(jul_net#168)#503 AS jul_net#539, sum(aug_net#169)#504 AS aug_net#540, sum(sep_net#170)#505 AS sep_net#541, sum(oct_net#171)#506 AS oct_net#542, sum(nov_net#172)#507 AS nov_net#543, sum(dec_net#173)#508 AS dec_net#544] (55) TakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, may_sales_per_sq_foot#537, jun_sales_per_sq_foot#538, jul_sales_per_sq_foot#539, aug_sales_per_sq_foot#540, sep_sales_per_sq_foot#541, oct_sales_per_sq_foot#542, nov_sales_per_sq_foot#543, dec_sales_per_sq_foot#544, jan_net#545, feb_net#546, mar_net#547, apr_net#548, may_net#549, jun_net#550, jul_net#551, aug_net#552, sep_net#553, oct_net#554, nov_net#555, dec_net#556] -Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, ... 20 more fields] +Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, may_sales_per_sq_foot#525, jun_sales_per_sq_foot#526, jul_sales_per_sq_foot#527, aug_sales_per_sq_foot#528, sep_sales_per_sq_foot#529, oct_sales_per_sq_foot#530, nov_sales_per_sq_foot#531, dec_sales_per_sq_foot#532, jan_net#533, feb_net#534, mar_net#535, apr_net#536, may_net#537, jun_net#538, jul_net#539, aug_net#540, sep_net#541, oct_net#542, nov_net#543, dec_net#544] +Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, ... 20 more fields] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt index b6a609ec193b4..744fe4b5a594d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt @@ -230,55 +230,55 @@ Input [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_ Arguments: [ss_addr_sk#12 ASC NULLS FIRST], false, 0 (41) ReusedExchange [Reuses operator id: 9] -Output [2]: [ca_address_sk#32, ca_city#33] +Output [2]: [ca_address_sk#6, ca_city#7] (42) Sort [codegen id : 13] -Input [2]: [ca_address_sk#32, ca_city#33] -Arguments: [ca_address_sk#32 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#6, ca_city#7] +Arguments: [ca_address_sk#6 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 14] Left keys [1]: [ss_addr_sk#12] -Right keys [1]: [ca_address_sk#32] +Right keys [1]: [ca_address_sk#6] Join condition: None (44) Project [codegen id : 14] -Output [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#33] -Input [8]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_address_sk#32, ca_city#33] +Output [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#7] +Input [8]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_address_sk#6, ca_city#7] (45) HashAggregate [codegen id : 14] -Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#33] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33] +Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#7] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#15)), partial_sum(UnscaledValue(ss_ext_list_price#16)), partial_sum(UnscaledValue(ss_ext_tax#17))] -Aggregate Attributes [3]: [sum#34, sum#35, sum#36] -Results [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33, sum#37, sum#38, sum#39] +Aggregate Attributes [3]: [sum#32, sum#33, sum#34] +Results [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#35, sum#36, sum#37] (46) HashAggregate [codegen id : 14] -Input [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33, sum#37, sum#38, sum#39] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33] +Input [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#35, sum#36, sum#37] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#15)), sum(UnscaledValue(ss_ext_list_price#16)), sum(UnscaledValue(ss_ext_tax#17))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#15))#40, sum(UnscaledValue(ss_ext_list_price#16))#41, sum(UnscaledValue(ss_ext_tax#17))#42] -Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#33 AS bought_city#43, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#40,17,2) AS extended_price#44, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#16))#41,17,2) AS list_price#45, MakeDecimal(sum(UnscaledValue(ss_ext_tax#17))#42,17,2) AS extended_tax#46] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#15))#38, sum(UnscaledValue(ss_ext_list_price#16))#39, sum(UnscaledValue(ss_ext_tax#17))#40] +Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#7 AS bought_city#41, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#38,17,2) AS extended_price#42, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#16))#39,17,2) AS list_price#43, MakeDecimal(sum(UnscaledValue(ss_ext_tax#17))#40,17,2) AS extended_tax#44] (47) Exchange -Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#43, extended_price#44, list_price#45, extended_tax#46] -Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#41, extended_price#42, list_price#43, extended_tax#44] +Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#45] (48) Sort [codegen id : 15] -Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#43, extended_price#44, list_price#45, extended_tax#46] +Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#41, extended_price#42, list_price#43, extended_tax#44] Arguments: [ss_customer_sk#10 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 16] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#10] -Join condition: NOT (ca_city#7 = bought_city#43) +Join condition: NOT (ca_city#7 = bought_city#41) (50) Project [codegen id : 16] -Output [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#43, ss_ticket_number#14, extended_price#44, extended_tax#46, list_price#45] -Input [10]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#43, extended_price#44, list_price#45, extended_tax#46] +Output [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#41, ss_ticket_number#14, extended_price#42, extended_tax#44, list_price#43] +Input [10]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#41, extended_price#42, list_price#43, extended_tax#44] (51) TakeOrderedAndProject -Input [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#43, ss_ticket_number#14, extended_price#44, extended_tax#46, list_price#45] -Arguments: 100, [c_last_name#4 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#43, ss_ticket_number#14, extended_price#44, extended_tax#46, list_price#45] +Input [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#41, ss_ticket_number#14, extended_price#42, extended_tax#44, list_price#43] +Arguments: 100, [c_last_name#4 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#41, ss_ticket_number#14, extended_price#42, extended_tax#44, list_price#43] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt index 101e4f272f98a..1a9fafda6250b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt @@ -225,20 +225,20 @@ Output [8]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#3 Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_customer_sk#39, c_current_addr_sk#40, c_first_name#41, c_last_name#42] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#44, ca_city#45] +Output [2]: [ca_address_sk#22, ca_city#23] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#40] -Right keys [1]: [ca_address_sk#44] -Join condition: NOT (ca_city#45 = bought_city#35) +Right keys [1]: [ca_address_sk#22] +Join condition: NOT (ca_city#23 = bought_city#35) (42) Project [codegen id : 8] -Output [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#44, ca_city#45] +Output [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#22, ca_city#23] (43) TakeOrderedAndProject -Input [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt index a0532462eff64..97d7a81baf2a7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt @@ -139,20 +139,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#8] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] +Right keys [1]: [d_date_sk#8] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#13] -Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] +Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] (23) Exchange Input [1]: [ws_bill_customer_sk#13] -Arguments: hashpartitioning(ws_bill_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#16] +Arguments: hashpartitioning(ws_bill_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#15] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#13] @@ -164,38 +164,38 @@ Right keys [1]: [ws_bill_customer_sk#13] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#19] +Output [1]: [d_date_sk#8] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#19] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#8] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#17] -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] +Output [1]: [cs_ship_customer_sk#16] +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] (31) Exchange -Input [1]: [cs_ship_customer_sk#17] -Arguments: hashpartitioning(cs_ship_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [cs_ship_customer_sk#16] +Arguments: hashpartitioning(cs_ship_customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#18] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#17] -Arguments: [cs_ship_customer_sk#17 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#16] +Arguments: [cs_ship_customer_sk#16 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#17] +Right keys [1]: [cs_ship_customer_sk#16] Join condition: None (34) Project [codegen id : 13] @@ -203,84 +203,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (35) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] +Output [2]: [ca_address_sk#19, ca_state#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 12] -Input [2]: [ca_address_sk#21, ca_state#22] +Input [2]: [ca_address_sk#19, ca_state#20] (37) Filter [codegen id : 12] -Input [2]: [ca_address_sk#21, ca_state#22] -Condition : (ca_state#22 IN (KY,GA,NM) AND isnotnull(ca_address_sk#21)) +Input [2]: [ca_address_sk#19, ca_state#20] +Condition : (ca_state#20 IN (KY,GA,NM) AND isnotnull(ca_address_sk#19)) (38) Project [codegen id : 12] -Output [1]: [ca_address_sk#21] -Input [2]: [ca_address_sk#21, ca_state#22] +Output [1]: [ca_address_sk#19] +Input [2]: [ca_address_sk#19, ca_state#20] (39) BroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [ca_address_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#19] Join condition: None (41) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] (42) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (43) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (44) ColumnarToRow -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] (45) Filter -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] -Condition : isnotnull(cd_demo_sk#25) +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Condition : isnotnull(cd_demo_sk#23) (46) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#23] Join condition: None (47) Project [codegen id : 14] -Output [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] (48) HashAggregate [codegen id : 14] -Input [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] -Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#31] -Results [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] +Aggregate Attributes [1]: [count#29] +Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] (49) Exchange -Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] +Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, 5), ENSURE_REQUIREMENTS, [id=#31] (50) HashAggregate [codegen id : 15] -Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] -Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] +Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#34] -Results [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#34 AS cnt1#35, cd_purchase_estimate#29, count(1)#34 AS cnt2#36, cd_credit_rating#30, count(1)#34 AS cnt3#37] +Aggregate Attributes [1]: [count(1)#32] +Results [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#32 AS cnt1#33, cd_purchase_estimate#27, count(1)#32 AS cnt2#34, cd_credit_rating#28, count(1)#32 AS cnt3#35] (51) TakeOrderedAndProject -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#35, cd_purchase_estimate#29, cnt2#36, cd_credit_rating#30, cnt3#37] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#35, cd_purchase_estimate#29, cnt2#36, cd_credit_rating#30, cnt3#37] +Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#33, cd_purchase_estimate#27, cnt2#34, cd_credit_rating#28, cnt3#35] +Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#33, cd_purchase_estimate#27, cnt2#34, cd_credit_rating#28, cnt3#35] ===== Subqueries ===== @@ -293,6 +293,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index d0b3c2231d997..b5307abaf688c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -122,20 +122,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#7] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#7] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#12] -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] @@ -143,34 +143,34 @@ Right keys [1]: [ws_bill_customer_sk#12] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#7] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#18] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#7] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#16] -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#18] +Output [1]: [cs_ship_customer_sk#15] +Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [cs_ship_customer_sk#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#16] +Right keys [1]: [cs_ship_customer_sk#15] Join condition: None (29) Project [codegen id : 9] @@ -178,84 +178,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (30) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +Output [2]: [ca_address_sk#18, ca_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_state#21] +Input [2]: [ca_address_sk#18, ca_state#19] (32) Filter [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : (ca_state#21 IN (KY,GA,NM) AND isnotnull(ca_address_sk#20)) +Input [2]: [ca_address_sk#18, ca_state#19] +Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) (33) Project [codegen id : 7] -Output [1]: [ca_address_sk#20] -Input [2]: [ca_address_sk#20, ca_state#21] +Output [1]: [ca_address_sk#18] +Input [2]: [ca_address_sk#18, ca_state#19] (34) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [ca_address_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#18] Join condition: None (36) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] (37) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] (39) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Condition : isnotnull(cd_demo_sk#23) +Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Condition : isnotnull(cd_demo_sk#21) (40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#21] Join condition: None (42) Project [codegen id : 9] -Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Output [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] (43) HashAggregate [codegen id : 9] -Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#30] -Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#31] +Aggregate Attributes [1]: [count#28] +Results [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] (44) Exchange -Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#31] -Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] +Arguments: hashpartitioning(cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, 5), ENSURE_REQUIREMENTS, [id=#30] (45) HashAggregate [codegen id : 10] -Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#31] -Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] +Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#33] -Results [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#33 AS cnt1#34, cd_purchase_estimate#27, count(1)#33 AS cnt2#35, cd_credit_rating#28, count(1)#33 AS cnt3#36] +Aggregate Attributes [1]: [count(1)#31] +Results [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, count(1)#31 AS cnt1#32, cd_purchase_estimate#25, count(1)#31 AS cnt2#33, cd_credit_rating#26, count(1)#31 AS cnt3#34] (46) TakeOrderedAndProject -Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#34, cd_purchase_estimate#27, cnt2#35, cd_credit_rating#28, cnt3#36] -Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#34, cd_purchase_estimate#27, cnt2#35, cd_credit_rating#28, cnt3#36] +Input [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#32, cd_purchase_estimate#25, cnt2#33, cd_credit_rating#26, cnt3#34] +Arguments: 100, [cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_education_status#24 ASC NULLS FIRST, cd_purchase_estimate#25 ASC NULLS FIRST, cd_credit_rating#26 ASC NULLS FIRST], [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#32, cd_purchase_estimate#25, cnt2#33, cd_credit_rating#26, cnt3#34] ===== Subqueries ===== @@ -268,6 +268,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index d31dbc3498ead..9e2d33f2bca19 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -194,7 +194,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) +Condition : (ranking#19 <= 5) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index 26fec145f4211..cd12e362c44f0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -194,7 +194,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) +Condition : (ranking#19 <= 5) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt index 3daa8b66851f7..cd88a84ede266 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt @@ -180,7 +180,7 @@ Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#15, sold_item_sk#16, Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [Or(EqualTo(t_meal_time,breakfast),EqualTo(t_meal_time,dinner)), IsNotNull(t_time_sk)] +PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] @@ -188,7 +188,7 @@ Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] (33) Filter [codegen id : 8] Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Condition : (((t_meal_time#35 = breakfast) OR (t_meal_time#35 = dinner)) AND isnotnull(t_time_sk#32)) +Condition : (((t_meal_time#35 = breakfast ) OR (t_meal_time#35 = dinner )) AND isnotnull(t_time_sk#32)) (34) Project [codegen id : 8] Output [3]: [t_time_sk#32, t_hour#33, t_minute#34] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index 3daa8b66851f7..cd88a84ede266 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt @@ -180,7 +180,7 @@ Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#15, sold_item_sk#16, Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [Or(EqualTo(t_meal_time,breakfast),EqualTo(t_meal_time,dinner)), IsNotNull(t_time_sk)] +PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] @@ -188,7 +188,7 @@ Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] (33) Filter [codegen id : 8] Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Condition : (((t_meal_time#35 = breakfast) OR (t_meal_time#35 = dinner)) AND isnotnull(t_time_sk#32)) +Condition : (((t_meal_time#35 = breakfast ) OR (t_meal_time#35 = dinner )) AND isnotnull(t_time_sk#32)) (34) Project [codegen id : 8] Output [3]: [t_time_sk#32, t_hour#33, t_minute#34] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt index 9b2ead7ea96f7..74cbccc79fdc5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt @@ -199,285 +199,285 @@ Input [2]: [customer_id#18, year_total#19] Arguments: [customer_id#18 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Output [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] (27) Filter [codegen id : 10] -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#21) +Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_customer_sk#1) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#25, d_year#26] +Output [2]: [d_date_sk#5, d_year#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#25, d_year#26] +Input [2]: [d_date_sk#5, d_year#6] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) +Input [2]: [d_date_sk#5, d_year#6] +Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2002)) AND d_year#6 IN (2001,2002)) AND isnotnull(d_date_sk#5)) (31) BroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [2]: [d_date_sk#5, d_year#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#23] -Right keys [1]: [d_date_sk#25] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] Join condition: None (33) Project [codegen id : 10] -Output [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] -Input [5]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] +Output [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] +Input [5]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6] (34) Exchange -Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] -Arguments: hashpartitioning(ss_customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] +Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#23] (35) Sort [codegen id : 11] -Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] -Arguments: [ss_customer_sk#21 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] +Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (37) Sort [codegen id : 13] -Input [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#21] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#9] Join condition: None (39) Project [codegen id : 14] -Output [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] -Input [7]: [ss_customer_sk#21, ss_net_paid#22, d_year#26, c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] +Input [7]: [ss_customer_sk#1, ss_net_paid#2, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (40) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] -Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] -Aggregate Attributes [1]: [sum#33] -Results [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#2))] +Aggregate Attributes [1]: [sum#24] +Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] (41) Exchange -Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] -Arguments: hashpartitioning(c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] +Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#26] (42) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] -Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] -Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#22))#36] -Results [4]: [c_customer_id#30 AS customer_id#37, c_first_name#31 AS customer_first_name#38, c_last_name#32 AS customer_last_name#39, MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#36,17,2) AS year_total#40] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [sum(UnscaledValue(ss_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#2))#27] +Results [4]: [c_customer_id#10 AS customer_id#28, c_first_name#11 AS customer_first_name#29, c_last_name#12 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#2))#27,17,2) AS year_total#31] (43) Exchange -Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] -Arguments: hashpartitioning(customer_id#37, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +Arguments: hashpartitioning(customer_id#28, 5), ENSURE_REQUIREMENTS, [id=#32] (44) Sort [codegen id : 16] -Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] -Arguments: [customer_id#37 ASC NULLS FIRST], false, 0 +Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +Arguments: [customer_id#28 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#37] +Right keys [1]: [customer_id#28] Join condition: None (46) 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#33, ws_net_paid#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] (48) Filter [codegen id : 19] -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#33, ws_net_paid#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_bill_customer_sk#33) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#45, d_year#46] +Output [2]: [d_date_sk#5, d_year#6] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#45] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#5] Join condition: None (51) Project [codegen id : 19] -Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] -Input [5]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] +Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] (52) Exchange -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] -Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#36] (53) Sort [codegen id : 20] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] -Arguments: [ws_bill_customer_sk#42 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (55) Sort [codegen id : 22] -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Arguments: [c_customer_sk#48 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#42] -Right keys [1]: [c_customer_sk#48] +Left keys [1]: [ws_bill_customer_sk#33] +Right keys [1]: [c_customer_sk#9] Join condition: None (57) Project [codegen id : 23] -Output [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] -Input [7]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46, c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] +Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (58) HashAggregate [codegen id : 23] -Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] -Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum#52] -Results [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum#37] +Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] (59) Exchange -Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] -Arguments: hashpartitioning(c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] +Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#39] (60) HashAggregate [codegen id : 24] -Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] -Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] -Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#55] -Results [2]: [c_customer_id#49 AS customer_id#56, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#55,17,2) AS year_total#57] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#40] +Results [2]: [c_customer_id#10 AS customer_id#41, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#40,17,2) AS year_total#42] (61) Filter [codegen id : 24] -Input [2]: [customer_id#56, year_total#57] -Condition : (isnotnull(year_total#57) AND (year_total#57 > 0.00)) +Input [2]: [customer_id#41, year_total#42] +Condition : (isnotnull(year_total#42) AND (year_total#42 > 0.00)) (62) Project [codegen id : 24] -Output [2]: [customer_id#56 AS customer_id#58, year_total#57 AS year_total#59] -Input [2]: [customer_id#56, year_total#57] +Output [2]: [customer_id#41 AS customer_id#43, year_total#42 AS year_total#44] +Input [2]: [customer_id#41, year_total#42] (63) Exchange -Input [2]: [customer_id#58, year_total#59] -Arguments: hashpartitioning(customer_id#58, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [2]: [customer_id#43, year_total#44] +Arguments: hashpartitioning(customer_id#43, 5), ENSURE_REQUIREMENTS, [id=#45] (64) Sort [codegen id : 25] -Input [2]: [customer_id#58, year_total#59] -Arguments: [customer_id#58 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#43, year_total#44] +Arguments: [customer_id#43 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#58] +Right keys [1]: [customer_id#43] Join condition: None (66) Project [codegen id : 26] -Output [7]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59] -Input [8]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, customer_id#58, year_total#59] +Output [7]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44] +Input [8]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#43, year_total#44] (67) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] (69) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_customer_sk#61) +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_bill_customer_sk#33) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#64, d_year#65] +Output [2]: [d_date_sk#5, d_year#6] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#63] -Right keys [1]: [d_date_sk#64] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#5] Join condition: None (72) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] -Input [5]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65] +Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] (73) Exchange -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] -Arguments: hashpartitioning(ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#46] (74) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] -Arguments: [ws_bill_customer_sk#61 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] +Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (76) Sort [codegen id : 31] -Input [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] -Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#61] -Right keys [1]: [c_customer_sk#67] +Left keys [1]: [ws_bill_customer_sk#33] +Right keys [1]: [c_customer_sk#9] Join condition: None (78) Project [codegen id : 32] -Output [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] -Input [7]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65, c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] +Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] +Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (79) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] -Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#62))] -Aggregate Attributes [1]: [sum#71] -Results [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum#47] +Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] (80) Exchange -Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] -Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] +Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#49] (81) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] -Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] -Functions [1]: [sum(UnscaledValue(ws_net_paid#62))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#62))#74] -Results [2]: [c_customer_id#68 AS customer_id#75, MakeDecimal(sum(UnscaledValue(ws_net_paid#62))#74,17,2) AS year_total#76] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#50] +Results [2]: [c_customer_id#10 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#50,17,2) AS year_total#52] (82) Exchange -Input [2]: [customer_id#75, year_total#76] -Arguments: hashpartitioning(customer_id#75, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [2]: [customer_id#51, year_total#52] +Arguments: hashpartitioning(customer_id#51, 5), ENSURE_REQUIREMENTS, [id=#53] (83) Sort [codegen id : 34] -Input [2]: [customer_id#75, year_total#76] -Arguments: [customer_id#75 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#51, year_total#52] +Arguments: [customer_id#51 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#75] -Join condition: (CASE WHEN (year_total#59 > 0.00) THEN CheckOverflow((promote_precision(year_total#76) / promote_precision(year_total#59)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#40) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#51] +Join condition: (CASE WHEN (year_total#44 > 0.00) THEN CheckOverflow((promote_precision(year_total#52) / promote_precision(year_total#44)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#31) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) (85) Project [codegen id : 35] -Output [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] -Input [9]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59, customer_id#75, year_total#76] +Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Input [9]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44, customer_id#51, year_total#52] (86) TakeOrderedAndProject -Input [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] -Arguments: 100, [customer_id#37 ASC NULLS FIRST, customer_id#37 ASC NULLS FIRST, customer_id#37 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39] +Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Arguments: 100, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] ===== Subqueries ===== @@ -488,15 +488,15 @@ ReusedExchange (87) (87) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#5, d_year#6] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#21 ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#25, d_year#26] +Output [2]: [d_date_sk#5, d_year#6] -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#24 +Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#21 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt index 9fccc4c4ba66d..db3a0a43be591 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt @@ -171,282 +171,282 @@ Input [2]: [customer_id#17, year_total#18] Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) (20) Scan parquet default.customer -Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] (22) Filter [codegen id : 6] -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Condition : (isnotnull(c_customer_sk#19) AND isnotnull(c_customer_id#20)) +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (23) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#19)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] (25) Filter [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_customer_sk#23) +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Condition : isnotnull(ss_customer_sk#5) (26) BroadcastExchange -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#19] -Right keys [1]: [ss_customer_sk#23] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#5] Join condition: None (28) Project [codegen id : 6] -Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25] -Input [7]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] +Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#28, d_year#29] +Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#28, d_year#29] +Input [2]: [d_date_sk#10, d_year#11] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#28, d_year#29] -Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) +Input [2]: [d_date_sk#10, d_year#11] +Condition : (((isnotnull(d_year#11) AND (d_year#11 = 2002)) AND d_year#11 IN (2001,2002)) AND isnotnull(d_date_sk#10)) (32) BroadcastExchange -Input [2]: [d_date_sk#28, d_year#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [2]: [d_date_sk#10, d_year#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#25] -Right keys [1]: [d_date_sk#28] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#10] Join condition: None (34) Project [codegen id : 6] -Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] -Input [7]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#28, d_year#29] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] +Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#10, d_year#11] (35) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] -Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum#31] -Results [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] +Aggregate Attributes [1]: [sum#22] +Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] (36) Exchange -Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] -Arguments: hashpartitioning(c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#24] (37) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] -Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] -Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#34] -Results [4]: [c_customer_id#20 AS customer_id#35, c_first_name#21 AS customer_first_name#36, c_last_name#22 AS customer_last_name#37, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#34,17,2) AS year_total#38] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#25] +Results [4]: [c_customer_id#2 AS customer_id#26, c_first_name#3 AS customer_first_name#27, c_last_name#4 AS customer_last_name#28, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#25,17,2) AS year_total#29] (38) BroadcastExchange -Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#39] +Input [4]: [customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#35] +Right keys [1]: [customer_id#26] Join condition: None (40) Scan parquet default.customer -Output [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] +Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] (42) Filter [codegen id : 10] -Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] -Condition : (isnotnull(c_customer_sk#40) AND isnotnull(c_customer_id#41)) +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (43) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] (45) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] -Condition : isnotnull(ws_bill_customer_sk#44) +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Condition : isnotnull(ws_bill_customer_sk#31) (46) BroadcastExchange -Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#40] -Right keys [1]: [ws_bill_customer_sk#44] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#31] Join condition: None (48) Project [codegen id : 10] -Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46] -Input [7]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] +Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#48, d_year#49] +Output [2]: [d_date_sk#10, d_year#11] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#46] -Right keys [1]: [d_date_sk#48] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#10] Join condition: None (51) Project [codegen id : 10] -Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] -Input [7]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46, d_date_sk#48, d_year#49] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] +Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] (52) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] -Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#45))] -Aggregate Attributes [1]: [sum#50] -Results [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] +Aggregate Attributes [1]: [sum#35] +Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] (53) Exchange -Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] -Arguments: hashpartitioning(c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#37] (54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] -Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] -Functions [1]: [sum(UnscaledValue(ws_net_paid#45))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#45))#53] -Results [2]: [c_customer_id#41 AS customer_id#54, MakeDecimal(sum(UnscaledValue(ws_net_paid#45))#53,17,2) AS year_total#55] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#38] +Results [2]: [c_customer_id#2 AS customer_id#39, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#38,17,2) AS year_total#40] (55) Filter [codegen id : 11] -Input [2]: [customer_id#54, year_total#55] -Condition : (isnotnull(year_total#55) AND (year_total#55 > 0.00)) +Input [2]: [customer_id#39, year_total#40] +Condition : (isnotnull(year_total#40) AND (year_total#40 > 0.00)) (56) Project [codegen id : 11] -Output [2]: [customer_id#54 AS customer_id#56, year_total#55 AS year_total#57] -Input [2]: [customer_id#54, year_total#55] +Output [2]: [customer_id#39 AS customer_id#41, year_total#40 AS year_total#42] +Input [2]: [customer_id#39, year_total#40] (57) BroadcastExchange -Input [2]: [customer_id#56, year_total#57] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#58] +Input [2]: [customer_id#41, year_total#42] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#43] (58) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#56] +Right keys [1]: [customer_id#41] Join condition: None (59) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57] -Input [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, customer_id#56, year_total#57] +Output [7]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42] +Input [8]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, customer_id#41, year_total#42] (60) Scan parquet default.customer -Output [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] +Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] (62) Filter [codegen id : 14] -Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] -Condition : (isnotnull(c_customer_sk#59) AND isnotnull(c_customer_id#60)) +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (63) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#19)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] (65) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] -Condition : isnotnull(ws_bill_customer_sk#63) +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Condition : isnotnull(ws_bill_customer_sk#31) (66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#59] -Right keys [1]: [ws_bill_customer_sk#63] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#31] Join condition: None (68) Project [codegen id : 14] -Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65] -Input [7]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] +Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#67, d_year#68] +Output [2]: [d_date_sk#10, d_year#11] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#67] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#10] Join condition: None (71) Project [codegen id : 14] -Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] -Input [7]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65, d_date_sk#67, d_year#68] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] +Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] (72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] -Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#64))] -Aggregate Attributes [1]: [sum#69] -Results [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] +Aggregate Attributes [1]: [sum#45] +Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] (73) Exchange -Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] -Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#47] (74) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] -Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] -Functions [1]: [sum(UnscaledValue(ws_net_paid#64))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#64))#72] -Results [2]: [c_customer_id#60 AS customer_id#73, MakeDecimal(sum(UnscaledValue(ws_net_paid#64))#72,17,2) AS year_total#74] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#48] +Results [2]: [c_customer_id#2 AS customer_id#49, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#48,17,2) AS year_total#50] (75) BroadcastExchange -Input [2]: [customer_id#73, year_total#74] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#75] +Input [2]: [customer_id#49, year_total#50] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#51] (76) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#73] -Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#57)), 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#49] +Join condition: (CASE WHEN (year_total#42 > 0.00) THEN CheckOverflow((promote_precision(year_total#50) / promote_precision(year_total#42)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#29) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END) (77) Project [codegen id : 16] -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#57, customer_id#73, year_total#74] +Output [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] +Input [9]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42, customer_id#49, year_total#50] (78) TakeOrderedAndProject -Input [3]: [customer_id#35, customer_first_name#36, customer_last_name#37] -Arguments: 100, [customer_id#35 ASC NULLS FIRST, customer_id#35 ASC NULLS FIRST, customer_id#35 ASC NULLS FIRST], [customer_id#35, customer_first_name#36, customer_last_name#37] +Input [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] +Arguments: 100, [customer_id#26 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST], [customer_id#26, customer_first_name#27, customer_last_name#28] ===== Subqueries ===== @@ -457,15 +457,15 @@ ReusedExchange (79) (79) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#10, d_year#11] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#19 ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#28, d_year#29] +Output [2]: [d_date_sk#10, d_year#11] -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#26 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#19 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt index 8279413489807..9472a8935eb8b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#31] +Right keys [1]: [i_item_sk#7] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#36, d_year#37] +Output [2]: [d_date_sk#14, d_year#15] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] +Right keys [1]: [d_date_sk#14] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] -Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] (42) Exchange -Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] -Condition : isnotnull(ws_item_sk#47) +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_item_sk#40) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#47] -Right keys [1]: [i_item_sk#52] +Left keys [1]: [ws_item_sk#40] +Right keys [1]: [i_item_sk#7] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] -Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#57, d_year#58] +Output [2]: [d_date_sk#14, d_year#15] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#57] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#14] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] (55) Exchange -Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] (61) Exchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] -Right keys [2]: [wr_order_number#61, wr_item_sk#60] +Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] +Right keys [2]: [wr_order_number#47, wr_item_sk#46] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] -Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] +Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#69, sum#70] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Aggregate Attributes [2]: [sum#55, sum#56] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] +Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] -Condition : isnotnull(cs_item_sk#79) +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Condition : isnotnull(cs_item_sk#1) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#79] -Right keys [1]: [i_item_sk#85] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#66] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] -Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#71, d_year#72] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#90, d_year#91] +Input [2]: [d_date_sk#71, d_year#72] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#90, d_year#91] -Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) +Input [2]: [d_date_sk#71, d_year#72] +Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) (83) BroadcastExchange -Input [2]: [d_date_sk#90, d_year#91] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] +Input [2]: [d_date_sk#71, d_year#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#83] -Right keys [1]: [d_date_sk#90] +Left keys [1]: [cs_sold_date_sk#5] +Right keys [1]: [d_date_sk#71] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (86) Exchange -Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] -Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#80, cs_item_sk#79] -Right keys [2]: [cr_order_number#95, cr_item_sk#94] +Left keys [2]: [cs_order_number#2, cs_item_sk#1] +Right keys [2]: [cr_order_number#19, cr_item_sk#18] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] -Condition : isnotnull(ss_item_sk#98) +Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Condition : isnotnull(ss_item_sk#26) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#98] -Right keys [1]: [i_item_sk#103] +Left keys [1]: [ss_item_sk#26] +Right keys [1]: [i_item_sk#66] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] -Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#108, d_year#109] +Output [2]: [d_date_sk#71, d_year#72] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#102] -Right keys [1]: [d_date_sk#108] +Left keys [1]: [ss_sold_date_sk#30] +Right keys [1]: [d_date_sk#71] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (101) Exchange -Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] -Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] -Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] +Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] -Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] -Condition : isnotnull(ws_item_sk#116) +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_item_sk#40) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#116] -Right keys [1]: [i_item_sk#121] +Left keys [1]: [ws_item_sk#40] +Right keys [1]: [i_item_sk#66] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] -Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#126, d_year#127] +Output [2]: [d_date_sk#71, d_year#72] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#120] -Right keys [1]: [d_date_sk#126] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#71] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (116) Exchange -Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] -Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] -Right keys [2]: [wr_order_number#130, wr_item_sk#129] +Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] +Right keys [2]: [wr_order_number#47, wr_item_sk#46] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] -Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] +Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#135, sum#136] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Aggregate Attributes [2]: [sum#82, sum#83] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] (127) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] -Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] -Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] +Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] (129) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] -Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] (130) Sort [codegen id : 50] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] -Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 (131) 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#86, i_class_id#87, i_category_id#88, i_manufact_id#89] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] (133) TakeOrderedAndProject -Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] -Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#71, d_year#72] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt index 8279413489807..9472a8935eb8b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#31] +Right keys [1]: [i_item_sk#7] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#36, d_year#37] +Output [2]: [d_date_sk#14, d_year#15] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] +Right keys [1]: [d_date_sk#14] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] -Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] (42) Exchange -Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] -Condition : isnotnull(ws_item_sk#47) +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_item_sk#40) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#47] -Right keys [1]: [i_item_sk#52] +Left keys [1]: [ws_item_sk#40] +Right keys [1]: [i_item_sk#7] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] -Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#57, d_year#58] +Output [2]: [d_date_sk#14, d_year#15] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#57] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#14] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] (55) Exchange -Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] (61) Exchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] -Right keys [2]: [wr_order_number#61, wr_item_sk#60] +Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] +Right keys [2]: [wr_order_number#47, wr_item_sk#46] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] -Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] +Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#69, sum#70] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Aggregate Attributes [2]: [sum#55, sum#56] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] +Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] -Condition : isnotnull(cs_item_sk#79) +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Condition : isnotnull(cs_item_sk#1) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#79] -Right keys [1]: [i_item_sk#85] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#66] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] -Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#71, d_year#72] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#90, d_year#91] +Input [2]: [d_date_sk#71, d_year#72] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#90, d_year#91] -Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) +Input [2]: [d_date_sk#71, d_year#72] +Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) (83) BroadcastExchange -Input [2]: [d_date_sk#90, d_year#91] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] +Input [2]: [d_date_sk#71, d_year#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#83] -Right keys [1]: [d_date_sk#90] +Left keys [1]: [cs_sold_date_sk#5] +Right keys [1]: [d_date_sk#71] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (86) Exchange -Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] -Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#80, cs_item_sk#79] -Right keys [2]: [cr_order_number#95, cr_item_sk#94] +Left keys [2]: [cs_order_number#2, cs_item_sk#1] +Right keys [2]: [cr_order_number#19, cr_item_sk#18] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] -Condition : isnotnull(ss_item_sk#98) +Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Condition : isnotnull(ss_item_sk#26) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#98] -Right keys [1]: [i_item_sk#103] +Left keys [1]: [ss_item_sk#26] +Right keys [1]: [i_item_sk#66] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] -Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#108, d_year#109] +Output [2]: [d_date_sk#71, d_year#72] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#102] -Right keys [1]: [d_date_sk#108] +Left keys [1]: [ss_sold_date_sk#30] +Right keys [1]: [d_date_sk#71] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (101) Exchange -Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] -Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] -Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] +Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] -Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] -Condition : isnotnull(ws_item_sk#116) +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_item_sk#40) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#116] -Right keys [1]: [i_item_sk#121] +Left keys [1]: [ws_item_sk#40] +Right keys [1]: [i_item_sk#66] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] -Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#126, d_year#127] +Output [2]: [d_date_sk#71, d_year#72] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#120] -Right keys [1]: [d_date_sk#126] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#71] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (116) Exchange -Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] -Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] -Right keys [2]: [wr_order_number#130, wr_item_sk#129] +Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] +Right keys [2]: [wr_order_number#47, wr_item_sk#46] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] -Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] +Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#135, sum#136] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Aggregate Attributes [2]: [sum#82, sum#83] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] (127) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] -Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] -Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] +Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] (129) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] -Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] (130) Sort [codegen id : 50] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] -Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 (131) 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#86, i_class_id#87, i_category_id#88, i_manufact_id#89] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] (133) TakeOrderedAndProject -Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] -Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#71, d_year#72] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt index 9284172139688..0092354e54be0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt @@ -134,93 +134,93 @@ Input [4]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sol Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [id=#19] (20) Scan parquet default.date_dim -Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] +Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow -Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] (22) Filter -Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] -Condition : isnotnull(d_date_sk#20) +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Condition : isnotnull(d_date_sk#5) (23) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_sold_date_sk#18] -Right keys [1]: [d_date_sk#20] +Right keys [1]: [d_date_sk#5] Join condition: None (24) Project [codegen id : 5] -Output [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#21, d_qoy#22] -Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, d_date_sk#20, d_year#21, d_qoy#22] +Output [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#6, d_qoy#7] +Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, d_date_sk#5, d_year#6, d_qoy#7] (25) BroadcastExchange -Input [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#21, d_qoy#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#6, d_qoy#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (26) Scan parquet default.item -Output [2]: [i_item_sk#24, i_category#25] +Output [2]: [i_item_sk#9, i_category#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (27) ColumnarToRow -Input [2]: [i_item_sk#24, i_category#25] +Input [2]: [i_item_sk#9, i_category#10] (28) Filter -Input [2]: [i_item_sk#24, i_category#25] -Condition : isnotnull(i_item_sk#24) +Input [2]: [i_item_sk#9, i_category#10] +Condition : isnotnull(i_item_sk#9) (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#15] -Right keys [1]: [i_item_sk#24] +Right keys [1]: [i_item_sk#9] Join condition: None (30) Project [codegen id : 6] -Output [6]: [web AS channel#26, ws_ship_customer_sk#16 AS col_name#27, d_year#21, d_qoy#22, i_category#25, ws_ext_sales_price#17 AS ext_sales_price#28] -Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#21, d_qoy#22, i_item_sk#24, i_category#25] +Output [6]: [web AS channel#21, ws_ship_customer_sk#16 AS col_name#22, d_year#6, d_qoy#7, i_category#10, ws_ext_sales_price#17 AS ext_sales_price#23] +Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#6, d_qoy#7, i_item_sk#9, i_category#10] (31) Scan parquet default.catalog_sales -Output [4]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32] +Output [4]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#32)] +PartitionFilters: [isnotnull(cs_sold_date_sk#27)] PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 9] -Input [4]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32] +Input [4]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] (33) Filter [codegen id : 9] -Input [4]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32] -Condition : (isnull(cs_ship_addr_sk#29) AND isnotnull(cs_item_sk#30)) +Input [4]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +Condition : (isnull(cs_ship_addr_sk#24) AND isnotnull(cs_item_sk#25)) (34) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] (35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#32] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [cs_sold_date_sk#27] +Right keys [1]: [d_date_sk#5] Join condition: None (36) Project [codegen id : 9] -Output [5]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, d_year#34, d_qoy#35] -Input [7]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32, d_date_sk#33, d_year#34, d_qoy#35] +Output [5]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_year#6, d_qoy#7] +Input [7]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#5, d_year#6, d_qoy#7] (37) ReusedExchange [Reuses operator id: 13] -Output [2]: [i_item_sk#36, i_category#37] +Output [2]: [i_item_sk#9, i_category#10] (38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#30] -Right keys [1]: [i_item_sk#36] +Left keys [1]: [cs_item_sk#25] +Right keys [1]: [i_item_sk#9] Join condition: None (39) Project [codegen id : 9] -Output [6]: [catalog AS channel#38, cs_ship_addr_sk#29 AS col_name#39, d_year#34, d_qoy#35, i_category#37, cs_ext_sales_price#31 AS ext_sales_price#40] -Input [7]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, d_year#34, d_qoy#35, i_item_sk#36, i_category#37] +Output [6]: [catalog AS channel#28, cs_ship_addr_sk#24 AS col_name#29, d_year#6, d_qoy#7, i_category#10, cs_ext_sales_price#26 AS ext_sales_price#30] +Input [7]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_year#6, d_qoy#7, i_item_sk#9, i_category#10] (40) Union @@ -228,21 +228,21 @@ Input [7]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, d_year#34, Input [6]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, ext_sales_price#14] Keys [5]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10] Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count#41, sum#42] -Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#43, sum#44] +Aggregate Attributes [2]: [count#31, sum#32] +Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#33, sum#34] (42) Exchange -Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#43, sum#44] -Arguments: hashpartitioning(channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#33, sum#34] +Arguments: hashpartitioning(channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, 5), ENSURE_REQUIREMENTS, [id=#35] (43) HashAggregate [codegen id : 11] -Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#43, sum#44] +Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#33, sum#34] Keys [5]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count(1)#46, sum(UnscaledValue(ext_sales_price#14))#47] -Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count(1)#46 AS sales_cnt#48, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#47,17,2) AS sales_amt#49] +Aggregate Attributes [2]: [count(1)#36, sum(UnscaledValue(ext_sales_price#14))#37] +Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count(1)#36 AS sales_cnt#38, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#37,17,2) AS sales_amt#39] (44) TakeOrderedAndProject -Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#48, sales_amt#49] -Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#6 ASC NULLS FIRST, d_qoy#7 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#48, sales_amt#49] +Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#38, sales_amt#39] +Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#6 ASC NULLS FIRST, d_qoy#7 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#38, sales_amt#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt index 026c9396cd025..1250725d1782e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt @@ -124,67 +124,67 @@ Input [4]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sol Condition : (isnull(ws_ship_customer_sk#16) AND isnotnull(ws_item_sk#15)) (19) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#19, i_category#20] +Output [2]: [i_item_sk#5, i_category#6] (20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#15] -Right keys [1]: [i_item_sk#19] +Right keys [1]: [i_item_sk#5] Join condition: None (21) Project [codegen id : 6] -Output [4]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#20] -Input [6]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_item_sk#19, i_category#20] +Output [4]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#6] +Input [6]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_item_sk#5, i_category#6] (22) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] +Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] (23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#18] -Right keys [1]: [d_date_sk#21] +Right keys [1]: [d_date_sk#8] Join condition: None (24) Project [codegen id : 6] -Output [6]: [web AS channel#24, ws_ship_customer_sk#16 AS col_name#25, d_year#22, d_qoy#23, i_category#20, ws_ext_sales_price#17 AS ext_sales_price#26] -Input [7]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#20, d_date_sk#21, d_year#22, d_qoy#23] +Output [6]: [web AS channel#19, ws_ship_customer_sk#16 AS col_name#20, d_year#9, d_qoy#10, i_category#6, ws_ext_sales_price#17 AS ext_sales_price#21] +Input [7]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#6, d_date_sk#8, d_year#9, d_qoy#10] (25) Scan parquet default.catalog_sales -Output [4]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30] +Output [4]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#30)] +PartitionFilters: [isnotnull(cs_sold_date_sk#25)] PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 9] -Input [4]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30] +Input [4]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25] (27) Filter [codegen id : 9] -Input [4]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30] -Condition : (isnull(cs_ship_addr_sk#27) AND isnotnull(cs_item_sk#28)) +Input [4]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25] +Condition : (isnull(cs_ship_addr_sk#22) AND isnotnull(cs_item_sk#23)) (28) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#31, i_category#32] +Output [2]: [i_item_sk#5, i_category#6] (29) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#28] -Right keys [1]: [i_item_sk#31] +Left keys [1]: [cs_item_sk#23] +Right keys [1]: [i_item_sk#5] Join condition: None (30) Project [codegen id : 9] -Output [4]: [cs_ship_addr_sk#27, cs_ext_sales_price#29, cs_sold_date_sk#30, i_category#32] -Input [6]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30, i_item_sk#31, i_category#32] +Output [4]: [cs_ship_addr_sk#22, cs_ext_sales_price#24, cs_sold_date_sk#25, i_category#6] +Input [6]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25, i_item_sk#5, i_category#6] (31) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#30] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [cs_sold_date_sk#25] +Right keys [1]: [d_date_sk#8] Join condition: None (33) Project [codegen id : 9] -Output [6]: [catalog AS channel#36, cs_ship_addr_sk#27 AS col_name#37, d_year#34, d_qoy#35, i_category#32, cs_ext_sales_price#29 AS ext_sales_price#38] -Input [7]: [cs_ship_addr_sk#27, cs_ext_sales_price#29, cs_sold_date_sk#30, i_category#32, d_date_sk#33, d_year#34, d_qoy#35] +Output [6]: [catalog AS channel#26, cs_ship_addr_sk#22 AS col_name#27, d_year#9, d_qoy#10, i_category#6, cs_ext_sales_price#24 AS ext_sales_price#28] +Input [7]: [cs_ship_addr_sk#22, cs_ext_sales_price#24, cs_sold_date_sk#25, i_category#6, d_date_sk#8, d_year#9, d_qoy#10] (34) Union @@ -192,21 +192,21 @@ Input [7]: [cs_ship_addr_sk#27, cs_ext_sales_price#29, cs_sold_date_sk#30, i_cat Input [6]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, ext_sales_price#14] Keys [5]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6] Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count#39, sum#40] -Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#41, sum#42] +Aggregate Attributes [2]: [count#29, sum#30] +Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] (36) Exchange -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#41, sum#42] -Arguments: hashpartitioning(channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] +Arguments: hashpartitioning(channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, 5), ENSURE_REQUIREMENTS, [id=#33] (37) HashAggregate [codegen id : 11] -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#41, sum#42] +Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] Keys [5]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count(1)#44, sum(UnscaledValue(ext_sales_price#14))#45] -Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count(1)#44 AS sales_cnt#46, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#45,17,2) AS sales_amt#47] +Aggregate Attributes [2]: [count(1)#34, sum(UnscaledValue(ext_sales_price#14))#35] +Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count(1)#34 AS sales_cnt#36, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#35,17,2) AS sales_amt#37] (38) TakeOrderedAndProject -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#46, sales_amt#47] -Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#46, sales_amt#47] +Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#36, sales_amt#37] +Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#36, sales_amt#37] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt index 4b2299ca2e749..a3626dfe3293d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt @@ -209,38 +209,38 @@ Output [4]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_s Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25] (26) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#26] +Output [1]: [d_date_sk#6] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#26 as bigint)] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (28) Project [codegen id : 6] Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#26] +Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#6] (29) HashAggregate [codegen id : 6] Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] Keys [1]: [s_store_sk#25] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#27, sum#28] -Results [3]: [s_store_sk#25, sum#29, sum#30] +Aggregate Attributes [2]: [sum#26, sum#27] +Results [3]: [s_store_sk#25, sum#28, sum#29] (30) Exchange -Input [3]: [s_store_sk#25, sum#29, sum#30] -Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [s_store_sk#25, sum#28, sum#29] +Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#25, sum#29, sum#30] +Input [3]: [s_store_sk#25, sum#28, sum#29] Keys [1]: [s_store_sk#25] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] -Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] +Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] (32) BroadcastExchange -Input [3]: [s_store_sk#25, returns#34, profit_loss#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [3]: [s_store_sk#25, returns#33, profit_loss#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] @@ -248,269 +248,269 @@ Right keys [1]: [s_store_sk#25] Join condition: None (34) Project [codegen id : 8] -Output [5]: [sales#18, coalesce(returns#34, 0.00) AS returns#37, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#38, store channel AS channel#39, s_store_sk#9 AS id#40] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#34, profit_loss#35] +Output [5]: [sales#18, coalesce(returns#33, 0.00) AS returns#36, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#37, store channel AS channel#38, s_store_sk#9 AS id#39] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] +Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] +Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#45] +Output [1]: [d_date_sk#6] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#44] -Right keys [1]: [d_date_sk#45] +Left keys [1]: [cs_sold_date_sk#43] +Right keys [1]: [d_date_sk#6] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] -Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] +Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] +Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] -Keys [1]: [cs_call_center_sk#41] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] -Aggregate Attributes [2]: [sum#46, sum#47] -Results [3]: [cs_call_center_sk#41, sum#48, sum#49] +Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] +Keys [1]: [cs_call_center_sk#40] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] +Aggregate Attributes [2]: [sum#44, sum#45] +Results [3]: [cs_call_center_sk#40, sum#46, sum#47] (41) Exchange -Input [3]: [cs_call_center_sk#41, sum#48, sum#49] -Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [3]: [cs_call_center_sk#40, sum#46, sum#47] +Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] (42) HashAggregate [codegen id : 14] -Input [3]: [cs_call_center_sk#41, sum#48, sum#49] -Keys [1]: [cs_call_center_sk#41] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] -Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] +Input [3]: [cs_call_center_sk#40, sum#46, sum#47] +Keys [1]: [cs_call_center_sk#40] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] +Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] (43) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Output [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#55), dynamicpruningexpression(cr_returned_date_sk#55 IN dynamicpruning#5)] ReadSchema: struct (44) ColumnarToRow [codegen id : 12] -Input [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Input [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] (45) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#58] +Output [1]: [d_date_sk#6] (46) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cr_returned_date_sk#57] -Right keys [1]: [d_date_sk#58] +Left keys [1]: [cr_returned_date_sk#55] +Right keys [1]: [d_date_sk#6] Join condition: None (47) Project [codegen id : 12] -Output [2]: [cr_return_amount#55, cr_net_loss#56] -Input [4]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57, d_date_sk#58] +Output [2]: [cr_return_amount#53, cr_net_loss#54] +Input [4]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55, d_date_sk#6] (48) HashAggregate [codegen id : 12] -Input [2]: [cr_return_amount#55, cr_net_loss#56] +Input [2]: [cr_return_amount#53, cr_net_loss#54] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#55)), partial_sum(UnscaledValue(cr_net_loss#56))] -Aggregate Attributes [2]: [sum#59, sum#60] -Results [2]: [sum#61, sum#62] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#53)), partial_sum(UnscaledValue(cr_net_loss#54))] +Aggregate Attributes [2]: [sum#56, sum#57] +Results [2]: [sum#58, sum#59] (49) Exchange -Input [2]: [sum#61, sum#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] +Input [2]: [sum#58, sum#59] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#60] (50) HashAggregate [codegen id : 13] -Input [2]: [sum#61, sum#62] +Input [2]: [sum#58, sum#59] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#55)), sum(UnscaledValue(cr_net_loss#56))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#55))#64, sum(UnscaledValue(cr_net_loss#56))#65] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#55))#64,17,2) AS returns#66, MakeDecimal(sum(UnscaledValue(cr_net_loss#56))#65,17,2) AS profit_loss#67] +Functions [2]: [sum(UnscaledValue(cr_return_amount#53)), sum(UnscaledValue(cr_net_loss#54))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#53))#61, sum(UnscaledValue(cr_net_loss#54))#62] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#53))#61,17,2) AS returns#63, MakeDecimal(sum(UnscaledValue(cr_net_loss#54))#62,17,2) AS profit_loss#64] (51) BroadcastExchange -Input [2]: [returns#66, profit_loss#67] -Arguments: IdentityBroadcastMode, [id=#68] +Input [2]: [returns#63, profit_loss#64] +Arguments: IdentityBroadcastMode, [id=#65] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [sales#53, returns#66, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#67 as decimal(18,2)))), DecimalType(18,2), true) AS profit#69, catalog channel AS channel#70, cs_call_center_sk#41 AS id#71] -Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#66, profit_loss#67] +Output [5]: [sales#51, returns#63, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#64 as decimal(18,2)))), DecimalType(18,2), true) AS profit#66, catalog channel AS channel#67, cs_call_center_sk#40 AS id#68] +Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#63, profit_loss#64] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] -Condition : isnotnull(ws_web_page_sk#72) +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Condition : isnotnull(ws_web_page_sk#69) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#76] +Output [1]: [d_date_sk#6] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#75] -Right keys [1]: [d_date_sk#76] +Left keys [1]: [ws_sold_date_sk#72] +Right keys [1]: [d_date_sk#6] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] -Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] +Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] +Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#77] +Output [1]: [wp_web_page_sk#73] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#77] +Input [1]: [wp_web_page_sk#73] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#77] -Condition : isnotnull(wp_web_page_sk#77) +Input [1]: [wp_web_page_sk#73] +Condition : isnotnull(wp_web_page_sk#73) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#77] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] +Input [1]: [wp_web_page_sk#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#72] -Right keys [1]: [wp_web_page_sk#77] +Left keys [1]: [ws_web_page_sk#69] +Right keys [1]: [wp_web_page_sk#73] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] -Aggregate Attributes [2]: [sum#79, sum#80] -Results [3]: [wp_web_page_sk#77, sum#81, sum#82] +Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Keys [1]: [wp_web_page_sk#73] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] +Aggregate Attributes [2]: [sum#75, sum#76] +Results [3]: [wp_web_page_sk#73, sum#77, sum#78] (67) Exchange -Input [3]: [wp_web_page_sk#77, sum#81, sum#82] -Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [3]: [wp_web_page_sk#73, sum#77, sum#78] +Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#77, sum#81, sum#82] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] -Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] +Input [3]: [wp_web_page_sk#73, sum#77, sum#78] +Keys [1]: [wp_web_page_sk#73] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] +Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] -Condition : isnotnull(wr_web_page_sk#88) +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Condition : isnotnull(wr_web_page_sk#84) (72) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#92] +Output [1]: [wp_web_page_sk#88] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#88] -Right keys [1]: [cast(wp_web_page_sk#92 as bigint)] +Left keys [1]: [wr_web_page_sk#84] +Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [4]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] -Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] +Output [4]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] +Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] (75) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#93] +Output [1]: [d_date_sk#6] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#91] -Right keys [1]: [cast(d_date_sk#93 as bigint)] +Left keys [1]: [wr_returned_date_sk#87] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] -Input [5]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92, d_date_sk#93] +Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Input [5]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88, d_date_sk#6] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] -Keys [1]: [wp_web_page_sk#92] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] -Aggregate Attributes [2]: [sum#94, sum#95] -Results [3]: [wp_web_page_sk#92, sum#96, sum#97] +Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Keys [1]: [wp_web_page_sk#88] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] +Aggregate Attributes [2]: [sum#89, sum#90] +Results [3]: [wp_web_page_sk#88, sum#91, sum#92] (79) Exchange -Input [3]: [wp_web_page_sk#92, sum#96, sum#97] -Arguments: hashpartitioning(wp_web_page_sk#92, 5), ENSURE_REQUIREMENTS, [id=#98] +Input [3]: [wp_web_page_sk#88, sum#91, sum#92] +Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#92, sum#96, sum#97] -Keys [1]: [wp_web_page_sk#92] -Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] -Results [3]: [wp_web_page_sk#92, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] +Input [3]: [wp_web_page_sk#88, sum#91, sum#92] +Keys [1]: [wp_web_page_sk#88] +Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] +Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#92, returns#101, profit_loss#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] +Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#77] -Right keys [1]: [wp_web_page_sk#92] +Left keys [1]: [wp_web_page_sk#73] +Right keys [1]: [wp_web_page_sk#88] Join condition: None (83) Project [codegen id : 22] -Output [5]: [sales#86, coalesce(returns#101, 0.00) AS returns#104, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#105, web channel AS channel#106, wp_web_page_sk#77 AS id#107] -Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#92, returns#101, profit_loss#102] +Output [5]: [sales#82, coalesce(returns#96, 0.00) AS returns#99, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#100, web channel AS channel#101, wp_web_page_sk#73 AS id#102] +Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] (84) Union (85) Expand [codegen id : 23] -Input [5]: [sales#18, returns#37, profit#38, channel#39, id#40] -Arguments: [List(sales#18, returns#37, profit#38, channel#39, id#40, 0), List(sales#18, returns#37, profit#38, channel#39, null, 1), List(sales#18, returns#37, profit#38, null, null, 3)], [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] +Input [5]: [sales#18, returns#36, profit#37, channel#38, id#39] +Arguments: [List(sales#18, returns#36, profit#37, channel#38, id#39, 0), List(sales#18, returns#36, profit#37, channel#38, null, 1), List(sales#18, returns#36, profit#37, null, null, 3)], [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] (86) HashAggregate [codegen id : 23] -Input [6]: [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] -Keys [3]: [channel#108, id#109, spark_grouping_id#110] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#37), partial_sum(profit#38)] -Aggregate Attributes [6]: [sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] -Results [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Input [6]: [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] +Keys [3]: [channel#103, id#104, spark_grouping_id#105] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#36), partial_sum(profit#37)] +Aggregate Attributes [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] +Results [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] (87) Exchange -Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Arguments: hashpartitioning(channel#108, id#109, spark_grouping_id#110, 5), ENSURE_REQUIREMENTS, [id=#123] +Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Arguments: hashpartitioning(channel#103, id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, [id=#118] (88) HashAggregate [codegen id : 24] -Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Keys [3]: [channel#108, id#109, spark_grouping_id#110] -Functions [3]: [sum(sales#18), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#18)#124, sum(returns#37)#125, sum(profit#38)#126] -Results [5]: [channel#108, id#109, sum(sales#18)#124 AS sales#127, sum(returns#37)#125 AS returns#128, sum(profit#38)#126 AS profit#129] +Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Keys [3]: [channel#103, id#104, spark_grouping_id#105] +Functions [3]: [sum(sales#18), sum(returns#36), sum(profit#37)] +Aggregate Attributes [3]: [sum(sales#18)#119, sum(returns#36)#120, sum(profit#37)#121] +Results [5]: [channel#103, id#104, sum(sales#18)#119 AS sales#122, sum(returns#36)#120 AS returns#123, sum(profit#37)#121 AS profit#124] (89) TakeOrderedAndProject -Input [5]: [channel#108, id#109, sales#127, returns#128, profit#129] -Arguments: 100, [channel#108 ASC NULLS FIRST, id#109 ASC NULLS FIRST], [channel#108, id#109, sales#127, returns#128, profit#129] +Input [5]: [channel#103, id#104, sales#122, returns#123, profit#124] +Arguments: 100, [channel#103 ASC NULLS FIRST, id#104 ASC NULLS FIRST], [channel#103, id#104, sales#122, returns#123, profit#124] ===== Subqueries ===== @@ -526,14 +526,14 @@ ReusedExchange (91) (91) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#26] +Output [1]: [d_date_sk#6] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#55 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt index 618da39637e23..7a277d5362e32 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt @@ -197,320 +197,320 @@ Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_s Condition : isnotnull(sr_store_sk#20) (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#6] (24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#25 as bigint)] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (25) Project [codegen id : 6] Output [3]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22] -Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#25] +Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#6] (26) ReusedExchange [Reuses operator id: 14] -Output [1]: [s_store_sk#26] +Output [1]: [s_store_sk#25] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_store_sk#20] -Right keys [1]: [cast(s_store_sk#26 as bigint)] +Right keys [1]: [cast(s_store_sk#25 as bigint)] Join condition: None (28) Project [codegen id : 6] -Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] -Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#26] +Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] +Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#25] (29) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] -Keys [1]: [s_store_sk#26] +Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] +Keys [1]: [s_store_sk#25] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#27, sum#28] -Results [3]: [s_store_sk#26, sum#29, sum#30] +Aggregate Attributes [2]: [sum#26, sum#27] +Results [3]: [s_store_sk#25, sum#28, sum#29] (30) Exchange -Input [3]: [s_store_sk#26, sum#29, sum#30] -Arguments: hashpartitioning(s_store_sk#26, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [s_store_sk#25, sum#28, sum#29] +Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#26, sum#29, sum#30] -Keys [1]: [s_store_sk#26] +Input [3]: [s_store_sk#25, sum#28, sum#29] +Keys [1]: [s_store_sk#25] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] -Results [3]: [s_store_sk#26, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] +Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] (32) BroadcastExchange -Input [3]: [s_store_sk#26, returns#34, profit_loss#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [3]: [s_store_sk#25, returns#33, profit_loss#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] -Right keys [1]: [s_store_sk#26] +Right keys [1]: [s_store_sk#25] Join condition: None (34) Project [codegen id : 8] -Output [5]: [sales#18, coalesce(returns#34, 0.00) AS returns#37, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#38, store channel AS channel#39, s_store_sk#9 AS id#40] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#26, returns#34, profit_loss#35] +Output [5]: [sales#18, coalesce(returns#33, 0.00) AS returns#36, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#37, store channel AS channel#38, s_store_sk#9 AS id#39] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] +Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] +Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#45] +Output [1]: [d_date_sk#6] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#44] -Right keys [1]: [d_date_sk#45] +Left keys [1]: [cs_sold_date_sk#43] +Right keys [1]: [d_date_sk#6] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] -Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] +Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] +Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] -Keys [1]: [cs_call_center_sk#41] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] -Aggregate Attributes [2]: [sum#46, sum#47] -Results [3]: [cs_call_center_sk#41, sum#48, sum#49] +Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] +Keys [1]: [cs_call_center_sk#40] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] +Aggregate Attributes [2]: [sum#44, sum#45] +Results [3]: [cs_call_center_sk#40, sum#46, sum#47] (41) Exchange -Input [3]: [cs_call_center_sk#41, sum#48, sum#49] -Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [3]: [cs_call_center_sk#40, sum#46, sum#47] +Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#41, sum#48, sum#49] -Keys [1]: [cs_call_center_sk#41] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] -Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] +Input [3]: [cs_call_center_sk#40, sum#46, sum#47] +Keys [1]: [cs_call_center_sk#40] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] +Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] (43) BroadcastExchange -Input [3]: [cs_call_center_sk#41, sales#53, profit#54] -Arguments: IdentityBroadcastMode, [id=#55] +Input [3]: [cs_call_center_sk#40, sales#51, profit#52] +Arguments: IdentityBroadcastMode, [id=#53] (44) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] +Output [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#58), dynamicpruningexpression(cr_returned_date_sk#58 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] ReadSchema: struct (45) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] +Input [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] (46) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#59] +Output [1]: [d_date_sk#6] (47) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#58] -Right keys [1]: [d_date_sk#59] +Left keys [1]: [cr_returned_date_sk#56] +Right keys [1]: [d_date_sk#6] Join condition: None (48) Project [codegen id : 13] -Output [2]: [cr_return_amount#56, cr_net_loss#57] -Input [4]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58, d_date_sk#59] +Output [2]: [cr_return_amount#54, cr_net_loss#55] +Input [4]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56, d_date_sk#6] (49) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#56, cr_net_loss#57] +Input [2]: [cr_return_amount#54, cr_net_loss#55] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#56)), partial_sum(UnscaledValue(cr_net_loss#57))] -Aggregate Attributes [2]: [sum#60, sum#61] -Results [2]: [sum#62, sum#63] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#54)), partial_sum(UnscaledValue(cr_net_loss#55))] +Aggregate Attributes [2]: [sum#57, sum#58] +Results [2]: [sum#59, sum#60] (50) Exchange -Input [2]: [sum#62, sum#63] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#64] +Input [2]: [sum#59, sum#60] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] (51) HashAggregate -Input [2]: [sum#62, sum#63] +Input [2]: [sum#59, sum#60] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#56)), sum(UnscaledValue(cr_net_loss#57))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#56))#65, sum(UnscaledValue(cr_net_loss#57))#66] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#56))#65,17,2) AS returns#67, MakeDecimal(sum(UnscaledValue(cr_net_loss#57))#66,17,2) AS profit_loss#68] +Functions [2]: [sum(UnscaledValue(cr_return_amount#54)), sum(UnscaledValue(cr_net_loss#55))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#54))#62, sum(UnscaledValue(cr_net_loss#55))#63] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#54))#62,17,2) AS returns#64, MakeDecimal(sum(UnscaledValue(cr_net_loss#55))#63,17,2) AS profit_loss#65] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [sales#53, returns#67, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#68 as decimal(18,2)))), DecimalType(18,2), true) AS profit#69, catalog channel AS channel#70, cs_call_center_sk#41 AS id#71] -Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#67, profit_loss#68] +Output [5]: [sales#51, returns#64, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#65 as decimal(18,2)))), DecimalType(18,2), true) AS profit#66, catalog channel AS channel#67, cs_call_center_sk#40 AS id#68] +Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#64, profit_loss#65] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] -Condition : isnotnull(ws_web_page_sk#72) +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Condition : isnotnull(ws_web_page_sk#69) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#76] +Output [1]: [d_date_sk#6] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#75] -Right keys [1]: [d_date_sk#76] +Left keys [1]: [ws_sold_date_sk#72] +Right keys [1]: [d_date_sk#6] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] -Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] +Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] +Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#77] +Output [1]: [wp_web_page_sk#73] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#77] +Input [1]: [wp_web_page_sk#73] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#77] -Condition : isnotnull(wp_web_page_sk#77) +Input [1]: [wp_web_page_sk#73] +Condition : isnotnull(wp_web_page_sk#73) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#77] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] +Input [1]: [wp_web_page_sk#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#72] -Right keys [1]: [wp_web_page_sk#77] +Left keys [1]: [ws_web_page_sk#69] +Right keys [1]: [wp_web_page_sk#73] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] -Aggregate Attributes [2]: [sum#79, sum#80] -Results [3]: [wp_web_page_sk#77, sum#81, sum#82] +Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Keys [1]: [wp_web_page_sk#73] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] +Aggregate Attributes [2]: [sum#75, sum#76] +Results [3]: [wp_web_page_sk#73, sum#77, sum#78] (67) Exchange -Input [3]: [wp_web_page_sk#77, sum#81, sum#82] -Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [3]: [wp_web_page_sk#73, sum#77, sum#78] +Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#77, sum#81, sum#82] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] -Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] +Input [3]: [wp_web_page_sk#73, sum#77, sum#78] +Keys [1]: [wp_web_page_sk#73] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] +Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] -Condition : isnotnull(wr_web_page_sk#88) +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Condition : isnotnull(wr_web_page_sk#84) (72) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#92] +Output [1]: [d_date_sk#6] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#91] -Right keys [1]: [cast(d_date_sk#92 as bigint)] +Left keys [1]: [wr_returned_date_sk#87] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90] -Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, d_date_sk#92] +Output [3]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86] +Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, d_date_sk#6] (75) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#93] +Output [1]: [wp_web_page_sk#88] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#88] -Right keys [1]: [cast(wp_web_page_sk#93 as bigint)] +Left keys [1]: [wr_web_page_sk#84] +Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] +Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] -Keys [1]: [wp_web_page_sk#93] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] -Aggregate Attributes [2]: [sum#94, sum#95] -Results [3]: [wp_web_page_sk#93, sum#96, sum#97] +Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Keys [1]: [wp_web_page_sk#88] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] +Aggregate Attributes [2]: [sum#89, sum#90] +Results [3]: [wp_web_page_sk#88, sum#91, sum#92] (79) Exchange -Input [3]: [wp_web_page_sk#93, sum#96, sum#97] -Arguments: hashpartitioning(wp_web_page_sk#93, 5), ENSURE_REQUIREMENTS, [id=#98] +Input [3]: [wp_web_page_sk#88, sum#91, sum#92] +Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#93, sum#96, sum#97] -Keys [1]: [wp_web_page_sk#93] -Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] -Results [3]: [wp_web_page_sk#93, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] +Input [3]: [wp_web_page_sk#88, sum#91, sum#92] +Keys [1]: [wp_web_page_sk#88] +Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] +Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#93, returns#101, profit_loss#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] +Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#77] -Right keys [1]: [wp_web_page_sk#93] +Left keys [1]: [wp_web_page_sk#73] +Right keys [1]: [wp_web_page_sk#88] Join condition: None (83) Project [codegen id : 22] -Output [5]: [sales#86, coalesce(returns#101, 0.00) AS returns#104, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#105, web channel AS channel#106, wp_web_page_sk#77 AS id#107] -Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#93, returns#101, profit_loss#102] +Output [5]: [sales#82, coalesce(returns#96, 0.00) AS returns#99, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#100, web channel AS channel#101, wp_web_page_sk#73 AS id#102] +Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] (84) Union (85) Expand [codegen id : 23] -Input [5]: [sales#18, returns#37, profit#38, channel#39, id#40] -Arguments: [List(sales#18, returns#37, profit#38, channel#39, id#40, 0), List(sales#18, returns#37, profit#38, channel#39, null, 1), List(sales#18, returns#37, profit#38, null, null, 3)], [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] +Input [5]: [sales#18, returns#36, profit#37, channel#38, id#39] +Arguments: [List(sales#18, returns#36, profit#37, channel#38, id#39, 0), List(sales#18, returns#36, profit#37, channel#38, null, 1), List(sales#18, returns#36, profit#37, null, null, 3)], [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] (86) HashAggregate [codegen id : 23] -Input [6]: [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] -Keys [3]: [channel#108, id#109, spark_grouping_id#110] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#37), partial_sum(profit#38)] -Aggregate Attributes [6]: [sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] -Results [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Input [6]: [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] +Keys [3]: [channel#103, id#104, spark_grouping_id#105] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#36), partial_sum(profit#37)] +Aggregate Attributes [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] +Results [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] (87) Exchange -Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Arguments: hashpartitioning(channel#108, id#109, spark_grouping_id#110, 5), ENSURE_REQUIREMENTS, [id=#123] +Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Arguments: hashpartitioning(channel#103, id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, [id=#118] (88) HashAggregate [codegen id : 24] -Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] -Keys [3]: [channel#108, id#109, spark_grouping_id#110] -Functions [3]: [sum(sales#18), sum(returns#37), sum(profit#38)] -Aggregate Attributes [3]: [sum(sales#18)#124, sum(returns#37)#125, sum(profit#38)#126] -Results [5]: [channel#108, id#109, sum(sales#18)#124 AS sales#127, sum(returns#37)#125 AS returns#128, sum(profit#38)#126 AS profit#129] +Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Keys [3]: [channel#103, id#104, spark_grouping_id#105] +Functions [3]: [sum(sales#18), sum(returns#36), sum(profit#37)] +Aggregate Attributes [3]: [sum(sales#18)#119, sum(returns#36)#120, sum(profit#37)#121] +Results [5]: [channel#103, id#104, sum(sales#18)#119 AS sales#122, sum(returns#36)#120 AS returns#123, sum(profit#37)#121 AS profit#124] (89) TakeOrderedAndProject -Input [5]: [channel#108, id#109, sales#127, returns#128, profit#129] -Arguments: 100, [channel#108 ASC NULLS FIRST, id#109 ASC NULLS FIRST], [channel#108, id#109, sales#127, returns#128, profit#129] +Input [5]: [channel#103, id#104, sales#122, returns#123, profit#124] +Arguments: 100, [channel#103 ASC NULLS FIRST, id#104 ASC NULLS FIRST], [channel#103, id#104, sales#122, returns#123, profit#124] ===== Subqueries ===== @@ -526,14 +526,14 @@ ReusedExchange (91) (91) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#6] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#58 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt index b74247d7aef3f..ec22a1048c116 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt @@ -248,164 +248,164 @@ Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale Input [9]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#14, d_year#15] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#14] Join condition: None (41) Project [codegen id : 13] -Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] -Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#43, d_year#44] +Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] +Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#14, d_year#15] (42) HashAggregate [codegen id : 13] -Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] -Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] +Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [partial_sum(cs_quantity#34), partial_sum(UnscaledValue(cs_wholesale_cost#35)), partial_sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum#45, sum#46, sum#47] -Results [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] +Aggregate Attributes [3]: [sum#43, sum#44, sum#45] +Results [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] (43) Exchange -Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] -Arguments: hashpartitioning(d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] +Arguments: hashpartitioning(d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#49] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] -Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] +Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [sum(cs_quantity#34), sum(UnscaledValue(cs_wholesale_cost#35)), sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum(cs_quantity#34)#52, sum(UnscaledValue(cs_wholesale_cost#35))#53, sum(UnscaledValue(cs_sales_price#36))#54] -Results [6]: [d_year#44 AS cs_sold_year#55, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#56, sum(cs_quantity#34)#52 AS cs_qty#57, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#53,17,2) AS cs_wc#58, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#54,17,2) AS cs_sp#59] +Aggregate Attributes [3]: [sum(cs_quantity#34)#50, sum(UnscaledValue(cs_wholesale_cost#35))#51, sum(UnscaledValue(cs_sales_price#36))#52] +Results [6]: [d_year#15 AS cs_sold_year#53, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#54, sum(cs_quantity#34)#50 AS cs_qty#55, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#51,17,2) AS cs_wc#56, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#52,17,2) AS cs_sp#57] (45) Filter [codegen id : 14] -Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] -Condition : (coalesce(cs_qty#57, 0) > 0) +Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] +Condition : (coalesce(cs_qty#55, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] -Arguments: [cs_sold_year#55 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#56 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] +Arguments: [cs_sold_year#53 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#54 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56] +Right keys [3]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] (49) Scan parquet default.web_sales -Output [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Output [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] (51) Filter [codegen id : 16] -Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] -Condition : (isnotnull(ws_item_sk#60) AND isnotnull(ws_bill_customer_sk#61)) +Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Condition : (isnotnull(ws_item_sk#58) AND isnotnull(ws_bill_customer_sk#59)) (52) Exchange -Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] -Arguments: hashpartitioning(cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint), 5), ENSURE_REQUIREMENTS, [id=#67] +Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Arguments: hashpartitioning(cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint), 5), ENSURE_REQUIREMENTS, [id=#65] (53) Sort [codegen id : 17] -Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] -Arguments: [cast(ws_order_number#62 as bigint) ASC NULLS FIRST, cast(ws_item_sk#60 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Arguments: [cast(ws_order_number#60 as bigint) ASC NULLS FIRST, cast(ws_item_sk#58 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.web_returns -Output [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Output [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] (56) Filter [codegen id : 18] -Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] -Condition : (isnotnull(wr_order_number#69) AND isnotnull(wr_item_sk#68)) +Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Condition : (isnotnull(wr_order_number#67) AND isnotnull(wr_item_sk#66)) (57) Project [codegen id : 18] -Output [2]: [wr_item_sk#68, wr_order_number#69] -Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Output [2]: [wr_item_sk#66, wr_order_number#67] +Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] (58) Exchange -Input [2]: [wr_item_sk#68, wr_order_number#69] -Arguments: hashpartitioning(wr_order_number#69, wr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [2]: [wr_item_sk#66, wr_order_number#67] +Arguments: hashpartitioning(wr_order_number#67, wr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] (59) Sort [codegen id : 19] -Input [2]: [wr_item_sk#68, wr_order_number#69] -Arguments: [wr_order_number#69 ASC NULLS FIRST, wr_item_sk#68 ASC NULLS FIRST], false, 0 +Input [2]: [wr_item_sk#66, wr_order_number#67] +Arguments: [wr_order_number#67 ASC NULLS FIRST, wr_item_sk#66 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint)] -Right keys [2]: [wr_order_number#69, wr_item_sk#68] +Left keys [2]: [cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint)] +Right keys [2]: [wr_order_number#67, wr_item_sk#66] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] -Condition : isnull(wr_order_number#69) +Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] +Condition : isnull(wr_order_number#67) (62) Project [codegen id : 21] -Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] -Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] +Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#72, d_year#73] +Output [2]: [d_date_sk#14, d_year#15] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [ws_sold_date_sk#66] -Right keys [1]: [d_date_sk#72] +Left keys [1]: [ws_sold_date_sk#64] +Right keys [1]: [d_date_sk#14] Join condition: None (65) Project [codegen id : 21] -Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] -Input [8]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, d_date_sk#72, d_year#73] +Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] +Input [8]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, d_date_sk#14, d_year#15] (66) HashAggregate [codegen id : 21] -Input [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] -Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] -Functions [3]: [partial_sum(ws_quantity#63), partial_sum(UnscaledValue(ws_wholesale_cost#64)), partial_sum(UnscaledValue(ws_sales_price#65))] -Aggregate Attributes [3]: [sum#74, sum#75, sum#76] -Results [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] +Input [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] +Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] +Functions [3]: [partial_sum(ws_quantity#61), partial_sum(UnscaledValue(ws_wholesale_cost#62)), partial_sum(UnscaledValue(ws_sales_price#63))] +Aggregate Attributes [3]: [sum#70, sum#71, sum#72] +Results [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] (67) Exchange -Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] -Arguments: hashpartitioning(d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] +Arguments: hashpartitioning(d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#76] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] -Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] -Functions [3]: [sum(ws_quantity#63), sum(UnscaledValue(ws_wholesale_cost#64)), sum(UnscaledValue(ws_sales_price#65))] -Aggregate Attributes [3]: [sum(ws_quantity#63)#81, sum(UnscaledValue(ws_wholesale_cost#64))#82, sum(UnscaledValue(ws_sales_price#65))#83] -Results [6]: [d_year#73 AS ws_sold_year#84, ws_item_sk#60, ws_bill_customer_sk#61 AS ws_customer_sk#85, sum(ws_quantity#63)#81 AS ws_qty#86, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#64))#82,17,2) AS ws_wc#87, MakeDecimal(sum(UnscaledValue(ws_sales_price#65))#83,17,2) AS ws_sp#88] +Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] +Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] +Functions [3]: [sum(ws_quantity#61), sum(UnscaledValue(ws_wholesale_cost#62)), sum(UnscaledValue(ws_sales_price#63))] +Aggregate Attributes [3]: [sum(ws_quantity#61)#77, sum(UnscaledValue(ws_wholesale_cost#62))#78, sum(UnscaledValue(ws_sales_price#63))#79] +Results [6]: [d_year#15 AS ws_sold_year#80, ws_item_sk#58, ws_bill_customer_sk#59 AS ws_customer_sk#81, sum(ws_quantity#61)#77 AS ws_qty#82, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#62))#78,17,2) AS ws_wc#83, MakeDecimal(sum(UnscaledValue(ws_sales_price#63))#79,17,2) AS ws_sp#84] (69) Filter [codegen id : 22] -Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] -Condition : (coalesce(ws_qty#86, 0) > 0) +Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] +Condition : (coalesce(ws_qty#82, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] -Arguments: [ws_sold_year#84 ASC NULLS FIRST, ws_item_sk#60 ASC NULLS FIRST, ws_customer_sk#85 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] +Arguments: [ws_sold_year#80 ASC NULLS FIRST, ws_item_sk#58 ASC NULLS FIRST, ws_customer_sk#81 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85] +Right keys [3]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81] Join condition: None (72) Project [codegen id : 23] -Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#86 + cs_qty#57), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#86, 0) + coalesce(cs_qty#57, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#87, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#58, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#88, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#59, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#86, cs_qty#57] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59, ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] +Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#82 + cs_qty#55), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#82, 0) + coalesce(cs_qty#55, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#83, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#56, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#84, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#57, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#82, cs_qty#55] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57, ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] (73) TakeOrderedAndProject -Input [12]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#86, cs_qty#57] -Arguments: 100, [ratio#89 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#86 + cs_qty#57), 1) as double)), 2) ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] +Input [12]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#82, cs_qty#55] +Arguments: 100, [ratio#85 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#82 + cs_qty#55), 1) as double)), 2) ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt index c7da3a1e23c62..f5eff516f9b36 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt @@ -248,164 +248,164 @@ Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale Input [9]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_order_number#33, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, wr_item_sk#39, wr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#14, d_year#15] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#37] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#14] Join condition: None (41) Project [codegen id : 13] -Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] -Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#43, d_year#44] +Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] +Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#14, d_year#15] (42) HashAggregate [codegen id : 13] -Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] -Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] +Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [partial_sum(ws_quantity#34), partial_sum(UnscaledValue(ws_wholesale_cost#35)), partial_sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum#45, sum#46, sum#47] -Results [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] +Aggregate Attributes [3]: [sum#43, sum#44, sum#45] +Results [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] (43) Exchange -Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] -Arguments: hashpartitioning(d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] +Arguments: hashpartitioning(d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#49] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] -Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] +Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [sum(ws_quantity#34), sum(UnscaledValue(ws_wholesale_cost#35)), sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum(ws_quantity#34)#52, sum(UnscaledValue(ws_wholesale_cost#35))#53, sum(UnscaledValue(ws_sales_price#36))#54] -Results [6]: [d_year#44 AS ws_sold_year#55, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#56, sum(ws_quantity#34)#52 AS ws_qty#57, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#53,17,2) AS ws_wc#58, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#54,17,2) AS ws_sp#59] +Aggregate Attributes [3]: [sum(ws_quantity#34)#50, sum(UnscaledValue(ws_wholesale_cost#35))#51, sum(UnscaledValue(ws_sales_price#36))#52] +Results [6]: [d_year#15 AS ws_sold_year#53, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#54, sum(ws_quantity#34)#50 AS ws_qty#55, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#51,17,2) AS ws_wc#56, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#52,17,2) AS ws_sp#57] (45) Filter [codegen id : 14] -Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] -Condition : (coalesce(ws_qty#57, 0) > 0) +Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] +Condition : (coalesce(ws_qty#55, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] -Arguments: [ws_sold_year#55 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#56 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] +Arguments: [ws_sold_year#53 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#54 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56] +Right keys [3]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] (49) Scan parquet default.catalog_sales -Output [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Output [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#66), dynamicpruningexpression(cs_sold_date_sk#66 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#8)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] (51) Filter [codegen id : 16] -Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] -Condition : (isnotnull(cs_item_sk#61) AND isnotnull(cs_bill_customer_sk#60)) +Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Condition : (isnotnull(cs_item_sk#59) AND isnotnull(cs_bill_customer_sk#58)) (52) Exchange -Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] -Arguments: hashpartitioning(cs_order_number#62, cs_item_sk#61, 5), ENSURE_REQUIREMENTS, [id=#67] +Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Arguments: hashpartitioning(cs_order_number#60, cs_item_sk#59, 5), ENSURE_REQUIREMENTS, [id=#65] (53) Sort [codegen id : 17] -Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] -Arguments: [cs_order_number#62 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST], false, 0 +Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Arguments: [cs_order_number#60 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST], false, 0 (54) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Output [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] (56) Filter [codegen id : 18] -Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] -Condition : (isnotnull(cr_order_number#69) AND isnotnull(cr_item_sk#68)) +Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Condition : (isnotnull(cr_order_number#67) AND isnotnull(cr_item_sk#66)) (57) Project [codegen id : 18] -Output [2]: [cr_item_sk#68, cr_order_number#69] -Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Output [2]: [cr_item_sk#66, cr_order_number#67] +Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] (58) Exchange -Input [2]: [cr_item_sk#68, cr_order_number#69] -Arguments: hashpartitioning(cr_order_number#69, cr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [2]: [cr_item_sk#66, cr_order_number#67] +Arguments: hashpartitioning(cr_order_number#67, cr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] (59) Sort [codegen id : 19] -Input [2]: [cr_item_sk#68, cr_order_number#69] -Arguments: [cr_order_number#69 ASC NULLS FIRST, cr_item_sk#68 ASC NULLS FIRST], false, 0 +Input [2]: [cr_item_sk#66, cr_order_number#67] +Arguments: [cr_order_number#67 ASC NULLS FIRST, cr_item_sk#66 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cs_order_number#62, cs_item_sk#61] -Right keys [2]: [cr_order_number#69, cr_item_sk#68] +Left keys [2]: [cs_order_number#60, cs_item_sk#59] +Right keys [2]: [cr_order_number#67, cr_item_sk#66] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] -Condition : isnull(cr_order_number#69) +Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] +Condition : isnull(cr_order_number#67) (62) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] -Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] +Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#72, d_year#73] +Output [2]: [d_date_sk#14, d_year#15] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [cs_sold_date_sk#66] -Right keys [1]: [d_date_sk#72] +Left keys [1]: [cs_sold_date_sk#64] +Right keys [1]: [d_date_sk#14] Join condition: None (65) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] -Input [8]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, d_date_sk#72, d_year#73] +Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] +Input [8]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, d_date_sk#14, d_year#15] (66) HashAggregate [codegen id : 21] -Input [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] -Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] -Functions [3]: [partial_sum(cs_quantity#63), partial_sum(UnscaledValue(cs_wholesale_cost#64)), partial_sum(UnscaledValue(cs_sales_price#65))] -Aggregate Attributes [3]: [sum#74, sum#75, sum#76] -Results [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] +Input [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] +Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] +Functions [3]: [partial_sum(cs_quantity#61), partial_sum(UnscaledValue(cs_wholesale_cost#62)), partial_sum(UnscaledValue(cs_sales_price#63))] +Aggregate Attributes [3]: [sum#70, sum#71, sum#72] +Results [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] (67) Exchange -Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] -Arguments: hashpartitioning(d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] +Arguments: hashpartitioning(d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, 5), ENSURE_REQUIREMENTS, [id=#76] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] -Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] -Functions [3]: [sum(cs_quantity#63), sum(UnscaledValue(cs_wholesale_cost#64)), sum(UnscaledValue(cs_sales_price#65))] -Aggregate Attributes [3]: [sum(cs_quantity#63)#81, sum(UnscaledValue(cs_wholesale_cost#64))#82, sum(UnscaledValue(cs_sales_price#65))#83] -Results [6]: [d_year#73 AS cs_sold_year#84, cs_item_sk#61, cs_bill_customer_sk#60 AS cs_customer_sk#85, sum(cs_quantity#63)#81 AS cs_qty#86, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#64))#82,17,2) AS cs_wc#87, MakeDecimal(sum(UnscaledValue(cs_sales_price#65))#83,17,2) AS cs_sp#88] +Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] +Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] +Functions [3]: [sum(cs_quantity#61), sum(UnscaledValue(cs_wholesale_cost#62)), sum(UnscaledValue(cs_sales_price#63))] +Aggregate Attributes [3]: [sum(cs_quantity#61)#77, sum(UnscaledValue(cs_wholesale_cost#62))#78, sum(UnscaledValue(cs_sales_price#63))#79] +Results [6]: [d_year#15 AS cs_sold_year#80, cs_item_sk#59, cs_bill_customer_sk#58 AS cs_customer_sk#81, sum(cs_quantity#61)#77 AS cs_qty#82, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#62))#78,17,2) AS cs_wc#83, MakeDecimal(sum(UnscaledValue(cs_sales_price#63))#79,17,2) AS cs_sp#84] (69) Filter [codegen id : 22] -Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] -Condition : (coalesce(cs_qty#86, 0) > 0) +Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] +Condition : (coalesce(cs_qty#82, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] -Arguments: [cs_sold_year#84 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST, cs_customer_sk#85 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] +Arguments: [cs_sold_year#80 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST, cs_customer_sk#81 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85] +Right keys [3]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81] Join condition: None (72) Project [codegen id : 23] -Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#57 + cs_qty#86), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#57, 0) + coalesce(cs_qty#86, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#58, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#87, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#59, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#88, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, cs_qty#86] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59, cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] +Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#55 + cs_qty#82), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#55, 0) + coalesce(cs_qty#82, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#56, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#83, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#57, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#84, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, cs_qty#82] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57, cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] (73) TakeOrderedAndProject -Input [12]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, cs_qty#86] -Arguments: 100, [ratio#89 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#57 + cs_qty#86), 1) as double)), 2) ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] +Input [12]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, cs_qty#82] +Arguments: 100, [ratio#85 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#55 + cs_qty#82), 1) as double)), 2) ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt index d13b9623d1b34..48521c72281f4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt @@ -149,157 +149,157 @@ Input [1]: [ca_zip#14] Condition : (substr(ca_zip#14, 1, 5) INSET (56910,69952,63792,39371,74351,11101,25003,97189,57834,73134,62377,51200,32754,22752,86379,14171,91110,40162,98569,28709,13394,66162,25733,25782,26065,18383,51949,87343,50298,83849,33786,64528,23470,67030,46136,25280,46820,77721,99076,18426,31880,17871,98235,45748,49156,18652,72013,51622,43848,78567,41248,13695,44165,67853,54917,53179,64034,10567,71791,68908,55565,59402,64147,85816,57855,61547,27700,68100,28810,58263,15723,83933,51103,58058,90578,82276,81096,81426,96451,77556,38607,76638,18906,62971,57047,48425,35576,11928,30625,83444,73520,51650,57647,60099,30122,94983,24128,10445,41368,26233,26859,21756,24676,19849,36420,38193,58470,39127,13595,87501,24317,15455,69399,98025,81019,48033,11376,39516,67875,92712,14867,38122,29741,42961,30469,51211,56458,15559,16021,33123,33282,33515,72823,54601,76698,56240,72175,60279,20004,68806,72325,28488,43933,50412,45200,22246,78668,79777,96765,67301,73273,49448,82636,23932,47305,29839,39192,18799,61265,37125,58943,64457,88424,24610,84935,89360,68893,30431,28898,10336,90257,59166,46081,26105,96888,36634,86284,35258,39972,22927,73241,53268,24206,27385,99543,31671,14663,30903,39861,24996,63089,88086,83921,21076,67897,66708,45721,60576,25103,52867,30450,36233,30010,96576,73171,56571,56575,64544,13955,78451,43285,18119,16725,83041,76107,79994,54364,35942,56691,19769,63435,34102,18845,22744,13354,75691,45549,23968,31387,83144,13375,15765,28577,88190,19736,73650,37930,25989,83926,94898,51798,39736,22437,55253,38415,71256,18376,42029,25858,44438,19515,38935,51649,71954,15882,18767,63193,25486,49130,37126,40604,34425,17043,12305,11634,26653,94167,36446,10516,67473,66864,72425,63981,18842,22461,42666,47770,69035,70372,28587,45266,15371,15798,45375,90225,16807,31016,68014,21337,19505,50016,10144,84093,21286,19430,34322,91068,94945,72305,24671,58048,65084,28545,21195,20548,22245,77191,96976,48583,76231,15734,61810,11356,68621,68786,98359,41367,26689,69913,76614,68101,88885,50308,79077,18270,28915,29178,53672,62878,10390,14922,68341,56529,41766,68309,56616,15126,61860,97789,11489,45692,41918,72151,72550,27156,36495,70738,17879,53535,17920,68880,78890,35850,14089,58078,65164,27068,26231,13376,57665,32213,77610,87816,21309,15146,86198,91137,55307,67467,40558,94627,82136,22351,89091,20260,23006,91393,47537,62496,98294,18840,71286,81312,31029,70466,35458,14060,22685,28286,25631,19512,40081,63837,14328,35474,22152,76232,51061,86057,17183) AND isnotnull(substr(ca_zip#14, 1, 5))) (22) Scan parquet default.customer_address -Output [2]: [ca_address_sk#15, ca_zip#16] +Output [2]: [ca_address_sk#15, ca_zip#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 5] -Input [2]: [ca_address_sk#15, ca_zip#16] +Input [2]: [ca_address_sk#15, ca_zip#14] (24) Filter [codegen id : 5] -Input [2]: [ca_address_sk#15, ca_zip#16] +Input [2]: [ca_address_sk#15, ca_zip#14] Condition : isnotnull(ca_address_sk#15) (25) Exchange -Input [2]: [ca_address_sk#15, ca_zip#16] -Arguments: hashpartitioning(ca_address_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [2]: [ca_address_sk#15, ca_zip#14] +Arguments: hashpartitioning(ca_address_sk#15, 5), ENSURE_REQUIREMENTS, [id=#16] (26) Sort [codegen id : 6] -Input [2]: [ca_address_sk#15, ca_zip#16] +Input [2]: [ca_address_sk#15, ca_zip#14] Arguments: [ca_address_sk#15 ASC NULLS FIRST], false, 0 (27) Scan parquet default.customer -Output [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] +Output [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 7] -Input [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] +Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] (29) Filter [codegen id : 7] -Input [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] -Condition : ((isnotnull(c_preferred_cust_flag#19) AND (c_preferred_cust_flag#19 = Y)) AND isnotnull(c_current_addr_sk#18)) +Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] +Condition : ((isnotnull(c_preferred_cust_flag#18) AND (c_preferred_cust_flag#18 = Y)) AND isnotnull(c_current_addr_sk#17)) (30) Project [codegen id : 7] -Output [1]: [c_current_addr_sk#18] -Input [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] +Output [1]: [c_current_addr_sk#17] +Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] (31) Exchange -Input [1]: [c_current_addr_sk#18] -Arguments: hashpartitioning(c_current_addr_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [c_current_addr_sk#17] +Arguments: hashpartitioning(c_current_addr_sk#17, 5), ENSURE_REQUIREMENTS, [id=#19] (32) Sort [codegen id : 8] -Input [1]: [c_current_addr_sk#18] -Arguments: [c_current_addr_sk#18 ASC NULLS FIRST], false, 0 +Input [1]: [c_current_addr_sk#17] +Arguments: [c_current_addr_sk#17 ASC NULLS FIRST], false, 0 (33) SortMergeJoin [codegen id : 9] Left keys [1]: [ca_address_sk#15] -Right keys [1]: [c_current_addr_sk#18] +Right keys [1]: [c_current_addr_sk#17] Join condition: None (34) Project [codegen id : 9] -Output [1]: [ca_zip#16] -Input [3]: [ca_address_sk#15, ca_zip#16, c_current_addr_sk#18] +Output [1]: [ca_zip#14] +Input [3]: [ca_address_sk#15, ca_zip#14, c_current_addr_sk#17] (35) HashAggregate [codegen id : 9] -Input [1]: [ca_zip#16] -Keys [1]: [ca_zip#16] +Input [1]: [ca_zip#14] +Keys [1]: [ca_zip#14] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#21] -Results [2]: [ca_zip#16, count#22] +Aggregate Attributes [1]: [count#20] +Results [2]: [ca_zip#14, count#21] (36) Exchange -Input [2]: [ca_zip#16, count#22] -Arguments: hashpartitioning(ca_zip#16, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [2]: [ca_zip#14, count#21] +Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, [id=#22] (37) HashAggregate [codegen id : 10] -Input [2]: [ca_zip#16, count#22] -Keys [1]: [ca_zip#16] +Input [2]: [ca_zip#14, count#21] +Keys [1]: [ca_zip#14] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#24] -Results [2]: [substr(ca_zip#16, 1, 5) AS ca_zip#25, count(1)#24 AS count(1)#26] +Aggregate Attributes [1]: [count(1)#23] +Results [2]: [substr(ca_zip#14, 1, 5) AS ca_zip#24, count(1)#23 AS count(1)#25] (38) Filter [codegen id : 10] -Input [2]: [ca_zip#25, count(1)#26] -Condition : (count(1)#26 > 10) +Input [2]: [ca_zip#24, count(1)#25] +Condition : (count(1)#25 > 10) (39) Project [codegen id : 10] -Output [1]: [ca_zip#25] -Input [2]: [ca_zip#25, count(1)#26] +Output [1]: [ca_zip#24] +Input [2]: [ca_zip#24, count(1)#25] (40) BroadcastExchange -Input [1]: [ca_zip#25] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#27] +Input [1]: [ca_zip#24] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#26] (41) BroadcastHashJoin [codegen id : 11] Left keys [2]: [coalesce(substr(ca_zip#14, 1, 5), ), isnull(substr(ca_zip#14, 1, 5))] -Right keys [2]: [coalesce(ca_zip#25, ), isnull(ca_zip#25)] +Right keys [2]: [coalesce(ca_zip#24, ), isnull(ca_zip#24)] Join condition: None (42) Project [codegen id : 11] -Output [1]: [substr(ca_zip#14, 1, 5) AS ca_zip#28] +Output [1]: [substr(ca_zip#14, 1, 5) AS ca_zip#27] Input [1]: [ca_zip#14] (43) HashAggregate [codegen id : 11] -Input [1]: [ca_zip#28] -Keys [1]: [ca_zip#28] +Input [1]: [ca_zip#27] +Keys [1]: [ca_zip#27] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#28] +Results [1]: [ca_zip#27] (44) Exchange -Input [1]: [ca_zip#28] -Arguments: hashpartitioning(ca_zip#28, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [1]: [ca_zip#27] +Arguments: hashpartitioning(ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#28] (45) HashAggregate [codegen id : 12] -Input [1]: [ca_zip#28] -Keys [1]: [ca_zip#28] +Input [1]: [ca_zip#27] +Keys [1]: [ca_zip#27] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#28] +Results [1]: [ca_zip#27] (46) Exchange -Input [1]: [ca_zip#28] -Arguments: hashpartitioning(substr(ca_zip#28, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#30] +Input [1]: [ca_zip#27] +Arguments: hashpartitioning(substr(ca_zip#27, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#29] (47) Sort [codegen id : 13] -Input [1]: [ca_zip#28] -Arguments: [substr(ca_zip#28, 1, 2) ASC NULLS FIRST], false, 0 +Input [1]: [ca_zip#27] +Arguments: [substr(ca_zip#27, 1, 2) ASC NULLS FIRST], false, 0 (48) SortMergeJoin [codegen id : 14] Left keys [1]: [substr(s_zip#11, 1, 2)] -Right keys [1]: [substr(ca_zip#28, 1, 2)] +Right keys [1]: [substr(ca_zip#27, 1, 2)] Join condition: None (49) Project [codegen id : 14] Output [2]: [ss_net_profit#2, s_store_name#10] -Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#28] +Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#27] (50) HashAggregate [codegen id : 14] Input [2]: [ss_net_profit#2, s_store_name#10] Keys [1]: [s_store_name#10] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#31] -Results [2]: [s_store_name#10, sum#32] +Aggregate Attributes [1]: [sum#30] +Results [2]: [s_store_name#10, sum#31] (51) Exchange -Input [2]: [s_store_name#10, sum#32] -Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [2]: [s_store_name#10, sum#31] +Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#32] (52) HashAggregate [codegen id : 15] -Input [2]: [s_store_name#10, sum#32] +Input [2]: [s_store_name#10, sum#31] Keys [1]: [s_store_name#10] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#34] -Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS sum(ss_net_profit)#35] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#33] +Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#33,17,2) AS sum(ss_net_profit)#34] (53) TakeOrderedAndProject -Input [2]: [s_store_name#10, sum(ss_net_profit)#35] -Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#35] +Input [2]: [s_store_name#10, sum(ss_net_profit)#34] +Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#34] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt index dbb5e1f606a8d..09e1189003ed6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt @@ -135,141 +135,141 @@ Input [1]: [ca_zip#13] Condition : (substr(ca_zip#13, 1, 5) INSET (56910,69952,63792,39371,74351,11101,25003,97189,57834,73134,62377,51200,32754,22752,86379,14171,91110,40162,98569,28709,13394,66162,25733,25782,26065,18383,51949,87343,50298,83849,33786,64528,23470,67030,46136,25280,46820,77721,99076,18426,31880,17871,98235,45748,49156,18652,72013,51622,43848,78567,41248,13695,44165,67853,54917,53179,64034,10567,71791,68908,55565,59402,64147,85816,57855,61547,27700,68100,28810,58263,15723,83933,51103,58058,90578,82276,81096,81426,96451,77556,38607,76638,18906,62971,57047,48425,35576,11928,30625,83444,73520,51650,57647,60099,30122,94983,24128,10445,41368,26233,26859,21756,24676,19849,36420,38193,58470,39127,13595,87501,24317,15455,69399,98025,81019,48033,11376,39516,67875,92712,14867,38122,29741,42961,30469,51211,56458,15559,16021,33123,33282,33515,72823,54601,76698,56240,72175,60279,20004,68806,72325,28488,43933,50412,45200,22246,78668,79777,96765,67301,73273,49448,82636,23932,47305,29839,39192,18799,61265,37125,58943,64457,88424,24610,84935,89360,68893,30431,28898,10336,90257,59166,46081,26105,96888,36634,86284,35258,39972,22927,73241,53268,24206,27385,99543,31671,14663,30903,39861,24996,63089,88086,83921,21076,67897,66708,45721,60576,25103,52867,30450,36233,30010,96576,73171,56571,56575,64544,13955,78451,43285,18119,16725,83041,76107,79994,54364,35942,56691,19769,63435,34102,18845,22744,13354,75691,45549,23968,31387,83144,13375,15765,28577,88190,19736,73650,37930,25989,83926,94898,51798,39736,22437,55253,38415,71256,18376,42029,25858,44438,19515,38935,51649,71954,15882,18767,63193,25486,49130,37126,40604,34425,17043,12305,11634,26653,94167,36446,10516,67473,66864,72425,63981,18842,22461,42666,47770,69035,70372,28587,45266,15371,15798,45375,90225,16807,31016,68014,21337,19505,50016,10144,84093,21286,19430,34322,91068,94945,72305,24671,58048,65084,28545,21195,20548,22245,77191,96976,48583,76231,15734,61810,11356,68621,68786,98359,41367,26689,69913,76614,68101,88885,50308,79077,18270,28915,29178,53672,62878,10390,14922,68341,56529,41766,68309,56616,15126,61860,97789,11489,45692,41918,72151,72550,27156,36495,70738,17879,53535,17920,68880,78890,35850,14089,58078,65164,27068,26231,13376,57665,32213,77610,87816,21309,15146,86198,91137,55307,67467,40558,94627,82136,22351,89091,20260,23006,91393,47537,62496,98294,18840,71286,81312,31029,70466,35458,14060,22685,28286,25631,19512,40081,63837,14328,35474,22152,76232,51061,86057,17183) AND isnotnull(substr(ca_zip#13, 1, 5))) (20) Scan parquet default.customer_address -Output [2]: [ca_address_sk#14, ca_zip#15] +Output [2]: [ca_address_sk#14, ca_zip#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#14, ca_zip#15] +Input [2]: [ca_address_sk#14, ca_zip#13] (22) Filter [codegen id : 4] -Input [2]: [ca_address_sk#14, ca_zip#15] +Input [2]: [ca_address_sk#14, ca_zip#13] Condition : isnotnull(ca_address_sk#14) (23) Scan parquet default.customer -Output [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Output [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 3] -Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] (25) Filter [codegen id : 3] -Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] -Condition : ((isnotnull(c_preferred_cust_flag#17) AND (c_preferred_cust_flag#17 = Y)) AND isnotnull(c_current_addr_sk#16)) +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Condition : ((isnotnull(c_preferred_cust_flag#16) AND (c_preferred_cust_flag#16 = Y)) AND isnotnull(c_current_addr_sk#15)) (26) Project [codegen id : 3] -Output [1]: [c_current_addr_sk#16] -Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Output [1]: [c_current_addr_sk#15] +Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] (27) BroadcastExchange -Input [1]: [c_current_addr_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Input [1]: [c_current_addr_sk#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (28) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ca_address_sk#14] -Right keys [1]: [c_current_addr_sk#16] +Right keys [1]: [c_current_addr_sk#15] Join condition: None (29) Project [codegen id : 4] -Output [1]: [ca_zip#15] -Input [3]: [ca_address_sk#14, ca_zip#15, c_current_addr_sk#16] +Output [1]: [ca_zip#13] +Input [3]: [ca_address_sk#14, ca_zip#13, c_current_addr_sk#15] (30) HashAggregate [codegen id : 4] -Input [1]: [ca_zip#15] -Keys [1]: [ca_zip#15] +Input [1]: [ca_zip#13] +Keys [1]: [ca_zip#13] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#19] -Results [2]: [ca_zip#15, count#20] +Aggregate Attributes [1]: [count#18] +Results [2]: [ca_zip#13, count#19] (31) Exchange -Input [2]: [ca_zip#15, count#20] -Arguments: hashpartitioning(ca_zip#15, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [2]: [ca_zip#13, count#19] +Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, [id=#20] (32) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#15, count#20] -Keys [1]: [ca_zip#15] +Input [2]: [ca_zip#13, count#19] +Keys [1]: [ca_zip#13] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#22] -Results [2]: [substr(ca_zip#15, 1, 5) AS ca_zip#23, count(1)#22 AS count(1)#24] +Aggregate Attributes [1]: [count(1)#21] +Results [2]: [substr(ca_zip#13, 1, 5) AS ca_zip#22, count(1)#21 AS count(1)#23] (33) Filter [codegen id : 5] -Input [2]: [ca_zip#23, count(1)#24] -Condition : (count(1)#24 > 10) +Input [2]: [ca_zip#22, count(1)#23] +Condition : (count(1)#23 > 10) (34) Project [codegen id : 5] -Output [1]: [ca_zip#23] -Input [2]: [ca_zip#23, count(1)#24] +Output [1]: [ca_zip#22] +Input [2]: [ca_zip#22, count(1)#23] (35) BroadcastExchange -Input [1]: [ca_zip#23] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#25] +Input [1]: [ca_zip#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#24] (36) BroadcastHashJoin [codegen id : 6] Left keys [2]: [coalesce(substr(ca_zip#13, 1, 5), ), isnull(substr(ca_zip#13, 1, 5))] -Right keys [2]: [coalesce(ca_zip#23, ), isnull(ca_zip#23)] +Right keys [2]: [coalesce(ca_zip#22, ), isnull(ca_zip#22)] Join condition: None (37) Project [codegen id : 6] -Output [1]: [substr(ca_zip#13, 1, 5) AS ca_zip#26] +Output [1]: [substr(ca_zip#13, 1, 5) AS ca_zip#25] Input [1]: [ca_zip#13] (38) HashAggregate [codegen id : 6] -Input [1]: [ca_zip#26] -Keys [1]: [ca_zip#26] +Input [1]: [ca_zip#25] +Keys [1]: [ca_zip#25] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#26] +Results [1]: [ca_zip#25] (39) Exchange -Input [1]: [ca_zip#26] -Arguments: hashpartitioning(ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [1]: [ca_zip#25] +Arguments: hashpartitioning(ca_zip#25, 5), ENSURE_REQUIREMENTS, [id=#26] (40) HashAggregate [codegen id : 7] -Input [1]: [ca_zip#26] -Keys [1]: [ca_zip#26] +Input [1]: [ca_zip#25] +Keys [1]: [ca_zip#25] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#26] +Results [1]: [ca_zip#25] (41) BroadcastExchange -Input [1]: [ca_zip#26] -Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [id=#28] +Input [1]: [ca_zip#25] +Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [id=#27] (42) BroadcastHashJoin [codegen id : 8] Left keys [1]: [substr(s_zip#11, 1, 2)] -Right keys [1]: [substr(ca_zip#26, 1, 2)] +Right keys [1]: [substr(ca_zip#25, 1, 2)] Join condition: None (43) Project [codegen id : 8] Output [2]: [ss_net_profit#2, s_store_name#10] -Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#26] +Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#25] (44) HashAggregate [codegen id : 8] Input [2]: [ss_net_profit#2, s_store_name#10] Keys [1]: [s_store_name#10] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#29] -Results [2]: [s_store_name#10, sum#30] +Aggregate Attributes [1]: [sum#28] +Results [2]: [s_store_name#10, sum#29] (45) Exchange -Input [2]: [s_store_name#10, sum#30] -Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [2]: [s_store_name#10, sum#29] +Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#30] (46) HashAggregate [codegen id : 9] -Input [2]: [s_store_name#10, sum#30] +Input [2]: [s_store_name#10, sum#29] Keys [1]: [s_store_name#10] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#32] -Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#32,17,2) AS sum(ss_net_profit)#33] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] +Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS sum(ss_net_profit)#32] (47) TakeOrderedAndProject -Input [2]: [s_store_name#10, sum(ss_net_profit)#33] -Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#33] +Input [2]: [s_store_name#10, sum(ss_net_profit)#32] +Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#32] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt index bce6f2aa4b789..6e86260f0aba8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt @@ -367,252 +367,252 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#61] +Output [1]: [i_item_sk#16] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#61] +Right keys [1]: [i_item_sk#16] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#61] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#16] (60) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#62] +Output [1]: [p_promo_sk#19] (61) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#62] +Right keys [1]: [p_promo_sk#19] Join condition: None (62) Project [codegen id : 19] Output [6]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#62] +Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#19] (63) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#63] +Output [1]: [d_date_sk#22] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#63] +Right keys [1]: [d_date_sk#22] Join condition: None (65) Project [codegen id : 19] Output [5]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#63] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#22] (66) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 18] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] (68) Filter [codegen id : 18] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Condition : isnotnull(cp_catalog_page_sk#64) +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Condition : isnotnull(cp_catalog_page_sk#61) (69) BroadcastExchange -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#64] +Right keys [1]: [cp_catalog_page_sk#61] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#64, cp_catalog_page_id#65] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] -Keys [1]: [cp_catalog_page_id#65] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Keys [1]: [cp_catalog_page_id#62] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Results [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] (73) Exchange -Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] -Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] -Keys [1]: [cp_catalog_page_id#65] +Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Keys [1]: [cp_catalog_page_id#62] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#81, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#82, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#83, catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#65) AS id#85] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] +Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#80, catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#62) AS id#82] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) (78) Exchange -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] (84) Exchange -Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] +Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] -Right keys [2]: [wr_item_sk#94, wr_order_number#95] +Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] +Right keys [2]: [wr_item_sk#91, wr_order_number#92] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] -Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] +Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#100] +Output [1]: [i_item_sk#16] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#86] -Right keys [1]: [i_item_sk#100] +Left keys [1]: [ws_item_sk#83] +Right keys [1]: [i_item_sk#16] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] -Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, i_item_sk#100] +Output [7]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] +Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, i_item_sk#16] (91) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#101] +Output [1]: [p_promo_sk#19] (92) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#88] -Right keys [1]: [p_promo_sk#101] +Left keys [1]: [ws_promo_sk#85] +Right keys [1]: [p_promo_sk#19] Join condition: None (93) Project [codegen id : 29] -Output [6]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] -Input [8]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, p_promo_sk#101] +Output [6]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] +Input [8]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, p_promo_sk#19] (94) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#102] +Output [1]: [d_date_sk#22] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#92] -Right keys [1]: [d_date_sk#102] +Left keys [1]: [ws_sold_date_sk#89] +Right keys [1]: [d_date_sk#22] Join condition: None (96) Project [codegen id : 29] -Output [5]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] -Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#102] +Output [5]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] +Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#22] (97) Scan parquet default.web_site -Output [2]: [web_site_sk#103, web_site_id#104] +Output [2]: [web_site_sk#97, web_site_id#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (98) ColumnarToRow [codegen id : 28] -Input [2]: [web_site_sk#103, web_site_id#104] +Input [2]: [web_site_sk#97, web_site_id#98] (99) Filter [codegen id : 28] -Input [2]: [web_site_sk#103, web_site_id#104] -Condition : isnotnull(web_site_sk#103) +Input [2]: [web_site_sk#97, web_site_id#98] +Condition : isnotnull(web_site_sk#97) (100) BroadcastExchange -Input [2]: [web_site_sk#103, web_site_id#104] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#105] +Input [2]: [web_site_sk#97, web_site_id#98] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#87] -Right keys [1]: [web_site_sk#103] +Left keys [1]: [ws_web_site_sk#84] +Right keys [1]: [web_site_sk#97] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] -Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#103, web_site_id#104] +Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] -Keys [1]: [web_site_id#104] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Results [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Keys [1]: [web_site_id#98] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] (104) Exchange -Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, [id=#116] +Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Keys [1]: [web_site_id#104] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#120, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#121, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#122, web channel AS channel#123, concat(web_site, web_site_id#104) AS id#124] +Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Keys [1]: [web_site_id#98] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] +Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#114, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#115, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#116, web channel AS channel#117, concat(web_site, web_site_id#98) AS id#118] (106) Union (107) Expand [codegen id : 31] Input [5]: [sales#42, returns#43, profit#44, channel#45, id#46] -Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] +Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] (108) HashAggregate [codegen id : 31] -Input [6]: [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] -Keys [3]: [channel#125, id#126, spark_grouping_id#127] +Input [6]: [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] +Keys [3]: [channel#119, id#120, spark_grouping_id#121] Functions [3]: [partial_sum(sales#42), partial_sum(returns#43), partial_sum(profit#44)] -Aggregate Attributes [6]: [sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] -Results [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Aggregate Attributes [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] +Results [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] (109) Exchange -Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Arguments: hashpartitioning(channel#125, id#126, spark_grouping_id#127, 5), ENSURE_REQUIREMENTS, [id=#140] +Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Arguments: hashpartitioning(channel#119, id#120, spark_grouping_id#121, 5), ENSURE_REQUIREMENTS, [id=#134] (110) HashAggregate [codegen id : 32] -Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Keys [3]: [channel#125, id#126, spark_grouping_id#127] +Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Keys [3]: [channel#119, id#120, spark_grouping_id#121] Functions [3]: [sum(sales#42), sum(returns#43), sum(profit#44)] -Aggregate Attributes [3]: [sum(sales#42)#141, sum(returns#43)#142, sum(profit#44)#143] -Results [5]: [channel#125, id#126, sum(sales#42)#141 AS sales#144, sum(returns#43)#142 AS returns#145, sum(profit#44)#143 AS profit#146] +Aggregate Attributes [3]: [sum(sales#42)#135, sum(returns#43)#136, sum(profit#44)#137] +Results [5]: [channel#119, id#120, sum(sales#42)#135 AS sales#138, sum(returns#43)#136 AS returns#139, sum(profit#44)#137 AS profit#140] (111) TakeOrderedAndProject -Input [5]: [channel#125, id#126, sales#144, returns#145, profit#146] -Arguments: 100, [channel#125 ASC NULLS FIRST, id#126 ASC NULLS FIRST], [channel#125, id#126, sales#144, returns#145, profit#146] +Input [5]: [channel#119, id#120, sales#138, returns#139, profit#140] +Arguments: 100, [channel#119 ASC NULLS FIRST, id#120 ASC NULLS FIRST], [channel#119, id#120, sales#138, returns#139, profit#140] ===== Subqueries ===== @@ -625,6 +625,6 @@ Output [1]: [d_date_sk#22] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt index 57aa93b41482b..87abdc02b21a7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt @@ -367,252 +367,252 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#61] +Output [1]: [d_date_sk#16] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#61] +Right keys [1]: [d_date_sk#16] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#61] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#16] (60) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] (62) Filter [codegen id : 16] -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Condition : isnotnull(cp_catalog_page_sk#61) (63) BroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#62] +Right keys [1]: [cp_catalog_page_sk#61] Join condition: None (65) Project [codegen id : 19] -Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#62, cp_catalog_page_id#63] +Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] (66) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#65] +Output [1]: [i_item_sk#22] (67) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#65] +Right keys [1]: [i_item_sk#22] Join condition: None (68) Project [codegen id : 19] -Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] -Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, i_item_sk#65] +Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, i_item_sk#22] (69) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#66] +Output [1]: [p_promo_sk#25] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#66] +Right keys [1]: [p_promo_sk#25] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] -Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, p_promo_sk#66] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, p_promo_sk#25] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] -Keys [1]: [cp_catalog_page_id#63] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Keys [1]: [cp_catalog_page_id#62] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Results [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] (73) Exchange -Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] -Arguments: hashpartitioning(cp_catalog_page_id#63, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] -Keys [1]: [cp_catalog_page_id#63] +Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Keys [1]: [cp_catalog_page_id#62] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#81, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#82, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#83, catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#63) AS id#85] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] +Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#80, catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#62) AS id#82] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) (78) Exchange -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] (84) Exchange -Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] +Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] -Right keys [2]: [wr_item_sk#94, wr_order_number#95] +Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] +Right keys [2]: [wr_item_sk#91, wr_order_number#92] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] -Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] +Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#100] +Output [1]: [d_date_sk#16] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#92] -Right keys [1]: [d_date_sk#100] +Left keys [1]: [ws_sold_date_sk#89] +Right keys [1]: [d_date_sk#16] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] -Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#100] +Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] +Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#16] (91) Scan parquet default.web_site -Output [2]: [web_site_sk#101, web_site_id#102] +Output [2]: [web_site_sk#97, web_site_id#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 26] -Input [2]: [web_site_sk#101, web_site_id#102] +Input [2]: [web_site_sk#97, web_site_id#98] (93) Filter [codegen id : 26] -Input [2]: [web_site_sk#101, web_site_id#102] -Condition : isnotnull(web_site_sk#101) +Input [2]: [web_site_sk#97, web_site_id#98] +Condition : isnotnull(web_site_sk#97) (94) BroadcastExchange -Input [2]: [web_site_sk#101, web_site_id#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] +Input [2]: [web_site_sk#97, web_site_id#98] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#87] -Right keys [1]: [web_site_sk#101] +Left keys [1]: [ws_web_site_sk#84] +Right keys [1]: [web_site_sk#97] Join condition: None (96) Project [codegen id : 29] -Output [7]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] -Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#101, web_site_id#102] +Output [7]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] (97) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#104] +Output [1]: [i_item_sk#22] (98) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#86] -Right keys [1]: [i_item_sk#104] +Left keys [1]: [ws_item_sk#83] +Right keys [1]: [i_item_sk#22] Join condition: None (99) Project [codegen id : 29] -Output [6]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] -Input [8]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, i_item_sk#104] +Output [6]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Input [8]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, i_item_sk#22] (100) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#105] +Output [1]: [p_promo_sk#25] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#88] -Right keys [1]: [p_promo_sk#105] +Left keys [1]: [ws_promo_sk#85] +Right keys [1]: [p_promo_sk#25] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] -Input [7]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, p_promo_sk#105] +Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Input [7]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, p_promo_sk#25] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] -Keys [1]: [web_site_id#102] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Results [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Keys [1]: [web_site_id#98] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] (104) Exchange -Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Arguments: hashpartitioning(web_site_id#102, 5), ENSURE_REQUIREMENTS, [id=#116] +Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Keys [1]: [web_site_id#102] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#120, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#121, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#122, web channel AS channel#123, concat(web_site, web_site_id#102) AS id#124] +Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Keys [1]: [web_site_id#98] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] +Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#114, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#115, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#116, web channel AS channel#117, concat(web_site, web_site_id#98) AS id#118] (106) Union (107) Expand [codegen id : 31] Input [5]: [sales#42, returns#43, profit#44, channel#45, id#46] -Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] +Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] (108) HashAggregate [codegen id : 31] -Input [6]: [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] -Keys [3]: [channel#125, id#126, spark_grouping_id#127] +Input [6]: [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] +Keys [3]: [channel#119, id#120, spark_grouping_id#121] Functions [3]: [partial_sum(sales#42), partial_sum(returns#43), partial_sum(profit#44)] -Aggregate Attributes [6]: [sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] -Results [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Aggregate Attributes [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] +Results [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] (109) Exchange -Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Arguments: hashpartitioning(channel#125, id#126, spark_grouping_id#127, 5), ENSURE_REQUIREMENTS, [id=#140] +Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Arguments: hashpartitioning(channel#119, id#120, spark_grouping_id#121, 5), ENSURE_REQUIREMENTS, [id=#134] (110) HashAggregate [codegen id : 32] -Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Keys [3]: [channel#125, id#126, spark_grouping_id#127] +Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Keys [3]: [channel#119, id#120, spark_grouping_id#121] Functions [3]: [sum(sales#42), sum(returns#43), sum(profit#44)] -Aggregate Attributes [3]: [sum(sales#42)#141, sum(returns#43)#142, sum(profit#44)#143] -Results [5]: [channel#125, id#126, sum(sales#42)#141 AS sales#144, sum(returns#43)#142 AS returns#145, sum(profit#44)#143 AS profit#146] +Aggregate Attributes [3]: [sum(sales#42)#135, sum(returns#43)#136, sum(profit#44)#137] +Results [5]: [channel#119, id#120, sum(sales#42)#135 AS sales#138, sum(returns#43)#136 AS returns#139, sum(profit#44)#137 AS profit#140] (111) TakeOrderedAndProject -Input [5]: [channel#125, id#126, sales#144, returns#145, profit#146] -Arguments: 100, [channel#125 ASC NULLS FIRST, id#126 ASC NULLS FIRST], [channel#125, id#126, sales#144, returns#145, profit#146] +Input [5]: [channel#119, id#120, sales#138, returns#139, profit#140] +Arguments: 100, [channel#119 ASC NULLS FIRST, id#120 ASC NULLS FIRST], [channel#119, id#120, sales#138, returns#139, profit#140] ===== Subqueries ===== @@ -625,6 +625,6 @@ Output [1]: [d_date_sk#16] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#8 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 d2d70bef8c3ee..167e142598ae8 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 @@ -170,178 +170,178 @@ Input [3]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19] Condition : isnotnull(ctr_total_return#19) (24) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] +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#23), dynamicpruningexpression(cr_returned_date_sk#23 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 (25) ColumnarToRow [codegen id : 8] -Input [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] +Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] (26) Filter [codegen id : 8] -Input [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] -Condition : isnotnull(cr_returning_addr_sk#21) +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) (27) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#24] +Output [1]: [d_date_sk#6] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cr_returned_date_sk#23] -Right keys [1]: [d_date_sk#24] +Left keys [1]: [cr_returned_date_sk#4] +Right keys [1]: [d_date_sk#6] Join condition: None (29) Project [codegen id : 8] -Output [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] -Input [5]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23, d_date_sk#24] +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] (30) Exchange -Input [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] -Arguments: hashpartitioning(cr_returning_addr_sk#21, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] +Arguments: hashpartitioning(cr_returning_addr_sk#2, 5), ENSURE_REQUIREMENTS, [id=#20] (31) Sort [codegen id : 9] -Input [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] -Arguments: [cr_returning_addr_sk#21 ASC NULLS FIRST], false, 0 +Input [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] +Arguments: [cr_returning_addr_sk#2 ASC NULLS FIRST], false, 0 (32) ReusedExchange [Reuses operator id: 16] -Output [2]: [ca_address_sk#26, ca_state#27] +Output [2]: [ca_address_sk#10, ca_state#11] (33) Sort [codegen id : 11] -Input [2]: [ca_address_sk#26, ca_state#27] -Arguments: [ca_address_sk#26 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#10, ca_state#11] +Arguments: [ca_address_sk#10 ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 12] -Left keys [1]: [cr_returning_addr_sk#21] -Right keys [1]: [ca_address_sk#26] +Left keys [1]: [cr_returning_addr_sk#2] +Right keys [1]: [ca_address_sk#10] Join condition: None (35) Project [codegen id : 12] -Output [3]: [cr_returning_customer_sk#20, cr_return_amt_inc_tax#22, ca_state#27] -Input [5]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, ca_address_sk#26, ca_state#27] +Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#11] +Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#10, ca_state#11] (36) HashAggregate [codegen id : 12] -Input [3]: [cr_returning_customer_sk#20, cr_return_amt_inc_tax#22, ca_state#27] -Keys [2]: [cr_returning_customer_sk#20, ca_state#27] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#22))] -Aggregate Attributes [1]: [sum#28] -Results [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] +Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#11] +Keys [2]: [cr_returning_customer_sk#1, ca_state#11] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] +Aggregate Attributes [1]: [sum#21] +Results [3]: [cr_returning_customer_sk#1, ca_state#11, sum#22] (37) Exchange -Input [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] -Arguments: hashpartitioning(cr_returning_customer_sk#20, ca_state#27, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [cr_returning_customer_sk#1, ca_state#11, sum#22] +Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#11, 5), ENSURE_REQUIREMENTS, [id=#23] (38) HashAggregate [codegen id : 13] -Input [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] -Keys [2]: [cr_returning_customer_sk#20, ca_state#27] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#22))#31] -Results [2]: [ca_state#27 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#22))#31,17,2) AS ctr_total_return#19] +Input [3]: [cr_returning_customer_sk#1, ca_state#11, sum#22] +Keys [2]: [cr_returning_customer_sk#1, ca_state#11] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#24] +Results [2]: [ca_state#11 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#24,17,2) AS ctr_total_return#19] (39) HashAggregate [codegen id : 13] Input [2]: [ctr_state#18, ctr_total_return#19] Keys [1]: [ctr_state#18] Functions [1]: [partial_avg(ctr_total_return#19)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [3]: [ctr_state#18, sum#34, count#35] +Aggregate Attributes [2]: [sum#25, count#26] +Results [3]: [ctr_state#18, sum#27, count#28] (40) Exchange -Input [3]: [ctr_state#18, sum#34, count#35] -Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ctr_state#18, sum#27, count#28] +Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#29] (41) HashAggregate [codegen id : 14] -Input [3]: [ctr_state#18, sum#34, count#35] +Input [3]: [ctr_state#18, sum#27, count#28] Keys [1]: [ctr_state#18] Functions [1]: [avg(ctr_total_return#19)] -Aggregate Attributes [1]: [avg(ctr_total_return#19)#37] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#37) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#38, ctr_state#18 AS ctr_state#18#39] +Aggregate Attributes [1]: [avg(ctr_total_return#19)#30] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#18 AS ctr_state#18#32] (42) Filter [codegen id : 14] -Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#38) +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) (43) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#40] +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#33] (44) BroadcastHashJoin [codegen id : 15] Left keys [1]: [ctr_state#18] -Right keys [1]: [ctr_state#18#39] -Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#38) +Right keys [1]: [ctr_state#18#32] +Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) (45) Project [codegen id : 15] Output [2]: [ctr_customer_sk#17, ctr_total_return#19] -Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] +Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] (46) Exchange Input [2]: [ctr_customer_sk#17, ctr_total_return#19] -Arguments: hashpartitioning(ctr_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#41] +Arguments: hashpartitioning(ctr_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#34] (47) Sort [codegen id : 16] Input [2]: [ctr_customer_sk#17, ctr_total_return#19] Arguments: [ctr_customer_sk#17 ASC NULLS FIRST], false, 0 (48) Scan parquet default.customer -Output [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] +Output [6]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#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_current_addr_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 18] -Input [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] +Input [6]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40] (50) Filter [codegen id : 18] -Input [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] -Condition : (isnotnull(c_customer_sk#42) AND isnotnull(c_current_addr_sk#44)) +Input [6]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40] +Condition : (isnotnull(c_customer_sk#35) AND isnotnull(c_current_addr_sk#37)) (51) Scan parquet default.customer_address -Output [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Output [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] 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 (52) ColumnarToRow [codegen id : 17] -Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Input [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] (53) Filter [codegen id : 17] -Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] -Condition : ((isnotnull(ca_state#55) AND (ca_state#55 = GA)) AND isnotnull(ca_address_sk#48)) +Input [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Condition : ((isnotnull(ca_state#11) AND (ca_state#11 = GA)) AND isnotnull(ca_address_sk#10)) (54) BroadcastExchange -Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#60] +Input [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] (55) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#44] -Right keys [1]: [ca_address_sk#48] +Left keys [1]: [c_current_addr_sk#37] +Right keys [1]: [ca_address_sk#10] Join condition: None (56) Project [codegen id : 18] -Output [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] -Input [18]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47, ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Output [16]: [c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Input [18]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40, ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] (57) Exchange -Input [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] -Arguments: hashpartitioning(c_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [16]: [c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Arguments: hashpartitioning(c_customer_sk#35, 5), ENSURE_REQUIREMENTS, [id=#52] (58) Sort [codegen id : 19] -Input [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] -Arguments: [c_customer_sk#42 ASC NULLS FIRST], false, 0 +Input [16]: [c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Arguments: [c_customer_sk#35 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 20] Left keys [1]: [ctr_customer_sk#17] -Right keys [1]: [c_customer_sk#42] +Right keys [1]: [c_customer_sk#35] Join condition: None (60) Project [codegen id : 20] -Output [16]: [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] -Input [18]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Output [16]: [c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#19] +Input [18]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] (61) TakeOrderedAndProject -Input [16]: [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] -Arguments: 100, [c_customer_id#43 ASC NULLS FIRST, c_salutation#45 ASC NULLS FIRST, c_first_name#46 ASC NULLS FIRST, c_last_name#47 ASC NULLS FIRST, ca_street_number#49 ASC NULLS FIRST, ca_street_name#50 ASC NULLS FIRST, ca_street_type#51 ASC NULLS FIRST, ca_suite_number#52 ASC NULLS FIRST, ca_city#53 ASC NULLS FIRST, ca_county#54 ASC NULLS FIRST, ca_state#55 ASC NULLS FIRST, ca_zip#56 ASC NULLS FIRST, ca_country#57 ASC NULLS FIRST, ca_gmt_offset#58 ASC NULLS FIRST, ca_location_type#59 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] +Input [16]: [c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#19] +Arguments: 100, [c_customer_id#36 ASC NULLS FIRST, c_salutation#38 ASC NULLS FIRST, c_first_name#39 ASC NULLS FIRST, c_last_name#40 ASC NULLS FIRST, ca_street_number#41 ASC NULLS FIRST, ca_street_name#42 ASC NULLS FIRST, ca_street_type#43 ASC NULLS FIRST, ca_suite_number#44 ASC NULLS FIRST, ca_city#45 ASC NULLS FIRST, ca_county#46 ASC NULLS FIRST, ca_state#11 ASC NULLS FIRST, ca_zip#47 ASC NULLS FIRST, ca_country#48 ASC NULLS FIRST, ca_gmt_offset#49 ASC NULLS FIRST, ca_location_type#50 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#19] ===== Subqueries ===== @@ -352,6 +352,6 @@ ReusedExchange (62) (62) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#23 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 24 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/q81/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt index 04371a7f43d2a..6a8f9e5264d49 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 @@ -149,154 +149,154 @@ Input [3]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18] Condition : isnotnull(ctr_total_return#18) (21) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22] +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#22), dynamicpruningexpression(cr_returned_date_sk#22 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 (22) ColumnarToRow [codegen id : 6] -Input [4]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22] +Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] (23) Filter [codegen id : 6] -Input [4]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22] -Condition : isnotnull(cr_returning_addr_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] +Condition : isnotnull(cr_returning_addr_sk#2) (24) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#23] +Output [1]: [d_date_sk#6] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#22] -Right keys [1]: [d_date_sk#23] +Left keys [1]: [cr_returned_date_sk#4] +Right keys [1]: [d_date_sk#6] Join condition: None (26) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21] -Input [5]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22, d_date_sk#23] +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] (27) ReusedExchange [Reuses operator id: 14] -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#9, ca_state#10] (28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#20] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [cr_returning_addr_sk#2] +Right keys [1]: [ca_address_sk#9] Join condition: None (29) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#19, cr_return_amt_inc_tax#21, ca_state#25] -Input [5]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, ca_address_sk#24, ca_state#25] +Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] +Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#9, ca_state#10] (30) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#19, cr_return_amt_inc_tax#21, ca_state#25] -Keys [2]: [cr_returning_customer_sk#19, ca_state#25] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#21))] -Aggregate Attributes [1]: [sum#26] -Results [3]: [cr_returning_customer_sk#19, ca_state#25, sum#27] +Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] +Keys [2]: [cr_returning_customer_sk#1, ca_state#10] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] +Aggregate Attributes [1]: [sum#19] +Results [3]: [cr_returning_customer_sk#1, ca_state#10, sum#20] (31) Exchange -Input [3]: [cr_returning_customer_sk#19, ca_state#25, sum#27] -Arguments: hashpartitioning(cr_returning_customer_sk#19, ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#20] +Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#21] (32) HashAggregate [codegen id : 7] -Input [3]: [cr_returning_customer_sk#19, ca_state#25, sum#27] -Keys [2]: [cr_returning_customer_sk#19, ca_state#25] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#21))#29] -Results [2]: [ca_state#25 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#21))#29,17,2) AS ctr_total_return#18] +Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#20] +Keys [2]: [cr_returning_customer_sk#1, ca_state#10] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#22] +Results [2]: [ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#22,17,2) AS ctr_total_return#18] (33) HashAggregate [codegen id : 7] Input [2]: [ctr_state#17, ctr_total_return#18] Keys [1]: [ctr_state#17] Functions [1]: [partial_avg(ctr_total_return#18)] -Aggregate Attributes [2]: [sum#30, count#31] -Results [3]: [ctr_state#17, sum#32, count#33] +Aggregate Attributes [2]: [sum#23, count#24] +Results [3]: [ctr_state#17, sum#25, count#26] (34) Exchange -Input [3]: [ctr_state#17, sum#32, count#33] -Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [ctr_state#17, sum#25, count#26] +Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#27] (35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#17, sum#32, count#33] +Input [3]: [ctr_state#17, sum#25, count#26] Keys [1]: [ctr_state#17] Functions [1]: [avg(ctr_total_return#18)] -Aggregate Attributes [1]: [avg(ctr_total_return#18)#35] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#35) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#36, ctr_state#17 AS ctr_state#17#37] +Aggregate Attributes [1]: [avg(ctr_total_return#18)#28] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#28) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#29, ctr_state#17 AS ctr_state#17#30] (36) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#36) +Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) (37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#38] +Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#31] (38) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#17] -Right keys [1]: [ctr_state#17#37] -Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#36) +Right keys [1]: [ctr_state#17#30] +Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) (39) Project [codegen id : 11] Output [2]: [ctr_customer_sk#16, ctr_total_return#18] -Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] +Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] (40) Scan parquet default.customer -Output [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] +Output [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] +Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] (42) Filter [codegen id : 9] -Input [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] -Condition : (isnotnull(c_customer_sk#39) AND isnotnull(c_current_addr_sk#41)) +Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34)) (43) BroadcastExchange -Input [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] +Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] (44) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#16] -Right keys [1]: [c_customer_sk#39] +Right keys [1]: [c_customer_sk#32] Join condition: None (45) Project [codegen id : 11] -Output [6]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] -Input [8]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] +Output [6]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +Input [8]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] (46) Scan parquet default.customer_address -Output [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] +Output [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] 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 (47) ColumnarToRow [codegen id : 10] -Input [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] +Input [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] (48) Filter [codegen id : 10] -Input [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] -Condition : ((isnotnull(ca_state#53) AND (ca_state#53 = GA)) AND isnotnull(ca_address_sk#46)) +Input [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] +Condition : ((isnotnull(ca_state#10) AND (ca_state#10 = GA)) AND isnotnull(ca_address_sk#9)) (49) BroadcastExchange -Input [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] +Input [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#41] -Right keys [1]: [ca_address_sk#46] +Left keys [1]: [c_current_addr_sk#34] +Right keys [1]: [ca_address_sk#9] Join condition: None (51) Project [codegen id : 11] -Output [16]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57, ctr_total_return#18] -Input [18]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] +Output [16]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48, ctr_total_return#18] +Input [18]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] (52) TakeOrderedAndProject -Input [16]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57, ctr_total_return#18] -Arguments: 100, [c_customer_id#40 ASC NULLS FIRST, c_salutation#42 ASC NULLS FIRST, c_first_name#43 ASC NULLS FIRST, c_last_name#44 ASC NULLS FIRST, ca_street_number#47 ASC NULLS FIRST, ca_street_name#48 ASC NULLS FIRST, ca_street_type#49 ASC NULLS FIRST, ca_suite_number#50 ASC NULLS FIRST, ca_city#51 ASC NULLS FIRST, ca_county#52 ASC NULLS FIRST, ca_state#53 ASC NULLS FIRST, ca_zip#54 ASC NULLS FIRST, ca_country#55 ASC NULLS FIRST, ca_gmt_offset#56 ASC NULLS FIRST, ca_location_type#57 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57, ctr_total_return#18] +Input [16]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48, ctr_total_return#18] +Arguments: 100, [c_customer_id#33 ASC NULLS FIRST, c_salutation#35 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, c_last_name#37 ASC NULLS FIRST, ca_street_number#39 ASC NULLS FIRST, ca_street_name#40 ASC NULLS FIRST, ca_street_type#41 ASC NULLS FIRST, ca_suite_number#42 ASC NULLS FIRST, ca_city#43 ASC NULLS FIRST, ca_county#44 ASC NULLS FIRST, ca_state#10 ASC NULLS FIRST, ca_zip#45 ASC NULLS FIRST, ca_country#46 ASC NULLS FIRST, ca_gmt_offset#47 ASC NULLS FIRST, ca_location_type#48 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48, ctr_total_return#18] ===== Subqueries ===== @@ -307,6 +307,6 @@ ReusedExchange (53) (53) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 21 Hosting Expression = cr_returned_date_sk#22 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 21 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/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt index a7c71a0668676..d006b61d20c33 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt @@ -92,51 +92,51 @@ Input [2]: [d_date_sk#4, d_date#5] Condition : isnotnull(d_date_sk#4) (7) Scan parquet default.date_dim -Output [2]: [d_date#6, d_week_seq#7] +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (8) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#6, d_week_seq#7] +Input [2]: [d_date#5, d_week_seq#6] (9) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#5, d_week_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (10) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#5, d_week_seq#6] (11) Filter [codegen id : 1] -Input [2]: [d_date#8, d_week_seq#9] -Condition : cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) +Input [2]: [d_date#5, d_week_seq#6] +Condition : cast(d_date#5 as string) IN (2000-06-30,2000-09-27,2000-11-17) (12) Project [codegen id : 1] -Output [1]: [d_week_seq#9] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_week_seq#6 AS d_week_seq#6#7] +Input [2]: [d_date#5, d_week_seq#6] (13) BroadcastExchange -Input [1]: [d_week_seq#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] +Input [1]: [d_week_seq#6#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] (14) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_week_seq#7] -Right keys [1]: [d_week_seq#9] +Left keys [1]: [d_week_seq#6] +Right keys [1]: [d_week_seq#6#7] Join condition: None (15) Project [codegen id : 2] -Output [1]: [d_date#6] -Input [2]: [d_date#6, d_week_seq#7] +Output [1]: [d_date#5 AS d_date#5#9] +Input [2]: [d_date#5, d_week_seq#6] (16) BroadcastExchange -Input [1]: [d_date#6] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#11] +Input [1]: [d_date#5#9] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] (17) BroadcastHashJoin [codegen id : 3] Left keys [1]: [d_date#5] -Right keys [1]: [d_date#6] +Right keys [1]: [d_date#5#9] Join condition: None (18) Project [codegen id : 3] @@ -145,7 +145,7 @@ Input [2]: [d_date_sk#4, d_date#5] (19) BroadcastExchange Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (20) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] @@ -157,191 +157,191 @@ Output [2]: [sr_item_sk#1, sr_return_quantity#2] Input [4]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, d_date_sk#4] (22) Scan parquet default.item -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#12, i_item_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (23) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#13, i_item_id#14] +Input [2]: [i_item_sk#12, i_item_id#13] (24) Filter [codegen id : 4] -Input [2]: [i_item_sk#13, i_item_id#14] -Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) +Input [2]: [i_item_sk#12, i_item_id#13] +Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) (25) BroadcastExchange -Input [2]: [i_item_sk#13, i_item_id#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] +Input [2]: [i_item_sk#12, i_item_id#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] (26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_item_sk#1] -Right keys [1]: [cast(i_item_sk#13 as bigint)] +Right keys [1]: [cast(i_item_sk#12 as bigint)] Join condition: None (27) Project [codegen id : 5] -Output [2]: [sr_return_quantity#2, i_item_id#14] -Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#13, i_item_id#14] +Output [2]: [sr_return_quantity#2, i_item_id#13] +Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#12, i_item_id#13] (28) HashAggregate [codegen id : 5] -Input [2]: [sr_return_quantity#2, i_item_id#14] -Keys [1]: [i_item_id#14] +Input [2]: [sr_return_quantity#2, i_item_id#13] +Keys [1]: [i_item_id#13] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#16] -Results [2]: [i_item_id#14, sum#17] +Aggregate Attributes [1]: [sum#15] +Results [2]: [i_item_id#13, sum#16] (29) Exchange -Input [2]: [i_item_id#14, sum#17] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [2]: [i_item_id#13, sum#16] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#17] (30) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#17] -Keys [1]: [i_item_id#14] +Input [2]: [i_item_id#13, sum#16] +Keys [1]: [i_item_id#13] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] -Results [2]: [i_item_id#14 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#18] +Results [2]: [i_item_id#13 AS item_id#19, sum(sr_return_quantity#2)#18 AS sr_item_qty#20] (31) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +Output [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] (33) Filter [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] -Condition : isnotnull(cr_item_sk#22) +Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Condition : isnotnull(cr_item_sk#21) (34) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#4] (35) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#24] -Right keys [1]: [d_date_sk#25] +Left keys [1]: [cr_returned_date_sk#23] +Right keys [1]: [d_date_sk#4] Join condition: None (36) Project [codegen id : 10] -Output [2]: [cr_item_sk#22, cr_return_quantity#23] -Input [4]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, d_date_sk#25] +Output [2]: [cr_item_sk#21, cr_return_quantity#22] +Input [4]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, d_date_sk#4] (37) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#26, i_item_id#27] +Output [2]: [i_item_sk#12, i_item_id#13] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#22] -Right keys [1]: [i_item_sk#26] +Left keys [1]: [cr_item_sk#21] +Right keys [1]: [i_item_sk#12] Join condition: None (39) Project [codegen id : 10] -Output [2]: [cr_return_quantity#23, i_item_id#27] -Input [4]: [cr_item_sk#22, cr_return_quantity#23, i_item_sk#26, i_item_id#27] +Output [2]: [cr_return_quantity#22, i_item_id#13] +Input [4]: [cr_item_sk#21, cr_return_quantity#22, i_item_sk#12, i_item_id#13] (40) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#23, i_item_id#27] -Keys [1]: [i_item_id#27] -Functions [1]: [partial_sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#27, sum#29] +Input [2]: [cr_return_quantity#22, i_item_id#13] +Keys [1]: [i_item_id#13] +Functions [1]: [partial_sum(cr_return_quantity#22)] +Aggregate Attributes [1]: [sum#24] +Results [2]: [i_item_id#13, sum#25] (41) Exchange -Input [2]: [i_item_id#27, sum#29] -Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [i_item_id#13, sum#25] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#26] (42) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#27, sum#29] -Keys [1]: [i_item_id#27] -Functions [1]: [sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum(cr_return_quantity#23)#31] -Results [2]: [i_item_id#27 AS item_id#32, sum(cr_return_quantity#23)#31 AS cr_item_qty#33] +Input [2]: [i_item_id#13, sum#25] +Keys [1]: [i_item_id#13] +Functions [1]: [sum(cr_return_quantity#22)] +Aggregate Attributes [1]: [sum(cr_return_quantity#22)#27] +Results [2]: [i_item_id#13 AS item_id#28, sum(cr_return_quantity#22)#27 AS cr_item_qty#29] (43) BroadcastExchange -Input [2]: [item_id#32, cr_item_qty#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] +Input [2]: [item_id#28, cr_item_qty#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] (44) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#32] +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#28] Join condition: None (45) Project [codegen id : 18] -Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#33] -Input [4]: [item_id#20, sr_item_qty#21, item_id#32, cr_item_qty#33] +Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] +Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] (46) Scan parquet default.web_returns -Output [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +Output [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#37), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(wr_returned_date_sk#33), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] (48) Filter [codegen id : 16] -Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] -Condition : isnotnull(wr_item_sk#35) +Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +Condition : isnotnull(wr_item_sk#31) (49) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#38] +Output [1]: [d_date_sk#4] (50) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#37] -Right keys [1]: [cast(d_date_sk#38 as bigint)] +Left keys [1]: [wr_returned_date_sk#33] +Right keys [1]: [cast(d_date_sk#4 as bigint)] Join condition: None (51) Project [codegen id : 16] -Output [2]: [wr_item_sk#35, wr_return_quantity#36] -Input [4]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37, d_date_sk#38] +Output [2]: [wr_item_sk#31, wr_return_quantity#32] +Input [4]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33, d_date_sk#4] (52) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#39, i_item_id#40] +Output [2]: [i_item_sk#12, i_item_id#13] (53) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#35] -Right keys [1]: [cast(i_item_sk#39 as bigint)] +Left keys [1]: [wr_item_sk#31] +Right keys [1]: [cast(i_item_sk#12 as bigint)] Join condition: None (54) Project [codegen id : 16] -Output [2]: [wr_return_quantity#36, i_item_id#40] -Input [4]: [wr_item_sk#35, wr_return_quantity#36, i_item_sk#39, i_item_id#40] +Output [2]: [wr_return_quantity#32, i_item_id#13] +Input [4]: [wr_item_sk#31, wr_return_quantity#32, i_item_sk#12, i_item_id#13] (55) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#36, i_item_id#40] -Keys [1]: [i_item_id#40] -Functions [1]: [partial_sum(wr_return_quantity#36)] -Aggregate Attributes [1]: [sum#41] -Results [2]: [i_item_id#40, sum#42] +Input [2]: [wr_return_quantity#32, i_item_id#13] +Keys [1]: [i_item_id#13] +Functions [1]: [partial_sum(wr_return_quantity#32)] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#13, sum#35] (56) Exchange -Input [2]: [i_item_id#40, sum#42] -Arguments: hashpartitioning(i_item_id#40, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [2]: [i_item_id#13, sum#35] +Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#36] (57) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#40, sum#42] -Keys [1]: [i_item_id#40] -Functions [1]: [sum(wr_return_quantity#36)] -Aggregate Attributes [1]: [sum(wr_return_quantity#36)#44] -Results [2]: [i_item_id#40 AS item_id#45, sum(wr_return_quantity#36)#44 AS wr_item_qty#46] +Input [2]: [i_item_id#13, sum#35] +Keys [1]: [i_item_id#13] +Functions [1]: [sum(wr_return_quantity#32)] +Aggregate Attributes [1]: [sum(wr_return_quantity#32)#37] +Results [2]: [i_item_id#13 AS item_id#38, sum(wr_return_quantity#32)#37 AS wr_item_qty#39] (58) BroadcastExchange -Input [2]: [item_id#45, wr_item_qty#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#47] +Input [2]: [item_id#38, wr_item_qty#39] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] (59) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#45] +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#38] Join condition: None (60) Project [codegen id : 18] -Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS sr_dev#48, cr_item_qty#33, (((cast(cr_item_qty#33 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS cr_dev#49, wr_item_qty#46, (((cast(wr_item_qty#46 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS wr_dev#50, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#51] -Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#33, item_id#45, wr_item_qty#46] +Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] +Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] (61) TakeOrderedAndProject -Input [8]: [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] -Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] +Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index fd73cee998f71..c926ab7e3ccb6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -119,51 +119,51 @@ Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) (13) Scan parquet default.date_dim -Output [2]: [d_date#9, d_week_seq#10] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] -Input [2]: [d_date#9, d_week_seq#10] +Input [2]: [d_date#8, d_week_seq#9] (15) Scan parquet default.date_dim -Output [2]: [d_date#11, d_week_seq#12] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (16) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#11, d_week_seq#12] +Input [2]: [d_date#8, d_week_seq#9] (17) Filter [codegen id : 2] -Input [2]: [d_date#11, d_week_seq#12] -Condition : cast(d_date#11 as string) IN (2000-06-30,2000-09-27,2000-11-17) +Input [2]: [d_date#8, d_week_seq#9] +Condition : cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) (18) Project [codegen id : 2] -Output [1]: [d_week_seq#12] -Input [2]: [d_date#11, d_week_seq#12] +Output [1]: [d_week_seq#9 AS d_week_seq#9#10] +Input [2]: [d_date#8, d_week_seq#9] (19) BroadcastExchange -Input [1]: [d_week_seq#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] +Input [1]: [d_week_seq#9#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#12] +Left keys [1]: [d_week_seq#9] +Right keys [1]: [d_week_seq#9#10] Join condition: None (21) Project [codegen id : 3] -Output [1]: [d_date#9] -Input [2]: [d_date#9, d_week_seq#10] +Output [1]: [d_date#8 AS d_date#8#12] +Input [2]: [d_date#8, d_week_seq#9] (22) BroadcastExchange -Input [1]: [d_date#9] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#14] +Input [1]: [d_date#8#12] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] (23) BroadcastHashJoin [codegen id : 4] Left keys [1]: [d_date#8] -Right keys [1]: [d_date#9] +Right keys [1]: [d_date#8#12] Join condition: None (24) Project [codegen id : 4] @@ -172,7 +172,7 @@ Input [2]: [d_date_sk#7, d_date#8] (25) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] @@ -187,161 +187,161 @@ Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#5, d_date_sk# Input [2]: [sr_return_quantity#2, i_item_id#5] Keys [1]: [i_item_id#5] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#16] -Results [2]: [i_item_id#5, sum#17] +Aggregate Attributes [1]: [sum#15] +Results [2]: [i_item_id#5, sum#16] (29) Exchange -Input [2]: [i_item_id#5, sum#17] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [2]: [i_item_id#5, sum#16] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#17] (30) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#5, sum#17] +Input [2]: [i_item_id#5, sum#16] Keys [1]: [i_item_id#5] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] -Results [2]: [i_item_id#5 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#18] +Results [2]: [i_item_id#5 AS item_id#19, sum(sr_return_quantity#2)#18 AS sr_item_qty#20] (31) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +Output [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] (33) Filter [codegen id : 10] -Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] -Condition : isnotnull(cr_item_sk#22) +Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Condition : isnotnull(cr_item_sk#21) (34) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#25, i_item_id#26] +Output [2]: [i_item_sk#4, i_item_id#5] (35) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#22] -Right keys [1]: [i_item_sk#25] +Left keys [1]: [cr_item_sk#21] +Right keys [1]: [i_item_sk#4] Join condition: None (36) Project [codegen id : 10] -Output [3]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#26] -Input [5]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, i_item_sk#25, i_item_id#26] +Output [3]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#5] +Input [5]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, i_item_sk#4, i_item_id#5] (37) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#27] +Output [1]: [d_date_sk#7] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#24] -Right keys [1]: [d_date_sk#27] +Left keys [1]: [cr_returned_date_sk#23] +Right keys [1]: [d_date_sk#7] Join condition: None (39) Project [codegen id : 10] -Output [2]: [cr_return_quantity#23, i_item_id#26] -Input [4]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#26, d_date_sk#27] +Output [2]: [cr_return_quantity#22, i_item_id#5] +Input [4]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#5, d_date_sk#7] (40) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#23, i_item_id#26] -Keys [1]: [i_item_id#26] -Functions [1]: [partial_sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#26, sum#29] +Input [2]: [cr_return_quantity#22, i_item_id#5] +Keys [1]: [i_item_id#5] +Functions [1]: [partial_sum(cr_return_quantity#22)] +Aggregate Attributes [1]: [sum#24] +Results [2]: [i_item_id#5, sum#25] (41) Exchange -Input [2]: [i_item_id#26, sum#29] -Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [i_item_id#5, sum#25] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#26] (42) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#26, sum#29] -Keys [1]: [i_item_id#26] -Functions [1]: [sum(cr_return_quantity#23)] -Aggregate Attributes [1]: [sum(cr_return_quantity#23)#31] -Results [2]: [i_item_id#26 AS item_id#32, sum(cr_return_quantity#23)#31 AS cr_item_qty#33] +Input [2]: [i_item_id#5, sum#25] +Keys [1]: [i_item_id#5] +Functions [1]: [sum(cr_return_quantity#22)] +Aggregate Attributes [1]: [sum(cr_return_quantity#22)#27] +Results [2]: [i_item_id#5 AS item_id#28, sum(cr_return_quantity#22)#27 AS cr_item_qty#29] (43) BroadcastExchange -Input [2]: [item_id#32, cr_item_qty#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] +Input [2]: [item_id#28, cr_item_qty#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] (44) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#32] +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#28] Join condition: None (45) Project [codegen id : 18] -Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#33] -Input [4]: [item_id#20, sr_item_qty#21, item_id#32, cr_item_qty#33] +Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] +Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] (46) Scan parquet default.web_returns -Output [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +Output [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#37), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(wr_returned_date_sk#33), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] (48) Filter [codegen id : 16] -Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] -Condition : isnotnull(wr_item_sk#35) +Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +Condition : isnotnull(wr_item_sk#31) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#38, i_item_id#39] +Output [2]: [i_item_sk#4, i_item_id#5] (50) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#35] -Right keys [1]: [cast(i_item_sk#38 as bigint)] +Left keys [1]: [wr_item_sk#31] +Right keys [1]: [cast(i_item_sk#4 as bigint)] Join condition: None (51) Project [codegen id : 16] -Output [3]: [wr_return_quantity#36, wr_returned_date_sk#37, i_item_id#39] -Input [5]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37, i_item_sk#38, i_item_id#39] +Output [3]: [wr_return_quantity#32, wr_returned_date_sk#33, i_item_id#5] +Input [5]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33, i_item_sk#4, i_item_id#5] (52) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#40] +Output [1]: [d_date_sk#7] (53) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#37] -Right keys [1]: [cast(d_date_sk#40 as bigint)] +Left keys [1]: [wr_returned_date_sk#33] +Right keys [1]: [cast(d_date_sk#7 as bigint)] Join condition: None (54) Project [codegen id : 16] -Output [2]: [wr_return_quantity#36, i_item_id#39] -Input [4]: [wr_return_quantity#36, wr_returned_date_sk#37, i_item_id#39, d_date_sk#40] +Output [2]: [wr_return_quantity#32, i_item_id#5] +Input [4]: [wr_return_quantity#32, wr_returned_date_sk#33, i_item_id#5, d_date_sk#7] (55) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#36, i_item_id#39] -Keys [1]: [i_item_id#39] -Functions [1]: [partial_sum(wr_return_quantity#36)] -Aggregate Attributes [1]: [sum#41] -Results [2]: [i_item_id#39, sum#42] +Input [2]: [wr_return_quantity#32, i_item_id#5] +Keys [1]: [i_item_id#5] +Functions [1]: [partial_sum(wr_return_quantity#32)] +Aggregate Attributes [1]: [sum#34] +Results [2]: [i_item_id#5, sum#35] (56) Exchange -Input [2]: [i_item_id#39, sum#42] -Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [2]: [i_item_id#5, sum#35] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#36] (57) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#39, sum#42] -Keys [1]: [i_item_id#39] -Functions [1]: [sum(wr_return_quantity#36)] -Aggregate Attributes [1]: [sum(wr_return_quantity#36)#44] -Results [2]: [i_item_id#39 AS item_id#45, sum(wr_return_quantity#36)#44 AS wr_item_qty#46] +Input [2]: [i_item_id#5, sum#35] +Keys [1]: [i_item_id#5] +Functions [1]: [sum(wr_return_quantity#32)] +Aggregate Attributes [1]: [sum(wr_return_quantity#32)#37] +Results [2]: [i_item_id#5 AS item_id#38, sum(wr_return_quantity#32)#37 AS wr_item_qty#39] (58) BroadcastExchange -Input [2]: [item_id#45, wr_item_qty#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#47] +Input [2]: [item_id#38, wr_item_qty#39] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] (59) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#20] -Right keys [1]: [item_id#45] +Left keys [1]: [item_id#19] +Right keys [1]: [item_id#38] Join condition: None (60) Project [codegen id : 18] -Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS sr_dev#48, cr_item_qty#33, (((cast(cr_item_qty#33 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS cr_dev#49, wr_item_qty#46, (((cast(wr_item_qty#46 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS wr_dev#50, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#51] -Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#33, item_id#45, wr_item_qty#46] +Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] +Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] (61) TakeOrderedAndProject -Input [8]: [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] -Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] +Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt index e72928545d080..f9d73c29db3d8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt @@ -313,687 +313,687 @@ Aggregate Attributes [1]: [count(1)#19] Results [1]: [count(1)#19 AS h8_30_to_9#20] (29) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (31) Filter [codegen id : 8] -Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] -Condition : ((isnotnull(ss_hdemo_sk#22) AND isnotnull(ss_sold_time_sk#21)) AND isnotnull(ss_store_sk#23)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (32) Project [codegen id : 8] -Output [3]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23] -Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (33) Scan parquet default.time_dim -Output [3]: [t_time_sk#25, t_hour#26, t_minute#27] +Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 5] -Input [3]: [t_time_sk#25, t_hour#26, t_minute#27] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (35) Filter [codegen id : 5] -Input [3]: [t_time_sk#25, t_hour#26, t_minute#27] -Condition : ((((isnotnull(t_hour#26) AND isnotnull(t_minute#27)) AND (t_hour#26 = 9)) AND (t_minute#27 < 30)) AND isnotnull(t_time_sk#25)) +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 9)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) (36) Project [codegen id : 5] -Output [1]: [t_time_sk#25] -Input [3]: [t_time_sk#25, t_hour#26, t_minute#27] +Output [1]: [t_time_sk#5] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (37) BroadcastExchange -Input [1]: [t_time_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [t_time_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (38) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_time_sk#21] -Right keys [1]: [t_time_sk#25] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#5] Join condition: None (39) Project [codegen id : 8] -Output [2]: [ss_hdemo_sk#22, ss_store_sk#23] -Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, t_time_sk#25] +Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] (40) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#29] +Output [1]: [s_store_sk#9] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#23] -Right keys [1]: [s_store_sk#29] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#9] Join condition: None (42) Project [codegen id : 8] -Output [1]: [ss_hdemo_sk#22] -Input [3]: [ss_hdemo_sk#22, ss_store_sk#23, s_store_sk#29] +Output [1]: [ss_hdemo_sk#2] +Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] (43) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#30] +Output [1]: [hd_demo_sk#12] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_hdemo_sk#22] -Right keys [1]: [hd_demo_sk#30] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#12] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ss_hdemo_sk#22, hd_demo_sk#30] +Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#31] -Results [1]: [count#32] +Aggregate Attributes [1]: [count#22] +Results [1]: [count#23] (47) Exchange -Input [1]: [count#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] +Input [1]: [count#23] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] (48) HashAggregate [codegen id : 9] -Input [1]: [count#32] +Input [1]: [count#23] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#34] -Results [1]: [count(1)#34 AS h9_to_9_30#35] +Aggregate Attributes [1]: [count(1)#25] +Results [1]: [count(1)#25 AS h9_to_9_30#26] (49) BroadcastExchange -Input [1]: [h9_to_9_30#35] -Arguments: IdentityBroadcastMode, [id=#36] +Input [1]: [h9_to_9_30#26] +Arguments: IdentityBroadcastMode, [id=#27] (50) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (51) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 13] -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (53) Filter [codegen id : 13] -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (54) Project [codegen id : 13] -Output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (55) Scan parquet default.time_dim -Output [3]: [t_time_sk#41, t_hour#42, t_minute#43] +Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 10] -Input [3]: [t_time_sk#41, t_hour#42, t_minute#43] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (57) Filter [codegen id : 10] -Input [3]: [t_time_sk#41, t_hour#42, t_minute#43] -Condition : ((((isnotnull(t_hour#42) AND isnotnull(t_minute#43)) AND (t_hour#42 = 9)) AND (t_minute#43 >= 30)) AND isnotnull(t_time_sk#41)) +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 9)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5)) (58) Project [codegen id : 10] -Output [1]: [t_time_sk#41] -Input [3]: [t_time_sk#41, t_hour#42, t_minute#43] +Output [1]: [t_time_sk#5] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (59) BroadcastExchange -Input [1]: [t_time_sk#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] +Input [1]: [t_time_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (60) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_time_sk#37] -Right keys [1]: [t_time_sk#41] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#5] Join condition: None (61) Project [codegen id : 13] -Output [2]: [ss_hdemo_sk#38, ss_store_sk#39] -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, t_time_sk#41] +Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] (62) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#45] +Output [1]: [s_store_sk#9] (63) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#39] -Right keys [1]: [s_store_sk#45] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#9] Join condition: None (64) Project [codegen id : 13] -Output [1]: [ss_hdemo_sk#38] -Input [3]: [ss_hdemo_sk#38, ss_store_sk#39, s_store_sk#45] +Output [1]: [ss_hdemo_sk#2] +Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] (65) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#46] +Output [1]: [hd_demo_sk#12] (66) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_hdemo_sk#38] -Right keys [1]: [hd_demo_sk#46] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#12] Join condition: None (67) Project [codegen id : 13] Output: [] -Input [2]: [ss_hdemo_sk#38, hd_demo_sk#46] +Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] (68) HashAggregate [codegen id : 13] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#47] -Results [1]: [count#48] +Aggregate Attributes [1]: [count#29] +Results [1]: [count#30] (69) Exchange -Input [1]: [count#48] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] +Input [1]: [count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] (70) HashAggregate [codegen id : 14] -Input [1]: [count#48] +Input [1]: [count#30] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#50] -Results [1]: [count(1)#50 AS h9_30_to_10#51] +Aggregate Attributes [1]: [count(1)#32] +Results [1]: [count(1)#32 AS h9_30_to_10#33] (71) BroadcastExchange -Input [1]: [h9_30_to_10#51] -Arguments: IdentityBroadcastMode, [id=#52] +Input [1]: [h9_30_to_10#33] +Arguments: IdentityBroadcastMode, [id=#34] (72) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (73) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 18] -Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (75) Filter [codegen id : 18] -Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] -Condition : ((isnotnull(ss_hdemo_sk#54) AND isnotnull(ss_sold_time_sk#53)) AND isnotnull(ss_store_sk#55)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (76) Project [codegen id : 18] -Output [3]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55] -Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (77) Scan parquet default.time_dim -Output [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (78) ColumnarToRow [codegen id : 15] -Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (79) Filter [codegen id : 15] -Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] -Condition : ((((isnotnull(t_hour#58) AND isnotnull(t_minute#59)) AND (t_hour#58 = 10)) AND (t_minute#59 < 30)) AND isnotnull(t_time_sk#57)) +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 10)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) (80) Project [codegen id : 15] -Output [1]: [t_time_sk#57] -Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Output [1]: [t_time_sk#5] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (81) BroadcastExchange -Input [1]: [t_time_sk#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] +Input [1]: [t_time_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] (82) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_time_sk#53] -Right keys [1]: [t_time_sk#57] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#5] Join condition: None (83) Project [codegen id : 18] -Output [2]: [ss_hdemo_sk#54, ss_store_sk#55] -Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, t_time_sk#57] +Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] (84) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#61] +Output [1]: [s_store_sk#9] (85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#55] -Right keys [1]: [s_store_sk#61] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#9] Join condition: None (86) Project [codegen id : 18] -Output [1]: [ss_hdemo_sk#54] -Input [3]: [ss_hdemo_sk#54, ss_store_sk#55, s_store_sk#61] +Output [1]: [ss_hdemo_sk#2] +Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] (87) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#62] +Output [1]: [hd_demo_sk#12] (88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#54] -Right keys [1]: [hd_demo_sk#62] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#12] Join condition: None (89) Project [codegen id : 18] Output: [] -Input [2]: [ss_hdemo_sk#54, hd_demo_sk#62] +Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] (90) HashAggregate [codegen id : 18] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#63] -Results [1]: [count#64] +Aggregate Attributes [1]: [count#36] +Results [1]: [count#37] (91) Exchange -Input [1]: [count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] +Input [1]: [count#37] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#38] (92) HashAggregate [codegen id : 19] -Input [1]: [count#64] +Input [1]: [count#37] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#66] -Results [1]: [count(1)#66 AS h10_to_10_30#67] +Aggregate Attributes [1]: [count(1)#39] +Results [1]: [count(1)#39 AS h10_to_10_30#40] (93) BroadcastExchange -Input [1]: [h10_to_10_30#67] -Arguments: IdentityBroadcastMode, [id=#68] +Input [1]: [h10_to_10_30#40] +Arguments: IdentityBroadcastMode, [id=#41] (94) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (95) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (96) ColumnarToRow [codegen id : 23] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (97) Filter [codegen id : 23] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_hdemo_sk#70) AND isnotnull(ss_sold_time_sk#69)) AND isnotnull(ss_store_sk#71)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (98) Project [codegen id : 23] -Output [3]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (99) Scan parquet default.time_dim -Output [3]: [t_time_sk#73, t_hour#74, t_minute#75] +Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (100) ColumnarToRow [codegen id : 20] -Input [3]: [t_time_sk#73, t_hour#74, t_minute#75] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (101) Filter [codegen id : 20] -Input [3]: [t_time_sk#73, t_hour#74, t_minute#75] -Condition : ((((isnotnull(t_hour#74) AND isnotnull(t_minute#75)) AND (t_hour#74 = 10)) AND (t_minute#75 >= 30)) AND isnotnull(t_time_sk#73)) +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 10)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5)) (102) Project [codegen id : 20] -Output [1]: [t_time_sk#73] -Input [3]: [t_time_sk#73, t_hour#74, t_minute#75] +Output [1]: [t_time_sk#5] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (103) BroadcastExchange -Input [1]: [t_time_sk#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] +Input [1]: [t_time_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (104) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_sold_time_sk#69] -Right keys [1]: [t_time_sk#73] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#5] Join condition: None (105) Project [codegen id : 23] -Output [2]: [ss_hdemo_sk#70, ss_store_sk#71] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, t_time_sk#73] +Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] (106) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#77] +Output [1]: [s_store_sk#9] (107) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_store_sk#71] -Right keys [1]: [s_store_sk#77] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#9] Join condition: None (108) Project [codegen id : 23] -Output [1]: [ss_hdemo_sk#70] -Input [3]: [ss_hdemo_sk#70, ss_store_sk#71, s_store_sk#77] +Output [1]: [ss_hdemo_sk#2] +Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] (109) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#78] +Output [1]: [hd_demo_sk#12] (110) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_hdemo_sk#70] -Right keys [1]: [hd_demo_sk#78] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#12] Join condition: None (111) Project [codegen id : 23] Output: [] -Input [2]: [ss_hdemo_sk#70, hd_demo_sk#78] +Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] (112) HashAggregate [codegen id : 23] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#79] -Results [1]: [count#80] +Aggregate Attributes [1]: [count#43] +Results [1]: [count#44] (113) Exchange -Input [1]: [count#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +Input [1]: [count#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45] (114) HashAggregate [codegen id : 24] -Input [1]: [count#80] +Input [1]: [count#44] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#82] -Results [1]: [count(1)#82 AS h10_30_to_11#83] +Aggregate Attributes [1]: [count(1)#46] +Results [1]: [count(1)#46 AS h10_30_to_11#47] (115) BroadcastExchange -Input [1]: [h10_30_to_11#83] -Arguments: IdentityBroadcastMode, [id=#84] +Input [1]: [h10_30_to_11#47] +Arguments: IdentityBroadcastMode, [id=#48] (116) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (117) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (118) ColumnarToRow [codegen id : 28] -Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (119) Filter [codegen id : 28] -Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] -Condition : ((isnotnull(ss_hdemo_sk#86) AND isnotnull(ss_sold_time_sk#85)) AND isnotnull(ss_store_sk#87)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (120) Project [codegen id : 28] -Output [3]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87] -Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (121) Scan parquet default.time_dim -Output [3]: [t_time_sk#89, t_hour#90, t_minute#91] +Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (122) ColumnarToRow [codegen id : 25] -Input [3]: [t_time_sk#89, t_hour#90, t_minute#91] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (123) Filter [codegen id : 25] -Input [3]: [t_time_sk#89, t_hour#90, t_minute#91] -Condition : ((((isnotnull(t_hour#90) AND isnotnull(t_minute#91)) AND (t_hour#90 = 11)) AND (t_minute#91 < 30)) AND isnotnull(t_time_sk#89)) +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 11)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) (124) Project [codegen id : 25] -Output [1]: [t_time_sk#89] -Input [3]: [t_time_sk#89, t_hour#90, t_minute#91] +Output [1]: [t_time_sk#5] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (125) BroadcastExchange -Input [1]: [t_time_sk#89] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#92] +Input [1]: [t_time_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] (126) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_sold_time_sk#85] -Right keys [1]: [t_time_sk#89] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#5] Join condition: None (127) Project [codegen id : 28] -Output [2]: [ss_hdemo_sk#86, ss_store_sk#87] -Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, t_time_sk#89] +Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] (128) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#93] +Output [1]: [s_store_sk#9] (129) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_store_sk#87] -Right keys [1]: [s_store_sk#93] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#9] Join condition: None (130) Project [codegen id : 28] -Output [1]: [ss_hdemo_sk#86] -Input [3]: [ss_hdemo_sk#86, ss_store_sk#87, s_store_sk#93] +Output [1]: [ss_hdemo_sk#2] +Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] (131) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#94] +Output [1]: [hd_demo_sk#12] (132) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_hdemo_sk#86] -Right keys [1]: [hd_demo_sk#94] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#12] Join condition: None (133) Project [codegen id : 28] Output: [] -Input [2]: [ss_hdemo_sk#86, hd_demo_sk#94] +Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] (134) HashAggregate [codegen id : 28] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#95] -Results [1]: [count#96] +Aggregate Attributes [1]: [count#50] +Results [1]: [count#51] (135) Exchange -Input [1]: [count#96] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#97] +Input [1]: [count#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] (136) HashAggregate [codegen id : 29] -Input [1]: [count#96] +Input [1]: [count#51] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#98] -Results [1]: [count(1)#98 AS h11_to_11_30#99] +Aggregate Attributes [1]: [count(1)#53] +Results [1]: [count(1)#53 AS h11_to_11_30#54] (137) BroadcastExchange -Input [1]: [h11_to_11_30#99] -Arguments: IdentityBroadcastMode, [id=#100] +Input [1]: [h11_to_11_30#54] +Arguments: IdentityBroadcastMode, [id=#55] (138) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (139) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (140) ColumnarToRow [codegen id : 33] -Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (141) Filter [codegen id : 33] -Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] -Condition : ((isnotnull(ss_hdemo_sk#102) AND isnotnull(ss_sold_time_sk#101)) AND isnotnull(ss_store_sk#103)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (142) Project [codegen id : 33] -Output [3]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103] -Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (143) Scan parquet default.time_dim -Output [3]: [t_time_sk#105, t_hour#106, t_minute#107] +Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (144) ColumnarToRow [codegen id : 30] -Input [3]: [t_time_sk#105, t_hour#106, t_minute#107] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (145) Filter [codegen id : 30] -Input [3]: [t_time_sk#105, t_hour#106, t_minute#107] -Condition : ((((isnotnull(t_hour#106) AND isnotnull(t_minute#107)) AND (t_hour#106 = 11)) AND (t_minute#107 >= 30)) AND isnotnull(t_time_sk#105)) +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 11)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5)) (146) Project [codegen id : 30] -Output [1]: [t_time_sk#105] -Input [3]: [t_time_sk#105, t_hour#106, t_minute#107] +Output [1]: [t_time_sk#5] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (147) BroadcastExchange -Input [1]: [t_time_sk#105] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#108] +Input [1]: [t_time_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] (148) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_time_sk#101] -Right keys [1]: [t_time_sk#105] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#5] Join condition: None (149) Project [codegen id : 33] -Output [2]: [ss_hdemo_sk#102, ss_store_sk#103] -Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, t_time_sk#105] +Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] (150) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#109] +Output [1]: [s_store_sk#9] (151) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#103] -Right keys [1]: [s_store_sk#109] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#9] Join condition: None (152) Project [codegen id : 33] -Output [1]: [ss_hdemo_sk#102] -Input [3]: [ss_hdemo_sk#102, ss_store_sk#103, s_store_sk#109] +Output [1]: [ss_hdemo_sk#2] +Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] (153) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#110] +Output [1]: [hd_demo_sk#12] (154) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#102] -Right keys [1]: [hd_demo_sk#110] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#12] Join condition: None (155) Project [codegen id : 33] Output: [] -Input [2]: [ss_hdemo_sk#102, hd_demo_sk#110] +Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] (156) HashAggregate [codegen id : 33] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#111] -Results [1]: [count#112] +Aggregate Attributes [1]: [count#57] +Results [1]: [count#58] (157) Exchange -Input [1]: [count#112] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] +Input [1]: [count#58] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#59] (158) HashAggregate [codegen id : 34] -Input [1]: [count#112] +Input [1]: [count#58] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#114] -Results [1]: [count(1)#114 AS h11_30_to_12#115] +Aggregate Attributes [1]: [count(1)#60] +Results [1]: [count(1)#60 AS h11_30_to_12#61] (159) BroadcastExchange -Input [1]: [h11_30_to_12#115] -Arguments: IdentityBroadcastMode, [id=#116] +Input [1]: [h11_30_to_12#61] +Arguments: IdentityBroadcastMode, [id=#62] (160) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (161) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (162) ColumnarToRow [codegen id : 38] -Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (163) Filter [codegen id : 38] -Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] -Condition : ((isnotnull(ss_hdemo_sk#118) AND isnotnull(ss_sold_time_sk#117)) AND isnotnull(ss_store_sk#119)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (164) Project [codegen id : 38] -Output [3]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119] -Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (165) Scan parquet default.time_dim -Output [3]: [t_time_sk#121, t_hour#122, t_minute#123] +Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (166) ColumnarToRow [codegen id : 35] -Input [3]: [t_time_sk#121, t_hour#122, t_minute#123] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (167) Filter [codegen id : 35] -Input [3]: [t_time_sk#121, t_hour#122, t_minute#123] -Condition : ((((isnotnull(t_hour#122) AND isnotnull(t_minute#123)) AND (t_hour#122 = 12)) AND (t_minute#123 < 30)) AND isnotnull(t_time_sk#121)) +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 12)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) (168) Project [codegen id : 35] -Output [1]: [t_time_sk#121] -Input [3]: [t_time_sk#121, t_hour#122, t_minute#123] +Output [1]: [t_time_sk#5] +Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] (169) BroadcastExchange -Input [1]: [t_time_sk#121] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#124] +Input [1]: [t_time_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] (170) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_sold_time_sk#117] -Right keys [1]: [t_time_sk#121] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#5] Join condition: None (171) Project [codegen id : 38] -Output [2]: [ss_hdemo_sk#118, ss_store_sk#119] -Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, t_time_sk#121] +Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] (172) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#125] +Output [1]: [s_store_sk#9] (173) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_store_sk#119] -Right keys [1]: [s_store_sk#125] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#9] Join condition: None (174) Project [codegen id : 38] -Output [1]: [ss_hdemo_sk#118] -Input [3]: [ss_hdemo_sk#118, ss_store_sk#119, s_store_sk#125] +Output [1]: [ss_hdemo_sk#2] +Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] (175) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#126] +Output [1]: [hd_demo_sk#12] (176) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_hdemo_sk#118] -Right keys [1]: [hd_demo_sk#126] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#12] Join condition: None (177) Project [codegen id : 38] Output: [] -Input [2]: [ss_hdemo_sk#118, hd_demo_sk#126] +Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] (178) HashAggregate [codegen id : 38] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#127] -Results [1]: [count#128] +Aggregate Attributes [1]: [count#64] +Results [1]: [count#65] (179) Exchange -Input [1]: [count#128] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] +Input [1]: [count#65] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] (180) HashAggregate [codegen id : 39] -Input [1]: [count#128] +Input [1]: [count#65] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#130] -Results [1]: [count(1)#130 AS h12_to_12_30#131] +Aggregate Attributes [1]: [count(1)#67] +Results [1]: [count(1)#67 AS h12_to_12_30#68] (181) BroadcastExchange -Input [1]: [h12_to_12_30#131] -Arguments: IdentityBroadcastMode, [id=#132] +Input [1]: [h12_to_12_30#68] +Arguments: IdentityBroadcastMode, [id=#69] (182) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt index 9f56c71154a66..453906c2b5f34 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt @@ -313,687 +313,687 @@ Aggregate Attributes [1]: [count(1)#19] Results [1]: [count(1)#19 AS h8_30_to_9#20] (29) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (31) Filter [codegen id : 8] -Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] -Condition : ((isnotnull(ss_hdemo_sk#22) AND isnotnull(ss_sold_time_sk#21)) AND isnotnull(ss_store_sk#23)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (32) Project [codegen id : 8] -Output [3]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23] -Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (33) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#25] +Output [1]: [hd_demo_sk#5] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_hdemo_sk#22] -Right keys [1]: [hd_demo_sk#25] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] Join condition: None (35) Project [codegen id : 8] -Output [2]: [ss_sold_time_sk#21, ss_store_sk#23] -Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, hd_demo_sk#25] +Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] (36) Scan parquet default.time_dim -Output [3]: [t_time_sk#26, t_hour#27, t_minute#28] +Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [3]: [t_time_sk#26, t_hour#27, t_minute#28] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (38) Filter [codegen id : 6] -Input [3]: [t_time_sk#26, t_hour#27, t_minute#28] -Condition : ((((isnotnull(t_hour#27) AND isnotnull(t_minute#28)) AND (t_hour#27 = 9)) AND (t_minute#28 < 30)) AND isnotnull(t_time_sk#26)) +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 9)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) (39) Project [codegen id : 6] -Output [1]: [t_time_sk#26] -Input [3]: [t_time_sk#26, t_hour#27, t_minute#28] +Output [1]: [t_time_sk#9] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (40) BroadcastExchange -Input [1]: [t_time_sk#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] +Input [1]: [t_time_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_time_sk#21] -Right keys [1]: [t_time_sk#26] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#9] Join condition: None (42) Project [codegen id : 8] -Output [1]: [ss_store_sk#23] -Input [3]: [ss_sold_time_sk#21, ss_store_sk#23, t_time_sk#26] +Output [1]: [ss_store_sk#3] +Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] (43) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#30] +Output [1]: [s_store_sk#13] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#23] -Right keys [1]: [s_store_sk#30] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#13] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ss_store_sk#23, s_store_sk#30] +Input [2]: [ss_store_sk#3, s_store_sk#13] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#31] -Results [1]: [count#32] +Aggregate Attributes [1]: [count#22] +Results [1]: [count#23] (47) Exchange -Input [1]: [count#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] +Input [1]: [count#23] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] (48) HashAggregate [codegen id : 9] -Input [1]: [count#32] +Input [1]: [count#23] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#34] -Results [1]: [count(1)#34 AS h9_to_9_30#35] +Aggregate Attributes [1]: [count(1)#25] +Results [1]: [count(1)#25 AS h9_to_9_30#26] (49) BroadcastExchange -Input [1]: [h9_to_9_30#35] -Arguments: IdentityBroadcastMode, [id=#36] +Input [1]: [h9_to_9_30#26] +Arguments: IdentityBroadcastMode, [id=#27] (50) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (51) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 13] -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (53) Filter [codegen id : 13] -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] -Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (54) Project [codegen id : 13] -Output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (55) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#41] +Output [1]: [hd_demo_sk#5] (56) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_hdemo_sk#38] -Right keys [1]: [hd_demo_sk#41] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] Join condition: None (57) Project [codegen id : 13] -Output [2]: [ss_sold_time_sk#37, ss_store_sk#39] -Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] +Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] (58) Scan parquet default.time_dim -Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] +Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 11] -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (60) Filter [codegen id : 11] -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] -Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 9)) AND (t_minute#44 >= 30)) AND isnotnull(t_time_sk#42)) +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 9)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9)) (61) Project [codegen id : 11] -Output [1]: [t_time_sk#42] -Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] +Output [1]: [t_time_sk#9] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (62) BroadcastExchange -Input [1]: [t_time_sk#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#45] +Input [1]: [t_time_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (63) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_time_sk#37] -Right keys [1]: [t_time_sk#42] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#9] Join condition: None (64) Project [codegen id : 13] -Output [1]: [ss_store_sk#39] -Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] +Output [1]: [ss_store_sk#3] +Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] (65) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#46] +Output [1]: [s_store_sk#13] (66) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#39] -Right keys [1]: [s_store_sk#46] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#13] Join condition: None (67) Project [codegen id : 13] Output: [] -Input [2]: [ss_store_sk#39, s_store_sk#46] +Input [2]: [ss_store_sk#3, s_store_sk#13] (68) HashAggregate [codegen id : 13] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#47] -Results [1]: [count#48] +Aggregate Attributes [1]: [count#29] +Results [1]: [count#30] (69) Exchange -Input [1]: [count#48] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] +Input [1]: [count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] (70) HashAggregate [codegen id : 14] -Input [1]: [count#48] +Input [1]: [count#30] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#50] -Results [1]: [count(1)#50 AS h9_30_to_10#51] +Aggregate Attributes [1]: [count(1)#32] +Results [1]: [count(1)#32 AS h9_30_to_10#33] (71) BroadcastExchange -Input [1]: [h9_30_to_10#51] -Arguments: IdentityBroadcastMode, [id=#52] +Input [1]: [h9_30_to_10#33] +Arguments: IdentityBroadcastMode, [id=#34] (72) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (73) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 18] -Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (75) Filter [codegen id : 18] -Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] -Condition : ((isnotnull(ss_hdemo_sk#54) AND isnotnull(ss_sold_time_sk#53)) AND isnotnull(ss_store_sk#55)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (76) Project [codegen id : 18] -Output [3]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55] -Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (77) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#57] +Output [1]: [hd_demo_sk#5] (78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#54] -Right keys [1]: [hd_demo_sk#57] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] Join condition: None (79) Project [codegen id : 18] -Output [2]: [ss_sold_time_sk#53, ss_store_sk#55] -Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, hd_demo_sk#57] +Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] (80) Scan parquet default.time_dim -Output [3]: [t_time_sk#58, t_hour#59, t_minute#60] +Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (81) ColumnarToRow [codegen id : 16] -Input [3]: [t_time_sk#58, t_hour#59, t_minute#60] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (82) Filter [codegen id : 16] -Input [3]: [t_time_sk#58, t_hour#59, t_minute#60] -Condition : ((((isnotnull(t_hour#59) AND isnotnull(t_minute#60)) AND (t_hour#59 = 10)) AND (t_minute#60 < 30)) AND isnotnull(t_time_sk#58)) +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 10)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) (83) Project [codegen id : 16] -Output [1]: [t_time_sk#58] -Input [3]: [t_time_sk#58, t_hour#59, t_minute#60] +Output [1]: [t_time_sk#9] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (84) BroadcastExchange -Input [1]: [t_time_sk#58] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] +Input [1]: [t_time_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] (85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_time_sk#53] -Right keys [1]: [t_time_sk#58] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#9] Join condition: None (86) Project [codegen id : 18] -Output [1]: [ss_store_sk#55] -Input [3]: [ss_sold_time_sk#53, ss_store_sk#55, t_time_sk#58] +Output [1]: [ss_store_sk#3] +Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] (87) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#62] +Output [1]: [s_store_sk#13] (88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#55] -Right keys [1]: [s_store_sk#62] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#13] Join condition: None (89) Project [codegen id : 18] Output: [] -Input [2]: [ss_store_sk#55, s_store_sk#62] +Input [2]: [ss_store_sk#3, s_store_sk#13] (90) HashAggregate [codegen id : 18] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#63] -Results [1]: [count#64] +Aggregate Attributes [1]: [count#36] +Results [1]: [count#37] (91) Exchange -Input [1]: [count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] +Input [1]: [count#37] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#38] (92) HashAggregate [codegen id : 19] -Input [1]: [count#64] +Input [1]: [count#37] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#66] -Results [1]: [count(1)#66 AS h10_to_10_30#67] +Aggregate Attributes [1]: [count(1)#39] +Results [1]: [count(1)#39 AS h10_to_10_30#40] (93) BroadcastExchange -Input [1]: [h10_to_10_30#67] -Arguments: IdentityBroadcastMode, [id=#68] +Input [1]: [h10_to_10_30#40] +Arguments: IdentityBroadcastMode, [id=#41] (94) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (95) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (96) ColumnarToRow [codegen id : 23] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (97) Filter [codegen id : 23] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] -Condition : ((isnotnull(ss_hdemo_sk#70) AND isnotnull(ss_sold_time_sk#69)) AND isnotnull(ss_store_sk#71)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (98) Project [codegen id : 23] -Output [3]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (99) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#73] +Output [1]: [hd_demo_sk#5] (100) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_hdemo_sk#70] -Right keys [1]: [hd_demo_sk#73] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] Join condition: None (101) Project [codegen id : 23] -Output [2]: [ss_sold_time_sk#69, ss_store_sk#71] -Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, hd_demo_sk#73] +Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] (102) Scan parquet default.time_dim -Output [3]: [t_time_sk#74, t_hour#75, t_minute#76] +Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (103) ColumnarToRow [codegen id : 21] -Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (104) Filter [codegen id : 21] -Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] -Condition : ((((isnotnull(t_hour#75) AND isnotnull(t_minute#76)) AND (t_hour#75 = 10)) AND (t_minute#76 >= 30)) AND isnotnull(t_time_sk#74)) +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 10)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9)) (105) Project [codegen id : 21] -Output [1]: [t_time_sk#74] -Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] +Output [1]: [t_time_sk#9] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (106) BroadcastExchange -Input [1]: [t_time_sk#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] +Input [1]: [t_time_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (107) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_sold_time_sk#69] -Right keys [1]: [t_time_sk#74] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#9] Join condition: None (108) Project [codegen id : 23] -Output [1]: [ss_store_sk#71] -Input [3]: [ss_sold_time_sk#69, ss_store_sk#71, t_time_sk#74] +Output [1]: [ss_store_sk#3] +Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] (109) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#78] +Output [1]: [s_store_sk#13] (110) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_store_sk#71] -Right keys [1]: [s_store_sk#78] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#13] Join condition: None (111) Project [codegen id : 23] Output: [] -Input [2]: [ss_store_sk#71, s_store_sk#78] +Input [2]: [ss_store_sk#3, s_store_sk#13] (112) HashAggregate [codegen id : 23] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#79] -Results [1]: [count#80] +Aggregate Attributes [1]: [count#43] +Results [1]: [count#44] (113) Exchange -Input [1]: [count#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +Input [1]: [count#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45] (114) HashAggregate [codegen id : 24] -Input [1]: [count#80] +Input [1]: [count#44] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#82] -Results [1]: [count(1)#82 AS h10_30_to_11#83] +Aggregate Attributes [1]: [count(1)#46] +Results [1]: [count(1)#46 AS h10_30_to_11#47] (115) BroadcastExchange -Input [1]: [h10_30_to_11#83] -Arguments: IdentityBroadcastMode, [id=#84] +Input [1]: [h10_30_to_11#47] +Arguments: IdentityBroadcastMode, [id=#48] (116) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (117) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (118) ColumnarToRow [codegen id : 28] -Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (119) Filter [codegen id : 28] -Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] -Condition : ((isnotnull(ss_hdemo_sk#86) AND isnotnull(ss_sold_time_sk#85)) AND isnotnull(ss_store_sk#87)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (120) Project [codegen id : 28] -Output [3]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87] -Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (121) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#89] +Output [1]: [hd_demo_sk#5] (122) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_hdemo_sk#86] -Right keys [1]: [hd_demo_sk#89] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] Join condition: None (123) Project [codegen id : 28] -Output [2]: [ss_sold_time_sk#85, ss_store_sk#87] -Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, hd_demo_sk#89] +Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] (124) Scan parquet default.time_dim -Output [3]: [t_time_sk#90, t_hour#91, t_minute#92] +Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (125) ColumnarToRow [codegen id : 26] -Input [3]: [t_time_sk#90, t_hour#91, t_minute#92] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (126) Filter [codegen id : 26] -Input [3]: [t_time_sk#90, t_hour#91, t_minute#92] -Condition : ((((isnotnull(t_hour#91) AND isnotnull(t_minute#92)) AND (t_hour#91 = 11)) AND (t_minute#92 < 30)) AND isnotnull(t_time_sk#90)) +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 11)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) (127) Project [codegen id : 26] -Output [1]: [t_time_sk#90] -Input [3]: [t_time_sk#90, t_hour#91, t_minute#92] +Output [1]: [t_time_sk#9] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (128) BroadcastExchange -Input [1]: [t_time_sk#90] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] +Input [1]: [t_time_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] (129) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_sold_time_sk#85] -Right keys [1]: [t_time_sk#90] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#9] Join condition: None (130) Project [codegen id : 28] -Output [1]: [ss_store_sk#87] -Input [3]: [ss_sold_time_sk#85, ss_store_sk#87, t_time_sk#90] +Output [1]: [ss_store_sk#3] +Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] (131) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#94] +Output [1]: [s_store_sk#13] (132) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_store_sk#87] -Right keys [1]: [s_store_sk#94] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#13] Join condition: None (133) Project [codegen id : 28] Output: [] -Input [2]: [ss_store_sk#87, s_store_sk#94] +Input [2]: [ss_store_sk#3, s_store_sk#13] (134) HashAggregate [codegen id : 28] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#95] -Results [1]: [count#96] +Aggregate Attributes [1]: [count#50] +Results [1]: [count#51] (135) Exchange -Input [1]: [count#96] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#97] +Input [1]: [count#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] (136) HashAggregate [codegen id : 29] -Input [1]: [count#96] +Input [1]: [count#51] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#98] -Results [1]: [count(1)#98 AS h11_to_11_30#99] +Aggregate Attributes [1]: [count(1)#53] +Results [1]: [count(1)#53 AS h11_to_11_30#54] (137) BroadcastExchange -Input [1]: [h11_to_11_30#99] -Arguments: IdentityBroadcastMode, [id=#100] +Input [1]: [h11_to_11_30#54] +Arguments: IdentityBroadcastMode, [id=#55] (138) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (139) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (140) ColumnarToRow [codegen id : 33] -Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (141) Filter [codegen id : 33] -Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] -Condition : ((isnotnull(ss_hdemo_sk#102) AND isnotnull(ss_sold_time_sk#101)) AND isnotnull(ss_store_sk#103)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (142) Project [codegen id : 33] -Output [3]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103] -Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (143) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#105] +Output [1]: [hd_demo_sk#5] (144) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#102] -Right keys [1]: [hd_demo_sk#105] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] Join condition: None (145) Project [codegen id : 33] -Output [2]: [ss_sold_time_sk#101, ss_store_sk#103] -Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, hd_demo_sk#105] +Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] (146) Scan parquet default.time_dim -Output [3]: [t_time_sk#106, t_hour#107, t_minute#108] +Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (147) ColumnarToRow [codegen id : 31] -Input [3]: [t_time_sk#106, t_hour#107, t_minute#108] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (148) Filter [codegen id : 31] -Input [3]: [t_time_sk#106, t_hour#107, t_minute#108] -Condition : ((((isnotnull(t_hour#107) AND isnotnull(t_minute#108)) AND (t_hour#107 = 11)) AND (t_minute#108 >= 30)) AND isnotnull(t_time_sk#106)) +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 11)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9)) (149) Project [codegen id : 31] -Output [1]: [t_time_sk#106] -Input [3]: [t_time_sk#106, t_hour#107, t_minute#108] +Output [1]: [t_time_sk#9] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (150) BroadcastExchange -Input [1]: [t_time_sk#106] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#109] +Input [1]: [t_time_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] (151) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_time_sk#101] -Right keys [1]: [t_time_sk#106] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#9] Join condition: None (152) Project [codegen id : 33] -Output [1]: [ss_store_sk#103] -Input [3]: [ss_sold_time_sk#101, ss_store_sk#103, t_time_sk#106] +Output [1]: [ss_store_sk#3] +Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] (153) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#110] +Output [1]: [s_store_sk#13] (154) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#103] -Right keys [1]: [s_store_sk#110] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#13] Join condition: None (155) Project [codegen id : 33] Output: [] -Input [2]: [ss_store_sk#103, s_store_sk#110] +Input [2]: [ss_store_sk#3, s_store_sk#13] (156) HashAggregate [codegen id : 33] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#111] -Results [1]: [count#112] +Aggregate Attributes [1]: [count#57] +Results [1]: [count#58] (157) Exchange -Input [1]: [count#112] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] +Input [1]: [count#58] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#59] (158) HashAggregate [codegen id : 34] -Input [1]: [count#112] +Input [1]: [count#58] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#114] -Results [1]: [count(1)#114 AS h11_30_to_12#115] +Aggregate Attributes [1]: [count(1)#60] +Results [1]: [count(1)#60 AS h11_30_to_12#61] (159) BroadcastExchange -Input [1]: [h11_30_to_12#115] -Arguments: IdentityBroadcastMode, [id=#116] +Input [1]: [h11_30_to_12#61] +Arguments: IdentityBroadcastMode, [id=#62] (160) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (161) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (162) ColumnarToRow [codegen id : 38] -Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (163) Filter [codegen id : 38] -Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] -Condition : ((isnotnull(ss_hdemo_sk#118) AND isnotnull(ss_sold_time_sk#117)) AND isnotnull(ss_store_sk#119)) +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) (164) Project [codegen id : 38] -Output [3]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119] -Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] (165) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#121] +Output [1]: [hd_demo_sk#5] (166) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_hdemo_sk#118] -Right keys [1]: [hd_demo_sk#121] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] Join condition: None (167) Project [codegen id : 38] -Output [2]: [ss_sold_time_sk#117, ss_store_sk#119] -Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, hd_demo_sk#121] +Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] (168) Scan parquet default.time_dim -Output [3]: [t_time_sk#122, t_hour#123, t_minute#124] +Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (169) ColumnarToRow [codegen id : 36] -Input [3]: [t_time_sk#122, t_hour#123, t_minute#124] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (170) Filter [codegen id : 36] -Input [3]: [t_time_sk#122, t_hour#123, t_minute#124] -Condition : ((((isnotnull(t_hour#123) AND isnotnull(t_minute#124)) AND (t_hour#123 = 12)) AND (t_minute#124 < 30)) AND isnotnull(t_time_sk#122)) +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 12)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) (171) Project [codegen id : 36] -Output [1]: [t_time_sk#122] -Input [3]: [t_time_sk#122, t_hour#123, t_minute#124] +Output [1]: [t_time_sk#9] +Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] (172) BroadcastExchange -Input [1]: [t_time_sk#122] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#125] +Input [1]: [t_time_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] (173) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_sold_time_sk#117] -Right keys [1]: [t_time_sk#122] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#9] Join condition: None (174) Project [codegen id : 38] -Output [1]: [ss_store_sk#119] -Input [3]: [ss_sold_time_sk#117, ss_store_sk#119, t_time_sk#122] +Output [1]: [ss_store_sk#3] +Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] (175) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#126] +Output [1]: [s_store_sk#13] (176) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_store_sk#119] -Right keys [1]: [s_store_sk#126] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#13] Join condition: None (177) Project [codegen id : 38] Output: [] -Input [2]: [ss_store_sk#119, s_store_sk#126] +Input [2]: [ss_store_sk#3, s_store_sk#13] (178) HashAggregate [codegen id : 38] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#127] -Results [1]: [count#128] +Aggregate Attributes [1]: [count#64] +Results [1]: [count#65] (179) Exchange -Input [1]: [count#128] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] +Input [1]: [count#65] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] (180) HashAggregate [codegen id : 39] -Input [1]: [count#128] +Input [1]: [count#65] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#130] -Results [1]: [count(1)#130 AS h12_to_12_30#131] +Aggregate Attributes [1]: [count(1)#67] +Results [1]: [count(1)#67 AS h12_to_12_30#68] (181) BroadcastExchange -Input [1]: [h12_to_12_30#131] -Arguments: IdentityBroadcastMode, [id=#132] +Input [1]: [h12_to_12_30#68] +Arguments: IdentityBroadcastMode, [id=#69] (182) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt index 8736c9861a5ce..65c66eb083f55 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt @@ -82,40 +82,40 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (12) Scan parquet default.store_sales -Output [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (13) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (14) Filter [codegen id : 1] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#44) AND (ss_quantity#44 >= 1)) AND (ss_quantity#44 <= 20)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) (15) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#45] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (16) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#45] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#45))] -Aggregate Attributes [2]: [sum#47, count#48] -Results [2]: [sum#49, count#50] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#45, count#46] +Results [2]: [sum#47, count#48] (17) Exchange -Input [2]: [sum#49, count#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#51] +Input [2]: [sum#47, count#48] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] (18) HashAggregate [codegen id : 2] -Input [2]: [sum#49, count#50] +Input [2]: [sum#47, count#48] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))#52] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#45))#52 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#53] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#50] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#50 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#51] Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#6, [id=#7] * HashAggregate (25) @@ -128,40 +128,40 @@ Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (19) Scan parquet default.store_sales -Output [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (20) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (21) Filter [codegen id : 1] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] -Condition : ((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 1)) AND (ss_quantity#54 <= 20)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) (22) Project [codegen id : 1] -Output [1]: [ss_net_paid#55] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (23) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#55] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [2]: [sum#57, count#58] -Results [2]: [sum#59, count#60] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#53, count#54] +Results [2]: [sum#55, count#56] (24) Exchange -Input [2]: [sum#59, count#60] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] +Input [2]: [sum#55, count#56] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#57] (25) HashAggregate [codegen id : 2] -Input [2]: [sum#59, count#60] +Input [2]: [sum#55, count#56] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#55))#62] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#55))#62 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#63] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#58] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#58 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#59] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#9, [id=#10] * HashAggregate (32) @@ -174,40 +174,40 @@ Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (26) Scan parquet default.store_sales -Output [2]: [ss_quantity#64, ss_sold_date_sk#65] +Output [2]: [ss_quantity#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (27) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (28) Filter [codegen id : 1] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] -Condition : ((isnotnull(ss_quantity#64) AND (ss_quantity#64 >= 21)) AND (ss_quantity#64 <= 40)) +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) (29) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (30) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#66] -Results [1]: [count#67] +Aggregate Attributes [1]: [count#60] +Results [1]: [count#61] (31) Exchange -Input [1]: [count#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#68] +Input [1]: [count#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] (32) HashAggregate [codegen id : 2] -Input [1]: [count#67] +Input [1]: [count#61] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#69] -Results [1]: [count(1)#69 AS count(1)#70] +Aggregate Attributes [1]: [count(1)#63] +Results [1]: [count(1)#63 AS count(1)#64] Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] * HashAggregate (39) @@ -220,40 +220,40 @@ Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (33) Scan parquet default.store_sales -Output [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (34) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (35) Filter [codegen id : 1] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] -Condition : ((isnotnull(ss_quantity#71) AND (ss_quantity#71 >= 21)) AND (ss_quantity#71 <= 40)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) (36) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#72] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (37) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#72] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#72))] -Aggregate Attributes [2]: [sum#74, count#75] -Results [2]: [sum#76, count#77] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#65, count#66] +Results [2]: [sum#67, count#68] (38) Exchange -Input [2]: [sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] +Input [2]: [sum#67, count#68] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#69] (39) HashAggregate [codegen id : 2] -Input [2]: [sum#76, count#77] +Input [2]: [sum#67, count#68] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))#79] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#72))#79 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#80] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#70] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#70 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#71] Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#13, [id=#14] * HashAggregate (46) @@ -266,40 +266,40 @@ Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (40) Scan parquet default.store_sales -Output [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (41) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (42) Filter [codegen id : 1] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] -Condition : ((isnotnull(ss_quantity#81) AND (ss_quantity#81 >= 21)) AND (ss_quantity#81 <= 40)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) (43) Project [codegen id : 1] -Output [1]: [ss_net_paid#82] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (44) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#82] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [2]: [sum#84, count#85] -Results [2]: [sum#86, count#87] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#72, count#73] +Results [2]: [sum#74, count#75] (45) Exchange -Input [2]: [sum#86, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] +Input [2]: [sum#74, count#75] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] (46) HashAggregate [codegen id : 2] -Input [2]: [sum#86, count#87] +Input [2]: [sum#74, count#75] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#82))#89] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#82))#89 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#90] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#77] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#77 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#78] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#16, [id=#17] * HashAggregate (53) @@ -312,40 +312,40 @@ Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (47) Scan parquet default.store_sales -Output [2]: [ss_quantity#91, ss_sold_date_sk#92] +Output [2]: [ss_quantity#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (48) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (49) Filter [codegen id : 1] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] -Condition : ((isnotnull(ss_quantity#91) AND (ss_quantity#91 >= 41)) AND (ss_quantity#91 <= 60)) +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) (50) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (51) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#93] -Results [1]: [count#94] +Aggregate Attributes [1]: [count#79] +Results [1]: [count#80] (52) Exchange -Input [1]: [count#94] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] +Input [1]: [count#80] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] (53) HashAggregate [codegen id : 2] -Input [1]: [count#94] +Input [1]: [count#80] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#96] -Results [1]: [count(1)#96 AS count(1)#97] +Aggregate Attributes [1]: [count(1)#82] +Results [1]: [count(1)#82 AS count(1)#83] Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#18, [id=#19] * HashAggregate (60) @@ -358,40 +358,40 @@ Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (54) Scan parquet default.store_sales -Output [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (55) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (56) Filter [codegen id : 1] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] -Condition : ((isnotnull(ss_quantity#98) AND (ss_quantity#98 >= 41)) AND (ss_quantity#98 <= 60)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) (57) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#99] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (58) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#99] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#99))] -Aggregate Attributes [2]: [sum#101, count#102] -Results [2]: [sum#103, count#104] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#84, count#85] +Results [2]: [sum#86, count#87] (59) Exchange -Input [2]: [sum#103, count#104] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] +Input [2]: [sum#86, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] (60) HashAggregate [codegen id : 2] -Input [2]: [sum#103, count#104] +Input [2]: [sum#86, count#87] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))#106] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#99))#106 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#107] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#89] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#89 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#90] Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#20, [id=#21] * HashAggregate (67) @@ -404,40 +404,40 @@ Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (61) Scan parquet default.store_sales -Output [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (62) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (63) Filter [codegen id : 1] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] -Condition : ((isnotnull(ss_quantity#108) AND (ss_quantity#108 >= 41)) AND (ss_quantity#108 <= 60)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) (64) Project [codegen id : 1] -Output [1]: [ss_net_paid#109] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (65) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#109] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#109))] -Aggregate Attributes [2]: [sum#111, count#112] -Results [2]: [sum#113, count#114] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#91, count#92] +Results [2]: [sum#93, count#94] (66) Exchange -Input [2]: [sum#113, count#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] +Input [2]: [sum#93, count#94] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] (67) HashAggregate [codegen id : 2] -Input [2]: [sum#113, count#114] +Input [2]: [sum#93, count#94] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#109))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#109))#116] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#109))#116 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#117] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#96] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#96 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#97] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#23, [id=#24] * HashAggregate (74) @@ -450,40 +450,40 @@ Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (68) Scan parquet default.store_sales -Output [2]: [ss_quantity#118, ss_sold_date_sk#119] +Output [2]: [ss_quantity#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (69) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (70) Filter [codegen id : 1] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] -Condition : ((isnotnull(ss_quantity#118) AND (ss_quantity#118 >= 61)) AND (ss_quantity#118 <= 80)) +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) (71) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (72) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#120] -Results [1]: [count#121] +Aggregate Attributes [1]: [count#98] +Results [1]: [count#99] (73) Exchange -Input [1]: [count#121] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#122] +Input [1]: [count#99] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#100] (74) HashAggregate [codegen id : 2] -Input [1]: [count#121] +Input [1]: [count#99] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#123] -Results [1]: [count(1)#123 AS count(1)#124] +Aggregate Attributes [1]: [count(1)#101] +Results [1]: [count(1)#101 AS count(1)#102] Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#25, [id=#26] * HashAggregate (81) @@ -496,40 +496,40 @@ Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (75) Scan parquet default.store_sales -Output [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (76) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (77) Filter [codegen id : 1] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] -Condition : ((isnotnull(ss_quantity#125) AND (ss_quantity#125 >= 61)) AND (ss_quantity#125 <= 80)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) (78) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#126] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (79) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#126] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#126))] -Aggregate Attributes [2]: [sum#128, count#129] -Results [2]: [sum#130, count#131] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#103, count#104] +Results [2]: [sum#105, count#106] (80) Exchange -Input [2]: [sum#130, count#131] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#132] +Input [2]: [sum#105, count#106] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#107] (81) HashAggregate [codegen id : 2] -Input [2]: [sum#130, count#131] +Input [2]: [sum#105, count#106] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))#133] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#126))#133 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#134] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#108] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#108 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#109] Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#27, [id=#28] * HashAggregate (88) @@ -542,40 +542,40 @@ Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (82) Scan parquet default.store_sales -Output [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (83) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (84) Filter [codegen id : 1] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] -Condition : ((isnotnull(ss_quantity#135) AND (ss_quantity#135 >= 61)) AND (ss_quantity#135 <= 80)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) (85) Project [codegen id : 1] -Output [1]: [ss_net_paid#136] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (86) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#136] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#136))] -Aggregate Attributes [2]: [sum#138, count#139] -Results [2]: [sum#140, count#141] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#110, count#111] +Results [2]: [sum#112, count#113] (87) Exchange -Input [2]: [sum#140, count#141] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#142] +Input [2]: [sum#112, count#113] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] (88) HashAggregate [codegen id : 2] -Input [2]: [sum#140, count#141] +Input [2]: [sum#112, count#113] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#136))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#136))#143] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#136))#143 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#144] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#115] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#115 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#116] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#30, [id=#31] * HashAggregate (95) @@ -588,40 +588,40 @@ Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (89) Scan parquet default.store_sales -Output [2]: [ss_quantity#145, ss_sold_date_sk#146] +Output [2]: [ss_quantity#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (90) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (91) Filter [codegen id : 1] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] -Condition : ((isnotnull(ss_quantity#145) AND (ss_quantity#145 >= 81)) AND (ss_quantity#145 <= 100)) +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) (92) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (93) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#147] -Results [1]: [count#148] +Aggregate Attributes [1]: [count#117] +Results [1]: [count#118] (94) Exchange -Input [1]: [count#148] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#149] +Input [1]: [count#118] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#119] (95) HashAggregate [codegen id : 2] -Input [1]: [count#148] +Input [1]: [count#118] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#150] -Results [1]: [count(1)#150 AS count(1)#151] +Aggregate Attributes [1]: [count(1)#120] +Results [1]: [count(1)#120 AS count(1)#121] Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#32, [id=#33] * HashAggregate (102) @@ -634,40 +634,40 @@ Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (96) Scan parquet default.store_sales -Output [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (97) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (98) Filter [codegen id : 1] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] -Condition : ((isnotnull(ss_quantity#152) AND (ss_quantity#152 >= 81)) AND (ss_quantity#152 <= 100)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) (99) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#153] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (100) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#153] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#153))] -Aggregate Attributes [2]: [sum#155, count#156] -Results [2]: [sum#157, count#158] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#122, count#123] +Results [2]: [sum#124, count#125] (101) Exchange -Input [2]: [sum#157, count#158] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#159] +Input [2]: [sum#124, count#125] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#126] (102) HashAggregate [codegen id : 2] -Input [2]: [sum#157, count#158] +Input [2]: [sum#124, count#125] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))#160] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#153))#160 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#161] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#127] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#127 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#128] Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#34, [id=#35] * HashAggregate (109) @@ -680,39 +680,39 @@ Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (103) Scan parquet default.store_sales -Output [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (105) Filter [codegen id : 1] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] -Condition : ((isnotnull(ss_quantity#162) AND (ss_quantity#162 >= 81)) AND (ss_quantity#162 <= 100)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) (106) Project [codegen id : 1] -Output [1]: [ss_net_paid#163] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (107) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#163] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#163))] -Aggregate Attributes [2]: [sum#165, count#166] -Results [2]: [sum#167, count#168] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#129, count#130] +Results [2]: [sum#131, count#132] (108) Exchange -Input [2]: [sum#167, count#168] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#169] +Input [2]: [sum#131, count#132] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#133] (109) HashAggregate [codegen id : 2] -Input [2]: [sum#167, count#168] +Input [2]: [sum#131, count#132] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#163))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#163))#170] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#163))#170 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#171] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#134] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#134 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#135] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt index 8736c9861a5ce..65c66eb083f55 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt @@ -82,40 +82,40 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (12) Scan parquet default.store_sales -Output [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (13) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (14) Filter [codegen id : 1] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] -Condition : ((isnotnull(ss_quantity#44) AND (ss_quantity#44 >= 1)) AND (ss_quantity#44 <= 20)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) (15) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#45] -Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (16) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#45] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#45))] -Aggregate Attributes [2]: [sum#47, count#48] -Results [2]: [sum#49, count#50] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#45, count#46] +Results [2]: [sum#47, count#48] (17) Exchange -Input [2]: [sum#49, count#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#51] +Input [2]: [sum#47, count#48] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] (18) HashAggregate [codegen id : 2] -Input [2]: [sum#49, count#50] +Input [2]: [sum#47, count#48] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))#52] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#45))#52 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#53] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#50] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#50 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#51] Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#6, [id=#7] * HashAggregate (25) @@ -128,40 +128,40 @@ Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (19) Scan parquet default.store_sales -Output [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (20) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (21) Filter [codegen id : 1] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] -Condition : ((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 1)) AND (ss_quantity#54 <= 20)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) (22) Project [codegen id : 1] -Output [1]: [ss_net_paid#55] -Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (23) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#55] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [2]: [sum#57, count#58] -Results [2]: [sum#59, count#60] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#53, count#54] +Results [2]: [sum#55, count#56] (24) Exchange -Input [2]: [sum#59, count#60] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] +Input [2]: [sum#55, count#56] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#57] (25) HashAggregate [codegen id : 2] -Input [2]: [sum#59, count#60] +Input [2]: [sum#55, count#56] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#55))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#55))#62] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#55))#62 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#63] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#58] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#58 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#59] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#9, [id=#10] * HashAggregate (32) @@ -174,40 +174,40 @@ Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (26) Scan parquet default.store_sales -Output [2]: [ss_quantity#64, ss_sold_date_sk#65] +Output [2]: [ss_quantity#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (27) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (28) Filter [codegen id : 1] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] -Condition : ((isnotnull(ss_quantity#64) AND (ss_quantity#64 >= 21)) AND (ss_quantity#64 <= 40)) +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) (29) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (30) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#66] -Results [1]: [count#67] +Aggregate Attributes [1]: [count#60] +Results [1]: [count#61] (31) Exchange -Input [1]: [count#67] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#68] +Input [1]: [count#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] (32) HashAggregate [codegen id : 2] -Input [1]: [count#67] +Input [1]: [count#61] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#69] -Results [1]: [count(1)#69 AS count(1)#70] +Aggregate Attributes [1]: [count(1)#63] +Results [1]: [count(1)#63 AS count(1)#64] Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] * HashAggregate (39) @@ -220,40 +220,40 @@ Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (33) Scan parquet default.store_sales -Output [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (34) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (35) Filter [codegen id : 1] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] -Condition : ((isnotnull(ss_quantity#71) AND (ss_quantity#71 >= 21)) AND (ss_quantity#71 <= 40)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) (36) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#72] -Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (37) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#72] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#72))] -Aggregate Attributes [2]: [sum#74, count#75] -Results [2]: [sum#76, count#77] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#65, count#66] +Results [2]: [sum#67, count#68] (38) Exchange -Input [2]: [sum#76, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] +Input [2]: [sum#67, count#68] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#69] (39) HashAggregate [codegen id : 2] -Input [2]: [sum#76, count#77] +Input [2]: [sum#67, count#68] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))#79] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#72))#79 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#80] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#70] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#70 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#71] Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#13, [id=#14] * HashAggregate (46) @@ -266,40 +266,40 @@ Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (40) Scan parquet default.store_sales -Output [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (41) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (42) Filter [codegen id : 1] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] -Condition : ((isnotnull(ss_quantity#81) AND (ss_quantity#81 >= 21)) AND (ss_quantity#81 <= 40)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) (43) Project [codegen id : 1] -Output [1]: [ss_net_paid#82] -Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (44) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#82] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [2]: [sum#84, count#85] -Results [2]: [sum#86, count#87] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#72, count#73] +Results [2]: [sum#74, count#75] (45) Exchange -Input [2]: [sum#86, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] +Input [2]: [sum#74, count#75] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] (46) HashAggregate [codegen id : 2] -Input [2]: [sum#86, count#87] +Input [2]: [sum#74, count#75] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#82))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#82))#89] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#82))#89 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#90] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#77] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#77 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#78] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#16, [id=#17] * HashAggregate (53) @@ -312,40 +312,40 @@ Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (47) Scan parquet default.store_sales -Output [2]: [ss_quantity#91, ss_sold_date_sk#92] +Output [2]: [ss_quantity#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (48) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (49) Filter [codegen id : 1] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] -Condition : ((isnotnull(ss_quantity#91) AND (ss_quantity#91 >= 41)) AND (ss_quantity#91 <= 60)) +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) (50) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (51) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#93] -Results [1]: [count#94] +Aggregate Attributes [1]: [count#79] +Results [1]: [count#80] (52) Exchange -Input [1]: [count#94] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] +Input [1]: [count#80] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] (53) HashAggregate [codegen id : 2] -Input [1]: [count#94] +Input [1]: [count#80] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#96] -Results [1]: [count(1)#96 AS count(1)#97] +Aggregate Attributes [1]: [count(1)#82] +Results [1]: [count(1)#82 AS count(1)#83] Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#18, [id=#19] * HashAggregate (60) @@ -358,40 +358,40 @@ Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (54) Scan parquet default.store_sales -Output [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (55) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (56) Filter [codegen id : 1] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] -Condition : ((isnotnull(ss_quantity#98) AND (ss_quantity#98 >= 41)) AND (ss_quantity#98 <= 60)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) (57) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#99] -Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (58) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#99] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#99))] -Aggregate Attributes [2]: [sum#101, count#102] -Results [2]: [sum#103, count#104] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#84, count#85] +Results [2]: [sum#86, count#87] (59) Exchange -Input [2]: [sum#103, count#104] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] +Input [2]: [sum#86, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] (60) HashAggregate [codegen id : 2] -Input [2]: [sum#103, count#104] +Input [2]: [sum#86, count#87] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))#106] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#99))#106 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#107] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#89] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#89 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#90] Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#20, [id=#21] * HashAggregate (67) @@ -404,40 +404,40 @@ Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (61) Scan parquet default.store_sales -Output [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (62) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (63) Filter [codegen id : 1] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] -Condition : ((isnotnull(ss_quantity#108) AND (ss_quantity#108 >= 41)) AND (ss_quantity#108 <= 60)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) (64) Project [codegen id : 1] -Output [1]: [ss_net_paid#109] -Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (65) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#109] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#109))] -Aggregate Attributes [2]: [sum#111, count#112] -Results [2]: [sum#113, count#114] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#91, count#92] +Results [2]: [sum#93, count#94] (66) Exchange -Input [2]: [sum#113, count#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] +Input [2]: [sum#93, count#94] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] (67) HashAggregate [codegen id : 2] -Input [2]: [sum#113, count#114] +Input [2]: [sum#93, count#94] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#109))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#109))#116] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#109))#116 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#117] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#96] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#96 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#97] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#23, [id=#24] * HashAggregate (74) @@ -450,40 +450,40 @@ Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (68) Scan parquet default.store_sales -Output [2]: [ss_quantity#118, ss_sold_date_sk#119] +Output [2]: [ss_quantity#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (69) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (70) Filter [codegen id : 1] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] -Condition : ((isnotnull(ss_quantity#118) AND (ss_quantity#118 >= 61)) AND (ss_quantity#118 <= 80)) +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) (71) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (72) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#120] -Results [1]: [count#121] +Aggregate Attributes [1]: [count#98] +Results [1]: [count#99] (73) Exchange -Input [1]: [count#121] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#122] +Input [1]: [count#99] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#100] (74) HashAggregate [codegen id : 2] -Input [1]: [count#121] +Input [1]: [count#99] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#123] -Results [1]: [count(1)#123 AS count(1)#124] +Aggregate Attributes [1]: [count(1)#101] +Results [1]: [count(1)#101 AS count(1)#102] Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#25, [id=#26] * HashAggregate (81) @@ -496,40 +496,40 @@ Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (75) Scan parquet default.store_sales -Output [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (76) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (77) Filter [codegen id : 1] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] -Condition : ((isnotnull(ss_quantity#125) AND (ss_quantity#125 >= 61)) AND (ss_quantity#125 <= 80)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) (78) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#126] -Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (79) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#126] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#126))] -Aggregate Attributes [2]: [sum#128, count#129] -Results [2]: [sum#130, count#131] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#103, count#104] +Results [2]: [sum#105, count#106] (80) Exchange -Input [2]: [sum#130, count#131] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#132] +Input [2]: [sum#105, count#106] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#107] (81) HashAggregate [codegen id : 2] -Input [2]: [sum#130, count#131] +Input [2]: [sum#105, count#106] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))#133] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#126))#133 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#134] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#108] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#108 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#109] Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#27, [id=#28] * HashAggregate (88) @@ -542,40 +542,40 @@ Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (82) Scan parquet default.store_sales -Output [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (83) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (84) Filter [codegen id : 1] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] -Condition : ((isnotnull(ss_quantity#135) AND (ss_quantity#135 >= 61)) AND (ss_quantity#135 <= 80)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) (85) Project [codegen id : 1] -Output [1]: [ss_net_paid#136] -Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (86) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#136] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#136))] -Aggregate Attributes [2]: [sum#138, count#139] -Results [2]: [sum#140, count#141] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#110, count#111] +Results [2]: [sum#112, count#113] (87) Exchange -Input [2]: [sum#140, count#141] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#142] +Input [2]: [sum#112, count#113] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] (88) HashAggregate [codegen id : 2] -Input [2]: [sum#140, count#141] +Input [2]: [sum#112, count#113] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#136))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#136))#143] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#136))#143 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#144] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#115] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#115 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#116] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#30, [id=#31] * HashAggregate (95) @@ -588,40 +588,40 @@ Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (89) Scan parquet default.store_sales -Output [2]: [ss_quantity#145, ss_sold_date_sk#146] +Output [2]: [ss_quantity#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (90) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (91) Filter [codegen id : 1] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] -Condition : ((isnotnull(ss_quantity#145) AND (ss_quantity#145 >= 81)) AND (ss_quantity#145 <= 100)) +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) (92) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#145, ss_sold_date_sk#146] +Input [2]: [ss_quantity#37, ss_sold_date_sk#38] (93) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#147] -Results [1]: [count#148] +Aggregate Attributes [1]: [count#117] +Results [1]: [count#118] (94) Exchange -Input [1]: [count#148] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#149] +Input [1]: [count#118] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#119] (95) HashAggregate [codegen id : 2] -Input [1]: [count#148] +Input [1]: [count#118] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#150] -Results [1]: [count(1)#150 AS count(1)#151] +Aggregate Attributes [1]: [count(1)#120] +Results [1]: [count(1)#120 AS count(1)#121] Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#32, [id=#33] * HashAggregate (102) @@ -634,40 +634,40 @@ Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (96) Scan parquet default.store_sales -Output [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (97) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (98) Filter [codegen id : 1] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] -Condition : ((isnotnull(ss_quantity#152) AND (ss_quantity#152 >= 81)) AND (ss_quantity#152 <= 100)) +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) (99) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#153] -Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Output [1]: [ss_ext_discount_amt#44] +Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] (100) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#153] +Input [1]: [ss_ext_discount_amt#44] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#153))] -Aggregate Attributes [2]: [sum#155, count#156] -Results [2]: [sum#157, count#158] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [2]: [sum#122, count#123] +Results [2]: [sum#124, count#125] (101) Exchange -Input [2]: [sum#157, count#158] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#159] +Input [2]: [sum#124, count#125] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#126] (102) HashAggregate [codegen id : 2] -Input [2]: [sum#157, count#158] +Input [2]: [sum#124, count#125] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))#160] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#153))#160 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#161] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#127] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#127 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#128] Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#34, [id=#35] * HashAggregate (109) @@ -680,39 +680,39 @@ Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (103) Scan parquet default.store_sales -Output [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (105) Filter [codegen id : 1] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] -Condition : ((isnotnull(ss_quantity#162) AND (ss_quantity#162 >= 81)) AND (ss_quantity#162 <= 100)) +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) (106) Project [codegen id : 1] -Output [1]: [ss_net_paid#163] -Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Output [1]: [ss_net_paid#52] +Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] (107) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#163] +Input [1]: [ss_net_paid#52] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#163))] -Aggregate Attributes [2]: [sum#165, count#166] -Results [2]: [sum#167, count#168] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [2]: [sum#129, count#130] +Results [2]: [sum#131, count#132] (108) Exchange -Input [2]: [sum#167, count#168] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#169] +Input [2]: [sum#131, count#132] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#133] (109) HashAggregate [codegen id : 2] -Input [2]: [sum#167, count#168] +Input [2]: [sum#131, count#132] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#163))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#163))#170] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#163))#170 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#171] +Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#134] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#134 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#135] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt index 5226fce4ef512..a74b232e8c667 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt @@ -183,108 +183,108 @@ Aggregate Attributes [1]: [count(1)#17] Results [1]: [count(1)#17 AS amc#18] (29) Scan parquet default.web_sales -Output [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] (31) Filter [codegen id : 8] -Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] -Condition : ((isnotnull(ws_ship_hdemo_sk#20) AND isnotnull(ws_sold_time_sk#19)) AND isnotnull(ws_web_page_sk#21)) +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) (32) Project [codegen id : 8] -Output [3]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21] -Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] (33) ReusedExchange [Reuses operator id: 9] -Output [1]: [wp_web_page_sk#23] +Output [1]: [wp_web_page_sk#5] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#21] -Right keys [1]: [wp_web_page_sk#23] +Left keys [1]: [ws_web_page_sk#3] +Right keys [1]: [wp_web_page_sk#5] Join condition: None (35) Project [codegen id : 8] -Output [2]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20] -Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, wp_web_page_sk#23] +Output [2]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2] +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, wp_web_page_sk#5] (36) ReusedExchange [Reuses operator id: 16] -Output [1]: [hd_demo_sk#24] +Output [1]: [hd_demo_sk#8] (37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_ship_hdemo_sk#20] -Right keys [1]: [hd_demo_sk#24] +Left keys [1]: [ws_ship_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#8] Join condition: None (38) Project [codegen id : 8] -Output [1]: [ws_sold_time_sk#19] -Input [3]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, hd_demo_sk#24] +Output [1]: [ws_sold_time_sk#1] +Input [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, hd_demo_sk#8] (39) Scan parquet default.time_dim -Output [2]: [t_time_sk#25, t_hour#26] +Output [2]: [t_time_sk#11, t_hour#12] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 7] -Input [2]: [t_time_sk#25, t_hour#26] +Input [2]: [t_time_sk#11, t_hour#12] (41) Filter [codegen id : 7] -Input [2]: [t_time_sk#25, t_hour#26] -Condition : (((isnotnull(t_hour#26) AND (t_hour#26 >= 19)) AND (t_hour#26 <= 20)) AND isnotnull(t_time_sk#25)) +Input [2]: [t_time_sk#11, t_hour#12] +Condition : (((isnotnull(t_hour#12) AND (t_hour#12 >= 19)) AND (t_hour#12 <= 20)) AND isnotnull(t_time_sk#11)) (42) Project [codegen id : 7] -Output [1]: [t_time_sk#25] -Input [2]: [t_time_sk#25, t_hour#26] +Output [1]: [t_time_sk#11] +Input [2]: [t_time_sk#11, t_hour#12] (43) BroadcastExchange -Input [1]: [t_time_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] +Input [1]: [t_time_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_time_sk#19] -Right keys [1]: [t_time_sk#25] +Left keys [1]: [ws_sold_time_sk#1] +Right keys [1]: [t_time_sk#11] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ws_sold_time_sk#19, t_time_sk#25] +Input [2]: [ws_sold_time_sk#1, t_time_sk#11] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [1]: [count#29] +Aggregate Attributes [1]: [count#20] +Results [1]: [count#21] (47) Exchange -Input [1]: [count#29] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#30] +Input [1]: [count#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] (48) HashAggregate [codegen id : 9] -Input [1]: [count#29] +Input [1]: [count#21] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [1]: [count(1)#31 AS pmc#32] +Aggregate Attributes [1]: [count(1)#23] +Results [1]: [count(1)#23 AS pmc#24] (49) BroadcastExchange -Input [1]: [pmc#32] -Arguments: IdentityBroadcastMode, [id=#33] +Input [1]: [pmc#24] +Arguments: IdentityBroadcastMode, [id=#25] (50) BroadcastNestedLoopJoin [codegen id : 10] Join condition: None (51) Project [codegen id : 10] -Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#32 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#34] -Input [2]: [amc#18, pmc#32] +Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#24 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#26] +Input [2]: [amc#18, pmc#24] (52) Sort [codegen id : 10] -Input [1]: [am_pm_ratio#34] -Arguments: [am_pm_ratio#34 ASC NULLS FIRST], true, 0 +Input [1]: [am_pm_ratio#26] +Arguments: [am_pm_ratio#26 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt index e369a027040d3..070b497ac4d18 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt @@ -183,108 +183,108 @@ Aggregate Attributes [1]: [count(1)#17] Results [1]: [count(1)#17 AS amc#18] (29) Scan parquet default.web_sales -Output [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] (31) Filter [codegen id : 8] -Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] -Condition : ((isnotnull(ws_ship_hdemo_sk#20) AND isnotnull(ws_sold_time_sk#19)) AND isnotnull(ws_web_page_sk#21)) +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) (32) Project [codegen id : 8] -Output [3]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21] -Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] (33) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#23] +Output [1]: [hd_demo_sk#5] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_ship_hdemo_sk#20] -Right keys [1]: [hd_demo_sk#23] +Left keys [1]: [ws_ship_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] Join condition: None (35) Project [codegen id : 8] -Output [2]: [ws_sold_time_sk#19, ws_web_page_sk#21] -Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, hd_demo_sk#23] +Output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] (36) Scan parquet default.time_dim -Output [2]: [t_time_sk#24, t_hour#25] +Output [2]: [t_time_sk#8, t_hour#9] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [2]: [t_time_sk#24, t_hour#25] +Input [2]: [t_time_sk#8, t_hour#9] (38) Filter [codegen id : 6] -Input [2]: [t_time_sk#24, t_hour#25] -Condition : (((isnotnull(t_hour#25) AND (t_hour#25 >= 19)) AND (t_hour#25 <= 20)) AND isnotnull(t_time_sk#24)) +Input [2]: [t_time_sk#8, t_hour#9] +Condition : (((isnotnull(t_hour#9) AND (t_hour#9 >= 19)) AND (t_hour#9 <= 20)) AND isnotnull(t_time_sk#8)) (39) Project [codegen id : 6] -Output [1]: [t_time_sk#24] -Input [2]: [t_time_sk#24, t_hour#25] +Output [1]: [t_time_sk#8] +Input [2]: [t_time_sk#8, t_hour#9] (40) BroadcastExchange -Input [1]: [t_time_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [t_time_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_time_sk#19] -Right keys [1]: [t_time_sk#24] +Left keys [1]: [ws_sold_time_sk#1] +Right keys [1]: [t_time_sk#8] Join condition: None (42) Project [codegen id : 8] -Output [1]: [ws_web_page_sk#21] -Input [3]: [ws_sold_time_sk#19, ws_web_page_sk#21, t_time_sk#24] +Output [1]: [ws_web_page_sk#3] +Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#8] (43) ReusedExchange [Reuses operator id: 23] -Output [1]: [wp_web_page_sk#27] +Output [1]: [wp_web_page_sk#11] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#21] -Right keys [1]: [wp_web_page_sk#27] +Left keys [1]: [ws_web_page_sk#3] +Right keys [1]: [wp_web_page_sk#11] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ws_web_page_sk#21, wp_web_page_sk#27] +Input [2]: [ws_web_page_sk#3, wp_web_page_sk#11] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [1]: [count#29] +Aggregate Attributes [1]: [count#20] +Results [1]: [count#21] (47) Exchange -Input [1]: [count#29] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#30] +Input [1]: [count#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] (48) HashAggregate [codegen id : 9] -Input [1]: [count#29] +Input [1]: [count#21] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [1]: [count(1)#31 AS pmc#32] +Aggregate Attributes [1]: [count(1)#23] +Results [1]: [count(1)#23 AS pmc#24] (49) BroadcastExchange -Input [1]: [pmc#32] -Arguments: IdentityBroadcastMode, [id=#33] +Input [1]: [pmc#24] +Arguments: IdentityBroadcastMode, [id=#25] (50) BroadcastNestedLoopJoin [codegen id : 10] Join condition: None (51) Project [codegen id : 10] -Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#32 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#34] -Input [2]: [amc#18, pmc#32] +Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#24 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#26] +Input [2]: [amc#18, pmc#24] (52) Sort [codegen id : 10] -Input [1]: [am_pm_ratio#34] -Arguments: [am_pm_ratio#34 ASC NULLS FIRST], true, 0 +Input [1]: [am_pm_ratio#26] +Arguments: [am_pm_ratio#26 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index faf82026138f6..7d525630d8a9a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -82,119 +82,124 @@ Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] (11) Scan parquet default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] +Output [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] +Input [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] (13) Filter [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) +Input [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] +Condition : isnotnull(ws_item_sk#8) (14) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#9] +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#8, d_date#9] +Input [2]: [d_date_sk#11, d_date#12] (16) Filter [codegen id : 2] +<<<<<<< HEAD +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) +======= Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) +>>>>>>> abfd9b23cd7c21e9525df85a16e0611ef0f35908 (17) Project [codegen id : 2] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#9] +Output [1]: [d_date_sk#11] +Input [2]: [d_date_sk#11, d_date#12] (18) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] (19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ws_sold_date_sk#10] +Right keys [1]: [d_date_sk#11] Join condition: None (20) Project [codegen id : 3] -Output [2]: [ws_item_sk#1, ws_ext_discount_amt#2] -Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#8] +Output [2]: [ws_item_sk#8, ws_ext_discount_amt#9] +Input [4]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10, d_date_sk#11] (21) HashAggregate [codegen id : 3] -Input [2]: [ws_item_sk#1, ws_ext_discount_amt#2] -Keys [1]: [ws_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [2]: [sum#11, count#12] -Results [3]: [ws_item_sk#1, sum#13, count#14] +Input [2]: [ws_item_sk#8, ws_ext_discount_amt#9] +Keys [1]: [ws_item_sk#8] +Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#9))] +Aggregate Attributes [2]: [sum#14, count#15] +Results [3]: [ws_item_sk#8, sum#16, count#17] (22) Exchange -Input [3]: [ws_item_sk#1, sum#13, count#14] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [3]: [ws_item_sk#8, sum#16, count#17] +Arguments: hashpartitioning(ws_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#18] (23) HashAggregate [codegen id : 4] -Input [3]: [ws_item_sk#1, sum#13, count#14] -Keys [1]: [ws_item_sk#1] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#2))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#2))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1 AS ws_item_sk#1#18] +Input [3]: [ws_item_sk#8, sum#16, count#17] +Keys [1]: [ws_item_sk#8] +Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#9))] +Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#9))#19] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#9))#19 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] (24) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] +Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#20) (25) BroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#19] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#21] (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] -Right keys [1]: [ws_item_sk#1#18] -Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) +Right keys [1]: [ws_item_sk#8] +Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#20) (27) Project [codegen id : 6] Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] +Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] (28) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 6] Output [1]: [ws_ext_discount_amt#2] -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#8] +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#22] (31) HashAggregate [codegen id : 6] Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#20] -Results [1]: [sum#21] +Aggregate Attributes [1]: [sum#23] +Results [1]: [sum#24] (32) Exchange -Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [sum#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#25] (33) HashAggregate [codegen id : 7] -Input [1]: [sum#21] +Input [1]: [sum#24] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#23] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#23,17,2) AS Excess Discount Amount #24] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#26] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#26,17,2) AS Excess Discount Amount #27] (34) Sort [codegen id : 7] -Input [1]: [Excess Discount Amount #24] -Arguments: [Excess Discount Amount #24 ASC NULLS FIRST], true, 0 +Input [1]: [Excess Discount Amount #27] +Arguments: [Excess Discount Amount #27 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -203,8 +208,8 @@ ReusedExchange (35) (35) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index bee7110ecd6dd..c77590bf71044 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -175,7 +175,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Output [2]: [web_site_sk#20, web_company_name#21] 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)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 10] @@ -183,7 +183,7 @@ Input [2]: [web_site_sk#20, web_company_name#21] (31) Filter [codegen id : 10] Input [2]: [web_site_sk#20, web_company_name#21] -Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri)) AND isnotnull(web_site_sk#20)) +Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri )) AND isnotnull(web_site_sk#20)) (32) Project [codegen id : 10] Output [1]: [web_site_sk#20] @@ -214,7 +214,7 @@ Input [2]: [d_date_sk#23, d_date#24] (38) Filter [codegen id : 11] Input [2]: [d_date_sk#23, d_date#24] -Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 10623)) AND (d_date#24 <= 10683)) AND isnotnull(d_date_sk#23)) +Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 1999-02-01)) AND (d_date#24 <= 1999-04-02)) AND isnotnull(d_date_sk#23)) (39) Project [codegen id : 11] Output [1]: [d_date_sk#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index efa09c2f625cd..9558a01423452 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -152,7 +152,7 @@ Input [2]: [d_date_sk#17, d_date#18] (24) Filter [codegen id : 9] Input [2]: [d_date_sk#17, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 10623)) AND (d_date#18 <= 10683)) AND isnotnull(d_date_sk#17)) +Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-01)) AND (d_date#18 <= 1999-04-02)) AND isnotnull(d_date_sk#17)) (25) Project [codegen id : 9] Output [1]: [d_date_sk#17] @@ -206,7 +206,7 @@ Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_ Output [2]: [web_site_sk#23, web_company_name#24] 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)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 11] @@ -214,7 +214,7 @@ Input [2]: [web_site_sk#23, web_company_name#24] (38) Filter [codegen id : 11] Input [2]: [web_site_sk#23, web_company_name#24] -Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri)) AND isnotnull(web_site_sk#23)) +Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri )) AND isnotnull(web_site_sk#23)) (39) Project [codegen id : 11] Output [1]: [web_site_sk#23] 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 725939ef24f1b..320a93e19bb27 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 @@ -252,7 +252,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num 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)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct (45) ColumnarToRow [codegen id : 19] @@ -260,7 +260,7 @@ Input [2]: [web_site_sk#24, web_company_name#25] (46) Filter [codegen id : 19] 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)) +Condition : ((isnotnull(web_company_name#25) AND (web_company_name#25 = pri )) AND isnotnull(web_site_sk#24)) (47) Project [codegen id : 19] Output [1]: [web_site_sk#24] @@ -291,7 +291,7 @@ Input [2]: [d_date_sk#27, d_date#28] (53) Filter [codegen id : 20] Input [2]: [d_date_sk#27, d_date#28] -Condition : (((isnotnull(d_date#28) AND (d_date#28 >= 10623)) AND (d_date#28 <= 10683)) AND isnotnull(d_date_sk#27)) +Condition : (((isnotnull(d_date#28) AND (d_date#28 >= 1999-02-01)) AND (d_date#28 <= 1999-04-02)) AND isnotnull(d_date_sk#27)) (54) Project [codegen id : 20] Output [1]: [d_date_sk#27] 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 9f15375b5cfc0..2f719a4be1630 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 @@ -229,7 +229,7 @@ Input [2]: [d_date_sk#21, d_date#22] (39) Filter [codegen id : 18] Input [2]: [d_date_sk#21, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 10623)) AND (d_date#22 <= 10683)) AND isnotnull(d_date_sk#21)) +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-01)) AND (d_date#22 <= 1999-04-02)) AND isnotnull(d_date_sk#21)) (40) Project [codegen id : 18] Output [1]: [d_date_sk#21] @@ -283,7 +283,7 @@ Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_ Output [2]: [web_site_sk#27, web_company_name#28] 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)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 20] @@ -291,7 +291,7 @@ Input [2]: [web_site_sk#27, web_company_name#28] (53) Filter [codegen id : 20] Input [2]: [web_site_sk#27, web_company_name#28] -Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri)) AND isnotnull(web_site_sk#27)) +Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri )) AND isnotnull(web_site_sk#27)) (54) Project [codegen id : 20] Output [1]: [web_site_sk#27] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt index c01f3465ed693..245357ef5560c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt @@ -102,16 +102,16 @@ ReadSchema: struct Input [3]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13] (16) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#5] (17) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#5] Join condition: None (18) Project [codegen id : 5] Output [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#14] +Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#5] (19) HashAggregate [codegen id : 5] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] @@ -122,45 +122,45 @@ Results [2]: [cs_bill_customer_sk#11, cs_item_sk#12] (20) Exchange Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#15] +Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#14] (21) HashAggregate [codegen id : 6] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Keys [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Functions: [] Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#11 AS customer_sk#16, cs_item_sk#12 AS item_sk#17] +Results [2]: [cs_bill_customer_sk#11 AS customer_sk#15, cs_item_sk#12 AS item_sk#16] (22) Sort [codegen id : 6] -Input [2]: [customer_sk#16, item_sk#17] -Arguments: [customer_sk#16 ASC NULLS FIRST, item_sk#17 ASC NULLS FIRST], false, 0 +Input [2]: [customer_sk#15, item_sk#16] +Arguments: [customer_sk#15 ASC NULLS FIRST, item_sk#16 ASC NULLS FIRST], false, 0 (23) SortMergeJoin Left keys [2]: [customer_sk#9, item_sk#10] -Right keys [2]: [customer_sk#16, item_sk#17] +Right keys [2]: [customer_sk#15, item_sk#16] Join condition: None (24) Project [codegen id : 7] -Output [2]: [customer_sk#9, customer_sk#16] -Input [4]: [customer_sk#9, item_sk#10, customer_sk#16, item_sk#17] +Output [2]: [customer_sk#9, customer_sk#15] +Input [4]: [customer_sk#9, item_sk#10, customer_sk#15, item_sk#16] (25) HashAggregate [codegen id : 7] -Input [2]: [customer_sk#9, customer_sk#16] +Input [2]: [customer_sk#9, customer_sk#15] Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum#18, sum#19, sum#20] -Results [3]: [sum#21, sum#22, sum#23] +Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum#17, sum#18, sum#19] +Results [3]: [sum#20, sum#21, sum#22] (26) Exchange -Input [3]: [sum#21, sum#22, sum#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [sum#20, sum#21, sum#22] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#23] (27) HashAggregate [codegen id : 8] -Input [3]: [sum#21, sum#22, sum#23] +Input [3]: [sum#20, sum#21, sum#22] Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27] -Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25 AS store_only#28, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26 AS catalog_only#29, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27 AS store_and_catalog#30] +Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26] +Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24 AS store_only#27, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25 AS catalog_only#28, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26 AS store_and_catalog#29] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt index c01f3465ed693..245357ef5560c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt @@ -102,16 +102,16 @@ ReadSchema: struct Input [3]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13] (16) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#5] (17) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#5] Join condition: None (18) Project [codegen id : 5] Output [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#14] +Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#5] (19) HashAggregate [codegen id : 5] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] @@ -122,45 +122,45 @@ Results [2]: [cs_bill_customer_sk#11, cs_item_sk#12] (20) Exchange Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#15] +Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#14] (21) HashAggregate [codegen id : 6] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Keys [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Functions: [] Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#11 AS customer_sk#16, cs_item_sk#12 AS item_sk#17] +Results [2]: [cs_bill_customer_sk#11 AS customer_sk#15, cs_item_sk#12 AS item_sk#16] (22) Sort [codegen id : 6] -Input [2]: [customer_sk#16, item_sk#17] -Arguments: [customer_sk#16 ASC NULLS FIRST, item_sk#17 ASC NULLS FIRST], false, 0 +Input [2]: [customer_sk#15, item_sk#16] +Arguments: [customer_sk#15 ASC NULLS FIRST, item_sk#16 ASC NULLS FIRST], false, 0 (23) SortMergeJoin Left keys [2]: [customer_sk#9, item_sk#10] -Right keys [2]: [customer_sk#16, item_sk#17] +Right keys [2]: [customer_sk#15, item_sk#16] Join condition: None (24) Project [codegen id : 7] -Output [2]: [customer_sk#9, customer_sk#16] -Input [4]: [customer_sk#9, item_sk#10, customer_sk#16, item_sk#17] +Output [2]: [customer_sk#9, customer_sk#15] +Input [4]: [customer_sk#9, item_sk#10, customer_sk#15, item_sk#16] (25) HashAggregate [codegen id : 7] -Input [2]: [customer_sk#9, customer_sk#16] +Input [2]: [customer_sk#9, customer_sk#15] Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum#18, sum#19, sum#20] -Results [3]: [sum#21, sum#22, sum#23] +Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum#17, sum#18, sum#19] +Results [3]: [sum#20, sum#21, sum#22] (26) Exchange -Input [3]: [sum#21, sum#22, sum#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [sum#20, sum#21, sum#22] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#23] (27) HashAggregate [codegen id : 8] -Input [3]: [sum#21, sum#22, sum#23] +Input [3]: [sum#20, sum#21, sum#22] Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27] -Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25 AS store_only#28, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26 AS catalog_only#29, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27 AS store_and_catalog#30] +Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26] +Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24 AS store_only#27, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25 AS catalog_only#28, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26 AS store_and_catalog#29] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt index 2e710b7eda4b3..58d28b28dae10 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt @@ -137,52 +137,52 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#8] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] +Right keys [1]: [d_date_sk#8] Join condition: None (22) Project [codegen id : 7] -Output [1]: [ws_bill_customer_sk#13 AS customer_sk#16] -Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] +Output [1]: [ws_bill_customer_sk#13 AS customer_sk#15] +Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] (23) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] ReadSchema: struct (24) ColumnarToRow [codegen id : 9] -Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] (25) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#19] +Output [1]: [d_date_sk#8] (26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#19] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#8] Join condition: None (27) Project [codegen id : 9] -Output [1]: [cs_ship_customer_sk#17 AS customer_sk#20] -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] +Output [1]: [cs_ship_customer_sk#16 AS customer_sk#18] +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] (28) Union (29) Exchange -Input [1]: [customer_sk#16] -Arguments: hashpartitioning(customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [1]: [customer_sk#15] +Arguments: hashpartitioning(customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#19] (30) Sort [codegen id : 10] -Input [1]: [customer_sk#16] -Arguments: [customer_sk#16 ASC NULLS FIRST], false, 0 +Input [1]: [customer_sk#15] +Arguments: [customer_sk#15 ASC NULLS FIRST], false, 0 (31) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#16] +Right keys [1]: [customer_sk#15] Join condition: None (32) Project [codegen id : 12] @@ -190,84 +190,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (33) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_county#23] +Output [2]: [ca_address_sk#20, ca_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -Input [2]: [ca_address_sk#22, ca_county#23] +Input [2]: [ca_address_sk#20, ca_county#21] (35) Filter [codegen id : 11] -Input [2]: [ca_address_sk#22, ca_county#23] -Condition : (ca_county#23 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#22)) +Input [2]: [ca_address_sk#20, ca_county#21] +Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) (36) Project [codegen id : 11] -Output [1]: [ca_address_sk#22] -Input [2]: [ca_address_sk#22, ca_county#23] +Output [1]: [ca_address_sk#20] +Input [2]: [ca_address_sk#20, ca_county#21] (37) 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#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (38) BroadcastHashJoin [codegen id : 12] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#20] Join condition: None (39) Project [codegen id : 12] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#22] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] (40) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (41) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Output [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (42) ColumnarToRow -Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] (43) Filter -Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] -Condition : isnotnull(cd_demo_sk#26) +Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Condition : isnotnull(cd_demo_sk#24) (44) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#26] +Right keys [1]: [cd_demo_sk#24] Join condition: None (45) Project [codegen id : 13] -Output [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Output [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] (46) HashAggregate [codegen id : 13] -Input [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] -Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Input [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#35] -Results [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] +Aggregate Attributes [1]: [count#33] +Results [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] (47) Exchange -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] -Arguments: hashpartitioning(cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] +Arguments: hashpartitioning(cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#35] (48) HashAggregate [codegen id : 14] -Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] -Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] +Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, count(1)#38 AS cnt1#39, cd_purchase_estimate#30, count(1)#38 AS cnt2#40, cd_credit_rating#31, count(1)#38 AS cnt3#41, cd_dep_count#32, count(1)#38 AS cnt4#42, cd_dep_employed_count#33, count(1)#38 AS cnt5#43, cd_dep_college_count#34, count(1)#38 AS cnt6#44] +Aggregate Attributes [1]: [count(1)#36] +Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count(1)#36 AS cnt1#37, cd_purchase_estimate#28, count(1)#36 AS cnt2#38, cd_credit_rating#29, count(1)#36 AS cnt3#39, cd_dep_count#30, count(1)#36 AS cnt4#40, cd_dep_employed_count#31, count(1)#36 AS cnt5#41, cd_dep_college_count#32, count(1)#36 AS cnt6#42] (49) TakeOrderedAndProject -Input [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#39, cd_purchase_estimate#30, cnt2#40, cd_credit_rating#31, cnt3#41, cd_dep_count#32, cnt4#42, cd_dep_employed_count#33, cnt5#43, cd_dep_college_count#34, cnt6#44] -Arguments: 100, [cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_education_status#29 ASC NULLS FIRST, cd_purchase_estimate#30 ASC NULLS FIRST, cd_credit_rating#31 ASC NULLS FIRST, cd_dep_count#32 ASC NULLS FIRST, cd_dep_employed_count#33 ASC NULLS FIRST, cd_dep_college_count#34 ASC NULLS FIRST], [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#39, cd_purchase_estimate#30, cnt2#40, cd_credit_rating#31, cnt3#41, cd_dep_count#32, cnt4#42, cd_dep_employed_count#33, cnt5#43, cd_dep_college_count#34, cnt6#44] +Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] +Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] ===== Subqueries ===== @@ -280,6 +280,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index 62054fe96dd5c..eac6acc3543ca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -121,48 +121,48 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#7] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#7] Join condition: None (19) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#12 AS customer_sk#15] -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] +Output [1]: [ws_bill_customer_sk#12 AS customer_sk#14] +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] (20) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] (22) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#7] (23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#18] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#7] Join condition: None (24) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#16 AS customer_sk#19] -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#18] +Output [1]: [cs_ship_customer_sk#15 AS customer_sk#17] +Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] (25) Union (26) BroadcastExchange -Input [1]: [customer_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +Input [1]: [customer_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (27) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#15] +Right keys [1]: [customer_sk#14] Join condition: None (28) Project [codegen id : 9] @@ -170,84 +170,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_county#22] +Output [2]: [ca_address_sk#19, ca_county#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#21, ca_county#22] +Input [2]: [ca_address_sk#19, ca_county#20] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#21, ca_county#22] -Condition : (ca_county#22 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#21)) +Input [2]: [ca_address_sk#19, ca_county#20] +Condition : (ca_county#20 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#19)) (32) Project [codegen id : 7] -Output [1]: [ca_address_sk#21] -Input [2]: [ca_address_sk#21, ca_county#22] +Output [1]: [ca_address_sk#19] +Input [2]: [ca_address_sk#19, ca_county#20] (33) BroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [ca_address_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (34) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#19] Join condition: None (35) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] (36) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (38) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Condition : isnotnull(cd_demo_sk#24) +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#22) (39) BroadcastExchange -Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#24] +Right keys [1]: [cd_demo_sk#22] Join condition: None (41) Project [codegen id : 9] -Output [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Output [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (42) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#34] -Results [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#35] +Aggregate Attributes [1]: [count#32] +Results [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] (43) Exchange -Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#35] -Arguments: hashpartitioning(cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] +Arguments: hashpartitioning(cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#34] (44) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#35] -Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] +Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#37] -Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count(1)#37 AS cnt1#38, cd_purchase_estimate#28, count(1)#37 AS cnt2#39, cd_credit_rating#29, count(1)#37 AS cnt3#40, cd_dep_count#30, count(1)#37 AS cnt4#41, cd_dep_employed_count#31, count(1)#37 AS cnt5#42, cd_dep_college_count#32, count(1)#37 AS cnt6#43] +Aggregate Attributes [1]: [count(1)#35] +Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(1)#35 AS cnt1#36, cd_purchase_estimate#26, count(1)#35 AS cnt2#37, cd_credit_rating#27, count(1)#35 AS cnt3#38, cd_dep_count#28, count(1)#35 AS cnt4#39, cd_dep_employed_count#29, count(1)#35 AS cnt5#40, cd_dep_college_count#30, count(1)#35 AS cnt6#41] (45) TakeOrderedAndProject -Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#38, cd_purchase_estimate#28, cnt2#39, cd_credit_rating#29, cnt3#40, cd_dep_count#30, cnt4#41, cd_dep_employed_count#31, cnt5#42, cd_dep_college_count#32, cnt6#43] -Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#38, cd_purchase_estimate#28, cnt2#39, cd_credit_rating#29, cnt3#40, cd_dep_count#30, cnt4#41, cd_dep_employed_count#31, cnt5#42, cd_dep_college_count#32, cnt6#43] +Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] +Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] ===== Subqueries ===== @@ -260,6 +260,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 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 453a35d8d5175..6eb3be35a05bc 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 @@ -199,285 +199,285 @@ Input [2]: [customer_id#23, year_total#24] Arguments: [customer_id#23 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] +Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] +Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] (27) Filter [codegen id : 10] -Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] -Condition : isnotnull(ss_customer_sk#26) +Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] +Condition : isnotnull(ss_customer_sk#1) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#31, d_year#32] +Output [2]: [d_date_sk#6, d_year#7] 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 (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#31, d_year#32] +Input [2]: [d_date_sk#6, d_year#7] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#31, d_year#32] -Condition : ((isnotnull(d_year#32) AND (d_year#32 = 2002)) AND isnotnull(d_date_sk#31)) +Input [2]: [d_date_sk#6, d_year#7] +Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) (31) BroadcastExchange -Input [2]: [d_date_sk#31, d_year#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +Input [2]: [d_date_sk#6, d_year#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#29] -Right keys [1]: [d_date_sk#31] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#6] Join condition: None (33) Project [codegen id : 10] -Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Input [6]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29, d_date_sk#31, d_year#32] +Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4, d_date_sk#6, d_year#7] (34) Exchange -Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#28] (35) Sort [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (37) Sort [codegen id : 13] -Input [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] -Arguments: [c_customer_sk#35 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#35] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#10] Join condition: None (39) Project [codegen id : 14] -Output [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Input [12]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32, c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Input [12]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (40) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] -Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#43] -Results [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] +Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#29] +Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] (41) Exchange -Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] -Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] +Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, 5), ENSURE_REQUIREMENTS, [id=#31] (42) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] -Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46] -Results [5]: [c_customer_id#36 AS customer_id#47, c_first_name#37 AS customer_first_name#48, c_last_name#38 AS customer_last_name#49, c_email_address#42 AS customer_email_address#50, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46,18,2) AS year_total#51] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32] +Results [5]: [c_customer_id#11 AS customer_id#33, c_first_name#12 AS customer_first_name#34, c_last_name#13 AS customer_last_name#35, c_email_address#17 AS customer_email_address#36, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32,18,2) AS year_total#37] (43) Exchange -Input [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] -Arguments: hashpartitioning(customer_id#47, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [5]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37] +Arguments: hashpartitioning(customer_id#33, 5), ENSURE_REQUIREMENTS, [id=#38] (44) Sort [codegen id : 16] -Input [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] -Arguments: [customer_id#47 ASC NULLS FIRST], false, 0 +Input [5]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37] +Arguments: [customer_id#33 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#47] +Right keys [1]: [customer_id#33] Join condition: None (46) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56] +Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#56), dynamicpruningexpression(ws_sold_date_sk#56 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56] +Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] (48) Filter [codegen id : 19] -Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56] -Condition : isnotnull(ws_bill_customer_sk#53) +Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] +Condition : isnotnull(ws_bill_customer_sk#39) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#57, d_year#58] +Output [2]: [d_date_sk#6, d_year#7] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#56] -Right keys [1]: [d_date_sk#57] +Left keys [1]: [ws_sold_date_sk#42] +Right keys [1]: [d_date_sk#6] Join condition: None (51) Project [codegen id : 19] -Output [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] -Input [6]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56, d_date_sk#57, d_year#58] +Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Input [6]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42, d_date_sk#6, d_year#7] (52) Exchange -Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] -Arguments: hashpartitioning(ws_bill_customer_sk#53, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Arguments: hashpartitioning(ws_bill_customer_sk#39, 5), ENSURE_REQUIREMENTS, [id=#43] (53) Sort [codegen id : 20] -Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] -Arguments: [ws_bill_customer_sk#53 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Arguments: [ws_bill_customer_sk#39 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -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#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (55) Sort [codegen id : 22] -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] -Arguments: [c_customer_sk#60 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#53] -Right keys [1]: [c_customer_sk#60] +Left keys [1]: [ws_bill_customer_sk#39] +Right keys [1]: [c_customer_sk#10] Join condition: None (57) Project [codegen id : 23] -Output [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, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] -Input [12]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58, 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 [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Input [12]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (58) HashAggregate [codegen id : 23] -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, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] -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#58] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#68] -Results [9]: [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#58, sum#69] +Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#44] +Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#45] (59) Exchange -Input [9]: [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#58, sum#69] -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#58, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#45] +Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#46] (60) HashAggregate [codegen id : 24] -Input [9]: [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#58, sum#69] -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#58] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))#71] -Results [2]: [c_customer_id#61 AS customer_id#72, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))#71,18,2) AS year_total#73] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#45] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#47] +Results [2]: [c_customer_id#11 AS customer_id#48, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#47,18,2) AS year_total#49] (61) Filter [codegen id : 24] -Input [2]: [customer_id#72, year_total#73] -Condition : (isnotnull(year_total#73) AND (year_total#73 > 0.00)) +Input [2]: [customer_id#48, year_total#49] +Condition : (isnotnull(year_total#49) AND (year_total#49 > 0.00)) (62) Project [codegen id : 24] -Output [2]: [customer_id#72 AS customer_id#74, year_total#73 AS year_total#75] -Input [2]: [customer_id#72, year_total#73] +Output [2]: [customer_id#48 AS customer_id#50, year_total#49 AS year_total#51] +Input [2]: [customer_id#48, year_total#49] (63) Exchange -Input [2]: [customer_id#74, year_total#75] -Arguments: hashpartitioning(customer_id#74, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [2]: [customer_id#50, year_total#51] +Arguments: hashpartitioning(customer_id#50, 5), ENSURE_REQUIREMENTS, [id=#52] (64) Sort [codegen id : 25] -Input [2]: [customer_id#74, year_total#75] -Arguments: [customer_id#74 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#50, year_total#51] +Arguments: [customer_id#50 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#74] +Right keys [1]: [customer_id#50] Join condition: None (66) Project [codegen id : 26] -Output [8]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75] -Input [9]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, customer_id#74, year_total#75] +Output [8]: [customer_id#23, year_total#24, customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37, year_total#51] +Input [9]: [customer_id#23, year_total#24, customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37, customer_id#50, year_total#51] (67) 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#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -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#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] (69) Filter [codegen id : 28] -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#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] +Condition : isnotnull(ws_bill_customer_sk#39) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#81, d_year#82] +Output [2]: [d_date_sk#6, d_year#7] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#80] -Right keys [1]: [d_date_sk#81] +Left keys [1]: [ws_sold_date_sk#42] +Right keys [1]: [d_date_sk#6] Join condition: None (72) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] -Input [6]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80, d_date_sk#81, d_year#82] +Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Input [6]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42, d_date_sk#6, d_year#7] (73) Exchange -Input [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] -Arguments: hashpartitioning(ws_bill_customer_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Arguments: hashpartitioning(ws_bill_customer_sk#39, 5), ENSURE_REQUIREMENTS, [id=#53] (74) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] -Arguments: [ws_bill_customer_sk#77 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Arguments: [ws_bill_customer_sk#39 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#84, c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91] +Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (76) Sort [codegen id : 31] -Input [8]: [c_customer_sk#84, c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91] -Arguments: [c_customer_sk#84 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#77] -Right keys [1]: [c_customer_sk#84] +Left keys [1]: [ws_bill_customer_sk#39] +Right keys [1]: [c_customer_sk#10] Join condition: None (78) Project [codegen id : 32] -Output [10]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] -Input [12]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82, c_customer_sk#84, c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91] +Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Input [12]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] (79) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] -Keys [8]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82] -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#92] -Results [9]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, sum#93] +Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#54] +Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#55] (80) Exchange -Input [9]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, sum#93] -Arguments: hashpartitioning(c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, 5), ENSURE_REQUIREMENTS, [id=#94] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#55] +Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#56] (81) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, sum#93] -Keys [8]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82] -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)))#95] -Results [2]: [c_customer_id#85 AS customer_id#96, 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)))#95,18,2) AS year_total#97] +Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#55] +Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#57] +Results [2]: [c_customer_id#11 AS customer_id#58, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#57,18,2) AS year_total#59] (82) Exchange -Input [2]: [customer_id#96, year_total#97] -Arguments: hashpartitioning(customer_id#96, 5), ENSURE_REQUIREMENTS, [id=#98] +Input [2]: [customer_id#58, year_total#59] +Arguments: hashpartitioning(customer_id#58, 5), ENSURE_REQUIREMENTS, [id=#60] (83) Sort [codegen id : 34] -Input [2]: [customer_id#96, year_total#97] -Arguments: [customer_id#96 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#58, year_total#59] +Arguments: [customer_id#58 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#96] -Join condition: (CASE WHEN (year_total#75 > 0.00) THEN CheckOverflow((promote_precision(year_total#97) / promote_precision(year_total#75)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#51) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE 0E-20 END) +Right keys [1]: [customer_id#58] +Join condition: (CASE WHEN (year_total#51 > 0.00) THEN CheckOverflow((promote_precision(year_total#59) / promote_precision(year_total#51)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#37) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE 0E-20 END) (85) Project [codegen id : 35] -Output [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] -Input [10]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75, customer_id#96, year_total#97] +Output [4]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36] +Input [10]: [customer_id#23, year_total#24, customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37, year_total#51, customer_id#58, year_total#59] (86) TakeOrderedAndProject -Input [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] -Arguments: 100, [customer_id#47 ASC NULLS FIRST, customer_first_name#48 ASC NULLS FIRST, customer_last_name#49 ASC NULLS FIRST, customer_email_address#50 ASC NULLS FIRST], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] +Input [4]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36] +Arguments: 100, [customer_id#33 ASC NULLS FIRST, customer_first_name#34 ASC NULLS FIRST, customer_last_name#35 ASC NULLS FIRST, customer_email_address#36 ASC NULLS FIRST], [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36] ===== Subqueries ===== @@ -488,15 +488,15 @@ ReusedExchange (87) (87) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#6, d_year#7] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#30 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#26 ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#31, d_year#32] +Output [2]: [d_date_sk#6, d_year#7] -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#56 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#30 +Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#26 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 17d16033bb5be..8f7b2faaa1220 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 @@ -171,282 +171,282 @@ Input [2]: [customer_id#22, year_total#23] Condition : (isnotnull(year_total#23) AND (year_total#23 > 0.00)) (20) Scan parquet default.customer -Output [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] -Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_customer_id#25)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (23) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] 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#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] (25) Filter [codegen id : 4] -Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] -Condition : isnotnull(ss_customer_sk#32) +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_customer_sk#9) (26) BroadcastExchange -Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#24] -Right keys [1]: [ss_customer_sk#32] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#9] Join condition: None (28) Project [codegen id : 6] -Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] -Input [12]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#38, d_year#39] +Output [2]: [d_date_sk#15, d_year#16] 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 (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#38, d_year#39] +Input [2]: [d_date_sk#15, d_year#16] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#38, d_year#39] -Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) +Input [2]: [d_date_sk#15, d_year#16] +Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2002)) AND isnotnull(d_date_sk#15)) (32) BroadcastExchange -Input [2]: [d_date_sk#38, d_year#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] +Input [2]: [d_date_sk#15, d_year#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#35] -Right keys [1]: [d_date_sk#38] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#15] Join condition: None (34) Project [codegen id : 6] -Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] -Input [12]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#38, d_year#39] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#15, d_year#16] (35) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] -Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#41] -Results [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#27] +Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] (36) Exchange -Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] -Arguments: hashpartitioning(c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [id=#29] (37) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] -Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44] -Results [5]: [c_customer_id#25 AS customer_id#45, c_first_name#26 AS customer_first_name#46, c_last_name#27 AS customer_last_name#47, c_email_address#31 AS customer_email_address#48, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44,18,2) AS year_total#49] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30] +Results [5]: [c_customer_id#2 AS customer_id#31, c_first_name#3 AS customer_first_name#32, c_last_name#4 AS customer_last_name#33, c_email_address#8 AS customer_email_address#34, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30,18,2) AS year_total#35] (38) BroadcastExchange -Input [5]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] +Input [5]: [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#36] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#45] +Right keys [1]: [customer_id#31] Join condition: None (40) Scan parquet default.customer -Output [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (42) Filter [codegen id : 10] -Input [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] -Condition : (isnotnull(c_customer_sk#51) AND isnotnull(c_customer_id#52)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (43) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] +Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] (45) Filter [codegen id : 8] -Input [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] -Condition : isnotnull(ws_bill_customer_sk#59) +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Condition : isnotnull(ws_bill_customer_sk#37) (46) BroadcastExchange -Input [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#41] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#51] -Right keys [1]: [ws_bill_customer_sk#59] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#37] Join condition: None (48) Project [codegen id : 10] -Output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] -Input [12]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#64, d_year#65] +Output [2]: [d_date_sk#15, d_year#16] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#62] -Right keys [1]: [d_date_sk#64] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#15] Join condition: None (51) Project [codegen id : 10] -Output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, d_year#65] -Input [12]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62, d_date_sk#64, d_year#65] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#15, d_year#16] (52) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, d_year#65] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#66] -Results [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, sum#67] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#42] +Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#43] (53) Exchange -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, sum#67] -Arguments: hashpartitioning(c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#43] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#44] (54) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, sum#67] -Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))#69] -Results [2]: [c_customer_id#52 AS customer_id#70, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))#69,18,2) AS year_total#71] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#43] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45] +Results [2]: [c_customer_id#2 AS customer_id#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45,18,2) AS year_total#47] (55) Filter [codegen id : 11] -Input [2]: [customer_id#70, year_total#71] -Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.00)) +Input [2]: [customer_id#46, year_total#47] +Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) (56) Project [codegen id : 11] -Output [2]: [customer_id#70 AS customer_id#72, year_total#71 AS year_total#73] -Input [2]: [customer_id#70, year_total#71] +Output [2]: [customer_id#46 AS customer_id#48, year_total#47 AS year_total#49] +Input [2]: [customer_id#46, year_total#47] (57) BroadcastExchange -Input [2]: [customer_id#72, year_total#73] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#74] +Input [2]: [customer_id#48, year_total#49] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] (58) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#72] +Right keys [1]: [customer_id#48] Join condition: None (59) Project [codegen id : 16] -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#73] -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#72, year_total#73] +Output [8]: [customer_id#22, year_total#23, customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35, year_total#49] +Input [9]: [customer_id#22, year_total#23, customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35, customer_id#48, year_total#49] (60) Scan parquet default.customer -Output [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] (62) Filter [codegen id : 14] -Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] -Condition : (isnotnull(c_customer_sk#75) AND isnotnull(c_customer_id#76)) +Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (63) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] (65) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] -Condition : isnotnull(ws_bill_customer_sk#83) +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Condition : isnotnull(ws_bill_customer_sk#37) (66) BroadcastExchange -Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#87] +Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#75] -Right keys [1]: [ws_bill_customer_sk#83] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#37] Join condition: None (68) Project [codegen id : 14] -Output [10]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] -Input [12]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#88, d_year#89] +Output [2]: [d_date_sk#15, d_year#16] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#86] -Right keys [1]: [d_date_sk#88] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#15] Join condition: None (71) Project [codegen id : 14] -Output [10]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] -Input [12]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86, d_date_sk#88, d_year#89] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#15, d_year#16] (72) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] -Keys [8]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#90] -Results [9]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, sum#91] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#52] +Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#53] (73) Exchange -Input [9]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, sum#91] -Arguments: hashpartitioning(c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#53] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#54] (74) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, sum#91] -Keys [8]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))#93] -Results [2]: [c_customer_id#76 AS customer_id#94, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))#93,18,2) AS year_total#95] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#53] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55] +Results [2]: [c_customer_id#2 AS customer_id#56, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55,18,2) AS year_total#57] (75) BroadcastExchange -Input [2]: [customer_id#94, year_total#95] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#96] +Input [2]: [customer_id#56, year_total#57] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#58] (76) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#94] -Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#95) / promote_precision(year_total#73)), 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#56] +Join condition: (CASE WHEN (year_total#49 > 0.00) THEN CheckOverflow((promote_precision(year_total#57) / promote_precision(year_total#49)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#35) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE 0E-20 END) (77) Project [codegen id : 16] -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#73, customer_id#94, year_total#95] +Output [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34] +Input [10]: [customer_id#22, year_total#23, customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35, year_total#49, customer_id#56, year_total#57] (78) 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#31, customer_first_name#32, customer_last_name#33, customer_email_address#34] +Arguments: 100, [customer_id#31 ASC NULLS FIRST, customer_first_name#32 ASC NULLS FIRST, customer_last_name#33 ASC NULLS FIRST, customer_email_address#34 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34] ===== Subqueries ===== @@ -457,15 +457,15 @@ ReusedExchange (79) (79) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#15, d_year#16] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#24 ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#38, d_year#39] +Output [2]: [d_date_sk#15, d_year#16] -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#36 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#24 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 56fa48006d894..15f74a7802b27 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 @@ -393,484 +393,484 @@ Aggregate Attributes [7]: [avg(agg1#37)#73, avg(agg2#38)#74, avg(agg3#39)#75, av Results [11]: [i_item_id#20, ca_country#34, ca_state#33, ca_county#32, avg(agg1#37)#73 AS agg1#80, avg(agg2#38)#74 AS agg2#81, avg(agg3#39)#75 AS agg3#82, avg(agg4#40)#76 AS agg4#83, avg(agg5#41)#77 AS agg5#84, avg(agg6#42)#78 AS agg6#85, avg(agg7#43)#79 AS agg7#86] (52) ReusedExchange [Reuses operator id: unknown] -Output [8]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94] +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#20] (53) Sort [codegen id : 19] -Input [8]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94] -Arguments: [cs_bill_customer_sk#87 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#20] +Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (54) Scan parquet default.customer -Output [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] +Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 21] -Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] (56) Filter [codegen id : 21] -Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] -Condition : (((c_birth_month#98 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#95)) AND isnotnull(c_current_cdemo_sk#96)) AND isnotnull(c_current_addr_sk#97)) +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) (57) Project [codegen id : 21] -Output [4]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#99] -Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] +Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] (58) Scan parquet default.customer_address -Output [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Output [3]: [ca_address_sk#31, ca_state#33, ca_country#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] (60) Filter [codegen id : 20] -Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] -Condition : (ca_state#101 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#100)) +Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Condition : (ca_state#33 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#31)) (61) BroadcastExchange -Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] +Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#87] (62) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [c_current_addr_sk#97] -Right keys [1]: [ca_address_sk#100] +Left keys [1]: [c_current_addr_sk#25] +Right keys [1]: [ca_address_sk#31] Join condition: None (63) Project [codegen id : 21] -Output [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] -Input [7]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#99, ca_address_sk#100, ca_state#101, ca_country#102] +Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34] +Input [7]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31, ca_state#33, ca_country#34] (64) Exchange -Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] -Arguments: hashpartitioning(c_current_cdemo_sk#96, 5), ENSURE_REQUIREMENTS, [id=#104] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34] +Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#88] (65) Sort [codegen id : 22] -Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] -Arguments: [c_current_cdemo_sk#96 ASC NULLS FIRST], false, 0 +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34] +Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 (66) ReusedExchange [Reuses operator id: 35] -Output [1]: [cd_demo_sk#105] +Output [1]: [cd_demo_sk#89] (67) Sort [codegen id : 24] -Input [1]: [cd_demo_sk#105] -Arguments: [cd_demo_sk#105 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#89] +Arguments: [cd_demo_sk#89 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 25] -Left keys [1]: [c_current_cdemo_sk#96] -Right keys [1]: [cd_demo_sk#105] +Left keys [1]: [c_current_cdemo_sk#24] +Right keys [1]: [cd_demo_sk#89] Join condition: None (69) Project [codegen id : 25] -Output [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] -Input [6]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102, cd_demo_sk#105] +Output [4]: [c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] +Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34, cd_demo_sk#89] (70) Exchange -Input [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] -Arguments: hashpartitioning(c_customer_sk#95, 5), ENSURE_REQUIREMENTS, [id=#106] +Input [4]: [c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#90] (71) Sort [codegen id : 26] -Input [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] -Arguments: [c_customer_sk#95 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] +Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 (72) SortMergeJoin [codegen id : 27] -Left keys [1]: [cs_bill_customer_sk#87] -Right keys [1]: [c_customer_sk#95] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#23] Join condition: None (73) Project [codegen id : 27] -Output [10]: [i_item_id#94, ca_country#102, ca_state#101, cast(cs_quantity#88 as decimal(12,2)) AS agg1#37, cast(cs_list_price#89 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#91 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#90 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#92 as decimal(12,2)) AS agg5#41, cast(c_birth_year#99 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#93 as decimal(12,2)) AS agg7#43] -Input [12]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94, c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Output [10]: [i_item_id#20, ca_country#34, ca_state#33, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] +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#20, c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] (74) HashAggregate [codegen id : 27] -Input [10]: [i_item_id#94, ca_country#102, ca_state#101, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] -Keys [3]: [i_item_id#94, ca_country#102, ca_state#101] +Input [10]: [i_item_id#20, ca_country#34, ca_state#33, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [3]: [i_item_id#20, ca_country#34, ca_state#33] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] -Results [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] +Aggregate Attributes [14]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100, sum#101, count#102, sum#103, count#104] +Results [17]: [i_item_id#20, ca_country#34, ca_state#33, 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] (75) Exchange -Input [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] -Arguments: hashpartitioning(i_item_id#94, ca_country#102, ca_state#101, 5), ENSURE_REQUIREMENTS, [id=#135] +Input [17]: [i_item_id#20, ca_country#34, ca_state#33, 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] +Arguments: hashpartitioning(i_item_id#20, ca_country#34, ca_state#33, 5), ENSURE_REQUIREMENTS, [id=#119] (76) HashAggregate [codegen id : 28] -Input [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] -Keys [3]: [i_item_id#94, ca_country#102, ca_state#101] +Input [17]: [i_item_id#20, ca_country#34, ca_state#33, 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] +Keys [3]: [i_item_id#20, ca_country#34, ca_state#33] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#136, avg(agg2#38)#137, avg(agg3#39)#138, avg(agg4#40)#139, avg(agg5#41)#140, avg(agg6#42)#141, avg(agg7#43)#142] -Results [11]: [i_item_id#94, ca_country#102, ca_state#101, null AS county#143, avg(agg1#37)#136 AS agg1#144, avg(agg2#38)#137 AS agg2#145, avg(agg3#39)#138 AS agg3#146, avg(agg4#40)#139 AS agg4#147, avg(agg5#41)#140 AS agg5#148, avg(agg6#42)#141 AS agg6#149, avg(agg7#43)#142 AS agg7#150] +Aggregate Attributes [7]: [avg(agg1#37)#120, avg(agg2#38)#121, avg(agg3#39)#122, avg(agg4#40)#123, avg(agg5#41)#124, avg(agg6#42)#125, avg(agg7#43)#126] +Results [11]: [i_item_id#20, ca_country#34, ca_state#33, null AS county#127, avg(agg1#37)#120 AS agg1#128, avg(agg2#38)#121 AS agg2#129, avg(agg3#39)#122 AS agg3#130, avg(agg4#40)#123 AS agg4#131, avg(agg5#41)#124 AS agg5#132, avg(agg6#42)#125 AS agg6#133, avg(agg7#43)#126 AS agg7#134] (77) ReusedExchange [Reuses operator id: unknown] -Output [8]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158] +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#20] (78) Sort [codegen id : 33] -Input [8]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158] -Arguments: [cs_bill_customer_sk#151 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#20] +Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer -Output [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] +Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (80) ColumnarToRow [codegen id : 35] -Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] (81) Filter [codegen id : 35] -Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] -Condition : (((c_birth_month#162 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#159)) AND isnotnull(c_current_cdemo_sk#160)) AND isnotnull(c_current_addr_sk#161)) +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) (82) Project [codegen id : 35] -Output [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_year#163] -Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] +Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] (83) Scan parquet default.customer_address -Output [3]: [ca_address_sk#164, ca_state#165, ca_country#166] +Output [3]: [ca_address_sk#31, ca_state#33, ca_country#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (84) ColumnarToRow [codegen id : 34] -Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] +Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] (85) Filter [codegen id : 34] -Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] -Condition : (ca_state#165 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#164)) +Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Condition : (ca_state#33 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#31)) (86) Project [codegen id : 34] -Output [2]: [ca_address_sk#164, ca_country#166] -Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] +Output [2]: [ca_address_sk#31, ca_country#34] +Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] (87) BroadcastExchange -Input [2]: [ca_address_sk#164, ca_country#166] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#167] +Input [2]: [ca_address_sk#31, ca_country#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#135] (88) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [c_current_addr_sk#161] -Right keys [1]: [ca_address_sk#164] +Left keys [1]: [c_current_addr_sk#25] +Right keys [1]: [ca_address_sk#31] Join condition: None (89) Project [codegen id : 35] -Output [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] -Input [6]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_year#163, ca_address_sk#164, ca_country#166] +Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34] +Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31, ca_country#34] (90) Exchange -Input [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] -Arguments: hashpartitioning(c_current_cdemo_sk#160, 5), ENSURE_REQUIREMENTS, [id=#168] +Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34] +Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#136] (91) Sort [codegen id : 36] -Input [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] -Arguments: [c_current_cdemo_sk#160 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34] +Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 (92) ReusedExchange [Reuses operator id: 35] -Output [1]: [cd_demo_sk#169] +Output [1]: [cd_demo_sk#137] (93) Sort [codegen id : 38] -Input [1]: [cd_demo_sk#169] -Arguments: [cd_demo_sk#169 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#137] +Arguments: [cd_demo_sk#137 ASC NULLS FIRST], false, 0 (94) SortMergeJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#160] -Right keys [1]: [cd_demo_sk#169] +Left keys [1]: [c_current_cdemo_sk#24] +Right keys [1]: [cd_demo_sk#137] Join condition: None (95) Project [codegen id : 39] -Output [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] -Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166, cd_demo_sk#169] +Output [3]: [c_customer_sk#23, c_birth_year#27, ca_country#34] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34, cd_demo_sk#137] (96) Exchange -Input [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] -Arguments: hashpartitioning(c_customer_sk#159, 5), ENSURE_REQUIREMENTS, [id=#170] +Input [3]: [c_customer_sk#23, c_birth_year#27, ca_country#34] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#138] (97) Sort [codegen id : 40] -Input [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] -Arguments: [c_customer_sk#159 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#23, c_birth_year#27, ca_country#34] +Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 41] -Left keys [1]: [cs_bill_customer_sk#151] -Right keys [1]: [c_customer_sk#159] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#23] Join condition: None (99) Project [codegen id : 41] -Output [9]: [i_item_id#158, ca_country#166, cast(cs_quantity#152 as decimal(12,2)) AS agg1#37, cast(cs_list_price#153 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#155 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#154 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#156 as decimal(12,2)) AS agg5#41, cast(c_birth_year#163 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#157 as decimal(12,2)) AS agg7#43] -Input [11]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158, c_customer_sk#159, c_birth_year#163, ca_country#166] +Output [9]: [i_item_id#20, ca_country#34, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] +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#20, c_customer_sk#23, c_birth_year#27, ca_country#34] (100) HashAggregate [codegen id : 41] -Input [9]: [i_item_id#158, ca_country#166, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] -Keys [2]: [i_item_id#158, ca_country#166] +Input [9]: [i_item_id#20, ca_country#34, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [2]: [i_item_id#20, ca_country#34] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178, sum#179, count#180, sum#181, count#182, sum#183, count#184] -Results [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] +Aggregate Attributes [14]: [sum#139, count#140, sum#141, count#142, sum#143, count#144, sum#145, count#146, sum#147, count#148, sum#149, count#150, sum#151, count#152] +Results [16]: [i_item_id#20, ca_country#34, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] (101) Exchange -Input [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] -Arguments: hashpartitioning(i_item_id#158, ca_country#166, 5), ENSURE_REQUIREMENTS, [id=#199] +Input [16]: [i_item_id#20, ca_country#34, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] +Arguments: hashpartitioning(i_item_id#20, ca_country#34, 5), ENSURE_REQUIREMENTS, [id=#167] (102) HashAggregate [codegen id : 42] -Input [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] -Keys [2]: [i_item_id#158, ca_country#166] +Input [16]: [i_item_id#20, ca_country#34, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] +Keys [2]: [i_item_id#20, ca_country#34] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#200, avg(agg2#38)#201, avg(agg3#39)#202, avg(agg4#40)#203, avg(agg5#41)#204, avg(agg6#42)#205, avg(agg7#43)#206] -Results [11]: [i_item_id#158, ca_country#166, null AS ca_state#207, null AS county#208, avg(agg1#37)#200 AS agg1#209, avg(agg2#38)#201 AS agg2#210, avg(agg3#39)#202 AS agg3#211, avg(agg4#40)#203 AS agg4#212, avg(agg5#41)#204 AS agg5#213, avg(agg6#42)#205 AS agg6#214, avg(agg7#43)#206 AS agg7#215] +Aggregate Attributes [7]: [avg(agg1#37)#168, avg(agg2#38)#169, avg(agg3#39)#170, avg(agg4#40)#171, avg(agg5#41)#172, avg(agg6#42)#173, avg(agg7#43)#174] +Results [11]: [i_item_id#20, ca_country#34, null AS ca_state#175, null AS county#176, avg(agg1#37)#168 AS agg1#177, avg(agg2#38)#169 AS agg2#178, avg(agg3#39)#170 AS agg3#179, avg(agg4#40)#171 AS agg4#180, avg(agg5#41)#172 AS agg5#181, avg(agg6#42)#173 AS agg6#182, avg(agg7#43)#174 AS agg7#183] (103) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] +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#224), dynamicpruningexpression(cs_sold_date_sk#224 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 (104) ColumnarToRow [codegen id : 49] -Input [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] +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] (105) Filter [codegen id : 49] -Input [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] -Condition : ((isnotnull(cs_bill_cdemo_sk#217) AND isnotnull(cs_bill_customer_sk#216)) AND isnotnull(cs_item_sk#218)) +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)) (106) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#225, cd_dep_count#226] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (107) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_cdemo_sk#217] -Right keys [1]: [cd_demo_sk#225] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (108) Project [codegen id : 49] -Output [9]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_dep_count#226] -Input [11]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_demo_sk#225, cd_dep_count#226] +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] (109) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#227] +Output [1]: [d_date_sk#16] (110) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_sold_date_sk#224] -Right keys [1]: [d_date_sk#227] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (111) Project [codegen id : 49] -Output [8]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226] -Input [10]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_dep_count#226, d_date_sk#227] +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] (112) Scan parquet default.customer -Output [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] +Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (113) ColumnarToRow [codegen id : 46] -Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] (114) Filter [codegen id : 46] -Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] -Condition : (((c_birth_month#231 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#228)) AND isnotnull(c_current_cdemo_sk#229)) AND isnotnull(c_current_addr_sk#230)) +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) (115) Project [codegen id : 46] -Output [4]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_year#232] -Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] +Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] (116) Scan parquet default.customer_address -Output [2]: [ca_address_sk#233, ca_state#234] +Output [2]: [ca_address_sk#31, ca_state#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (117) ColumnarToRow [codegen id : 45] -Input [2]: [ca_address_sk#233, ca_state#234] +Input [2]: [ca_address_sk#31, ca_state#33] (118) Filter [codegen id : 45] -Input [2]: [ca_address_sk#233, ca_state#234] -Condition : (ca_state#234 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#233)) +Input [2]: [ca_address_sk#31, ca_state#33] +Condition : (ca_state#33 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#31)) (119) Project [codegen id : 45] -Output [1]: [ca_address_sk#233] -Input [2]: [ca_address_sk#233, ca_state#234] +Output [1]: [ca_address_sk#31] +Input [2]: [ca_address_sk#31, ca_state#33] (120) BroadcastExchange -Input [1]: [ca_address_sk#233] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#235] +Input [1]: [ca_address_sk#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#184] (121) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [c_current_addr_sk#230] -Right keys [1]: [ca_address_sk#233] +Left keys [1]: [c_current_addr_sk#25] +Right keys [1]: [ca_address_sk#31] Join condition: None (122) Project [codegen id : 46] -Output [3]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232] -Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_year#232, ca_address_sk#233] +Output [3]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31] (123) BroadcastExchange -Input [3]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#236] +Input [3]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#185] (124) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#237] +Output [1]: [cd_demo_sk#186] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (125) ColumnarToRow -Input [1]: [cd_demo_sk#237] +Input [1]: [cd_demo_sk#186] (126) Filter -Input [1]: [cd_demo_sk#237] -Condition : isnotnull(cd_demo_sk#237) +Input [1]: [cd_demo_sk#186] +Condition : isnotnull(cd_demo_sk#186) (127) BroadcastHashJoin [codegen id : 47] -Left keys [1]: [c_current_cdemo_sk#229] -Right keys [1]: [cd_demo_sk#237] +Left keys [1]: [c_current_cdemo_sk#24] +Right keys [1]: [cd_demo_sk#186] Join condition: None (128) Project [codegen id : 47] -Output [2]: [c_customer_sk#228, c_birth_year#232] -Input [4]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232, cd_demo_sk#237] +Output [2]: [c_customer_sk#23, c_birth_year#27] +Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, cd_demo_sk#186] (129) BroadcastExchange -Input [2]: [c_customer_sk#228, c_birth_year#232] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#238] +Input [2]: [c_customer_sk#23, c_birth_year#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#187] (130) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_customer_sk#216] -Right keys [1]: [c_customer_sk#228] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#23] Join condition: None (131) Project [codegen id : 49] -Output [8]: [cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_birth_year#232] -Input [10]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_customer_sk#228, c_birth_year#232] +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#27] +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#23, c_birth_year#27] (132) ReusedExchange [Reuses operator id: 21] -Output [2]: [i_item_sk#239, i_item_id#240] +Output [2]: [i_item_sk#19, i_item_id#20] (133) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_item_sk#218] -Right keys [1]: [i_item_sk#239] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#19] Join condition: None (134) Project [codegen id : 49] -Output [8]: [i_item_id#240, cast(cs_quantity#219 as decimal(12,2)) AS agg1#37, cast(cs_list_price#220 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#222 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#221 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#223 as decimal(12,2)) AS agg5#41, cast(c_birth_year#232 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#226 as decimal(12,2)) AS agg7#43] -Input [10]: [cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_birth_year#232, i_item_sk#239, i_item_id#240] +Output [8]: [i_item_id#20, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] +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#27, i_item_sk#19, i_item_id#20] (135) HashAggregate [codegen id : 49] -Input [8]: [i_item_id#240, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] -Keys [1]: [i_item_id#240] +Input [8]: [i_item_id#20, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [1]: [i_item_id#20] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#241, count#242, sum#243, count#244, sum#245, count#246, sum#247, count#248, sum#249, count#250, sum#251, count#252, sum#253, count#254] -Results [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] +Aggregate Attributes [14]: [sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201] +Results [15]: [i_item_id#20, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215] (136) Exchange -Input [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] -Arguments: hashpartitioning(i_item_id#240, 5), ENSURE_REQUIREMENTS, [id=#269] +Input [15]: [i_item_id#20, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215] +Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, [id=#216] (137) HashAggregate [codegen id : 50] -Input [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] -Keys [1]: [i_item_id#240] +Input [15]: [i_item_id#20, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215] +Keys [1]: [i_item_id#20] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#270, avg(agg2#38)#271, avg(agg3#39)#272, avg(agg4#40)#273, avg(agg5#41)#274, avg(agg6#42)#275, avg(agg7#43)#276] -Results [11]: [i_item_id#240, null AS ca_country#277, null AS ca_state#278, null AS county#279, avg(agg1#37)#270 AS agg1#280, avg(agg2#38)#271 AS agg2#281, avg(agg3#39)#272 AS agg3#282, avg(agg4#40)#273 AS agg4#283, avg(agg5#41)#274 AS agg5#284, avg(agg6#42)#275 AS agg6#285, avg(agg7#43)#276 AS agg7#286] +Aggregate Attributes [7]: [avg(agg1#37)#217, avg(agg2#38)#218, avg(agg3#39)#219, avg(agg4#40)#220, avg(agg5#41)#221, avg(agg6#42)#222, avg(agg7#43)#223] +Results [11]: [i_item_id#20, null AS ca_country#224, null AS ca_state#225, null AS county#226, avg(agg1#37)#217 AS agg1#227, avg(agg2#38)#218 AS agg2#228, avg(agg3#39)#219 AS agg3#229, avg(agg4#40)#220 AS agg4#230, avg(agg5#41)#221 AS agg5#231, avg(agg6#42)#222 AS agg6#232, avg(agg7#43)#223 AS agg7#233] (138) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] +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#295), dynamicpruningexpression(cs_sold_date_sk#295 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 (139) ColumnarToRow [codegen id : 57] -Input [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] +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] (140) Filter [codegen id : 57] -Input [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] -Condition : ((isnotnull(cs_bill_cdemo_sk#288) AND isnotnull(cs_bill_customer_sk#287)) AND isnotnull(cs_item_sk#289)) +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)) (141) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#296, cd_dep_count#297] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (142) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_cdemo_sk#288] -Right keys [1]: [cd_demo_sk#296] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (143) Project [codegen id : 57] -Output [9]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_dep_count#297] -Input [11]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_demo_sk#296, cd_dep_count#297] +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] (144) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#298] +Output [1]: [d_date_sk#16] (145) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_sold_date_sk#295] -Right keys [1]: [d_date_sk#298] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (146) Project [codegen id : 57] -Output [8]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297] -Input [10]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_dep_count#297, d_date_sk#298] +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] (147) Scan parquet default.item -Output [1]: [i_item_sk#299] +Output [1]: [i_item_sk#19] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (148) ColumnarToRow [codegen id : 53] -Input [1]: [i_item_sk#299] +Input [1]: [i_item_sk#19] (149) Filter [codegen id : 53] -Input [1]: [i_item_sk#299] -Condition : isnotnull(i_item_sk#299) +Input [1]: [i_item_sk#19] +Condition : isnotnull(i_item_sk#19) (150) BroadcastExchange -Input [1]: [i_item_sk#299] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#300] +Input [1]: [i_item_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#234] (151) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_item_sk#289] -Right keys [1]: [i_item_sk#299] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#19] Join condition: None (152) Project [codegen id : 57] -Output [7]: [cs_bill_customer_sk#287, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297] -Input [9]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297, i_item_sk#299] +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#19] (153) ReusedExchange [Reuses operator id: 129] -Output [2]: [c_customer_sk#301, c_birth_year#302] +Output [2]: [c_customer_sk#23, c_birth_year#27] (154) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_customer_sk#287] -Right keys [1]: [c_customer_sk#301] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#23] Join condition: None (155) Project [codegen id : 57] -Output [7]: [cast(cs_quantity#290 as decimal(12,2)) AS agg1#37, cast(cs_list_price#291 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#293 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#292 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#294 as decimal(12,2)) AS agg5#41, cast(c_birth_year#302 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#297 as decimal(12,2)) AS agg7#43] -Input [9]: [cs_bill_customer_sk#287, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297, c_customer_sk#301, c_birth_year#302] +Output [7]: [cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] +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#23, c_birth_year#27] (156) HashAggregate [codegen id : 57] Input [7]: [agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] Keys: [] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#303, count#304, sum#305, count#306, sum#307, count#308, sum#309, count#310, sum#311, count#312, sum#313, count#314, sum#315, count#316] -Results [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] +Aggregate Attributes [14]: [sum#235, count#236, sum#237, count#238, sum#239, count#240, sum#241, count#242, sum#243, count#244, sum#245, count#246, sum#247, count#248] +Results [14]: [sum#249, count#250, sum#251, count#252, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262] (157) Exchange -Input [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#331] +Input [14]: [sum#249, count#250, sum#251, count#252, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#263] (158) HashAggregate [codegen id : 58] -Input [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] +Input [14]: [sum#249, count#250, sum#251, count#252, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262] Keys: [] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#332, avg(agg2#38)#333, avg(agg3#39)#334, avg(agg4#40)#335, avg(agg5#41)#336, avg(agg6#42)#337, avg(agg7#43)#338] -Results [11]: [null AS i_item_id#339, null AS ca_country#340, null AS ca_state#341, null AS county#342, avg(agg1#37)#332 AS agg1#343, avg(agg2#38)#333 AS agg2#344, avg(agg3#39)#334 AS agg3#345, avg(agg4#40)#335 AS agg4#346, avg(agg5#41)#336 AS agg5#347, avg(agg6#42)#337 AS agg6#348, avg(agg7#43)#338 AS agg7#349] +Aggregate Attributes [7]: [avg(agg1#37)#264, avg(agg2#38)#265, avg(agg3#39)#266, avg(agg4#40)#267, avg(agg5#41)#268, avg(agg6#42)#269, avg(agg7#43)#270] +Results [11]: [null AS i_item_id#271, null AS ca_country#272, null AS ca_state#273, null AS county#274, avg(agg1#37)#264 AS agg1#275, avg(agg2#38)#265 AS agg2#276, avg(agg3#39)#266 AS agg3#277, avg(agg4#40)#267 AS agg4#278, avg(agg5#41)#268 AS agg5#279, avg(agg6#42)#269 AS agg6#280, avg(agg7#43)#270 AS agg7#281] (159) Union @@ -887,8 +887,8 @@ ReusedExchange (161) (161) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#16] -Subquery:2 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#224 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 138 Hosting Expression = cs_sold_date_sk#295 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 138 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 8e3c5958d2831..0cb9a331b406a 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 @@ -366,492 +366,492 @@ Aggregate Attributes [7]: [avg(agg1#35)#71, avg(agg2#36)#72, avg(agg3#37)#73, av Results [11]: [i_item_id#33, ca_country#27, ca_state#26, ca_county#25, 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] (46) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93] +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#93), dynamicpruningexpression(cs_sold_date_sk#93 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 (47) ColumnarToRow [codegen id : 15] -Input [9]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93] +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] (48) Filter [codegen id : 15] -Input [9]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93] -Condition : ((isnotnull(cs_bill_cdemo_sk#86) AND isnotnull(cs_bill_customer_sk#85)) AND isnotnull(cs_item_sk#87)) +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)) (49) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#94, cd_dep_count#95] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (50) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_cdemo_sk#86] -Right keys [1]: [cd_demo_sk#94] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (51) Project [codegen id : 15] -Output [9]: [cs_bill_customer_sk#85, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95] -Input [11]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_demo_sk#94, cd_dep_count#95] +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] (52) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#96, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99] +Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] (53) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_customer_sk#85] -Right keys [1]: [c_customer_sk#96] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#16] Join condition: None (54) Project [codegen id : 15] -Output [11]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99] -Input [13]: [cs_bill_customer_sk#85, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_customer_sk#96, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99] +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] (55) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#100] +Output [1]: [cd_demo_sk#85] (56) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_cdemo_sk#97] -Right keys [1]: [cd_demo_sk#100] +Left keys [1]: [c_current_cdemo_sk#17] +Right keys [1]: [cd_demo_sk#85] Join condition: None (57) Project [codegen id : 15] -Output [10]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_addr_sk#98, c_birth_year#99] -Input [12]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99, cd_demo_sk#100] +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#85] (58) Scan parquet default.customer_address -Output [3]: [ca_address_sk#101, ca_state#102, ca_country#103] +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, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 12] -Input [3]: [ca_address_sk#101, ca_state#102, ca_country#103] +Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] (60) Filter [codegen id : 12] -Input [3]: [ca_address_sk#101, ca_state#102, ca_country#103] -Condition : (ca_state#102 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#101)) +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)) (61) BroadcastExchange -Input [3]: [ca_address_sk#101, ca_state#102, ca_country#103] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#104] +Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#86] (62) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_addr_sk#98] -Right keys [1]: [ca_address_sk#101] +Left keys [1]: [c_current_addr_sk#18] +Right keys [1]: [ca_address_sk#24] Join condition: None (63) Project [codegen id : 15] -Output [11]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103] -Input [13]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_addr_sk#98, c_birth_year#99, ca_address_sk#101, ca_state#102, ca_country#103] +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] (64) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#105] +Output [1]: [d_date_sk#29] (65) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#93] -Right keys [1]: [d_date_sk#105] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#29] Join condition: None (66) Project [codegen id : 15] -Output [10]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103] -Input [12]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103, d_date_sk#105] +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] (67) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#106, i_item_id#107] +Output [2]: [i_item_sk#32, i_item_id#33] (68) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_item_sk#87] -Right keys [1]: [i_item_sk#106] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#32] Join condition: None (69) Project [codegen id : 15] -Output [10]: [i_item_id#107, ca_country#103, ca_state#102, cast(cs_quantity#88 as decimal(12,2)) AS agg1#35, cast(cs_list_price#89 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#91 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#90 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#92 as decimal(12,2)) AS agg5#39, cast(c_birth_year#99 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#95 as decimal(12,2)) AS agg7#41] -Input [12]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103, i_item_sk#106, i_item_id#107] +Output [10]: [i_item_id#33, ca_country#27, ca_state#26, 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#20 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] +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#32, i_item_id#33] (70) HashAggregate [codegen id : 15] -Input [10]: [i_item_id#107, ca_country#103, ca_state#102, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [3]: [i_item_id#107, ca_country#103, ca_state#102] +Input [10]: [i_item_id#33, ca_country#27, ca_state#26, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [3]: [i_item_id#33, ca_country#27, ca_state#26] 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#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117, sum#118, count#119, sum#120, count#121] -Results [17]: [i_item_id#107, ca_country#103, ca_state#102, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] +Aggregate Attributes [14]: [sum#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100] +Results [17]: [i_item_id#33, ca_country#27, ca_state#26, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] (71) Exchange -Input [17]: [i_item_id#107, ca_country#103, ca_state#102, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] -Arguments: hashpartitioning(i_item_id#107, ca_country#103, ca_state#102, 5), ENSURE_REQUIREMENTS, [id=#136] +Input [17]: [i_item_id#33, ca_country#27, ca_state#26, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] +Arguments: hashpartitioning(i_item_id#33, ca_country#27, ca_state#26, 5), ENSURE_REQUIREMENTS, [id=#115] (72) HashAggregate [codegen id : 16] -Input [17]: [i_item_id#107, ca_country#103, ca_state#102, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] -Keys [3]: [i_item_id#107, ca_country#103, ca_state#102] +Input [17]: [i_item_id#33, ca_country#27, ca_state#26, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] +Keys [3]: [i_item_id#33, ca_country#27, ca_state#26] 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)#137, avg(agg2#36)#138, avg(agg3#37)#139, avg(agg4#38)#140, avg(agg5#39)#141, avg(agg6#40)#142, avg(agg7#41)#143] -Results [11]: [i_item_id#107, ca_country#103, ca_state#102, null AS county#144, avg(agg1#35)#137 AS agg1#145, avg(agg2#36)#138 AS agg2#146, avg(agg3#37)#139 AS agg3#147, avg(agg4#38)#140 AS agg4#148, avg(agg5#39)#141 AS agg5#149, avg(agg6#40)#142 AS agg6#150, avg(agg7#41)#143 AS agg7#151] +Aggregate Attributes [7]: [avg(agg1#35)#116, avg(agg2#36)#117, avg(agg3#37)#118, avg(agg4#38)#119, avg(agg5#39)#120, avg(agg6#40)#121, avg(agg7#41)#122] +Results [11]: [i_item_id#33, ca_country#27, ca_state#26, null AS county#123, avg(agg1#35)#116 AS agg1#124, avg(agg2#36)#117 AS agg2#125, avg(agg3#37)#118 AS agg3#126, avg(agg4#38)#119 AS agg4#127, avg(agg5#39)#120 AS agg5#128, avg(agg6#40)#121 AS agg6#129, avg(agg7#41)#122 AS agg7#130] (73) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160] +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#160), dynamicpruningexpression(cs_sold_date_sk#160 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 (74) ColumnarToRow [codegen id : 23] -Input [9]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160] +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] (75) Filter [codegen id : 23] -Input [9]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160] -Condition : ((isnotnull(cs_bill_cdemo_sk#153) AND isnotnull(cs_bill_customer_sk#152)) AND isnotnull(cs_item_sk#154)) +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)) (76) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#161, cd_dep_count#162] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (77) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_cdemo_sk#153] -Right keys [1]: [cd_demo_sk#161] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (78) Project [codegen id : 23] -Output [9]: [cs_bill_customer_sk#152, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162] -Input [11]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_demo_sk#161, cd_dep_count#162] +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] (79) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#163, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] +Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] (80) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#152] -Right keys [1]: [c_customer_sk#163] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#16] Join condition: None (81) Project [codegen id : 23] -Output [11]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] -Input [13]: [cs_bill_customer_sk#152, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_customer_sk#163, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] +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] (82) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#167] +Output [1]: [cd_demo_sk#131] (83) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_cdemo_sk#164] -Right keys [1]: [cd_demo_sk#167] +Left keys [1]: [c_current_cdemo_sk#17] +Right keys [1]: [cd_demo_sk#131] Join condition: None (84) Project [codegen id : 23] -Output [10]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_addr_sk#165, c_birth_year#166] -Input [12]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166, cd_demo_sk#167] +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#131] (85) Scan parquet default.customer_address -Output [3]: [ca_address_sk#168, ca_state#169, ca_country#170] +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, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (86) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] +Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] (87) Filter [codegen id : 20] -Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] -Condition : (ca_state#169 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#168)) +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)) (88) Project [codegen id : 20] -Output [2]: [ca_address_sk#168, ca_country#170] -Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] +Output [2]: [ca_address_sk#24, ca_country#27] +Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] (89) BroadcastExchange -Input [2]: [ca_address_sk#168, ca_country#170] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#171] +Input [2]: [ca_address_sk#24, ca_country#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#132] (90) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_addr_sk#165] -Right keys [1]: [ca_address_sk#168] +Left keys [1]: [c_current_addr_sk#18] +Right keys [1]: [ca_address_sk#24] Join condition: None (91) Project [codegen id : 23] -Output [10]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_birth_year#166, ca_country#170] -Input [12]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_addr_sk#165, c_birth_year#166, ca_address_sk#168, ca_country#170] +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] (92) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#172] +Output [1]: [d_date_sk#29] (93) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_sold_date_sk#160] -Right keys [1]: [d_date_sk#172] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#29] Join condition: None (94) Project [codegen id : 23] -Output [9]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cd_dep_count#162, c_birth_year#166, ca_country#170] -Input [11]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_birth_year#166, ca_country#170, d_date_sk#172] +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] (95) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#173, i_item_id#174] +Output [2]: [i_item_sk#32, i_item_id#33] (96) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#154] -Right keys [1]: [i_item_sk#173] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#32] Join condition: None (97) Project [codegen id : 23] -Output [9]: [i_item_id#174, ca_country#170, cast(cs_quantity#155 as decimal(12,2)) AS agg1#35, cast(cs_list_price#156 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#158 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#157 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#159 as decimal(12,2)) AS agg5#39, cast(c_birth_year#166 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#162 as decimal(12,2)) AS agg7#41] -Input [11]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cd_dep_count#162, c_birth_year#166, ca_country#170, i_item_sk#173, i_item_id#174] +Output [9]: [i_item_id#33, ca_country#27, 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#20 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] +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#32, i_item_id#33] (98) HashAggregate [codegen id : 23] -Input [9]: [i_item_id#174, ca_country#170, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [2]: [i_item_id#174, ca_country#170] +Input [9]: [i_item_id#33, ca_country#27, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [2]: [i_item_id#33, ca_country#27] 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#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 [16]: [i_item_id#174, ca_country#170, 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] +Aggregate Attributes [14]: [sum#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140, sum#141, count#142, sum#143, count#144, sum#145, count#146] +Results [16]: [i_item_id#33, ca_country#27, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160] (99) Exchange -Input [16]: [i_item_id#174, ca_country#170, 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#174, ca_country#170, 5), ENSURE_REQUIREMENTS, [id=#203] +Input [16]: [i_item_id#33, ca_country#27, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160] +Arguments: hashpartitioning(i_item_id#33, ca_country#27, 5), ENSURE_REQUIREMENTS, [id=#161] (100) HashAggregate [codegen id : 24] -Input [16]: [i_item_id#174, ca_country#170, 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 [2]: [i_item_id#174, ca_country#170] +Input [16]: [i_item_id#33, ca_country#27, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160] +Keys [2]: [i_item_id#33, ca_country#27] 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)#204, avg(agg2#36)#205, avg(agg3#37)#206, avg(agg4#38)#207, avg(agg5#39)#208, avg(agg6#40)#209, avg(agg7#41)#210] -Results [11]: [i_item_id#174, ca_country#170, null AS ca_state#211, null AS county#212, avg(agg1#35)#204 AS agg1#213, avg(agg2#36)#205 AS agg2#214, avg(agg3#37)#206 AS agg3#215, avg(agg4#38)#207 AS agg4#216, avg(agg5#39)#208 AS agg5#217, avg(agg6#40)#209 AS agg6#218, avg(agg7#41)#210 AS agg7#219] +Aggregate Attributes [7]: [avg(agg1#35)#162, avg(agg2#36)#163, avg(agg3#37)#164, avg(agg4#38)#165, avg(agg5#39)#166, avg(agg6#40)#167, avg(agg7#41)#168] +Results [11]: [i_item_id#33, ca_country#27, null AS ca_state#169, null AS county#170, avg(agg1#35)#162 AS agg1#171, avg(agg2#36)#163 AS agg2#172, avg(agg3#37)#164 AS agg3#173, avg(agg4#38)#165 AS agg4#174, avg(agg5#39)#166 AS agg5#175, avg(agg6#40)#167 AS agg6#176, avg(agg7#41)#168 AS agg7#177] (101) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228] +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#228), dynamicpruningexpression(cs_sold_date_sk#228 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 (102) ColumnarToRow [codegen id : 31] -Input [9]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228] +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] (103) Filter [codegen id : 31] -Input [9]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228] -Condition : ((isnotnull(cs_bill_cdemo_sk#221) AND isnotnull(cs_bill_customer_sk#220)) AND isnotnull(cs_item_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] +Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) (104) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#229, cd_dep_count#230] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (105) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_cdemo_sk#221] -Right keys [1]: [cd_demo_sk#229] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (106) Project [codegen id : 31] -Output [9]: [cs_bill_customer_sk#220, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230] -Input [11]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_demo_sk#229, cd_dep_count#230] +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] (107) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#231, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234] +Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] (108) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_customer_sk#220] -Right keys [1]: [c_customer_sk#231] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#16] Join condition: None (109) Project [codegen id : 31] -Output [11]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234] -Input [13]: [cs_bill_customer_sk#220, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_customer_sk#231, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234] +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] (110) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#235] +Output [1]: [cd_demo_sk#178] (111) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_cdemo_sk#232] -Right keys [1]: [cd_demo_sk#235] +Left keys [1]: [c_current_cdemo_sk#17] +Right keys [1]: [cd_demo_sk#178] Join condition: None (112) Project [codegen id : 31] -Output [10]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_addr_sk#233, c_birth_year#234] -Input [12]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234, cd_demo_sk#235] +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#178] (113) Scan parquet default.customer_address -Output [2]: [ca_address_sk#236, ca_state#237] +Output [2]: [ca_address_sk#24, ca_state#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (114) ColumnarToRow [codegen id : 28] -Input [2]: [ca_address_sk#236, ca_state#237] +Input [2]: [ca_address_sk#24, ca_state#26] (115) Filter [codegen id : 28] -Input [2]: [ca_address_sk#236, ca_state#237] -Condition : (ca_state#237 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#236)) +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)) (116) Project [codegen id : 28] -Output [1]: [ca_address_sk#236] -Input [2]: [ca_address_sk#236, ca_state#237] +Output [1]: [ca_address_sk#24] +Input [2]: [ca_address_sk#24, ca_state#26] (117) BroadcastExchange -Input [1]: [ca_address_sk#236] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#238] +Input [1]: [ca_address_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#179] (118) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_addr_sk#233] -Right keys [1]: [ca_address_sk#236] +Left keys [1]: [c_current_addr_sk#18] +Right keys [1]: [ca_address_sk#24] Join condition: None (119) Project [codegen id : 31] -Output [9]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_birth_year#234] -Input [11]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_addr_sk#233, c_birth_year#234, ca_address_sk#236] +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] (120) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#239] +Output [1]: [d_date_sk#29] (121) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_sold_date_sk#228] -Right keys [1]: [d_date_sk#239] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#29] Join condition: None (122) Project [codegen id : 31] -Output [8]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cd_dep_count#230, c_birth_year#234] -Input [10]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_birth_year#234, d_date_sk#239] +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] (123) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#240, i_item_id#241] +Output [2]: [i_item_sk#32, i_item_id#33] (124) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_item_sk#222] -Right keys [1]: [i_item_sk#240] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#32] Join condition: None (125) Project [codegen id : 31] -Output [8]: [i_item_id#241, cast(cs_quantity#223 as decimal(12,2)) AS agg1#35, cast(cs_list_price#224 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#226 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#225 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#227 as decimal(12,2)) AS agg5#39, cast(c_birth_year#234 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#230 as decimal(12,2)) AS agg7#41] -Input [10]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cd_dep_count#230, c_birth_year#234, i_item_sk#240, i_item_id#241] +Output [8]: [i_item_id#33, 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#20 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#20, i_item_sk#32, i_item_id#33] (126) HashAggregate [codegen id : 31] -Input [8]: [i_item_id#241, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [1]: [i_item_id#241] +Input [8]: [i_item_id#33, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [1]: [i_item_id#33] 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#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] -Results [15]: [i_item_id#241, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] +Aggregate Attributes [14]: [sum#180, count#181, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193] +Results [15]: [i_item_id#33, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] (127) Exchange -Input [15]: [i_item_id#241, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] -Arguments: hashpartitioning(i_item_id#241, 5), ENSURE_REQUIREMENTS, [id=#270] +Input [15]: [i_item_id#33, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] +Arguments: hashpartitioning(i_item_id#33, 5), ENSURE_REQUIREMENTS, [id=#208] (128) HashAggregate [codegen id : 32] -Input [15]: [i_item_id#241, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] -Keys [1]: [i_item_id#241] +Input [15]: [i_item_id#33, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] +Keys [1]: [i_item_id#33] 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)#271, avg(agg2#36)#272, avg(agg3#37)#273, avg(agg4#38)#274, avg(agg5#39)#275, avg(agg6#40)#276, avg(agg7#41)#277] -Results [11]: [i_item_id#241, null AS ca_country#278, null AS ca_state#279, null AS county#280, avg(agg1#35)#271 AS agg1#281, avg(agg2#36)#272 AS agg2#282, avg(agg3#37)#273 AS agg3#283, avg(agg4#38)#274 AS agg4#284, avg(agg5#39)#275 AS agg5#285, avg(agg6#40)#276 AS agg6#286, avg(agg7#41)#277 AS agg7#287] +Aggregate Attributes [7]: [avg(agg1#35)#209, avg(agg2#36)#210, avg(agg3#37)#211, avg(agg4#38)#212, avg(agg5#39)#213, avg(agg6#40)#214, avg(agg7#41)#215] +Results [11]: [i_item_id#33, null AS ca_country#216, null AS ca_state#217, null AS county#218, avg(agg1#35)#209 AS agg1#219, avg(agg2#36)#210 AS agg2#220, avg(agg3#37)#211 AS agg3#221, avg(agg4#38)#212 AS agg4#222, avg(agg5#39)#213 AS agg5#223, avg(agg6#40)#214 AS agg6#224, avg(agg7#41)#215 AS agg7#225] (129) Scan parquet default.catalog_sales -Output [9]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296] +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#296), dynamicpruningexpression(cs_sold_date_sk#296 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 (130) ColumnarToRow [codegen id : 39] -Input [9]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296] +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] (131) Filter [codegen id : 39] -Input [9]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296] -Condition : ((isnotnull(cs_bill_cdemo_sk#289) AND isnotnull(cs_bill_customer_sk#288)) AND isnotnull(cs_item_sk#290)) +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)) (132) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#297, cd_dep_count#298] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] (133) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_cdemo_sk#289] -Right keys [1]: [cd_demo_sk#297] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (134) Project [codegen id : 39] -Output [9]: [cs_bill_customer_sk#288, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298] -Input [11]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_demo_sk#297, cd_dep_count#298] +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] (135) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#299, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302] +Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] (136) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_customer_sk#288] -Right keys [1]: [c_customer_sk#299] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#16] Join condition: None (137) Project [codegen id : 39] -Output [11]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302] -Input [13]: [cs_bill_customer_sk#288, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_customer_sk#299, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302] +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] (138) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#303] +Output [1]: [cd_demo_sk#226] (139) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#300] -Right keys [1]: [cd_demo_sk#303] +Left keys [1]: [c_current_cdemo_sk#17] +Right keys [1]: [cd_demo_sk#226] Join condition: None (140) Project [codegen id : 39] -Output [10]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_addr_sk#301, c_birth_year#302] -Input [12]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302, cd_demo_sk#303] +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#226] (141) ReusedExchange [Reuses operator id: 117] -Output [1]: [ca_address_sk#304] +Output [1]: [ca_address_sk#24] (142) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_addr_sk#301] -Right keys [1]: [ca_address_sk#304] +Left keys [1]: [c_current_addr_sk#18] +Right keys [1]: [ca_address_sk#24] Join condition: None (143) Project [codegen id : 39] -Output [9]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_birth_year#302] -Input [11]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_addr_sk#301, c_birth_year#302, ca_address_sk#304] +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] (144) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#305] +Output [1]: [d_date_sk#29] (145) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_sold_date_sk#296] -Right keys [1]: [d_date_sk#305] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#29] Join condition: None (146) Project [codegen id : 39] -Output [8]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cd_dep_count#298, c_birth_year#302] -Input [10]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_birth_year#302, d_date_sk#305] +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] (147) Scan parquet default.item -Output [1]: [i_item_sk#306] +Output [1]: [i_item_sk#32] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (148) ColumnarToRow [codegen id : 38] -Input [1]: [i_item_sk#306] +Input [1]: [i_item_sk#32] (149) Filter [codegen id : 38] -Input [1]: [i_item_sk#306] -Condition : isnotnull(i_item_sk#306) +Input [1]: [i_item_sk#32] +Condition : isnotnull(i_item_sk#32) (150) BroadcastExchange -Input [1]: [i_item_sk#306] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#307] +Input [1]: [i_item_sk#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#227] (151) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_item_sk#290] -Right keys [1]: [i_item_sk#306] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#32] Join condition: None (152) Project [codegen id : 39] -Output [7]: [cast(cs_quantity#291 as decimal(12,2)) AS agg1#35, cast(cs_list_price#292 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#294 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#293 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#295 as decimal(12,2)) AS agg5#39, cast(c_birth_year#302 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#298 as decimal(12,2)) AS agg7#41] -Input [9]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cd_dep_count#298, c_birth_year#302, i_item_sk#306] +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#20 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] +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#32] (153) HashAggregate [codegen id : 39] 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#308, count#309, sum#310, count#311, sum#312, count#313, sum#314, count#315, sum#316, count#317, sum#318, count#319, sum#320, count#321] -Results [14]: [sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335] +Aggregate Attributes [14]: [sum#228, count#229, sum#230, count#231, sum#232, count#233, sum#234, count#235, sum#236, count#237, sum#238, count#239, sum#240, count#241] +Results [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] (154) Exchange -Input [14]: [sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#336] +Input [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#256] (155) HashAggregate [codegen id : 40] -Input [14]: [sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335] +Input [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] 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)#337, avg(agg2#36)#338, avg(agg3#37)#339, avg(agg4#38)#340, avg(agg5#39)#341, avg(agg6#40)#342, avg(agg7#41)#343] -Results [11]: [null AS i_item_id#344, null AS ca_country#345, null AS ca_state#346, null AS county#347, avg(agg1#35)#337 AS agg1#348, avg(agg2#36)#338 AS agg2#349, avg(agg3#37)#339 AS agg3#350, avg(agg4#38)#340 AS agg4#351, avg(agg5#39)#341 AS agg5#352, avg(agg6#40)#342 AS agg6#353, avg(agg7#41)#343 AS agg7#354] +Aggregate Attributes [7]: [avg(agg1#35)#257, avg(agg2#36)#258, avg(agg3#37)#259, avg(agg4#38)#260, avg(agg5#39)#261, avg(agg6#40)#262, avg(agg7#41)#263] +Results [11]: [null AS i_item_id#264, null AS ca_country#265, null AS ca_state#266, null AS county#267, avg(agg1#35)#257 AS agg1#268, avg(agg2#36)#258 AS agg2#269, avg(agg3#37)#259 AS agg3#270, avg(agg4#38)#260 AS agg4#271, avg(agg5#39)#261 AS agg5#272, avg(agg6#40)#262 AS agg6#273, avg(agg7#41)#263 AS agg7#274] (156) Union @@ -868,12 +868,12 @@ ReusedExchange (158) (158) ReusedExchange [Reuses operator id: 34] Output [1]: [d_date_sk#29] -Subquery:2 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#93 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#160 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 101 Hosting Expression = cs_sold_date_sk#228 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 101 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 129 Hosting Expression = cs_sold_date_sk#296 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 129 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 f5a7a9135cf29..014b89cecd475 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 @@ -198,116 +198,116 @@ Aggregate Attributes [1]: [avg(qoh#24)#29] Results [5]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, avg(qoh#24)#29 AS qoh#30] (31) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#31, i_brand#32, i_class#33, i_category#34, sum#35, count#36] +Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#31, count#32] (32) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#31, i_brand#32, i_class#33, i_category#34, sum#35, count#36] -Keys [4]: [i_product_name#31, i_brand#32, i_class#33, i_category#34] -Functions [1]: [avg(inv_quantity_on_hand#37)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#37)#38] -Results [4]: [i_product_name#31, i_brand#32, i_class#33, avg(inv_quantity_on_hand#37)#38 AS qoh#24] +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#31, count#32] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#33] +Results [4]: [i_product_name#16, i_brand#13, i_class#14, avg(inv_quantity_on_hand#3)#33 AS qoh#24] (33) HashAggregate [codegen id : 16] -Input [4]: [i_product_name#31, i_brand#32, i_class#33, qoh#24] -Keys [3]: [i_product_name#31, i_brand#32, i_class#33] +Input [4]: [i_product_name#16, i_brand#13, i_class#14, qoh#24] +Keys [3]: [i_product_name#16, i_brand#13, i_class#14] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#39, count#40] -Results [5]: [i_product_name#31, i_brand#32, i_class#33, sum#41, count#42] +Aggregate Attributes [2]: [sum#34, count#35] +Results [5]: [i_product_name#16, i_brand#13, i_class#14, sum#36, count#37] (34) Exchange -Input [5]: [i_product_name#31, i_brand#32, i_class#33, sum#41, count#42] -Arguments: hashpartitioning(i_product_name#31, i_brand#32, i_class#33, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#36, count#37] +Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, 5), ENSURE_REQUIREMENTS, [id=#38] (35) HashAggregate [codegen id : 17] -Input [5]: [i_product_name#31, i_brand#32, i_class#33, sum#41, count#42] -Keys [3]: [i_product_name#31, i_brand#32, i_class#33] +Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#36, count#37] +Keys [3]: [i_product_name#16, i_brand#13, i_class#14] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#44] -Results [5]: [i_product_name#31, i_brand#32, i_class#33, null AS i_category#45, avg(qoh#24)#44 AS qoh#46] +Aggregate Attributes [1]: [avg(qoh#24)#39] +Results [5]: [i_product_name#16, i_brand#13, i_class#14, null AS i_category#40, avg(qoh#24)#39 AS qoh#41] (36) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#47, i_brand#48, i_class#49, i_category#50, sum#51, count#52] +Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#42, count#43] (37) HashAggregate [codegen id : 25] -Input [6]: [i_product_name#47, i_brand#48, i_class#49, i_category#50, sum#51, count#52] -Keys [4]: [i_product_name#47, i_brand#48, i_class#49, i_category#50] -Functions [1]: [avg(inv_quantity_on_hand#53)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#53)#54] -Results [3]: [i_product_name#47, i_brand#48, avg(inv_quantity_on_hand#53)#54 AS qoh#24] +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#42, count#43] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#44] +Results [3]: [i_product_name#16, i_brand#13, avg(inv_quantity_on_hand#3)#44 AS qoh#24] (38) HashAggregate [codegen id : 25] -Input [3]: [i_product_name#47, i_brand#48, qoh#24] -Keys [2]: [i_product_name#47, i_brand#48] +Input [3]: [i_product_name#16, i_brand#13, qoh#24] +Keys [2]: [i_product_name#16, i_brand#13] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#55, count#56] -Results [4]: [i_product_name#47, i_brand#48, sum#57, count#58] +Aggregate Attributes [2]: [sum#45, count#46] +Results [4]: [i_product_name#16, i_brand#13, sum#47, count#48] (39) Exchange -Input [4]: [i_product_name#47, i_brand#48, sum#57, count#58] -Arguments: hashpartitioning(i_product_name#47, i_brand#48, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [4]: [i_product_name#16, i_brand#13, sum#47, count#48] +Arguments: hashpartitioning(i_product_name#16, i_brand#13, 5), ENSURE_REQUIREMENTS, [id=#49] (40) HashAggregate [codegen id : 26] -Input [4]: [i_product_name#47, i_brand#48, sum#57, count#58] -Keys [2]: [i_product_name#47, i_brand#48] +Input [4]: [i_product_name#16, i_brand#13, sum#47, count#48] +Keys [2]: [i_product_name#16, i_brand#13] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#60] -Results [5]: [i_product_name#47, i_brand#48, null AS i_class#61, null AS i_category#62, avg(qoh#24)#60 AS qoh#63] +Aggregate Attributes [1]: [avg(qoh#24)#50] +Results [5]: [i_product_name#16, i_brand#13, null AS i_class#51, null AS i_category#52, avg(qoh#24)#50 AS qoh#53] (41) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#64, i_brand#65, i_class#66, i_category#67, sum#68, count#69] +Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#54, count#55] (42) HashAggregate [codegen id : 34] -Input [6]: [i_product_name#64, i_brand#65, i_class#66, i_category#67, sum#68, count#69] -Keys [4]: [i_product_name#64, i_brand#65, i_class#66, i_category#67] -Functions [1]: [avg(inv_quantity_on_hand#70)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#70)#71] -Results [2]: [i_product_name#64, avg(inv_quantity_on_hand#70)#71 AS qoh#24] +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#54, count#55] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#56] +Results [2]: [i_product_name#16, avg(inv_quantity_on_hand#3)#56 AS qoh#24] (43) HashAggregate [codegen id : 34] -Input [2]: [i_product_name#64, qoh#24] -Keys [1]: [i_product_name#64] +Input [2]: [i_product_name#16, qoh#24] +Keys [1]: [i_product_name#16] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#72, count#73] -Results [3]: [i_product_name#64, sum#74, count#75] +Aggregate Attributes [2]: [sum#57, count#58] +Results [3]: [i_product_name#16, sum#59, count#60] (44) Exchange -Input [3]: [i_product_name#64, sum#74, count#75] -Arguments: hashpartitioning(i_product_name#64, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [3]: [i_product_name#16, sum#59, count#60] +Arguments: hashpartitioning(i_product_name#16, 5), ENSURE_REQUIREMENTS, [id=#61] (45) HashAggregate [codegen id : 35] -Input [3]: [i_product_name#64, sum#74, count#75] -Keys [1]: [i_product_name#64] +Input [3]: [i_product_name#16, sum#59, count#60] +Keys [1]: [i_product_name#16] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#77] -Results [5]: [i_product_name#64, null AS i_brand#78, null AS i_class#79, null AS i_category#80, avg(qoh#24)#77 AS qoh#81] +Aggregate Attributes [1]: [avg(qoh#24)#62] +Results [5]: [i_product_name#16, null AS i_brand#63, null AS i_class#64, null AS i_category#65, avg(qoh#24)#62 AS qoh#66] (46) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#82, i_brand#83, i_class#84, i_category#85, sum#86, count#87] +Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#67, count#68] (47) HashAggregate [codegen id : 43] -Input [6]: [i_product_name#82, i_brand#83, i_class#84, i_category#85, sum#86, count#87] -Keys [4]: [i_product_name#82, i_brand#83, i_class#84, i_category#85] -Functions [1]: [avg(inv_quantity_on_hand#88)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#88)#89] -Results [1]: [avg(inv_quantity_on_hand#88)#89 AS qoh#24] +Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#67, count#68] +Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#69] +Results [1]: [avg(inv_quantity_on_hand#3)#69 AS qoh#24] (48) HashAggregate [codegen id : 43] Input [1]: [qoh#24] Keys: [] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#90, count#91] -Results [2]: [sum#92, count#93] +Aggregate Attributes [2]: [sum#70, count#71] +Results [2]: [sum#72, count#73] (49) Exchange -Input [2]: [sum#92, count#93] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] +Input [2]: [sum#72, count#73] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#74] (50) HashAggregate [codegen id : 44] -Input [2]: [sum#92, count#93] +Input [2]: [sum#72, count#73] Keys: [] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#95] -Results [5]: [null AS i_product_name#96, null AS i_brand#97, null AS i_class#98, null AS i_category#99, avg(qoh#24)#95 AS qoh#100] +Aggregate Attributes [1]: [avg(qoh#24)#75] +Results [5]: [null AS i_product_name#76, null AS i_brand#77, null AS i_class#78, null AS i_category#79, avg(qoh#24)#75 AS qoh#80] (51) Union 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 527d2eb5e86a4..74dd076851c1d 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 @@ -183,116 +183,116 @@ Aggregate Attributes [1]: [avg(qoh#23)#28] Results [5]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, avg(qoh#23)#28 AS qoh#29] (28) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#30, i_brand#31, i_class#32, i_category#33, sum#34, count#35] +Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#30, count#31] (29) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#30, i_brand#31, i_class#32, i_category#33, sum#34, count#35] -Keys [4]: [i_product_name#30, i_brand#31, i_class#32, i_category#33] -Functions [1]: [avg(inv_quantity_on_hand#36)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#36)#37] -Results [4]: [i_product_name#30, i_brand#31, i_class#32, avg(inv_quantity_on_hand#36)#37 AS qoh#23] +Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#30, count#31] +Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#32] +Results [4]: [i_product_name#13, i_brand#10, i_class#11, avg(inv_quantity_on_hand#3)#32 AS qoh#23] (30) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#30, i_brand#31, i_class#32, qoh#23] -Keys [3]: [i_product_name#30, i_brand#31, i_class#32] +Input [4]: [i_product_name#13, i_brand#10, i_class#11, qoh#23] +Keys [3]: [i_product_name#13, i_brand#10, i_class#11] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#38, count#39] -Results [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] +Aggregate Attributes [2]: [sum#33, count#34] +Results [5]: [i_product_name#13, i_brand#10, i_class#11, sum#35, count#36] (31) Exchange -Input [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] -Arguments: hashpartitioning(i_product_name#30, i_brand#31, i_class#32, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [5]: [i_product_name#13, i_brand#10, i_class#11, sum#35, count#36] +Arguments: hashpartitioning(i_product_name#13, i_brand#10, i_class#11, 5), ENSURE_REQUIREMENTS, [id=#37] (32) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] -Keys [3]: [i_product_name#30, i_brand#31, i_class#32] +Input [5]: [i_product_name#13, i_brand#10, i_class#11, sum#35, count#36] +Keys [3]: [i_product_name#13, i_brand#10, i_class#11] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#43] -Results [5]: [i_product_name#30, i_brand#31, i_class#32, null AS i_category#44, avg(qoh#23)#43 AS qoh#45] +Aggregate Attributes [1]: [avg(qoh#23)#38] +Results [5]: [i_product_name#13, i_brand#10, i_class#11, null AS i_category#39, avg(qoh#23)#38 AS qoh#40] (33) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#46, i_brand#47, i_class#48, i_category#49, sum#50, count#51] +Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#41, count#42] (34) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#46, i_brand#47, i_class#48, i_category#49, sum#50, count#51] -Keys [4]: [i_product_name#46, i_brand#47, i_class#48, i_category#49] -Functions [1]: [avg(inv_quantity_on_hand#52)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#52)#53] -Results [3]: [i_product_name#46, i_brand#47, avg(inv_quantity_on_hand#52)#53 AS qoh#23] +Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#41, count#42] +Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#43] +Results [3]: [i_product_name#13, i_brand#10, avg(inv_quantity_on_hand#3)#43 AS qoh#23] (35) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#46, i_brand#47, qoh#23] -Keys [2]: [i_product_name#46, i_brand#47] +Input [3]: [i_product_name#13, i_brand#10, qoh#23] +Keys [2]: [i_product_name#13, i_brand#10] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#54, count#55] -Results [4]: [i_product_name#46, i_brand#47, sum#56, count#57] +Aggregate Attributes [2]: [sum#44, count#45] +Results [4]: [i_product_name#13, i_brand#10, sum#46, count#47] (36) Exchange -Input [4]: [i_product_name#46, i_brand#47, sum#56, count#57] -Arguments: hashpartitioning(i_product_name#46, i_brand#47, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [4]: [i_product_name#13, i_brand#10, sum#46, count#47] +Arguments: hashpartitioning(i_product_name#13, i_brand#10, 5), ENSURE_REQUIREMENTS, [id=#48] (37) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#46, i_brand#47, sum#56, count#57] -Keys [2]: [i_product_name#46, i_brand#47] +Input [4]: [i_product_name#13, i_brand#10, sum#46, count#47] +Keys [2]: [i_product_name#13, i_brand#10] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#59] -Results [5]: [i_product_name#46, i_brand#47, null AS i_class#60, null AS i_category#61, avg(qoh#23)#59 AS qoh#62] +Aggregate Attributes [1]: [avg(qoh#23)#49] +Results [5]: [i_product_name#13, i_brand#10, null AS i_class#50, null AS i_category#51, avg(qoh#23)#49 AS qoh#52] (38) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, sum#67, count#68] +Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#53, count#54] (39) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, sum#67, count#68] -Keys [4]: [i_product_name#63, i_brand#64, i_class#65, i_category#66] -Functions [1]: [avg(inv_quantity_on_hand#69)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#69)#70] -Results [2]: [i_product_name#63, avg(inv_quantity_on_hand#69)#70 AS qoh#23] +Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#53, count#54] +Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#55] +Results [2]: [i_product_name#13, avg(inv_quantity_on_hand#3)#55 AS qoh#23] (40) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#63, qoh#23] -Keys [1]: [i_product_name#63] +Input [2]: [i_product_name#13, qoh#23] +Keys [1]: [i_product_name#13] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#71, count#72] -Results [3]: [i_product_name#63, sum#73, count#74] +Aggregate Attributes [2]: [sum#56, count#57] +Results [3]: [i_product_name#13, sum#58, count#59] (41) Exchange -Input [3]: [i_product_name#63, sum#73, count#74] -Arguments: hashpartitioning(i_product_name#63, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [3]: [i_product_name#13, sum#58, count#59] +Arguments: hashpartitioning(i_product_name#13, 5), ENSURE_REQUIREMENTS, [id=#60] (42) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#63, sum#73, count#74] -Keys [1]: [i_product_name#63] +Input [3]: [i_product_name#13, sum#58, count#59] +Keys [1]: [i_product_name#13] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#76] -Results [5]: [i_product_name#63, null AS i_brand#77, null AS i_class#78, null AS i_category#79, avg(qoh#23)#76 AS qoh#80] +Aggregate Attributes [1]: [avg(qoh#23)#61] +Results [5]: [i_product_name#13, null AS i_brand#62, null AS i_class#63, null AS i_category#64, avg(qoh#23)#61 AS qoh#65] (43) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#81, i_brand#82, i_class#83, i_category#84, sum#85, count#86] +Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#66, count#67] (44) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#81, i_brand#82, i_class#83, i_category#84, sum#85, count#86] -Keys [4]: [i_product_name#81, i_brand#82, i_class#83, i_category#84] -Functions [1]: [avg(inv_quantity_on_hand#87)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#87)#88] -Results [1]: [avg(inv_quantity_on_hand#87)#88 AS qoh#23] +Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#66, count#67] +Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#68] +Results [1]: [avg(inv_quantity_on_hand#3)#68 AS qoh#23] (45) HashAggregate [codegen id : 28] Input [1]: [qoh#23] Keys: [] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#89, count#90] -Results [2]: [sum#91, count#92] +Aggregate Attributes [2]: [sum#69, count#70] +Results [2]: [sum#71, count#72] (46) Exchange -Input [2]: [sum#91, count#92] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] +Input [2]: [sum#71, count#72] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#73] (47) HashAggregate [codegen id : 29] -Input [2]: [sum#91, count#92] +Input [2]: [sum#71, count#72] Keys: [] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#94] -Results [5]: [null AS i_product_name#95, null AS i_brand#96, null AS i_class#97, null AS i_category#98, avg(qoh#23)#94 AS qoh#99] +Aggregate Attributes [1]: [avg(qoh#23)#74] +Results [5]: [null AS i_product_name#75, null AS i_brand#76, null AS i_class#77, null AS i_category#78, avg(qoh#23)#74 AS qoh#79] (48) Union 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 03fca5e05cf3f..d4d0015a9fec0 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 @@ -329,224 +329,224 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (50) Scan parquet default.store -Output [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] +Output [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (51) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] +Input [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] (52) Filter [codegen id : 1] -Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] -Condition : (((isnotnull(s_market_id#54) AND (s_market_id#54 = 8)) AND isnotnull(s_store_sk#52)) AND isnotnull(s_zip#56)) +Input [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] +Condition : (((isnotnull(s_market_id#3) AND (s_market_id#3 = 8)) AND isnotnull(s_store_sk#1)) AND isnotnull(s_zip#5)) (53) Project [codegen id : 1] -Output [4]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56] -Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] +Output [4]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5] +Input [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] (54) BroadcastExchange -Input [4]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56] -Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#57] +Input [4]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5] +Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#52] (55) Scan parquet default.customer_address -Output [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] +Output [4]: [ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (56) ColumnarToRow -Input [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] +Input [4]: [ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] (57) Filter -Input [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] -Condition : ((isnotnull(ca_address_sk#58) AND isnotnull(ca_country#61)) AND isnotnull(ca_zip#60)) +Input [4]: [ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] +Condition : ((isnotnull(ca_address_sk#7) AND isnotnull(ca_country#10)) AND isnotnull(ca_zip#9)) (58) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [s_zip#56] -Right keys [1]: [ca_zip#60] +Left keys [1]: [s_zip#5] +Right keys [1]: [ca_zip#9] Join condition: None (59) Project [codegen id : 2] -Output [6]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61] -Input [8]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56, ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] +Output [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10] +Input [8]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5, ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] (60) BroadcastExchange -Input [6]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61] -Arguments: HashedRelationBroadcastMode(List(input[3, int, true], upper(input[5, string, true])),false), [id=#62] +Input [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10] +Arguments: HashedRelationBroadcastMode(List(input[3, int, true], upper(input[5, string, true])),false), [id=#53] (61) Scan parquet default.customer -Output [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] +Output [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct (62) ColumnarToRow -Input [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] +Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] (63) Filter -Input [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] -Condition : ((isnotnull(c_customer_sk#63) AND isnotnull(c_current_addr_sk#64)) AND isnotnull(c_birth_country#67)) +Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] +Condition : ((isnotnull(c_customer_sk#12) AND isnotnull(c_current_addr_sk#13)) AND isnotnull(c_birth_country#16)) (64) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ca_address_sk#58, upper(ca_country#61)] -Right keys [2]: [c_current_addr_sk#64, c_birth_country#67] +Left keys [2]: [ca_address_sk#7, upper(ca_country#10)] +Right keys [2]: [c_current_addr_sk#13, c_birth_country#16] Join condition: None (65) Project [codegen id : 3] -Output [7]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66] -Input [11]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61, c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] +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] +Input [11]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10, c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] (66) BroadcastExchange -Input [7]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#68] +Input [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] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#54] (67) Scan parquet default.store_sales -Output [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] +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 (68) ColumnarToRow -Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] +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] (69) Filter -Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] -Condition : (((isnotnull(ss_ticket_number#72) AND isnotnull(ss_item_sk#69)) AND isnotnull(ss_store_sk#71)) AND isnotnull(ss_customer_sk#70)) +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)) (70) Project -Output [5]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73] -Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] +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] (71) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [s_store_sk#52, c_customer_sk#63] -Right keys [2]: [ss_store_sk#71, ss_customer_sk#70] +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 (72) Project [codegen id : 4] -Output [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] -Input [12]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73] +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] (73) Exchange -Input [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] -Arguments: hashpartitioning(ss_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] +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=#55] (74) Sort [codegen id : 5] -Input [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] -Arguments: [ss_item_sk#69 ASC NULLS FIRST], false, 0 +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 (75) Scan parquet default.item -Output [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +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 (76) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] (77) Filter [codegen id : 6] -Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Condition : isnotnull(i_item_sk#76) +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) (78) Exchange -Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Arguments: hashpartitioning(i_item_sk#76, 5), ENSURE_REQUIREMENTS, [id=#82] +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=#56] (79) Sort [codegen id : 7] -Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Arguments: [i_item_sk#76 ASC NULLS FIRST], false, 0 +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 (80) SortMergeJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#69] -Right keys [1]: [i_item_sk#76] +Left keys [1]: [ss_item_sk#18] +Right keys [1]: [i_item_sk#24] Join condition: None (81) Project [codegen id : 8] -Output [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Input [14]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +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] (82) Exchange -Input [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Arguments: hashpartitioning(cast(ss_ticket_number#72 as bigint), cast(ss_item_sk#69 as bigint), 5), ENSURE_REQUIREMENTS, [id=#83] +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(cast(ss_ticket_number#21 as bigint), cast(ss_item_sk#18 as bigint), 5), ENSURE_REQUIREMENTS, [id=#57] (83) Sort [codegen id : 9] -Input [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] -Arguments: [cast(ss_ticket_number#72 as bigint) ASC NULLS FIRST, cast(ss_item_sk#69 as bigint) 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: [cast(ss_ticket_number#21 as bigint) ASC NULLS FIRST, cast(ss_item_sk#18 as bigint) ASC NULLS FIRST], false, 0 (84) Scan parquet default.store_returns -Output [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] +Output [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (85) ColumnarToRow [codegen id : 10] -Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] +Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] (86) Filter [codegen id : 10] -Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] -Condition : (isnotnull(sr_ticket_number#85) AND isnotnull(sr_item_sk#84)) +Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) (87) Project [codegen id : 10] -Output [2]: [sr_item_sk#84, sr_ticket_number#85] -Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] +Output [2]: [sr_item_sk#32, sr_ticket_number#33] +Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] (88) Exchange -Input [2]: [sr_item_sk#84, sr_ticket_number#85] -Arguments: hashpartitioning(sr_ticket_number#85, sr_item_sk#84, 5), ENSURE_REQUIREMENTS, [id=#87] +Input [2]: [sr_item_sk#32, sr_ticket_number#33] +Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#58] (89) Sort [codegen id : 11] -Input [2]: [sr_item_sk#84, sr_ticket_number#85] -Arguments: [sr_ticket_number#85 ASC NULLS FIRST, sr_item_sk#84 ASC NULLS FIRST], false, 0 +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 (90) SortMergeJoin [codegen id : 12] -Left keys [2]: [cast(ss_ticket_number#72 as bigint), cast(ss_item_sk#69 as bigint)] -Right keys [2]: [sr_ticket_number#85, sr_item_sk#84] +Left keys [2]: [cast(ss_ticket_number#21 as bigint), cast(ss_item_sk#18 as bigint)] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (91) Project [codegen id : 12] -Output [11]: [ss_net_paid#73, s_store_name#53, s_state#55, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, c_first_name#65, c_last_name#66, ca_state#59] -Input [15]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, sr_item_sk#84, sr_ticket_number#85] +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] (92) HashAggregate [codegen id : 12] -Input [11]: [ss_net_paid#73, s_store_name#53, s_state#55, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, c_first_name#65, c_last_name#66, ca_state#59] -Keys [10]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#73))] -Aggregate Attributes [1]: [sum#88] -Results [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] +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#59] +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#60] (93) Exchange -Input [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] -Arguments: hashpartitioning(c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, 5), ENSURE_REQUIREMENTS, [id=#90] +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#60] +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=#61] (94) HashAggregate [codegen id : 13] -Input [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] -Keys [10]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78] -Functions [1]: [sum(UnscaledValue(ss_net_paid#73))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#73))#91] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#73))#91,17,2) AS netpaid#40] +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#60] +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))#62] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#62,17,2) AS netpaid#40] (95) HashAggregate [codegen id : 13] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#92, count#93] -Results [2]: [sum#94, count#95] +Aggregate Attributes [2]: [sum#63, count#64] +Results [2]: [sum#65, count#66] (96) Exchange -Input [2]: [sum#94, count#95] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#96] +Input [2]: [sum#65, count#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] (97) HashAggregate [codegen id : 14] -Input [2]: [sum#94, count#95] +Input [2]: [sum#65, count#66] Keys: [] Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#97] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#97)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#98] +Aggregate Attributes [1]: [avg(netpaid#40)#68] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#68)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#69] 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 ea7a18f3c5dae..6835ff84f5bc7 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 @@ -326,212 +326,212 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (50) Scan parquet default.store_sales -Output [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] +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 : 1] -Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] +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 : 1] -Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] -Condition : (((isnotnull(ss_ticket_number#55) AND isnotnull(ss_item_sk#52)) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_customer_sk#53)) +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 : 1] -Output [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] -Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] +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) Exchange -Input [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] -Arguments: hashpartitioning(cast(ss_ticket_number#55 as bigint), cast(ss_item_sk#52 as bigint), 5), ENSURE_REQUIREMENTS, [id=#58] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#52] (55) Sort [codegen id : 2] -Input [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] -Arguments: [cast(ss_ticket_number#55 as bigint) ASC NULLS FIRST, cast(ss_item_sk#52 as bigint) 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: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (56) Scan parquet default.store_returns -Output [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] +Output [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] (58) Filter [codegen id : 3] -Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] -Condition : (isnotnull(sr_ticket_number#60) AND isnotnull(sr_item_sk#59)) +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Condition : (isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#8)) (59) Project [codegen id : 3] -Output [2]: [sr_item_sk#59, sr_ticket_number#60] -Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] +Output [2]: [sr_item_sk#8, sr_ticket_number#9] +Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] (60) Exchange -Input [2]: [sr_item_sk#59, sr_ticket_number#60] -Arguments: hashpartitioning(sr_ticket_number#60, sr_item_sk#59, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [2]: [sr_item_sk#8, sr_ticket_number#9] +Arguments: hashpartitioning(sr_ticket_number#9, sr_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#53] (61) Sort [codegen id : 4] -Input [2]: [sr_item_sk#59, sr_ticket_number#60] -Arguments: [sr_ticket_number#60 ASC NULLS FIRST, sr_item_sk#59 ASC NULLS FIRST], false, 0 +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 (62) SortMergeJoin [codegen id : 9] -Left keys [2]: [cast(ss_ticket_number#55 as bigint), cast(ss_item_sk#52 as bigint)] -Right keys [2]: [sr_ticket_number#60, sr_item_sk#59] +Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] +Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] Join condition: None (63) Project [codegen id : 9] -Output [4]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_net_paid#56] -Input [7]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, sr_item_sk#59, sr_ticket_number#60] +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] (64) Scan parquet default.store -Output [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] +Output [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (65) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] (66) Filter [codegen id : 5] -Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] -Condition : (((isnotnull(s_market_id#65) AND (s_market_id#65 = 8)) AND isnotnull(s_store_sk#63)) AND isnotnull(s_zip#67)) +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Condition : (((isnotnull(s_market_id#14) AND (s_market_id#14 = 8)) AND isnotnull(s_store_sk#12)) AND isnotnull(s_zip#16)) (67) Project [codegen id : 5] -Output [4]: [s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] -Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] +Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] +Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] (68) BroadcastExchange -Input [4]: [s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#68] +Input [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] (69) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#54] -Right keys [1]: [s_store_sk#63] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#12] Join condition: None (70) Project [codegen id : 9] -Output [6]: [ss_item_sk#52, ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67] -Input [8]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_net_paid#56, s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] +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] (71) Scan parquet default.item -Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +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 (72) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] (73) Filter [codegen id : 6] -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Condition : isnotnull(i_item_sk#69) +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) (74) BroadcastExchange -Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] +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=#55] (75) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#52] -Right keys [1]: [i_item_sk#69] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#18] Join condition: None (76) Project [codegen id : 9] -Output [10]: [ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] -Input [12]: [ss_item_sk#52, ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +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] (77) Scan parquet default.customer -Output [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +Output [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct (78) ColumnarToRow [codegen id : 7] -Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +Input [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] (79) Filter [codegen id : 7] -Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] -Condition : ((isnotnull(c_customer_sk#76) AND isnotnull(c_current_addr_sk#77)) AND isnotnull(c_birth_country#80)) +Input [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] +Condition : ((isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#26)) AND isnotnull(c_birth_country#29)) (80) BroadcastExchange -Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] +Input [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#56] (81) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#53] -Right keys [1]: [c_customer_sk#76] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#25] Join condition: None (82) Project [codegen id : 9] -Output [13]: [ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] -Input [15]: [ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +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] (83) Scan parquet default.customer_address -Output [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] +Output [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (84) ColumnarToRow [codegen id : 8] -Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] +Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] (85) Filter [codegen id : 8] -Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] -Condition : ((isnotnull(ca_address_sk#82) AND isnotnull(ca_country#85)) AND isnotnull(ca_zip#84)) +Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] +Condition : ((isnotnull(ca_address_sk#31) AND isnotnull(ca_country#34)) AND isnotnull(ca_zip#33)) (86) BroadcastExchange -Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] -Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [id=#86] +Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [id=#57] (87) BroadcastHashJoin [codegen id : 9] -Left keys [3]: [c_current_addr_sk#77, c_birth_country#80, s_zip#67] -Right keys [3]: [ca_address_sk#82, upper(ca_country#85), ca_zip#84] +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 (88) Project [codegen id : 9] -Output [11]: [ss_net_paid#56, s_store_name#64, s_state#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#78, c_last_name#79, ca_state#83] -Input [17]: [ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80, ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] +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] (89) HashAggregate [codegen id : 9] -Input [11]: [ss_net_paid#56, s_store_name#64, s_state#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#78, c_last_name#79, ca_state#83] -Keys [10]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#56))] -Aggregate Attributes [1]: [sum#87] -Results [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] +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#58] +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#59] (90) Exchange -Input [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] -Arguments: hashpartitioning(c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#89] +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#59] +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=#60] (91) HashAggregate [codegen id : 10] -Input [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] -Keys [10]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] -Functions [1]: [sum(UnscaledValue(ss_net_paid#56))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#56))#90] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#56))#90,17,2) AS netpaid#40] +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#59] +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))#61] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#61,17,2) AS netpaid#40] (92) HashAggregate [codegen id : 10] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#91, count#92] -Results [2]: [sum#93, count#94] +Aggregate Attributes [2]: [sum#62, count#63] +Results [2]: [sum#64, count#65] (93) Exchange -Input [2]: [sum#93, count#94] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] +Input [2]: [sum#64, count#65] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] (94) HashAggregate [codegen id : 11] -Input [2]: [sum#93, count#94] +Input [2]: [sum#64, count#65] Keys: [] Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#96] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#96)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#97] +Aggregate Attributes [1]: [avg(netpaid#40)#67] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#67)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#68] 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 1fb9533024b2f..6c48953be5aeb 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +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#61), dynamicpruningexpression(ss_sold_date_sk#61 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 (34) ColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +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] (35) Filter [codegen id : 11] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] -Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) +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)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#62] +Output [1]: [cd_demo_sk#10] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#55] -Right keys [1]: [cd_demo_sk#62] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (38) Project [codegen id : 11] -Output [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] -Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, cd_demo_sk#62] +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] (39) Scan parquet default.store -Output [2]: [s_store_sk#63, s_state#64] +Output [2]: [s_store_sk#18, s_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 8] -Input [2]: [s_store_sk#63, s_state#64] +Input [2]: [s_store_sk#18, s_state#19] (41) Filter [codegen id : 8] -Input [2]: [s_store_sk#63, s_state#64] -Condition : ((isnotnull(s_state#64) AND (s_state#64 = TN)) AND isnotnull(s_store_sk#63)) +Input [2]: [s_store_sk#18, s_state#19] +Condition : ((isnotnull(s_state#19) AND (s_state#19 = TN)) AND isnotnull(s_store_sk#18)) (42) Project [codegen id : 8] -Output [1]: [s_store_sk#63] -Input [2]: [s_store_sk#63, s_state#64] +Output [1]: [s_store_sk#18] +Input [2]: [s_store_sk#18, s_state#19] (43) BroadcastExchange -Input [1]: [s_store_sk#63] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] +Input [1]: [s_store_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] (44) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#56] -Right keys [1]: [s_store_sk#63] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None (45) Project [codegen id : 11] -Output [6]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] -Input [8]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, s_store_sk#63] +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#18] (46) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#66] +Output [1]: [d_date_sk#15] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#61] -Right keys [1]: [d_date_sk#66] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (48) Project [codegen id : 11] -Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] -Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#66] +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] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#67, i_item_id#68] +Output [2]: [i_item_sk#21, i_item_id#22] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#67] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#21] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] -Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] +Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] +Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#68] +Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#22] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] -Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] +Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] (53) Exchange -Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Keys [1]: [i_item_id#68] +Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Keys [1]: [i_item_id#22] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] -Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] +Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] +Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] (55) Scan parquet default.store_sales -Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +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#103), dynamicpruningexpression(ss_sold_date_sk#103 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 (56) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +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] (57) Filter [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) +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)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#104] +Output [1]: [cd_demo_sk#10] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#97] -Right keys [1]: [cd_demo_sk#104] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (60) Project [codegen id : 17] -Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] +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] (61) ReusedExchange [Reuses operator id: 43] -Output [1]: [s_store_sk#105] +Output [1]: [s_store_sk#18] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#98] -Right keys [1]: [s_store_sk#105] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, s_store_sk#105] +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#18] (64) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#106] +Output [1]: [d_date_sk#15] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#103] -Right keys [1]: [d_date_sk#106] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [7]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#106] +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] (67) Scan parquet default.item -Output [1]: [i_item_sk#107] +Output [1]: [i_item_sk#21] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#107] +Input [1]: [i_item_sk#21] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#107] -Condition : isnotnull(i_item_sk#107) +Input [1]: [i_item_sk#21] +Condition : isnotnull(i_item_sk#21) (70) BroadcastExchange -Input [1]: [i_item_sk#107] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] +Input [1]: [i_item_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#96] -Right keys [1]: [i_item_sk#107] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#21] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] -Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] +Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] +Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] -Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] +Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] (74) Exchange -Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] +Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] -Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] +Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] +Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#15] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 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 f05f4e9f2e43e..04c79a5413ad9 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +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#61), dynamicpruningexpression(ss_sold_date_sk#61 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 (34) ColumnarToRow [codegen id : 11] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +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] (35) Filter [codegen id : 11] -Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] -Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) +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)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#62] +Output [1]: [cd_demo_sk#10] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#55] -Right keys [1]: [cd_demo_sk#62] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (38) Project [codegen id : 11] -Output [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] -Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, cd_demo_sk#62] +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] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#63] +Output [1]: [d_date_sk#15] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#61] -Right keys [1]: [d_date_sk#63] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (41) Project [codegen id : 11] -Output [6]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] -Input [8]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#63] +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] (42) Scan parquet default.store -Output [2]: [s_store_sk#64, s_state#65] +Output [2]: [s_store_sk#18, s_state#19] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (43) ColumnarToRow [codegen id : 9] -Input [2]: [s_store_sk#64, s_state#65] +Input [2]: [s_store_sk#18, s_state#19] (44) Filter [codegen id : 9] -Input [2]: [s_store_sk#64, s_state#65] -Condition : ((isnotnull(s_state#65) AND (s_state#65 = TN)) AND isnotnull(s_store_sk#64)) +Input [2]: [s_store_sk#18, s_state#19] +Condition : ((isnotnull(s_state#19) AND (s_state#19 = TN)) AND isnotnull(s_store_sk#18)) (45) Project [codegen id : 9] -Output [1]: [s_store_sk#64] -Input [2]: [s_store_sk#64, s_state#65] +Output [1]: [s_store_sk#18] +Input [2]: [s_store_sk#18, s_state#19] (46) BroadcastExchange -Input [1]: [s_store_sk#64] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#66] +Input [1]: [s_store_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#56] -Right keys [1]: [s_store_sk#64] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None (48) Project [codegen id : 11] -Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] -Input [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, s_store_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_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#18] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#67, i_item_id#68] +Output [2]: [i_item_sk#21, i_item_id#22] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#54] -Right keys [1]: [i_item_sk#67] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#21] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] -Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] +Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] +Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#68] +Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#22] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] -Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] +Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] (53) Exchange -Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] -Keys [1]: [i_item_id#68] +Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Keys [1]: [i_item_id#22] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] -Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] +Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] +Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] (55) Scan parquet default.store_sales -Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +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#103), dynamicpruningexpression(ss_sold_date_sk#103 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 (56) ColumnarToRow [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +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] (57) Filter [codegen id : 17] -Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) +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)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#104] +Output [1]: [cd_demo_sk#10] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#97] -Right keys [1]: [cd_demo_sk#104] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] Join condition: None (60) Project [codegen id : 17] -Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] -Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] +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] (61) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#105] +Output [1]: [d_date_sk#15] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#103] -Right keys [1]: [d_date_sk#105] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#105] +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] (64) ReusedExchange [Reuses operator id: 46] -Output [1]: [s_store_sk#106] +Output [1]: [s_store_sk#18] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#98] -Right keys [1]: [s_store_sk#106] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#18] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] -Input [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#106] +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#18] (67) Scan parquet default.item -Output [1]: [i_item_sk#107] +Output [1]: [i_item_sk#21] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#107] +Input [1]: [i_item_sk#21] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#107] -Condition : isnotnull(i_item_sk#107) +Input [1]: [i_item_sk#21] +Condition : isnotnull(i_item_sk#21) (70) BroadcastExchange -Input [1]: [i_item_sk#107] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] +Input [1]: [i_item_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#96] -Right keys [1]: [i_item_sk#107] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#21] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] -Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] +Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] +Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] -Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] +Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] (74) Exchange -Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] +Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] -Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] +Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] +Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#15] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 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/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt index f7fff421817c0..0a861f4ce8e6d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt @@ -145,20 +145,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#17] +Output [1]: [d_date_sk#10] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#17] +Right keys [1]: [d_date_sk#10] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] -Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] +Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] (23) Exchange Input [1]: [ws_bill_customer_sk#15] -Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] +Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] @@ -170,38 +170,38 @@ Right keys [1]: [ws_bill_customer_sk#15] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] +Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] +Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#21] +Output [1]: [d_date_sk#10] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#20] -Right keys [1]: [d_date_sk#21] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#10] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#19] -Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] +Output [1]: [cs_ship_customer_sk#18] +Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] (31) Exchange -Input [1]: [cs_ship_customer_sk#19] -Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [cs_ship_customer_sk#18] +Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#19] -Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#18] +Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#19] +Right keys [1]: [cs_ship_customer_sk#18] Join condition: None (34) Filter [codegen id : 12] @@ -214,103 +214,103 @@ Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2 (36) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#23] +Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#21] (37) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 (38) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#22, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 14] -Input [2]: [ca_address_sk#24, ca_state#25] +Input [2]: [ca_address_sk#22, ca_state#23] (40) Filter [codegen id : 14] -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : isnotnull(ca_address_sk#24) +Input [2]: [ca_address_sk#22, ca_state#23] +Condition : isnotnull(ca_address_sk#22) (41) Exchange -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [ca_address_sk#22, ca_state#23] +Arguments: hashpartitioning(ca_address_sk#22, 5), ENSURE_REQUIREMENTS, [id=#24] (42) Sort [codegen id : 15] -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#22, ca_state#23] +Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#24] +Right keys [1]: [ca_address_sk#22] Join condition: None (44) Project [codegen id : 16] -Output [2]: [c_current_cdemo_sk#4, ca_state#25] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24, ca_state#25] +Output [2]: [c_current_cdemo_sk#4, ca_state#23] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] (45) Exchange -Input [2]: [c_current_cdemo_sk#4, ca_state#25] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [2]: [c_current_cdemo_sk#4, ca_state#23] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#25] (46) Sort [codegen id : 17] -Input [2]: [c_current_cdemo_sk#4, ca_state#25] +Input [2]: [c_current_cdemo_sk#4, ca_state#23] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 (47) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 18] -Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] (49) Filter [codegen id : 18] -Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Condition : isnotnull(cd_demo_sk#28) +Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Condition : isnotnull(cd_demo_sk#26) (50) Exchange -Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Arguments: hashpartitioning(cd_demo_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] (51) Sort [codegen id : 19] -Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 +Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Arguments: [cd_demo_sk#26 ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#28] +Right keys [1]: [cd_demo_sk#26] Join condition: None (53) Project [codegen id : 20] -Output [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Input [8]: [c_current_cdemo_sk#4, ca_state#25, cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] (54) HashAggregate [codegen id : 20] -Input [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#31), partial_max(cd_dep_count#31), partial_sum(cd_dep_count#31), partial_avg(cd_dep_employed_count#32), partial_max(cd_dep_employed_count#32), partial_sum(cd_dep_employed_count#32), partial_avg(cd_dep_college_count#33), partial_max(cd_dep_college_count#33), partial_sum(cd_dep_college_count#33)] -Aggregate Attributes [13]: [count#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47] -Results [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] +Input [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#29), partial_max(cd_dep_count#29), partial_sum(cd_dep_count#29), partial_avg(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_sum(cd_dep_employed_count#30), partial_avg(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_sum(cd_dep_college_count#31)] +Aggregate Attributes [13]: [count#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45] +Results [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] (55) Exchange -Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] -Arguments: hashpartitioning(ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] +Arguments: hashpartitioning(ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#59] (56) HashAggregate [codegen id : 21] -Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] -Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Functions [10]: [count(1), avg(cd_dep_count#31), max(cd_dep_count#31), sum(cd_dep_count#31), avg(cd_dep_employed_count#32), max(cd_dep_employed_count#32), sum(cd_dep_employed_count#32), avg(cd_dep_college_count#33), max(cd_dep_college_count#33), sum(cd_dep_college_count#33)] -Aggregate Attributes [10]: [count(1)#62, avg(cd_dep_count#31)#63, max(cd_dep_count#31)#64, sum(cd_dep_count#31)#65, avg(cd_dep_employed_count#32)#66, max(cd_dep_employed_count#32)#67, sum(cd_dep_employed_count#32)#68, avg(cd_dep_college_count#33)#69, max(cd_dep_college_count#33)#70, sum(cd_dep_college_count#33)#71] -Results [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, count(1)#62 AS cnt1#72, avg(cd_dep_count#31)#63 AS avg(cd_dep_count)#73, max(cd_dep_count#31)#64 AS max(cd_dep_count)#74, sum(cd_dep_count#31)#65 AS sum(cd_dep_count)#75, cd_dep_employed_count#32, count(1)#62 AS cnt2#76, avg(cd_dep_employed_count#32)#66 AS avg(cd_dep_employed_count)#77, max(cd_dep_employed_count#32)#67 AS max(cd_dep_employed_count)#78, sum(cd_dep_employed_count#32)#68 AS sum(cd_dep_employed_count)#79, cd_dep_college_count#33, count(1)#62 AS cnt3#80, avg(cd_dep_college_count#33)#69 AS avg(cd_dep_college_count)#81, max(cd_dep_college_count#33)#70 AS max(cd_dep_college_count)#82, sum(cd_dep_college_count#33)#71 AS sum(cd_dep_college_count)#83] +Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] +Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Functions [10]: [count(1), avg(cd_dep_count#29), max(cd_dep_count#29), sum(cd_dep_count#29), avg(cd_dep_employed_count#30), max(cd_dep_employed_count#30), sum(cd_dep_employed_count#30), avg(cd_dep_college_count#31), max(cd_dep_college_count#31), sum(cd_dep_college_count#31)] +Aggregate Attributes [10]: [count(1)#60, avg(cd_dep_count#29)#61, max(cd_dep_count#29)#62, sum(cd_dep_count#29)#63, avg(cd_dep_employed_count#30)#64, max(cd_dep_employed_count#30)#65, sum(cd_dep_employed_count#30)#66, avg(cd_dep_college_count#31)#67, max(cd_dep_college_count#31)#68, sum(cd_dep_college_count#31)#69] +Results [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, count(1)#60 AS cnt1#70, avg(cd_dep_count#29)#61 AS avg(cd_dep_count)#71, max(cd_dep_count#29)#62 AS max(cd_dep_count)#72, sum(cd_dep_count#29)#63 AS sum(cd_dep_count)#73, cd_dep_employed_count#30, count(1)#60 AS cnt2#74, avg(cd_dep_employed_count#30)#64 AS avg(cd_dep_employed_count)#75, max(cd_dep_employed_count#30)#65 AS max(cd_dep_employed_count)#76, sum(cd_dep_employed_count#30)#66 AS sum(cd_dep_employed_count)#77, cd_dep_college_count#31, count(1)#60 AS cnt3#78, avg(cd_dep_college_count#31)#67 AS avg(cd_dep_college_count)#79, max(cd_dep_college_count#31)#68 AS max(cd_dep_college_count)#80, sum(cd_dep_college_count#31)#69 AS sum(cd_dep_college_count)#81] (57) TakeOrderedAndProject -Input [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cnt1#72, avg(cd_dep_count)#73, max(cd_dep_count)#74, sum(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, avg(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, sum(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, avg(cd_dep_college_count)#81, max(cd_dep_college_count)#82, sum(cd_dep_college_count)#83] -Arguments: 100, [ca_state#25 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cnt1#72, avg(cd_dep_count)#73, max(cd_dep_count)#74, sum(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, avg(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, sum(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, avg(cd_dep_college_count)#81, max(cd_dep_college_count)#82, sum(cd_dep_college_count)#83] +Input [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#70, avg(cd_dep_count)#71, max(cd_dep_count)#72, sum(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, avg(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, sum(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, avg(cd_dep_college_count)#79, max(cd_dep_college_count)#80, sum(cd_dep_college_count)#81] +Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#70, avg(cd_dep_count)#71, max(cd_dep_count)#72, sum(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, avg(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, sum(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, avg(cd_dep_college_count)#79, max(cd_dep_college_count)#80, sum(cd_dep_college_count)#81] ===== Subqueries ===== @@ -323,6 +323,6 @@ Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index 642bf989159d3..12c8230c7e3bc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -122,20 +122,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#9] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#9] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#16] +Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] @@ -143,34 +143,34 @@ Right keys [1]: [ws_bill_customer_sk#14] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#20] +Output [1]: [d_date_sk#9] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#20] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#9] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#18] -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#20] +Output [1]: [cs_ship_customer_sk#17] +Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [cs_ship_customer_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#17] Join condition: None (29) Filter [codegen id : 9] @@ -182,80 +182,80 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_state#23] +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#22, ca_state#23] +Input [2]: [ca_address_sk#20, ca_state#21] (33) Filter [codegen id : 7] -Input [2]: [ca_address_sk#22, ca_state#23] -Condition : isnotnull(ca_address_sk#22) +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : isnotnull(ca_address_sk#20) (34) BroadcastExchange -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] +Input [2]: [ca_address_sk#20, ca_state#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] (35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#20] Join condition: None (36) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#23] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] +Output [2]: [c_current_cdemo_sk#4, ca_state#21] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] (37) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (39) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#25) +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Condition : isnotnull(cd_demo_sk#23) (40) BroadcastExchange -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#23] Join condition: None (42) Project [codegen id : 9] -Output [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] (43) HashAggregate [codegen id : 9] -Input [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#28), partial_max(cd_dep_count#28), partial_sum(cd_dep_count#28), partial_avg(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_sum(cd_dep_employed_count#29), partial_avg(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_sum(cd_dep_college_count#30)] -Aggregate Attributes [13]: [count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] -Results [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] +Input [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] +Aggregate Attributes [13]: [count#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42] +Results [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] (44) Exchange -Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] -Arguments: hashpartitioning(ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] +Arguments: hashpartitioning(ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [id=#56] (45) HashAggregate [codegen id : 10] -Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] -Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [10]: [count(1), avg(cd_dep_count#28), max(cd_dep_count#28), sum(cd_dep_count#28), avg(cd_dep_employed_count#29), max(cd_dep_employed_count#29), sum(cd_dep_employed_count#29), avg(cd_dep_college_count#30), max(cd_dep_college_count#30), sum(cd_dep_college_count#30)] -Aggregate Attributes [10]: [count(1)#59, avg(cd_dep_count#28)#60, max(cd_dep_count#28)#61, sum(cd_dep_count#28)#62, avg(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, sum(cd_dep_employed_count#29)#65, avg(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, sum(cd_dep_college_count#30)#68] -Results [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, count(1)#59 AS cnt1#69, avg(cd_dep_count#28)#60 AS avg(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, sum(cd_dep_count#28)#62 AS sum(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, avg(cd_dep_employed_count#29)#63 AS avg(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, sum(cd_dep_employed_count#29)#65 AS sum(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, avg(cd_dep_college_count#30)#66 AS avg(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, sum(cd_dep_college_count#30)#68 AS sum(cd_dep_college_count)#80] +Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] +Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] +Aggregate Attributes [10]: [count(1)#57, avg(cd_dep_count#26)#58, max(cd_dep_count#26)#59, sum(cd_dep_count#26)#60, avg(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, sum(cd_dep_employed_count#27)#63, avg(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, sum(cd_dep_college_count#28)#66] +Results [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, count(1)#57 AS cnt1#67, avg(cd_dep_count#26)#58 AS avg(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, sum(cd_dep_count#26)#60 AS sum(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, avg(cd_dep_employed_count#27)#61 AS avg(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, sum(cd_dep_employed_count#27)#63 AS sum(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, avg(cd_dep_college_count#28)#64 AS avg(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, sum(cd_dep_college_count#28)#66 AS sum(cd_dep_college_count)#78] (46) TakeOrderedAndProject -Input [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] -Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] +Input [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] +Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] ===== Subqueries ===== @@ -268,6 +268,6 @@ Output [1]: [d_date_sk#9] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt index df75bec72326d..1c1b8ad01abfa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt @@ -142,52 +142,52 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#8] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] -Right keys [1]: [d_date_sk#15] +Right keys [1]: [d_date_sk#8] Join condition: None (22) Project [codegen id : 7] -Output [1]: [ws_bill_customer_sk#13 AS customsk#16] -Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] +Output [1]: [ws_bill_customer_sk#13 AS customsk#15] +Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] (23) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] ReadSchema: struct (24) ColumnarToRow [codegen id : 9] -Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] (25) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#19] +Output [1]: [d_date_sk#8] (26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#19] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#8] Join condition: None (27) Project [codegen id : 9] -Output [1]: [cs_ship_customer_sk#17 AS customsk#20] -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] +Output [1]: [cs_ship_customer_sk#16 AS customsk#18] +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] (28) Union (29) Exchange -Input [1]: [customsk#16] -Arguments: hashpartitioning(customsk#16, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [1]: [customsk#15] +Arguments: hashpartitioning(customsk#15, 5), ENSURE_REQUIREMENTS, [id=#19] (30) Sort [codegen id : 10] -Input [1]: [customsk#16] -Arguments: [customsk#16 ASC NULLS FIRST], false, 0 +Input [1]: [customsk#15] +Arguments: [customsk#15 ASC NULLS FIRST], false, 0 (31) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#16] +Right keys [1]: [customsk#15] Join condition: None (32) Project [codegen id : 11] @@ -196,103 +196,103 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (33) Exchange Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: hashpartitioning(c_current_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#22] +Arguments: hashpartitioning(c_current_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#20] (34) Sort [codegen id : 12] Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: [c_current_addr_sk#3 ASC NULLS FIRST], false, 0 (35) Scan parquet default.customer_address -Output [2]: [ca_address_sk#23, ca_state#24] +Output [2]: [ca_address_sk#21, ca_state#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 13] -Input [2]: [ca_address_sk#23, ca_state#24] +Input [2]: [ca_address_sk#21, ca_state#22] (37) Filter [codegen id : 13] -Input [2]: [ca_address_sk#23, ca_state#24] -Condition : isnotnull(ca_address_sk#23) +Input [2]: [ca_address_sk#21, ca_state#22] +Condition : isnotnull(ca_address_sk#21) (38) Exchange -Input [2]: [ca_address_sk#23, ca_state#24] -Arguments: hashpartitioning(ca_address_sk#23, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [2]: [ca_address_sk#21, ca_state#22] +Arguments: hashpartitioning(ca_address_sk#21, 5), ENSURE_REQUIREMENTS, [id=#23] (39) Sort [codegen id : 14] -Input [2]: [ca_address_sk#23, ca_state#24] -Arguments: [ca_address_sk#23 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#21, ca_state#22] +Arguments: [ca_address_sk#21 ASC NULLS FIRST], false, 0 (40) SortMergeJoin [codegen id : 15] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#23] +Right keys [1]: [ca_address_sk#21] Join condition: None (41) Project [codegen id : 15] -Output [2]: [c_current_cdemo_sk#2, ca_state#24] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#23, ca_state#24] +Output [2]: [c_current_cdemo_sk#2, ca_state#22] +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21, ca_state#22] (42) Exchange -Input [2]: [c_current_cdemo_sk#2, ca_state#24] -Arguments: hashpartitioning(c_current_cdemo_sk#2, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [c_current_cdemo_sk#2, ca_state#22] +Arguments: hashpartitioning(c_current_cdemo_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] (43) Sort [codegen id : 16] -Input [2]: [c_current_cdemo_sk#2, ca_state#24] +Input [2]: [c_current_cdemo_sk#2, ca_state#22] Arguments: [c_current_cdemo_sk#2 ASC NULLS FIRST], false, 0 (44) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (45) ColumnarToRow [codegen id : 17] -Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (46) Filter [codegen id : 17] -Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Condition : isnotnull(cd_demo_sk#27) +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#25) (47) Exchange -Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Arguments: hashpartitioning(cd_demo_sk#27, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: hashpartitioning(cd_demo_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] (48) Sort [codegen id : 18] -Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Arguments: [cd_demo_sk#27 ASC NULLS FIRST], false, 0 +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: [cd_demo_sk#25 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 19] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#27] +Right keys [1]: [cd_demo_sk#25] Join condition: None (50) Project [codegen id : 19] -Output [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Input [8]: [c_current_cdemo_sk#2, ca_state#24, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Output [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (51) HashAggregate [codegen id : 19] -Input [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Keys [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#30), partial_max(cd_dep_count#30), partial_sum(cd_dep_count#30), partial_avg(cd_dep_employed_count#31), partial_max(cd_dep_employed_count#31), partial_sum(cd_dep_employed_count#31), partial_avg(cd_dep_college_count#32), partial_max(cd_dep_college_count#32), partial_sum(cd_dep_college_count#32)] -Aggregate Attributes [13]: [count#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42, sum#43, count#44, max#45, sum#46] -Results [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] +Input [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#28), partial_max(cd_dep_count#28), partial_sum(cd_dep_count#28), partial_avg(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_sum(cd_dep_employed_count#29), partial_avg(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_sum(cd_dep_college_count#30)] +Aggregate Attributes [13]: [count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] +Results [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] (52) Exchange -Input [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] -Arguments: hashpartitioning(ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] +Arguments: hashpartitioning(ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] (53) HashAggregate [codegen id : 20] -Input [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] -Keys [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Functions [10]: [count(1), avg(cd_dep_count#30), max(cd_dep_count#30), sum(cd_dep_count#30), avg(cd_dep_employed_count#31), max(cd_dep_employed_count#31), sum(cd_dep_employed_count#31), avg(cd_dep_college_count#32), max(cd_dep_college_count#32), sum(cd_dep_college_count#32)] -Aggregate Attributes [10]: [count(1)#61, avg(cd_dep_count#30)#62, max(cd_dep_count#30)#63, sum(cd_dep_count#30)#64, avg(cd_dep_employed_count#31)#65, max(cd_dep_employed_count#31)#66, sum(cd_dep_employed_count#31)#67, avg(cd_dep_college_count#32)#68, max(cd_dep_college_count#32)#69, sum(cd_dep_college_count#32)#70] -Results [18]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, count(1)#61 AS cnt1#71, avg(cd_dep_count#30)#62 AS avg(cd_dep_count)#72, max(cd_dep_count#30)#63 AS max(cd_dep_count)#73, sum(cd_dep_count#30)#64 AS sum(cd_dep_count)#74, cd_dep_employed_count#31, count(1)#61 AS cnt2#75, avg(cd_dep_employed_count#31)#65 AS avg(cd_dep_employed_count)#76, max(cd_dep_employed_count#31)#66 AS max(cd_dep_employed_count)#77, sum(cd_dep_employed_count#31)#67 AS sum(cd_dep_employed_count)#78, cd_dep_college_count#32, count(1)#61 AS cnt3#79, avg(cd_dep_college_count#32)#68 AS avg(cd_dep_college_count)#80, max(cd_dep_college_count#32)#69 AS max(cd_dep_college_count)#81, sum(cd_dep_college_count#32)#70 AS sum(cd_dep_college_count)#82] +Input [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] +Keys [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Functions [10]: [count(1), avg(cd_dep_count#28), max(cd_dep_count#28), sum(cd_dep_count#28), avg(cd_dep_employed_count#29), max(cd_dep_employed_count#29), sum(cd_dep_employed_count#29), avg(cd_dep_college_count#30), max(cd_dep_college_count#30), sum(cd_dep_college_count#30)] +Aggregate Attributes [10]: [count(1)#59, avg(cd_dep_count#28)#60, max(cd_dep_count#28)#61, sum(cd_dep_count#28)#62, avg(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, sum(cd_dep_employed_count#29)#65, avg(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, sum(cd_dep_college_count#30)#68] +Results [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, count(1)#59 AS cnt1#69, avg(cd_dep_count#28)#60 AS avg(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, sum(cd_dep_count#28)#62 AS sum(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, avg(cd_dep_employed_count#29)#63 AS avg(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, sum(cd_dep_employed_count#29)#65 AS sum(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, avg(cd_dep_college_count#30)#66 AS avg(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, sum(cd_dep_college_count#30)#68 AS sum(cd_dep_college_count)#80] (54) TakeOrderedAndProject -Input [18]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cnt1#71, avg(cd_dep_count)#72, max(cd_dep_count)#73, sum(cd_dep_count)#74, cd_dep_employed_count#31, cnt2#75, avg(cd_dep_employed_count)#76, max(cd_dep_employed_count)#77, sum(cd_dep_employed_count)#78, cd_dep_college_count#32, cnt3#79, avg(cd_dep_college_count)#80, max(cd_dep_college_count)#81, sum(cd_dep_college_count)#82] -Arguments: 100, [ca_state#24 ASC NULLS FIRST, cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cnt1#71, avg(cd_dep_count)#72, max(cd_dep_count)#73, sum(cd_dep_count)#74, cd_dep_employed_count#31, cnt2#75, avg(cd_dep_employed_count)#76, max(cd_dep_employed_count)#77, sum(cd_dep_employed_count)#78, cd_dep_college_count#32, cnt3#79, avg(cd_dep_college_count)#80, max(cd_dep_college_count)#81, sum(cd_dep_college_count)#82] +Input [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] +Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] ===== Subqueries ===== @@ -305,6 +305,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index 57710f427f0f8..ffb07d398de71 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -120,48 +120,48 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#7] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#7] Join condition: None (19) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#12 AS customsk#15] -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] +Output [1]: [ws_bill_customer_sk#12 AS customsk#14] +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] (20) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] (22) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#7] (23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#18] +Left keys [1]: [cs_sold_date_sk#16] +Right keys [1]: [d_date_sk#7] Join condition: None (24) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#16 AS customsk#19] -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#18] +Output [1]: [cs_ship_customer_sk#15 AS customsk#17] +Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] (25) Union (26) BroadcastExchange -Input [1]: [customsk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +Input [1]: [customsk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (27) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#15] +Right keys [1]: [customsk#14] Join condition: None (28) Project [codegen id : 9] @@ -169,80 +169,80 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] +Output [2]: [ca_address_sk#19, ca_state#20] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#21, ca_state#22] +Input [2]: [ca_address_sk#19, ca_state#20] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#21, ca_state#22] -Condition : isnotnull(ca_address_sk#21) +Input [2]: [ca_address_sk#19, ca_state#20] +Condition : isnotnull(ca_address_sk#19) (32) BroadcastExchange -Input [2]: [ca_address_sk#21, ca_state#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] +Input [2]: [ca_address_sk#19, ca_state#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (33) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#19] Join condition: None (34) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, ca_state#22] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21, ca_state#22] +Output [2]: [c_current_cdemo_sk#2, ca_state#20] +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19, ca_state#20] (35) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] (37) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Condition : isnotnull(cd_demo_sk#24) +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Condition : isnotnull(cd_demo_sk#22) (38) BroadcastExchange -Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] (39) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#24] +Right keys [1]: [cd_demo_sk#22] Join condition: None (40) Project [codegen id : 9] -Output [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Output [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Input [8]: [c_current_cdemo_sk#2, ca_state#20, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] (41) HashAggregate [codegen id : 9] -Input [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Keys [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#27), partial_max(cd_dep_count#27), partial_sum(cd_dep_count#27), partial_avg(cd_dep_employed_count#28), partial_max(cd_dep_employed_count#28), partial_sum(cd_dep_employed_count#28), partial_avg(cd_dep_college_count#29), partial_max(cd_dep_college_count#29), partial_sum(cd_dep_college_count#29)] -Aggregate Attributes [13]: [count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] -Results [19]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] +Input [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_dep_count#25), partial_sum(cd_dep_count#25), partial_avg(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_sum(cd_dep_employed_count#26), partial_avg(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_sum(cd_dep_college_count#27)] +Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] +Results [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] (42) Exchange -Input [19]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -Arguments: hashpartitioning(ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +Arguments: hashpartitioning(ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [id=#55] (43) HashAggregate [codegen id : 10] -Input [19]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] -Keys [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] -Functions [10]: [count(1), avg(cd_dep_count#27), max(cd_dep_count#27), sum(cd_dep_count#27), avg(cd_dep_employed_count#28), max(cd_dep_employed_count#28), sum(cd_dep_employed_count#28), avg(cd_dep_college_count#29), max(cd_dep_college_count#29), sum(cd_dep_college_count#29)] -Aggregate Attributes [10]: [count(1)#58, avg(cd_dep_count#27)#59, max(cd_dep_count#27)#60, sum(cd_dep_count#27)#61, avg(cd_dep_employed_count#28)#62, max(cd_dep_employed_count#28)#63, sum(cd_dep_employed_count#28)#64, avg(cd_dep_college_count#29)#65, max(cd_dep_college_count#29)#66, sum(cd_dep_college_count#29)#67] -Results [18]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, count(1)#58 AS cnt1#68, avg(cd_dep_count#27)#59 AS avg(cd_dep_count)#69, max(cd_dep_count#27)#60 AS max(cd_dep_count)#70, sum(cd_dep_count#27)#61 AS sum(cd_dep_count)#71, cd_dep_employed_count#28, count(1)#58 AS cnt2#72, avg(cd_dep_employed_count#28)#62 AS avg(cd_dep_employed_count)#73, max(cd_dep_employed_count#28)#63 AS max(cd_dep_employed_count)#74, sum(cd_dep_employed_count#28)#64 AS sum(cd_dep_employed_count)#75, cd_dep_college_count#29, count(1)#58 AS cnt3#76, avg(cd_dep_college_count#29)#65 AS avg(cd_dep_college_count)#77, max(cd_dep_college_count#29)#66 AS max(cd_dep_college_count)#78, sum(cd_dep_college_count#29)#67 AS sum(cd_dep_college_count)#79] +Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_dep_count#25), avg(cd_dep_employed_count#26), max(cd_dep_employed_count#26), sum(cd_dep_employed_count#26), avg(cd_dep_college_count#27), max(cd_dep_college_count#27), sum(cd_dep_college_count#27)] +Aggregate Attributes [10]: [count(1)#56, avg(cd_dep_count#25)#57, max(cd_dep_count#25)#58, sum(cd_dep_count#25)#59, avg(cd_dep_employed_count#26)#60, max(cd_dep_employed_count#26)#61, sum(cd_dep_employed_count#26)#62, avg(cd_dep_college_count#27)#63, max(cd_dep_college_count#27)#64, sum(cd_dep_college_count#27)#65] +Results [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, count(1)#56 AS cnt1#66, avg(cd_dep_count#25)#57 AS avg(cd_dep_count)#67, max(cd_dep_count#25)#58 AS max(cd_dep_count)#68, sum(cd_dep_count#25)#59 AS sum(cd_dep_count)#69, cd_dep_employed_count#26, count(1)#56 AS cnt2#70, avg(cd_dep_employed_count#26)#60 AS avg(cd_dep_employed_count)#71, max(cd_dep_employed_count#26)#61 AS max(cd_dep_employed_count)#72, sum(cd_dep_employed_count#26)#62 AS sum(cd_dep_employed_count)#73, cd_dep_college_count#27, count(1)#56 AS cnt3#74, avg(cd_dep_college_count#27)#63 AS avg(cd_dep_college_count)#75, max(cd_dep_college_count#27)#64 AS max(cd_dep_college_count)#76, sum(cd_dep_college_count#27)#65 AS sum(cd_dep_college_count)#77] (44) TakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#28, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#29, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79] -Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_dep_count#27 ASC NULLS FIRST, cd_dep_employed_count#28 ASC NULLS FIRST, cd_dep_college_count#29 ASC NULLS FIRST], [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#28, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#29, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79] +Input [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] +Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] ===== Subqueries ===== @@ -255,6 +255,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 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 79037befbd6b5..9f594df6754ba 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 @@ -169,60 +169,60 @@ Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#22, sum(UnscaledV Results [6]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#22,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#23,17,2))), DecimalType(37,20), true) as decimal(38,20)) AS gross_margin#24, i_category#15, i_class#14, 0 AS t_category#25, 0 AS t_class#26, 0 AS lochierarchy#27] (27) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#28, i_class#29, sum#30, sum#31] +Output [4]: [i_category#15, i_class#14, sum#28, sum#29] (28) HashAggregate [codegen id : 10] -Input [4]: [i_category#28, i_class#29, sum#30, sum#31] -Keys [2]: [i_category#28, i_class#29] -Functions [2]: [sum(UnscaledValue(ss_net_profit#32)), sum(UnscaledValue(ss_ext_sales_price#33))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#32))#34, sum(UnscaledValue(ss_ext_sales_price#33))#35] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#32))#34,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#33))#35,17,2) AS ss_ext_sales_price#37, i_category#28] +Input [4]: [i_category#15, i_class#14, sum#28, sum#29] +Keys [2]: [i_category#15, i_class#14] +Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#30, sum(UnscaledValue(ss_ext_sales_price#3))#31] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#30,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#31,17,2) AS ss_ext_sales_price#33, i_category#15] (29) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#36, ss_ext_sales_price#37, i_category#28] -Keys [1]: [i_category#28] -Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] -Aggregate Attributes [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] -Results [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] +Input [3]: [ss_net_profit#32, ss_ext_sales_price#33, i_category#15] +Keys [1]: [i_category#15] +Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] +Aggregate Attributes [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] +Results [5]: [i_category#15, sum#38, isEmpty#39, sum#40, isEmpty#41] (30) Exchange -Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [5]: [i_category#15, sum#38, isEmpty#39, sum#40, isEmpty#41] +Arguments: hashpartitioning(i_category#15, 5), ENSURE_REQUIREMENTS, [id=#42] (31) HashAggregate [codegen id : 11] -Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] -Keys [1]: [i_category#28] -Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] -Aggregate Attributes [2]: [sum(ss_net_profit#36)#47, sum(ss_ext_sales_price#37)#48] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#47) / promote_precision(sum(ss_ext_sales_price#37)#48)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#49, i_category#28, null AS i_class#50, 0 AS t_category#51, 1 AS t_class#52, 1 AS lochierarchy#53] +Input [5]: [i_category#15, sum#38, isEmpty#39, sum#40, isEmpty#41] +Keys [1]: [i_category#15] +Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] +Aggregate Attributes [2]: [sum(ss_net_profit#32)#43, sum(ss_ext_sales_price#33)#44] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#43) / promote_precision(sum(ss_ext_sales_price#33)#44)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#45, i_category#15, null AS i_class#46, 0 AS t_category#47, 1 AS t_class#48, 1 AS lochierarchy#49] (32) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#54, i_class#55, sum#56, sum#57] +Output [4]: [i_category#15, i_class#14, sum#50, sum#51] (33) HashAggregate [codegen id : 16] -Input [4]: [i_category#54, i_class#55, sum#56, sum#57] -Keys [2]: [i_category#54, i_class#55] -Functions [2]: [sum(UnscaledValue(ss_net_profit#58)), sum(UnscaledValue(ss_ext_sales_price#59))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#58))#60, sum(UnscaledValue(ss_ext_sales_price#59))#61] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#58))#60,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#59))#61,17,2) AS ss_ext_sales_price#37] +Input [4]: [i_category#15, i_class#14, sum#50, sum#51] +Keys [2]: [i_category#15, i_class#14] +Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#52, sum(UnscaledValue(ss_ext_sales_price#3))#53] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#52,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#53,17,2) AS ss_ext_sales_price#33] (34) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#36, ss_ext_sales_price#37] +Input [2]: [ss_net_profit#32, ss_ext_sales_price#33] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] -Aggregate Attributes [4]: [sum#62, isEmpty#63, sum#64, isEmpty#65] -Results [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] +Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] +Aggregate Attributes [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] +Results [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] (35) Exchange -Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#70] +Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] (36) HashAggregate [codegen id : 17] -Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] +Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] Keys: [] -Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] -Aggregate Attributes [2]: [sum(ss_net_profit#36)#71, sum(ss_ext_sales_price#37)#72] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#71) / promote_precision(sum(ss_ext_sales_price#37)#72)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#73, null AS i_category#74, null AS i_class#75, 1 AS t_category#76, 1 AS t_class#77, 2 AS lochierarchy#78] +Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] +Aggregate Attributes [2]: [sum(ss_net_profit#32)#63, sum(ss_ext_sales_price#33)#64] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#63) / promote_precision(sum(ss_ext_sales_price#33)#64)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#65, null AS i_category#66, null AS i_class#67, 1 AS t_category#68, 1 AS t_class#69, 2 AS lochierarchy#70] (37) Union @@ -235,34 +235,34 @@ Results [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class (39) Exchange Input [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27] -Arguments: hashpartitioning(gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#79] +Arguments: hashpartitioning(gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#71] (40) HashAggregate [codegen id : 19] Input [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27] Keys [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27] Functions: [] Aggregate Attributes: [] -Results [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#15 END AS _w0#80] +Results [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#15 END AS _w0#72] (41) Exchange -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80] -Arguments: hashpartitioning(lochierarchy#27, _w0#80, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72] +Arguments: hashpartitioning(lochierarchy#27, _w0#72, 5), ENSURE_REQUIREMENTS, [id=#73] (42) Sort [codegen id : 20] -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80] -Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#80 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72] +Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 (43) Window -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80] -Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#80, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#82], [lochierarchy#27, _w0#80], [gross_margin#24 ASC NULLS FIRST] +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72] +Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#72, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#74], [lochierarchy#27, _w0#72], [gross_margin#24 ASC NULLS FIRST] (44) Project [codegen id : 21] -Output [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#82] -Input [6]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80, rank_within_parent#82] +Output [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#74] +Input [6]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72, rank_within_parent#74] (45) TakeOrderedAndProject -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#82] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#82 ASC NULLS FIRST], [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#82] +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#74] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#74 ASC NULLS FIRST], [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#74] ===== Subqueries ===== 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 bc2931129868b..5e58913557f22 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 @@ -169,60 +169,60 @@ Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#22, sum(UnscaledV Results [6]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#22,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#23,17,2))), DecimalType(37,20), true) as decimal(38,20)) AS gross_margin#24, i_category#12, i_class#11, 0 AS t_category#25, 0 AS t_class#26, 0 AS lochierarchy#27] (27) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#28, i_class#29, sum#30, sum#31] +Output [4]: [i_category#12, i_class#11, sum#28, sum#29] (28) HashAggregate [codegen id : 10] -Input [4]: [i_category#28, i_class#29, sum#30, sum#31] -Keys [2]: [i_category#28, i_class#29] -Functions [2]: [sum(UnscaledValue(ss_net_profit#32)), sum(UnscaledValue(ss_ext_sales_price#33))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#32))#34, sum(UnscaledValue(ss_ext_sales_price#33))#35] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#32))#34,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#33))#35,17,2) AS ss_ext_sales_price#37, i_category#28] +Input [4]: [i_category#12, i_class#11, sum#28, sum#29] +Keys [2]: [i_category#12, i_class#11] +Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#30, sum(UnscaledValue(ss_ext_sales_price#3))#31] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#30,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#31,17,2) AS ss_ext_sales_price#33, i_category#12] (29) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#36, ss_ext_sales_price#37, i_category#28] -Keys [1]: [i_category#28] -Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] -Aggregate Attributes [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] -Results [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] +Input [3]: [ss_net_profit#32, ss_ext_sales_price#33, i_category#12] +Keys [1]: [i_category#12] +Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] +Aggregate Attributes [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] +Results [5]: [i_category#12, sum#38, isEmpty#39, sum#40, isEmpty#41] (30) Exchange -Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [5]: [i_category#12, sum#38, isEmpty#39, sum#40, isEmpty#41] +Arguments: hashpartitioning(i_category#12, 5), ENSURE_REQUIREMENTS, [id=#42] (31) HashAggregate [codegen id : 11] -Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] -Keys [1]: [i_category#28] -Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] -Aggregate Attributes [2]: [sum(ss_net_profit#36)#47, sum(ss_ext_sales_price#37)#48] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#47) / promote_precision(sum(ss_ext_sales_price#37)#48)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#49, i_category#28, null AS i_class#50, 0 AS t_category#51, 1 AS t_class#52, 1 AS lochierarchy#53] +Input [5]: [i_category#12, sum#38, isEmpty#39, sum#40, isEmpty#41] +Keys [1]: [i_category#12] +Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] +Aggregate Attributes [2]: [sum(ss_net_profit#32)#43, sum(ss_ext_sales_price#33)#44] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#43) / promote_precision(sum(ss_ext_sales_price#33)#44)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#45, i_category#12, null AS i_class#46, 0 AS t_category#47, 1 AS t_class#48, 1 AS lochierarchy#49] (32) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#54, i_class#55, sum#56, sum#57] +Output [4]: [i_category#12, i_class#11, sum#50, sum#51] (33) HashAggregate [codegen id : 16] -Input [4]: [i_category#54, i_class#55, sum#56, sum#57] -Keys [2]: [i_category#54, i_class#55] -Functions [2]: [sum(UnscaledValue(ss_net_profit#58)), sum(UnscaledValue(ss_ext_sales_price#59))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#58))#60, sum(UnscaledValue(ss_ext_sales_price#59))#61] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#58))#60,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#59))#61,17,2) AS ss_ext_sales_price#37] +Input [4]: [i_category#12, i_class#11, sum#50, sum#51] +Keys [2]: [i_category#12, i_class#11] +Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#52, sum(UnscaledValue(ss_ext_sales_price#3))#53] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#52,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#53,17,2) AS ss_ext_sales_price#33] (34) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#36, ss_ext_sales_price#37] +Input [2]: [ss_net_profit#32, ss_ext_sales_price#33] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] -Aggregate Attributes [4]: [sum#62, isEmpty#63, sum#64, isEmpty#65] -Results [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] +Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] +Aggregate Attributes [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] +Results [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] (35) Exchange -Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#70] +Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] (36) HashAggregate [codegen id : 17] -Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] +Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] Keys: [] -Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] -Aggregate Attributes [2]: [sum(ss_net_profit#36)#71, sum(ss_ext_sales_price#37)#72] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#71) / promote_precision(sum(ss_ext_sales_price#37)#72)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#73, null AS i_category#74, null AS i_class#75, 1 AS t_category#76, 1 AS t_class#77, 2 AS lochierarchy#78] +Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] +Aggregate Attributes [2]: [sum(ss_net_profit#32)#63, sum(ss_ext_sales_price#33)#64] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#63) / promote_precision(sum(ss_ext_sales_price#33)#64)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#65, null AS i_category#66, null AS i_class#67, 1 AS t_category#68, 1 AS t_class#69, 2 AS lochierarchy#70] (37) Union @@ -235,34 +235,34 @@ Results [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class (39) Exchange Input [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27] -Arguments: hashpartitioning(gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#79] +Arguments: hashpartitioning(gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#71] (40) HashAggregate [codegen id : 19] Input [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27] Keys [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27] Functions: [] Aggregate Attributes: [] -Results [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#12 END AS _w0#80] +Results [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#12 END AS _w0#72] (41) Exchange -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80] -Arguments: hashpartitioning(lochierarchy#27, _w0#80, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72] +Arguments: hashpartitioning(lochierarchy#27, _w0#72, 5), ENSURE_REQUIREMENTS, [id=#73] (42) Sort [codegen id : 20] -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80] -Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#80 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72] +Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 (43) Window -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80] -Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#80, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#82], [lochierarchy#27, _w0#80], [gross_margin#24 ASC NULLS FIRST] +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72] +Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#72, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#74], [lochierarchy#27, _w0#72], [gross_margin#24 ASC NULLS FIRST] (44) Project [codegen id : 21] -Output [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#82] -Input [6]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80, rank_within_parent#82] +Output [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#74] +Input [6]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72, rank_within_parent#74] (45) TakeOrderedAndProject -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#82] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#82 ASC NULLS FIRST], [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#82] +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#74] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#74 ASC NULLS FIRST], [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#74] ===== Subqueries ===== 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 636fa32f94212..a0737e88bd760 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 @@ -229,78 +229,78 @@ Output [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_ye (39) HashAggregate [codegen id : 21] Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum#36] Keys [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35] -Functions [1]: [sum(UnscaledValue(ss_sales_price#37))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#37))#38] -Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#37))#38,17,2) AS sum_sales#39] +Functions [1]: [sum(UnscaledValue(ss_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#3))#37] +Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#3))#37,17,2) AS sum_sales#38] (40) Exchange -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#39] (41) Sort [codegen id : 22] -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST], false, 0 (42) Window -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] (43) Project [codegen id : 23] -Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] -Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] +Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] +Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38, rn#40] (44) Exchange -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1), 5), ENSURE_REQUIREMENTS, [id=#42] +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1), 5), ENSURE_REQUIREMENTS, [id=#41] (45) Sort [codegen id : 24] -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#41 + 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] +Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#40 + 1) ASC NULLS FIRST], false, 0 (46) SortMergeJoin [codegen id : 25] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1)] +Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1)] Join condition: None (47) Project [codegen id : 25] -Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39] -Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38] +Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] (48) ReusedExchange [Reuses operator id: 40] -Output [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] +Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] (49) Sort [codegen id : 34] -Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] -Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 (50) Window -Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] -Arguments: [rank(d_year#47, d_moy#48) windowspecdefinition(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#50], [i_category#43, i_brand#44, s_store_name#45, s_company_name#46], [d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST] +Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] (51) Project [codegen id : 35] -Output [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] -Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] +Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] +Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] (52) Exchange -Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] -Arguments: hashpartitioning(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1), 5), ENSURE_REQUIREMENTS, [id=#51] +Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] +Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1), 5), ENSURE_REQUIREMENTS, [id=#50] (53) Sort [codegen id : 36] -Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] -Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, (rn#50 - 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] +Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (rn#49 - 1) ASC NULLS FIRST], false, 0 (54) SortMergeJoin [codegen id : 37] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1)] +Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1)] Join condition: None (55) Project [codegen id : 37] -Output [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#39 AS psum#52, sum_sales#49 AS nsum#53] -Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39, i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Output [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#38 AS psum#51, sum_sales#48 AS nsum#52] +Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] (56) TakeOrderedAndProject -Input [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] +Input [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] ===== Subqueries ===== 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 734eec1be3565..b0a231fc01379 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 @@ -202,70 +202,70 @@ Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_ye (34) HashAggregate [codegen id : 13] 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] +Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#35] +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#6))#35,17,2) AS sum_sales#36] (35) 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#36] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#37] (36) Sort [codegen id : 14] -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#36] 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 (37) 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#36] +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#38], [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] (38) Project [codegen id : 15] -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#36, rn#38] +Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] (39) BroadcastExchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] -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=#40] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] +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=#39] (40) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -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#38 + 1)] Join condition: None (41) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] -Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] +Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] (42) ReusedExchange [Reuses operator id: 35] -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#46] (43) Sort [codegen id : 21] -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#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, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 (44) 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#46] +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#47], [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] (45) Project [codegen id : 22] -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#46, rn#47] +Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] (46) BroadcastExchange -Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] -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=#49] +Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] +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=#48] (47) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -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#47 - 1)] Join condition: None (48) Project [codegen id : 23] -Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] -Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, 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#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#49, sum_sales#46 AS nsum#50] +Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] (49) TakeOrderedAndProject -Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, 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#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#22, psum#50, nsum#51] +Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, 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#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#22, psum#49, nsum#50] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt index a1c7e4bfb35dc..57e57951851aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt @@ -250,225 +250,225 @@ Output [5]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_net_profit#47, cs_sold_date_sk#48] (36) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#49] +Output [1]: [d_date_sk#8] (37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_sold_date_sk#48] -Right keys [1]: [d_date_sk#49] +Right keys [1]: [d_date_sk#8] Join condition: None (38) Project [codegen id : 12] Output [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#49] +Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#8] (39) Exchange Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#50] +Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#49] (40) Sort [codegen id : 13] Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] Arguments: [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST], false, 0 (41) Scan parquet default.catalog_returns -Output [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Output [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (42) ColumnarToRow [codegen id : 14] -Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] (43) Filter [codegen id : 14] -Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] -Condition : (((isnotnull(cr_return_amount#54) AND (cr_return_amount#54 > 10000.00)) AND isnotnull(cr_order_number#52)) AND isnotnull(cr_item_sk#51)) +Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Condition : (((isnotnull(cr_return_amount#53) AND (cr_return_amount#53 > 10000.00)) AND isnotnull(cr_order_number#51)) AND isnotnull(cr_item_sk#50)) (44) Project [codegen id : 14] -Output [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] -Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Output [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] (45) Exchange -Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] -Arguments: hashpartitioning(cr_order_number#52, cr_item_sk#51, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#55] (46) Sort [codegen id : 15] -Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] -Arguments: [cr_order_number#52 ASC NULLS FIRST, cr_item_sk#51 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Arguments: [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 16] Left keys [2]: [cs_order_number#44, cs_item_sk#43] -Right keys [2]: [cr_order_number#52, cr_item_sk#51] +Right keys [2]: [cr_order_number#51, cr_item_sk#50] Join condition: None (48) Project [codegen id : 16] -Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] -Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] +Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] (49) HashAggregate [codegen id : 16] -Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] +Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] Keys [1]: [cs_item_sk#43] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#53, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] -Results [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#52, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#56, sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] +Results [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] (50) Exchange -Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#68] (51) HashAggregate [codegen id : 17] -Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] Keys [1]: [cs_item_sk#43] -Functions [4]: [sum(coalesce(cr_return_quantity#53, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#53, 0))#70, sum(coalesce(cs_quantity#45, 0))#71, sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73] -Results [3]: [cs_item_sk#43 AS item#74, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#53, 0))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#71 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#75, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#76] +Functions [4]: [sum(coalesce(cr_return_quantity#52, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#52, 0))#69, sum(coalesce(cs_quantity#45, 0))#70, sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72] +Results [3]: [cs_item_sk#43 AS item#73, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#52, 0))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#70 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#74, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#75] (52) Exchange -Input [3]: [item#74, return_ratio#75, currency_ratio#76] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#77] +Input [3]: [item#73, return_ratio#74, currency_ratio#75] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] (53) Sort [codegen id : 18] -Input [3]: [item#74, return_ratio#75, currency_ratio#76] -Arguments: [return_ratio#75 ASC NULLS FIRST], false, 0 +Input [3]: [item#73, return_ratio#74, currency_ratio#75] +Arguments: [return_ratio#74 ASC NULLS FIRST], false, 0 (54) Window -Input [3]: [item#74, return_ratio#75, currency_ratio#76] -Arguments: [rank(return_ratio#75) windowspecdefinition(return_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#78], [return_ratio#75 ASC NULLS FIRST] +Input [3]: [item#73, return_ratio#74, currency_ratio#75] +Arguments: [rank(return_ratio#74) windowspecdefinition(return_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#77], [return_ratio#74 ASC NULLS FIRST] (55) Sort [codegen id : 19] -Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] -Arguments: [currency_ratio#76 ASC NULLS FIRST], false, 0 +Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] +Arguments: [currency_ratio#75 ASC NULLS FIRST], false, 0 (56) Window -Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] -Arguments: [rank(currency_ratio#76) windowspecdefinition(currency_ratio#76 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#79], [currency_ratio#76 ASC NULLS FIRST] +Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] +Arguments: [rank(currency_ratio#75) windowspecdefinition(currency_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#78], [currency_ratio#75 ASC NULLS FIRST] (57) Filter [codegen id : 20] -Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] -Condition : ((return_rank#78 <= 10) OR (currency_rank#79 <= 10)) +Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] +Condition : ((return_rank#77 <= 10) OR (currency_rank#78 <= 10)) (58) Project [codegen id : 20] -Output [5]: [catalog AS channel#80, item#74, return_ratio#75, return_rank#78, currency_rank#79] -Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] +Output [5]: [catalog AS channel#79, item#73, return_ratio#74, return_rank#77, currency_rank#78] +Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] (59) Scan parquet default.store_sales -Output [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Output [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_sold_date_sk#85 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 22] -Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] (61) Filter [codegen id : 22] -Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] -Condition : (((((((isnotnull(ss_net_profit#85) AND isnotnull(ss_net_paid#84)) AND isnotnull(ss_quantity#83)) AND (ss_net_profit#85 > 1.00)) AND (ss_net_paid#84 > 0.00)) AND (ss_quantity#83 > 0)) AND isnotnull(ss_ticket_number#82)) AND isnotnull(ss_item_sk#81)) +Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Condition : (((((((isnotnull(ss_net_profit#84) AND isnotnull(ss_net_paid#83)) AND isnotnull(ss_quantity#82)) AND (ss_net_profit#84 > 1.00)) AND (ss_net_paid#83 > 0.00)) AND (ss_quantity#82 > 0)) AND isnotnull(ss_ticket_number#81)) AND isnotnull(ss_item_sk#80)) (62) Project [codegen id : 22] -Output [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86] -Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Output [5]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85] +Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] (63) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#87] +Output [1]: [d_date_sk#8] (64) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ss_sold_date_sk#86] -Right keys [1]: [d_date_sk#87] +Left keys [1]: [ss_sold_date_sk#85] +Right keys [1]: [d_date_sk#8] Join condition: None (65) Project [codegen id : 22] -Output [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] -Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86, d_date_sk#87] +Output [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] +Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85, d_date_sk#8] (66) Exchange -Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] -Arguments: hashpartitioning(cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint), 5), ENSURE_REQUIREMENTS, [id=#88] +Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] +Arguments: hashpartitioning(cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint), 5), ENSURE_REQUIREMENTS, [id=#86] (67) Sort [codegen id : 23] -Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] -Arguments: [cast(ss_ticket_number#82 as bigint) ASC NULLS FIRST, cast(ss_item_sk#81 as bigint) ASC NULLS FIRST], false, 0 +Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] +Arguments: [cast(ss_ticket_number#81 as bigint) ASC NULLS FIRST, cast(ss_item_sk#80 as bigint) ASC NULLS FIRST], false, 0 (68) Scan parquet default.store_returns -Output [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Output [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 24] -Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] (70) Filter [codegen id : 24] -Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] -Condition : (((isnotnull(sr_return_amt#92) AND (sr_return_amt#92 > 10000.00)) AND isnotnull(sr_ticket_number#90)) AND isnotnull(sr_item_sk#89)) +Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Condition : (((isnotnull(sr_return_amt#90) AND (sr_return_amt#90 > 10000.00)) AND isnotnull(sr_ticket_number#88)) AND isnotnull(sr_item_sk#87)) (71) Project [codegen id : 24] -Output [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] -Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Output [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] (72) Exchange -Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] -Arguments: hashpartitioning(sr_ticket_number#90, sr_item_sk#89, 5), ENSURE_REQUIREMENTS, [id=#94] +Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Arguments: hashpartitioning(sr_ticket_number#88, sr_item_sk#87, 5), ENSURE_REQUIREMENTS, [id=#92] (73) Sort [codegen id : 25] -Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] -Arguments: [sr_ticket_number#90 ASC NULLS FIRST, sr_item_sk#89 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Arguments: [sr_ticket_number#88 ASC NULLS FIRST, sr_item_sk#87 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 26] -Left keys [2]: [cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint)] -Right keys [2]: [sr_ticket_number#90, sr_item_sk#89] +Left keys [2]: [cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint)] +Right keys [2]: [sr_ticket_number#88, sr_item_sk#87] Join condition: None (75) Project [codegen id : 26] -Output [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] -Input [8]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Output [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] +Input [8]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] (76) HashAggregate [codegen id : 26] -Input [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] -Keys [1]: [ss_item_sk#81] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#91, 0)), partial_sum(coalesce(ss_quantity#83, 0)), partial_sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#95, sum#96, sum#97, isEmpty#98, sum#99, isEmpty#100] -Results [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] +Input [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] +Keys [1]: [ss_item_sk#80] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#89, 0)), partial_sum(coalesce(ss_quantity#82, 0)), partial_sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#93, sum#94, sum#95, isEmpty#96, sum#97, isEmpty#98] +Results [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] (77) Exchange -Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] -Arguments: hashpartitioning(ss_item_sk#81, 5), ENSURE_REQUIREMENTS, [id=#107] +Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Arguments: hashpartitioning(ss_item_sk#80, 5), ENSURE_REQUIREMENTS, [id=#105] (78) HashAggregate [codegen id : 27] -Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] -Keys [1]: [ss_item_sk#81] -Functions [4]: [sum(coalesce(sr_return_quantity#91, 0)), sum(coalesce(ss_quantity#83, 0)), sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#91, 0))#108, sum(coalesce(ss_quantity#83, 0))#109, sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110, sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111] -Results [3]: [ss_item_sk#81 AS item#112, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#91, 0))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#83, 0))#109 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#113, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#114] +Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Keys [1]: [ss_item_sk#80] +Functions [4]: [sum(coalesce(sr_return_quantity#89, 0)), sum(coalesce(ss_quantity#82, 0)), sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#89, 0))#106, sum(coalesce(ss_quantity#82, 0))#107, sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108, sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109] +Results [3]: [ss_item_sk#80 AS item#110, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#89, 0))#106 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#82, 0))#107 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#111, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#112] (79) Exchange -Input [3]: [item#112, return_ratio#113, currency_ratio#114] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] +Input [3]: [item#110, return_ratio#111, currency_ratio#112] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] (80) Sort [codegen id : 28] -Input [3]: [item#112, return_ratio#113, currency_ratio#114] -Arguments: [return_ratio#113 ASC NULLS FIRST], false, 0 +Input [3]: [item#110, return_ratio#111, currency_ratio#112] +Arguments: [return_ratio#111 ASC NULLS FIRST], false, 0 (81) Window -Input [3]: [item#112, return_ratio#113, currency_ratio#114] -Arguments: [rank(return_ratio#113) windowspecdefinition(return_ratio#113 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#116], [return_ratio#113 ASC NULLS FIRST] +Input [3]: [item#110, return_ratio#111, currency_ratio#112] +Arguments: [rank(return_ratio#111) windowspecdefinition(return_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#114], [return_ratio#111 ASC NULLS FIRST] (82) Sort [codegen id : 29] -Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] -Arguments: [currency_ratio#114 ASC NULLS FIRST], false, 0 +Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] +Arguments: [currency_ratio#112 ASC NULLS FIRST], false, 0 (83) Window -Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] -Arguments: [rank(currency_ratio#114) windowspecdefinition(currency_ratio#114 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#117], [currency_ratio#114 ASC NULLS FIRST] +Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] +Arguments: [rank(currency_ratio#112) windowspecdefinition(currency_ratio#112 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#115], [currency_ratio#112 ASC NULLS FIRST] (84) Filter [codegen id : 30] -Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] -Condition : ((return_rank#116 <= 10) OR (currency_rank#117 <= 10)) +Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] +Condition : ((return_rank#114 <= 10) OR (currency_rank#115 <= 10)) (85) Project [codegen id : 30] -Output [5]: [store AS channel#118, item#112, return_ratio#113, return_rank#116, currency_rank#117] -Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] +Output [5]: [store AS channel#116, item#110, return_ratio#111, return_rank#114, currency_rank#115] +Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] (86) Union @@ -481,7 +481,7 @@ Results [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_ran (88) Exchange Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] -Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#119] +Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#117] (89) HashAggregate [codegen id : 32] Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] @@ -505,6 +505,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#85 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index 74b789e4a7a65..ada8c0415c672 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -260,170 +260,170 @@ Output [6]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, Input [9]: [cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_item_sk#49, cr_order_number#50, cr_return_quantity#51, cr_return_amount#52] (40) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#54] +Output [1]: [d_date_sk#14] (41) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#47] -Right keys [1]: [d_date_sk#54] +Right keys [1]: [d_date_sk#14] Join condition: None (42) Project [codegen id : 10] Output [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] -Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#54] +Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#14] (43) HashAggregate [codegen id : 10] Input [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] Keys [1]: [cs_item_sk#42] Functions [4]: [partial_sum(coalesce(cr_return_quantity#51, 0)), partial_sum(coalesce(cs_quantity#44, 0)), partial_sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] -Results [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Aggregate Attributes [6]: [sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] +Results [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] (44) Exchange -Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] -Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#67] +Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] +Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#66] (45) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] Keys [1]: [cs_item_sk#42] Functions [4]: [sum(coalesce(cr_return_quantity#51, 0)), sum(coalesce(cs_quantity#44, 0)), sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#68, sum(coalesce(cs_quantity#44, 0))#69, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71] -Results [3]: [cs_item_sk#42 AS item#72, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#68 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#69 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#73, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#74] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#67, sum(coalesce(cs_quantity#44, 0))#68, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70] +Results [3]: [cs_item_sk#42 AS item#71, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#67 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#68 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#72, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#73] (46) Exchange -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#75] +Input [3]: [item#71, return_ratio#72, currency_ratio#73] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#74] (47) Sort [codegen id : 12] -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [return_ratio#73 ASC NULLS FIRST], false, 0 +Input [3]: [item#71, return_ratio#72, currency_ratio#73] +Arguments: [return_ratio#72 ASC NULLS FIRST], false, 0 (48) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#76], [return_ratio#73 ASC NULLS FIRST] +Input [3]: [item#71, return_ratio#72, currency_ratio#73] +Arguments: [rank(return_ratio#72) windowspecdefinition(return_ratio#72 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#72 ASC NULLS FIRST] (49) Sort [codegen id : 13] -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 +Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] +Arguments: [currency_ratio#73 ASC NULLS FIRST], false, 0 (50) Window -Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#77], [currency_ratio#74 ASC NULLS FIRST] +Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] +Arguments: [rank(currency_ratio#73) windowspecdefinition(currency_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#73 ASC NULLS FIRST] (51) Filter [codegen id : 14] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] -Condition : ((return_rank#76 <= 10) OR (currency_rank#77 <= 10)) +Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] +Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) (52) Project [codegen id : 14] -Output [5]: [catalog AS channel#78, item#72, return_ratio#73, return_rank#76, currency_rank#77] -Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] +Output [5]: [catalog AS channel#77, item#71, return_ratio#72, return_rank#75, currency_rank#76] +Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] (53) Scan parquet default.store_sales -Output [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Output [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#84), dynamicpruningexpression(ss_sold_date_sk#84 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#83), dynamicpruningexpression(ss_sold_date_sk#83 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 15] -Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] (55) Filter [codegen id : 15] -Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] -Condition : (((((((isnotnull(ss_net_profit#83) AND isnotnull(ss_net_paid#82)) AND isnotnull(ss_quantity#81)) AND (ss_net_profit#83 > 1.00)) AND (ss_net_paid#82 > 0.00)) AND (ss_quantity#81 > 0)) AND isnotnull(ss_ticket_number#80)) AND isnotnull(ss_item_sk#79)) +Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Condition : (((((((isnotnull(ss_net_profit#82) AND isnotnull(ss_net_paid#81)) AND isnotnull(ss_quantity#80)) AND (ss_net_profit#82 > 1.00)) AND (ss_net_paid#81 > 0.00)) AND (ss_quantity#80 > 0)) AND isnotnull(ss_ticket_number#79)) AND isnotnull(ss_item_sk#78)) (56) Project [codegen id : 15] -Output [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] -Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Output [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] +Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] (57) BroadcastExchange -Input [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#85] +Input [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#84] (58) Scan parquet default.store_returns -Output [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Output [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (59) ColumnarToRow -Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] (60) Filter -Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] -Condition : (((isnotnull(sr_return_amt#89) AND (sr_return_amt#89 > 10000.00)) AND isnotnull(sr_ticket_number#87)) AND isnotnull(sr_item_sk#86)) +Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Condition : (((isnotnull(sr_return_amt#88) AND (sr_return_amt#88 > 10000.00)) AND isnotnull(sr_ticket_number#86)) AND isnotnull(sr_item_sk#85)) (61) Project -Output [4]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] -Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Output [4]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] +Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] (62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [cast(ss_ticket_number#80 as bigint), cast(ss_item_sk#79 as bigint)] -Right keys [2]: [sr_ticket_number#87, sr_item_sk#86] +Left keys [2]: [cast(ss_ticket_number#79 as bigint), cast(ss_item_sk#78 as bigint)] +Right keys [2]: [sr_ticket_number#86, sr_item_sk#85] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89] -Input [9]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] +Output [6]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88] +Input [9]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] (64) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#91] +Output [1]: [d_date_sk#14] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#84] -Right keys [1]: [d_date_sk#91] +Left keys [1]: [ss_sold_date_sk#83] +Right keys [1]: [d_date_sk#14] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] -Input [7]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89, d_date_sk#91] +Output [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] +Input [7]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88, d_date_sk#14] (67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] -Keys [1]: [ss_item_sk#79] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#88, 0)), partial_sum(coalesce(ss_quantity#81, 0)), partial_sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] -Results [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Input [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] +Keys [1]: [ss_item_sk#78] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#87, 0)), partial_sum(coalesce(ss_quantity#80, 0)), partial_sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Results [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] (68) Exchange -Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] -Arguments: hashpartitioning(ss_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#104] +Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Arguments: hashpartitioning(ss_item_sk#78, 5), ENSURE_REQUIREMENTS, [id=#102] (69) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] -Keys [1]: [ss_item_sk#79] -Functions [4]: [sum(coalesce(sr_return_quantity#88, 0)), sum(coalesce(ss_quantity#81, 0)), sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#88, 0))#105, sum(coalesce(ss_quantity#81, 0))#106, sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107, sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108] -Results [3]: [ss_item_sk#79 AS item#109, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#88, 0))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#81, 0))#106 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#110, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#111] +Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Keys [1]: [ss_item_sk#78] +Functions [4]: [sum(coalesce(sr_return_quantity#87, 0)), sum(coalesce(ss_quantity#80, 0)), sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#87, 0))#103, sum(coalesce(ss_quantity#80, 0))#104, sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105, sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106] +Results [3]: [ss_item_sk#78 AS item#107, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#87, 0))#103 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#80, 0))#104 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#108, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#109] (70) Exchange -Input [3]: [item#109, return_ratio#110, currency_ratio#111] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#112] +Input [3]: [item#107, return_ratio#108, currency_ratio#109] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] (71) Sort [codegen id : 19] -Input [3]: [item#109, return_ratio#110, currency_ratio#111] -Arguments: [return_ratio#110 ASC NULLS FIRST], false, 0 +Input [3]: [item#107, return_ratio#108, currency_ratio#109] +Arguments: [return_ratio#108 ASC NULLS FIRST], false, 0 (72) Window -Input [3]: [item#109, return_ratio#110, currency_ratio#111] -Arguments: [rank(return_ratio#110) windowspecdefinition(return_ratio#110 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#113], [return_ratio#110 ASC NULLS FIRST] +Input [3]: [item#107, return_ratio#108, currency_ratio#109] +Arguments: [rank(return_ratio#108) windowspecdefinition(return_ratio#108 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#111], [return_ratio#108 ASC NULLS FIRST] (73) Sort [codegen id : 20] -Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] -Arguments: [currency_ratio#111 ASC NULLS FIRST], false, 0 +Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] +Arguments: [currency_ratio#109 ASC NULLS FIRST], false, 0 (74) Window -Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] -Arguments: [rank(currency_ratio#111) windowspecdefinition(currency_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#114], [currency_ratio#111 ASC NULLS FIRST] +Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] +Arguments: [rank(currency_ratio#109) windowspecdefinition(currency_ratio#109 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#112], [currency_ratio#109 ASC NULLS FIRST] (75) Filter [codegen id : 21] -Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] -Condition : ((return_rank#113 <= 10) OR (currency_rank#114 <= 10)) +Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] +Condition : ((return_rank#111 <= 10) OR (currency_rank#112 <= 10)) (76) Project [codegen id : 21] -Output [5]: [store AS channel#115, item#109, return_ratio#110, return_rank#113, currency_rank#114] -Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] +Output [5]: [store AS channel#113, item#107, return_ratio#108, return_rank#111, currency_rank#112] +Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] (77) Union @@ -436,7 +436,7 @@ Results [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_ran (79) Exchange Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] -Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#116] +Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#114] (80) HashAggregate [codegen id : 23] Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] @@ -460,6 +460,6 @@ Output [1]: [d_date_sk#14] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#84 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#83 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt index 0a8fdb66edb77..bcf47f68f2e59 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 @@ -164,252 +164,252 @@ Input [4]: [item_sk#13, d_date#6, sumws#14, rk#16] Arguments: [item_sk#13 ASC NULLS FIRST], false, 0 (20) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] +Output [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] (21) Sort [codegen id : 10] -Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] -Arguments: [ws_item_sk#21 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] +Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST], false, 0 (22) Window -Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] -Arguments: [row_number() windowspecdefinition(ws_item_sk#21, d_date#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#22], [ws_item_sk#21], [d_date#19 ASC NULLS FIRST] +Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#21], [ws_item_sk#1], [d_date#19 ASC NULLS FIRST] (23) Project [codegen id : 11] -Output [3]: [item_sk#18, sumws#20, rk#22] -Input [5]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21, rk#22] +Output [3]: [item_sk#18, sumws#20, rk#21] +Input [5]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1, rk#21] (24) Exchange -Input [3]: [item_sk#18, sumws#20, rk#22] -Arguments: hashpartitioning(item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [item_sk#18, sumws#20, rk#21] +Arguments: hashpartitioning(item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#22] (25) Sort [codegen id : 12] -Input [3]: [item_sk#18, sumws#20, rk#22] +Input [3]: [item_sk#18, sumws#20, rk#21] Arguments: [item_sk#18 ASC NULLS FIRST], false, 0 (26) SortMergeJoin [codegen id : 13] Left keys [1]: [item_sk#13] Right keys [1]: [item_sk#18] -Join condition: (rk#16 >= rk#22) +Join condition: (rk#16 >= rk#21) (27) Project [codegen id : 13] Output [4]: [item_sk#13, d_date#6, sumws#14, sumws#20] -Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#18, sumws#20, rk#22] +Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#18, sumws#20, rk#21] (28) HashAggregate [codegen id : 13] Input [4]: [item_sk#13, d_date#6, sumws#14, sumws#20] Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [partial_sum(sumws#20)] -Aggregate Attributes [2]: [sum#24, isEmpty#25] -Results [5]: [item_sk#13, d_date#6, sumws#14, sum#26, isEmpty#27] +Aggregate Attributes [2]: [sum#23, isEmpty#24] +Results [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] (29) HashAggregate [codegen id : 13] -Input [5]: [item_sk#13, d_date#6, sumws#14, sum#26, isEmpty#27] +Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [sum(sumws#20)] -Aggregate Attributes [1]: [sum(sumws#20)#28] -Results [3]: [item_sk#13, d_date#6, sum(sumws#20)#28 AS cume_sales#29] +Aggregate Attributes [1]: [sum(sumws#20)#27] +Results [3]: [item_sk#13, d_date#6, sum(sumws#20)#27 AS cume_sales#28] (30) Exchange -Input [3]: [item_sk#13, d_date#6, cume_sales#29] -Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [item_sk#13, d_date#6, cume_sales#28] +Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#29] (31) Sort [codegen id : 14] -Input [3]: [item_sk#13, d_date#6, cume_sales#29] +Input [3]: [item_sk#13, d_date#6, cume_sales#28] Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 (32) Scan parquet default.store_sales -Output [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] +Output [3]: [ss_item_sk#30, ss_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#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (33) ColumnarToRow [codegen id : 16] -Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] +Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] (34) Filter [codegen id : 16] -Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] -Condition : isnotnull(ss_item_sk#31) +Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +Condition : isnotnull(ss_item_sk#30) (35) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#34, d_date#35] +Output [2]: [d_date_sk#33, d_date#34] (36) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ss_sold_date_sk#33] -Right keys [1]: [d_date_sk#34] +Left keys [1]: [ss_sold_date_sk#32] +Right keys [1]: [d_date_sk#33] Join condition: None (37) Project [codegen id : 16] -Output [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] -Input [5]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33, d_date_sk#34, d_date#35] +Output [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] +Input [5]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32, d_date_sk#33, d_date#34] (38) HashAggregate [codegen id : 16] -Input [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] -Keys [2]: [ss_item_sk#31, d_date#35] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#32))] -Aggregate Attributes [1]: [sum#36] -Results [3]: [ss_item_sk#31, d_date#35, sum#37] +Input [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] +Keys [2]: [ss_item_sk#30, d_date#34] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#31))] +Aggregate Attributes [1]: [sum#35] +Results [3]: [ss_item_sk#30, d_date#34, sum#36] (39) Exchange -Input [3]: [ss_item_sk#31, d_date#35, sum#37] -Arguments: hashpartitioning(ss_item_sk#31, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [3]: [ss_item_sk#30, d_date#34, sum#36] +Arguments: hashpartitioning(ss_item_sk#30, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#37] (40) HashAggregate [codegen id : 17] -Input [3]: [ss_item_sk#31, d_date#35, sum#37] -Keys [2]: [ss_item_sk#31, d_date#35] -Functions [1]: [sum(UnscaledValue(ss_sales_price#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#32))#39] -Results [4]: [ss_item_sk#31 AS item_sk#40, d_date#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#32))#39,17,2) AS sumss#41, ss_item_sk#31] +Input [3]: [ss_item_sk#30, d_date#34, sum#36] +Keys [2]: [ss_item_sk#30, d_date#34] +Functions [1]: [sum(UnscaledValue(ss_sales_price#31))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#31))#38] +Results [4]: [ss_item_sk#30 AS item_sk#39, d_date#34, MakeDecimal(sum(UnscaledValue(ss_sales_price#31))#38,17,2) AS sumss#40, ss_item_sk#30] (41) Exchange -Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] -Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] +Arguments: hashpartitioning(ss_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#41] (42) Sort [codegen id : 18] -Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] +Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 (43) Window -Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] -Arguments: [row_number() windowspecdefinition(ss_item_sk#31, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [ss_item_sk#31], [d_date#35 ASC NULLS FIRST] +Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] +Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#42], [ss_item_sk#30], [d_date#34 ASC NULLS FIRST] (44) Project [codegen id : 19] -Output [4]: [item_sk#40, d_date#35, sumss#41, rk#43] -Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] +Output [4]: [item_sk#39, d_date#34, sumss#40, rk#42] +Input [5]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30, rk#42] (45) Exchange -Input [4]: [item_sk#40, d_date#35, sumss#41, rk#43] -Arguments: hashpartitioning(item_sk#40, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [4]: [item_sk#39, d_date#34, sumss#40, rk#42] +Arguments: hashpartitioning(item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#43] (46) Sort [codegen id : 20] -Input [4]: [item_sk#40, d_date#35, sumss#41, rk#43] -Arguments: [item_sk#40 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#39, d_date#34, sumss#40, rk#42] +Arguments: [item_sk#39 ASC NULLS FIRST], false, 0 (47) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] +Output [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30] (48) Sort [codegen id : 24] -Input [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] -Arguments: [ss_item_sk#48 ASC NULLS FIRST, d_date#46 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30] +Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], false, 0 (49) Window -Input [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] -Arguments: [row_number() windowspecdefinition(ss_item_sk#48, d_date#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [ss_item_sk#48], [d_date#46 ASC NULLS FIRST] +Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30] +Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#47], [ss_item_sk#30], [d_date#45 ASC NULLS FIRST] (50) Project [codegen id : 25] -Output [3]: [item_sk#45, sumss#47, rk#49] -Input [5]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48, rk#49] +Output [3]: [item_sk#44, sumss#46, rk#47] +Input [5]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30, rk#47] (51) Exchange -Input [3]: [item_sk#45, sumss#47, rk#49] -Arguments: hashpartitioning(item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [3]: [item_sk#44, sumss#46, rk#47] +Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, [id=#48] (52) Sort [codegen id : 26] -Input [3]: [item_sk#45, sumss#47, rk#49] -Arguments: [item_sk#45 ASC NULLS FIRST], false, 0 +Input [3]: [item_sk#44, sumss#46, rk#47] +Arguments: [item_sk#44 ASC NULLS FIRST], false, 0 (53) SortMergeJoin [codegen id : 27] -Left keys [1]: [item_sk#40] -Right keys [1]: [item_sk#45] -Join condition: (rk#43 >= rk#49) +Left keys [1]: [item_sk#39] +Right keys [1]: [item_sk#44] +Join condition: (rk#42 >= rk#47) (54) Project [codegen id : 27] -Output [4]: [item_sk#40, d_date#35, sumss#41, sumss#47] -Input [7]: [item_sk#40, d_date#35, sumss#41, rk#43, item_sk#45, sumss#47, rk#49] +Output [4]: [item_sk#39, d_date#34, sumss#40, sumss#46] +Input [7]: [item_sk#39, d_date#34, sumss#40, rk#42, item_sk#44, sumss#46, rk#47] (55) HashAggregate [codegen id : 27] -Input [4]: [item_sk#40, d_date#35, sumss#41, sumss#47] -Keys [3]: [item_sk#40, d_date#35, sumss#41] -Functions [1]: [partial_sum(sumss#47)] -Aggregate Attributes [2]: [sum#51, isEmpty#52] -Results [5]: [item_sk#40, d_date#35, sumss#41, sum#53, isEmpty#54] +Input [4]: [item_sk#39, d_date#34, sumss#40, sumss#46] +Keys [3]: [item_sk#39, d_date#34, sumss#40] +Functions [1]: [partial_sum(sumss#46)] +Aggregate Attributes [2]: [sum#49, isEmpty#50] +Results [5]: [item_sk#39, d_date#34, sumss#40, sum#51, isEmpty#52] (56) HashAggregate [codegen id : 27] -Input [5]: [item_sk#40, d_date#35, sumss#41, sum#53, isEmpty#54] -Keys [3]: [item_sk#40, d_date#35, sumss#41] -Functions [1]: [sum(sumss#47)] -Aggregate Attributes [1]: [sum(sumss#47)#55] -Results [3]: [item_sk#40, d_date#35, sum(sumss#47)#55 AS cume_sales#56] +Input [5]: [item_sk#39, d_date#34, sumss#40, sum#51, isEmpty#52] +Keys [3]: [item_sk#39, d_date#34, sumss#40] +Functions [1]: [sum(sumss#46)] +Aggregate Attributes [1]: [sum(sumss#46)#53] +Results [3]: [item_sk#39, d_date#34, sum(sumss#46)#53 AS cume_sales#54] (57) Exchange -Input [3]: [item_sk#40, d_date#35, cume_sales#56] -Arguments: hashpartitioning(item_sk#40, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [3]: [item_sk#39, d_date#34, cume_sales#54] +Arguments: hashpartitioning(item_sk#39, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#55] (58) Sort [codegen id : 28] -Input [3]: [item_sk#40, d_date#35, cume_sales#56] -Arguments: [item_sk#40 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 +Input [3]: [item_sk#39, d_date#34, cume_sales#54] +Arguments: [item_sk#39 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 (59) SortMergeJoin Left keys [2]: [item_sk#13, d_date#6] -Right keys [2]: [item_sk#40, d_date#35] +Right keys [2]: [item_sk#39, d_date#34] Join condition: None (60) Filter [codegen id : 29] -Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END) +Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END) (61) Project [codegen id : 29] -Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END AS item_sk#58, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#35 END AS d_date#59, cume_sales#29 AS web_sales#60, cume_sales#56 AS store_sales#61] -Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] +Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END AS item_sk#56, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#34 END AS d_date#57, cume_sales#28 AS web_sales#58, cume_sales#54 AS store_sales#59] +Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] (62) Exchange -Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Arguments: hashpartitioning(item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] +Arguments: hashpartitioning(item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#60] (63) Sort [codegen id : 30] -Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Arguments: [item_sk#58 ASC NULLS FIRST, d_date#59 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 (64) Window -Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Arguments: [row_number() windowspecdefinition(item_sk#58, d_date#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#63], [item_sk#58], [d_date#59 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#61], [item_sk#56], [d_date#57 ASC NULLS FIRST] (65) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] +Output [4]: [item_sk#62, d_date#63, web_sales#64, store_sales#65] (66) Sort [codegen id : 60] -Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] -Arguments: [item_sk#64 ASC NULLS FIRST, d_date#65 ASC NULLS FIRST], false, 0 +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 (67) Window -Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] -Arguments: [row_number() windowspecdefinition(item_sk#64, d_date#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#68], [item_sk#64], [d_date#65 ASC NULLS FIRST] +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] (68) Project [codegen id : 61] -Output [4]: [item_sk#64, web_sales#66, store_sales#67, rk#68] -Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] +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] (69) SortMergeJoin [codegen id : 62] -Left keys [1]: [item_sk#58] -Right keys [1]: [item_sk#64] -Join condition: (rk#63 >= rk#68) +Left keys [1]: [item_sk#56] +Right keys [1]: [item_sk#62] +Join condition: (rk#61 >= rk#66) (70) Project [codegen id : 62] -Output [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] -Input [9]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63, item_sk#64, web_sales#66, store_sales#67, rk#68] +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] (71) HashAggregate [codegen id : 62] -Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] -Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Functions [2]: [partial_max(web_sales#66), partial_max(store_sales#67)] -Aggregate Attributes [2]: [max#69, max#70] -Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#71, max#72] +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] (72) HashAggregate [codegen id : 62] -Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#71, max#72] -Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Functions [2]: [max(web_sales#66), max(store_sales#67)] -Aggregate Attributes [2]: [max(web_sales#66)#73, max(store_sales#67)#74] -Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max(web_sales#66)#73 AS web_cumulative#75, max(store_sales#67)#74 AS store_cumulative#76] +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)#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] (73) Filter [codegen id : 62] -Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] -Condition : ((isnotnull(web_cumulative#75) AND isnotnull(store_cumulative#76)) AND (web_cumulative#75 > store_cumulative#76)) +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)) (74) TakeOrderedAndProject -Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] -Arguments: 100, [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] +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 ===== @@ -420,6 +420,6 @@ ReusedExchange (75) (75) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#5, d_date#6] -Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#33 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#32 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 b7d3461ce34ea..8aaaf48a8162c 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 @@ -153,248 +153,248 @@ Output [4]: [item_sk#13, d_date#6, sumws#14, rk#16] Input [5]: [item_sk#13, d_date#6, sumws#14, ws_item_sk#1, rk#16] (18) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] +Output [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] (19) Sort [codegen id : 8] -Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] -Arguments: [ws_item_sk#20 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] +Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 (20) Window -Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] -Arguments: [row_number() windowspecdefinition(ws_item_sk#20, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#21], [ws_item_sk#20], [d_date#18 ASC NULLS FIRST] +Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#20], [ws_item_sk#1], [d_date#18 ASC NULLS FIRST] (21) Project [codegen id : 9] -Output [3]: [item_sk#17, sumws#19, rk#21] -Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20, rk#21] +Output [3]: [item_sk#17, sumws#19, rk#20] +Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1, rk#20] (22) BroadcastExchange -Input [3]: [item_sk#17, sumws#19, rk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [3]: [item_sk#17, sumws#19, rk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (23) BroadcastHashJoin [codegen id : 10] Left keys [1]: [item_sk#13] Right keys [1]: [item_sk#17] -Join condition: (rk#16 >= rk#21) +Join condition: (rk#16 >= rk#20) (24) Project [codegen id : 10] Output [4]: [item_sk#13, d_date#6, sumws#14, sumws#19] -Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#17, sumws#19, rk#21] +Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#17, sumws#19, rk#20] (25) HashAggregate [codegen id : 10] Input [4]: [item_sk#13, d_date#6, sumws#14, sumws#19] Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [partial_sum(sumws#19)] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +Aggregate Attributes [2]: [sum#22, isEmpty#23] +Results [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] (26) Exchange -Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] -Arguments: hashpartitioning(item_sk#13, d_date#6, sumws#14, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] +Arguments: hashpartitioning(item_sk#13, d_date#6, sumws#14, 5), ENSURE_REQUIREMENTS, [id=#26] (27) HashAggregate [codegen id : 11] -Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +Input [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [sum(sumws#19)] -Aggregate Attributes [1]: [sum(sumws#19)#28] -Results [3]: [item_sk#13, d_date#6, sum(sumws#19)#28 AS cume_sales#29] +Aggregate Attributes [1]: [sum(sumws#19)#27] +Results [3]: [item_sk#13, d_date#6, sum(sumws#19)#27 AS cume_sales#28] (28) Exchange -Input [3]: [item_sk#13, d_date#6, cume_sales#29] -Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [item_sk#13, d_date#6, cume_sales#28] +Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#29] (29) Sort [codegen id : 12] -Input [3]: [item_sk#13, d_date#6, cume_sales#29] +Input [3]: [item_sk#13, d_date#6, cume_sales#28] Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 (30) Scan parquet default.store_sales -Output [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] +Output [3]: [ss_item_sk#30, ss_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#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 14] -Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] +Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] (32) Filter [codegen id : 14] -Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] -Condition : isnotnull(ss_item_sk#31) +Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +Condition : isnotnull(ss_item_sk#30) (33) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#34, d_date#35] +Output [2]: [d_date_sk#33, d_date#34] (34) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_sold_date_sk#33] -Right keys [1]: [d_date_sk#34] +Left keys [1]: [ss_sold_date_sk#32] +Right keys [1]: [d_date_sk#33] Join condition: None (35) Project [codegen id : 14] -Output [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] -Input [5]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33, d_date_sk#34, d_date#35] +Output [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] +Input [5]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32, d_date_sk#33, d_date#34] (36) HashAggregate [codegen id : 14] -Input [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] -Keys [2]: [ss_item_sk#31, d_date#35] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#32))] -Aggregate Attributes [1]: [sum#36] -Results [3]: [ss_item_sk#31, d_date#35, sum#37] +Input [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] +Keys [2]: [ss_item_sk#30, d_date#34] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#31))] +Aggregate Attributes [1]: [sum#35] +Results [3]: [ss_item_sk#30, d_date#34, sum#36] (37) Exchange -Input [3]: [ss_item_sk#31, d_date#35, sum#37] -Arguments: hashpartitioning(ss_item_sk#31, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [3]: [ss_item_sk#30, d_date#34, sum#36] +Arguments: hashpartitioning(ss_item_sk#30, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#37] (38) HashAggregate [codegen id : 15] -Input [3]: [ss_item_sk#31, d_date#35, sum#37] -Keys [2]: [ss_item_sk#31, d_date#35] -Functions [1]: [sum(UnscaledValue(ss_sales_price#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#32))#39] -Results [4]: [ss_item_sk#31 AS item_sk#40, d_date#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#32))#39,17,2) AS sumss#41, ss_item_sk#31] +Input [3]: [ss_item_sk#30, d_date#34, sum#36] +Keys [2]: [ss_item_sk#30, d_date#34] +Functions [1]: [sum(UnscaledValue(ss_sales_price#31))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#31))#38] +Results [4]: [ss_item_sk#30 AS item_sk#39, d_date#34, MakeDecimal(sum(UnscaledValue(ss_sales_price#31))#38,17,2) AS sumss#40, ss_item_sk#30] (39) Exchange -Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] -Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] +Arguments: hashpartitioning(ss_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#41] (40) Sort [codegen id : 16] -Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] +Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 (41) Window -Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] -Arguments: [row_number() windowspecdefinition(ss_item_sk#31, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [ss_item_sk#31], [d_date#35 ASC NULLS FIRST] +Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] +Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#42], [ss_item_sk#30], [d_date#34 ASC NULLS FIRST] (42) Project [codegen id : 22] -Output [4]: [item_sk#40, d_date#35, sumss#41, rk#43] -Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] +Output [4]: [item_sk#39, d_date#34, sumss#40, rk#42] +Input [5]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30, rk#42] (43) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] +Output [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] (44) Sort [codegen id : 20] -Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] +Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#44 ASC NULLS FIRST], false, 0 (45) Window -Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] -Arguments: [row_number() windowspecdefinition(ss_item_sk#47, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [ss_item_sk#47], [d_date#45 ASC NULLS FIRST] +Input [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] +Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#46], [ss_item_sk#30], [d_date#44 ASC NULLS FIRST] (46) Project [codegen id : 21] -Output [3]: [item_sk#44, sumss#46, rk#48] -Input [5]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47, rk#48] +Output [3]: [item_sk#43, sumss#45, rk#46] +Input [5]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30, rk#46] (47) BroadcastExchange -Input [3]: [item_sk#44, sumss#46, rk#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] +Input [3]: [item_sk#43, sumss#45, rk#46] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] (48) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [item_sk#40] -Right keys [1]: [item_sk#44] -Join condition: (rk#43 >= rk#48) +Left keys [1]: [item_sk#39] +Right keys [1]: [item_sk#43] +Join condition: (rk#42 >= rk#46) (49) Project [codegen id : 22] -Output [4]: [item_sk#40, d_date#35, sumss#41, sumss#46] -Input [7]: [item_sk#40, d_date#35, sumss#41, rk#43, item_sk#44, sumss#46, rk#48] +Output [4]: [item_sk#39, d_date#34, sumss#40, sumss#45] +Input [7]: [item_sk#39, d_date#34, sumss#40, rk#42, item_sk#43, sumss#45, rk#46] (50) HashAggregate [codegen id : 22] -Input [4]: [item_sk#40, d_date#35, sumss#41, sumss#46] -Keys [3]: [item_sk#40, d_date#35, sumss#41] -Functions [1]: [partial_sum(sumss#46)] -Aggregate Attributes [2]: [sum#50, isEmpty#51] -Results [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] +Input [4]: [item_sk#39, d_date#34, sumss#40, sumss#45] +Keys [3]: [item_sk#39, d_date#34, sumss#40] +Functions [1]: [partial_sum(sumss#45)] +Aggregate Attributes [2]: [sum#48, isEmpty#49] +Results [5]: [item_sk#39, d_date#34, sumss#40, sum#50, isEmpty#51] (51) Exchange -Input [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] -Arguments: hashpartitioning(item_sk#40, d_date#35, sumss#41, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [5]: [item_sk#39, d_date#34, sumss#40, sum#50, isEmpty#51] +Arguments: hashpartitioning(item_sk#39, d_date#34, sumss#40, 5), ENSURE_REQUIREMENTS, [id=#52] (52) HashAggregate [codegen id : 23] -Input [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] -Keys [3]: [item_sk#40, d_date#35, sumss#41] -Functions [1]: [sum(sumss#46)] -Aggregate Attributes [1]: [sum(sumss#46)#55] -Results [3]: [item_sk#40, d_date#35, sum(sumss#46)#55 AS cume_sales#56] +Input [5]: [item_sk#39, d_date#34, sumss#40, sum#50, isEmpty#51] +Keys [3]: [item_sk#39, d_date#34, sumss#40] +Functions [1]: [sum(sumss#45)] +Aggregate Attributes [1]: [sum(sumss#45)#53] +Results [3]: [item_sk#39, d_date#34, sum(sumss#45)#53 AS cume_sales#54] (53) Exchange -Input [3]: [item_sk#40, d_date#35, cume_sales#56] -Arguments: hashpartitioning(item_sk#40, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [3]: [item_sk#39, d_date#34, cume_sales#54] +Arguments: hashpartitioning(item_sk#39, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#55] (54) Sort [codegen id : 24] -Input [3]: [item_sk#40, d_date#35, cume_sales#56] -Arguments: [item_sk#40 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 +Input [3]: [item_sk#39, d_date#34, cume_sales#54] +Arguments: [item_sk#39 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 (55) SortMergeJoin Left keys [2]: [item_sk#13, d_date#6] -Right keys [2]: [item_sk#40, d_date#35] +Right keys [2]: [item_sk#39, d_date#34] Join condition: None (56) Filter [codegen id : 25] -Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END) +Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END) (57) Project [codegen id : 25] -Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END AS item_sk#58, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#35 END AS d_date#59, cume_sales#29 AS web_sales#60, cume_sales#56 AS store_sales#61] -Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] +Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END AS item_sk#56, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#34 END AS d_date#57, cume_sales#28 AS web_sales#58, cume_sales#54 AS store_sales#59] +Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] (58) Exchange -Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Arguments: hashpartitioning(item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] +Arguments: hashpartitioning(item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#60] (59) Sort [codegen id : 26] -Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Arguments: [item_sk#58 ASC NULLS FIRST, d_date#59 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 (60) Window -Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Arguments: [row_number() windowspecdefinition(item_sk#58, d_date#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#63], [item_sk#58], [d_date#59 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#61], [item_sk#56], [d_date#57 ASC NULLS FIRST] (61) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] +Output [4]: [item_sk#62, d_date#63, web_sales#64, store_sales#65] (62) Sort [codegen id : 52] -Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] -Arguments: [item_sk#64 ASC NULLS FIRST, d_date#65 ASC NULLS FIRST], false, 0 +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 (63) Window -Input [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] -Arguments: [row_number() windowspecdefinition(item_sk#64, d_date#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#68], [item_sk#64], [d_date#65 ASC NULLS FIRST] +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] (64) Project [codegen id : 53] -Output [4]: [item_sk#64, web_sales#66, store_sales#67, rk#68] -Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] +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] (65) BroadcastExchange -Input [4]: [item_sk#64, web_sales#66, store_sales#67, rk#68] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] +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] (66) BroadcastHashJoin [codegen id : 54] -Left keys [1]: [item_sk#58] -Right keys [1]: [item_sk#64] -Join condition: (rk#63 >= rk#68) +Left keys [1]: [item_sk#56] +Right keys [1]: [item_sk#62] +Join condition: (rk#61 >= rk#66) (67) Project [codegen id : 54] -Output [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] -Input [9]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63, item_sk#64, web_sales#66, store_sales#67, rk#68] +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] (68) HashAggregate [codegen id : 54] -Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] -Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Functions [2]: [partial_max(web_sales#66), partial_max(store_sales#67)] -Aggregate Attributes [2]: [max#70, max#71] -Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#72, max#73] +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] (69) HashAggregate [codegen id : 54] -Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#72, max#73] -Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] -Functions [2]: [max(web_sales#66), max(store_sales#67)] -Aggregate Attributes [2]: [max(web_sales#66)#74, max(store_sales#67)#75] -Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max(web_sales#66)#74 AS web_cumulative#76, max(store_sales#67)#75 AS store_cumulative#77] +Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#70, max#71] +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] (70) Filter [codegen id : 54] -Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] -Condition : ((isnotnull(web_cumulative#76) AND isnotnull(store_cumulative#77)) AND (web_cumulative#76 > store_cumulative#77)) +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)) (71) TakeOrderedAndProject -Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] -Arguments: 100, [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] +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] ===== Subqueries ===== @@ -405,6 +405,6 @@ ReusedExchange (72) (72) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#5, d_date#6] -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#33 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#32 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 e3de4a7a053b1..ed0399519f579 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 @@ -229,78 +229,78 @@ Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] (39) HashAggregate [codegen id : 21] Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] Keys [5]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33] -Functions [1]: [sum(UnscaledValue(cs_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#35))#36] -Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#35))#36,17,2) AS sum_sales#37] +Functions [1]: [sum(UnscaledValue(cs_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#3))#35] +Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#3))#35,17,2) AS sum_sales#36] (40) Exchange -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#37] (41) Sort [codegen id : 22] -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 (42) Window -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] -Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] (43) Project [codegen id : 23] -Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] -Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] +Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] (44) Exchange -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#38 + 1), 5), ENSURE_REQUIREMENTS, [id=#39] (45) Sort [codegen id : 24] -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] -Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] +Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#38 + 1) ASC NULLS FIRST], false, 0 (46) SortMergeJoin [codegen id : 25] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#39 + 1)] +Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#38 + 1)] Join condition: None (47) Project [codegen id : 25] -Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] -Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] +Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] (48) ReusedExchange [Reuses operator id: 40] -Output [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] +Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] (49) Sort [codegen id : 34] -Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 (50) Window -Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] -Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#41, i_brand#42, cc_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#41, i_brand#42, cc_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] +Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] (51) Project [codegen id : 35] -Output [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] -Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] +Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] +Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] (52) Exchange -Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] -Arguments: hashpartitioning(i_category#41, i_brand#42, cc_name#43, (rn#47 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] +Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] +Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (rn#46 - 1), 5), ENSURE_REQUIREMENTS, [id=#47] (53) Sort [codegen id : 36] -Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] -Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, (rn#47 - 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] +Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 (54) SortMergeJoin [codegen id : 37] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#41, i_brand#42, cc_name#43, (rn#47 - 1)] +Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (rn#46 - 1)] Join condition: None (55) Project [codegen id : 37] -Output [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#49, sum_sales#46 AS nsum#50] -Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Output [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#48, sum_sales#45 AS nsum#49] +Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] (56) TakeOrderedAndProject -Input [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] +Input [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 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#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] ===== Subqueries ===== 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 a6742cf4ab1cf..c347bbbe3b4ad 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 @@ -202,70 +202,70 @@ Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] (34) HashAggregate [codegen id : 13] 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] +Functions [1]: [sum(UnscaledValue(cs_sales_price#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#33] +Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#33,17,2) AS sum_sales#34] (35) 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#34] +Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#35] (36) Sort [codegen id : 14] -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#34] 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 (37) 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#34] +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#36], [i_category#27, i_brand#28, cc_name#29], [d_year#30 ASC NULLS FIRST, d_moy#31 ASC NULLS FIRST] (38) Project [codegen id : 15] -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#34, rn#36] +Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34, rn#36] (39) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#38] +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#37] (40) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -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#36 + 1)] Join condition: None (41) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35] -Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34] +Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] (42) ReusedExchange [Reuses operator id: 35] -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#43] (43) Sort [codegen id : 21] -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#43] +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 (44) 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#43] +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#44], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] (45) Project [codegen id : 22] -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#43, rn#44] +Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43, rn#44] (46) BroadcastExchange -Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#46] +Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#45] (47) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -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#44 - 1)] Join condition: None (48) Project [codegen id : 23] -Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] -Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#34 AS psum#46, sum_sales#43 AS nsum#47] +Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34, i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] (49) TakeOrderedAndProject -Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, 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#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#21, psum#47, nsum#48] +Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, 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#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#21, psum#46, nsum#47] ===== Subqueries ===== 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 8ca76e48002b7..e04148fad545e 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 @@ -278,165 +278,165 @@ Output [6]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, Input [8]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] (41) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#70] +Output [1]: [d_date_sk#25] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#70] +Right keys [1]: [d_date_sk#25] Join condition: None (43) Project [codegen id : 11] Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#70] +Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#25] (44) HashAggregate [codegen id : 11] Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] -Results [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] +Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] +Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] (45) Exchange -Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] -Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] +Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] -Results [5]: [catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#68) AS id#85, MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS returns#87, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#88] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] +Results [5]: [catalog channel AS channel#83, concat(catalog_page, cp_catalog_page_id#68) AS id#84, MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#85, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS returns#86, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#87] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Condition : isnotnull(ws_web_site_sk#89) +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Condition : isnotnull(ws_web_site_sk#88) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] (53) Exchange -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] -Arguments: hashpartitioning(wr_item_sk#99, wr_order_number#100, 5), ENSURE_REQUIREMENTS, [id=#104] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Arguments: hashpartitioning(wr_item_sk#98, wr_order_number#99, 5), ENSURE_REQUIREMENTS, [id=#103] (54) Sort [codegen id : 15] -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] -Arguments: [wr_item_sk#99 ASC NULLS FIRST, wr_order_number#100 ASC NULLS FIRST], false, 0 +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Arguments: [wr_item_sk#98 ASC NULLS FIRST, wr_order_number#99 ASC NULLS FIRST], false, 0 (55) Scan parquet default.web_sales -Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 16] -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] (57) Filter [codegen id : 16] -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] -Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) (58) Project [codegen id : 16] -Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] (59) Exchange -Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] -Arguments: hashpartitioning(cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint), 5), ENSURE_REQUIREMENTS, [id=#109] +Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Arguments: hashpartitioning(cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint), 5), ENSURE_REQUIREMENTS, [id=#106] (60) Sort [codegen id : 17] -Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] -Arguments: [cast(ws_item_sk#105 as bigint) ASC NULLS FIRST, cast(ws_order_number#107 as bigint) ASC NULLS FIRST], false, 0 +Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Arguments: [cast(ws_item_sk#104 as bigint) ASC NULLS FIRST, cast(ws_order_number#105 as bigint) ASC NULLS FIRST], false, 0 (61) SortMergeJoin [codegen id : 18] -Left keys [2]: [wr_item_sk#99, wr_order_number#100] -Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] +Left keys [2]: [wr_item_sk#98, wr_order_number#99] +Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None (62) Project [codegen id : 18] -Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#110, wr_returned_date_sk#103 AS date_sk#111, 0.00 AS sales_price#112, 0.00 AS profit#113, wr_return_amt#101 AS return_amt#114, wr_net_loss#102 AS net_loss#115] -Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#107, wr_returned_date_sk#102 AS date_sk#108, 0.00 AS sales_price#109, 0.00 AS profit#110, wr_return_amt#100 AS return_amt#111, wr_net_loss#101 AS net_loss#112] +Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] (63) Union (64) Scan parquet default.web_site -Output [2]: [web_site_sk#116, web_site_id#117] +Output [2]: [web_site_sk#113, web_site_id#114] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 19] -Input [2]: [web_site_sk#116, web_site_id#117] +Input [2]: [web_site_sk#113, web_site_id#114] (66) Filter [codegen id : 19] -Input [2]: [web_site_sk#116, web_site_id#117] -Condition : isnotnull(web_site_sk#116) +Input [2]: [web_site_sk#113, web_site_id#114] +Condition : isnotnull(web_site_sk#113) (67) BroadcastExchange -Input [2]: [web_site_sk#116, web_site_id#117] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] +Input [2]: [web_site_sk#113, web_site_id#114] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] (68) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [wsr_web_site_sk#93] -Right keys [1]: [web_site_sk#116] +Left keys [1]: [wsr_web_site_sk#92] +Right keys [1]: [web_site_sk#113] Join condition: None (69) Project [codegen id : 21] -Output [6]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Input [8]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] +Output [6]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] +Input [8]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#113, web_site_id#114] (70) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#119] +Output [1]: [d_date_sk#25] (71) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [date_sk#94] -Right keys [1]: [cast(d_date_sk#119 as bigint)] +Left keys [1]: [date_sk#93] +Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None (72) Project [codegen id : 21] -Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Input [7]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117, d_date_sk#119] +Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] +Input [7]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114, d_date_sk#25] (73) HashAggregate [codegen id : 21] -Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Keys [1]: [web_site_id#117] -Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] -Aggregate Attributes [4]: [sum#120, sum#121, sum#122, sum#123] -Results [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] +Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] +Keys [1]: [web_site_id#114] +Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] +Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] +Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] (74) Exchange -Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] -Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#128] +Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] +Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] (75) HashAggregate [codegen id : 22] -Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] -Keys [1]: [web_site_id#117] -Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#129, sum(UnscaledValue(return_amt#97))#130, sum(UnscaledValue(profit#96))#131, sum(UnscaledValue(net_loss#98))#132] -Results [5]: [web channel AS channel#133, concat(web_site, web_site_id#117) AS id#134, MakeDecimal(sum(UnscaledValue(sales_price#95))#129,17,2) AS sales#135, MakeDecimal(sum(UnscaledValue(return_amt#97))#130,17,2) AS returns#136, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#131,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#132,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#137] +Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] +Keys [1]: [web_site_id#114] +Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#125, sum(UnscaledValue(return_amt#96))#126, sum(UnscaledValue(profit#95))#127, sum(UnscaledValue(net_loss#97))#128] +Results [5]: [web channel AS channel#129, concat(web_site, web_site_id#114) AS id#130, MakeDecimal(sum(UnscaledValue(sales_price#94))#125,17,2) AS sales#131, MakeDecimal(sum(UnscaledValue(return_amt#96))#126,17,2) AS returns#132, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#133] (76) Union @@ -444,99 +444,99 @@ Results [5]: [web channel AS channel#133, concat(web_site, web_site_id#117) AS i Input [5]: [channel#41, id#42, sales#43, returns#44, profit#45] Keys [2]: [channel#41, id#42] Functions [3]: [partial_sum(sales#43), partial_sum(returns#44), partial_sum(profit#45)] -Aggregate Attributes [6]: [sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Results [8]: [channel#41, id#42, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] +Aggregate Attributes [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Results [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] (78) Exchange -Input [8]: [channel#41, id#42, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] -Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#150] +Input [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#146] (79) HashAggregate [codegen id : 24] -Input [8]: [channel#41, id#42, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] +Input [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#45)] -Aggregate Attributes [3]: [sum(sales#43)#151, sum(returns#44)#152, sum(profit#45)#153] -Results [5]: [channel#41, id#42, cast(sum(sales#43)#151 as decimal(37,2)) AS sales#154, cast(sum(returns#44)#152 as decimal(37,2)) AS returns#155, cast(sum(profit#45)#153 as decimal(38,2)) AS profit#156] +Aggregate Attributes [3]: [sum(sales#43)#147, sum(returns#44)#148, sum(profit#45)#149] +Results [5]: [channel#41, id#42, cast(sum(sales#43)#147 as decimal(37,2)) AS sales#150, cast(sum(returns#44)#148 as decimal(37,2)) AS returns#151, cast(sum(profit#45)#149 as decimal(38,2)) AS profit#152] (80) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] +Output [8]: [channel#41, id#42, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] (81) HashAggregate [codegen id : 48] -Input [8]: [channel#41, id#42, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] +Input [8]: [channel#41, id#42, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#163)] -Aggregate Attributes [3]: [sum(sales#43)#164, sum(returns#44)#165, sum(profit#163)#166] -Results [4]: [channel#41, sum(sales#43)#164 AS sales#167, sum(returns#44)#165 AS returns#168, sum(profit#163)#166 AS profit#169] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#159)] +Aggregate Attributes [3]: [sum(sales#43)#160, sum(returns#44)#161, sum(profit#159)#162] +Results [4]: [channel#41, sum(sales#43)#160 AS sales#163, sum(returns#44)#161 AS returns#164, sum(profit#159)#162 AS profit#165] (82) HashAggregate [codegen id : 48] -Input [4]: [channel#41, sales#167, returns#168, profit#169] +Input [4]: [channel#41, sales#163, returns#164, profit#165] Keys [1]: [channel#41] -Functions [3]: [partial_sum(sales#167), partial_sum(returns#168), partial_sum(profit#169)] -Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] -Results [7]: [channel#41, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +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#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] (83) Exchange -Input [7]: [channel#41, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] -Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#182] +Input [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] +Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#178] (84) HashAggregate [codegen id : 49] -Input [7]: [channel#41, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Input [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] Keys [1]: [channel#41] -Functions [3]: [sum(sales#167), sum(returns#168), sum(profit#169)] -Aggregate Attributes [3]: [sum(sales#167)#183, sum(returns#168)#184, sum(profit#169)#185] -Results [5]: [channel#41, null AS id#186, sum(sales#167)#183 AS sum(sales)#187, sum(returns#168)#184 AS sum(returns)#188, sum(profit#169)#185 AS sum(profit)#189] +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#41, 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] (85) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#190, isEmpty#191, sum#192, isEmpty#193, sum#194, isEmpty#195] +Output [8]: [channel#41, id#42, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] (86) HashAggregate [codegen id : 73] -Input [8]: [channel#41, id#42, sum#190, isEmpty#191, sum#192, isEmpty#193, sum#194, isEmpty#195] +Input [8]: [channel#41, id#42, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#196)] -Aggregate Attributes [3]: [sum(sales#43)#197, sum(returns#44)#198, sum(profit#196)#199] -Results [3]: [sum(sales#43)#197 AS sales#167, sum(returns#44)#198 AS returns#168, sum(profit#196)#199 AS profit#169] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#192)] +Aggregate Attributes [3]: [sum(sales#43)#193, sum(returns#44)#194, sum(profit#192)#195] +Results [3]: [sum(sales#43)#193 AS sales#163, sum(returns#44)#194 AS returns#164, sum(profit#192)#195 AS profit#165] (87) HashAggregate [codegen id : 73] -Input [3]: [sales#167, returns#168, profit#169] +Input [3]: [sales#163, returns#164, profit#165] Keys: [] -Functions [3]: [partial_sum(sales#167), partial_sum(returns#168), partial_sum(profit#169)] -Aggregate Attributes [6]: [sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205] -Results [6]: [sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211] +Functions [3]: [partial_sum(sales#163), partial_sum(returns#164), partial_sum(profit#165)] +Aggregate Attributes [6]: [sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201] +Results [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] (88) Exchange -Input [6]: [sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#212] +Input [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#208] (89) HashAggregate [codegen id : 74] -Input [6]: [sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211] +Input [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] Keys: [] -Functions [3]: [sum(sales#167), sum(returns#168), sum(profit#169)] -Aggregate Attributes [3]: [sum(sales#167)#213, sum(returns#168)#214, sum(profit#169)#215] -Results [5]: [null AS channel#216, null AS id#217, sum(sales#167)#213 AS sum(sales)#218, sum(returns#168)#214 AS sum(returns)#219, sum(profit#169)#215 AS sum(profit)#220] +Functions [3]: [sum(sales#163), sum(returns#164), sum(profit#165)] +Aggregate Attributes [3]: [sum(sales#163)#209, sum(returns#164)#210, sum(profit#165)#211] +Results [5]: [null AS channel#212, null AS id#213, sum(sales#163)#209 AS sum(sales)#214, sum(returns#164)#210 AS sum(returns)#215, sum(profit#165)#211 AS sum(profit)#216] (90) Union (91) HashAggregate [codegen id : 75] -Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] -Keys [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Keys [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Results [5]: [channel#41, id#42, sales#150, returns#151, profit#152] (92) Exchange -Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] -Arguments: hashpartitioning(channel#41, id#42, sales#154, returns#155, profit#156, 5), ENSURE_REQUIREMENTS, [id=#221] +Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Arguments: hashpartitioning(channel#41, id#42, sales#150, returns#151, profit#152, 5), ENSURE_REQUIREMENTS, [id=#217] (93) HashAggregate [codegen id : 76] -Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] -Keys [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Keys [5]: [channel#41, id#42, sales#150, returns#151, profit#152] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Results [5]: [channel#41, id#42, sales#150, returns#151, profit#152] (94) TakeOrderedAndProject -Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] -Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#154, returns#155, profit#156] +Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#150, returns#151, profit#152] ===== Subqueries ===== @@ -554,12 +554,12 @@ ReusedExchange (96) (96) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#70] +Output [1]: [d_date_sk#25] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 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 fe1900d0c2d27..36b1ff63b2065 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 @@ -248,180 +248,180 @@ Input [4]: [cr_catalog_page_sk#57, cr_return_amount#58, cr_net_loss#59, cr_retur (34) Union (35) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#67] +Output [1]: [d_date_sk#22] (36) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#67] +Right keys [1]: [d_date_sk#22] Join condition: None (37) Project [codegen id : 11] Output [5]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56] -Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#67] +Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#22] (38) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Output [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] (40) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] -Condition : isnotnull(cp_catalog_page_sk#68) +Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Condition : isnotnull(cp_catalog_page_sk#67) (41) BroadcastExchange -Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#70] +Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#69] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [page_sk#51] -Right keys [1]: [cp_catalog_page_sk#68] +Right keys [1]: [cp_catalog_page_sk#67] Join condition: None (43) Project [codegen id : 11] -Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] -Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#68, cp_catalog_page_id#69] +Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] +Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] (44) HashAggregate [codegen id : 11] -Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] -Keys [1]: [cp_catalog_page_id#69] +Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] +Keys [1]: [cp_catalog_page_id#68] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] -Results [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] +Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] +Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] (45) Exchange -Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] -Arguments: hashpartitioning(cp_catalog_page_id#69, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] -Keys [1]: [cp_catalog_page_id#69] +Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Keys [1]: [cp_catalog_page_id#68] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] -Results [5]: [catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#69) AS id#85, MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS returns#87, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#88] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] +Results [5]: [catalog channel AS channel#83, concat(catalog_page, cp_catalog_page_id#68) AS id#84, MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#85, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS returns#86, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#87] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Condition : isnotnull(ws_web_site_sk#89) +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Condition : isnotnull(ws_web_site_sk#88) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] -Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] +Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] (53) BroadcastExchange -Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#104] +Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#103] (54) Scan parquet default.web_sales -Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (55) ColumnarToRow -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] (56) Filter -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] -Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) (57) Project -Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] -Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] (58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#99, wr_order_number#100] -Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] +Left keys [2]: [wr_item_sk#98, wr_order_number#99] +Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] Join condition: None (59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#109, wr_returned_date_sk#103 AS date_sk#110, 0.00 AS sales_price#111, 0.00 AS profit#112, wr_return_amt#101 AS return_amt#113, wr_net_loss#102 AS net_loss#114] -Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#106, wr_returned_date_sk#102 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#100 AS return_amt#110, wr_net_loss#101 AS net_loss#111] +Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] (60) Union (61) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#115] +Output [1]: [d_date_sk#22] (62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#94] -Right keys [1]: [cast(d_date_sk#115 as bigint)] +Left keys [1]: [date_sk#93] +Right keys [1]: [cast(d_date_sk#22 as bigint)] Join condition: None (63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98] -Input [7]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, d_date_sk#115] +Output [5]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97] +Input [7]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, d_date_sk#22] (64) Scan parquet default.web_site -Output [2]: [web_site_sk#116, web_site_id#117] +Output [2]: [web_site_sk#112, web_site_id#113] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#116, web_site_id#117] +Input [2]: [web_site_sk#112, web_site_id#113] (66) Filter [codegen id : 17] -Input [2]: [web_site_sk#116, web_site_id#117] -Condition : isnotnull(web_site_sk#116) +Input [2]: [web_site_sk#112, web_site_id#113] +Condition : isnotnull(web_site_sk#112) (67) BroadcastExchange -Input [2]: [web_site_sk#116, web_site_id#117] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] +Input [2]: [web_site_sk#112, web_site_id#113] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#114] (68) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#93] -Right keys [1]: [web_site_sk#116] +Left keys [1]: [wsr_web_site_sk#92] +Right keys [1]: [web_site_sk#112] Join condition: None (69) Project [codegen id : 18] -Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Input [7]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] +Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] +Input [7]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#112, web_site_id#113] (70) HashAggregate [codegen id : 18] -Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] -Keys [1]: [web_site_id#117] -Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] -Aggregate Attributes [4]: [sum#119, sum#120, sum#121, sum#122] -Results [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] +Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] +Keys [1]: [web_site_id#113] +Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] +Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] +Results [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] (71) Exchange -Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] -Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#127] +Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] +Arguments: hashpartitioning(web_site_id#113, 5), ENSURE_REQUIREMENTS, [id=#123] (72) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] -Keys [1]: [web_site_id#117] -Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#128, sum(UnscaledValue(return_amt#97))#129, sum(UnscaledValue(profit#96))#130, sum(UnscaledValue(net_loss#98))#131] -Results [5]: [web channel AS channel#132, concat(web_site, web_site_id#117) AS id#133, MakeDecimal(sum(UnscaledValue(sales_price#95))#128,17,2) AS sales#134, MakeDecimal(sum(UnscaledValue(return_amt#97))#129,17,2) AS returns#135, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#130,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#131,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#136] +Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] +Keys [1]: [web_site_id#113] +Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#124, sum(UnscaledValue(return_amt#96))#125, sum(UnscaledValue(profit#95))#126, sum(UnscaledValue(net_loss#97))#127] +Results [5]: [web channel AS channel#128, concat(web_site, web_site_id#113) AS id#129, MakeDecimal(sum(UnscaledValue(sales_price#94))#124,17,2) AS sales#130, MakeDecimal(sum(UnscaledValue(return_amt#96))#125,17,2) AS returns#131, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#126,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#127,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#132] (73) Union @@ -429,99 +429,99 @@ Results [5]: [web channel AS channel#132, concat(web_site, web_site_id#117) AS i Input [5]: [channel#41, id#42, sales#43, returns#44, profit#45] Keys [2]: [channel#41, id#42] Functions [3]: [partial_sum(sales#43), partial_sum(returns#44), partial_sum(profit#45)] -Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] -Results [8]: [channel#41, id#42, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Aggregate Attributes [6]: [sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] +Results [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] (75) Exchange -Input [8]: [channel#41, id#42, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#149] +Input [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#145] (76) HashAggregate [codegen id : 21] -Input [8]: [channel#41, id#42, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Input [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#45)] -Aggregate Attributes [3]: [sum(sales#43)#150, sum(returns#44)#151, sum(profit#45)#152] -Results [5]: [channel#41, id#42, cast(sum(sales#43)#150 as decimal(37,2)) AS sales#153, cast(sum(returns#44)#151 as decimal(37,2)) AS returns#154, cast(sum(profit#45)#152 as decimal(38,2)) AS profit#155] +Aggregate Attributes [3]: [sum(sales#43)#146, sum(returns#44)#147, sum(profit#45)#148] +Results [5]: [channel#41, id#42, cast(sum(sales#43)#146 as decimal(37,2)) AS sales#149, cast(sum(returns#44)#147 as decimal(37,2)) AS returns#150, cast(sum(profit#45)#148 as decimal(38,2)) AS profit#151] (77) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Output [8]: [channel#41, id#42, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] (78) HashAggregate [codegen id : 42] -Input [8]: [channel#41, id#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Input [8]: [channel#41, id#42, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#162)] -Aggregate Attributes [3]: [sum(sales#43)#163, sum(returns#44)#164, sum(profit#162)#165] -Results [4]: [channel#41, sum(sales#43)#163 AS sales#166, sum(returns#44)#164 AS returns#167, sum(profit#162)#165 AS profit#168] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#158)] +Aggregate Attributes [3]: [sum(sales#43)#159, sum(returns#44)#160, sum(profit#158)#161] +Results [4]: [channel#41, sum(sales#43)#159 AS sales#162, sum(returns#44)#160 AS returns#163, sum(profit#158)#161 AS profit#164] (79) HashAggregate [codegen id : 42] -Input [4]: [channel#41, sales#166, returns#167, profit#168] +Input [4]: [channel#41, sales#162, returns#163, profit#164] Keys [1]: [channel#41] -Functions [3]: [partial_sum(sales#166), partial_sum(returns#167), partial_sum(profit#168)] -Aggregate Attributes [6]: [sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174] -Results [7]: [channel#41, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] +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#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] (80) Exchange -Input [7]: [channel#41, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] -Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#181] +Input [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#177] (81) HashAggregate [codegen id : 43] -Input [7]: [channel#41, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] +Input [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] Keys [1]: [channel#41] -Functions [3]: [sum(sales#166), sum(returns#167), sum(profit#168)] -Aggregate Attributes [3]: [sum(sales#166)#182, sum(returns#167)#183, sum(profit#168)#184] -Results [5]: [channel#41, null AS id#185, sum(sales#166)#182 AS sum(sales)#186, sum(returns#167)#183 AS sum(returns)#187, sum(profit#168)#184 AS sum(profit)#188] +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#41, 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] (82) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194] +Output [8]: [channel#41, id#42, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] (83) HashAggregate [codegen id : 64] -Input [8]: [channel#41, id#42, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194] +Input [8]: [channel#41, id#42, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#195)] -Aggregate Attributes [3]: [sum(sales#43)#196, sum(returns#44)#197, sum(profit#195)#198] -Results [3]: [sum(sales#43)#196 AS sales#166, sum(returns#44)#197 AS returns#167, sum(profit#195)#198 AS profit#168] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#191)] +Aggregate Attributes [3]: [sum(sales#43)#192, sum(returns#44)#193, sum(profit#191)#194] +Results [3]: [sum(sales#43)#192 AS sales#162, sum(returns#44)#193 AS returns#163, sum(profit#191)#194 AS profit#164] (84) HashAggregate [codegen id : 64] -Input [3]: [sales#166, returns#167, profit#168] +Input [3]: [sales#162, returns#163, profit#164] Keys: [] -Functions [3]: [partial_sum(sales#166), partial_sum(returns#167), partial_sum(profit#168)] -Aggregate Attributes [6]: [sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204] -Results [6]: [sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210] +Functions [3]: [partial_sum(sales#162), partial_sum(returns#163), partial_sum(profit#164)] +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] (85) Exchange -Input [6]: [sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#211] +Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#207] (86) HashAggregate [codegen id : 65] -Input [6]: [sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210] +Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] Keys: [] -Functions [3]: [sum(sales#166), sum(returns#167), sum(profit#168)] -Aggregate Attributes [3]: [sum(sales#166)#212, sum(returns#167)#213, sum(profit#168)#214] -Results [5]: [null AS channel#215, null AS id#216, sum(sales#166)#212 AS sum(sales)#217, sum(returns#167)#213 AS sum(returns)#218, sum(profit#168)#214 AS sum(profit)#219] +Functions [3]: [sum(sales#162), sum(returns#163), sum(profit#164)] +Aggregate Attributes [3]: [sum(sales#162)#208, sum(returns#163)#209, sum(profit#164)#210] +Results [5]: [null AS channel#211, null AS id#212, sum(sales#162)#208 AS sum(sales)#213, sum(returns#163)#209 AS sum(returns)#214, sum(profit#164)#210 AS sum(profit)#215] (87) Union (88) HashAggregate [codegen id : 66] -Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] -Keys [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Keys [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Results [5]: [channel#41, id#42, sales#149, returns#150, profit#151] (89) Exchange -Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] -Arguments: hashpartitioning(channel#41, id#42, sales#153, returns#154, profit#155, 5), ENSURE_REQUIREMENTS, [id=#220] +Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Arguments: hashpartitioning(channel#41, id#42, sales#149, returns#150, profit#151, 5), ENSURE_REQUIREMENTS, [id=#216] (90) HashAggregate [codegen id : 67] -Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] -Keys [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Keys [5]: [channel#41, id#42, sales#149, returns#150, profit#151] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Results [5]: [channel#41, id#42, sales#149, returns#150, profit#151] (91) TakeOrderedAndProject -Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] -Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#153, returns#154, profit#155] +Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#149, returns#150, profit#151] ===== Subqueries ===== @@ -539,12 +539,12 @@ ReusedExchange (93) (93) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#67] +Output [1]: [d_date_sk#22] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 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 ad5cf43c6febc..c096a28386361 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 @@ -795,353 +795,353 @@ Input [17]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_str Arguments: [item_sk#112 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, store_zip#114 ASC NULLS FIRST], false, 0 (132) Scan parquet default.store_sales -Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#140), dynamicpruningexpression(ss_sold_date_sk#140 IN dynamicpruning#141)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#129)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (133) ColumnarToRow [codegen id : 44] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] (134) Filter [codegen id : 44] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Condition : (((((((isnotnull(ss_item_sk#129) AND isnotnull(ss_ticket_number#136)) AND isnotnull(ss_store_sk#134)) AND isnotnull(ss_customer_sk#130)) AND isnotnull(ss_cdemo_sk#131)) AND isnotnull(ss_promo_sk#135)) AND isnotnull(ss_hdemo_sk#132)) AND isnotnull(ss_addr_sk#133)) +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) (135) Exchange -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Arguments: hashpartitioning(cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint), 5), ENSURE_REQUIREMENTS, [id=#142] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: hashpartitioning(cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint), 5), ENSURE_REQUIREMENTS, [id=#130] (136) Sort [codegen id : 45] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Arguments: [cast(ss_item_sk#129 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#136 as bigint) ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: [cast(ss_item_sk#1 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#8 as bigint) ASC NULLS FIRST], false, 0 (137) ReusedExchange [Reuses operator id: 10] -Output [2]: [sr_item_sk#143, sr_ticket_number#144] +Output [2]: [sr_item_sk#15, sr_ticket_number#16] (138) Sort [codegen id : 47] -Input [2]: [sr_item_sk#143, sr_ticket_number#144] -Arguments: [sr_item_sk#143 ASC NULLS FIRST, sr_ticket_number#144 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#15, sr_ticket_number#16] +Arguments: [sr_item_sk#15 ASC NULLS FIRST, sr_ticket_number#16 ASC NULLS FIRST], false, 0 (139) SortMergeJoin [codegen id : 56] -Left keys [2]: [cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint)] -Right keys [2]: [sr_item_sk#143, sr_ticket_number#144] +Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] +Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] Join condition: None (140) Project [codegen id : 56] -Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, sr_item_sk#143, sr_ticket_number#144] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] (141) ReusedExchange [Reuses operator id: 33] -Output [1]: [cs_item_sk#145] +Output [1]: [cs_item_sk#19] (142) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_item_sk#129] -Right keys [1]: [cs_item_sk#145] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [cs_item_sk#19] Join condition: None (143) Project [codegen id : 56] -Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, cs_item_sk#145] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] (144) Scan parquet default.date_dim -Output [2]: [d_date_sk#146, d_year#147] +Output [2]: [d_date_sk#43, d_year#44] 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 (145) ColumnarToRow [codegen id : 54] -Input [2]: [d_date_sk#146, d_year#147] +Input [2]: [d_date_sk#43, d_year#44] (146) Filter [codegen id : 54] -Input [2]: [d_date_sk#146, d_year#147] -Condition : ((isnotnull(d_year#147) AND (d_year#147 = 2000)) AND isnotnull(d_date_sk#146)) +Input [2]: [d_date_sk#43, d_year#44] +Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#43)) (147) BroadcastExchange -Input [2]: [d_date_sk#146, d_year#147] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#148] +Input [2]: [d_date_sk#43, d_year#44] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#131] (148) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_sold_date_sk#140] -Right keys [1]: [d_date_sk#146] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#43] Join condition: None (149) Project [codegen id : 56] -Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147] -Input [13]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, d_date_sk#146, d_year#147] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#43, d_year#44] (150) ReusedExchange [Reuses operator id: 45] -Output [3]: [s_store_sk#149, s_store_name#150, s_zip#151] +Output [3]: [s_store_sk#46, s_store_name#47, s_zip#48] (151) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_store_sk#134] -Right keys [1]: [s_store_sk#149] +Left keys [1]: [ss_store_sk#6] +Right keys [1]: [s_store_sk#46] Join condition: None (152) Project [codegen id : 56] -Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] -Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_sk#149, s_store_name#150, s_zip#151] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_sk#46, s_store_name#47, s_zip#48] (153) Exchange -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] -Arguments: hashpartitioning(ss_customer_sk#130, 5), ENSURE_REQUIREMENTS, [id=#152] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#132] (154) Sort [codegen id : 57] -Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] -Arguments: [ss_customer_sk#130 ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (155) ReusedExchange [Reuses operator id: 53] -Output [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Output [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] (156) Sort [codegen id : 59] -Input [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] -Arguments: [c_customer_sk#153 ASC NULLS FIRST], false, 0 +Input [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Arguments: [c_customer_sk#51 ASC NULLS FIRST], false, 0 (157) SortMergeJoin [codegen id : 62] -Left keys [1]: [ss_customer_sk#130] -Right keys [1]: [c_customer_sk#153] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#51] Join condition: None (158) Project [codegen id : 62] -Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] -Input [18]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] (159) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#159, d_year#160] +Output [2]: [d_date_sk#133, d_year#134] (160) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_sales_date_sk#158] -Right keys [1]: [d_date_sk#159] +Left keys [1]: [c_first_sales_date_sk#56] +Right keys [1]: [d_date_sk#133] Join condition: None (161) Project [codegen id : 62] -Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160] -Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158, d_date_sk#159, d_year#160] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56, d_date_sk#133, d_year#134] (162) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#161, d_year#162] +Output [2]: [d_date_sk#135, d_year#136] (163) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_shipto_date_sk#157] -Right keys [1]: [d_date_sk#161] +Left keys [1]: [c_first_shipto_date_sk#55] +Right keys [1]: [d_date_sk#135] Join condition: None (164) Project [codegen id : 62] -Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160, d_date_sk#161, d_year#162] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134, d_date_sk#135, d_year#136] (165) Exchange -Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Arguments: hashpartitioning(ss_cdemo_sk#131, 5), ENSURE_REQUIREMENTS, [id=#163] +Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Arguments: hashpartitioning(ss_cdemo_sk#3, 5), ENSURE_REQUIREMENTS, [id=#137] (166) Sort [codegen id : 63] -Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Arguments: [ss_cdemo_sk#131 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Arguments: [ss_cdemo_sk#3 ASC NULLS FIRST], false, 0 (167) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#164, cd_marital_status#165] +Output [2]: [cd_demo_sk#64, cd_marital_status#65] (168) Sort [codegen id : 65] -Input [2]: [cd_demo_sk#164, cd_marital_status#165] -Arguments: [cd_demo_sk#164 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#64, cd_marital_status#65] +Arguments: [cd_demo_sk#64 ASC NULLS FIRST], false, 0 (169) SortMergeJoin [codegen id : 66] -Left keys [1]: [ss_cdemo_sk#131] -Right keys [1]: [cd_demo_sk#164] +Left keys [1]: [ss_cdemo_sk#3] +Right keys [1]: [cd_demo_sk#64] Join condition: None (170) Project [codegen id : 66] -Output [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] -Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_demo_sk#164, cd_marital_status#165] +Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_demo_sk#64, cd_marital_status#65] (171) Exchange -Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] -Arguments: hashpartitioning(c_current_cdemo_sk#154, 5), ENSURE_REQUIREMENTS, [id=#166] +Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] +Arguments: hashpartitioning(c_current_cdemo_sk#52, 5), ENSURE_REQUIREMENTS, [id=#138] (172) Sort [codegen id : 67] -Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] -Arguments: [c_current_cdemo_sk#154 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] +Arguments: [c_current_cdemo_sk#52 ASC NULLS FIRST], false, 0 (173) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#167, cd_marital_status#168] +Output [2]: [cd_demo_sk#139, cd_marital_status#140] (174) Sort [codegen id : 69] -Input [2]: [cd_demo_sk#167, cd_marital_status#168] -Arguments: [cd_demo_sk#167 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#139, cd_marital_status#140] +Arguments: [cd_demo_sk#139 ASC NULLS FIRST], false, 0 (175) SortMergeJoin [codegen id : 73] -Left keys [1]: [c_current_cdemo_sk#154] -Right keys [1]: [cd_demo_sk#167] -Join condition: NOT (cd_marital_status#165 = cd_marital_status#168) +Left keys [1]: [c_current_cdemo_sk#52] +Right keys [1]: [cd_demo_sk#139] +Join condition: NOT (cd_marital_status#65 = cd_marital_status#140) (176) Project [codegen id : 73] -Output [14]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Input [18]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165, cd_demo_sk#167, cd_marital_status#168] +Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65, cd_demo_sk#139, cd_marital_status#140] (177) ReusedExchange [Reuses operator id: 84] -Output [1]: [p_promo_sk#169] +Output [1]: [p_promo_sk#70] (178) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_promo_sk#135] -Right keys [1]: [p_promo_sk#169] +Left keys [1]: [ss_promo_sk#7] +Right keys [1]: [p_promo_sk#70] Join condition: None (179) Project [codegen id : 73] -Output [13]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] -Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, p_promo_sk#169] +Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, p_promo_sk#70] (180) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#170, hd_income_band_sk#171] +Output [2]: [hd_demo_sk#72, hd_income_band_sk#73] (181) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_hdemo_sk#132] -Right keys [1]: [hd_demo_sk#170] +Left keys [1]: [ss_hdemo_sk#4] +Right keys [1]: [hd_demo_sk#72] Join condition: None (182) Project [codegen id : 73] -Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171] -Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_demo_sk#170, hd_income_band_sk#171] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_demo_sk#72, hd_income_band_sk#73] (183) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#172, hd_income_band_sk#173] +Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] (184) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [c_current_hdemo_sk#155] -Right keys [1]: [hd_demo_sk#172] +Left keys [1]: [c_current_hdemo_sk#53] +Right keys [1]: [hd_demo_sk#141] Join condition: None (185) Project [codegen id : 73] -Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] -Input [15]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_demo_sk#172, hd_income_band_sk#173] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_demo_sk#141, hd_income_band_sk#142] (186) Exchange -Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] -Arguments: hashpartitioning(ss_addr_sk#133, 5), ENSURE_REQUIREMENTS, [id=#174] +Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] +Arguments: hashpartitioning(ss_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#143] (187) Sort [codegen id : 74] -Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] -Arguments: [ss_addr_sk#133 ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] +Arguments: [ss_addr_sk#5 ASC NULLS FIRST], false, 0 (188) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Output [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] (189) Sort [codegen id : 76] -Input [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -Arguments: [ca_address_sk#175 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Arguments: [ca_address_sk#78 ASC NULLS FIRST], false, 0 (190) SortMergeJoin [codegen id : 77] -Left keys [1]: [ss_addr_sk#133] -Right keys [1]: [ca_address_sk#175] +Left keys [1]: [ss_addr_sk#5] +Right keys [1]: [ca_address_sk#78] Join condition: None (191) Project [codegen id : 77] -Output [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -Input [18]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] (192) Exchange -Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -Arguments: hashpartitioning(c_current_addr_sk#156, 5), ENSURE_REQUIREMENTS, [id=#180] +Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Arguments: hashpartitioning(c_current_addr_sk#54, 5), ENSURE_REQUIREMENTS, [id=#144] (193) Sort [codegen id : 78] -Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] -Arguments: [c_current_addr_sk#156 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Arguments: [c_current_addr_sk#54 ASC NULLS FIRST], false, 0 (194) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Output [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] (195) Sort [codegen id : 80] -Input [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] -Arguments: [ca_address_sk#181 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Arguments: [ca_address_sk#145 ASC NULLS FIRST], false, 0 (196) SortMergeJoin [codegen id : 84] -Left keys [1]: [c_current_addr_sk#156] -Right keys [1]: [ca_address_sk#181] +Left keys [1]: [c_current_addr_sk#54] +Right keys [1]: [ca_address_sk#145] Join condition: None (197) Project [codegen id : 84] -Output [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] -Input [21]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] (198) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#186] +Output [1]: [ib_income_band_sk#90] (199) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#171] -Right keys [1]: [ib_income_band_sk#186] +Left keys [1]: [hd_income_band_sk#73] +Right keys [1]: [ib_income_band_sk#90] Join condition: None (200) Project [codegen id : 84] -Output [18]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] -Input [20]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#186] +Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#90] (201) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#187] +Output [1]: [ib_income_band_sk#150] (202) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#173] -Right keys [1]: [ib_income_band_sk#187] +Left keys [1]: [hd_income_band_sk#142] +Right keys [1]: [ib_income_band_sk#150] Join condition: None (203) Project [codegen id : 84] -Output [17]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] -Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#187] +Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#150] (204) ReusedExchange [Reuses operator id: 124] -Output [2]: [i_item_sk#188, i_product_name#189] +Output [2]: [i_item_sk#93, i_product_name#96] (205) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#129] -Right keys [1]: [i_item_sk#188] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#93] Join condition: None (206) Project [codegen id : 84] -Output [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] -Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] +Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] (207) HashAggregate [codegen id : 84] -Input [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] -Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#137)), partial_sum(UnscaledValue(ss_list_price#138)), partial_sum(UnscaledValue(ss_coupon_amt#139))] -Aggregate Attributes [4]: [count#190, sum#191, sum#192, sum#193] -Results [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] +Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count#151, sum#152, sum#153, sum#154] +Results [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] (208) Exchange -Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] -Arguments: hashpartitioning(i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, 5), ENSURE_REQUIREMENTS, [id=#198] +Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] +Arguments: hashpartitioning(i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, 5), ENSURE_REQUIREMENTS, [id=#159] (209) HashAggregate [codegen id : 85] -Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] -Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#137)), sum(UnscaledValue(ss_list_price#138)), sum(UnscaledValue(ss_coupon_amt#139))] -Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#137))#200, sum(UnscaledValue(ss_list_price#138))#201, sum(UnscaledValue(ss_coupon_amt#139))#202] -Results [8]: [i_item_sk#188 AS item_sk#203, s_store_name#150 AS store_name#204, s_zip#151 AS store_zip#205, d_year#147 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#137))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#138))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#139))#202,17,2) AS s3#210] +Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] +Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count(1)#160, sum(UnscaledValue(ss_wholesale_cost#9))#161, sum(UnscaledValue(ss_list_price#10))#162, sum(UnscaledValue(ss_coupon_amt#11))#163] +Results [8]: [i_item_sk#93 AS item_sk#164, s_store_name#47 AS store_name#165, s_zip#48 AS store_zip#166, d_year#44 AS syear#167, count(1)#160 AS cnt#168, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#161,17,2) AS s1#169, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#162,17,2) AS s2#170, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#163,17,2) AS s3#171] (210) Exchange -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] +Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] +Arguments: hashpartitioning(item_sk#164, store_name#165, store_zip#166, 5), ENSURE_REQUIREMENTS, [id=#172] (211) Sort [codegen id : 86] -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] +Arguments: [item_sk#164 ASC NULLS FIRST, store_name#165 ASC NULLS FIRST, store_zip#166 ASC NULLS FIRST], false, 0 (212) SortMergeJoin [codegen id : 87] Left keys [3]: [item_sk#112, store_name#113, store_zip#114] -Right keys [3]: [item_sk#203, store_name#204, store_zip#205] -Join condition: (cnt#207 <= cnt#124) +Right keys [3]: [item_sk#164, store_name#165, store_zip#166] +Join condition: (cnt#168 <= cnt#124) (213) Project [codegen id : 87] -Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] -Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] +Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] (214) Exchange -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#208 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] +Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#169 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#173] (215) Sort [codegen id : 88] -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#208 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] +Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#169 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1152,11 +1152,11 @@ ReusedExchange (216) (216) ReusedExchange [Reuses operator id: 39] Output [2]: [d_date_sk#43, d_year#44] -Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#140 IN dynamicpruning#141 +Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#129 ReusedExchange (217) (217) ReusedExchange [Reuses operator id: 147] -Output [2]: [d_date_sk#146, d_year#147] +Output [2]: [d_date_sk#43, d_year#44] 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 804b4980e5275..9424b7f1e2225 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 @@ -701,323 +701,323 @@ Input [17]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_str Arguments: [item_sk#105 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, store_zip#107 ASC NULLS FIRST], false, 0 (115) Scan parquet default.store_sales -Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#133), dynamicpruningexpression(ss_sold_date_sk#133 IN dynamicpruning#134)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#122)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (116) ColumnarToRow [codegen id : 27] -Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] (117) Filter [codegen id : 27] -Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Condition : (((((((isnotnull(ss_item_sk#122) AND isnotnull(ss_ticket_number#129)) AND isnotnull(ss_store_sk#127)) AND isnotnull(ss_customer_sk#123)) AND isnotnull(ss_cdemo_sk#124)) AND isnotnull(ss_promo_sk#128)) AND isnotnull(ss_hdemo_sk#125)) AND isnotnull(ss_addr_sk#126)) +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) (118) BroadcastExchange -Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#135] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#123] (119) Scan parquet default.store_returns -Output [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Output [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct (120) ColumnarToRow -Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] (121) Filter -Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] -Condition : (isnotnull(sr_item_sk#136) AND isnotnull(sr_ticket_number#137)) +Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Condition : (isnotnull(sr_item_sk#15) AND isnotnull(sr_ticket_number#16)) (122) Project -Output [2]: [sr_item_sk#136, sr_ticket_number#137] -Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Output [2]: [sr_item_sk#15, sr_ticket_number#16] +Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] (123) BroadcastHashJoin [codegen id : 28] -Left keys [2]: [cast(ss_item_sk#122 as bigint), cast(ss_ticket_number#129 as bigint)] -Right keys [2]: [sr_item_sk#136, sr_ticket_number#137] +Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] +Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] Join condition: None (124) Project [codegen id : 28] -Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, sr_item_sk#136, sr_ticket_number#137] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] (125) Exchange -Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Arguments: hashpartitioning(ss_item_sk#122, 5), ENSURE_REQUIREMENTS, [id=#139] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#124] (126) Sort [codegen id : 29] -Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Arguments: [ss_item_sk#122 ASC NULLS FIRST], false, 0 +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (127) ReusedExchange [Reuses operator id: 28] -Output [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] +Output [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] (128) HashAggregate [codegen id : 35] -Input [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] -Keys [1]: [cs_item_sk#140] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#144)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#144))#148, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149] -Results [3]: [cs_item_sk#140, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#144))#148,17,2) AS sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] +Input [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] +Keys [1]: [cs_item_sk#19] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#21)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#21))#128, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129] +Results [3]: [cs_item_sk#19, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#21))#128,17,2) AS sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] (129) Filter [codegen id : 35] -Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] -Condition : (isnotnull(sum(cs_ext_list_price#144)#150) AND (cast(sum(cs_ext_list_price#144)#150 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151)), DecimalType(21,2), true))) +Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] +Condition : (isnotnull(sum(cs_ext_list_price#21)#130) AND (cast(sum(cs_ext_list_price#21)#130 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131)), DecimalType(21,2), true))) (130) Project [codegen id : 35] -Output [1]: [cs_item_sk#140] -Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] +Output [1]: [cs_item_sk#19] +Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] (131) Sort [codegen id : 35] -Input [1]: [cs_item_sk#140] -Arguments: [cs_item_sk#140 ASC NULLS FIRST], false, 0 +Input [1]: [cs_item_sk#19] +Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 (132) SortMergeJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#122] -Right keys [1]: [cs_item_sk#140] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [cs_item_sk#19] Join condition: None (133) Project [codegen id : 51] -Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] -Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, cs_item_sk#140] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] (134) Scan parquet default.date_dim -Output [2]: [d_date_sk#152, d_year#153] +Output [2]: [d_date_sk#42, d_year#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (135) ColumnarToRow [codegen id : 36] -Input [2]: [d_date_sk#152, d_year#153] +Input [2]: [d_date_sk#42, d_year#43] (136) Filter [codegen id : 36] -Input [2]: [d_date_sk#152, d_year#153] -Condition : ((isnotnull(d_year#153) AND (d_year#153 = 2000)) AND isnotnull(d_date_sk#152)) +Input [2]: [d_date_sk#42, d_year#43] +Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2000)) AND isnotnull(d_date_sk#42)) (137) BroadcastExchange -Input [2]: [d_date_sk#152, d_year#153] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#154] +Input [2]: [d_date_sk#42, d_year#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#132] (138) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_sold_date_sk#133] -Right keys [1]: [d_date_sk#152] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#42] Join condition: None (139) Project [codegen id : 51] -Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153] -Input [13]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, d_date_sk#152, d_year#153] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#42, d_year#43] (140) ReusedExchange [Reuses operator id: 44] -Output [3]: [s_store_sk#155, s_store_name#156, s_zip#157] +Output [3]: [s_store_sk#45, s_store_name#46, s_zip#47] (141) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_store_sk#127] -Right keys [1]: [s_store_sk#155] +Left keys [1]: [ss_store_sk#6] +Right keys [1]: [s_store_sk#45] Join condition: None (142) Project [codegen id : 51] -Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157] -Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_sk#155, s_store_name#156, s_zip#157] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_sk#45, s_store_name#46, s_zip#47] (143) ReusedExchange [Reuses operator id: 50] -Output [6]: [c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] +Output [6]: [c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] (144) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_customer_sk#123] -Right keys [1]: [c_customer_sk#158] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#49] Join condition: None (145) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] -Input [18]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] (146) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#164, d_year#165] +Output [2]: [d_date_sk#133, d_year#134] (147) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_sales_date_sk#163] -Right keys [1]: [d_date_sk#164] +Left keys [1]: [c_first_sales_date_sk#54] +Right keys [1]: [d_date_sk#133] Join condition: None (148) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165] -Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163, d_date_sk#164, d_year#165] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54, d_date_sk#133, d_year#134] (149) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#166, d_year#167] +Output [2]: [d_date_sk#135, d_year#136] (150) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_shipto_date_sk#162] -Right keys [1]: [d_date_sk#166] +Left keys [1]: [c_first_shipto_date_sk#53] +Right keys [1]: [d_date_sk#135] Join condition: None (151) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] -Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165, d_date_sk#166, d_year#167] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134, d_date_sk#135, d_year#136] (152) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#168, cd_marital_status#169] +Output [2]: [cd_demo_sk#61, cd_marital_status#62] (153) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_cdemo_sk#124] -Right keys [1]: [cd_demo_sk#168] +Left keys [1]: [ss_cdemo_sk#3] +Right keys [1]: [cd_demo_sk#61] Join condition: None (154) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169] -Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_demo_sk#168, cd_marital_status#169] +Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_demo_sk#61, cd_marital_status#62] (155) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#170, cd_marital_status#171] +Output [2]: [cd_demo_sk#137, cd_marital_status#138] (156) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_cdemo_sk#159] -Right keys [1]: [cd_demo_sk#170] -Join condition: NOT (cd_marital_status#169 = cd_marital_status#171) +Left keys [1]: [c_current_cdemo_sk#50] +Right keys [1]: [cd_demo_sk#137] +Join condition: NOT (cd_marital_status#62 = cd_marital_status#138) (157) Project [codegen id : 51] -Output [14]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] -Input [18]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169, cd_demo_sk#170, cd_marital_status#171] +Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62, cd_demo_sk#137, cd_marital_status#138] (158) ReusedExchange [Reuses operator id: 74] -Output [1]: [p_promo_sk#172] +Output [1]: [p_promo_sk#66] (159) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_promo_sk#128] -Right keys [1]: [p_promo_sk#172] +Left keys [1]: [ss_promo_sk#7] +Right keys [1]: [p_promo_sk#66] Join condition: None (160) Project [codegen id : 51] -Output [13]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] -Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, p_promo_sk#172] +Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, p_promo_sk#66] (161) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#173, hd_income_band_sk#174] +Output [2]: [hd_demo_sk#68, hd_income_band_sk#69] (162) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_hdemo_sk#125] -Right keys [1]: [hd_demo_sk#173] +Left keys [1]: [ss_hdemo_sk#4] +Right keys [1]: [hd_demo_sk#68] Join condition: None (163) Project [codegen id : 51] -Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174] -Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_demo_sk#173, hd_income_band_sk#174] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_demo_sk#68, hd_income_band_sk#69] (164) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#175, hd_income_band_sk#176] +Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] (165) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_hdemo_sk#160] -Right keys [1]: [hd_demo_sk#175] +Left keys [1]: [c_current_hdemo_sk#51] +Right keys [1]: [hd_demo_sk#139] Join condition: None (166) Project [codegen id : 51] -Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176] -Input [15]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_demo_sk#175, hd_income_band_sk#176] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140] +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_demo_sk#139, hd_income_band_sk#140] (167) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] +Output [5]: [ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] (168) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_addr_sk#126] -Right keys [1]: [ca_address_sk#177] +Left keys [1]: [ss_addr_sk#5] +Right keys [1]: [ca_address_sk#73] Join condition: None (169) Project [codegen id : 51] -Output [16]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] -Input [18]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] +Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] (170) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] (171) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_addr_sk#161] -Right keys [1]: [ca_address_sk#182] +Left keys [1]: [c_current_addr_sk#52] +Right keys [1]: [ca_address_sk#141] Join condition: None (172) Project [codegen id : 51] -Output [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] -Input [21]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] (173) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#187] +Output [1]: [ib_income_band_sk#84] (174) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#174] -Right keys [1]: [ib_income_band_sk#187] +Left keys [1]: [hd_income_band_sk#69] +Right keys [1]: [ib_income_band_sk#84] Join condition: None (175) Project [codegen id : 51] -Output [18]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] -Input [20]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#187] +Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#84] (176) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#188] +Output [1]: [ib_income_band_sk#146] (177) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#176] -Right keys [1]: [ib_income_band_sk#188] +Left keys [1]: [hd_income_band_sk#140] +Right keys [1]: [ib_income_band_sk#146] Join condition: None (178) Project [codegen id : 51] -Output [17]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] -Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#188] +Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#146] (179) ReusedExchange [Reuses operator id: 108] -Output [2]: [i_item_sk#189, i_product_name#190] +Output [2]: [i_item_sk#87, i_product_name#90] (180) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#122] -Right keys [1]: [i_item_sk#189] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#87] Join condition: None (181) Project [codegen id : 51] -Output [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] -Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] (182) HashAggregate [codegen id : 51] -Input [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] -Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#130)), partial_sum(UnscaledValue(ss_list_price#131)), partial_sum(UnscaledValue(ss_coupon_amt#132))] -Aggregate Attributes [4]: [count#191, sum#192, sum#193, sum#194] -Results [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] +Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count#147, sum#148, sum#149, sum#150] +Results [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] (183) HashAggregate [codegen id : 51] -Input [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] -Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#130)), sum(UnscaledValue(ss_list_price#131)), sum(UnscaledValue(ss_coupon_amt#132))] -Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#130))#200, sum(UnscaledValue(ss_list_price#131))#201, sum(UnscaledValue(ss_coupon_amt#132))#202] -Results [8]: [i_item_sk#189 AS item_sk#203, s_store_name#156 AS store_name#204, s_zip#157 AS store_zip#205, d_year#153 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#130))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#131))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#132))#202,17,2) AS s3#210] +Input [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] +Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count(1)#155, sum(UnscaledValue(ss_wholesale_cost#9))#156, sum(UnscaledValue(ss_list_price#10))#157, sum(UnscaledValue(ss_coupon_amt#11))#158] +Results [8]: [i_item_sk#87 AS item_sk#159, s_store_name#46 AS store_name#160, s_zip#47 AS store_zip#161, d_year#43 AS syear#162, count(1)#155 AS cnt#163, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#156,17,2) AS s1#164, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#157,17,2) AS s2#165, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#158,17,2) AS s3#166] (184) Exchange -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] +Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] +Arguments: hashpartitioning(item_sk#159, store_name#160, store_zip#161, 5), ENSURE_REQUIREMENTS, [id=#167] (185) Sort [codegen id : 52] -Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] -Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] +Arguments: [item_sk#159 ASC NULLS FIRST, store_name#160 ASC NULLS FIRST, store_zip#161 ASC NULLS FIRST], false, 0 (186) SortMergeJoin [codegen id : 53] Left keys [3]: [item_sk#105, store_name#106, store_zip#107] -Right keys [3]: [item_sk#203, store_name#204, store_zip#205] -Join condition: (cnt#207 <= cnt#117) +Right keys [3]: [item_sk#159, store_name#160, store_zip#161] +Join condition: (cnt#163 <= cnt#117) (187) Project [codegen id : 53] -Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] -Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] +Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] (188) Exchange -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#208 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] +Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#164 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#168] (189) Sort [codegen id : 54] -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] -Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#208 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] +Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#164 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1028,11 +1028,11 @@ ReusedExchange (190) (190) ReusedExchange [Reuses operator id: 38] Output [2]: [d_date_sk#42, d_year#43] -Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#133 IN dynamicpruning#134 +Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#122 ReusedExchange (191) (191) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#152, d_year#153] +Output [2]: [d_date_sk#42, d_year#43] 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 6d9adf1d38106..413a918da6a0f 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 @@ -445,7 +445,7 @@ Arguments: [rank(sumsales#29) windowspecdefinition(i_category#20, sumsales#29 DE (73) Filter [codegen id : 82] Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] -Condition : (isnotnull(rk#148) AND (rk#148 <= 100)) +Condition : (rk#148 <= 100) (74) TakeOrderedAndProject Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] 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 ae6b3ff4d7542..fa822f3ac9ed7 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 @@ -430,7 +430,7 @@ Arguments: [rank(sumsales#28) windowspecdefinition(i_category#19, sumsales#28 DE (70) Filter [codegen id : 55] Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] -Condition : (isnotnull(rk#147) AND (rk#147 <= 100)) +Condition : (rk#147 <= 100) (71) TakeOrderedAndProject Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] 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 3c65529504320..e024d06c710a7 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 @@ -207,7 +207,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) +Condition : (ranking#19 <= 5) (33) Project [codegen id : 7] Output [1]: [s_state#16] 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 406acb0e0a27f..5ae5fd82839cc 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 @@ -207,7 +207,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) +Condition : (ranking#19 <= 5) (33) Project [codegen id : 7] Output [1]: [s_state#16] 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 7eea8040043d6..035f89d3a04f7 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 @@ -199,285 +199,285 @@ Input [2]: [customer_id#18, year_total#19] Arguments: [customer_id#18 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Output [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] (27) Filter [codegen id : 10] -Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_customer_sk#21) +Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_customer_sk#1) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#25, d_year#26] +Output [2]: [d_date_sk#5, d_year#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#25, d_year#26] +Input [2]: [d_date_sk#5, d_year#6] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#25, d_year#26] -Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) +Input [2]: [d_date_sk#5, d_year#6] +Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2002)) AND d_year#6 IN (2001,2002)) AND isnotnull(d_date_sk#5)) (31) BroadcastExchange -Input [2]: [d_date_sk#25, d_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [2]: [d_date_sk#5, d_year#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#23] -Right keys [1]: [d_date_sk#25] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] Join condition: None (33) Project [codegen id : 10] -Output [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] -Input [5]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] +Output [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] +Input [5]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6] (34) Exchange -Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] -Arguments: hashpartitioning(ss_customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] +Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#23] (35) Sort [codegen id : 11] -Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] -Arguments: [ss_customer_sk#21 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] +Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (37) Sort [codegen id : 13] -Input [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#21] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#9] Join condition: None (39) Project [codegen id : 14] -Output [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] -Input [7]: [ss_customer_sk#21, ss_net_paid#22, d_year#26, c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] +Input [7]: [ss_customer_sk#1, ss_net_paid#2, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (40) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] -Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] -Aggregate Attributes [1]: [sum#33] -Results [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#2))] +Aggregate Attributes [1]: [sum#24] +Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] (41) Exchange -Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] -Arguments: hashpartitioning(c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] +Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#26] (42) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] -Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] -Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#22))#36] -Results [4]: [c_customer_id#30 AS customer_id#37, c_first_name#31 AS customer_first_name#38, c_last_name#32 AS customer_last_name#39, MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#36,17,2) AS year_total#40] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [sum(UnscaledValue(ss_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#2))#27] +Results [4]: [c_customer_id#10 AS customer_id#28, c_first_name#11 AS customer_first_name#29, c_last_name#12 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#2))#27,17,2) AS year_total#31] (43) Exchange -Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] -Arguments: hashpartitioning(customer_id#37, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +Arguments: hashpartitioning(customer_id#28, 5), ENSURE_REQUIREMENTS, [id=#32] (44) Sort [codegen id : 16] -Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] -Arguments: [customer_id#37 ASC NULLS FIRST], false, 0 +Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +Arguments: [customer_id#28 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#37] +Right keys [1]: [customer_id#28] Join condition: None (46) 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#33, ws_net_paid#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] (48) Filter [codegen id : 19] -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#33, ws_net_paid#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_bill_customer_sk#33) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#45, d_year#46] +Output [2]: [d_date_sk#5, d_year#6] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#45] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#5] Join condition: None (51) Project [codegen id : 19] -Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] -Input [5]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] +Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] (52) Exchange -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] -Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#36] (53) Sort [codegen id : 20] -Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] -Arguments: [ws_bill_customer_sk#42 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (55) Sort [codegen id : 22] -Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] -Arguments: [c_customer_sk#48 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#42] -Right keys [1]: [c_customer_sk#48] +Left keys [1]: [ws_bill_customer_sk#33] +Right keys [1]: [c_customer_sk#9] Join condition: None (57) Project [codegen id : 23] -Output [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] -Input [7]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46, c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] +Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (58) HashAggregate [codegen id : 23] -Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] -Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum#52] -Results [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum#37] +Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] (59) Exchange -Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] -Arguments: hashpartitioning(c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] +Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#39] (60) HashAggregate [codegen id : 24] -Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] -Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] -Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#55] -Results [2]: [c_customer_id#49 AS customer_id#56, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#55,17,2) AS year_total#57] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#40] +Results [2]: [c_customer_id#10 AS customer_id#41, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#40,17,2) AS year_total#42] (61) Filter [codegen id : 24] -Input [2]: [customer_id#56, year_total#57] -Condition : (isnotnull(year_total#57) AND (year_total#57 > 0.00)) +Input [2]: [customer_id#41, year_total#42] +Condition : (isnotnull(year_total#42) AND (year_total#42 > 0.00)) (62) Project [codegen id : 24] -Output [2]: [customer_id#56 AS customer_id#58, year_total#57 AS year_total#59] -Input [2]: [customer_id#56, year_total#57] +Output [2]: [customer_id#41 AS customer_id#43, year_total#42 AS year_total#44] +Input [2]: [customer_id#41, year_total#42] (63) Exchange -Input [2]: [customer_id#58, year_total#59] -Arguments: hashpartitioning(customer_id#58, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [2]: [customer_id#43, year_total#44] +Arguments: hashpartitioning(customer_id#43, 5), ENSURE_REQUIREMENTS, [id=#45] (64) Sort [codegen id : 25] -Input [2]: [customer_id#58, year_total#59] -Arguments: [customer_id#58 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#43, year_total#44] +Arguments: [customer_id#43 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#58] +Right keys [1]: [customer_id#43] Join condition: None (66) Project [codegen id : 26] -Output [7]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59] -Input [8]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, customer_id#58, year_total#59] +Output [7]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44] +Input [8]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#43, year_total#44] (67) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] (69) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] -Condition : isnotnull(ws_bill_customer_sk#61) +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_bill_customer_sk#33) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#64, d_year#65] +Output [2]: [d_date_sk#5, d_year#6] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#63] -Right keys [1]: [d_date_sk#64] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#5] Join condition: None (72) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] -Input [5]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65] +Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] (73) Exchange -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] -Arguments: hashpartitioning(ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#46] (74) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] -Arguments: [ws_bill_customer_sk#61 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] +Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] +Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (76) Sort [codegen id : 31] -Input [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] -Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#61] -Right keys [1]: [c_customer_sk#67] +Left keys [1]: [ws_bill_customer_sk#33] +Right keys [1]: [c_customer_sk#9] Join condition: None (78) Project [codegen id : 32] -Output [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] -Input [7]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65, c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] +Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] +Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] (79) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] -Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#62))] -Aggregate Attributes [1]: [sum#71] -Results [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum#47] +Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] (80) Exchange -Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] -Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] +Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#49] (81) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] -Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] -Functions [1]: [sum(UnscaledValue(ws_net_paid#62))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#62))#74] -Results [2]: [c_customer_id#68 AS customer_id#75, MakeDecimal(sum(UnscaledValue(ws_net_paid#62))#74,17,2) AS year_total#76] +Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] +Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] +Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#50] +Results [2]: [c_customer_id#10 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#50,17,2) AS year_total#52] (82) Exchange -Input [2]: [customer_id#75, year_total#76] -Arguments: hashpartitioning(customer_id#75, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [2]: [customer_id#51, year_total#52] +Arguments: hashpartitioning(customer_id#51, 5), ENSURE_REQUIREMENTS, [id=#53] (83) Sort [codegen id : 34] -Input [2]: [customer_id#75, year_total#76] -Arguments: [customer_id#75 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#51, year_total#52] +Arguments: [customer_id#51 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#75] -Join condition: (CASE WHEN (year_total#59 > 0.00) THEN CheckOverflow((promote_precision(year_total#76) / promote_precision(year_total#59)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#40) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#51] +Join condition: (CASE WHEN (year_total#44 > 0.00) THEN CheckOverflow((promote_precision(year_total#52) / promote_precision(year_total#44)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#31) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) (85) Project [codegen id : 35] -Output [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] -Input [9]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59, customer_id#75, year_total#76] +Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Input [9]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44, customer_id#51, year_total#52] (86) TakeOrderedAndProject -Input [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] -Arguments: 100, [customer_first_name#38 ASC NULLS FIRST, customer_id#37 ASC NULLS FIRST, customer_last_name#39 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39] +Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Arguments: 100, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] ===== Subqueries ===== @@ -488,15 +488,15 @@ ReusedExchange (87) (87) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#5, d_year#6] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#21 ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#25, d_year#26] +Output [2]: [d_date_sk#5, d_year#6] -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#24 +Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#21 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 3ee6b56189888..44a902cf6da07 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 @@ -171,282 +171,282 @@ Input [2]: [customer_id#17, year_total#18] Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) (20) Scan parquet default.customer -Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] (22) Filter [codegen id : 6] -Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] -Condition : (isnotnull(c_customer_sk#19) AND isnotnull(c_customer_id#20)) +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (23) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#19)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] (25) Filter [codegen id : 4] -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_customer_sk#23) +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Condition : isnotnull(ss_customer_sk#5) (26) BroadcastExchange -Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#19] -Right keys [1]: [ss_customer_sk#23] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#5] Join condition: None (28) Project [codegen id : 6] -Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25] -Input [7]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] +Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#28, d_year#29] +Output [2]: [d_date_sk#10, d_year#11] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#28, d_year#29] +Input [2]: [d_date_sk#10, d_year#11] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#28, d_year#29] -Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) +Input [2]: [d_date_sk#10, d_year#11] +Condition : (((isnotnull(d_year#11) AND (d_year#11 = 2002)) AND d_year#11 IN (2001,2002)) AND isnotnull(d_date_sk#10)) (32) BroadcastExchange -Input [2]: [d_date_sk#28, d_year#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [2]: [d_date_sk#10, d_year#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#25] -Right keys [1]: [d_date_sk#28] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#10] Join condition: None (34) Project [codegen id : 6] -Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] -Input [7]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#28, d_year#29] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] +Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#10, d_year#11] (35) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] -Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum#31] -Results [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] +Aggregate Attributes [1]: [sum#22] +Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] (36) Exchange -Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] -Arguments: hashpartitioning(c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#24] (37) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] -Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] -Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#34] -Results [4]: [c_customer_id#20 AS customer_id#35, c_first_name#21 AS customer_first_name#36, c_last_name#22 AS customer_last_name#37, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#34,17,2) AS year_total#38] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#25] +Results [4]: [c_customer_id#2 AS customer_id#26, c_first_name#3 AS customer_first_name#27, c_last_name#4 AS customer_last_name#28, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#25,17,2) AS year_total#29] (38) BroadcastExchange -Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#39] +Input [4]: [customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#35] +Right keys [1]: [customer_id#26] Join condition: None (40) Scan parquet default.customer -Output [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] +Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] (42) Filter [codegen id : 10] -Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] -Condition : (isnotnull(c_customer_sk#40) AND isnotnull(c_customer_id#41)) +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (43) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] (45) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] -Condition : isnotnull(ws_bill_customer_sk#44) +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Condition : isnotnull(ws_bill_customer_sk#31) (46) BroadcastExchange -Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#40] -Right keys [1]: [ws_bill_customer_sk#44] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#31] Join condition: None (48) Project [codegen id : 10] -Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46] -Input [7]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] +Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#48, d_year#49] +Output [2]: [d_date_sk#10, d_year#11] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#46] -Right keys [1]: [d_date_sk#48] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#10] Join condition: None (51) Project [codegen id : 10] -Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] -Input [7]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46, d_date_sk#48, d_year#49] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] +Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] (52) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] -Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#45))] -Aggregate Attributes [1]: [sum#50] -Results [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] +Aggregate Attributes [1]: [sum#35] +Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] (53) Exchange -Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] -Arguments: hashpartitioning(c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#37] (54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] -Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] -Functions [1]: [sum(UnscaledValue(ws_net_paid#45))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#45))#53] -Results [2]: [c_customer_id#41 AS customer_id#54, MakeDecimal(sum(UnscaledValue(ws_net_paid#45))#53,17,2) AS year_total#55] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#38] +Results [2]: [c_customer_id#2 AS customer_id#39, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#38,17,2) AS year_total#40] (55) Filter [codegen id : 11] -Input [2]: [customer_id#54, year_total#55] -Condition : (isnotnull(year_total#55) AND (year_total#55 > 0.00)) +Input [2]: [customer_id#39, year_total#40] +Condition : (isnotnull(year_total#40) AND (year_total#40 > 0.00)) (56) Project [codegen id : 11] -Output [2]: [customer_id#54 AS customer_id#56, year_total#55 AS year_total#57] -Input [2]: [customer_id#54, year_total#55] +Output [2]: [customer_id#39 AS customer_id#41, year_total#40 AS year_total#42] +Input [2]: [customer_id#39, year_total#40] (57) BroadcastExchange -Input [2]: [customer_id#56, year_total#57] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#58] +Input [2]: [customer_id#41, year_total#42] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#43] (58) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#56] +Right keys [1]: [customer_id#41] Join condition: None (59) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57] -Input [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, customer_id#56, year_total#57] +Output [7]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42] +Input [8]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, customer_id#41, year_total#42] (60) Scan parquet default.customer -Output [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] +Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] (62) Filter [codegen id : 14] -Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] -Condition : (isnotnull(c_customer_sk#59) AND isnotnull(c_customer_id#60)) +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) (63) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#19)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] (65) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] -Condition : isnotnull(ws_bill_customer_sk#63) +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Condition : isnotnull(ws_bill_customer_sk#31) (66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] +Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#59] -Right keys [1]: [ws_bill_customer_sk#63] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#31] Join condition: None (68) Project [codegen id : 14] -Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65] -Input [7]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] +Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#67, d_year#68] +Output [2]: [d_date_sk#10, d_year#11] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#65] -Right keys [1]: [d_date_sk#67] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#10] Join condition: None (71) Project [codegen id : 14] -Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] -Input [7]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65, d_date_sk#67, d_year#68] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] +Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] (72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] -Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#64))] -Aggregate Attributes [1]: [sum#69] -Results [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] +Aggregate Attributes [1]: [sum#45] +Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] (73) Exchange -Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] -Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#47] (74) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] -Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] -Functions [1]: [sum(UnscaledValue(ws_net_paid#64))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#64))#72] -Results [2]: [c_customer_id#60 AS customer_id#73, MakeDecimal(sum(UnscaledValue(ws_net_paid#64))#72,17,2) AS year_total#74] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] +Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#48] +Results [2]: [c_customer_id#2 AS customer_id#49, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#48,17,2) AS year_total#50] (75) BroadcastExchange -Input [2]: [customer_id#73, year_total#74] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#75] +Input [2]: [customer_id#49, year_total#50] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#51] (76) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#73] -Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#57)), 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#49] +Join condition: (CASE WHEN (year_total#42 > 0.00) THEN CheckOverflow((promote_precision(year_total#50) / promote_precision(year_total#42)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#29) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END) (77) Project [codegen id : 16] -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#57, customer_id#73, year_total#74] +Output [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] +Input [9]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42, customer_id#49, year_total#50] (78) 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#26, customer_first_name#27, customer_last_name#28] +Arguments: 100, [customer_first_name#27 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST, customer_last_name#28 ASC NULLS FIRST], [customer_id#26, customer_first_name#27, customer_last_name#28] ===== Subqueries ===== @@ -457,15 +457,15 @@ ReusedExchange (79) (79) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#10, d_year#11] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#19 ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#28, d_year#29] +Output [2]: [d_date_sk#10, d_year#11] -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#26 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#19 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 afd72661e3eef..955dde96eb177 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 @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#31] +Right keys [1]: [i_item_sk#7] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#36, d_year#37] +Output [2]: [d_date_sk#14, d_year#15] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] +Right keys [1]: [d_date_sk#14] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] -Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] (42) Exchange -Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] -Condition : isnotnull(ws_item_sk#47) +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_item_sk#40) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#47] -Right keys [1]: [i_item_sk#52] +Left keys [1]: [ws_item_sk#40] +Right keys [1]: [i_item_sk#7] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] -Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#57, d_year#58] +Output [2]: [d_date_sk#14, d_year#15] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#57] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#14] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] (55) Exchange -Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] (61) Exchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] -Right keys [2]: [wr_order_number#61, wr_item_sk#60] +Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] +Right keys [2]: [wr_order_number#47, wr_item_sk#46] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] -Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] +Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#69, sum#70] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Aggregate Attributes [2]: [sum#55, sum#56] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] +Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] -Condition : isnotnull(cs_item_sk#79) +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Condition : isnotnull(cs_item_sk#1) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#79] -Right keys [1]: [i_item_sk#85] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#66] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] -Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#71, d_year#72] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#90, d_year#91] +Input [2]: [d_date_sk#71, d_year#72] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#90, d_year#91] -Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) +Input [2]: [d_date_sk#71, d_year#72] +Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) (83) BroadcastExchange -Input [2]: [d_date_sk#90, d_year#91] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] +Input [2]: [d_date_sk#71, d_year#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#83] -Right keys [1]: [d_date_sk#90] +Left keys [1]: [cs_sold_date_sk#5] +Right keys [1]: [d_date_sk#71] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (86) Exchange -Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] -Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#80, cs_item_sk#79] -Right keys [2]: [cr_order_number#95, cr_item_sk#94] +Left keys [2]: [cs_order_number#2, cs_item_sk#1] +Right keys [2]: [cr_order_number#19, cr_item_sk#18] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] -Condition : isnotnull(ss_item_sk#98) +Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Condition : isnotnull(ss_item_sk#26) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#98] -Right keys [1]: [i_item_sk#103] +Left keys [1]: [ss_item_sk#26] +Right keys [1]: [i_item_sk#66] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] -Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#108, d_year#109] +Output [2]: [d_date_sk#71, d_year#72] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#102] -Right keys [1]: [d_date_sk#108] +Left keys [1]: [ss_sold_date_sk#30] +Right keys [1]: [d_date_sk#71] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (101) Exchange -Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] -Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] -Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] +Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] -Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] -Condition : isnotnull(ws_item_sk#116) +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_item_sk#40) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#116] -Right keys [1]: [i_item_sk#121] +Left keys [1]: [ws_item_sk#40] +Right keys [1]: [i_item_sk#66] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] -Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#126, d_year#127] +Output [2]: [d_date_sk#71, d_year#72] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#120] -Right keys [1]: [d_date_sk#126] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#71] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (116) Exchange -Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] -Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] -Right keys [2]: [wr_order_number#130, wr_item_sk#129] +Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] +Right keys [2]: [wr_order_number#47, wr_item_sk#46] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] -Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] +Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#135, sum#136] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Aggregate Attributes [2]: [sum#82, sum#83] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] (127) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] -Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] -Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] +Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] (129) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] -Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] (130) Sort [codegen id : 50] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] -Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 (131) 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#86, i_class_id#87, i_category_id#88, i_manufact_id#89] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] (133) TakeOrderedAndProject -Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] -Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST, sales_amt_diff#150 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST, sales_amt_diff#97 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#71, d_year#72] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 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 afd72661e3eef..955dde96eb177 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 @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#31] +Right keys [1]: [i_item_sk#7] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#36, d_year#37] +Output [2]: [d_date_sk#14, d_year#15] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#36] +Right keys [1]: [d_date_sk#14] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] -Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] (42) Exchange -Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] -Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] -Condition : isnotnull(ws_item_sk#47) +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_item_sk#40) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#47] -Right keys [1]: [i_item_sk#52] +Left keys [1]: [ws_item_sk#40] +Right keys [1]: [i_item_sk#7] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] -Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#57, d_year#58] +Output [2]: [d_date_sk#14, d_year#15] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#51] -Right keys [1]: [d_date_sk#57] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#14] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] (55) Exchange -Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] -Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] -Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] (61) Exchange -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] -Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] -Right keys [2]: [wr_order_number#61, wr_item_sk#60] +Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] +Right keys [2]: [wr_order_number#47, wr_item_sk#46] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] -Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] +Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#69, sum#70] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Aggregate Attributes [2]: [sum#55, sum#56] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] +Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] -Condition : isnotnull(cs_item_sk#79) +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Condition : isnotnull(cs_item_sk#1) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#79] -Right keys [1]: [i_item_sk#85] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#66] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] -Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#71, d_year#72] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#90, d_year#91] +Input [2]: [d_date_sk#71, d_year#72] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#90, d_year#91] -Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) +Input [2]: [d_date_sk#71, d_year#72] +Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) (83) BroadcastExchange -Input [2]: [d_date_sk#90, d_year#91] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] +Input [2]: [d_date_sk#71, d_year#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#83] -Right keys [1]: [d_date_sk#90] +Left keys [1]: [cs_sold_date_sk#5] +Right keys [1]: [d_date_sk#71] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (86) Exchange -Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] -Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] -Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#80, cs_item_sk#79] -Right keys [2]: [cr_order_number#95, cr_item_sk#94] +Left keys [2]: [cs_order_number#2, cs_item_sk#1] +Right keys [2]: [cr_order_number#19, cr_item_sk#18] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] -Condition : isnotnull(ss_item_sk#98) +Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Condition : isnotnull(ss_item_sk#26) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#98] -Right keys [1]: [i_item_sk#103] +Left keys [1]: [ss_item_sk#26] +Right keys [1]: [i_item_sk#66] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] -Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#108, d_year#109] +Output [2]: [d_date_sk#71, d_year#72] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#102] -Right keys [1]: [d_date_sk#108] +Left keys [1]: [ss_sold_date_sk#30] +Right keys [1]: [d_date_sk#71] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (101) Exchange -Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] -Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] -Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] -Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] +Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] +Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] -Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] -Condition : isnotnull(ws_item_sk#116) +Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_item_sk#40) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#116] -Right keys [1]: [i_item_sk#121] +Left keys [1]: [ws_item_sk#40] +Right keys [1]: [i_item_sk#66] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] -Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#126, d_year#127] +Output [2]: [d_date_sk#71, d_year#72] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#120] -Right keys [1]: [d_date_sk#126] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#71] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] +Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] (116) Exchange -Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] -Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] +Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] -Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] -Right keys [2]: [wr_order_number#130, wr_item_sk#129] +Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] +Right keys [2]: [wr_order_number#47, wr_item_sk#46] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] -Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] +Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#135, sum#136] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Aggregate Attributes [2]: [sum#82, sum#83] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] (127) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] -Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] -Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] -Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] +Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] +Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] (129) Exchange -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] -Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] (130) Sort [codegen id : 50] -Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] -Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 (131) 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#86, i_class_id#87, i_category_id#88, i_manufact_id#89] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] (133) TakeOrderedAndProject -Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] -Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST, sales_amt_diff#150 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST, sales_amt_diff#97 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#90, d_year#91] +Output [2]: [d_date_sk#71, d_year#72] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 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 422443509b417..e6ade3880e2d4 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 @@ -222,38 +222,38 @@ Output [4]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_s Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25] (26) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#26] +Output [1]: [d_date_sk#6] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#26 as bigint)] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (28) Project [codegen id : 6] Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#26] +Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#6] (29) HashAggregate [codegen id : 6] Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] Keys [1]: [s_store_sk#25] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#27, sum#28] -Results [3]: [s_store_sk#25, sum#29, sum#30] +Aggregate Attributes [2]: [sum#26, sum#27] +Results [3]: [s_store_sk#25, sum#28, sum#29] (30) Exchange -Input [3]: [s_store_sk#25, sum#29, sum#30] -Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [s_store_sk#25, sum#28, sum#29] +Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#25, sum#29, sum#30] +Input [3]: [s_store_sk#25, sum#28, sum#29] Keys [1]: [s_store_sk#25] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] -Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] +Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] (32) BroadcastExchange -Input [3]: [s_store_sk#25, returns#34, profit_loss#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [3]: [s_store_sk#25, returns#33, profit_loss#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] @@ -261,341 +261,341 @@ Right keys [1]: [s_store_sk#25] Join condition: None (34) Project [codegen id : 8] -Output [5]: [store channel AS channel#37, s_store_sk#9 AS id#38, sales#18, coalesce(returns#34, 0.00) AS returns#39, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#40] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#34, profit_loss#35] +Output [5]: [store channel AS channel#36, s_store_sk#9 AS id#37, sales#18, coalesce(returns#33, 0.00) AS returns#38, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#39] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] +Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] +Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#45] +Output [1]: [d_date_sk#6] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#44] -Right keys [1]: [d_date_sk#45] +Left keys [1]: [cs_sold_date_sk#43] +Right keys [1]: [d_date_sk#6] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] -Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] +Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] +Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] -Keys [1]: [cs_call_center_sk#41] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] -Aggregate Attributes [2]: [sum#46, sum#47] -Results [3]: [cs_call_center_sk#41, sum#48, sum#49] +Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] +Keys [1]: [cs_call_center_sk#40] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] +Aggregate Attributes [2]: [sum#44, sum#45] +Results [3]: [cs_call_center_sk#40, sum#46, sum#47] (41) Exchange -Input [3]: [cs_call_center_sk#41, sum#48, sum#49] -Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [3]: [cs_call_center_sk#40, sum#46, sum#47] +Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] (42) HashAggregate [codegen id : 14] -Input [3]: [cs_call_center_sk#41, sum#48, sum#49] -Keys [1]: [cs_call_center_sk#41] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] -Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] +Input [3]: [cs_call_center_sk#40, sum#46, sum#47] +Keys [1]: [cs_call_center_sk#40] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] +Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] (43) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Output [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#55), dynamicpruningexpression(cr_returned_date_sk#55 IN dynamicpruning#5)] ReadSchema: struct (44) ColumnarToRow [codegen id : 12] -Input [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] +Input [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] (45) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#58] +Output [1]: [d_date_sk#6] (46) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cr_returned_date_sk#57] -Right keys [1]: [d_date_sk#58] +Left keys [1]: [cr_returned_date_sk#55] +Right keys [1]: [d_date_sk#6] Join condition: None (47) Project [codegen id : 12] -Output [2]: [cr_return_amount#55, cr_net_loss#56] -Input [4]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57, d_date_sk#58] +Output [2]: [cr_return_amount#53, cr_net_loss#54] +Input [4]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55, d_date_sk#6] (48) HashAggregate [codegen id : 12] -Input [2]: [cr_return_amount#55, cr_net_loss#56] +Input [2]: [cr_return_amount#53, cr_net_loss#54] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#55)), partial_sum(UnscaledValue(cr_net_loss#56))] -Aggregate Attributes [2]: [sum#59, sum#60] -Results [2]: [sum#61, sum#62] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#53)), partial_sum(UnscaledValue(cr_net_loss#54))] +Aggregate Attributes [2]: [sum#56, sum#57] +Results [2]: [sum#58, sum#59] (49) Exchange -Input [2]: [sum#61, sum#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] +Input [2]: [sum#58, sum#59] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#60] (50) HashAggregate [codegen id : 13] -Input [2]: [sum#61, sum#62] +Input [2]: [sum#58, sum#59] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#55)), sum(UnscaledValue(cr_net_loss#56))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#55))#64, sum(UnscaledValue(cr_net_loss#56))#65] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#55))#64,17,2) AS returns#66, MakeDecimal(sum(UnscaledValue(cr_net_loss#56))#65,17,2) AS profit_loss#67] +Functions [2]: [sum(UnscaledValue(cr_return_amount#53)), sum(UnscaledValue(cr_net_loss#54))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#53))#61, sum(UnscaledValue(cr_net_loss#54))#62] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#53))#61,17,2) AS returns#63, MakeDecimal(sum(UnscaledValue(cr_net_loss#54))#62,17,2) AS profit_loss#64] (51) BroadcastExchange -Input [2]: [returns#66, profit_loss#67] -Arguments: IdentityBroadcastMode, [id=#68] +Input [2]: [returns#63, profit_loss#64] +Arguments: IdentityBroadcastMode, [id=#65] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#69, cs_call_center_sk#41 AS id#70, sales#53, returns#66, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#67 as decimal(18,2)))), DecimalType(18,2), true) AS profit#71] -Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#66, profit_loss#67] +Output [5]: [catalog channel AS channel#66, cs_call_center_sk#40 AS id#67, sales#51, returns#63, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#64 as decimal(18,2)))), DecimalType(18,2), true) AS profit#68] +Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#63, profit_loss#64] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] -Condition : isnotnull(ws_web_page_sk#72) +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Condition : isnotnull(ws_web_page_sk#69) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#76] +Output [1]: [d_date_sk#6] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#75] -Right keys [1]: [d_date_sk#76] +Left keys [1]: [ws_sold_date_sk#72] +Right keys [1]: [d_date_sk#6] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] -Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] +Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] +Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#77] +Output [1]: [wp_web_page_sk#73] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#77] +Input [1]: [wp_web_page_sk#73] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#77] -Condition : isnotnull(wp_web_page_sk#77) +Input [1]: [wp_web_page_sk#73] +Condition : isnotnull(wp_web_page_sk#73) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#77] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] +Input [1]: [wp_web_page_sk#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#72] -Right keys [1]: [wp_web_page_sk#77] +Left keys [1]: [ws_web_page_sk#69] +Right keys [1]: [wp_web_page_sk#73] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] -Aggregate Attributes [2]: [sum#79, sum#80] -Results [3]: [wp_web_page_sk#77, sum#81, sum#82] +Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Keys [1]: [wp_web_page_sk#73] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] +Aggregate Attributes [2]: [sum#75, sum#76] +Results [3]: [wp_web_page_sk#73, sum#77, sum#78] (67) Exchange -Input [3]: [wp_web_page_sk#77, sum#81, sum#82] -Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [3]: [wp_web_page_sk#73, sum#77, sum#78] +Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#77, sum#81, sum#82] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] -Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] +Input [3]: [wp_web_page_sk#73, sum#77, sum#78] +Keys [1]: [wp_web_page_sk#73] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] +Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] -Condition : isnotnull(wr_web_page_sk#88) +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Condition : isnotnull(wr_web_page_sk#84) (72) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#92] +Output [1]: [wp_web_page_sk#88] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#88] -Right keys [1]: [cast(wp_web_page_sk#92 as bigint)] +Left keys [1]: [wr_web_page_sk#84] +Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [4]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] -Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] +Output [4]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] +Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] (75) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#93] +Output [1]: [d_date_sk#6] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#91] -Right keys [1]: [cast(d_date_sk#93 as bigint)] +Left keys [1]: [wr_returned_date_sk#87] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] -Input [5]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92, d_date_sk#93] +Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Input [5]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88, d_date_sk#6] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] -Keys [1]: [wp_web_page_sk#92] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] -Aggregate Attributes [2]: [sum#94, sum#95] -Results [3]: [wp_web_page_sk#92, sum#96, sum#97] +Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Keys [1]: [wp_web_page_sk#88] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] +Aggregate Attributes [2]: [sum#89, sum#90] +Results [3]: [wp_web_page_sk#88, sum#91, sum#92] (79) Exchange -Input [3]: [wp_web_page_sk#92, sum#96, sum#97] -Arguments: hashpartitioning(wp_web_page_sk#92, 5), ENSURE_REQUIREMENTS, [id=#98] +Input [3]: [wp_web_page_sk#88, sum#91, sum#92] +Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#92, sum#96, sum#97] -Keys [1]: [wp_web_page_sk#92] -Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] -Results [3]: [wp_web_page_sk#92, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] +Input [3]: [wp_web_page_sk#88, sum#91, sum#92] +Keys [1]: [wp_web_page_sk#88] +Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] +Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#92, returns#101, profit_loss#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] +Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#77] -Right keys [1]: [wp_web_page_sk#92] +Left keys [1]: [wp_web_page_sk#73] +Right keys [1]: [wp_web_page_sk#88] Join condition: None (83) Project [codegen id : 22] -Output [5]: [web channel AS channel#104, wp_web_page_sk#77 AS id#105, sales#86, coalesce(returns#101, 0.00) AS returns#106, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#107] -Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#92, returns#101, profit_loss#102] +Output [5]: [web channel AS channel#99, wp_web_page_sk#73 AS id#100, sales#82, coalesce(returns#96, 0.00) AS returns#101, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#102] +Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] (84) Union (85) HashAggregate [codegen id : 23] -Input [5]: [channel#37, id#38, sales#18, returns#39, profit#40] -Keys [2]: [channel#37, id#38] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#39), partial_sum(profit#40)] -Aggregate Attributes [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -Results [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Input [5]: [channel#36, id#37, sales#18, returns#38, profit#39] +Keys [2]: [channel#36, id#37] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#38), partial_sum(profit#39)] +Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Results [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] (86) Exchange -Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [id=#120] +Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Arguments: hashpartitioning(channel#36, id#37, 5), ENSURE_REQUIREMENTS, [id=#115] (87) HashAggregate [codegen id : 24] -Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#40)] -Aggregate Attributes [3]: [sum(sales#18)#121, sum(returns#39)#122, sum(profit#40)#123] -Results [5]: [channel#37, id#38, cast(sum(sales#18)#121 as decimal(37,2)) AS sales#124, cast(sum(returns#39)#122 as decimal(37,2)) AS returns#125, cast(sum(profit#40)#123 as decimal(38,2)) AS profit#126] +Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#36, id#37] +Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#39)] +Aggregate Attributes [3]: [sum(sales#18)#116, sum(returns#38)#117, sum(profit#39)#118] +Results [5]: [channel#36, id#37, cast(sum(sales#18)#116 as decimal(37,2)) AS sales#119, cast(sum(returns#38)#117 as decimal(37,2)) AS returns#120, cast(sum(profit#39)#118 as decimal(38,2)) AS profit#121] (88) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Output [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] (89) HashAggregate [codegen id : 48] -Input [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#133)] -Aggregate Attributes [3]: [sum(sales#18)#134, sum(returns#39)#135, sum(profit#133)#136] -Results [4]: [channel#37, sum(sales#18)#134 AS sales#137, sum(returns#39)#135 AS returns#138, sum(profit#133)#136 AS profit#139] +Input [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] +Keys [2]: [channel#36, id#37] +Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#128)] +Aggregate Attributes [3]: [sum(sales#18)#129, sum(returns#38)#130, sum(profit#128)#131] +Results [4]: [channel#36, sum(sales#18)#129 AS sales#132, sum(returns#38)#130 AS returns#133, sum(profit#128)#131 AS profit#134] (90) HashAggregate [codegen id : 48] -Input [4]: [channel#37, sales#137, returns#138, profit#139] -Keys [1]: [channel#37] -Functions [3]: [partial_sum(sales#137), partial_sum(returns#138), partial_sum(profit#139)] -Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -Results [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Input [4]: [channel#36, sales#132, returns#133, profit#134] +Keys [1]: [channel#36] +Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] +Aggregate Attributes [6]: [sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +Results [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] (91) Exchange -Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [id=#152] +Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Arguments: hashpartitioning(channel#36, 5), ENSURE_REQUIREMENTS, [id=#147] (92) HashAggregate [codegen id : 49] -Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -Keys [1]: [channel#37] -Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] -Aggregate Attributes [3]: [sum(sales#137)#153, sum(returns#138)#154, sum(profit#139)#155] -Results [5]: [channel#37, null AS id#156, sum(sales#137)#153 AS sales#157, sum(returns#138)#154 AS returns#158, sum(profit#139)#155 AS profit#159] +Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Keys [1]: [channel#36] +Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] +Aggregate Attributes [3]: [sum(sales#132)#148, sum(returns#133)#149, sum(profit#134)#150] +Results [5]: [channel#36, null AS id#151, sum(sales#132)#148 AS sales#152, sum(returns#133)#149 AS returns#153, sum(profit#134)#150 AS profit#154] (93) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +Output [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] (94) HashAggregate [codegen id : 73] -Input [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#166)] -Aggregate Attributes [3]: [sum(sales#18)#167, sum(returns#39)#168, sum(profit#166)#169] -Results [3]: [sum(sales#18)#167 AS sales#137, sum(returns#39)#168 AS returns#138, sum(profit#166)#169 AS profit#139] +Input [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] +Keys [2]: [channel#36, id#37] +Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#161)] +Aggregate Attributes [3]: [sum(sales#18)#162, sum(returns#38)#163, sum(profit#161)#164] +Results [3]: [sum(sales#18)#162 AS sales#132, sum(returns#38)#163 AS returns#133, sum(profit#161)#164 AS profit#134] (95) HashAggregate [codegen id : 73] -Input [3]: [sales#137, returns#138, profit#139] +Input [3]: [sales#132, returns#133, profit#134] Keys: [] -Functions [3]: [partial_sum(sales#137), partial_sum(returns#138), partial_sum(profit#139)] -Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] -Results [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] +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] (96) Exchange -Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#182] +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#177] (97) HashAggregate [codegen id : 74] -Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] Keys: [] -Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] -Aggregate Attributes [3]: [sum(sales#137)#183, sum(returns#138)#184, sum(profit#139)#185] -Results [5]: [null AS channel#186, null AS id#187, sum(sales#137)#183 AS sales#188, sum(returns#138)#184 AS returns#189, sum(profit#139)#185 AS profit#190] +Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] +Aggregate Attributes [3]: [sum(sales#132)#178, sum(returns#133)#179, sum(profit#134)#180] +Results [5]: [null AS channel#181, null AS id#182, sum(sales#132)#178 AS sales#183, sum(returns#133)#179 AS returns#184, sum(profit#134)#180 AS profit#185] (98) Union (99) HashAggregate [codegen id : 75] -Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] -Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] (100) Exchange -Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] -Arguments: hashpartitioning(channel#37, id#38, sales#124, returns#125, profit#126, 5), ENSURE_REQUIREMENTS, [id=#191] +Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Arguments: hashpartitioning(channel#36, id#37, sales#119, returns#120, profit#121, 5), ENSURE_REQUIREMENTS, [id=#186] (101) HashAggregate [codegen id : 76] -Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] -Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] (102) TakeOrderedAndProject -Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] -Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#124, returns#125, profit#126] +Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Arguments: 100, [channel#36 ASC NULLS FIRST, id#37 ASC NULLS FIRST], [channel#36, id#37, sales#119, returns#120, profit#121] ===== Subqueries ===== @@ -611,14 +611,14 @@ ReusedExchange (104) (104) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#26] +Output [1]: [d_date_sk#6] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#55 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 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 705277b65ff0a..0ec2ed7cca356 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 @@ -210,392 +210,392 @@ Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_s Condition : isnotnull(sr_store_sk#20) (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#6] (24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#25 as bigint)] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (25) Project [codegen id : 6] Output [3]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22] -Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#25] +Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#6] (26) ReusedExchange [Reuses operator id: 14] -Output [1]: [s_store_sk#26] +Output [1]: [s_store_sk#25] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_store_sk#20] -Right keys [1]: [cast(s_store_sk#26 as bigint)] +Right keys [1]: [cast(s_store_sk#25 as bigint)] Join condition: None (28) Project [codegen id : 6] -Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] -Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#26] +Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] +Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#25] (29) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] -Keys [1]: [s_store_sk#26] +Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] +Keys [1]: [s_store_sk#25] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#27, sum#28] -Results [3]: [s_store_sk#26, sum#29, sum#30] +Aggregate Attributes [2]: [sum#26, sum#27] +Results [3]: [s_store_sk#25, sum#28, sum#29] (30) Exchange -Input [3]: [s_store_sk#26, sum#29, sum#30] -Arguments: hashpartitioning(s_store_sk#26, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [s_store_sk#25, sum#28, sum#29] +Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#26, sum#29, sum#30] -Keys [1]: [s_store_sk#26] +Input [3]: [s_store_sk#25, sum#28, sum#29] +Keys [1]: [s_store_sk#25] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] -Results [3]: [s_store_sk#26, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] +Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] (32) BroadcastExchange -Input [3]: [s_store_sk#26, returns#34, profit_loss#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [3]: [s_store_sk#25, returns#33, profit_loss#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] -Right keys [1]: [s_store_sk#26] +Right keys [1]: [s_store_sk#25] Join condition: None (34) Project [codegen id : 8] -Output [5]: [store channel AS channel#37, s_store_sk#9 AS id#38, sales#18, coalesce(returns#34, 0.00) AS returns#39, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#40] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#26, returns#34, profit_loss#35] +Output [5]: [store channel AS channel#36, s_store_sk#9 AS id#37, sales#18, coalesce(returns#33, 0.00) AS returns#38, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#39] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] +Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] +Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#45] +Output [1]: [d_date_sk#6] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#44] -Right keys [1]: [d_date_sk#45] +Left keys [1]: [cs_sold_date_sk#43] +Right keys [1]: [d_date_sk#6] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] -Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] +Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] +Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] -Keys [1]: [cs_call_center_sk#41] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] -Aggregate Attributes [2]: [sum#46, sum#47] -Results [3]: [cs_call_center_sk#41, sum#48, sum#49] +Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] +Keys [1]: [cs_call_center_sk#40] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] +Aggregate Attributes [2]: [sum#44, sum#45] +Results [3]: [cs_call_center_sk#40, sum#46, sum#47] (41) Exchange -Input [3]: [cs_call_center_sk#41, sum#48, sum#49] -Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [3]: [cs_call_center_sk#40, sum#46, sum#47] +Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#41, sum#48, sum#49] -Keys [1]: [cs_call_center_sk#41] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] -Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] +Input [3]: [cs_call_center_sk#40, sum#46, sum#47] +Keys [1]: [cs_call_center_sk#40] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] +Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] (43) BroadcastExchange -Input [3]: [cs_call_center_sk#41, sales#53, profit#54] -Arguments: IdentityBroadcastMode, [id=#55] +Input [3]: [cs_call_center_sk#40, sales#51, profit#52] +Arguments: IdentityBroadcastMode, [id=#53] (44) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] +Output [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#58), dynamicpruningexpression(cr_returned_date_sk#58 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] ReadSchema: struct (45) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] +Input [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] (46) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#59] +Output [1]: [d_date_sk#6] (47) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#58] -Right keys [1]: [d_date_sk#59] +Left keys [1]: [cr_returned_date_sk#56] +Right keys [1]: [d_date_sk#6] Join condition: None (48) Project [codegen id : 13] -Output [2]: [cr_return_amount#56, cr_net_loss#57] -Input [4]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58, d_date_sk#59] +Output [2]: [cr_return_amount#54, cr_net_loss#55] +Input [4]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56, d_date_sk#6] (49) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#56, cr_net_loss#57] +Input [2]: [cr_return_amount#54, cr_net_loss#55] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#56)), partial_sum(UnscaledValue(cr_net_loss#57))] -Aggregate Attributes [2]: [sum#60, sum#61] -Results [2]: [sum#62, sum#63] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#54)), partial_sum(UnscaledValue(cr_net_loss#55))] +Aggregate Attributes [2]: [sum#57, sum#58] +Results [2]: [sum#59, sum#60] (50) Exchange -Input [2]: [sum#62, sum#63] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#64] +Input [2]: [sum#59, sum#60] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] (51) HashAggregate -Input [2]: [sum#62, sum#63] +Input [2]: [sum#59, sum#60] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#56)), sum(UnscaledValue(cr_net_loss#57))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#56))#65, sum(UnscaledValue(cr_net_loss#57))#66] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#56))#65,17,2) AS returns#67, MakeDecimal(sum(UnscaledValue(cr_net_loss#57))#66,17,2) AS profit_loss#68] +Functions [2]: [sum(UnscaledValue(cr_return_amount#54)), sum(UnscaledValue(cr_net_loss#55))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#54))#62, sum(UnscaledValue(cr_net_loss#55))#63] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#54))#62,17,2) AS returns#64, MakeDecimal(sum(UnscaledValue(cr_net_loss#55))#63,17,2) AS profit_loss#65] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#69, cs_call_center_sk#41 AS id#70, sales#53, returns#67, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#68 as decimal(18,2)))), DecimalType(18,2), true) AS profit#71] -Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#67, profit_loss#68] +Output [5]: [catalog channel AS channel#66, cs_call_center_sk#40 AS id#67, sales#51, returns#64, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#65 as decimal(18,2)))), DecimalType(18,2), true) AS profit#68] +Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#64, profit_loss#65] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] -Condition : isnotnull(ws_web_page_sk#72) +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Condition : isnotnull(ws_web_page_sk#69) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#76] +Output [1]: [d_date_sk#6] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#75] -Right keys [1]: [d_date_sk#76] +Left keys [1]: [ws_sold_date_sk#72] +Right keys [1]: [d_date_sk#6] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] -Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] +Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] +Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#77] +Output [1]: [wp_web_page_sk#73] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#77] +Input [1]: [wp_web_page_sk#73] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#77] -Condition : isnotnull(wp_web_page_sk#77) +Input [1]: [wp_web_page_sk#73] +Condition : isnotnull(wp_web_page_sk#73) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#77] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] +Input [1]: [wp_web_page_sk#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#72] -Right keys [1]: [wp_web_page_sk#77] +Left keys [1]: [ws_web_page_sk#69] +Right keys [1]: [wp_web_page_sk#73] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] -Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] -Aggregate Attributes [2]: [sum#79, sum#80] -Results [3]: [wp_web_page_sk#77, sum#81, sum#82] +Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Keys [1]: [wp_web_page_sk#73] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] +Aggregate Attributes [2]: [sum#75, sum#76] +Results [3]: [wp_web_page_sk#73, sum#77, sum#78] (67) Exchange -Input [3]: [wp_web_page_sk#77, sum#81, sum#82] -Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [3]: [wp_web_page_sk#73, sum#77, sum#78] +Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#77, sum#81, sum#82] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] -Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] +Input [3]: [wp_web_page_sk#73, sum#77, sum#78] +Keys [1]: [wp_web_page_sk#73] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] +Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] -Condition : isnotnull(wr_web_page_sk#88) +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Condition : isnotnull(wr_web_page_sk#84) (72) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#92] +Output [1]: [d_date_sk#6] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#91] -Right keys [1]: [cast(d_date_sk#92 as bigint)] +Left keys [1]: [wr_returned_date_sk#87] +Right keys [1]: [cast(d_date_sk#6 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90] -Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, d_date_sk#92] +Output [3]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86] +Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, d_date_sk#6] (75) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#93] +Output [1]: [wp_web_page_sk#88] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#88] -Right keys [1]: [cast(wp_web_page_sk#93 as bigint)] +Left keys [1]: [wr_web_page_sk#84] +Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] -Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] +Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] -Keys [1]: [wp_web_page_sk#93] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] -Aggregate Attributes [2]: [sum#94, sum#95] -Results [3]: [wp_web_page_sk#93, sum#96, sum#97] +Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Keys [1]: [wp_web_page_sk#88] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] +Aggregate Attributes [2]: [sum#89, sum#90] +Results [3]: [wp_web_page_sk#88, sum#91, sum#92] (79) Exchange -Input [3]: [wp_web_page_sk#93, sum#96, sum#97] -Arguments: hashpartitioning(wp_web_page_sk#93, 5), ENSURE_REQUIREMENTS, [id=#98] +Input [3]: [wp_web_page_sk#88, sum#91, sum#92] +Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#93, sum#96, sum#97] -Keys [1]: [wp_web_page_sk#93] -Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] -Results [3]: [wp_web_page_sk#93, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] +Input [3]: [wp_web_page_sk#88, sum#91, sum#92] +Keys [1]: [wp_web_page_sk#88] +Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] +Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#93, returns#101, profit_loss#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] +Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#77] -Right keys [1]: [wp_web_page_sk#93] +Left keys [1]: [wp_web_page_sk#73] +Right keys [1]: [wp_web_page_sk#88] Join condition: None (83) Project [codegen id : 22] -Output [5]: [web channel AS channel#104, wp_web_page_sk#77 AS id#105, sales#86, coalesce(returns#101, 0.00) AS returns#106, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#107] -Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#93, returns#101, profit_loss#102] +Output [5]: [web channel AS channel#99, wp_web_page_sk#73 AS id#100, sales#82, coalesce(returns#96, 0.00) AS returns#101, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#102] +Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] (84) Union (85) HashAggregate [codegen id : 23] -Input [5]: [channel#37, id#38, sales#18, returns#39, profit#40] -Keys [2]: [channel#37, id#38] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#39), partial_sum(profit#40)] -Aggregate Attributes [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -Results [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Input [5]: [channel#36, id#37, sales#18, returns#38, profit#39] +Keys [2]: [channel#36, id#37] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#38), partial_sum(profit#39)] +Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] +Results [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] (86) Exchange -Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [id=#120] +Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Arguments: hashpartitioning(channel#36, id#37, 5), ENSURE_REQUIREMENTS, [id=#115] (87) HashAggregate [codegen id : 24] -Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#40)] -Aggregate Attributes [3]: [sum(sales#18)#121, sum(returns#39)#122, sum(profit#40)#123] -Results [5]: [channel#37, id#38, cast(sum(sales#18)#121 as decimal(37,2)) AS sales#124, cast(sum(returns#39)#122 as decimal(37,2)) AS returns#125, cast(sum(profit#40)#123 as decimal(38,2)) AS profit#126] +Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#36, id#37] +Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#39)] +Aggregate Attributes [3]: [sum(sales#18)#116, sum(returns#38)#117, sum(profit#39)#118] +Results [5]: [channel#36, id#37, cast(sum(sales#18)#116 as decimal(37,2)) AS sales#119, cast(sum(returns#38)#117 as decimal(37,2)) AS returns#120, cast(sum(profit#39)#118 as decimal(38,2)) AS profit#121] (88) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Output [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] (89) HashAggregate [codegen id : 48] -Input [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#133)] -Aggregate Attributes [3]: [sum(sales#18)#134, sum(returns#39)#135, sum(profit#133)#136] -Results [4]: [channel#37, sum(sales#18)#134 AS sales#137, sum(returns#39)#135 AS returns#138, sum(profit#133)#136 AS profit#139] +Input [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] +Keys [2]: [channel#36, id#37] +Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#128)] +Aggregate Attributes [3]: [sum(sales#18)#129, sum(returns#38)#130, sum(profit#128)#131] +Results [4]: [channel#36, sum(sales#18)#129 AS sales#132, sum(returns#38)#130 AS returns#133, sum(profit#128)#131 AS profit#134] (90) HashAggregate [codegen id : 48] -Input [4]: [channel#37, sales#137, returns#138, profit#139] -Keys [1]: [channel#37] -Functions [3]: [partial_sum(sales#137), partial_sum(returns#138), partial_sum(profit#139)] -Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -Results [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Input [4]: [channel#36, sales#132, returns#133, profit#134] +Keys [1]: [channel#36] +Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] +Aggregate Attributes [6]: [sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +Results [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] (91) Exchange -Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [id=#152] +Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Arguments: hashpartitioning(channel#36, 5), ENSURE_REQUIREMENTS, [id=#147] (92) HashAggregate [codegen id : 49] -Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -Keys [1]: [channel#37] -Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] -Aggregate Attributes [3]: [sum(sales#137)#153, sum(returns#138)#154, sum(profit#139)#155] -Results [5]: [channel#37, null AS id#156, sum(sales#137)#153 AS sales#157, sum(returns#138)#154 AS returns#158, sum(profit#139)#155 AS profit#159] +Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Keys [1]: [channel#36] +Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] +Aggregate Attributes [3]: [sum(sales#132)#148, sum(returns#133)#149, sum(profit#134)#150] +Results [5]: [channel#36, null AS id#151, sum(sales#132)#148 AS sales#152, sum(returns#133)#149 AS returns#153, sum(profit#134)#150 AS profit#154] (93) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +Output [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] (94) HashAggregate [codegen id : 73] -Input [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] -Keys [2]: [channel#37, id#38] -Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#166)] -Aggregate Attributes [3]: [sum(sales#18)#167, sum(returns#39)#168, sum(profit#166)#169] -Results [3]: [sum(sales#18)#167 AS sales#137, sum(returns#39)#168 AS returns#138, sum(profit#166)#169 AS profit#139] +Input [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] +Keys [2]: [channel#36, id#37] +Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#161)] +Aggregate Attributes [3]: [sum(sales#18)#162, sum(returns#38)#163, sum(profit#161)#164] +Results [3]: [sum(sales#18)#162 AS sales#132, sum(returns#38)#163 AS returns#133, sum(profit#161)#164 AS profit#134] (95) HashAggregate [codegen id : 73] -Input [3]: [sales#137, returns#138, profit#139] +Input [3]: [sales#132, returns#133, profit#134] Keys: [] -Functions [3]: [partial_sum(sales#137), partial_sum(returns#138), partial_sum(profit#139)] -Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] -Results [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] +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] (96) Exchange -Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#182] +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#177] (97) HashAggregate [codegen id : 74] -Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] Keys: [] -Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] -Aggregate Attributes [3]: [sum(sales#137)#183, sum(returns#138)#184, sum(profit#139)#185] -Results [5]: [null AS channel#186, null AS id#187, sum(sales#137)#183 AS sales#188, sum(returns#138)#184 AS returns#189, sum(profit#139)#185 AS profit#190] +Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] +Aggregate Attributes [3]: [sum(sales#132)#178, sum(returns#133)#179, sum(profit#134)#180] +Results [5]: [null AS channel#181, null AS id#182, sum(sales#132)#178 AS sales#183, sum(returns#133)#179 AS returns#184, sum(profit#134)#180 AS profit#185] (98) Union (99) HashAggregate [codegen id : 75] -Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] -Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] (100) Exchange -Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] -Arguments: hashpartitioning(channel#37, id#38, sales#124, returns#125, profit#126, 5), ENSURE_REQUIREMENTS, [id=#191] +Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Arguments: hashpartitioning(channel#36, id#37, sales#119, returns#120, profit#121, 5), ENSURE_REQUIREMENTS, [id=#186] (101) HashAggregate [codegen id : 76] -Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] -Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] (102) TakeOrderedAndProject -Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] -Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#124, returns#125, profit#126] +Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Arguments: 100, [channel#36 ASC NULLS FIRST, id#37 ASC NULLS FIRST], [channel#36, id#37, sales#119, returns#120, profit#121] ===== Subqueries ===== @@ -611,14 +611,14 @@ ReusedExchange (104) (104) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#6] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#58 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt index a12ab50a89f43..0bd6422fffa5a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt @@ -248,164 +248,164 @@ Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale Input [9]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#14, d_year#15] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#14] Join condition: None (41) Project [codegen id : 13] -Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] -Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#43, d_year#44] +Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] +Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#14, d_year#15] (42) HashAggregate [codegen id : 13] -Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] -Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] +Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [partial_sum(cs_quantity#34), partial_sum(UnscaledValue(cs_wholesale_cost#35)), partial_sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum#45, sum#46, sum#47] -Results [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] +Aggregate Attributes [3]: [sum#43, sum#44, sum#45] +Results [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] (43) Exchange -Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] -Arguments: hashpartitioning(d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] +Arguments: hashpartitioning(d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#49] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] -Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] +Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [sum(cs_quantity#34), sum(UnscaledValue(cs_wholesale_cost#35)), sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum(cs_quantity#34)#52, sum(UnscaledValue(cs_wholesale_cost#35))#53, sum(UnscaledValue(cs_sales_price#36))#54] -Results [6]: [d_year#44 AS cs_sold_year#55, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#56, sum(cs_quantity#34)#52 AS cs_qty#57, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#53,17,2) AS cs_wc#58, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#54,17,2) AS cs_sp#59] +Aggregate Attributes [3]: [sum(cs_quantity#34)#50, sum(UnscaledValue(cs_wholesale_cost#35))#51, sum(UnscaledValue(cs_sales_price#36))#52] +Results [6]: [d_year#15 AS cs_sold_year#53, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#54, sum(cs_quantity#34)#50 AS cs_qty#55, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#51,17,2) AS cs_wc#56, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#52,17,2) AS cs_sp#57] (45) Filter [codegen id : 14] -Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] -Condition : (coalesce(cs_qty#57, 0) > 0) +Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] +Condition : (coalesce(cs_qty#55, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] -Arguments: [cs_sold_year#55 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#56 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] +Arguments: [cs_sold_year#53 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#54 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56] +Right keys [3]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] (49) Scan parquet default.web_sales -Output [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Output [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] (51) Filter [codegen id : 16] -Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] -Condition : (isnotnull(ws_item_sk#60) AND isnotnull(ws_bill_customer_sk#61)) +Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Condition : (isnotnull(ws_item_sk#58) AND isnotnull(ws_bill_customer_sk#59)) (52) Exchange -Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] -Arguments: hashpartitioning(cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint), 5), ENSURE_REQUIREMENTS, [id=#67] +Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Arguments: hashpartitioning(cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint), 5), ENSURE_REQUIREMENTS, [id=#65] (53) Sort [codegen id : 17] -Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] -Arguments: [cast(ws_order_number#62 as bigint) ASC NULLS FIRST, cast(ws_item_sk#60 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Arguments: [cast(ws_order_number#60 as bigint) ASC NULLS FIRST, cast(ws_item_sk#58 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.web_returns -Output [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Output [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] (56) Filter [codegen id : 18] -Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] -Condition : (isnotnull(wr_order_number#69) AND isnotnull(wr_item_sk#68)) +Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Condition : (isnotnull(wr_order_number#67) AND isnotnull(wr_item_sk#66)) (57) Project [codegen id : 18] -Output [2]: [wr_item_sk#68, wr_order_number#69] -Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Output [2]: [wr_item_sk#66, wr_order_number#67] +Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] (58) Exchange -Input [2]: [wr_item_sk#68, wr_order_number#69] -Arguments: hashpartitioning(wr_order_number#69, wr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [2]: [wr_item_sk#66, wr_order_number#67] +Arguments: hashpartitioning(wr_order_number#67, wr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] (59) Sort [codegen id : 19] -Input [2]: [wr_item_sk#68, wr_order_number#69] -Arguments: [wr_order_number#69 ASC NULLS FIRST, wr_item_sk#68 ASC NULLS FIRST], false, 0 +Input [2]: [wr_item_sk#66, wr_order_number#67] +Arguments: [wr_order_number#67 ASC NULLS FIRST, wr_item_sk#66 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint)] -Right keys [2]: [wr_order_number#69, wr_item_sk#68] +Left keys [2]: [cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint)] +Right keys [2]: [wr_order_number#67, wr_item_sk#66] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] -Condition : isnull(wr_order_number#69) +Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] +Condition : isnull(wr_order_number#67) (62) Project [codegen id : 21] -Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] -Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] +Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#72, d_year#73] +Output [2]: [d_date_sk#14, d_year#15] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [ws_sold_date_sk#66] -Right keys [1]: [d_date_sk#72] +Left keys [1]: [ws_sold_date_sk#64] +Right keys [1]: [d_date_sk#14] Join condition: None (65) Project [codegen id : 21] -Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] -Input [8]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, d_date_sk#72, d_year#73] +Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] +Input [8]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, d_date_sk#14, d_year#15] (66) HashAggregate [codegen id : 21] -Input [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] -Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] -Functions [3]: [partial_sum(ws_quantity#63), partial_sum(UnscaledValue(ws_wholesale_cost#64)), partial_sum(UnscaledValue(ws_sales_price#65))] -Aggregate Attributes [3]: [sum#74, sum#75, sum#76] -Results [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] +Input [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] +Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] +Functions [3]: [partial_sum(ws_quantity#61), partial_sum(UnscaledValue(ws_wholesale_cost#62)), partial_sum(UnscaledValue(ws_sales_price#63))] +Aggregate Attributes [3]: [sum#70, sum#71, sum#72] +Results [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] (67) Exchange -Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] -Arguments: hashpartitioning(d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] +Arguments: hashpartitioning(d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#76] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] -Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] -Functions [3]: [sum(ws_quantity#63), sum(UnscaledValue(ws_wholesale_cost#64)), sum(UnscaledValue(ws_sales_price#65))] -Aggregate Attributes [3]: [sum(ws_quantity#63)#81, sum(UnscaledValue(ws_wholesale_cost#64))#82, sum(UnscaledValue(ws_sales_price#65))#83] -Results [6]: [d_year#73 AS ws_sold_year#84, ws_item_sk#60, ws_bill_customer_sk#61 AS ws_customer_sk#85, sum(ws_quantity#63)#81 AS ws_qty#86, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#64))#82,17,2) AS ws_wc#87, MakeDecimal(sum(UnscaledValue(ws_sales_price#65))#83,17,2) AS ws_sp#88] +Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] +Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] +Functions [3]: [sum(ws_quantity#61), sum(UnscaledValue(ws_wholesale_cost#62)), sum(UnscaledValue(ws_sales_price#63))] +Aggregate Attributes [3]: [sum(ws_quantity#61)#77, sum(UnscaledValue(ws_wholesale_cost#62))#78, sum(UnscaledValue(ws_sales_price#63))#79] +Results [6]: [d_year#15 AS ws_sold_year#80, ws_item_sk#58, ws_bill_customer_sk#59 AS ws_customer_sk#81, sum(ws_quantity#61)#77 AS ws_qty#82, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#62))#78,17,2) AS ws_wc#83, MakeDecimal(sum(UnscaledValue(ws_sales_price#63))#79,17,2) AS ws_sp#84] (69) Filter [codegen id : 22] -Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] -Condition : (coalesce(ws_qty#86, 0) > 0) +Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] +Condition : (coalesce(ws_qty#82, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] -Arguments: [ws_sold_year#84 ASC NULLS FIRST, ws_item_sk#60 ASC NULLS FIRST, ws_customer_sk#85 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] +Arguments: [ws_sold_year#80 ASC NULLS FIRST, ws_item_sk#58 ASC NULLS FIRST, ws_customer_sk#81 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85] +Right keys [3]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81] Join condition: None (72) Project [codegen id : 23] -Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#86 + cs_qty#57), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#86, 0) + coalesce(cs_qty#57, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#87, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#58, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#88, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#59, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59, ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] +Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#82 + cs_qty#55), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#82, 0) + coalesce(cs_qty#55, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#83, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#56, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#84, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#57, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57, ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] (73) TakeOrderedAndProject -Input [13]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, ratio#89 ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] +Input [13]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, ratio#85 ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt index b14ab47d28afe..2d84cacd5d09c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt @@ -248,164 +248,164 @@ Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale Input [9]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_order_number#33, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, wr_item_sk#39, wr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#14, d_year#15] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#37] -Right keys [1]: [d_date_sk#43] +Right keys [1]: [d_date_sk#14] Join condition: None (41) Project [codegen id : 13] -Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] -Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#43, d_year#44] +Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] +Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#14, d_year#15] (42) HashAggregate [codegen id : 13] -Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] -Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] +Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [partial_sum(ws_quantity#34), partial_sum(UnscaledValue(ws_wholesale_cost#35)), partial_sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum#45, sum#46, sum#47] -Results [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] +Aggregate Attributes [3]: [sum#43, sum#44, sum#45] +Results [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] (43) Exchange -Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] -Arguments: hashpartitioning(d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] +Arguments: hashpartitioning(d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#49] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] -Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] +Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [sum(ws_quantity#34), sum(UnscaledValue(ws_wholesale_cost#35)), sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum(ws_quantity#34)#52, sum(UnscaledValue(ws_wholesale_cost#35))#53, sum(UnscaledValue(ws_sales_price#36))#54] -Results [6]: [d_year#44 AS ws_sold_year#55, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#56, sum(ws_quantity#34)#52 AS ws_qty#57, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#53,17,2) AS ws_wc#58, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#54,17,2) AS ws_sp#59] +Aggregate Attributes [3]: [sum(ws_quantity#34)#50, sum(UnscaledValue(ws_wholesale_cost#35))#51, sum(UnscaledValue(ws_sales_price#36))#52] +Results [6]: [d_year#15 AS ws_sold_year#53, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#54, sum(ws_quantity#34)#50 AS ws_qty#55, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#51,17,2) AS ws_wc#56, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#52,17,2) AS ws_sp#57] (45) Filter [codegen id : 14] -Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] -Condition : (coalesce(ws_qty#57, 0) > 0) +Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] +Condition : (coalesce(ws_qty#55, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] -Arguments: [ws_sold_year#55 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#56 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] +Arguments: [ws_sold_year#53 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#54 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56] +Right keys [3]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] (49) Scan parquet default.catalog_sales -Output [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Output [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#66), dynamicpruningexpression(cs_sold_date_sk#66 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#8)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] (51) Filter [codegen id : 16] -Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] -Condition : (isnotnull(cs_item_sk#61) AND isnotnull(cs_bill_customer_sk#60)) +Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Condition : (isnotnull(cs_item_sk#59) AND isnotnull(cs_bill_customer_sk#58)) (52) Exchange -Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] -Arguments: hashpartitioning(cs_order_number#62, cs_item_sk#61, 5), ENSURE_REQUIREMENTS, [id=#67] +Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Arguments: hashpartitioning(cs_order_number#60, cs_item_sk#59, 5), ENSURE_REQUIREMENTS, [id=#65] (53) Sort [codegen id : 17] -Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] -Arguments: [cs_order_number#62 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST], false, 0 +Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Arguments: [cs_order_number#60 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST], false, 0 (54) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Output [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] (56) Filter [codegen id : 18] -Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] -Condition : (isnotnull(cr_order_number#69) AND isnotnull(cr_item_sk#68)) +Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Condition : (isnotnull(cr_order_number#67) AND isnotnull(cr_item_sk#66)) (57) Project [codegen id : 18] -Output [2]: [cr_item_sk#68, cr_order_number#69] -Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Output [2]: [cr_item_sk#66, cr_order_number#67] +Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] (58) Exchange -Input [2]: [cr_item_sk#68, cr_order_number#69] -Arguments: hashpartitioning(cr_order_number#69, cr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [2]: [cr_item_sk#66, cr_order_number#67] +Arguments: hashpartitioning(cr_order_number#67, cr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] (59) Sort [codegen id : 19] -Input [2]: [cr_item_sk#68, cr_order_number#69] -Arguments: [cr_order_number#69 ASC NULLS FIRST, cr_item_sk#68 ASC NULLS FIRST], false, 0 +Input [2]: [cr_item_sk#66, cr_order_number#67] +Arguments: [cr_order_number#67 ASC NULLS FIRST, cr_item_sk#66 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cs_order_number#62, cs_item_sk#61] -Right keys [2]: [cr_order_number#69, cr_item_sk#68] +Left keys [2]: [cs_order_number#60, cs_item_sk#59] +Right keys [2]: [cr_order_number#67, cr_item_sk#66] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] -Condition : isnull(cr_order_number#69) +Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] +Condition : isnull(cr_order_number#67) (62) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] -Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] +Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#72, d_year#73] +Output [2]: [d_date_sk#14, d_year#15] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [cs_sold_date_sk#66] -Right keys [1]: [d_date_sk#72] +Left keys [1]: [cs_sold_date_sk#64] +Right keys [1]: [d_date_sk#14] Join condition: None (65) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] -Input [8]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, d_date_sk#72, d_year#73] +Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] +Input [8]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, d_date_sk#14, d_year#15] (66) HashAggregate [codegen id : 21] -Input [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] -Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] -Functions [3]: [partial_sum(cs_quantity#63), partial_sum(UnscaledValue(cs_wholesale_cost#64)), partial_sum(UnscaledValue(cs_sales_price#65))] -Aggregate Attributes [3]: [sum#74, sum#75, sum#76] -Results [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] +Input [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] +Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] +Functions [3]: [partial_sum(cs_quantity#61), partial_sum(UnscaledValue(cs_wholesale_cost#62)), partial_sum(UnscaledValue(cs_sales_price#63))] +Aggregate Attributes [3]: [sum#70, sum#71, sum#72] +Results [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] (67) Exchange -Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] -Arguments: hashpartitioning(d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] +Arguments: hashpartitioning(d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, 5), ENSURE_REQUIREMENTS, [id=#76] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] -Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] -Functions [3]: [sum(cs_quantity#63), sum(UnscaledValue(cs_wholesale_cost#64)), sum(UnscaledValue(cs_sales_price#65))] -Aggregate Attributes [3]: [sum(cs_quantity#63)#81, sum(UnscaledValue(cs_wholesale_cost#64))#82, sum(UnscaledValue(cs_sales_price#65))#83] -Results [6]: [d_year#73 AS cs_sold_year#84, cs_item_sk#61, cs_bill_customer_sk#60 AS cs_customer_sk#85, sum(cs_quantity#63)#81 AS cs_qty#86, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#64))#82,17,2) AS cs_wc#87, MakeDecimal(sum(UnscaledValue(cs_sales_price#65))#83,17,2) AS cs_sp#88] +Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] +Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] +Functions [3]: [sum(cs_quantity#61), sum(UnscaledValue(cs_wholesale_cost#62)), sum(UnscaledValue(cs_sales_price#63))] +Aggregate Attributes [3]: [sum(cs_quantity#61)#77, sum(UnscaledValue(cs_wholesale_cost#62))#78, sum(UnscaledValue(cs_sales_price#63))#79] +Results [6]: [d_year#15 AS cs_sold_year#80, cs_item_sk#59, cs_bill_customer_sk#58 AS cs_customer_sk#81, sum(cs_quantity#61)#77 AS cs_qty#82, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#62))#78,17,2) AS cs_wc#83, MakeDecimal(sum(UnscaledValue(cs_sales_price#63))#79,17,2) AS cs_sp#84] (69) Filter [codegen id : 22] -Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] -Condition : (coalesce(cs_qty#86, 0) > 0) +Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] +Condition : (coalesce(cs_qty#82, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] -Arguments: [cs_sold_year#84 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST, cs_customer_sk#85 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] +Arguments: [cs_sold_year#80 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST, cs_customer_sk#81 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85] +Right keys [3]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81] Join condition: None (72) Project [codegen id : 23] -Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#57 + cs_qty#86), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#57, 0) + coalesce(cs_qty#86, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#58, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#87, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#59, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#88, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59, cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] +Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#55 + cs_qty#82), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#55, 0) + coalesce(cs_qty#82, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#56, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#83, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#57, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#84, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57, cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] (73) TakeOrderedAndProject -Input [13]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, ratio#89 ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] +Input [13]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, ratio#85 ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#8 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 92f95024afafb..e192ab8d637de 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 @@ -380,224 +380,224 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#61] +Output [1]: [i_item_sk#16] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#61] +Right keys [1]: [i_item_sk#16] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#61] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#16] (60) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#62] +Output [1]: [p_promo_sk#19] (61) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#62] +Right keys [1]: [p_promo_sk#19] Join condition: None (62) Project [codegen id : 19] Output [6]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#62] +Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#19] (63) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#63] +Output [1]: [d_date_sk#22] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#63] +Right keys [1]: [d_date_sk#22] Join condition: None (65) Project [codegen id : 19] Output [5]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#63] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#22] (66) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 18] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] (68) Filter [codegen id : 18] -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Condition : isnotnull(cp_catalog_page_sk#64) +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Condition : isnotnull(cp_catalog_page_sk#61) (69) BroadcastExchange -Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#64] +Right keys [1]: [cp_catalog_page_sk#61] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#64, cp_catalog_page_id#65] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] -Keys [1]: [cp_catalog_page_id#65] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Keys [1]: [cp_catalog_page_id#62] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Results [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] (73) Exchange -Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] -Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] -Keys [1]: [cp_catalog_page_id#65] +Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Keys [1]: [cp_catalog_page_id#62] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] -Results [5]: [catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#65) AS id#82, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#83, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#84, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#85] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] +Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_page_id#62) AS id#79, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#80, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#81, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#82] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) (78) Exchange -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] (84) Exchange -Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] +Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] -Right keys [2]: [wr_item_sk#94, wr_order_number#95] +Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] +Right keys [2]: [wr_item_sk#91, wr_order_number#92] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] -Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] +Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#100] +Output [1]: [i_item_sk#16] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#86] -Right keys [1]: [i_item_sk#100] +Left keys [1]: [ws_item_sk#83] +Right keys [1]: [i_item_sk#16] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] -Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, i_item_sk#100] +Output [7]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] +Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, i_item_sk#16] (91) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#101] +Output [1]: [p_promo_sk#19] (92) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#88] -Right keys [1]: [p_promo_sk#101] +Left keys [1]: [ws_promo_sk#85] +Right keys [1]: [p_promo_sk#19] Join condition: None (93) Project [codegen id : 29] -Output [6]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] -Input [8]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, p_promo_sk#101] +Output [6]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] +Input [8]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, p_promo_sk#19] (94) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#102] +Output [1]: [d_date_sk#22] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#92] -Right keys [1]: [d_date_sk#102] +Left keys [1]: [ws_sold_date_sk#89] +Right keys [1]: [d_date_sk#22] Join condition: None (96) Project [codegen id : 29] -Output [5]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] -Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#102] +Output [5]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] +Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#22] (97) Scan parquet default.web_site -Output [2]: [web_site_sk#103, web_site_id#104] +Output [2]: [web_site_sk#97, web_site_id#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (98) ColumnarToRow [codegen id : 28] -Input [2]: [web_site_sk#103, web_site_id#104] +Input [2]: [web_site_sk#97, web_site_id#98] (99) Filter [codegen id : 28] -Input [2]: [web_site_sk#103, web_site_id#104] -Condition : isnotnull(web_site_sk#103) +Input [2]: [web_site_sk#97, web_site_id#98] +Condition : isnotnull(web_site_sk#97) (100) BroadcastExchange -Input [2]: [web_site_sk#103, web_site_id#104] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#105] +Input [2]: [web_site_sk#97, web_site_id#98] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#87] -Right keys [1]: [web_site_sk#103] +Left keys [1]: [ws_web_site_sk#84] +Right keys [1]: [web_site_sk#97] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] -Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#103, web_site_id#104] +Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] -Keys [1]: [web_site_id#104] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Results [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Keys [1]: [web_site_id#98] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] (104) Exchange -Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, [id=#116] +Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Keys [1]: [web_site_id#104] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] -Results [5]: [web channel AS channel#120, concat(web_site, web_site_id#104) AS id#121, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#122, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#123, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#124] +Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Keys [1]: [web_site_id#98] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] +Results [5]: [web channel AS channel#114, concat(web_site, web_site_id#98) AS id#115, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#116, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#117, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#118] (106) Union @@ -605,99 +605,99 @@ Results [5]: [web channel AS channel#120, concat(web_site, web_site_id#104) AS i Input [5]: [channel#42, id#43, sales#44, returns#45, profit#46] Keys [2]: [channel#42, id#43] Functions [3]: [partial_sum(sales#44), partial_sum(returns#45), partial_sum(profit#46)] -Aggregate Attributes [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Results [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] +Results [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] (108) Exchange -Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] -Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#131] (109) HashAggregate [codegen id : 32] -Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#138, sum(returns#45)#139, sum(profit#46)#140] -Results [5]: [channel#42, id#43, cast(sum(sales#44)#138 as decimal(37,2)) AS sales#141, cast(sum(returns#45)#139 as decimal(38,2)) AS returns#142, cast(sum(profit#46)#140 as decimal(38,2)) AS profit#143] +Aggregate Attributes [3]: [sum(sales#44)#132, sum(returns#45)#133, sum(profit#46)#134] +Results [5]: [channel#42, id#43, cast(sum(sales#44)#132 as decimal(37,2)) AS sales#135, cast(sum(returns#45)#133 as decimal(38,2)) AS returns#136, cast(sum(profit#46)#134 as decimal(38,2)) AS profit#137] (110) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] +Output [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] (111) HashAggregate [codegen id : 64] -Input [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] +Input [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#150, sum(returns#45)#151, sum(profit#46)#152] -Results [4]: [channel#42, sum(sales#44)#150 AS sales#153, sum(returns#45)#151 AS returns#154, sum(profit#46)#152 AS profit#155] +Aggregate Attributes [3]: [sum(sales#44)#144, sum(returns#45)#145, sum(profit#46)#146] +Results [4]: [channel#42, sum(sales#44)#144 AS sales#147, sum(returns#45)#145 AS returns#148, sum(profit#46)#146 AS profit#149] (112) HashAggregate [codegen id : 64] -Input [4]: [channel#42, sales#153, returns#154, profit#155] +Input [4]: [channel#42, sales#147, returns#148, profit#149] Keys [1]: [channel#42] -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#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] +Aggregate Attributes [6]: [sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] +Results [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] (113) Exchange -Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] -Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#168] +Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#162] (114) HashAggregate [codegen id : 65] -Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] Keys [1]: [channel#42] -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#42, null AS id#172, sum(sales#153)#169 AS sales#173, sum(returns#154)#170 AS returns#174, sum(profit#155)#171 AS profit#175] +Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] +Aggregate Attributes [3]: [sum(sales#147)#163, sum(returns#148)#164, sum(profit#149)#165] +Results [5]: [channel#42, null AS id#166, sum(sales#147)#163 AS sales#167, sum(returns#148)#164 AS returns#168, sum(profit#149)#165 AS profit#169] (115) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Output [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] (116) HashAggregate [codegen id : 97] -Input [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Input [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#182, sum(returns#45)#183, sum(profit#46)#184] -Results [3]: [sum(sales#44)#182 AS sales#153, sum(returns#45)#183 AS returns#154, sum(profit#46)#184 AS profit#155] +Aggregate Attributes [3]: [sum(sales#44)#176, sum(returns#45)#177, sum(profit#46)#178] +Results [3]: [sum(sales#44)#176 AS sales#147, sum(returns#45)#177 AS returns#148, sum(profit#46)#178 AS profit#149] (117) HashAggregate [codegen id : 97] -Input [3]: [sales#153, returns#154, profit#155] +Input [3]: [sales#147, returns#148, profit#149] Keys: [] -Functions [3]: [partial_sum(sales#153), partial_sum(returns#154), partial_sum(profit#155)] -Aggregate Attributes [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] -Results [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] +Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] +Aggregate Attributes [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] +Results [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] (118) Exchange -Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#197] +Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#191] (119) HashAggregate [codegen id : 98] -Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] +Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] Keys: [] -Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] -Aggregate Attributes [3]: [sum(sales#153)#198, sum(returns#154)#199, sum(profit#155)#200] -Results [5]: [null AS channel#201, null AS id#202, sum(sales#153)#198 AS sales#203, sum(returns#154)#199 AS returns#204, sum(profit#155)#200 AS profit#205] +Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] +Aggregate Attributes [3]: [sum(sales#147)#192, sum(returns#148)#193, sum(profit#149)#194] +Results [5]: [null AS channel#195, null AS id#196, sum(sales#147)#192 AS sales#197, sum(returns#148)#193 AS returns#198, sum(profit#149)#194 AS profit#199] (120) Union (121) HashAggregate [codegen id : 99] -Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] -Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] (122) Exchange -Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] -Arguments: hashpartitioning(channel#42, id#43, sales#141, returns#142, profit#143, 5), ENSURE_REQUIREMENTS, [id=#206] +Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Arguments: hashpartitioning(channel#42, id#43, sales#135, returns#136, profit#137, 5), ENSURE_REQUIREMENTS, [id=#200] (123) HashAggregate [codegen id : 100] -Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] -Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] (124) TakeOrderedAndProject -Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] -Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#141, returns#142, profit#143] +Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#135, returns#136, profit#137] ===== Subqueries ===== @@ -710,6 +710,6 @@ Output [1]: [d_date_sk#22] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 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 cb36b5081db35..e027ef7e53e8a 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 @@ -380,224 +380,224 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#61] +Output [1]: [d_date_sk#16] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#61] +Right keys [1]: [d_date_sk#16] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#61] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#16] (60) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] (62) Filter [codegen id : 16] -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Condition : isnotnull(cp_catalog_page_sk#61) (63) BroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] +Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#62] +Right keys [1]: [cp_catalog_page_sk#61] Join condition: None (65) Project [codegen id : 19] -Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#62, cp_catalog_page_id#63] +Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] (66) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#65] +Output [1]: [i_item_sk#22] (67) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#65] +Right keys [1]: [i_item_sk#22] Join condition: None (68) Project [codegen id : 19] -Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] -Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, i_item_sk#65] +Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, i_item_sk#22] (69) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#66] +Output [1]: [p_promo_sk#25] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#66] +Right keys [1]: [p_promo_sk#25] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] -Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, p_promo_sk#66] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, p_promo_sk#25] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] -Keys [1]: [cp_catalog_page_id#63] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] +Keys [1]: [cp_catalog_page_id#62] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] -Results [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] (73) Exchange -Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] -Arguments: hashpartitioning(cp_catalog_page_id#63, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] -Keys [1]: [cp_catalog_page_id#63] +Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Keys [1]: [cp_catalog_page_id#62] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] -Results [5]: [catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#63) AS id#82, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#83, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#84, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#85] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] +Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_page_id#62) AS id#79, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#80, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#81, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#82] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) (78) Exchange -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] -Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] -Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] (84) Exchange -Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] +Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] -Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] -Right keys [2]: [wr_item_sk#94, wr_order_number#95] +Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] +Right keys [2]: [wr_item_sk#91, wr_order_number#92] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] -Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] +Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#100] +Output [1]: [d_date_sk#16] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#92] -Right keys [1]: [d_date_sk#100] +Left keys [1]: [ws_sold_date_sk#89] +Right keys [1]: [d_date_sk#16] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] -Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#100] +Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] +Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#16] (91) Scan parquet default.web_site -Output [2]: [web_site_sk#101, web_site_id#102] +Output [2]: [web_site_sk#97, web_site_id#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 26] -Input [2]: [web_site_sk#101, web_site_id#102] +Input [2]: [web_site_sk#97, web_site_id#98] (93) Filter [codegen id : 26] -Input [2]: [web_site_sk#101, web_site_id#102] -Condition : isnotnull(web_site_sk#101) +Input [2]: [web_site_sk#97, web_site_id#98] +Condition : isnotnull(web_site_sk#97) (94) BroadcastExchange -Input [2]: [web_site_sk#101, web_site_id#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] +Input [2]: [web_site_sk#97, web_site_id#98] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#87] -Right keys [1]: [web_site_sk#101] +Left keys [1]: [ws_web_site_sk#84] +Right keys [1]: [web_site_sk#97] Join condition: None (96) Project [codegen id : 29] -Output [7]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] -Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#101, web_site_id#102] +Output [7]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] (97) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#104] +Output [1]: [i_item_sk#22] (98) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#86] -Right keys [1]: [i_item_sk#104] +Left keys [1]: [ws_item_sk#83] +Right keys [1]: [i_item_sk#22] Join condition: None (99) Project [codegen id : 29] -Output [6]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] -Input [8]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, i_item_sk#104] +Output [6]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Input [8]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, i_item_sk#22] (100) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#105] +Output [1]: [p_promo_sk#25] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#88] -Right keys [1]: [p_promo_sk#105] +Left keys [1]: [ws_promo_sk#85] +Right keys [1]: [p_promo_sk#25] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] -Input [7]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, p_promo_sk#105] +Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Input [7]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, p_promo_sk#25] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] -Keys [1]: [web_site_id#102] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] -Results [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] +Keys [1]: [web_site_id#98] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] (104) Exchange -Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Arguments: hashpartitioning(web_site_id#102, 5), ENSURE_REQUIREMENTS, [id=#116] +Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] -Keys [1]: [web_site_id#102] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] -Results [5]: [web channel AS channel#120, concat(web_site, web_site_id#102) AS id#121, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#122, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#123, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#124] +Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Keys [1]: [web_site_id#98] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] +Results [5]: [web channel AS channel#114, concat(web_site, web_site_id#98) AS id#115, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#116, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#117, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#118] (106) Union @@ -605,99 +605,99 @@ Results [5]: [web channel AS channel#120, concat(web_site, web_site_id#102) AS i Input [5]: [channel#42, id#43, sales#44, returns#45, profit#46] Keys [2]: [channel#42, id#43] Functions [3]: [partial_sum(sales#44), partial_sum(returns#45), partial_sum(profit#46)] -Aggregate Attributes [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Results [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] +Results [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] (108) Exchange -Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] -Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#131] (109) HashAggregate [codegen id : 32] -Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#138, sum(returns#45)#139, sum(profit#46)#140] -Results [5]: [channel#42, id#43, cast(sum(sales#44)#138 as decimal(37,2)) AS sales#141, cast(sum(returns#45)#139 as decimal(38,2)) AS returns#142, cast(sum(profit#46)#140 as decimal(38,2)) AS profit#143] +Aggregate Attributes [3]: [sum(sales#44)#132, sum(returns#45)#133, sum(profit#46)#134] +Results [5]: [channel#42, id#43, cast(sum(sales#44)#132 as decimal(37,2)) AS sales#135, cast(sum(returns#45)#133 as decimal(38,2)) AS returns#136, cast(sum(profit#46)#134 as decimal(38,2)) AS profit#137] (110) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] +Output [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] (111) HashAggregate [codegen id : 64] -Input [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] +Input [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#150, sum(returns#45)#151, sum(profit#46)#152] -Results [4]: [channel#42, sum(sales#44)#150 AS sales#153, sum(returns#45)#151 AS returns#154, sum(profit#46)#152 AS profit#155] +Aggregate Attributes [3]: [sum(sales#44)#144, sum(returns#45)#145, sum(profit#46)#146] +Results [4]: [channel#42, sum(sales#44)#144 AS sales#147, sum(returns#45)#145 AS returns#148, sum(profit#46)#146 AS profit#149] (112) HashAggregate [codegen id : 64] -Input [4]: [channel#42, sales#153, returns#154, profit#155] +Input [4]: [channel#42, sales#147, returns#148, profit#149] Keys [1]: [channel#42] -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#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] +Aggregate Attributes [6]: [sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] +Results [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] (113) Exchange -Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] -Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#168] +Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#162] (114) HashAggregate [codegen id : 65] -Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] Keys [1]: [channel#42] -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#42, null AS id#172, sum(sales#153)#169 AS sales#173, sum(returns#154)#170 AS returns#174, sum(profit#155)#171 AS profit#175] +Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] +Aggregate Attributes [3]: [sum(sales#147)#163, sum(returns#148)#164, sum(profit#149)#165] +Results [5]: [channel#42, null AS id#166, sum(sales#147)#163 AS sales#167, sum(returns#148)#164 AS returns#168, sum(profit#149)#165 AS profit#169] (115) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Output [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] (116) HashAggregate [codegen id : 97] -Input [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Input [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#182, sum(returns#45)#183, sum(profit#46)#184] -Results [3]: [sum(sales#44)#182 AS sales#153, sum(returns#45)#183 AS returns#154, sum(profit#46)#184 AS profit#155] +Aggregate Attributes [3]: [sum(sales#44)#176, sum(returns#45)#177, sum(profit#46)#178] +Results [3]: [sum(sales#44)#176 AS sales#147, sum(returns#45)#177 AS returns#148, sum(profit#46)#178 AS profit#149] (117) HashAggregate [codegen id : 97] -Input [3]: [sales#153, returns#154, profit#155] +Input [3]: [sales#147, returns#148, profit#149] Keys: [] -Functions [3]: [partial_sum(sales#153), partial_sum(returns#154), partial_sum(profit#155)] -Aggregate Attributes [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] -Results [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] +Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] +Aggregate Attributes [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] +Results [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] (118) Exchange -Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#197] +Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#191] (119) HashAggregate [codegen id : 98] -Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] +Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] Keys: [] -Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] -Aggregate Attributes [3]: [sum(sales#153)#198, sum(returns#154)#199, sum(profit#155)#200] -Results [5]: [null AS channel#201, null AS id#202, sum(sales#153)#198 AS sales#203, sum(returns#154)#199 AS returns#204, sum(profit#155)#200 AS profit#205] +Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] +Aggregate Attributes [3]: [sum(sales#147)#192, sum(returns#148)#193, sum(profit#149)#194] +Results [5]: [null AS channel#195, null AS id#196, sum(sales#147)#192 AS sales#197, sum(returns#148)#193 AS returns#198, sum(profit#149)#194 AS profit#199] (120) Union (121) HashAggregate [codegen id : 99] -Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] -Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] (122) Exchange -Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] -Arguments: hashpartitioning(channel#42, id#43, sales#141, returns#142, profit#143, 5), ENSURE_REQUIREMENTS, [id=#206] +Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Arguments: hashpartitioning(channel#42, id#43, sales#135, returns#136, profit#137, 5), ENSURE_REQUIREMENTS, [id=#200] (123) HashAggregate [codegen id : 100] -Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] -Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] (124) TakeOrderedAndProject -Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] -Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#141, returns#142, profit#143] +Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#135, returns#136, profit#137] ===== Subqueries ===== @@ -710,6 +710,6 @@ Output [1]: [d_date_sk#16] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 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 64f6270e81183..3333d5d1aed52 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 @@ -131,60 +131,60 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#15] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#15,17,2) as decimal(27,2)) AS total_sum#16, i_category#10, i_class#9, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] (20) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#20, i_class#21, sum#22] +Output [3]: [i_category#10, i_class#9, sum#20] (21) HashAggregate [codegen id : 8] -Input [3]: [i_category#20, i_class#21, sum#22] -Keys [2]: [i_category#20, i_class#21] -Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#23))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#23))#24,17,2) AS total_sum#25, i_category#20] +Input [3]: [i_category#10, i_class#9, sum#20] +Keys [2]: [i_category#10, i_class#9] +Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#21] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#21,17,2) AS total_sum#22, i_category#10] (22) HashAggregate [codegen id : 8] -Input [2]: [total_sum#25, i_category#20] -Keys [1]: [i_category#20] -Functions [1]: [partial_sum(total_sum#25)] -Aggregate Attributes [2]: [sum#26, isEmpty#27] -Results [3]: [i_category#20, sum#28, isEmpty#29] +Input [2]: [total_sum#22, i_category#10] +Keys [1]: [i_category#10] +Functions [1]: [partial_sum(total_sum#22)] +Aggregate Attributes [2]: [sum#23, isEmpty#24] +Results [3]: [i_category#10, sum#25, isEmpty#26] (23) Exchange -Input [3]: [i_category#20, sum#28, isEmpty#29] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [i_category#10, sum#25, isEmpty#26] +Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, [id=#27] (24) HashAggregate [codegen id : 9] -Input [3]: [i_category#20, sum#28, isEmpty#29] -Keys [1]: [i_category#20] -Functions [1]: [sum(total_sum#25)] -Aggregate Attributes [1]: [sum(total_sum#25)#31] -Results [6]: [sum(total_sum#25)#31 AS total_sum#32, i_category#20, null AS i_class#33, 0 AS g_category#34, 1 AS g_class#35, 1 AS lochierarchy#36] +Input [3]: [i_category#10, sum#25, isEmpty#26] +Keys [1]: [i_category#10] +Functions [1]: [sum(total_sum#22)] +Aggregate Attributes [1]: [sum(total_sum#22)#28] +Results [6]: [sum(total_sum#22)#28 AS total_sum#29, i_category#10, null AS i_class#30, 0 AS g_category#31, 1 AS g_class#32, 1 AS lochierarchy#33] (25) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#37, i_class#38, sum#39] +Output [3]: [i_category#10, i_class#9, sum#34] (26) HashAggregate [codegen id : 13] -Input [3]: [i_category#37, i_class#38, sum#39] -Keys [2]: [i_category#37, i_class#38] -Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#40))#41] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#40))#41,17,2) AS total_sum#25] +Input [3]: [i_category#10, i_class#9, sum#34] +Keys [2]: [i_category#10, i_class#9] +Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#35] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#35,17,2) AS total_sum#22] (27) HashAggregate [codegen id : 13] -Input [1]: [total_sum#25] +Input [1]: [total_sum#22] Keys: [] -Functions [1]: [partial_sum(total_sum#25)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [2]: [sum#44, isEmpty#45] +Functions [1]: [partial_sum(total_sum#22)] +Aggregate Attributes [2]: [sum#36, isEmpty#37] +Results [2]: [sum#38, isEmpty#39] (28) Exchange -Input [2]: [sum#44, isEmpty#45] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#46] +Input [2]: [sum#38, isEmpty#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] (29) HashAggregate [codegen id : 14] -Input [2]: [sum#44, isEmpty#45] +Input [2]: [sum#38, isEmpty#39] Keys: [] -Functions [1]: [sum(total_sum#25)] -Aggregate Attributes [1]: [sum(total_sum#25)#47] -Results [6]: [sum(total_sum#25)#47 AS total_sum#48, null AS i_category#49, null AS i_class#50, 1 AS g_category#51, 1 AS g_class#52, 2 AS lochierarchy#53] +Functions [1]: [sum(total_sum#22)] +Aggregate Attributes [1]: [sum(total_sum#22)#41] +Results [6]: [sum(total_sum#22)#41 AS total_sum#42, null AS i_category#43, null AS i_class#44, 1 AS g_category#45, 1 AS g_class#46, 2 AS lochierarchy#47] (30) Union @@ -197,34 +197,34 @@ Results [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, (32) Exchange Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] -Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#54] +Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#48] (33) HashAggregate [codegen id : 16] Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Keys [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#55] +Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#49] (34) Exchange -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] -Arguments: hashpartitioning(lochierarchy#19, _w0#55, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] +Arguments: hashpartitioning(lochierarchy#19, _w0#49, 5), ENSURE_REQUIREMENTS, [id=#50] (35) Sort [codegen id : 17] -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] -Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] +Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#49 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 (36) Window -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] -Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#55, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#57], [lochierarchy#19, _w0#55], [total_sum#16 DESC NULLS LAST] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] +Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#49, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#51], [lochierarchy#19, _w0#49], [total_sum#16 DESC NULLS LAST] (37) Project [codegen id : 18] -Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] -Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55, rank_within_parent#57] +Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] +Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49, rank_within_parent#51] (38) TakeOrderedAndProject -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#57 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] +Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#51 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] ===== Subqueries ===== 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 64f6270e81183..3333d5d1aed52 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 @@ -131,60 +131,60 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#15] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#15,17,2) as decimal(27,2)) AS total_sum#16, i_category#10, i_class#9, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] (20) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#20, i_class#21, sum#22] +Output [3]: [i_category#10, i_class#9, sum#20] (21) HashAggregate [codegen id : 8] -Input [3]: [i_category#20, i_class#21, sum#22] -Keys [2]: [i_category#20, i_class#21] -Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#23))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#23))#24,17,2) AS total_sum#25, i_category#20] +Input [3]: [i_category#10, i_class#9, sum#20] +Keys [2]: [i_category#10, i_class#9] +Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#21] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#21,17,2) AS total_sum#22, i_category#10] (22) HashAggregate [codegen id : 8] -Input [2]: [total_sum#25, i_category#20] -Keys [1]: [i_category#20] -Functions [1]: [partial_sum(total_sum#25)] -Aggregate Attributes [2]: [sum#26, isEmpty#27] -Results [3]: [i_category#20, sum#28, isEmpty#29] +Input [2]: [total_sum#22, i_category#10] +Keys [1]: [i_category#10] +Functions [1]: [partial_sum(total_sum#22)] +Aggregate Attributes [2]: [sum#23, isEmpty#24] +Results [3]: [i_category#10, sum#25, isEmpty#26] (23) Exchange -Input [3]: [i_category#20, sum#28, isEmpty#29] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [i_category#10, sum#25, isEmpty#26] +Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, [id=#27] (24) HashAggregate [codegen id : 9] -Input [3]: [i_category#20, sum#28, isEmpty#29] -Keys [1]: [i_category#20] -Functions [1]: [sum(total_sum#25)] -Aggregate Attributes [1]: [sum(total_sum#25)#31] -Results [6]: [sum(total_sum#25)#31 AS total_sum#32, i_category#20, null AS i_class#33, 0 AS g_category#34, 1 AS g_class#35, 1 AS lochierarchy#36] +Input [3]: [i_category#10, sum#25, isEmpty#26] +Keys [1]: [i_category#10] +Functions [1]: [sum(total_sum#22)] +Aggregate Attributes [1]: [sum(total_sum#22)#28] +Results [6]: [sum(total_sum#22)#28 AS total_sum#29, i_category#10, null AS i_class#30, 0 AS g_category#31, 1 AS g_class#32, 1 AS lochierarchy#33] (25) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#37, i_class#38, sum#39] +Output [3]: [i_category#10, i_class#9, sum#34] (26) HashAggregate [codegen id : 13] -Input [3]: [i_category#37, i_class#38, sum#39] -Keys [2]: [i_category#37, i_class#38] -Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#40))#41] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#40))#41,17,2) AS total_sum#25] +Input [3]: [i_category#10, i_class#9, sum#34] +Keys [2]: [i_category#10, i_class#9] +Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#35] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#35,17,2) AS total_sum#22] (27) HashAggregate [codegen id : 13] -Input [1]: [total_sum#25] +Input [1]: [total_sum#22] Keys: [] -Functions [1]: [partial_sum(total_sum#25)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [2]: [sum#44, isEmpty#45] +Functions [1]: [partial_sum(total_sum#22)] +Aggregate Attributes [2]: [sum#36, isEmpty#37] +Results [2]: [sum#38, isEmpty#39] (28) Exchange -Input [2]: [sum#44, isEmpty#45] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#46] +Input [2]: [sum#38, isEmpty#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] (29) HashAggregate [codegen id : 14] -Input [2]: [sum#44, isEmpty#45] +Input [2]: [sum#38, isEmpty#39] Keys: [] -Functions [1]: [sum(total_sum#25)] -Aggregate Attributes [1]: [sum(total_sum#25)#47] -Results [6]: [sum(total_sum#25)#47 AS total_sum#48, null AS i_category#49, null AS i_class#50, 1 AS g_category#51, 1 AS g_class#52, 2 AS lochierarchy#53] +Functions [1]: [sum(total_sum#22)] +Aggregate Attributes [1]: [sum(total_sum#22)#41] +Results [6]: [sum(total_sum#22)#41 AS total_sum#42, null AS i_category#43, null AS i_class#44, 1 AS g_category#45, 1 AS g_class#46, 2 AS lochierarchy#47] (30) Union @@ -197,34 +197,34 @@ Results [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, (32) Exchange Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] -Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#54] +Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#48] (33) HashAggregate [codegen id : 16] Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Keys [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#55] +Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#49] (34) Exchange -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] -Arguments: hashpartitioning(lochierarchy#19, _w0#55, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] +Arguments: hashpartitioning(lochierarchy#19, _w0#49, 5), ENSURE_REQUIREMENTS, [id=#50] (35) Sort [codegen id : 17] -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] -Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] +Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#49 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 (36) Window -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] -Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#55, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#57], [lochierarchy#19, _w0#55], [total_sum#16 DESC NULLS LAST] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] +Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#49, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#51], [lochierarchy#19, _w0#49], [total_sum#16 DESC NULLS LAST] (37) Project [codegen id : 18] -Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] -Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55, rank_within_parent#57] +Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] +Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49, rank_within_parent#51] (38) TakeOrderedAndProject -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#57 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] +Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#51 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] ===== Subqueries ===== From f0a2754f1ff2d6d303ee1b84e69862082e6997bf Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 25 Mar 2021 15:37:36 +0800 Subject: [PATCH 29/38] revert --- .../q14a.sf100/explain.txt | 844 +++++----- .../q14a.sf100/simplified.txt | 158 +- .../approved-plans-v1_4/q14a/explain.txt | 678 ++++---- .../approved-plans-v1_4/q14a/simplified.txt | 80 +- .../q14b.sf100/explain.txt | 762 ++++----- .../q14b.sf100/simplified.txt | 146 +- .../approved-plans-v1_4/q14b/explain.txt | 630 +++---- .../approved-plans-v1_4/q14b/simplified.txt | 80 +- .../approved-plans-v1_4/q16.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q16/explain.txt | 2 +- .../approved-plans-v1_4/q17.sf100/explain.txt | 8 +- .../approved-plans-v1_4/q17/explain.txt | 8 +- .../approved-plans-v1_4/q32.sf100/explain.txt | 42 +- .../q32.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q32/explain.txt | 89 +- .../approved-plans-v1_4/q32/simplified.txt | 2 +- .../approved-plans-v1_4/q33.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q33/explain.txt | 4 +- .../q39a.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q39a/explain.txt | 4 +- .../q39b.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q39b/explain.txt | 4 +- .../approved-plans-v1_4/q41.sf100/explain.txt | 51 +- .../q41.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q41/explain.txt | 51 +- .../approved-plans-v1_4/q41/simplified.txt | 2 +- .../approved-plans-v1_4/q44.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q44/explain.txt | 4 +- .../approved-plans-v1_4/q58.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q58/explain.txt | 2 +- .../approved-plans-v1_4/q6.sf100/explain.txt | 186 +-- .../q6.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q6/explain.txt | 100 +- .../approved-plans-v1_4/q6/simplified.txt | 2 +- .../approved-plans-v1_4/q70.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q70/explain.txt | 2 +- .../approved-plans-v1_4/q71.sf100/explain.txt | 4 +- .../approved-plans-v1_4/q71/explain.txt | 4 +- .../approved-plans-v1_4/q92.sf100/explain.txt | 64 +- .../q92.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q92/explain.txt | 103 +- .../approved-plans-v1_4/q92/simplified.txt | 2 +- .../approved-plans-v1_4/q94.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q94/explain.txt | 6 +- .../approved-plans-v1_4/q95.sf100/explain.txt | 6 +- .../approved-plans-v1_4/q95/explain.txt | 6 +- .../approved-plans-v2_7/q14.sf100/explain.txt | 762 ++++----- .../q14.sf100/simplified.txt | 146 +- .../approved-plans-v2_7/q14/explain.txt | 630 +++---- .../approved-plans-v2_7/q14/simplified.txt | 80 +- .../q14a.sf100/explain.txt | 1446 ++++++++--------- .../q14a.sf100/simplified.txt | 210 +-- .../approved-plans-v2_7/q14a/explain.txt | 1158 ++++++------- .../approved-plans-v2_7/q14a/simplified.txt | 80 +- .../approved-plans-v2_7/q6.sf100/explain.txt | 186 +-- .../q6.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q6/explain.txt | 100 +- .../approved-plans-v2_7/q6/simplified.txt | 2 +- .../q67a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q67a/explain.txt | 2 +- .../q70a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q70a/explain.txt | 2 +- 62 files changed, 4486 insertions(+), 4498 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt index e4ec487623d2c..d4ab3f60494d9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt @@ -37,39 +37,39 @@ TakeOrderedAndProject (134) : : : : +- * HashAggregate (44) : : : : +- Exchange (43) : : : : +- * HashAggregate (42) - : : : : +- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * Project (18) - : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : :- * Filter (11) - : : : : : : +- * ColumnarToRow (10) - : : : : : : +- Scan parquet default.store_sales (9) - : : : : : +- BroadcastExchange (16) - : : : : : +- * Project (15) - : : : : : +- * Filter (14) - : : : : : +- * ColumnarToRow (13) - : : : : : +- Scan parquet default.date_dim (12) - : : : : +- BroadcastExchange (39) - : : : : +- SortMergeJoin LeftSemi (38) - : : : : :- * Sort (23) - : : : : : +- Exchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (37) - : : : : +- Exchange (36) - : : : : +- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (26) - : : : : : : +- * ColumnarToRow (25) - : : : : : : +- Scan parquet default.catalog_sales (24) - : : : : : +- ReusedExchange (27) - : : : : +- BroadcastExchange (33) - : : : : +- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet default.item (30) + : : : : +- SortMergeJoin LeftSemi (41) + : : : : :- * Sort (26) + : : : : : +- Exchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (40) + : : : : +- Exchange (39) + : : : : +- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.item (33) : : : +- * Sort (57) : : : +- Exchange (56) : : : +- * Project (55) @@ -165,588 +165,588 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 19] +(7) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 19] +(8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 10] -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +(10) ColumnarToRow [codegen id : 5] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -(11) Filter [codegen id : 10] -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_item_sk#11) +(11) Filter [codegen id : 5] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#12, d_year#13] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#14, d_year#15] +Input [2]: [d_date_sk#12, d_year#13] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#14, d_year#15] -Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1999)) AND (d_year#15 <= 2001)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_year#15] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (16) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(17) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#14] +(17) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None -(18) Project [codegen id : 10] -Output [1]: [ss_item_sk#11] -Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] +(18) Project [codegen id : 5] +Output [1]: [ss_item_sk#1] +Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] (19) Scan parquet default.item -Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) -(22) Exchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] +(22) BroadcastExchange +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] -(23) Sort [codegen id : 5] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 +(23) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#7] +Join condition: None -(24) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] +(24) Project [codegen id : 5] +Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] +Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] + +(25) Exchange +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] + +(26) Sort [codegen id : 6] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 + +(27) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 8] -Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +(28) ColumnarToRow [codegen id : 9] +Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -(26) Filter [codegen id : 8] -Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] -Condition : isnotnull(cs_item_sk#22) +(29) Filter [codegen id : 9] +Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +Condition : isnotnull(cs_item_sk#20) -(27) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#24] +(30) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#12] -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] +(31) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#12] Join condition: None -(29) Project [codegen id : 8] -Output [1]: [cs_item_sk#22] -Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] +(32) Project [codegen id : 9] +Output [1]: [cs_item_sk#20] +Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] -(30) Scan parquet default.item -Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +(33) Scan parquet default.item +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 7] -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(32) Filter [codegen id : 7] -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -Condition : isnotnull(i_item_sk#25) +(35) Filter [codegen id : 8] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : isnotnull(i_item_sk#7) -(33) BroadcastExchange -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +(36) BroadcastExchange +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#22] -Right keys [1]: [i_item_sk#25] +(37) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [cs_item_sk#20] +Right keys [1]: [i_item_sk#7] Join condition: None -(35) Project [codegen id : 8] -Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +(38) Project [codegen id : 9] +Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(36) Exchange -Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] +(39) Exchange +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] -(37) Sort [codegen id : 9] -Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 +(40) Sort [codegen id : 10] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 -(38) SortMergeJoin -Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] -Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] +(41) SortMergeJoin +Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] +Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] Join condition: None -(39) BroadcastExchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] - -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#11] -Right keys [1]: [i_item_sk#17] -Join condition: None - -(41) Project [codegen id : 10] -Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] -Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] - -(42) HashAggregate [codegen id : 10] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +(42) HashAggregate [codegen id : 11] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#16, class_id#17, category_id#18] (43) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] -(44) HashAggregate [codegen id : 11] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +(44) HashAggregate [codegen id : 12] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#16, class_id#17, category_id#18] (45) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] -(46) Sort [codegen id : 12] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 13] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 15] -Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +(48) ColumnarToRow [codegen id : 16] +Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -(49) Filter [codegen id : 15] -Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_item_sk#37) +(49) Filter [codegen id : 16] +Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Condition : isnotnull(ws_item_sk#26) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#39] +Output [1]: [d_date_sk#12] -(51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#39] +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None -(52) Project [codegen id : 15] -Output [1]: [ws_item_sk#37] -Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] +(52) Project [codegen id : 16] +Output [1]: [ws_item_sk#26] +Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] -(53) ReusedExchange [Reuses operator id: 33] -Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] +(53) ReusedExchange [Reuses operator id: 36] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(54) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#40] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [i_item_sk#7] Join condition: None -(55) Project [codegen id : 15] -Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] +(55) Project [codegen id : 16] +Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (56) Exchange -Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] -(57) Sort [codegen id : 16] -Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 17] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] -Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] +Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] +Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] Join condition: None -(59) HashAggregate [codegen id : 17] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +(59) HashAggregate [codegen id : 18] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#16, class_id#17, category_id#18] (60) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] -(61) HashAggregate [codegen id : 18] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +(61) HashAggregate [codegen id : 19] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#16, class_id#17, category_id#18] (62) BroadcastExchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] -(63) BroadcastHashJoin [codegen id : 19] +(63) BroadcastHashJoin [codegen id : 20] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#32, class_id#33, category_id#34] +Right keys [3]: [brand_id#16, class_id#17, category_id#18] Join condition: None -(64) Project [codegen id : 19] -Output [1]: [i_item_sk#7 AS ss_item_sk#47] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] +(64) Project [codegen id : 20] +Output [1]: [i_item_sk#7 AS ss_item_sk#31] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] (65) Exchange -Input [1]: [ss_item_sk#47] -Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [1]: [ss_item_sk#31] +Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] -(66) Sort [codegen id : 20] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 21] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#47] +Right keys [1]: [ss_item_sk#31] Join condition: None (68) Scan parquet default.date_dim -Output [3]: [d_date_sk#49, d_year#50, d_moy#51] +Output [3]: [d_date_sk#12, d_year#13, d_moy#33] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 21] -Input [3]: [d_date_sk#49, d_year#50, d_moy#51] +(69) ColumnarToRow [codegen id : 22] +Input [3]: [d_date_sk#12, d_year#13, d_moy#33] -(70) Filter [codegen id : 21] -Input [3]: [d_date_sk#49, d_year#50, d_moy#51] -Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2001)) AND (d_moy#51 = 11)) AND isnotnull(d_date_sk#49)) +(70) Filter [codegen id : 22] +Input [3]: [d_date_sk#12, d_year#13, d_moy#33] +Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#33)) AND (d_year#13 = 2001)) AND (d_moy#33 = 11)) AND isnotnull(d_date_sk#12)) -(71) Project [codegen id : 21] -Output [1]: [d_date_sk#49] -Input [3]: [d_date_sk#49, d_year#50, d_moy#51] +(71) Project [codegen id : 22] +Output [1]: [d_date_sk#12] +Input [3]: [d_date_sk#12, d_year#13, d_moy#33] (72) BroadcastExchange -Input [1]: [d_date_sk#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] -(73) BroadcastHashJoin [codegen id : 42] +(73) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#49] +Right keys [1]: [d_date_sk#12] Join condition: None -(74) Project [codegen id : 42] +(74) Project [codegen id : 44] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (75) Scan parquet default.item -Output [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 22] -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +(76) ColumnarToRow [codegen id : 23] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(77) Filter [codegen id : 22] -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Condition : isnotnull(i_item_sk#53) +(77) Filter [codegen id : 23] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : isnotnull(i_item_sk#7) (78) Exchange -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: hashpartitioning(i_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#35] -(79) Sort [codegen id : 23] -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [i_item_sk#53 ASC NULLS FIRST], false, 0 +(79) Sort [codegen id : 24] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#31] -(81) Sort [codegen id : 41] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 43] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#53] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [i_item_sk#7] +Right keys [1]: [ss_item_sk#31] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] -(84) BroadcastHashJoin [codegen id : 42] +(84) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#53] +Right keys [1]: [i_item_sk#7] Join condition: None -(85) Project [codegen id : 42] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +(85) Project [codegen id : 44] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(86) HashAggregate [codegen id : 42] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] -Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] +(86) HashAggregate [codegen id : 44] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#59, isEmpty#60, count#61] -Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] +Aggregate Attributes [3]: [sum#37, isEmpty#38, count#39] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] (87) Exchange -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] -Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#43] -(88) HashAggregate [codegen id : 43] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] -Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] +(88) HashAggregate [codegen id : 45] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66, count(1)#67] -Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44, count(1)#45] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sales#46, count(1)#45 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] -(89) Filter [codegen id : 43] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(32,6)) > cast(Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(89) Filter [codegen id : 45] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48 as decimal(32,6)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(90) Project [codegen id : 43] -Output [6]: [sales#68, number_sales#69, store AS channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +(90) Project [codegen id : 45] +Output [6]: [sales#46, number_sales#47, store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] (91) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Output [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#77), dynamicpruningexpression(cs_sold_date_sk#77 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 44] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +(92) ColumnarToRow [codegen id : 46] +Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -(93) Filter [codegen id : 44] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Condition : isnotnull(cs_item_sk#74) +(93) Filter [codegen id : 46] +Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Condition : isnotnull(cs_item_sk#20) (94) Exchange -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Arguments: hashpartitioning(cs_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#54] -(95) Sort [codegen id : 45] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Arguments: [cs_item_sk#74 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 47] +Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Arguments: [cs_item_sk#20 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#31] -(97) Sort [codegen id : 63] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 66] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [cs_item_sk#74] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [cs_item_sk#20] +Right keys [1]: [ss_item_sk#31] Join condition: None (99) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#79] +Output [1]: [d_date_sk#12] -(100) BroadcastHashJoin [codegen id : 85] -Left keys [1]: [cs_sold_date_sk#77] -Right keys [1]: [d_date_sk#79] +(100) BroadcastHashJoin [codegen id : 89] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#12] Join condition: None -(101) Project [codegen id : 85] -Output [3]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76] -Input [5]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77, d_date_sk#79] +(101) Project [codegen id : 89] +Output [3]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53] +Input [5]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] (102) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(103) BroadcastHashJoin [codegen id : 85] -Left keys [1]: [cs_item_sk#74] -Right keys [1]: [i_item_sk#80] +(103) BroadcastHashJoin [codegen id : 89] +Left keys [1]: [cs_item_sk#20] +Right keys [1]: [i_item_sk#7] Join condition: None -(104) Project [codegen id : 85] -Output [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] -Input [7]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] +(104) Project [codegen id : 89] +Output [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [7]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(105) HashAggregate [codegen id : 85] -Input [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] -Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +(105) HashAggregate [codegen id : 89] +Input [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] (106) Exchange -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] -Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#61] -(107) HashAggregate [codegen id : 86] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] -Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] +(107) HashAggregate [codegen id : 90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] -(108) Filter [codegen id : 86] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(108) Filter [codegen id : 90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(109) Project [codegen id : 86] -Output [6]: [sales#93, number_sales#94, catalog AS channel#96, i_brand_id#81, i_class_id#82, i_category_id#83] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] +(109) Project [codegen id : 90] +Output [6]: [sales#64, number_sales#65, catalog AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] (110) Scan parquet default.web_sales -Output [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Output [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#100), dynamicpruningexpression(ws_sold_date_sk#100 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 87] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +(111) ColumnarToRow [codegen id : 91] +Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -(112) Filter [codegen id : 87] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Condition : isnotnull(ws_item_sk#97) +(112) Filter [codegen id : 91] +Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Condition : isnotnull(ws_item_sk#26) (113) Exchange -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Arguments: hashpartitioning(ws_item_sk#97, 5), ENSURE_REQUIREMENTS, [id=#101] +Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Arguments: hashpartitioning(ws_item_sk#26, 5), ENSURE_REQUIREMENTS, [id=#70] -(114) Sort [codegen id : 88] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Arguments: [ws_item_sk#97 ASC NULLS FIRST], false, 0 +(114) Sort [codegen id : 92] +Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Arguments: [ws_item_sk#26 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#31] -(116) Sort [codegen id : 106] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +(116) Sort [codegen id : 111] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (117) SortMergeJoin -Left keys [1]: [ws_item_sk#97] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [ss_item_sk#31] Join condition: None (118) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#102] +Output [1]: [d_date_sk#12] -(119) BroadcastHashJoin [codegen id : 128] -Left keys [1]: [ws_sold_date_sk#100] -Right keys [1]: [d_date_sk#102] +(119) BroadcastHashJoin [codegen id : 134] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None -(120) Project [codegen id : 128] -Output [3]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99] -Input [5]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100, d_date_sk#102] +(120) Project [codegen id : 134] +Output [3]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69] +Input [5]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] (121) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(122) BroadcastHashJoin [codegen id : 128] -Left keys [1]: [ws_item_sk#97] -Right keys [1]: [i_item_sk#103] +(122) BroadcastHashJoin [codegen id : 134] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [i_item_sk#7] Join condition: None -(123) Project [codegen id : 128] -Output [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] -Input [7]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] +(123) Project [codegen id : 134] +Output [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [7]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(124) HashAggregate [codegen id : 128] -Input [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] -Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#107, isEmpty#108, count#109] -Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +(124) HashAggregate [codegen id : 134] +Input [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#71, isEmpty#72, count#73] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] (125) Exchange -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] -Arguments: hashpartitioning(i_brand_id#104, i_class_id#105, i_category_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#77] -(126) HashAggregate [codegen id : 129] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] -Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114, count(1)#115] -Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sales#116, count(1)#115 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] +(126) HashAggregate [codegen id : 135] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78, count(1)#79] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sales#80, count(1)#79 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] -(127) Filter [codegen id : 129] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(127) Filter [codegen id : 135] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(128) Project [codegen id : 129] -Output [6]: [sales#116, number_sales#117, web AS channel#119, i_brand_id#104, i_class_id#105, i_category_id#106] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] +(128) Project [codegen id : 135] +Output [6]: [sales#80, number_sales#81, web AS channel#83, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] (129) Union -(130) Expand [codegen id : 130] -Input [6]: [sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [List(sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, 0), List(sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, null, 1), List(sales#68, number_sales#69, channel#73, i_brand_id#54, null, null, 3), List(sales#68, number_sales#69, channel#73, null, null, null, 7), List(sales#68, number_sales#69, null, null, null, null, 15)], [sales#68, number_sales#69, channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] +(130) Expand [codegen id : 136] +Input [6]: [sales#46, number_sales#47, channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [List(sales#46, number_sales#47, channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 0), List(sales#46, number_sales#47, channel#51, i_brand_id#8, i_class_id#9, null, 1), List(sales#46, number_sales#47, channel#51, i_brand_id#8, null, null, 3), List(sales#46, number_sales#47, channel#51, null, null, null, 7), List(sales#46, number_sales#47, null, null, null, null, 15)], [sales#46, number_sales#47, channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] -(131) HashAggregate [codegen id : 130] -Input [7]: [sales#68, number_sales#69, channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] -Keys [5]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#125, isEmpty#126, sum#127] -Results [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] +(131) HashAggregate [codegen id : 136] +Input [7]: [sales#46, number_sales#47, channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] +Keys [5]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] +Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] +Aggregate Attributes [3]: [sum#89, isEmpty#90, sum#91] +Results [8]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, sum#92, isEmpty#93, sum#94] (132) Exchange -Input [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] -Arguments: hashpartitioning(channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, 5), ENSURE_REQUIREMENTS, [id=#131] +Input [8]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, sum#92, isEmpty#93, sum#94] +Arguments: hashpartitioning(channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, 5), ENSURE_REQUIREMENTS, [id=#95] -(133) HashAggregate [codegen id : 131] -Input [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] -Keys [5]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#132, sum(number_sales#69)#133] -Results [6]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales#68)#132 AS sum(sales)#134, sum(number_sales#69)#133 AS sum(number_sales)#135] +(133) HashAggregate [codegen id : 137] +Input [8]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, sum#92, isEmpty#93, sum#94] +Keys [5]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] +Functions [2]: [sum(sales#46), sum(number_sales#47)] +Aggregate Attributes [2]: [sum(sales#46)#96, sum(number_sales#47)#97] +Results [6]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, sum(sales#46)#96 AS sum(sales)#98, sum(number_sales#47)#97 AS sum(number_sales)#99] (134) TakeOrderedAndProject -Input [6]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales)#134, sum(number_sales)#135] -Arguments: 100, [channel#120 ASC NULLS FIRST, i_brand_id#121 ASC NULLS FIRST, i_class_id#122 ASC NULLS FIRST, i_category_id#123 ASC NULLS FIRST], [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales)#134, sum(number_sales)#135] +Input [6]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, sum(sales)#98, sum(number_sales)#99] +Arguments: 100, [channel#84 ASC NULLS FIRST, i_brand_id#85 ASC NULLS FIRST, i_class_id#86 ASC NULLS FIRST, i_category_id#87 ASC NULLS FIRST], [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, sum(sales)#98, sum(number_sales)#99] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#71, [id=#72] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#49, [id=#50] * HashAggregate (157) +- Exchange (156) +- * HashAggregate (155) @@ -773,145 +773,145 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (135) Scan parquet default.store_sales -Output [3]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138] +Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#138), dynamicpruningexpression(ss_sold_date_sk#138 IN dynamicpruning#139)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#100)] ReadSchema: struct (136) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138] +Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (137) Scan parquet default.date_dim -Output [2]: [d_date_sk#140, d_year#141] +Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (138) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#140, d_year#141] +Input [2]: [d_date_sk#12, d_year#13] (139) Filter [codegen id : 1] -Input [2]: [d_date_sk#140, d_year#141] -Condition : (((isnotnull(d_year#141) AND (d_year#141 >= 1999)) AND (d_year#141 <= 2001)) AND isnotnull(d_date_sk#140)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (140) Project [codegen id : 1] -Output [1]: [d_date_sk#140] -Input [2]: [d_date_sk#140, d_year#141] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (141) BroadcastExchange -Input [1]: [d_date_sk#140] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#142] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#101] (142) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#138] -Right keys [1]: [d_date_sk#140] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (143) Project [codegen id : 2] -Output [2]: [ss_quantity#136 AS quantity#143, ss_list_price#137 AS list_price#144] -Input [4]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138, d_date_sk#140] +Output [2]: [ss_quantity#2 AS quantity#102, ss_list_price#3 AS list_price#103] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (144) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147] +Output [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#147), dynamicpruningexpression(cs_sold_date_sk#147 IN dynamicpruning#139)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#100)] ReadSchema: struct (145) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147] +Input [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] (146) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#148] +Output [1]: [d_date_sk#12] (147) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#147] -Right keys [1]: [d_date_sk#148] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#12] Join condition: None (148) Project [codegen id : 4] -Output [2]: [cs_quantity#145 AS quantity#149, cs_list_price#146 AS list_price#150] -Input [4]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147, d_date_sk#148] +Output [2]: [cs_quantity#52 AS quantity#104, cs_list_price#53 AS list_price#105] +Input [4]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] (149) Scan parquet default.web_sales -Output [3]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153] +Output [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#153), dynamicpruningexpression(ws_sold_date_sk#153 IN dynamicpruning#139)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#100)] ReadSchema: struct (150) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153] +Input [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] (151) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#154] +Output [1]: [d_date_sk#12] (152) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#153] -Right keys [1]: [d_date_sk#154] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None (153) Project [codegen id : 6] -Output [2]: [ws_quantity#151 AS quantity#155, ws_list_price#152 AS list_price#156] -Input [4]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153, d_date_sk#154] +Output [2]: [ws_quantity#68 AS quantity#106, ws_list_price#69 AS list_price#107] +Input [4]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] (154) Union (155) HashAggregate [codegen id : 7] -Input [2]: [quantity#143, list_price#144] +Input [2]: [quantity#102, list_price#103] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#157, count#158] -Results [2]: [sum#159, count#160] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#108, count#109] +Results [2]: [sum#110, count#111] (156) Exchange -Input [2]: [sum#159, count#160] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#161] +Input [2]: [sum#110, count#111] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#112] (157) HashAggregate [codegen id : 8] -Input [2]: [sum#159, count#160] +Input [2]: [sum#110, count#111] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))#162] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))#162 AS average_sales#163] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))#113] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))#113 AS average_sales#114] -Subquery:2 Hosting operator id = 135 Hosting Expression = ss_sold_date_sk#138 IN dynamicpruning#139 +Subquery:2 Hosting operator id = 135 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#100 ReusedExchange (158) (158) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#140] +Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 144 Hosting Expression = cs_sold_date_sk#147 IN dynamicpruning#139 +Subquery:3 Hosting operator id = 144 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#100 -Subquery:4 Hosting operator id = 149 Hosting Expression = ws_sold_date_sk#153 IN dynamicpruning#139 +Subquery:4 Hosting operator id = 149 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#100 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (159) (159) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#49] +Output [1]: [d_date_sk#12] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 ReusedExchange (160) (160) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 +Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 -Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#77 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt index 2134091c43b82..4ee1ff31e9beb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (131) + WholeStageCodegen (137) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (130) + WholeStageCodegen (136) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] InputAdapter Union - WholeStageCodegen (43) + WholeStageCodegen (45) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #3 @@ -57,7 +57,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (42) + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -76,11 +76,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - WholeStageCodegen (20) + WholeStageCodegen (21) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #5 - WholeStageCodegen (19) + WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -89,82 +89,84 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 - WholeStageCodegen (18) + WholeStageCodegen (19) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (17) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (12) + WholeStageCodegen (13) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (10) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #10 + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #11 - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (9) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #13 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (7) - Filter [i_item_sk] + Exchange [brand_id,class_id,category_id] #10 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #11 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (16) + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #11 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (8) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (17) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (15) + WholeStageCodegen (16) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -175,12 +177,12 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #10 + ReusedExchange [d_date_sk] #11 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 InputAdapter BroadcastExchange #4 - WholeStageCodegen (21) + WholeStageCodegen (22) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow @@ -189,27 +191,27 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter BroadcastExchange #16 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (23) + WholeStageCodegen (24) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #17 - WholeStageCodegen (22) + WholeStageCodegen (23) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (41) + WholeStageCodegen (43) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 - WholeStageCodegen (86) + WholeStageCodegen (90) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (85) + WholeStageCodegen (89) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -217,17 +219,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [cs_item_sk,ss_item_sk] - WholeStageCodegen (45) + WholeStageCodegen (47) Sort [cs_item_sk] InputAdapter Exchange [cs_item_sk] #22 - WholeStageCodegen (44) + WholeStageCodegen (46) Filter [cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (63) + WholeStageCodegen (66) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 @@ -235,14 +237,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [d_date_sk] #4 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (129) + WholeStageCodegen (135) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #23 - WholeStageCodegen (128) + WholeStageCodegen (134) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] @@ -250,17 +252,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ws_item_sk,ss_item_sk] - WholeStageCodegen (88) + WholeStageCodegen (92) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #24 - WholeStageCodegen (87) + WholeStageCodegen (91) Filter [ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (106) + WholeStageCodegen (111) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 6f61fc8e96ae1..85a3a63413f92 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -31,36 +31,36 @@ TakeOrderedAndProject (115) : : : :- * HashAggregate (39) : : : : +- Exchange (38) : : : : +- * HashAggregate (37) - : : : : +- * Project (36) - : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : :- * Project (33) - : : : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet default.store_sales (7) - : : : : : +- BroadcastExchange (31) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Project (28) - : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet default.item (16) - : : : : : +- BroadcastExchange (26) - : : : : : +- * Project (25) - : : : : : +- * Filter (24) - : : : : : +- * ColumnarToRow (23) - : : : : : +- Scan parquet default.date_dim (22) - : : : : +- ReusedExchange (34) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : :- * Project (22) + : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : :- * Project (15) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (13) + : : : : : : +- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (20) + : : : : : +- * Project (19) + : : : : : +- * Filter (18) + : : : : : +- * ColumnarToRow (17) + : : : : : +- Scan parquet default.date_dim (16) + : : : : +- BroadcastExchange (35) + : : : : +- * Project (34) + : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * Project (31) + : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : :- * Filter (25) + : : : : : : +- * ColumnarToRow (24) + : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Filter (28) + : : : : : +- * ColumnarToRow (27) + : : : : : +- Scan parquet default.item (26) + : : : : +- ReusedExchange (32) : : : +- BroadcastExchange (49) : : : +- * Project (48) : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -146,512 +146,512 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (10) Scan parquet default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +(11) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(12) Filter [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) +(12) Filter [codegen id : 1] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(13) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +(13) BroadcastExchange +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +(14) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join condition: None -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) +(15) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(16) Scan parquet default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +(16) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_year#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +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 + +(17) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#12, d_year#13] -(17) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +(18) Filter [codegen id : 2] +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) -(18) Filter [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) +(19) Project [codegen id : 2] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] -(19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] +(20) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +(22) Project [codegen id : 6] +Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] +Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] -(22) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_year#25] +(23) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] 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 - -(23) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#24, d_year#25] +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(24) Filter [codegen id : 2] -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) +(24) ColumnarToRow [codegen id : 5] +Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -(25) Project [codegen id : 2] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_year#25] +(25) Filter [codegen id : 5] +Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +Condition : isnotnull(cs_item_sk#18) -(26) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +(26) Scan parquet default.item +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(27) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] -Join condition: None +(27) ColumnarToRow [codegen id : 3] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(28) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] +(28) Filter [codegen id : 3] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : isnotnull(i_item_sk#6) (29) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] -(30) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] +(30) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#6] Join condition: None -(31) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] +(31) Project [codegen id : 5] +Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] + +(32) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] -(32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] +(33) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None -(33) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +(34) Project [codegen id : 5] +Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] -(34) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#29] +(35) BroadcastExchange +Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] -(35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#29] +(36) BroadcastHashJoin [codegen id : 6] +Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] +Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None -(36) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] - (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#15, class_id#16, category_id#17] (38) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#15, class_id#16, category_id#17] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_item_sk#34) +Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Condition : isnotnull(ws_item_sk#23) -(43) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] +(43) ReusedExchange [Reuses operator id: 29] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#36] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [i_item_sk#6] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] +Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(46) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#40] +(46) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#40] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] +Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (49) BroadcastExchange -Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] +Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] -Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] +Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] +Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#15, class_id#16, category_id#17] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#15, class_id#16, category_id#17] (53) BroadcastExchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#30, class_id#31, category_id#32] +Right keys [3]: [brand_id#15, class_id#16, category_id#17] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#43] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] +Output [1]: [i_item_sk#6 AS ss_item_sk#27] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] (56) BroadcastExchange -Input [1]: [ss_item_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] +Input [1]: [ss_item_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#43] +Right keys [1]: [ss_item_sk#27] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] -Condition : isnotnull(i_item_sk#45) +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : isnotnull(i_item_sk#6) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#27] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#45] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [i_item_sk#6] +Right keys [1]: [ss_item_sk#27] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#45] +Right keys [1]: [i_item_sk#6] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (66) Scan parquet default.date_dim -Output [3]: [d_date_sk#50, d_year#51, d_moy#52] +Output [3]: [d_date_sk#12, d_year#13, d_moy#30] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [3]: [d_date_sk#50, d_year#51, d_moy#52] +Input [3]: [d_date_sk#12, d_year#13, d_moy#30] (68) Filter [codegen id : 24] -Input [3]: [d_date_sk#50, d_year#51, d_moy#52] -Condition : ((((isnotnull(d_year#51) AND isnotnull(d_moy#52)) AND (d_year#51 = 2001)) AND (d_moy#52 = 11)) AND isnotnull(d_date_sk#50)) +Input [3]: [d_date_sk#12, d_year#13, d_moy#30] +Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#30)) AND (d_year#13 = 2001)) AND (d_moy#30 = 11)) AND isnotnull(d_date_sk#12)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#50] -Input [3]: [d_date_sk#50, d_year#51, d_moy#52] +Output [1]: [d_date_sk#12] +Input [3]: [d_date_sk#12, d_year#13, d_moy#30] (70) BroadcastExchange -Input [1]: [d_date_sk#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#50] +Right keys [1]: [d_date_sk#12] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] -Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] +Aggregate Attributes [3]: [sum#32, isEmpty#33, count#34] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] (74) Exchange -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] -Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#38] (75) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] -Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] (76) Filter [codegen id : 26] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [sales#63, number_sales#64, store AS channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Output [6]: [sales#41, number_sales#42, store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] (78) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +Output [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] (80) Filter [codegen id : 51] -Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] -Condition : isnotnull(cs_item_sk#69) +Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Condition : isnotnull(cs_item_sk#18) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#27] (82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#69] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [ss_item_sk#27] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#69] -Right keys [1]: [i_item_sk#73] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#6] Join condition: None (85) Project [codegen id : 51] -Output [6]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76] -Input [8]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] +Output [6]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [8]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#77] +Output [1]: [d_date_sk#12] (87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#72] -Right keys [1]: [d_date_sk#77] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None (88) Project [codegen id : 51] -Output [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] -Input [7]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76, d_date_sk#77] +Output [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] -Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#78, isEmpty#79, count#80] -Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Input [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#49, isEmpty#50, count#51] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] (90) Exchange -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] -Arguments: hashpartitioning(i_brand_id#74, i_class_id#75, i_category_id#76, 5), ENSURE_REQUIREMENTS, [id=#84] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#55] (91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] -Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85, count(1)#86] -Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sales#87, count(1)#86 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56, count(1)#57] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sales#58, count(1)#57 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] (92) Filter [codegen id : 52] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [sales#87, number_sales#88, catalog AS channel#90, i_brand_id#74, i_class_id#75, i_category_id#76] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] +Output [6]: [sales#58, number_sales#59, catalog AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] (94) Scan parquet default.web_sales -Output [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Output [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] (96) Filter [codegen id : 77] -Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] -Condition : isnotnull(ws_item_sk#91) +Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Condition : isnotnull(ws_item_sk#23) (97) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#27] (98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#91] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [ss_item_sk#27] Join condition: None (99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#91] -Right keys [1]: [i_item_sk#95] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [i_item_sk#6] Join condition: None (101) Project [codegen id : 77] -Output [6]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98] -Input [8]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] +Output [6]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [8]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#99] +Output [1]: [d_date_sk#12] (103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#94] -Right keys [1]: [d_date_sk#99] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (104) Project [codegen id : 77] -Output [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] -Input [7]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98, d_date_sk#99] +Output [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] -Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#100, isEmpty#101, count#102] -Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Input [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] (106) Exchange -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] -Arguments: hashpartitioning(i_brand_id#96, i_class_id#97, i_category_id#98, 5), ENSURE_REQUIREMENTS, [id=#106] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#70] (107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] -Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] -Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#109, count(1)#108 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71, count(1)#72] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sales#73, count(1)#72 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] (108) Filter [codegen id : 78] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [sales#109, number_sales#110, web AS channel#112, i_brand_id#96, i_class_id#97, i_category_id#98] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] +Output [6]: [sales#73, number_sales#74, web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] (110) Union (111) Expand [codegen id : 79] -Input [6]: [sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] -Arguments: [List(sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, 0), List(sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, null, 1), List(sales#63, number_sales#64, channel#68, i_brand_id#46, null, null, 3), List(sales#63, number_sales#64, channel#68, null, null, null, 7), List(sales#63, number_sales#64, null, null, null, null, 15)], [sales#63, number_sales#64, channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] +Input [6]: [sales#41, number_sales#42, channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: [List(sales#41, number_sales#42, channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#41, number_sales#42, channel#46, i_brand_id#7, i_class_id#8, null, 1), List(sales#41, number_sales#42, channel#46, i_brand_id#7, null, null, 3), List(sales#41, number_sales#42, channel#46, null, null, null, 7), List(sales#41, number_sales#42, null, null, null, null, 15)], [sales#41, number_sales#42, channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] (112) HashAggregate [codegen id : 79] -Input [7]: [sales#63, number_sales#64, channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] -Keys [5]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#118, isEmpty#119, sum#120] -Results [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] +Input [7]: [sales#41, number_sales#42, channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] +Keys [5]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] +Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] +Aggregate Attributes [3]: [sum#82, isEmpty#83, sum#84] +Results [8]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, sum#85, isEmpty#86, sum#87] (113) Exchange -Input [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] -Arguments: hashpartitioning(channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [8]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, sum#85, isEmpty#86, sum#87] +Arguments: hashpartitioning(channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, 5), ENSURE_REQUIREMENTS, [id=#88] (114) HashAggregate [codegen id : 80] -Input [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] -Keys [5]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#125, sum(number_sales#64)#126] -Results [6]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales#63)#125 AS sum(sales)#127, sum(number_sales#64)#126 AS sum(number_sales)#128] +Input [8]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, sum#85, isEmpty#86, sum#87] +Keys [5]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] +Functions [2]: [sum(sales#41), sum(number_sales#42)] +Aggregate Attributes [2]: [sum(sales#41)#89, sum(number_sales#42)#90] +Results [6]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, sum(sales#41)#89 AS sum(sales)#91, sum(number_sales#42)#90 AS sum(number_sales)#92] (115) TakeOrderedAndProject -Input [6]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales)#127, sum(number_sales)#128] -Arguments: 100, [channel#113 ASC NULLS FIRST, i_brand_id#114 ASC NULLS FIRST, i_class_id#115 ASC NULLS FIRST, i_category_id#116 ASC NULLS FIRST], [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales)#127, sum(number_sales)#128] +Input [6]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, sum(sales)#91, sum(number_sales)#92] +Arguments: 100, [channel#77 ASC NULLS FIRST, i_brand_id#78 ASC NULLS FIRST, i_class_id#79 ASC NULLS FIRST, i_category_id#80 ASC NULLS FIRST], [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, sum(sales)#91, sum(number_sales)#92] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#66, [id=#67] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] * HashAggregate (138) +- Exchange (137) +- * HashAggregate (136) @@ -678,145 +678,145 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (116) Scan parquet default.store_sales -Output [3]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131] +Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#131), dynamicpruningexpression(ss_sold_date_sk#131 IN dynamicpruning#132)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#93)] ReadSchema: struct (117) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131] +Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (118) Scan parquet default.date_dim -Output [2]: [d_date_sk#133, d_year#134] +Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (119) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#133, d_year#134] +Input [2]: [d_date_sk#12, d_year#13] (120) Filter [codegen id : 1] -Input [2]: [d_date_sk#133, d_year#134] -Condition : (((isnotnull(d_year#134) AND (d_year#134 >= 1999)) AND (d_year#134 <= 2001)) AND isnotnull(d_date_sk#133)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (121) Project [codegen id : 1] -Output [1]: [d_date_sk#133] -Input [2]: [d_date_sk#133, d_year#134] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (122) BroadcastExchange -Input [1]: [d_date_sk#133] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#135] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#94] (123) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#131] -Right keys [1]: [d_date_sk#133] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (124) Project [codegen id : 2] -Output [2]: [ss_quantity#129 AS quantity#136, ss_list_price#130 AS list_price#137] -Input [4]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131, d_date_sk#133] +Output [2]: [ss_quantity#2 AS quantity#95, ss_list_price#3 AS list_price#96] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (125) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140] +Output [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#140), dynamicpruningexpression(cs_sold_date_sk#140 IN dynamicpruning#132)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#93)] ReadSchema: struct (126) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140] +Input [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] (127) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#141] +Output [1]: [d_date_sk#12] (128) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#140] -Right keys [1]: [d_date_sk#141] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None (129) Project [codegen id : 4] -Output [2]: [cs_quantity#138 AS quantity#142, cs_list_price#139 AS list_price#143] -Input [4]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140, d_date_sk#141] +Output [2]: [cs_quantity#47 AS quantity#97, cs_list_price#48 AS list_price#98] +Input [4]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, d_date_sk#12] (130) Scan parquet default.web_sales -Output [3]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146] +Output [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#146), dynamicpruningexpression(ws_sold_date_sk#146 IN dynamicpruning#132)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#93)] ReadSchema: struct (131) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146] +Input [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] (132) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#147] +Output [1]: [d_date_sk#12] (133) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#146] -Right keys [1]: [d_date_sk#147] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (134) Project [codegen id : 6] -Output [2]: [ws_quantity#144 AS quantity#148, ws_list_price#145 AS list_price#149] -Input [4]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146, d_date_sk#147] +Output [2]: [ws_quantity#62 AS quantity#99, ws_list_price#63 AS list_price#100] +Input [4]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, d_date_sk#12] (135) Union (136) HashAggregate [codegen id : 7] -Input [2]: [quantity#136, list_price#137] +Input [2]: [quantity#95, list_price#96] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#150, count#151] -Results [2]: [sum#152, count#153] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#101, count#102] +Results [2]: [sum#103, count#104] (137) Exchange -Input [2]: [sum#152, count#153] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#154] +Input [2]: [sum#103, count#104] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] (138) HashAggregate [codegen id : 8] -Input [2]: [sum#152, count#153] +Input [2]: [sum#103, count#104] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#155] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#155 AS average_sales#156] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))#106] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))#106 AS average_sales#107] -Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#131 IN dynamicpruning#132 +Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#93 ReusedExchange (139) (139) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#133] +Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#140 IN dynamicpruning#132 +Subquery:3 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#93 -Subquery:4 Hosting operator id = 130 Hosting Expression = ws_sold_date_sk#146 IN dynamicpruning#132 +Subquery:4 Hosting operator id = 130 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#93 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#50] +Output [1]: [d_date_sk#12] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 ReusedExchange (141) -(141) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#29] +(141) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 -Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index 9ae1a0e30e90e..f9b76560e363c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -90,53 +90,53 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Exchange [brand_id,class_id,category_id] #6 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (1) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #9 + BroadcastExchange #10 WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (1) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Filter [i_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #7 + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #11 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt index fab231dc80a6c..76f5ac48f96cc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt @@ -33,39 +33,39 @@ TakeOrderedAndProject (116) : : : : +- * HashAggregate (44) : : : : +- Exchange (43) : : : : +- * HashAggregate (42) - : : : : +- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * Project (18) - : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : :- * Filter (11) - : : : : : : +- * ColumnarToRow (10) - : : : : : : +- Scan parquet default.store_sales (9) - : : : : : +- BroadcastExchange (16) - : : : : : +- * Project (15) - : : : : : +- * Filter (14) - : : : : : +- * ColumnarToRow (13) - : : : : : +- Scan parquet default.date_dim (12) - : : : : +- BroadcastExchange (39) - : : : : +- SortMergeJoin LeftSemi (38) - : : : : :- * Sort (23) - : : : : : +- Exchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (37) - : : : : +- Exchange (36) - : : : : +- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (26) - : : : : : : +- * ColumnarToRow (25) - : : : : : : +- Scan parquet default.catalog_sales (24) - : : : : : +- ReusedExchange (27) - : : : : +- BroadcastExchange (33) - : : : : +- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet default.item (30) + : : : : +- SortMergeJoin LeftSemi (41) + : : : : :- * Sort (26) + : : : : : +- Exchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (40) + : : : : +- Exchange (39) + : : : : +- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.item (33) : : : +- * Sort (57) : : : +- Exchange (56) : : : +- * Project (55) @@ -147,507 +147,507 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 19] +(7) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 19] +(8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 10] -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +(10) ColumnarToRow [codegen id : 5] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -(11) Filter [codegen id : 10] -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_item_sk#11) +(11) Filter [codegen id : 5] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#12, d_year#13] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#14, d_year#15] +Input [2]: [d_date_sk#12, d_year#13] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#14, d_year#15] -Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1999)) AND (d_year#15 <= 2001)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_year#15] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (16) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(17) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#14] +(17) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None -(18) Project [codegen id : 10] -Output [1]: [ss_item_sk#11] -Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] +(18) Project [codegen id : 5] +Output [1]: [ss_item_sk#1] +Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] (19) Scan parquet default.item -Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) + +(22) BroadcastExchange +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] + +(23) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#7] +Join condition: None + +(24) Project [codegen id : 5] +Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] +Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(22) Exchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] +(25) Exchange +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] -(23) Sort [codegen id : 5] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 +(26) Sort [codegen id : 6] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 -(24) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] +(27) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 8] -Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +(28) ColumnarToRow [codegen id : 9] +Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -(26) Filter [codegen id : 8] -Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] -Condition : isnotnull(cs_item_sk#22) +(29) Filter [codegen id : 9] +Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +Condition : isnotnull(cs_item_sk#20) -(27) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#24] +(30) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#12] -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] +(31) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#12] Join condition: None -(29) Project [codegen id : 8] -Output [1]: [cs_item_sk#22] -Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] +(32) Project [codegen id : 9] +Output [1]: [cs_item_sk#20] +Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] -(30) Scan parquet default.item -Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +(33) Scan parquet default.item +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 7] -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(32) Filter [codegen id : 7] -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -Condition : isnotnull(i_item_sk#25) +(35) Filter [codegen id : 8] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : isnotnull(i_item_sk#7) -(33) BroadcastExchange -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +(36) BroadcastExchange +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#22] -Right keys [1]: [i_item_sk#25] +(37) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [cs_item_sk#20] +Right keys [1]: [i_item_sk#7] Join condition: None -(35) Project [codegen id : 8] -Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] - -(36) Exchange -Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] +(38) Project [codegen id : 9] +Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(37) Sort [codegen id : 9] -Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 +(39) Exchange +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] -(38) SortMergeJoin -Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] -Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] -Join condition: None - -(39) BroadcastExchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +(40) Sort [codegen id : 10] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#11] -Right keys [1]: [i_item_sk#17] +(41) SortMergeJoin +Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] +Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] Join condition: None -(41) Project [codegen id : 10] -Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] -Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] - -(42) HashAggregate [codegen id : 10] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +(42) HashAggregate [codegen id : 11] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#16, class_id#17, category_id#18] (43) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] -(44) HashAggregate [codegen id : 11] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +(44) HashAggregate [codegen id : 12] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#16, class_id#17, category_id#18] (45) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] -(46) Sort [codegen id : 12] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 13] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 15] -Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +(48) ColumnarToRow [codegen id : 16] +Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -(49) Filter [codegen id : 15] -Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_item_sk#37) +(49) Filter [codegen id : 16] +Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Condition : isnotnull(ws_item_sk#26) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#39] +Output [1]: [d_date_sk#12] -(51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#39] +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None -(52) Project [codegen id : 15] -Output [1]: [ws_item_sk#37] -Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] +(52) Project [codegen id : 16] +Output [1]: [ws_item_sk#26] +Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] -(53) ReusedExchange [Reuses operator id: 33] -Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] +(53) ReusedExchange [Reuses operator id: 36] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(54) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#40] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [i_item_sk#7] Join condition: None -(55) Project [codegen id : 15] -Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] +(55) Project [codegen id : 16] +Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (56) Exchange -Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] -(57) Sort [codegen id : 16] -Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 17] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] -Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] +Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] +Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] Join condition: None -(59) HashAggregate [codegen id : 17] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +(59) HashAggregate [codegen id : 18] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#16, class_id#17, category_id#18] (60) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] -(61) HashAggregate [codegen id : 18] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +(61) HashAggregate [codegen id : 19] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#16, class_id#17, category_id#18] (62) BroadcastExchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] -(63) BroadcastHashJoin [codegen id : 19] +(63) BroadcastHashJoin [codegen id : 20] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#32, class_id#33, category_id#34] +Right keys [3]: [brand_id#16, class_id#17, category_id#18] Join condition: None -(64) Project [codegen id : 19] -Output [1]: [i_item_sk#7 AS ss_item_sk#47] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] +(64) Project [codegen id : 20] +Output [1]: [i_item_sk#7 AS ss_item_sk#31] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] (65) Exchange -Input [1]: [ss_item_sk#47] -Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [1]: [ss_item_sk#31] +Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] -(66) Sort [codegen id : 20] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 21] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#47] +Right keys [1]: [ss_item_sk#31] Join condition: None (68) Scan parquet default.date_dim -Output [2]: [d_date_sk#49, d_week_seq#50] +Output [2]: [d_date_sk#12, d_week_seq#33] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 21] -Input [2]: [d_date_sk#49, d_week_seq#50] +(69) ColumnarToRow [codegen id : 22] +Input [2]: [d_date_sk#12, d_week_seq#33] -(70) Filter [codegen id : 21] -Input [2]: [d_date_sk#49, d_week_seq#50] -Condition : ((isnotnull(d_week_seq#50) AND (d_week_seq#50 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#49)) +(70) Filter [codegen id : 22] +Input [2]: [d_date_sk#12, d_week_seq#33] +Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#34, [id=#35])) AND isnotnull(d_date_sk#12)) -(71) Project [codegen id : 21] -Output [1]: [d_date_sk#49] -Input [2]: [d_date_sk#49, d_week_seq#50] +(71) Project [codegen id : 22] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_week_seq#33] (72) BroadcastExchange -Input [1]: [d_date_sk#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] -(73) BroadcastHashJoin [codegen id : 42] +(73) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#49] +Right keys [1]: [d_date_sk#12] Join condition: None -(74) Project [codegen id : 42] +(74) Project [codegen id : 44] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (75) Scan parquet default.item -Output [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 22] -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +(76) ColumnarToRow [codegen id : 23] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(77) Filter [codegen id : 22] -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -Condition : (((isnotnull(i_item_sk#54) AND isnotnull(i_brand_id#55)) AND isnotnull(i_class_id#56)) AND isnotnull(i_category_id#57)) +(77) Filter [codegen id : 23] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (78) Exchange -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -Arguments: hashpartitioning(i_item_sk#54, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#37] -(79) Sort [codegen id : 23] -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -Arguments: [i_item_sk#54 ASC NULLS FIRST], false, 0 +(79) Sort [codegen id : 24] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#31] -(81) Sort [codegen id : 41] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 43] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#54] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [i_item_sk#7] +Right keys [1]: [ss_item_sk#31] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#59] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] -(84) BroadcastHashJoin [codegen id : 42] +(84) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#54] +Right keys [1]: [i_item_sk#7] Join condition: None -(85) Project [codegen id : 42] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +(85) Project [codegen id : 44] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(86) HashAggregate [codegen id : 42] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] -Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] +(86) HashAggregate [codegen id : 44] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#60, isEmpty#61, count#62] -Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] +Aggregate Attributes [3]: [sum#39, isEmpty#40, count#41] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] (87) Exchange -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] -Arguments: hashpartitioning(i_brand_id#55, i_class_id#56, i_category_id#57, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#45] -(88) HashAggregate [codegen id : 86] -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] -Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] +(88) HashAggregate [codegen id : 90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67, count(1)#68] -Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#69, count(1)#68 AS number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46, count(1)#47] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sales#48, count(1)#47 AS number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] -(89) Filter [codegen id : 86] -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(32,6)) > cast(Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) +(89) Filter [codegen id : 90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(90) Project [codegen id : 86] -Output [6]: [store AS channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70] -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] +(90) Project [codegen id : 90] +Output [6]: [store AS channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] (91) Scan parquet default.store_sales -Output [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 43] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +(92) ColumnarToRow [codegen id : 45] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(93) Filter [codegen id : 43] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Condition : isnotnull(ss_item_sk#75) +(93) Filter [codegen id : 45] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (94) Exchange -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Arguments: hashpartitioning(ss_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#55] -(95) Sort [codegen id : 44] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Arguments: [ss_item_sk#75 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 46] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#31] -(97) Sort [codegen id : 62] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 65] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ss_item_sk#75] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [ss_item_sk#31] Join condition: None (99) Scan parquet default.date_dim -Output [2]: [d_date_sk#81, d_week_seq#82] +Output [2]: [d_date_sk#12, d_week_seq#33] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 63] -Input [2]: [d_date_sk#81, d_week_seq#82] +(100) ColumnarToRow [codegen id : 66] +Input [2]: [d_date_sk#12, d_week_seq#33] -(101) Filter [codegen id : 63] -Input [2]: [d_date_sk#81, d_week_seq#82] -Condition : ((isnotnull(d_week_seq#82) AND (d_week_seq#82 = Subquery scalar-subquery#83, [id=#84])) AND isnotnull(d_date_sk#81)) +(101) Filter [codegen id : 66] +Input [2]: [d_date_sk#12, d_week_seq#33] +Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#56, [id=#57])) AND isnotnull(d_date_sk#12)) -(102) Project [codegen id : 63] -Output [1]: [d_date_sk#81] -Input [2]: [d_date_sk#81, d_week_seq#82] +(102) Project [codegen id : 66] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_week_seq#33] (103) BroadcastExchange -Input [1]: [d_date_sk#81] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#85] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] -(104) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_sold_date_sk#78] -Right keys [1]: [d_date_sk#81] +(104) BroadcastHashJoin [codegen id : 88] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None -(105) Project [codegen id : 84] -Output [3]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77] -Input [5]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#81] +(105) Project [codegen id : 88] +Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (106) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] +Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(107) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#75] -Right keys [1]: [i_item_sk#86] +(107) BroadcastHashJoin [codegen id : 88] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#59] Join condition: None -(108) Project [codegen id : 84] -Output [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] -Input [7]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] +(108) Project [codegen id : 88] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(109) HashAggregate [codegen id : 84] -Input [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] -Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] -Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] +(109) HashAggregate [codegen id : 88] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] +Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65] +Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] (110) Exchange -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] -Arguments: hashpartitioning(i_brand_id#87, i_class_id#88, i_category_id#89, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] +Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [id=#69] -(111) HashAggregate [codegen id : 85] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] -Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] -Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#99, count(1)#98 AS number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] +(111) HashAggregate [codegen id : 89] +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] +Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70, count(1)#71] +Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sales#72, count(1)#71 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] -(112) Filter [codegen id : 85] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) +(112) Filter [codegen id : 89] +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(113) Project [codegen id : 85] -Output [6]: [store AS channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] +(113) Project [codegen id : 89] +Output [6]: [store AS channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] (114) BroadcastExchange -Input [6]: [channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#103] +Input [6]: [channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#76] -(115) BroadcastHashJoin [codegen id : 86] -Left keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] -Right keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +(115) BroadcastHashJoin [codegen id : 90] +Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Join condition: None (116) TakeOrderedAndProject -Input [12]: [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Arguments: 100, [i_brand_id#55 ASC NULLS FIRST, i_class_id#56 ASC NULLS FIRST, i_category_id#57 ASC NULLS FIRST], [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Input [12]: [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#72, [id=#73] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#51, [id=#52] * HashAggregate (139) +- Exchange (138) +- * HashAggregate (137) @@ -674,140 +674,140 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (117) Scan parquet default.store_sales -Output [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] +Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#106), dynamicpruningexpression(ss_sold_date_sk#106 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#77)] ReadSchema: struct (118) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] +Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (119) Scan parquet default.date_dim -Output [2]: [d_date_sk#108, d_year#109] +Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (120) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#108, d_year#109] +Input [2]: [d_date_sk#12, d_year#13] (121) Filter [codegen id : 1] -Input [2]: [d_date_sk#108, d_year#109] -Condition : (((isnotnull(d_year#109) AND (d_year#109 >= 1999)) AND (d_year#109 <= 2001)) AND isnotnull(d_date_sk#108)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (122) Project [codegen id : 1] -Output [1]: [d_date_sk#108] -Input [2]: [d_date_sk#108, d_year#109] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (123) BroadcastExchange -Input [1]: [d_date_sk#108] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#110] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] (124) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#106] -Right keys [1]: [d_date_sk#108] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (125) Project [codegen id : 2] -Output [2]: [ss_quantity#104 AS quantity#111, ss_list_price#105 AS list_price#112] -Input [4]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106, d_date_sk#108] +Output [2]: [ss_quantity#2 AS quantity#79, ss_list_price#3 AS list_price#80] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (126) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] +Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#115), dynamicpruningexpression(cs_sold_date_sk#115 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#77)] ReadSchema: struct (127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] +Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] (128) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#116] +Output [1]: [d_date_sk#12] (129) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#115] -Right keys [1]: [d_date_sk#116] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#12] Join condition: None (130) Project [codegen id : 4] -Output [2]: [cs_quantity#113 AS quantity#117, cs_list_price#114 AS list_price#118] -Input [4]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115, d_date_sk#116] +Output [2]: [cs_quantity#81 AS quantity#83, cs_list_price#82 AS list_price#84] +Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21, d_date_sk#12] (131) Scan parquet default.web_sales -Output [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] +Output [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#77)] ReadSchema: struct (132) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] +Input [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] (133) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#122] +Output [1]: [d_date_sk#12] (134) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#121] -Right keys [1]: [d_date_sk#122] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None (135) Project [codegen id : 6] -Output [2]: [ws_quantity#119 AS quantity#123, ws_list_price#120 AS list_price#124] -Input [4]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121, d_date_sk#122] +Output [2]: [ws_quantity#85 AS quantity#87, ws_list_price#86 AS list_price#88] +Input [4]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27, d_date_sk#12] (136) Union (137) HashAggregate [codegen id : 7] -Input [2]: [quantity#111, list_price#112] +Input [2]: [quantity#79, list_price#80] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#125, count#126] -Results [2]: [sum#127, count#128] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#89, count#90] +Results [2]: [sum#91, count#92] (138) Exchange -Input [2]: [sum#127, count#128] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] +Input [2]: [sum#91, count#92] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] (139) HashAggregate [codegen id : 8] -Input [2]: [sum#127, count#128] +Input [2]: [sum#91, count#92] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130 AS average_sales#131] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94 AS average_sales#95] -Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#106 IN dynamicpruning#107 +Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#77 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#108] +Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#115 IN dynamicpruning#107 +Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#77 -Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#107 +Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#77 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (141) (141) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#49] +Output [1]: [d_date_sk#12] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 ReusedExchange (142) (142) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 +Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 -Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#34, [id=#35] * Project (146) +- * Filter (145) +- * ColumnarToRow (144) @@ -815,33 +815,33 @@ Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquer (143) Scan parquet default.date_dim -Output [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] (145) Filter [codegen id : 1] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] -Condition : (((((isnotnull(d_year#133) AND isnotnull(d_moy#134)) AND isnotnull(d_dom#135)) AND (d_year#133 = 2000)) AND (d_moy#134 = 12)) AND (d_dom#135 = 11)) +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 2000)) AND (d_moy#96 = 12)) AND (d_dom#97 = 11)) (146) Project [codegen id : 1] -Output [1]: [d_week_seq#132] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Output [1]: [d_week_seq#33] +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] -Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] +Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] -Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#79 +Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#54 ReusedExchange (147) (147) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#81] +Output [1]: [d_date_sk#12] -Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#83, [id=#84] +Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#56, [id=#57] * Project (151) +- * Filter (150) +- * ColumnarToRow (149) @@ -849,21 +849,21 @@ Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subqu (148) Scan parquet default.date_dim -Output [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (149) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] (150) Filter [codegen id : 1] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] -Condition : (((((isnotnull(d_year#137) AND isnotnull(d_moy#138)) AND isnotnull(d_dom#139)) AND (d_year#137 = 1999)) AND (d_moy#138 = 12)) AND (d_dom#139 = 11)) +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 1999)) AND (d_moy#96 = 12)) AND (d_dom#97 = 11)) (151) Project [codegen id : 1] -Output [1]: [d_week_seq#136] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Output [1]: [d_week_seq#33] +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt index 1b41abbc97b06..1d5907e319658 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (86) + WholeStageCodegen (90) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] @@ -49,7 +49,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (42) + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -68,11 +68,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #3 - WholeStageCodegen (20) + WholeStageCodegen (21) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 - WholeStageCodegen (19) + WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -81,82 +81,84 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (18) + WholeStageCodegen (19) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (17) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (12) + WholeStageCodegen (13) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (10) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #9 + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #10 - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (9) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (7) - Filter [i_item_sk] + Exchange [brand_id,class_id,category_id] #9 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #10 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (16) + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #10 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (8) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (17) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (15) + WholeStageCodegen (16) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -167,12 +169,12 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #9 + ReusedExchange [d_date_sk] #10 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter BroadcastExchange #3 - WholeStageCodegen (21) + WholeStageCodegen (22) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #3 @@ -188,29 +190,29 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #15 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (23) + WholeStageCodegen (24) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #16 - WholeStageCodegen (22) + WholeStageCodegen (23) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (41) + WholeStageCodegen (43) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #20 - WholeStageCodegen (85) + WholeStageCodegen (89) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (84) + WholeStageCodegen (88) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -218,24 +220,24 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] - WholeStageCodegen (44) + WholeStageCodegen (46) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #22 - WholeStageCodegen (43) + WholeStageCodegen (45) Filter [ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #6 ReusedExchange [d_date_sk] #23 - WholeStageCodegen (62) + WholeStageCodegen (65) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #23 - WholeStageCodegen (63) + WholeStageCodegen (66) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index ae653b43971db..01bcb83622844 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -27,36 +27,36 @@ TakeOrderedAndProject (100) : : : :- * HashAggregate (39) : : : : +- Exchange (38) : : : : +- * HashAggregate (37) - : : : : +- * Project (36) - : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : :- * Project (33) - : : : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet default.store_sales (7) - : : : : : +- BroadcastExchange (31) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Project (28) - : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet default.item (16) - : : : : : +- BroadcastExchange (26) - : : : : : +- * Project (25) - : : : : : +- * Filter (24) - : : : : : +- * ColumnarToRow (23) - : : : : : +- Scan parquet default.date_dim (22) - : : : : +- ReusedExchange (34) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : :- * Project (22) + : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : :- * Project (15) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (13) + : : : : : : +- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (20) + : : : : : +- * Project (19) + : : : : : +- * Filter (18) + : : : : : +- * ColumnarToRow (17) + : : : : : +- Scan parquet default.date_dim (16) + : : : : +- BroadcastExchange (35) + : : : : +- * Project (34) + : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * Project (31) + : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : :- * Filter (25) + : : : : : : +- * ColumnarToRow (24) + : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Filter (28) + : : : : : +- * ColumnarToRow (27) + : : : : : +- Scan parquet default.item (26) + : : : : +- ReusedExchange (32) : : : +- BroadcastExchange (49) : : : +- * Project (48) : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -131,443 +131,443 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (10) Scan parquet default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +(11) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(12) Filter [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) +(12) Filter [codegen id : 1] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(13) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +(13) BroadcastExchange +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +(14) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join condition: None -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) +(15) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(16) Scan parquet default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +(16) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_year#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +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 + +(17) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#12, d_year#13] -(17) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +(18) Filter [codegen id : 2] +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) -(18) Filter [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) +(19) Project [codegen id : 2] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] -(19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] +(20) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +(22) Project [codegen id : 6] +Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] +Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] -(22) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_year#25] +(23) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] 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 - -(23) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#24, d_year#25] +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(24) Filter [codegen id : 2] -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) +(24) ColumnarToRow [codegen id : 5] +Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -(25) Project [codegen id : 2] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_year#25] +(25) Filter [codegen id : 5] +Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +Condition : isnotnull(cs_item_sk#18) -(26) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +(26) Scan parquet default.item +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(27) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] -Join condition: None +(27) ColumnarToRow [codegen id : 3] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(28) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] +(28) Filter [codegen id : 3] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : isnotnull(i_item_sk#6) (29) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] -(30) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] +(30) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#6] Join condition: None -(31) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] +(31) Project [codegen id : 5] +Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] + +(32) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] -(32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] +(33) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None -(33) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +(34) Project [codegen id : 5] +Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] -(34) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#29] +(35) BroadcastExchange +Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] -(35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#29] +(36) BroadcastHashJoin [codegen id : 6] +Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] +Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None -(36) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] - (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#15, class_id#16, category_id#17] (38) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#15, class_id#16, category_id#17] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_item_sk#34) +Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Condition : isnotnull(ws_item_sk#23) -(43) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] +(43) ReusedExchange [Reuses operator id: 29] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#36] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [i_item_sk#6] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] +Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(46) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#40] +(46) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#40] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] +Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (49) BroadcastExchange -Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] +Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] -Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] +Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] +Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#15, class_id#16, category_id#17] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#15, class_id#16, category_id#17] (53) BroadcastExchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#30, class_id#31, category_id#32] +Right keys [3]: [brand_id#15, class_id#16, category_id#17] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#43] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] +Output [1]: [i_item_sk#6 AS ss_item_sk#27] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] (56) BroadcastExchange -Input [1]: [ss_item_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] +Input [1]: [ss_item_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#43] +Right keys [1]: [ss_item_sk#27] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] -Condition : (((isnotnull(i_item_sk#45) AND isnotnull(i_brand_id#46)) AND isnotnull(i_class_id#47)) AND isnotnull(i_category_id#48)) +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#27] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#45] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [i_item_sk#6] +Right keys [1]: [ss_item_sk#27] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#45] +Right keys [1]: [i_item_sk#6] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (66) Scan parquet default.date_dim -Output [2]: [d_date_sk#50, d_week_seq#51] +Output [2]: [d_date_sk#12, d_week_seq#30] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [2]: [d_date_sk#50, d_week_seq#51] +Input [2]: [d_date_sk#12, d_week_seq#30] (68) Filter [codegen id : 24] -Input [2]: [d_date_sk#50, d_week_seq#51] -Condition : ((isnotnull(d_week_seq#51) AND (d_week_seq#51 = Subquery scalar-subquery#52, [id=#53])) AND isnotnull(d_date_sk#50)) +Input [2]: [d_date_sk#12, d_week_seq#30] +Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#31, [id=#32])) AND isnotnull(d_date_sk#12)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#50] -Input [2]: [d_date_sk#50, d_week_seq#51] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_week_seq#30] (70) BroadcastExchange -Input [1]: [d_date_sk#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#50] +Right keys [1]: [d_date_sk#12] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] -Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] +Aggregate Attributes [3]: [sum#34, isEmpty#35, count#36] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] (74) Exchange -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] -Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#40] (75) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] -Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41, count(1)#42] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sales#43, count(1)#42 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] (76) Filter [codegen id : 52] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45 as decimal(32,6)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) (77) Project [codegen id : 52] -Output [6]: [store AS channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] +Output [6]: [store AS channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] (78) Scan parquet default.store_sales -Output [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#49)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (80) Filter [codegen id : 50] -Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] -Condition : isnotnull(ss_item_sk#70) +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#27] (82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#70] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [ss_item_sk#27] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] +Output [4]: [i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] (84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#70] -Right keys [1]: [i_item_sk#75] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#50] Join condition: None (85) Project [codegen id : 50] -Output [6]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] -Input [8]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#79, d_week_seq#80] +Output [2]: [d_date_sk#12, d_week_seq#30] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#79, d_week_seq#80] +Input [2]: [d_date_sk#12, d_week_seq#30] (88) Filter [codegen id : 49] -Input [2]: [d_date_sk#79, d_week_seq#80] -Condition : ((isnotnull(d_week_seq#80) AND (d_week_seq#80 = Subquery scalar-subquery#81, [id=#82])) AND isnotnull(d_date_sk#79)) +Input [2]: [d_date_sk#12, d_week_seq#30] +Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#54, [id=#55])) AND isnotnull(d_date_sk#12)) (89) Project [codegen id : 49] -Output [1]: [d_date_sk#79] -Input [2]: [d_date_sk#79, d_week_seq#80] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_week_seq#30] (90) BroadcastExchange -Input [1]: [d_date_sk#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#83] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] (91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#73] -Right keys [1]: [d_date_sk#79] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (92) Project [codegen id : 50] -Output [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] -Input [7]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78, d_date_sk#79] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53, d_date_sk#12] (93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] -Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] +Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#57, isEmpty#58, count#59] +Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] (94) Exchange -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] -Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] +Arguments: hashpartitioning(i_brand_id#51, i_class_id#52, i_category_id#53, 5), ENSURE_REQUIREMENTS, [id=#63] (95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] -Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] +Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] +Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] +Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] (96) Filter [codegen id : 51] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) +Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) (97) Project [codegen id : 51] -Output [6]: [store AS channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] +Output [6]: [store AS channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] (98) BroadcastExchange -Input [6]: [channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#97] +Input [6]: [channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] (99) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] -Right keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Right keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] Join condition: None (100) TakeOrderedAndProject -Input [12]: [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Arguments: 100, [i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Input [12]: [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#67, [id=#68] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#46, [id=#47] * HashAggregate (123) +- Exchange (122) +- * HashAggregate (121) @@ -594,140 +594,140 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] +Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#100), dynamicpruningexpression(ss_sold_date_sk#100 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#71)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] +Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (103) Scan parquet default.date_dim -Output [2]: [d_date_sk#102, d_year#103] +Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#102, d_year#103] +Input [2]: [d_date_sk#12, d_year#13] (105) Filter [codegen id : 1] -Input [2]: [d_date_sk#102, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 <= 2001)) AND isnotnull(d_date_sk#102)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (106) Project [codegen id : 1] -Output [1]: [d_date_sk#102] -Input [2]: [d_date_sk#102, d_year#103] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (107) BroadcastExchange -Input [1]: [d_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#104] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] (108) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#100] -Right keys [1]: [d_date_sk#102] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (109) Project [codegen id : 2] -Output [2]: [ss_quantity#98 AS quantity#105, ss_list_price#99 AS list_price#106] -Input [4]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100, d_date_sk#102] +Output [2]: [ss_quantity#2 AS quantity#73, ss_list_price#3 AS list_price#74] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (110) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] +Output [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#109), dynamicpruningexpression(cs_sold_date_sk#109 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#71)] ReadSchema: struct (111) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] +Input [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] (112) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#110] +Output [1]: [d_date_sk#12] (113) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#109] -Right keys [1]: [d_date_sk#110] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None (114) Project [codegen id : 4] -Output [2]: [cs_quantity#107 AS quantity#111, cs_list_price#108 AS list_price#112] -Input [4]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109, d_date_sk#110] +Output [2]: [cs_quantity#75 AS quantity#77, cs_list_price#76 AS list_price#78] +Input [4]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19, d_date_sk#12] (115) Scan parquet default.web_sales -Output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] +Output [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#115), dynamicpruningexpression(ws_sold_date_sk#115 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#71)] ReadSchema: struct (116) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] +Input [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] (117) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#116] +Output [1]: [d_date_sk#12] (118) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#115] -Right keys [1]: [d_date_sk#116] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (119) Project [codegen id : 6] -Output [2]: [ws_quantity#113 AS quantity#117, ws_list_price#114 AS list_price#118] -Input [4]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115, d_date_sk#116] +Output [2]: [ws_quantity#79 AS quantity#81, ws_list_price#80 AS list_price#82] +Input [4]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24, d_date_sk#12] (120) Union (121) HashAggregate [codegen id : 7] -Input [2]: [quantity#105, list_price#106] +Input [2]: [quantity#73, list_price#74] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#119, count#120] -Results [2]: [sum#121, count#122] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#83, count#84] +Results [2]: [sum#85, count#86] (122) Exchange -Input [2]: [sum#121, count#122] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#123] +Input [2]: [sum#85, count#86] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#87] (123) HashAggregate [codegen id : 8] -Input [2]: [sum#121, count#122] +Input [2]: [sum#85, count#86] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124 AS average_sales#125] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88 AS average_sales#89] -Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#100 IN dynamicpruning#101 +Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#71 ReusedExchange (124) (124) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#102] +Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#109 IN dynamicpruning#101 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#71 -Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#115 IN dynamicpruning#101 +Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#71 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#50] +Output [1]: [d_date_sk#12] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 ReusedExchange (126) -(126) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#29] +(126) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 -Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#31, [id=#32] * Project (130) +- * Filter (129) +- * ColumnarToRow (128) @@ -735,33 +735,33 @@ Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer (127) Scan parquet default.date_dim -Output [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (128) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] (129) Filter [codegen id : 1] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] -Condition : (((((isnotnull(d_year#127) AND isnotnull(d_moy#128)) AND isnotnull(d_dom#129)) AND (d_year#127 = 2000)) AND (d_moy#128 = 12)) AND (d_dom#129 = 11)) +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 2000)) AND (d_moy#90 = 12)) AND (d_dom#91 = 11)) (130) Project [codegen id : 1] -Output [1]: [d_week_seq#126] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Output [1]: [d_week_seq#30] +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] -Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] +Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] -Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 +Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#49 ReusedExchange (131) (131) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#79] +Output [1]: [d_date_sk#12] -Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#81, [id=#82] +Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#54, [id=#55] * Project (135) +- * Filter (134) +- * ColumnarToRow (133) @@ -769,21 +769,21 @@ Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subque (132) Scan parquet default.date_dim -Output [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (133) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] (134) Filter [codegen id : 1] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] -Condition : (((((isnotnull(d_year#131) AND isnotnull(d_moy#132)) AND isnotnull(d_dom#133)) AND (d_year#131 = 1999)) AND (d_moy#132 = 12)) AND (d_dom#133 = 11)) +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 1999)) AND (d_moy#90 = 12)) AND (d_dom#91 = 11)) (135) Project [codegen id : 1] -Output [1]: [d_week_seq#130] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Output [1]: [d_week_seq#30] +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index cb9b410ea8568..3507d76bcf9b9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -82,53 +82,53 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Exchange [brand_id,class_id,category_id] #5 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (1) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #8 + BroadcastExchange #9 WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Filter [i_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #6 + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index 9c56438870f64..adfa3e44f996d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -203,7 +203,7 @@ Input [2]: [d_date_sk#22, d_date#23] (36) Filter [codegen id : 10] Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2002-02-01)) AND (d_date#23 <= 2002-04-02)) AND isnotnull(d_date_sk#22)) +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 11719)) AND (d_date#23 <= 11779)) AND isnotnull(d_date_sk#22)) (37) Project [codegen id : 10] Output [1]: [d_date_sk#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index dde6a9f564859..5640564564396 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -141,7 +141,7 @@ Input [2]: [d_date_sk#16, d_date#17] (22) Filter [codegen id : 8] Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2002-02-01)) AND (d_date#17 <= 2002-04-02)) AND isnotnull(d_date_sk#16)) +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 11719)) AND (d_date#17 <= 11779)) AND isnotnull(d_date_sk#16)) (23) Project [codegen id : 8] Output [1]: [d_date_sk#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index 8918c2a36e2ec..97467fe1dc1db 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -77,7 +77,7 @@ Condition : (((isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1)) AND isno Output [2]: [d_date_sk#8, d_quarter_name#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -85,7 +85,7 @@ Input [2]: [d_date_sk#8, d_quarter_name#9] (6) Filter [codegen id : 1] Input [2]: [d_date_sk#8, d_quarter_name#9] -Condition : ((isnotnull(d_quarter_name#9) AND (rpad(d_quarter_name#9, 6, ) = 2001Q1)) AND isnotnull(d_date_sk#8)) +Condition : ((isnotnull(d_quarter_name#9) AND (d_quarter_name#9 = 2001Q1)) AND isnotnull(d_date_sk#8)) (7) Project [codegen id : 1] Output [1]: [d_date_sk#8] @@ -197,7 +197,7 @@ Condition : ((isnotnull(sr_customer_sk#21) AND isnotnull(sr_item_sk#20)) AND isn Output [2]: [d_date_sk#26, d_quarter_name#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 9] @@ -205,7 +205,7 @@ Input [2]: [d_date_sk#26, d_quarter_name#27] (33) Filter [codegen id : 9] Input [2]: [d_date_sk#26, d_quarter_name#27] -Condition : (rpad(d_quarter_name#27, 6, ) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#26)) +Condition : (d_quarter_name#27 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#26)) (34) Project [codegen id : 9] Output [1]: [d_date_sk#26] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index dafa24943be82..32c8de9a1d16e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -124,7 +124,7 @@ Input [12]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_i Output [2]: [d_date_sk#21, d_quarter_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_quarter_name), IsNotNull(d_date_sk)] +PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] ReadSchema: struct (17) ColumnarToRow [codegen id : 3] @@ -132,7 +132,7 @@ Input [2]: [d_date_sk#21, d_quarter_name#22] (18) Filter [codegen id : 3] Input [2]: [d_date_sk#21, d_quarter_name#22] -Condition : ((isnotnull(d_quarter_name#22) AND (rpad(d_quarter_name#22, 6, ) = 2001Q1)) AND isnotnull(d_date_sk#21)) +Condition : ((isnotnull(d_quarter_name#22) AND (d_quarter_name#22 = 2001Q1)) AND isnotnull(d_date_sk#21)) (19) Project [codegen id : 3] Output [1]: [d_date_sk#21] @@ -155,7 +155,7 @@ Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_re Output [2]: [d_date_sk#24, d_quarter_name#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date_sk)] +PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] @@ -163,7 +163,7 @@ Input [2]: [d_date_sk#24, d_quarter_name#25] (25) Filter [codegen id : 4] Input [2]: [d_date_sk#24, d_quarter_name#25] -Condition : (rpad(d_quarter_name#25, 6, ) IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) +Condition : (d_quarter_name#25 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#24)) (26) Project [codegen id : 4] Output [1]: [d_date_sk#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt index 5c886e9f2295d..b70c36db4bc9d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt @@ -116,63 +116,63 @@ Input [3]: [cs_item_sk#4, sum#13, count#14] Keys [1]: [cs_item_sk#4] Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#5))] Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#5))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4 AS cs_item_sk#4#18] (19) Filter -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4#18] Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) (20) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#4] +Right keys [1]: [cs_item_sk#4#18] Join condition: None (21) Project [codegen id : 4] Output [2]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17] -Input [3]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4] +Input [3]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4#18] (22) BroadcastExchange Input [2]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (23) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] +Output [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#6), dynamicpruningexpression(cs_sold_date_sk#6 IN dynamicpruning#7)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] ReadSchema: struct (24) ColumnarToRow -Input [3]: [cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] +Input [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] (25) Filter -Input [3]: [cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] -Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_ext_discount_amt#20)) +Input [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] +Condition : (isnotnull(cs_item_sk#4) AND isnotnull(cs_ext_discount_amt#5)) (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#19] -Join condition: (cast(cs_ext_discount_amt#20 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) +Right keys [1]: [cs_item_sk#4] +Join condition: (cast(cs_ext_discount_amt#5 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) (27) Project [codegen id : 6] -Output [1]: [cs_sold_date_sk#21] -Input [5]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] +Output [1]: [cs_sold_date_sk#6] +Input [5]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] (28) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#8] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [cs_sold_date_sk#6] +Right keys [1]: [d_date_sk#8] Join condition: None (30) Project [codegen id : 6] -Output [1]: [1 AS excess discount amount #23] -Input [2]: [cs_sold_date_sk#21, d_date_sk#22] +Output [1]: [1 AS excess discount amount #20] +Input [2]: [cs_sold_date_sk#6, d_date_sk#8] (31) CollectLimit -Input [1]: [excess discount amount #23] +Input [1]: [excess discount amount #20] Arguments: 100 ===== Subqueries ===== @@ -184,6 +184,6 @@ ReusedExchange (32) (32) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#6 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt index 303bdf58604bf..d885ad3178181 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt @@ -18,7 +18,7 @@ CollectLimit InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] Filter [(1.3 * avg(cs_ext_discount_amt))] - HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] + HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,sum,count] InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index 0f35745fc2144..9537689459170 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -79,105 +79,100 @@ Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] (11) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] +Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] +Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] (13) Filter [codegen id : 3] -Input [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] -Condition : isnotnull(cs_item_sk#8) +Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] +Condition : isnotnull(cs_item_sk#1) (14) Scan parquet default.date_dim -Output [2]: [d_date_sk#11, d_date#12] +Output [2]: [d_date_sk#8, d_date#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#11, d_date#12] +Input [2]: [d_date_sk#8, d_date#9] (16) Filter [codegen id : 2] -<<<<<<< HEAD -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) -======= Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) ->>>>>>> abfd9b23cd7c21e9525df85a16e0611ef0f35908 +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) (17) Project [codegen id : 2] -Output [1]: [d_date_sk#11] -Input [2]: [d_date_sk#11, d_date#12] +Output [1]: [d_date_sk#8] +Input [2]: [d_date_sk#8, d_date#9] (18) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] +Input [1]: [d_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] (19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#10] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [cs_sold_date_sk#3] +Right keys [1]: [d_date_sk#8] Join condition: None (20) Project [codegen id : 3] -Output [2]: [cs_item_sk#8, cs_ext_discount_amt#9] -Input [4]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10, d_date_sk#11] +Output [2]: [cs_item_sk#1, cs_ext_discount_amt#2] +Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#8] (21) HashAggregate [codegen id : 3] -Input [2]: [cs_item_sk#8, cs_ext_discount_amt#9] -Keys [1]: [cs_item_sk#8] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#9))] -Aggregate Attributes [2]: [sum#14, count#15] -Results [3]: [cs_item_sk#8, sum#16, count#17] +Input [2]: [cs_item_sk#1, cs_ext_discount_amt#2] +Keys [1]: [cs_item_sk#1] +Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#2))] +Aggregate Attributes [2]: [sum#11, count#12] +Results [3]: [cs_item_sk#1, sum#13, count#14] (22) Exchange -Input [3]: [cs_item_sk#8, sum#16, count#17] -Arguments: hashpartitioning(cs_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [cs_item_sk#1, sum#13, count#14] +Arguments: hashpartitioning(cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#15] (23) HashAggregate [codegen id : 4] -Input [3]: [cs_item_sk#8, sum#16, count#17] -Keys [1]: [cs_item_sk#8] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#9))] -Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#9))#19] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#9))#19 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] +Input [3]: [cs_item_sk#1, sum#13, count#14] +Keys [1]: [cs_item_sk#1] +Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#2))] +Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#2))#16] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#2))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1 AS cs_item_sk#1#18] (24) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#20) +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] +Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) (25) BroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#21] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#19] (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] -Right keys [1]: [cs_item_sk#8] -Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#20) +Right keys [1]: [cs_item_sk#1#18] +Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) (27) Project [codegen id : 6] Output [1]: [cs_sold_date_sk#3] -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] +Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] (28) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#8] (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#8] Join condition: None (30) Project [codegen id : 6] -Output [1]: [1 AS excess discount amount #23] -Input [2]: [cs_sold_date_sk#3, d_date_sk#22] +Output [1]: [1 AS excess discount amount #20] +Input [2]: [cs_sold_date_sk#3, d_date_sk#8] (31) CollectLimit -Input [1]: [excess discount amount #23] +Input [1]: [excess discount amount #20] Arguments: 100 ===== Subqueries ===== @@ -187,8 +182,8 @@ ReusedExchange (32) (32) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index f3dd6d0954046..563281eef37bc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -24,7 +24,7 @@ CollectLimit BroadcastExchange #3 WholeStageCodegen (4) Filter [(1.3 * avg(cs_ext_discount_amt))] - HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] + HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,sum,count] InputAdapter Exchange [cs_item_sk] #4 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index 1349aa54f3cc2..cba099c2bb5b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -163,7 +163,7 @@ Condition : isnotnull(i_item_sk#13) Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics)) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index 1349aa54f3cc2..cba099c2bb5b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -163,7 +163,7 @@ Condition : isnotnull(i_item_sk#13) Output [2]: [i_category#15, i_manufact_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] @@ -171,7 +171,7 @@ Input [2]: [i_category#15, i_manufact_id#14] (23) Filter [codegen id : 3] Input [2]: [i_category#15, i_manufact_id#14] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) +Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics)) (24) Project [codegen id : 3] Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] 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 68ab7a8bbecab..a53bec582637d 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 @@ -176,7 +176,7 @@ Results [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stddev_samp(cast(inv_qua (26) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stdev#28, mean#29] -Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND (NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0))) +Condition : (((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) (27) Project [codegen id : 5] Output [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -280,7 +280,7 @@ Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stddev_samp(cast(inv_qu (49) Filter [codegen id : 11] Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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))) +Condition : (((isnotnull(mean#54) AND isnotnull(stdev#53)) AND NOT (mean#54 = 0.0)) AND ((stdev#53 / mean#54) > 1.0)) (50) Project [codegen id : 11] Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, CASE WHEN (mean#54 = 0.0) THEN null ELSE (stdev#53 / mean#54) END AS cov#55] 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 7abd3bb1acb11..c61480a05a4cf 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 @@ -173,7 +173,7 @@ Results [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stddev_samp(cast(inv_quan (26) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] -Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND (NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0))) +Condition : (((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) (27) Project [codegen id : 10] Output [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -269,7 +269,7 @@ Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_qu (47) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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))) +Condition : (((isnotnull(mean#53) AND isnotnull(stdev#52)) AND NOT (mean#53 = 0.0)) AND ((stdev#52 / mean#53) > 1.0)) (48) Project [codegen id : 9] Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] 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 f0dfbac4f3a0f..6bc5980231252 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 @@ -176,7 +176,7 @@ Results [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stddev_samp(cast(inv_qua (26) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, stdev#28, mean#29] -Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND ((NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0)) AND ((stdev#28 / mean#29) > 1.5))) +Condition : (((((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.5)) (27) Project [codegen id : 5] Output [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -280,7 +280,7 @@ Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stddev_samp(cast(inv_qu (49) Filter [codegen id : 11] Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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))) +Condition : (((isnotnull(mean#54) AND isnotnull(stdev#53)) AND NOT (mean#54 = 0.0)) AND ((stdev#53 / mean#54) > 1.0)) (50) Project [codegen id : 11] Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, CASE WHEN (mean#54 = 0.0) THEN null ELSE (stdev#53 / mean#54) END AS cov#55] 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 d1a5ecaa6a4d8..0ffe40240be2b 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 @@ -173,7 +173,7 @@ Results [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stddev_samp(cast(inv_quan (26) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] -Condition : ((isnotnull(mean#29) AND isnotnull(stdev#28)) AND ((NOT (mean#29 = 0.0) AND ((stdev#28 / mean#29) > 1.0)) AND ((stdev#28 / mean#29) > 1.5))) +Condition : (((((isnotnull(mean#29) AND isnotnull(stdev#28)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.0)) AND NOT (mean#29 = 0.0)) AND ((stdev#28 / mean#29) > 1.5)) (27) Project [codegen id : 10] Output [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, CASE WHEN (mean#29 = 0.0) THEN null ELSE (stdev#28 / mean#29) END AS cov#30] @@ -269,7 +269,7 @@ Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_qu (47) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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))) +Condition : (((isnotnull(mean#53) AND isnotnull(stdev#52)) AND NOT (mean#53 = 0.0)) AND ((stdev#52 / mean#53) > 1.0)) (48) Project [codegen id : 9] Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt index 0c227d75c67c4..13d73e61e1443 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt @@ -40,66 +40,61 @@ Output [2]: [i_manufact#2, i_product_name#3] Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] (5) Scan parquet default.item -Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Output [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [Or(Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,powder),EqualTo(i_color,khaki)),Or(EqualTo(i_units,Ounce),EqualTo(i_units,Oz))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,brown),EqualTo(i_color,honeydew)),Or(EqualTo(i_units,Bunch),EqualTo(i_units,Ton))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,floral),EqualTo(i_color,deep)),Or(EqualTo(i_units,N/A),EqualTo(i_units,Dozen))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,light),EqualTo(i_color,cornflower)),Or(EqualTo(i_units,Box),EqualTo(i_units,Pound))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large)))))),Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,midnight),EqualTo(i_color,snow)),Or(EqualTo(i_units,Pallet),EqualTo(i_units,Gross))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,cyan),EqualTo(i_color,papaya)),Or(EqualTo(i_units,Cup),EqualTo(i_units,Dram))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,orange),EqualTo(i_color,frosted)),Or(EqualTo(i_units,Each),EqualTo(i_units,Tbl))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,forest),EqualTo(i_color,ghost)),Or(EqualTo(i_units,Lb),EqualTo(i_units,Bundle))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))))))), IsNotNull(i_manufact)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] (7) Filter [codegen id : 1] -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Condition : (((((i_category#4 = Women) AND (((((i_color#7 = powder) OR (i_color#7 = khaki)) AND ((i_units#8 = Ounce) OR (i_units#8 = Oz))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = brown) OR (i_color#7 = honeydew)) AND ((i_units#8 = Bunch) OR (i_units#8 = Ton))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = floral) OR (i_color#7 = deep)) AND ((i_units#8 = N/A) OR (i_units#8 = Dozen))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = light) OR (i_color#7 = cornflower)) AND ((i_units#8 = Box) OR (i_units#8 = Pound))) AND ((i_size#6 = medium) OR (i_size#6 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#7 = midnight) OR (i_color#7 = snow)) AND ((i_units#8 = Pallet) OR (i_units#8 = Gross))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = cyan) OR (i_color#7 = papaya)) AND ((i_units#8 = Cup) OR (i_units#8 = Dram))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = orange) OR (i_color#7 = frosted)) AND ((i_units#8 = Each) OR (i_units#8 = Tbl))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = forest) OR (i_color#7 = ghost)) AND ((i_units#8 = Lb) OR (i_units#8 = Bundle))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))))))) AND isnotnull(i_manufact#5)) +Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Condition : (((((i_category#4 = Women) AND (((((i_color#6 = powder) OR (i_color#6 = khaki)) AND ((i_units#7 = Ounce) OR (i_units#7 = Oz))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = brown) OR (i_color#6 = honeydew)) AND ((i_units#7 = Bunch) OR (i_units#7 = Ton))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = floral) OR (i_color#6 = deep)) AND ((i_units#7 = N/A) OR (i_units#7 = Dozen))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = light) OR (i_color#6 = cornflower)) AND ((i_units#7 = Box) OR (i_units#7 = Pound))) AND ((i_size#5 = medium) OR (i_size#5 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#6 = midnight) OR (i_color#6 = snow)) AND ((i_units#7 = Pallet) OR (i_units#7 = Gross))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = cyan) OR (i_color#6 = papaya)) AND ((i_units#7 = Cup) OR (i_units#7 = Dram))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = orange) OR (i_color#6 = frosted)) AND ((i_units#7 = Each) OR (i_units#7 = Tbl))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = forest) OR (i_color#6 = ghost)) AND ((i_units#7 = Lb) OR (i_units#7 = Bundle))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))))))) AND isnotnull(i_manufact#2)) (8) Project [codegen id : 1] -Output [1]: [i_manufact#5] -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Output [1]: [i_manufact#2] +Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] (9) HashAggregate [codegen id : 1] -Input [1]: [i_manufact#5] -Keys [1]: [i_manufact#5] +Input [1]: [i_manufact#2] +Keys [1]: [i_manufact#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#9] -Results [2]: [i_manufact#5, count#10] +Aggregate Attributes [1]: [count#8] +Results [2]: [i_manufact#2, count#9] (10) Exchange -<<<<<<< HEAD -Input [2]: [i_manufact#5, count#10] -Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [id=#11] -======= Input [2]: [i_manufact#2, count#9] -Arguments: hashpartitioning(i_manufact#2, 5), ENSURE_REQUIREMENTS, [id=#10] ->>>>>>> abfd9b23cd7c21e9525df85a16e0611ef0f35908 +Arguments: hashpartitioning(i_manufact#2, 5), true, [id=#10] (11) HashAggregate [codegen id : 2] -Input [2]: [i_manufact#5, count#10] -Keys [1]: [i_manufact#5] +Input [2]: [i_manufact#2, count#9] +Keys [1]: [i_manufact#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#12] -Results [2]: [count(1)#12 AS item_cnt#13, i_manufact#5] +Aggregate Attributes [1]: [count(1)#11] +Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13] (12) Filter [codegen id : 2] -Input [2]: [item_cnt#13, i_manufact#5] -Condition : (item_cnt#13 > 0) +Input [2]: [item_cnt#12, i_manufact#2#13] +Condition : (item_cnt#12 > 0) (13) Project [codegen id : 2] -Output [1]: [i_manufact#5] -Input [2]: [item_cnt#13, i_manufact#5] +Output [1]: [i_manufact#2#13] +Input [2]: [item_cnt#12, i_manufact#2#13] (14) BroadcastExchange -Input [1]: [i_manufact#5] +Input [1]: [i_manufact#2#13] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14] (15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_manufact#2] -Right keys [1]: [i_manufact#5] +Right keys [1]: [i_manufact#2#13] Join condition: None (16) Project [codegen id : 3] Output [1]: [i_product_name#3] -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] +Input [3]: [i_manufact#2, i_product_name#3, i_manufact#2#13] (17) HashAggregate [codegen id : 3] Input [1]: [i_product_name#3] @@ -110,7 +105,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [id=#15] +Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt index d36800823bb3f..2d14d75ca9362 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (2) Project [i_manufact] Filter [item_cnt] - HashAggregate [i_manufact,count] [count(1),item_cnt,count] + HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,count] InputAdapter Exchange [i_manufact] #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index 0c227d75c67c4..13d73e61e1443 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -40,66 +40,61 @@ Output [2]: [i_manufact#2, i_product_name#3] Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] (5) Scan parquet default.item -Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Output [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [Or(Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,powder),EqualTo(i_color,khaki)),Or(EqualTo(i_units,Ounce),EqualTo(i_units,Oz))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,brown),EqualTo(i_color,honeydew)),Or(EqualTo(i_units,Bunch),EqualTo(i_units,Ton))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,floral),EqualTo(i_color,deep)),Or(EqualTo(i_units,N/A),EqualTo(i_units,Dozen))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,light),EqualTo(i_color,cornflower)),Or(EqualTo(i_units,Box),EqualTo(i_units,Pound))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large)))))),Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,midnight),EqualTo(i_color,snow)),Or(EqualTo(i_units,Pallet),EqualTo(i_units,Gross))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,cyan),EqualTo(i_color,papaya)),Or(EqualTo(i_units,Cup),EqualTo(i_units,Dram))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,orange),EqualTo(i_color,frosted)),Or(EqualTo(i_units,Each),EqualTo(i_units,Tbl))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,forest),EqualTo(i_color,ghost)),Or(EqualTo(i_units,Lb),EqualTo(i_units,Bundle))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))))))), IsNotNull(i_manufact)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] (7) Filter [codegen id : 1] -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] -Condition : (((((i_category#4 = Women) AND (((((i_color#7 = powder) OR (i_color#7 = khaki)) AND ((i_units#8 = Ounce) OR (i_units#8 = Oz))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = brown) OR (i_color#7 = honeydew)) AND ((i_units#8 = Bunch) OR (i_units#8 = Ton))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = floral) OR (i_color#7 = deep)) AND ((i_units#8 = N/A) OR (i_units#8 = Dozen))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = light) OR (i_color#7 = cornflower)) AND ((i_units#8 = Box) OR (i_units#8 = Pound))) AND ((i_size#6 = medium) OR (i_size#6 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#7 = midnight) OR (i_color#7 = snow)) AND ((i_units#8 = Pallet) OR (i_units#8 = Gross))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))) OR ((((i_color#7 = cyan) OR (i_color#7 = papaya)) AND ((i_units#8 = Cup) OR (i_units#8 = Dram))) AND ((i_size#6 = N/A) OR (i_size#6 = small))))) OR ((i_category#4 = Men) AND (((((i_color#7 = orange) OR (i_color#7 = frosted)) AND ((i_units#8 = Each) OR (i_units#8 = Tbl))) AND ((i_size#6 = petite) OR (i_size#6 = large))) OR ((((i_color#7 = forest) OR (i_color#7 = ghost)) AND ((i_units#8 = Lb) OR (i_units#8 = Bundle))) AND ((i_size#6 = medium) OR (i_size#6 = extra large))))))) AND isnotnull(i_manufact#5)) +Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Condition : (((((i_category#4 = Women) AND (((((i_color#6 = powder) OR (i_color#6 = khaki)) AND ((i_units#7 = Ounce) OR (i_units#7 = Oz))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = brown) OR (i_color#6 = honeydew)) AND ((i_units#7 = Bunch) OR (i_units#7 = Ton))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = floral) OR (i_color#6 = deep)) AND ((i_units#7 = N/A) OR (i_units#7 = Dozen))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = light) OR (i_color#6 = cornflower)) AND ((i_units#7 = Box) OR (i_units#7 = Pound))) AND ((i_size#5 = medium) OR (i_size#5 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#6 = midnight) OR (i_color#6 = snow)) AND ((i_units#7 = Pallet) OR (i_units#7 = Gross))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = cyan) OR (i_color#6 = papaya)) AND ((i_units#7 = Cup) OR (i_units#7 = Dram))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = orange) OR (i_color#6 = frosted)) AND ((i_units#7 = Each) OR (i_units#7 = Tbl))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = forest) OR (i_color#6 = ghost)) AND ((i_units#7 = Lb) OR (i_units#7 = Bundle))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))))))) AND isnotnull(i_manufact#2)) (8) Project [codegen id : 1] -Output [1]: [i_manufact#5] -Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Output [1]: [i_manufact#2] +Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] (9) HashAggregate [codegen id : 1] -Input [1]: [i_manufact#5] -Keys [1]: [i_manufact#5] +Input [1]: [i_manufact#2] +Keys [1]: [i_manufact#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#9] -Results [2]: [i_manufact#5, count#10] +Aggregate Attributes [1]: [count#8] +Results [2]: [i_manufact#2, count#9] (10) Exchange -<<<<<<< HEAD -Input [2]: [i_manufact#5, count#10] -Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [id=#11] -======= Input [2]: [i_manufact#2, count#9] -Arguments: hashpartitioning(i_manufact#2, 5), ENSURE_REQUIREMENTS, [id=#10] ->>>>>>> abfd9b23cd7c21e9525df85a16e0611ef0f35908 +Arguments: hashpartitioning(i_manufact#2, 5), true, [id=#10] (11) HashAggregate [codegen id : 2] -Input [2]: [i_manufact#5, count#10] -Keys [1]: [i_manufact#5] +Input [2]: [i_manufact#2, count#9] +Keys [1]: [i_manufact#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#12] -Results [2]: [count(1)#12 AS item_cnt#13, i_manufact#5] +Aggregate Attributes [1]: [count(1)#11] +Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13] (12) Filter [codegen id : 2] -Input [2]: [item_cnt#13, i_manufact#5] -Condition : (item_cnt#13 > 0) +Input [2]: [item_cnt#12, i_manufact#2#13] +Condition : (item_cnt#12 > 0) (13) Project [codegen id : 2] -Output [1]: [i_manufact#5] -Input [2]: [item_cnt#13, i_manufact#5] +Output [1]: [i_manufact#2#13] +Input [2]: [item_cnt#12, i_manufact#2#13] (14) BroadcastExchange -Input [1]: [i_manufact#5] +Input [1]: [i_manufact#2#13] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14] (15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_manufact#2] -Right keys [1]: [i_manufact#5] +Right keys [1]: [i_manufact#2#13] Join condition: None (16) Project [codegen id : 3] Output [1]: [i_product_name#3] -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] +Input [3]: [i_manufact#2, i_product_name#3, i_manufact#2#13] (17) HashAggregate [codegen id : 3] Input [1]: [i_product_name#3] @@ -110,7 +105,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [id=#15] +Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index d36800823bb3f..2d14d75ca9362 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (2) Project [i_manufact] Filter [item_cnt] - HashAggregate [i_manufact,count] [count(1),item_cnt,count] + HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,count] InputAdapter Exchange [i_manufact] #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt index fdb95bd5de9d3..35924d2ffd9a9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt @@ -95,7 +95,7 @@ Arguments: [rank(rank_col#12) windowspecdefinition(rank_col#12 ASC NULLS FIRST, (13) Filter [codegen id : 10] Input [3]: [item_sk#11, rank_col#12, rnk#17] -Condition : ((rnk#17 < 11) AND isnotnull(item_sk#11)) +Condition : ((isnotnull(rnk#17) AND (rnk#17 < 11)) AND isnotnull(item_sk#11)) (14) Project [codegen id : 10] Output [2]: [item_sk#11, rnk#17] @@ -133,7 +133,7 @@ Arguments: [rank(rank_col#22) windowspecdefinition(rank_col#22 DESC NULLS LAST, (22) Filter [codegen id : 7] Input [3]: [item_sk#21, rank_col#22, rnk#25] -Condition : ((rnk#25 < 11) AND isnotnull(item_sk#21)) +Condition : ((isnotnull(rnk#25) AND (rnk#25 < 11)) AND isnotnull(item_sk#21)) (23) Project [codegen id : 7] Output [2]: [item_sk#21, rnk#25] @@ -141,7 +141,7 @@ Input [3]: [item_sk#21, rank_col#22, rnk#25] (24) BroadcastExchange Input [2]: [item_sk#21, rnk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#26] (25) BroadcastHashJoin [codegen id : 10] Left keys [1]: [rnk#17] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index f52ad2e63762e..4ca133ffd838d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -96,7 +96,7 @@ Arguments: [rank(rank_col#12) windowspecdefinition(rank_col#12 ASC NULLS FIRST, (13) Filter [codegen id : 4] Input [3]: [item_sk#11, rank_col#12, rnk#17] -Condition : ((rnk#17 < 11) AND isnotnull(item_sk#11)) +Condition : ((isnotnull(rnk#17) AND (rnk#17 < 11)) AND isnotnull(item_sk#11)) (14) Project [codegen id : 4] Output [2]: [item_sk#11, rnk#17] @@ -138,7 +138,7 @@ Arguments: [rank(rank_col#22) windowspecdefinition(rank_col#22 DESC NULLS LAST, (23) Filter [codegen id : 8] Input [3]: [item_sk#21, rank_col#22, rnk#25] -Condition : ((rnk#25 < 11) AND isnotnull(item_sk#21)) +Condition : ((isnotnull(rnk#25) AND (rnk#25 < 11)) AND isnotnull(item_sk#21)) (24) Project [codegen id : 8] Output [2]: [item_sk#21, rnk#25] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index f6dd5a59de4d4..ee3abb39ed053 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -467,7 +467,7 @@ Input [2]: [d_date#5, d_week_seq#6] (82) Filter [codegen id : 1] Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_date#5) AND (d_date#5 = 2000-01-03)) +Condition : (isnotnull(d_date#5) AND (d_date#5 = 10959)) (83) Project [codegen id : 1] Output [1]: [d_week_seq#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index ab4c4ad4ae65b..dd37b8801e957 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -467,7 +467,7 @@ Input [2]: [d_date#8, d_week_seq#9] (82) Filter [codegen id : 1] Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_date#8) AND (d_date#8 = 2000-01-03)) +Condition : (isnotnull(d_date#8) AND (d_date#8 = 10959)) (83) Project [codegen id : 1] Output [1]: [d_week_seq#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt index 7e82b4d5df296..4c5b5f25108e0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt @@ -81,57 +81,57 @@ Input [3]: [i_item_sk#5, i_current_price#6, i_category#7] Condition : (isnotnull(i_current_price#6) AND isnotnull(i_item_sk#5)) (7) Scan parquet default.item -Output [2]: [i_current_price#8, i_category#9] +Output [2]: [i_current_price#6, i_category#7] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [i_current_price#8, i_category#9] +Input [2]: [i_current_price#6, i_category#7] (9) Filter [codegen id : 1] -Input [2]: [i_current_price#8, i_category#9] -Condition : isnotnull(i_category#9) +Input [2]: [i_current_price#6, i_category#7] +Condition : isnotnull(i_category#7) (10) HashAggregate [codegen id : 1] -Input [2]: [i_current_price#8, i_category#9] -Keys [1]: [i_category#9] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#8))] -Aggregate Attributes [2]: [sum#10, count#11] -Results [3]: [i_category#9, sum#12, count#13] +Input [2]: [i_current_price#6, i_category#7] +Keys [1]: [i_category#7] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#6))] +Aggregate Attributes [2]: [sum#8, count#9] +Results [3]: [i_category#7, sum#10, count#11] (11) Exchange -Input [3]: [i_category#9, sum#12, count#13] -Arguments: hashpartitioning(i_category#9, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [3]: [i_category#7, sum#10, count#11] +Arguments: hashpartitioning(i_category#7, 5), ENSURE_REQUIREMENTS, [id=#12] (12) HashAggregate [codegen id : 2] -Input [3]: [i_category#9, sum#12, count#13] -Keys [1]: [i_category#9] -Functions [1]: [avg(UnscaledValue(i_current_price#8))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#8))#15] -Results [2]: [cast((avg(UnscaledValue(i_current_price#8))#15 / 100.0) as decimal(11,6)) AS avg(i_current_price)#16, i_category#9] +Input [3]: [i_category#7, sum#10, count#11] +Keys [1]: [i_category#7] +Functions [1]: [avg(UnscaledValue(i_current_price#6))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#6))#13] +Results [2]: [cast((avg(UnscaledValue(i_current_price#6))#13 / 100.0) as decimal(11,6)) AS avg(i_current_price)#14, i_category#7 AS i_category#7#15] (13) BroadcastExchange -Input [2]: [avg(i_current_price)#16, i_category#9] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#17] +Input [2]: [avg(i_current_price)#14, i_category#7#15] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#16] (14) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_category#7] -Right keys [1]: [i_category#9] +Right keys [1]: [i_category#7#15] Join condition: None (15) Filter [codegen id : 3] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] -Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#16)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] +Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#14)), DecimalType(14,7), true)) (16) Project [codegen id : 3] Output [1]: [i_item_sk#5] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] (17) BroadcastExchange Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (18) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#1] @@ -143,143 +143,143 @@ Output [2]: [ss_customer_sk#2, ss_sold_date_sk#3] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, i_item_sk#5] (20) Scan parquet default.date_dim -Output [2]: [d_date_sk#19, d_month_seq#20] +Output [2]: [d_date_sk#18, d_month_seq#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#19, d_month_seq#20] +Input [2]: [d_date_sk#18, d_month_seq#19] (22) Filter [codegen id : 4] -Input [2]: [d_date_sk#19, d_month_seq#20] -Condition : ((isnotnull(d_month_seq#20) AND (d_month_seq#20 = Subquery scalar-subquery#21, [id=#22])) AND isnotnull(d_date_sk#19)) +Input [2]: [d_date_sk#18, d_month_seq#19] +Condition : ((isnotnull(d_month_seq#19) AND (d_month_seq#19 = Subquery scalar-subquery#20, [id=#21])) AND isnotnull(d_date_sk#18)) (23) Project [codegen id : 4] -Output [1]: [d_date_sk#19] -Input [2]: [d_date_sk#19, d_month_seq#20] +Output [1]: [d_date_sk#18] +Input [2]: [d_date_sk#18, d_month_seq#19] (24) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [d_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (25) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#19] +Right keys [1]: [d_date_sk#18] Join condition: None (26) Project [codegen id : 5] Output [1]: [ss_customer_sk#2] -Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#19] +Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#18] (27) Exchange Input [1]: [ss_customer_sk#2] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#23] (28) Sort [codegen id : 6] Input [1]: [ss_customer_sk#2] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#25, ca_state#26] +Output [2]: [ca_address_sk#24, ca_state#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#25, ca_state#26] +Input [2]: [ca_address_sk#24, ca_state#25] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#25, ca_state#26] -Condition : isnotnull(ca_address_sk#25) +Input [2]: [ca_address_sk#24, ca_state#25] +Condition : isnotnull(ca_address_sk#24) (32) Exchange -Input [2]: [ca_address_sk#25, ca_state#26] -Arguments: hashpartitioning(ca_address_sk#25, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] (33) Sort [codegen id : 8] -Input [2]: [ca_address_sk#25, ca_state#26] -Arguments: [ca_address_sk#25 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 (34) Scan parquet default.customer -Output [2]: [c_customer_sk#28, c_current_addr_sk#29] +Output [2]: [c_customer_sk#27, c_current_addr_sk#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 9] -Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Input [2]: [c_customer_sk#27, c_current_addr_sk#28] (36) Filter [codegen id : 9] -Input [2]: [c_customer_sk#28, c_current_addr_sk#29] -Condition : (isnotnull(c_current_addr_sk#29) AND isnotnull(c_customer_sk#28)) +Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Condition : (isnotnull(c_current_addr_sk#28) AND isnotnull(c_customer_sk#27)) (37) Exchange -Input [2]: [c_customer_sk#28, c_current_addr_sk#29] -Arguments: hashpartitioning(c_current_addr_sk#29, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Arguments: hashpartitioning(c_current_addr_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] (38) Sort [codegen id : 10] -Input [2]: [c_customer_sk#28, c_current_addr_sk#29] -Arguments: [c_current_addr_sk#29 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Arguments: [c_current_addr_sk#28 ASC NULLS FIRST], false, 0 (39) SortMergeJoin [codegen id : 11] -Left keys [1]: [ca_address_sk#25] -Right keys [1]: [c_current_addr_sk#29] +Left keys [1]: [ca_address_sk#24] +Right keys [1]: [c_current_addr_sk#28] Join condition: None (40) Project [codegen id : 11] -Output [2]: [ca_state#26, c_customer_sk#28] -Input [4]: [ca_address_sk#25, ca_state#26, c_customer_sk#28, c_current_addr_sk#29] +Output [2]: [ca_state#25, c_customer_sk#27] +Input [4]: [ca_address_sk#24, ca_state#25, c_customer_sk#27, c_current_addr_sk#28] (41) Exchange -Input [2]: [ca_state#26, c_customer_sk#28] -Arguments: hashpartitioning(c_customer_sk#28, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [2]: [ca_state#25, c_customer_sk#27] +Arguments: hashpartitioning(c_customer_sk#27, 5), ENSURE_REQUIREMENTS, [id=#30] (42) Sort [codegen id : 12] -Input [2]: [ca_state#26, c_customer_sk#28] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 +Input [2]: [ca_state#25, c_customer_sk#27] +Arguments: [c_customer_sk#27 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 13] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#28] +Right keys [1]: [c_customer_sk#27] Join condition: None (44) Project [codegen id : 13] -Output [1]: [ca_state#26] -Input [3]: [ss_customer_sk#2, ca_state#26, c_customer_sk#28] +Output [1]: [ca_state#25] +Input [3]: [ss_customer_sk#2, ca_state#25, c_customer_sk#27] (45) HashAggregate [codegen id : 13] -Input [1]: [ca_state#26] -Keys [1]: [ca_state#26] +Input [1]: [ca_state#25] +Keys [1]: [ca_state#25] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#32] -Results [2]: [ca_state#26, count#33] +Aggregate Attributes [1]: [count#31] +Results [2]: [ca_state#25, count#32] (46) Exchange -Input [2]: [ca_state#26, count#33] -Arguments: hashpartitioning(ca_state#26, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [2]: [ca_state#25, count#32] +Arguments: hashpartitioning(ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#33] (47) HashAggregate [codegen id : 14] -Input [2]: [ca_state#26, count#33] -Keys [1]: [ca_state#26] +Input [2]: [ca_state#25, count#32] +Keys [1]: [ca_state#25] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [3]: [ca_state#26 AS state#36, count(1)#35 AS cnt#37, count(1)#35 AS count(1)#38] +Aggregate Attributes [1]: [count(1)#34] +Results [3]: [ca_state#25 AS state#35, count(1)#34 AS cnt#36, count(1)#34 AS count(1)#37] (48) Filter [codegen id : 14] -Input [3]: [state#36, cnt#37, count(1)#38] -Condition : (count(1)#38 >= 10) +Input [3]: [state#35, cnt#36, count(1)#37] +Condition : (count(1)#37 >= 10) (49) Project [codegen id : 14] -Output [2]: [state#36, cnt#37] -Input [3]: [state#36, cnt#37, count(1)#38] +Output [2]: [state#35, cnt#36] +Input [3]: [state#35, cnt#36, count(1)#37] (50) TakeOrderedAndProject -Input [2]: [state#36, cnt#37] -Arguments: 100, [cnt#37 ASC NULLS FIRST], [state#36, cnt#37] +Input [2]: [state#35, cnt#36] +Arguments: 100, [cnt#36 ASC NULLS FIRST], [state#35, cnt#36] ===== Subqueries ===== @@ -288,9 +288,9 @@ ReusedExchange (51) (51) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#19] +Output [1]: [d_date_sk#18] -Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#21, [id=#22] +Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#20, [id=#21] * HashAggregate (58) +- Exchange (57) +- * HashAggregate (56) @@ -301,39 +301,39 @@ Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquer (52) Scan parquet default.date_dim -Output [3]: [d_month_seq#39, d_year#40, d_moy#41] +Output [3]: [d_month_seq#19, d_year#38, d_moy#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (53) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#39, d_year#40, d_moy#41] +Input [3]: [d_month_seq#19, d_year#38, d_moy#39] (54) Filter [codegen id : 1] -Input [3]: [d_month_seq#39, d_year#40, d_moy#41] -Condition : (((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2000)) AND (d_moy#41 = 1)) +Input [3]: [d_month_seq#19, d_year#38, d_moy#39] +Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) (55) Project [codegen id : 1] -Output [1]: [d_month_seq#39] -Input [3]: [d_month_seq#39, d_year#40, d_moy#41] +Output [1]: [d_month_seq#19] +Input [3]: [d_month_seq#19, d_year#38, d_moy#39] (56) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#39] -Keys [1]: [d_month_seq#39] +Input [1]: [d_month_seq#19] +Keys [1]: [d_month_seq#19] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#39] +Results [1]: [d_month_seq#19] (57) Exchange -Input [1]: [d_month_seq#39] -Arguments: hashpartitioning(d_month_seq#39, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [1]: [d_month_seq#19] +Arguments: hashpartitioning(d_month_seq#19, 5), ENSURE_REQUIREMENTS, [id=#40] (58) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#39] -Keys [1]: [d_month_seq#39] +Input [1]: [d_month_seq#19] +Keys [1]: [d_month_seq#19] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#39] +Results [1]: [d_month_seq#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt index 1cdb7a5df8e6f..a5097742ec146 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt @@ -38,7 +38,7 @@ TakeOrderedAndProject [cnt,state] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] InputAdapter Exchange [i_category] #6 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt index 0f98039fc0f7f..58fabaa8e44be 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt @@ -160,57 +160,57 @@ Input [3]: [i_item_sk#16, i_current_price#17, i_category#18] Condition : (isnotnull(i_current_price#17) AND isnotnull(i_item_sk#16)) (26) Scan parquet default.item -Output [2]: [i_current_price#19, i_category#20] +Output [2]: [i_current_price#17, i_category#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (27) ColumnarToRow [codegen id : 4] -Input [2]: [i_current_price#19, i_category#20] +Input [2]: [i_current_price#17, i_category#18] (28) Filter [codegen id : 4] -Input [2]: [i_current_price#19, i_category#20] -Condition : isnotnull(i_category#20) +Input [2]: [i_current_price#17, i_category#18] +Condition : isnotnull(i_category#18) (29) HashAggregate [codegen id : 4] -Input [2]: [i_current_price#19, i_category#20] -Keys [1]: [i_category#20] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#19))] -Aggregate Attributes [2]: [sum#21, count#22] -Results [3]: [i_category#20, sum#23, count#24] +Input [2]: [i_current_price#17, i_category#18] +Keys [1]: [i_category#18] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] +Aggregate Attributes [2]: [sum#19, count#20] +Results [3]: [i_category#18, sum#21, count#22] (30) Exchange -Input [3]: [i_category#20, sum#23, count#24] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [i_category#18, sum#21, count#22] +Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [id=#23] (31) HashAggregate [codegen id : 5] -Input [3]: [i_category#20, sum#23, count#24] -Keys [1]: [i_category#20] -Functions [1]: [avg(UnscaledValue(i_current_price#19))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#19))#26] -Results [2]: [cast((avg(UnscaledValue(i_current_price#19))#26 / 100.0) as decimal(11,6)) AS avg(i_current_price)#27, i_category#20] +Input [3]: [i_category#18, sum#21, count#22] +Keys [1]: [i_category#18] +Functions [1]: [avg(UnscaledValue(i_current_price#17))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#17))#24] +Results [2]: [cast((avg(UnscaledValue(i_current_price#17))#24 / 100.0) as decimal(11,6)) AS avg(i_current_price)#25, i_category#18 AS i_category#18#26] (32) BroadcastExchange -Input [2]: [avg(i_current_price)#27, i_category#20] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#28] +Input [2]: [avg(i_current_price)#25, i_category#18#26] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#27] (33) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_category#18] -Right keys [1]: [i_category#20] +Right keys [1]: [i_category#18#26] Join condition: None (34) Filter [codegen id : 6] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] -Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#27)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] +Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#25)), DecimalType(14,7), true)) (35) Project [codegen id : 6] Output [1]: [i_item_sk#16] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] (36) BroadcastExchange Input [1]: [i_item_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (37) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#6] @@ -225,31 +225,31 @@ Input [3]: [ca_state#2, ss_item_sk#6, i_item_sk#16] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#30] -Results [2]: [ca_state#2, count#31] +Aggregate Attributes [1]: [count#29] +Results [2]: [ca_state#2, count#30] (40) Exchange -Input [2]: [ca_state#2, count#31] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [2]: [ca_state#2, count#30] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#31] (41) HashAggregate [codegen id : 8] -Input [2]: [ca_state#2, count#31] +Input [2]: [ca_state#2, count#30] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#33] -Results [3]: [ca_state#2 AS state#34, count(1)#33 AS cnt#35, count(1)#33 AS count(1)#36] +Aggregate Attributes [1]: [count(1)#32] +Results [3]: [ca_state#2 AS state#33, count(1)#32 AS cnt#34, count(1)#32 AS count(1)#35] (42) Filter [codegen id : 8] -Input [3]: [state#34, cnt#35, count(1)#36] -Condition : (count(1)#36 >= 10) +Input [3]: [state#33, cnt#34, count(1)#35] +Condition : (count(1)#35 >= 10) (43) Project [codegen id : 8] -Output [2]: [state#34, cnt#35] -Input [3]: [state#34, cnt#35, count(1)#36] +Output [2]: [state#33, cnt#34] +Input [3]: [state#33, cnt#34, count(1)#35] (44) TakeOrderedAndProject -Input [2]: [state#34, cnt#35] -Arguments: 100, [cnt#35 ASC NULLS FIRST], [state#34, cnt#35] +Input [2]: [state#33, cnt#34] +Arguments: 100, [cnt#34 ASC NULLS FIRST], [state#33, cnt#34] ===== Subqueries ===== @@ -271,39 +271,39 @@ Subquery:2 Hosting operator id = 18 Hosting Expression = Subquery scalar-subquer (46) Scan parquet default.date_dim -Output [3]: [d_month_seq#37, d_year#38, d_moy#39] +Output [3]: [d_month_seq#12, d_year#36, d_moy#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (47) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#38, d_moy#39] +Input [3]: [d_month_seq#12, d_year#36, d_moy#37] (48) Filter [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#38, d_moy#39] -Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) +Input [3]: [d_month_seq#12, d_year#36, d_moy#37] +Condition : (((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2000)) AND (d_moy#37 = 1)) (49) Project [codegen id : 1] -Output [1]: [d_month_seq#37] -Input [3]: [d_month_seq#37, d_year#38, d_moy#39] +Output [1]: [d_month_seq#12] +Input [3]: [d_month_seq#12, d_year#36, d_moy#37] (50) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#37] -Keys [1]: [d_month_seq#37] +Input [1]: [d_month_seq#12] +Keys [1]: [d_month_seq#12] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#37] +Results [1]: [d_month_seq#12] (51) Exchange -Input [1]: [d_month_seq#37] -Arguments: hashpartitioning(d_month_seq#37, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [1]: [d_month_seq#12] +Arguments: hashpartitioning(d_month_seq#12, 5), ENSURE_REQUIREMENTS, [id=#38] (52) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#37] -Keys [1]: [d_month_seq#37] +Input [1]: [d_month_seq#12] +Keys [1]: [d_month_seq#12] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#37] +Results [1]: [d_month_seq#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index d7fad5948f64b..8b60c2f86a280 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -68,7 +68,7 @@ TakeOrderedAndProject [cnt,state] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] InputAdapter Exchange [i_category] #8 WholeStageCodegen (4) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index 9e2d33f2bca19..d31dbc3498ead 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -194,7 +194,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (ranking#19 <= 5) +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index cd12e362c44f0..26fec145f4211 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -194,7 +194,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (ranking#19 <= 5) +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#16] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt index cd88a84ede266..3daa8b66851f7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt @@ -180,7 +180,7 @@ Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#15, sold_item_sk#16, Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] +PushedFilters: [Or(EqualTo(t_meal_time,breakfast),EqualTo(t_meal_time,dinner)), IsNotNull(t_time_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] @@ -188,7 +188,7 @@ Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] (33) Filter [codegen id : 8] Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Condition : (((t_meal_time#35 = breakfast ) OR (t_meal_time#35 = dinner )) AND isnotnull(t_time_sk#32)) +Condition : (((t_meal_time#35 = breakfast) OR (t_meal_time#35 = dinner)) AND isnotnull(t_time_sk#32)) (34) Project [codegen id : 8] Output [3]: [t_time_sk#32, t_hour#33, t_minute#34] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index cd88a84ede266..3daa8b66851f7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt @@ -180,7 +180,7 @@ Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#15, sold_item_sk#16, Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] -PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] +PushedFilters: [Or(EqualTo(t_meal_time,breakfast),EqualTo(t_meal_time,dinner)), IsNotNull(t_time_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] @@ -188,7 +188,7 @@ Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] (33) Filter [codegen id : 8] Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Condition : (((t_meal_time#35 = breakfast ) OR (t_meal_time#35 = dinner )) AND isnotnull(t_time_sk#32)) +Condition : (((t_meal_time#35 = breakfast) OR (t_meal_time#35 = dinner)) AND isnotnull(t_time_sk#32)) (34) Project [codegen id : 8] Output [3]: [t_time_sk#32, t_hour#33, t_minute#34] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt index 8ae9d897a7d79..8297114e1e9ab 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt @@ -119,82 +119,82 @@ Input [3]: [ws_item_sk#4, sum#13, count#14] Keys [1]: [ws_item_sk#4] Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#5))] Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#5))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4 AS ws_item_sk#4#18] (19) Filter -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4#18] Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) (20) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#1] -Right keys [1]: [ws_item_sk#4] +Right keys [1]: [ws_item_sk#4#18] Join condition: None (21) Project [codegen id : 4] Output [2]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17] -Input [3]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4] +Input [3]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4#18] (22) BroadcastExchange Input [2]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (23) Scan parquet default.web_sales -Output [3]: [ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] +Output [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_ext_discount_amt)] ReadSchema: struct (24) ColumnarToRow -Input [3]: [ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] +Input [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] (25) Filter -Input [3]: [ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] -Condition : (isnotnull(ws_item_sk#19) AND isnotnull(ws_ext_discount_amt#20)) +Input [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] +Condition : (isnotnull(ws_item_sk#4) AND isnotnull(ws_ext_discount_amt#5)) (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#1] -Right keys [1]: [ws_item_sk#19] -Join condition: (cast(ws_ext_discount_amt#20 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) +Right keys [1]: [ws_item_sk#4] +Join condition: (cast(ws_ext_discount_amt#5 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) (27) Project [codegen id : 6] -Output [2]: [ws_ext_discount_amt#20, ws_sold_date_sk#21] -Input [5]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] +Output [2]: [ws_ext_discount_amt#5, ws_sold_date_sk#6] +Input [5]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] (28) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#8] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#21] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [ws_sold_date_sk#6] +Right keys [1]: [d_date_sk#8] Join condition: None (30) Project [codegen id : 6] -Output [1]: [ws_ext_discount_amt#20] -Input [3]: [ws_ext_discount_amt#20, ws_sold_date_sk#21, d_date_sk#22] +Output [1]: [ws_ext_discount_amt#5] +Input [3]: [ws_ext_discount_amt#5, ws_sold_date_sk#6, d_date_sk#8] (31) HashAggregate [codegen id : 6] -Input [1]: [ws_ext_discount_amt#20] +Input [1]: [ws_ext_discount_amt#5] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#20))] -Aggregate Attributes [1]: [sum#23] -Results [1]: [sum#24] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#5))] +Aggregate Attributes [1]: [sum#20] +Results [1]: [sum#21] (32) Exchange -Input [1]: [sum#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#25] +Input [1]: [sum#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] (33) HashAggregate [codegen id : 7] -Input [1]: [sum#24] +Input [1]: [sum#21] Keys: [] -Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#20))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#20))#26] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#20))#26,17,2) AS Excess Discount Amount #27] +Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#5))#23] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#5))#23,17,2) AS Excess Discount Amount #24] (34) Sort [codegen id : 7] -Input [1]: [Excess Discount Amount #27] -Arguments: [Excess Discount Amount #27 ASC NULLS FIRST], true, 0 +Input [1]: [Excess Discount Amount #24] +Arguments: [Excess Discount Amount #24 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -205,6 +205,6 @@ ReusedExchange (35) (35) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt index e83a3e67e5c6f..cc3ffa0de4bfd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt @@ -23,7 +23,7 @@ WholeStageCodegen (7) InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] Filter [(1.3 * avg(ws_ext_discount_amt))] - HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] + HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,sum,count] InputAdapter Exchange [ws_item_sk] #4 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index 7d525630d8a9a..faf82026138f6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -82,124 +82,119 @@ Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] (11) Scan parquet default.web_sales -Output [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] +Output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] +Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] (13) Filter [codegen id : 3] -Input [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] -Condition : isnotnull(ws_item_sk#8) +Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] +Condition : isnotnull(ws_item_sk#1) (14) Scan parquet default.date_dim -Output [2]: [d_date_sk#11, d_date#12] +Output [2]: [d_date_sk#8, d_date#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#11, d_date#12] +Input [2]: [d_date_sk#8, d_date#9] (16) Filter [codegen id : 2] -<<<<<<< HEAD -Input [2]: [d_date_sk#11, d_date#12] -Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) -======= Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) ->>>>>>> abfd9b23cd7c21e9525df85a16e0611ef0f35908 +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) (17) Project [codegen id : 2] -Output [1]: [d_date_sk#11] -Input [2]: [d_date_sk#11, d_date#12] +Output [1]: [d_date_sk#8] +Input [2]: [d_date_sk#8, d_date#9] (18) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] +Input [1]: [d_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] (19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#10] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#8] Join condition: None (20) Project [codegen id : 3] -Output [2]: [ws_item_sk#8, ws_ext_discount_amt#9] -Input [4]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10, d_date_sk#11] +Output [2]: [ws_item_sk#1, ws_ext_discount_amt#2] +Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#8] (21) HashAggregate [codegen id : 3] -Input [2]: [ws_item_sk#8, ws_ext_discount_amt#9] -Keys [1]: [ws_item_sk#8] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#9))] -Aggregate Attributes [2]: [sum#14, count#15] -Results [3]: [ws_item_sk#8, sum#16, count#17] +Input [2]: [ws_item_sk#1, ws_ext_discount_amt#2] +Keys [1]: [ws_item_sk#1] +Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#2))] +Aggregate Attributes [2]: [sum#11, count#12] +Results [3]: [ws_item_sk#1, sum#13, count#14] (22) Exchange -Input [3]: [ws_item_sk#8, sum#16, count#17] -Arguments: hashpartitioning(ws_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [ws_item_sk#1, sum#13, count#14] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#15] (23) HashAggregate [codegen id : 4] -Input [3]: [ws_item_sk#8, sum#16, count#17] -Keys [1]: [ws_item_sk#8] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#9))] -Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#9))#19] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#9))#19 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] +Input [3]: [ws_item_sk#1, sum#13, count#14] +Keys [1]: [ws_item_sk#1] +Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#2))] +Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#2))#16] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#2))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1 AS ws_item_sk#1#18] (24) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#20) +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] +Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) (25) BroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#21] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#19] (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] -Right keys [1]: [ws_item_sk#8] -Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#20) +Right keys [1]: [ws_item_sk#1#18] +Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) (27) Project [codegen id : 6] Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] +Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] (28) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#8] (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#8] Join condition: None (30) Project [codegen id : 6] Output [1]: [ws_ext_discount_amt#2] -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#22] +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#8] (31) HashAggregate [codegen id : 6] Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#23] -Results [1]: [sum#24] +Aggregate Attributes [1]: [sum#20] +Results [1]: [sum#21] (32) Exchange -Input [1]: [sum#24] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#25] +Input [1]: [sum#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] (33) HashAggregate [codegen id : 7] -Input [1]: [sum#24] +Input [1]: [sum#21] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#26] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#26,17,2) AS Excess Discount Amount #27] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#23] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#23,17,2) AS Excess Discount Amount #24] (34) Sort [codegen id : 7] -Input [1]: [Excess Discount Amount #27] -Arguments: [Excess Discount Amount #27 ASC NULLS FIRST], true, 0 +Input [1]: [Excess Discount Amount #24] +Arguments: [Excess Discount Amount #24 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -208,8 +203,8 @@ ReusedExchange (35) (35) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index 0df713c0e075e..3108c3bc82c7c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -29,7 +29,7 @@ WholeStageCodegen (7) BroadcastExchange #4 WholeStageCodegen (4) Filter [(1.3 * avg(ws_ext_discount_amt))] - HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] + HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,sum,count] InputAdapter Exchange [ws_item_sk] #5 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index c77590bf71044..bee7110ecd6dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -175,7 +175,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Output [2]: [web_site_sk#20, web_company_name#21] 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)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 10] @@ -183,7 +183,7 @@ Input [2]: [web_site_sk#20, web_company_name#21] (31) Filter [codegen id : 10] Input [2]: [web_site_sk#20, web_company_name#21] -Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri )) AND isnotnull(web_site_sk#20)) +Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri)) AND isnotnull(web_site_sk#20)) (32) Project [codegen id : 10] Output [1]: [web_site_sk#20] @@ -214,7 +214,7 @@ Input [2]: [d_date_sk#23, d_date#24] (38) Filter [codegen id : 11] Input [2]: [d_date_sk#23, d_date#24] -Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 1999-02-01)) AND (d_date#24 <= 1999-04-02)) AND isnotnull(d_date_sk#23)) +Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 10623)) AND (d_date#24 <= 10683)) AND isnotnull(d_date_sk#23)) (39) Project [codegen id : 11] Output [1]: [d_date_sk#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index 9558a01423452..efa09c2f625cd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -152,7 +152,7 @@ Input [2]: [d_date_sk#17, d_date#18] (24) Filter [codegen id : 9] Input [2]: [d_date_sk#17, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-01)) AND (d_date#18 <= 1999-04-02)) AND isnotnull(d_date_sk#17)) +Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 10623)) AND (d_date#18 <= 10683)) AND isnotnull(d_date_sk#17)) (25) Project [codegen id : 9] Output [1]: [d_date_sk#17] @@ -206,7 +206,7 @@ Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_ Output [2]: [web_site_sk#23, web_company_name#24] 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)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 11] @@ -214,7 +214,7 @@ Input [2]: [web_site_sk#23, web_company_name#24] (38) Filter [codegen id : 11] Input [2]: [web_site_sk#23, web_company_name#24] -Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri )) AND isnotnull(web_site_sk#23)) +Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri)) AND isnotnull(web_site_sk#23)) (39) Project [codegen id : 11] Output [1]: [web_site_sk#23] 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 320a93e19bb27..725939ef24f1b 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 @@ -252,7 +252,7 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num 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)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct (45) ColumnarToRow [codegen id : 19] @@ -260,7 +260,7 @@ Input [2]: [web_site_sk#24, web_company_name#25] (46) Filter [codegen id : 19] 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)) +Condition : ((isnotnull(web_company_name#25) AND (web_company_name#25 = pri)) AND isnotnull(web_site_sk#24)) (47) Project [codegen id : 19] Output [1]: [web_site_sk#24] @@ -291,7 +291,7 @@ Input [2]: [d_date_sk#27, d_date#28] (53) Filter [codegen id : 20] Input [2]: [d_date_sk#27, d_date#28] -Condition : (((isnotnull(d_date#28) AND (d_date#28 >= 1999-02-01)) AND (d_date#28 <= 1999-04-02)) AND isnotnull(d_date_sk#27)) +Condition : (((isnotnull(d_date#28) AND (d_date#28 >= 10623)) AND (d_date#28 <= 10683)) AND isnotnull(d_date_sk#27)) (54) Project [codegen id : 20] Output [1]: [d_date_sk#27] 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 2f719a4be1630..9f15375b5cfc0 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 @@ -229,7 +229,7 @@ Input [2]: [d_date_sk#21, d_date#22] (39) Filter [codegen id : 18] Input [2]: [d_date_sk#21, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-01)) AND (d_date#22 <= 1999-04-02)) AND isnotnull(d_date_sk#21)) +Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 10623)) AND (d_date#22 <= 10683)) AND isnotnull(d_date_sk#21)) (40) Project [codegen id : 18] Output [1]: [d_date_sk#21] @@ -283,7 +283,7 @@ Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_ Output [2]: [web_site_sk#27, web_company_name#28] 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)] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri), IsNotNull(web_site_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 20] @@ -291,7 +291,7 @@ Input [2]: [web_site_sk#27, web_company_name#28] (53) Filter [codegen id : 20] Input [2]: [web_site_sk#27, web_company_name#28] -Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri )) AND isnotnull(web_site_sk#27)) +Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri)) AND isnotnull(web_site_sk#27)) (54) Project [codegen id : 20] Output [1]: [web_site_sk#27] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt index 25baf22f2b4b5..312aba94bd9c6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt @@ -33,39 +33,39 @@ TakeOrderedAndProject (116) : : : : +- * HashAggregate (44) : : : : +- Exchange (43) : : : : +- * HashAggregate (42) - : : : : +- * Project (41) - : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : :- * Project (18) - : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : :- * Filter (11) - : : : : : : +- * ColumnarToRow (10) - : : : : : : +- Scan parquet default.store_sales (9) - : : : : : +- BroadcastExchange (16) - : : : : : +- * Project (15) - : : : : : +- * Filter (14) - : : : : : +- * ColumnarToRow (13) - : : : : : +- Scan parquet default.date_dim (12) - : : : : +- BroadcastExchange (39) - : : : : +- SortMergeJoin LeftSemi (38) - : : : : :- * Sort (23) - : : : : : +- Exchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (37) - : : : : +- Exchange (36) - : : : : +- * Project (35) - : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : :- * Project (29) - : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : :- * Filter (26) - : : : : : : +- * ColumnarToRow (25) - : : : : : : +- Scan parquet default.catalog_sales (24) - : : : : : +- ReusedExchange (27) - : : : : +- BroadcastExchange (33) - : : : : +- * Filter (32) - : : : : +- * ColumnarToRow (31) - : : : : +- Scan parquet default.item (30) + : : : : +- SortMergeJoin LeftSemi (41) + : : : : :- * Sort (26) + : : : : : +- Exchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (40) + : : : : +- Exchange (39) + : : : : +- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Filter (29) + : : : : : : +- * ColumnarToRow (28) + : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (36) + : : : : +- * Filter (35) + : : : : +- * ColumnarToRow (34) + : : : : +- Scan parquet default.item (33) : : : +- * Sort (57) : : : +- Exchange (56) : : : +- * Project (55) @@ -147,507 +147,507 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 19] +(7) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 19] +(8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 10] -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +(10) ColumnarToRow [codegen id : 5] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -(11) Filter [codegen id : 10] -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_item_sk#11) +(11) Filter [codegen id : 5] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#12, d_year#13] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#14, d_year#15] +Input [2]: [d_date_sk#12, d_year#13] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#14, d_year#15] -Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1998)) AND (d_year#15 <= 2000)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_year#15] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (16) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(17) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#14] +(17) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None -(18) Project [codegen id : 10] -Output [1]: [ss_item_sk#11] -Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] +(18) Project [codegen id : 5] +Output [1]: [ss_item_sk#1] +Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] (19) Scan parquet default.item -Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) + +(22) BroadcastExchange +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] + +(23) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#7] +Join condition: None + +(24) Project [codegen id : 5] +Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] +Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(22) Exchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] +(25) Exchange +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] -(23) Sort [codegen id : 5] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 +(26) Sort [codegen id : 6] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 -(24) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] +(27) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 8] -Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +(28) ColumnarToRow [codegen id : 9] +Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -(26) Filter [codegen id : 8] -Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] -Condition : isnotnull(cs_item_sk#22) +(29) Filter [codegen id : 9] +Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +Condition : isnotnull(cs_item_sk#20) -(27) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#24] +(30) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#12] -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] +(31) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#12] Join condition: None -(29) Project [codegen id : 8] -Output [1]: [cs_item_sk#22] -Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] +(32) Project [codegen id : 9] +Output [1]: [cs_item_sk#20] +Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] -(30) Scan parquet default.item -Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +(33) Scan parquet default.item +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 7] -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(32) Filter [codegen id : 7] -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -Condition : isnotnull(i_item_sk#25) +(35) Filter [codegen id : 8] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : isnotnull(i_item_sk#7) -(33) BroadcastExchange -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +(36) BroadcastExchange +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#22] -Right keys [1]: [i_item_sk#25] +(37) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [cs_item_sk#20] +Right keys [1]: [i_item_sk#7] Join condition: None -(35) Project [codegen id : 8] -Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] - -(36) Exchange -Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] +(38) Project [codegen id : 9] +Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(37) Sort [codegen id : 9] -Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 +(39) Exchange +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] -(38) SortMergeJoin -Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] -Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] -Join condition: None - -(39) BroadcastExchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +(40) Sort [codegen id : 10] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#11] -Right keys [1]: [i_item_sk#17] +(41) SortMergeJoin +Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] +Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] Join condition: None -(41) Project [codegen id : 10] -Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] -Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] - -(42) HashAggregate [codegen id : 10] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +(42) HashAggregate [codegen id : 11] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#16, class_id#17, category_id#18] (43) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] -(44) HashAggregate [codegen id : 11] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +(44) HashAggregate [codegen id : 12] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#16, class_id#17, category_id#18] (45) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] -(46) Sort [codegen id : 12] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 13] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 15] -Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +(48) ColumnarToRow [codegen id : 16] +Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -(49) Filter [codegen id : 15] -Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_item_sk#37) +(49) Filter [codegen id : 16] +Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Condition : isnotnull(ws_item_sk#26) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#39] +Output [1]: [d_date_sk#12] -(51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#39] +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None -(52) Project [codegen id : 15] -Output [1]: [ws_item_sk#37] -Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] +(52) Project [codegen id : 16] +Output [1]: [ws_item_sk#26] +Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] -(53) ReusedExchange [Reuses operator id: 33] -Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] +(53) ReusedExchange [Reuses operator id: 36] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(54) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#40] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [i_item_sk#7] Join condition: None -(55) Project [codegen id : 15] -Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] +(55) Project [codegen id : 16] +Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (56) Exchange -Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] -(57) Sort [codegen id : 16] -Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 17] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] -Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] +Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] +Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] Join condition: None -(59) HashAggregate [codegen id : 17] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +(59) HashAggregate [codegen id : 18] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#16, class_id#17, category_id#18] (60) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] -(61) HashAggregate [codegen id : 18] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +(61) HashAggregate [codegen id : 19] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#16, class_id#17, category_id#18] (62) BroadcastExchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] -(63) BroadcastHashJoin [codegen id : 19] +(63) BroadcastHashJoin [codegen id : 20] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#32, class_id#33, category_id#34] +Right keys [3]: [brand_id#16, class_id#17, category_id#18] Join condition: None -(64) Project [codegen id : 19] -Output [1]: [i_item_sk#7 AS ss_item_sk#47] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] +(64) Project [codegen id : 20] +Output [1]: [i_item_sk#7 AS ss_item_sk#31] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] (65) Exchange -Input [1]: [ss_item_sk#47] -Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [1]: [ss_item_sk#31] +Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] -(66) Sort [codegen id : 20] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 21] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#47] +Right keys [1]: [ss_item_sk#31] Join condition: None (68) Scan parquet default.date_dim -Output [2]: [d_date_sk#49, d_week_seq#50] +Output [2]: [d_date_sk#12, d_week_seq#33] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 21] -Input [2]: [d_date_sk#49, d_week_seq#50] +(69) ColumnarToRow [codegen id : 22] +Input [2]: [d_date_sk#12, d_week_seq#33] -(70) Filter [codegen id : 21] -Input [2]: [d_date_sk#49, d_week_seq#50] -Condition : ((isnotnull(d_week_seq#50) AND (d_week_seq#50 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#49)) +(70) Filter [codegen id : 22] +Input [2]: [d_date_sk#12, d_week_seq#33] +Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#34, [id=#35])) AND isnotnull(d_date_sk#12)) -(71) Project [codegen id : 21] -Output [1]: [d_date_sk#49] -Input [2]: [d_date_sk#49, d_week_seq#50] +(71) Project [codegen id : 22] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_week_seq#33] (72) BroadcastExchange -Input [1]: [d_date_sk#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] -(73) BroadcastHashJoin [codegen id : 42] +(73) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#49] +Right keys [1]: [d_date_sk#12] Join condition: None -(74) Project [codegen id : 42] +(74) Project [codegen id : 44] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (75) Scan parquet default.item -Output [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 22] -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +(76) ColumnarToRow [codegen id : 23] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(77) Filter [codegen id : 22] -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -Condition : (((isnotnull(i_item_sk#54) AND isnotnull(i_brand_id#55)) AND isnotnull(i_class_id#56)) AND isnotnull(i_category_id#57)) +(77) Filter [codegen id : 23] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (78) Exchange -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -Arguments: hashpartitioning(i_item_sk#54, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#37] -(79) Sort [codegen id : 23] -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -Arguments: [i_item_sk#54 ASC NULLS FIRST], false, 0 +(79) Sort [codegen id : 24] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#31] -(81) Sort [codegen id : 41] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 43] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#54] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [i_item_sk#7] +Right keys [1]: [ss_item_sk#31] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#59] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] -(84) BroadcastHashJoin [codegen id : 42] +(84) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#54] +Right keys [1]: [i_item_sk#7] Join condition: None -(85) Project [codegen id : 42] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +(85) Project [codegen id : 44] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(86) HashAggregate [codegen id : 42] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] -Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] +(86) HashAggregate [codegen id : 44] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#60, isEmpty#61, count#62] -Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] +Aggregate Attributes [3]: [sum#39, isEmpty#40, count#41] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] (87) Exchange -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] -Arguments: hashpartitioning(i_brand_id#55, i_class_id#56, i_category_id#57, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#45] -(88) HashAggregate [codegen id : 86] -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] -Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] +(88) HashAggregate [codegen id : 90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67, count(1)#68] -Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#69, count(1)#68 AS number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46, count(1)#47] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sales#48, count(1)#47 AS number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] -(89) Filter [codegen id : 86] -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(32,6)) > cast(Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) +(89) Filter [codegen id : 90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(90) Project [codegen id : 86] -Output [6]: [store AS channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70] -Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] +(90) Project [codegen id : 90] +Output [6]: [store AS channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] (91) Scan parquet default.store_sales -Output [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 43] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +(92) ColumnarToRow [codegen id : 45] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -(93) Filter [codegen id : 43] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Condition : isnotnull(ss_item_sk#75) +(93) Filter [codegen id : 45] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (94) Exchange -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Arguments: hashpartitioning(ss_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#55] -(95) Sort [codegen id : 44] -Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -Arguments: [ss_item_sk#75 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 46] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#31] -(97) Sort [codegen id : 62] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 65] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ss_item_sk#75] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [ss_item_sk#31] Join condition: None (99) Scan parquet default.date_dim -Output [2]: [d_date_sk#81, d_week_seq#82] +Output [2]: [d_date_sk#12, d_week_seq#33] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 63] -Input [2]: [d_date_sk#81, d_week_seq#82] +(100) ColumnarToRow [codegen id : 66] +Input [2]: [d_date_sk#12, d_week_seq#33] -(101) Filter [codegen id : 63] -Input [2]: [d_date_sk#81, d_week_seq#82] -Condition : ((isnotnull(d_week_seq#82) AND (d_week_seq#82 = Subquery scalar-subquery#83, [id=#84])) AND isnotnull(d_date_sk#81)) +(101) Filter [codegen id : 66] +Input [2]: [d_date_sk#12, d_week_seq#33] +Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#56, [id=#57])) AND isnotnull(d_date_sk#12)) -(102) Project [codegen id : 63] -Output [1]: [d_date_sk#81] -Input [2]: [d_date_sk#81, d_week_seq#82] +(102) Project [codegen id : 66] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_week_seq#33] (103) BroadcastExchange -Input [1]: [d_date_sk#81] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#85] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] -(104) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_sold_date_sk#78] -Right keys [1]: [d_date_sk#81] +(104) BroadcastHashJoin [codegen id : 88] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None -(105) Project [codegen id : 84] -Output [3]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77] -Input [5]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#81] +(105) Project [codegen id : 88] +Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (106) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] +Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(107) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#75] -Right keys [1]: [i_item_sk#86] +(107) BroadcastHashJoin [codegen id : 88] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#59] Join condition: None -(108) Project [codegen id : 84] -Output [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] -Input [7]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] +(108) Project [codegen id : 88] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] -(109) HashAggregate [codegen id : 84] -Input [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] -Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] -Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] +(109) HashAggregate [codegen id : 88] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] +Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65] +Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] (110) Exchange -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] -Arguments: hashpartitioning(i_brand_id#87, i_class_id#88, i_category_id#89, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] +Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [id=#69] -(111) HashAggregate [codegen id : 85] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] -Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] -Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#99, count(1)#98 AS number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] +(111) HashAggregate [codegen id : 89] +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] +Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70, count(1)#71] +Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sales#72, count(1)#71 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] -(112) Filter [codegen id : 85] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) +(112) Filter [codegen id : 89] +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) -(113) Project [codegen id : 85] -Output [6]: [store AS channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] +(113) Project [codegen id : 89] +Output [6]: [store AS channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] (114) BroadcastExchange -Input [6]: [channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#103] +Input [6]: [channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#76] -(115) BroadcastHashJoin [codegen id : 86] -Left keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] -Right keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +(115) BroadcastHashJoin [codegen id : 90] +Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] Join condition: None (116) TakeOrderedAndProject -Input [12]: [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] -Arguments: 100, [i_brand_id#55 ASC NULLS FIRST, i_class_id#56 ASC NULLS FIRST, i_category_id#57 ASC NULLS FIRST], [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Input [12]: [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#72, [id=#73] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#51, [id=#52] * HashAggregate (139) +- Exchange (138) +- * HashAggregate (137) @@ -674,140 +674,140 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (117) Scan parquet default.store_sales -Output [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] +Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#106), dynamicpruningexpression(ss_sold_date_sk#106 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#77)] ReadSchema: struct (118) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] +Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (119) Scan parquet default.date_dim -Output [2]: [d_date_sk#108, d_year#109] +Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (120) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#108, d_year#109] +Input [2]: [d_date_sk#12, d_year#13] (121) Filter [codegen id : 1] -Input [2]: [d_date_sk#108, d_year#109] -Condition : (((isnotnull(d_year#109) AND (d_year#109 >= 1998)) AND (d_year#109 <= 2000)) AND isnotnull(d_date_sk#108)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) (122) Project [codegen id : 1] -Output [1]: [d_date_sk#108] -Input [2]: [d_date_sk#108, d_year#109] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (123) BroadcastExchange -Input [1]: [d_date_sk#108] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#110] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] (124) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#106] -Right keys [1]: [d_date_sk#108] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (125) Project [codegen id : 2] -Output [2]: [ss_quantity#104 AS quantity#111, ss_list_price#105 AS list_price#112] -Input [4]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106, d_date_sk#108] +Output [2]: [ss_quantity#2 AS quantity#79, ss_list_price#3 AS list_price#80] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (126) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] +Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#115), dynamicpruningexpression(cs_sold_date_sk#115 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#77)] ReadSchema: struct (127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] +Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] (128) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#116] +Output [1]: [d_date_sk#12] (129) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#115] -Right keys [1]: [d_date_sk#116] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#12] Join condition: None (130) Project [codegen id : 4] -Output [2]: [cs_quantity#113 AS quantity#117, cs_list_price#114 AS list_price#118] -Input [4]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115, d_date_sk#116] +Output [2]: [cs_quantity#81 AS quantity#83, cs_list_price#82 AS list_price#84] +Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21, d_date_sk#12] (131) Scan parquet default.web_sales -Output [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] +Output [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#107)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#77)] ReadSchema: struct (132) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] +Input [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] (133) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#122] +Output [1]: [d_date_sk#12] (134) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#121] -Right keys [1]: [d_date_sk#122] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None (135) Project [codegen id : 6] -Output [2]: [ws_quantity#119 AS quantity#123, ws_list_price#120 AS list_price#124] -Input [4]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121, d_date_sk#122] +Output [2]: [ws_quantity#85 AS quantity#87, ws_list_price#86 AS list_price#88] +Input [4]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27, d_date_sk#12] (136) Union (137) HashAggregate [codegen id : 7] -Input [2]: [quantity#111, list_price#112] +Input [2]: [quantity#79, list_price#80] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#125, count#126] -Results [2]: [sum#127, count#128] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#89, count#90] +Results [2]: [sum#91, count#92] (138) Exchange -Input [2]: [sum#127, count#128] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] +Input [2]: [sum#91, count#92] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] (139) HashAggregate [codegen id : 8] -Input [2]: [sum#127, count#128] +Input [2]: [sum#91, count#92] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130 AS average_sales#131] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94 AS average_sales#95] -Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#106 IN dynamicpruning#107 +Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#77 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#108] +Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#115 IN dynamicpruning#107 +Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#77 -Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#107 +Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#77 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (141) (141) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#49] +Output [1]: [d_date_sk#12] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 ReusedExchange (142) (142) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 +Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 -Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#34, [id=#35] * Project (146) +- * Filter (145) +- * ColumnarToRow (144) @@ -815,33 +815,33 @@ Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquer (143) Scan parquet default.date_dim -Output [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] (145) Filter [codegen id : 1] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] -Condition : (((((isnotnull(d_year#133) AND isnotnull(d_moy#134)) AND isnotnull(d_dom#135)) AND (d_year#133 = 1999)) AND (d_moy#134 = 12)) AND (d_dom#135 = 16)) +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 1999)) AND (d_moy#96 = 12)) AND (d_dom#97 = 16)) (146) Project [codegen id : 1] -Output [1]: [d_week_seq#132] -Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Output [1]: [d_week_seq#33] +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] -Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] +Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] -Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#79 +Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#54 ReusedExchange (147) (147) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#81] +Output [1]: [d_date_sk#12] -Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#83, [id=#84] +Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#56, [id=#57] * Project (151) +- * Filter (150) +- * ColumnarToRow (149) @@ -849,21 +849,21 @@ Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subqu (148) Scan parquet default.date_dim -Output [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (149) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] (150) Filter [codegen id : 1] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] -Condition : (((((isnotnull(d_year#137) AND isnotnull(d_moy#138)) AND isnotnull(d_dom#139)) AND (d_year#137 = 1998)) AND (d_moy#138 = 12)) AND (d_dom#139 = 16)) +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 1998)) AND (d_moy#96 = 12)) AND (d_dom#97 = 16)) (151) Project [codegen id : 1] -Output [1]: [d_week_seq#136] -Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Output [1]: [d_week_seq#33] +Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt index 1b41abbc97b06..1d5907e319658 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (86) + WholeStageCodegen (90) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] @@ -49,7 +49,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (42) + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -68,11 +68,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #3 - WholeStageCodegen (20) + WholeStageCodegen (21) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 - WholeStageCodegen (19) + WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -81,82 +81,84 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (18) + WholeStageCodegen (19) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (17) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (12) + WholeStageCodegen (13) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (10) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #9 + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #10 - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #11 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (9) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #9 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (7) - Filter [i_item_sk] + Exchange [brand_id,class_id,category_id] #9 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #10 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (16) + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #10 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (8) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (17) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (15) + WholeStageCodegen (16) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -167,12 +169,12 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #9 + ReusedExchange [d_date_sk] #10 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter BroadcastExchange #3 - WholeStageCodegen (21) + WholeStageCodegen (22) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #3 @@ -188,29 +190,29 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #15 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (23) + WholeStageCodegen (24) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #16 - WholeStageCodegen (22) + WholeStageCodegen (23) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (41) + WholeStageCodegen (43) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #20 - WholeStageCodegen (85) + WholeStageCodegen (89) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (84) + WholeStageCodegen (88) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -218,24 +220,24 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] - WholeStageCodegen (44) + WholeStageCodegen (46) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #22 - WholeStageCodegen (43) + WholeStageCodegen (45) Filter [ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #6 ReusedExchange [d_date_sk] #23 - WholeStageCodegen (62) + WholeStageCodegen (65) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #23 - WholeStageCodegen (63) + WholeStageCodegen (66) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index bdafb17d69d45..258224e574cfb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -27,36 +27,36 @@ TakeOrderedAndProject (100) : : : :- * HashAggregate (39) : : : : +- Exchange (38) : : : : +- * HashAggregate (37) - : : : : +- * Project (36) - : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : :- * Project (33) - : : : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : : : :- * Filter (9) - : : : : : : +- * ColumnarToRow (8) - : : : : : : +- Scan parquet default.store_sales (7) - : : : : : +- BroadcastExchange (31) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : : : : : :- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Project (28) - : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : :- * Project (21) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : :- * Filter (15) - : : : : : : : +- * ColumnarToRow (14) - : : : : : : : +- Scan parquet default.catalog_sales (13) - : : : : : : +- BroadcastExchange (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet default.item (16) - : : : : : +- BroadcastExchange (26) - : : : : : +- * Project (25) - : : : : : +- * Filter (24) - : : : : : +- * ColumnarToRow (23) - : : : : : +- Scan parquet default.date_dim (22) - : : : : +- ReusedExchange (34) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : :- * Project (22) + : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : :- * Project (15) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (13) + : : : : : : +- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (20) + : : : : : +- * Project (19) + : : : : : +- * Filter (18) + : : : : : +- * ColumnarToRow (17) + : : : : : +- Scan parquet default.date_dim (16) + : : : : +- BroadcastExchange (35) + : : : : +- * Project (34) + : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * Project (31) + : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : :- * Filter (25) + : : : : : : +- * ColumnarToRow (24) + : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Filter (28) + : : : : : +- * ColumnarToRow (27) + : : : : : +- Scan parquet default.item (26) + : : : : +- ReusedExchange (32) : : : +- BroadcastExchange (49) : : : +- * Project (48) : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -131,443 +131,443 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (10) Scan parquet default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +(11) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(12) Filter [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) +(12) Filter [codegen id : 1] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(13) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +(13) BroadcastExchange +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +(14) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join condition: None -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) +(15) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(16) Scan parquet default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +(16) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_year#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +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 + +(17) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#12, d_year#13] -(17) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +(18) Filter [codegen id : 2] +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) -(18) Filter [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) +(19) Project [codegen id : 2] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] -(19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] +(20) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +(22) Project [codegen id : 6] +Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] +Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] -(22) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_year#25] +(23) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] 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 - -(23) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#24, d_year#25] +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(24) Filter [codegen id : 2] -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) +(24) ColumnarToRow [codegen id : 5] +Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -(25) Project [codegen id : 2] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_year#25] +(25) Filter [codegen id : 5] +Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +Condition : isnotnull(cs_item_sk#18) -(26) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +(26) Scan parquet default.item +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(27) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] -Join condition: None +(27) ColumnarToRow [codegen id : 3] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(28) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] +(28) Filter [codegen id : 3] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : isnotnull(i_item_sk#6) (29) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] -(30) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] +(30) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#6] Join condition: None -(31) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] +(31) Project [codegen id : 5] +Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] + +(32) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] -(32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] +(33) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None -(33) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +(34) Project [codegen id : 5] +Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] -(34) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#29] +(35) BroadcastExchange +Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] -(35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#29] +(36) BroadcastHashJoin [codegen id : 6] +Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] +Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None -(36) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] - (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#15, class_id#16, category_id#17] (38) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#15, class_id#16, category_id#17] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_item_sk#34) +Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Condition : isnotnull(ws_item_sk#23) -(43) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] +(43) ReusedExchange [Reuses operator id: 29] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#36] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [i_item_sk#6] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] +Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(46) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#40] +(46) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#40] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] +Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (49) BroadcastExchange -Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] +Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] -Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] +Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] +Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#15, class_id#16, category_id#17] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#15, class_id#16, category_id#17] (53) BroadcastExchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#30, class_id#31, category_id#32] +Right keys [3]: [brand_id#15, class_id#16, category_id#17] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#43] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] +Output [1]: [i_item_sk#6 AS ss_item_sk#27] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] (56) BroadcastExchange -Input [1]: [ss_item_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] +Input [1]: [ss_item_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#43] +Right keys [1]: [ss_item_sk#27] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] -Condition : (((isnotnull(i_item_sk#45) AND isnotnull(i_brand_id#46)) AND isnotnull(i_class_id#47)) AND isnotnull(i_category_id#48)) +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#27] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#45] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [i_item_sk#6] +Right keys [1]: [ss_item_sk#27] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#45] +Right keys [1]: [i_item_sk#6] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (66) Scan parquet default.date_dim -Output [2]: [d_date_sk#50, d_week_seq#51] +Output [2]: [d_date_sk#12, d_week_seq#30] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [2]: [d_date_sk#50, d_week_seq#51] +Input [2]: [d_date_sk#12, d_week_seq#30] (68) Filter [codegen id : 24] -Input [2]: [d_date_sk#50, d_week_seq#51] -Condition : ((isnotnull(d_week_seq#51) AND (d_week_seq#51 = Subquery scalar-subquery#52, [id=#53])) AND isnotnull(d_date_sk#50)) +Input [2]: [d_date_sk#12, d_week_seq#30] +Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#31, [id=#32])) AND isnotnull(d_date_sk#12)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#50] -Input [2]: [d_date_sk#50, d_week_seq#51] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_week_seq#30] (70) BroadcastExchange -Input [1]: [d_date_sk#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#50] +Right keys [1]: [d_date_sk#12] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] -Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] +Aggregate Attributes [3]: [sum#34, isEmpty#35, count#36] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] (74) Exchange -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] -Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#40] (75) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] -Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41, count(1)#42] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sales#43, count(1)#42 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] (76) Filter [codegen id : 52] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45 as decimal(32,6)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) (77) Project [codegen id : 52] -Output [6]: [store AS channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] +Output [6]: [store AS channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] (78) Scan parquet default.store_sales -Output [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#49)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (80) Filter [codegen id : 50] -Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] -Condition : isnotnull(ss_item_sk#70) +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#27] (82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#70] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [ss_item_sk#27] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] +Output [4]: [i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] (84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#70] -Right keys [1]: [i_item_sk#75] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#50] Join condition: None (85) Project [codegen id : 50] -Output [6]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] -Input [8]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#79, d_week_seq#80] +Output [2]: [d_date_sk#12, d_week_seq#30] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#79, d_week_seq#80] +Input [2]: [d_date_sk#12, d_week_seq#30] (88) Filter [codegen id : 49] -Input [2]: [d_date_sk#79, d_week_seq#80] -Condition : ((isnotnull(d_week_seq#80) AND (d_week_seq#80 = Subquery scalar-subquery#81, [id=#82])) AND isnotnull(d_date_sk#79)) +Input [2]: [d_date_sk#12, d_week_seq#30] +Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#54, [id=#55])) AND isnotnull(d_date_sk#12)) (89) Project [codegen id : 49] -Output [1]: [d_date_sk#79] -Input [2]: [d_date_sk#79, d_week_seq#80] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_week_seq#30] (90) BroadcastExchange -Input [1]: [d_date_sk#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#83] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] (91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#73] -Right keys [1]: [d_date_sk#79] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (92) Project [codegen id : 50] -Output [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] -Input [7]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78, d_date_sk#79] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53, d_date_sk#12] (93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] -Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] +Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#57, isEmpty#58, count#59] +Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] (94) Exchange -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] -Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] +Arguments: hashpartitioning(i_brand_id#51, i_class_id#52, i_category_id#53, 5), ENSURE_REQUIREMENTS, [id=#63] (95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] -Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] -Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] +Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] +Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] +Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] (96) Filter [codegen id : 51] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) +Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) (97) Project [codegen id : 51] -Output [6]: [store AS channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] +Output [6]: [store AS channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] (98) BroadcastExchange -Input [6]: [channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#97] +Input [6]: [channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] (99) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] -Right keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Right keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] Join condition: None (100) TakeOrderedAndProject -Input [12]: [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] -Arguments: 100, [i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Input [12]: [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#67, [id=#68] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#46, [id=#47] * HashAggregate (123) +- Exchange (122) +- * HashAggregate (121) @@ -594,140 +594,140 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] +Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#100), dynamicpruningexpression(ss_sold_date_sk#100 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#71)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] +Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (103) Scan parquet default.date_dim -Output [2]: [d_date_sk#102, d_year#103] +Output [2]: [d_date_sk#12, d_year#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#102, d_year#103] +Input [2]: [d_date_sk#12, d_year#13] (105) Filter [codegen id : 1] -Input [2]: [d_date_sk#102, d_year#103] -Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 <= 2000)) AND isnotnull(d_date_sk#102)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) (106) Project [codegen id : 1] -Output [1]: [d_date_sk#102] -Input [2]: [d_date_sk#102, d_year#103] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (107) BroadcastExchange -Input [1]: [d_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#104] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] (108) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#100] -Right keys [1]: [d_date_sk#102] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (109) Project [codegen id : 2] -Output [2]: [ss_quantity#98 AS quantity#105, ss_list_price#99 AS list_price#106] -Input [4]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100, d_date_sk#102] +Output [2]: [ss_quantity#2 AS quantity#73, ss_list_price#3 AS list_price#74] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (110) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] +Output [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#109), dynamicpruningexpression(cs_sold_date_sk#109 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#71)] ReadSchema: struct (111) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] +Input [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] (112) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#110] +Output [1]: [d_date_sk#12] (113) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#109] -Right keys [1]: [d_date_sk#110] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None (114) Project [codegen id : 4] -Output [2]: [cs_quantity#107 AS quantity#111, cs_list_price#108 AS list_price#112] -Input [4]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109, d_date_sk#110] +Output [2]: [cs_quantity#75 AS quantity#77, cs_list_price#76 AS list_price#78] +Input [4]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19, d_date_sk#12] (115) Scan parquet default.web_sales -Output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] +Output [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#115), dynamicpruningexpression(ws_sold_date_sk#115 IN dynamicpruning#101)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#71)] ReadSchema: struct (116) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] +Input [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] (117) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#116] +Output [1]: [d_date_sk#12] (118) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#115] -Right keys [1]: [d_date_sk#116] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (119) Project [codegen id : 6] -Output [2]: [ws_quantity#113 AS quantity#117, ws_list_price#114 AS list_price#118] -Input [4]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115, d_date_sk#116] +Output [2]: [ws_quantity#79 AS quantity#81, ws_list_price#80 AS list_price#82] +Input [4]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24, d_date_sk#12] (120) Union (121) HashAggregate [codegen id : 7] -Input [2]: [quantity#105, list_price#106] +Input [2]: [quantity#73, list_price#74] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#119, count#120] -Results [2]: [sum#121, count#122] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#83, count#84] +Results [2]: [sum#85, count#86] (122) Exchange -Input [2]: [sum#121, count#122] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#123] +Input [2]: [sum#85, count#86] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#87] (123) HashAggregate [codegen id : 8] -Input [2]: [sum#121, count#122] +Input [2]: [sum#85, count#86] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124 AS average_sales#125] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88 AS average_sales#89] -Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#100 IN dynamicpruning#101 +Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#71 ReusedExchange (124) (124) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#102] +Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#109 IN dynamicpruning#101 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#71 -Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#115 IN dynamicpruning#101 +Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#71 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#50] +Output [1]: [d_date_sk#12] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 ReusedExchange (126) -(126) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#29] +(126) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 -Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] +Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#31, [id=#32] * Project (130) +- * Filter (129) +- * ColumnarToRow (128) @@ -735,33 +735,33 @@ Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer (127) Scan parquet default.date_dim -Output [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (128) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] (129) Filter [codegen id : 1] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] -Condition : (((((isnotnull(d_year#127) AND isnotnull(d_moy#128)) AND isnotnull(d_dom#129)) AND (d_year#127 = 1999)) AND (d_moy#128 = 12)) AND (d_dom#129 = 16)) +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 1999)) AND (d_moy#90 = 12)) AND (d_dom#91 = 16)) (130) Project [codegen id : 1] -Output [1]: [d_week_seq#126] -Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Output [1]: [d_week_seq#30] +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] -Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] +Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] -Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 +Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#49 ReusedExchange (131) (131) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#79] +Output [1]: [d_date_sk#12] -Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#81, [id=#82] +Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#54, [id=#55] * Project (135) +- * Filter (134) +- * ColumnarToRow (133) @@ -769,21 +769,21 @@ Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subque (132) Scan parquet default.date_dim -Output [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (133) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] (134) Filter [codegen id : 1] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] -Condition : (((((isnotnull(d_year#131) AND isnotnull(d_moy#132)) AND isnotnull(d_dom#133)) AND (d_year#131 = 1998)) AND (d_moy#132 = 12)) AND (d_dom#133 = 16)) +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 1998)) AND (d_moy#90 = 12)) AND (d_dom#91 = 16)) (135) Project [codegen id : 1] -Output [1]: [d_week_seq#130] -Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Output [1]: [d_week_seq#30] +Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index cb9b410ea8568..3507d76bcf9b9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -82,53 +82,53 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Exchange [brand_id,class_id,category_id] #5 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (1) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #8 + BroadcastExchange #9 WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Filter [i_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #6 + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 WholeStageCodegen (9) 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 43ebe6e0b4d9b..9b47400d3b780 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 @@ -40,39 +40,39 @@ TakeOrderedAndProject (213) : : : : : +- * HashAggregate (44) : : : : : +- Exchange (43) : : : : : +- * HashAggregate (42) - : : : : : +- * Project (41) - : : : : : +- * BroadcastHashJoin Inner BuildRight (40) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (39) - : : : : : +- SortMergeJoin LeftSemi (38) - : : : : : :- * Sort (23) - : : : : : : +- Exchange (22) - : : : : : : +- * Filter (21) - : : : : : : +- * ColumnarToRow (20) - : : : : : : +- Scan parquet default.item (19) - : : : : : +- * Sort (37) - : : : : : +- Exchange (36) - : : : : : +- * Project (35) - : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : :- * Project (29) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) - : : : : : : :- * Filter (26) - : : : : : : : +- * ColumnarToRow (25) - : : : : : : : +- Scan parquet default.catalog_sales (24) - : : : : : : +- ReusedExchange (27) - : : : : : +- BroadcastExchange (33) - : : : : : +- * Filter (32) - : : : : : +- * ColumnarToRow (31) - : : : : : +- Scan parquet default.item (30) + : : : : : +- SortMergeJoin LeftSemi (41) + : : : : : :- * Sort (26) + : : : : : : +- Exchange (25) + : : : : : : +- * Project (24) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : : :- * Project (18) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : : :- * Filter (11) + : : : : : : : : +- * ColumnarToRow (10) + : : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : : +- BroadcastExchange (16) + : : : : : : : +- * Project (15) + : : : : : : : +- * Filter (14) + : : : : : : : +- * ColumnarToRow (13) + : : : : : : : +- Scan parquet default.date_dim (12) + : : : : : : +- BroadcastExchange (22) + : : : : : : +- * Filter (21) + : : : : : : +- * ColumnarToRow (20) + : : : : : : +- Scan parquet default.item (19) + : : : : : +- * Sort (40) + : : : : : +- Exchange (39) + : : : : : +- * Project (38) + : : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : : :- * Project (32) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : : :- * Filter (29) + : : : : : : : +- * ColumnarToRow (28) + : : : : : : : +- Scan parquet default.catalog_sales (27) + : : : : : : +- ReusedExchange (30) + : : : : : +- BroadcastExchange (36) + : : : : : +- * Filter (35) + : : : : : +- * ColumnarToRow (34) + : : : : : +- Scan parquet default.item (33) : : : : +- * Sort (57) : : : : +- Exchange (56) : : : : +- * Project (55) @@ -244,972 +244,972 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 19] +(7) ColumnarToRow [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 19] +(8) Filter [codegen id : 20] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 10] -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +(10) ColumnarToRow [codegen id : 5] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -(11) Filter [codegen id : 10] -Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_item_sk#11) +(11) Filter [codegen id : 5] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#12, d_year#13] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#14, d_year#15] +Input [2]: [d_date_sk#12, d_year#13] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#14, d_year#15] -Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1999)) AND (d_year#15 <= 2001)) AND isnotnull(d_date_sk#14)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#14] -Input [2]: [d_date_sk#14, d_year#15] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (16) BroadcastExchange -Input [1]: [d_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(17) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#14] +(17) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None -(18) Project [codegen id : 10] -Output [1]: [ss_item_sk#11] -Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] +(18) Project [codegen id : 5] +Output [1]: [ss_item_sk#1] +Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] (19) Scan parquet default.item -Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) -(22) Exchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] +(22) BroadcastExchange +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] + +(23) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#7] +Join condition: None -(23) Sort [codegen id : 5] -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 +(24) Project [codegen id : 5] +Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] +Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(24) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] +(25) Exchange +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] + +(26) Sort [codegen id : 6] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 + +(27) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 8] -Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +(28) ColumnarToRow [codegen id : 9] +Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -(26) Filter [codegen id : 8] -Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] -Condition : isnotnull(cs_item_sk#22) +(29) Filter [codegen id : 9] +Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +Condition : isnotnull(cs_item_sk#20) -(27) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#24] +(30) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#12] -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#24] +(31) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#12] Join condition: None -(29) Project [codegen id : 8] -Output [1]: [cs_item_sk#22] -Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] +(32) Project [codegen id : 9] +Output [1]: [cs_item_sk#20] +Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] -(30) Scan parquet default.item -Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +(33) Scan parquet default.item +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(31) ColumnarToRow [codegen id : 7] -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(32) Filter [codegen id : 7] -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -Condition : isnotnull(i_item_sk#25) +(35) Filter [codegen id : 8] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : isnotnull(i_item_sk#7) -(33) BroadcastExchange -Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +(36) BroadcastExchange +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cs_item_sk#22] -Right keys [1]: [i_item_sk#25] +(37) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [cs_item_sk#20] +Right keys [1]: [i_item_sk#7] Join condition: None -(35) Project [codegen id : 8] -Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +(38) Project [codegen id : 9] +Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(36) Exchange -Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] +(39) Exchange +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] -(37) Sort [codegen id : 9] -Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] -Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 +(40) Sort [codegen id : 10] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 -(38) SortMergeJoin -Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] -Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] +(41) SortMergeJoin +Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] +Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] Join condition: None -(39) BroadcastExchange -Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] - -(40) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#11] -Right keys [1]: [i_item_sk#17] -Join condition: None - -(41) Project [codegen id : 10] -Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] -Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] - -(42) HashAggregate [codegen id : 10] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +(42) HashAggregate [codegen id : 11] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#16, class_id#17, category_id#18] (43) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] -(44) HashAggregate [codegen id : 11] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +(44) HashAggregate [codegen id : 12] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#16, class_id#17, category_id#18] (45) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] -(46) Sort [codegen id : 12] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 13] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 15] -Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +(48) ColumnarToRow [codegen id : 16] +Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -(49) Filter [codegen id : 15] -Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_item_sk#37) +(49) Filter [codegen id : 16] +Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Condition : isnotnull(ws_item_sk#26) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#39] +Output [1]: [d_date_sk#12] -(51) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#39] +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None -(52) Project [codegen id : 15] -Output [1]: [ws_item_sk#37] -Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] +(52) Project [codegen id : 16] +Output [1]: [ws_item_sk#26] +Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] -(53) ReusedExchange [Reuses operator id: 33] -Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] +(53) ReusedExchange [Reuses operator id: 36] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(54) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ws_item_sk#37] -Right keys [1]: [i_item_sk#40] +(54) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [i_item_sk#7] Join condition: None -(55) Project [codegen id : 15] -Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] +(55) Project [codegen id : 16] +Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] (56) Exchange -Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] -(57) Sort [codegen id : 16] -Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] -Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 17] +Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] -Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] +Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] +Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] Join condition: None -(59) HashAggregate [codegen id : 17] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +(59) HashAggregate [codegen id : 18] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#16, class_id#17, category_id#18] (60) Exchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] -(61) HashAggregate [codegen id : 18] -Input [3]: [brand_id#32, class_id#33, category_id#34] -Keys [3]: [brand_id#32, class_id#33, category_id#34] +(61) HashAggregate [codegen id : 19] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Keys [3]: [brand_id#16, class_id#17, category_id#18] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#32, class_id#33, category_id#34] +Results [3]: [brand_id#16, class_id#17, category_id#18] (62) BroadcastExchange -Input [3]: [brand_id#32, class_id#33, category_id#34] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] +Input [3]: [brand_id#16, class_id#17, category_id#18] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] -(63) BroadcastHashJoin [codegen id : 19] +(63) BroadcastHashJoin [codegen id : 20] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#32, class_id#33, category_id#34] +Right keys [3]: [brand_id#16, class_id#17, category_id#18] Join condition: None -(64) Project [codegen id : 19] -Output [1]: [i_item_sk#7 AS ss_item_sk#47] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] +(64) Project [codegen id : 20] +Output [1]: [i_item_sk#7 AS ss_item_sk#31] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] (65) Exchange -Input [1]: [ss_item_sk#47] -Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [1]: [ss_item_sk#31] +Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] -(66) Sort [codegen id : 20] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 21] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#47] +Right keys [1]: [ss_item_sk#31] Join condition: None (68) Scan parquet default.date_dim -Output [3]: [d_date_sk#49, d_year#50, d_moy#51] +Output [3]: [d_date_sk#12, d_year#13, d_moy#33] 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 -(69) ColumnarToRow [codegen id : 21] -Input [3]: [d_date_sk#49, d_year#50, d_moy#51] +(69) ColumnarToRow [codegen id : 22] +Input [3]: [d_date_sk#12, d_year#13, d_moy#33] -(70) Filter [codegen id : 21] -Input [3]: [d_date_sk#49, d_year#50, d_moy#51] -Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2000)) AND (d_moy#51 = 11)) AND isnotnull(d_date_sk#49)) +(70) Filter [codegen id : 22] +Input [3]: [d_date_sk#12, d_year#13, d_moy#33] +Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#33)) AND (d_year#13 = 2000)) AND (d_moy#33 = 11)) AND isnotnull(d_date_sk#12)) -(71) Project [codegen id : 21] -Output [1]: [d_date_sk#49] -Input [3]: [d_date_sk#49, d_year#50, d_moy#51] +(71) Project [codegen id : 22] +Output [1]: [d_date_sk#12] +Input [3]: [d_date_sk#12, d_year#13, d_moy#33] (72) BroadcastExchange -Input [1]: [d_date_sk#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] -(73) BroadcastHashJoin [codegen id : 42] +(73) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#49] +Right keys [1]: [d_date_sk#12] Join condition: None -(74) Project [codegen id : 42] +(74) Project [codegen id : 44] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (75) Scan parquet default.item -Output [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 22] -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +(76) ColumnarToRow [codegen id : 23] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(77) Filter [codegen id : 22] -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Condition : isnotnull(i_item_sk#53) +(77) Filter [codegen id : 23] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Condition : isnotnull(i_item_sk#7) (78) Exchange -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: hashpartitioning(i_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#35] -(79) Sort [codegen id : 23] -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: [i_item_sk#53 ASC NULLS FIRST], false, 0 +(79) Sort [codegen id : 24] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#31] -(81) Sort [codegen id : 41] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 43] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#53] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [i_item_sk#7] +Right keys [1]: [ss_item_sk#31] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] +Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] -(84) BroadcastHashJoin [codegen id : 42] +(84) BroadcastHashJoin [codegen id : 44] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#53] +Right keys [1]: [i_item_sk#7] Join condition: None -(85) Project [codegen id : 42] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +(85) Project [codegen id : 44] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(86) HashAggregate [codegen id : 42] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] -Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] +(86) HashAggregate [codegen id : 44] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#59, isEmpty#60, count#61] -Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] +Aggregate Attributes [3]: [sum#37, isEmpty#38, count#39] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] (87) Exchange -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] -Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#43] -(88) HashAggregate [codegen id : 43] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] -Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] +(88) HashAggregate [codegen id : 45] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66, count(1)#67] -Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44, count(1)#45] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sales#46, count(1)#45 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] -(89) Filter [codegen id : 43] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(32,6)) > cast(Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(89) Filter [codegen id : 45] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48 as decimal(32,6)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(90) Project [codegen id : 43] -Output [6]: [store AS channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69] -Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +(90) Project [codegen id : 45] +Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] (91) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Output [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#77), dynamicpruningexpression(cs_sold_date_sk#77 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 44] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +(92) ColumnarToRow [codegen id : 46] +Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -(93) Filter [codegen id : 44] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Condition : isnotnull(cs_item_sk#74) +(93) Filter [codegen id : 46] +Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Condition : isnotnull(cs_item_sk#20) (94) Exchange -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Arguments: hashpartitioning(cs_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#54] -(95) Sort [codegen id : 45] -Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -Arguments: [cs_item_sk#74 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 47] +Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Arguments: [cs_item_sk#20 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#31] -(97) Sort [codegen id : 63] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 66] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [cs_item_sk#74] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [cs_item_sk#20] +Right keys [1]: [ss_item_sk#31] Join condition: None (99) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#79] +Output [1]: [d_date_sk#12] -(100) BroadcastHashJoin [codegen id : 85] -Left keys [1]: [cs_sold_date_sk#77] -Right keys [1]: [d_date_sk#79] +(100) BroadcastHashJoin [codegen id : 89] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#12] Join condition: None -(101) Project [codegen id : 85] -Output [3]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76] -Input [5]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77, d_date_sk#79] +(101) Project [codegen id : 89] +Output [3]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53] +Input [5]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] (102) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(103) BroadcastHashJoin [codegen id : 85] -Left keys [1]: [cs_item_sk#74] -Right keys [1]: [i_item_sk#80] +(103) BroadcastHashJoin [codegen id : 89] +Left keys [1]: [cs_item_sk#20] +Right keys [1]: [i_item_sk#7] Join condition: None -(104) Project [codegen id : 85] -Output [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] -Input [7]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] +(104) Project [codegen id : 89] +Output [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [7]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(105) HashAggregate [codegen id : 85] -Input [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] -Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +(105) HashAggregate [codegen id : 89] +Input [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] (106) Exchange -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] -Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#61] -(107) HashAggregate [codegen id : 86] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] -Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] -Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] +(107) HashAggregate [codegen id : 90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] -(108) Filter [codegen id : 86] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(108) Filter [codegen id : 90] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(109) Project [codegen id : 86] -Output [6]: [catalog AS channel#96, i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94] -Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] +(109) Project [codegen id : 90] +Output [6]: [catalog AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] (110) Scan parquet default.web_sales -Output [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Output [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#100), dynamicpruningexpression(ws_sold_date_sk#100 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 87] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +(111) ColumnarToRow [codegen id : 91] +Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -(112) Filter [codegen id : 87] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Condition : isnotnull(ws_item_sk#97) +(112) Filter [codegen id : 91] +Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Condition : isnotnull(ws_item_sk#26) (113) Exchange -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Arguments: hashpartitioning(ws_item_sk#97, 5), ENSURE_REQUIREMENTS, [id=#101] +Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Arguments: hashpartitioning(ws_item_sk#26, 5), ENSURE_REQUIREMENTS, [id=#70] -(114) Sort [codegen id : 88] -Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -Arguments: [ws_item_sk#97 ASC NULLS FIRST], false, 0 +(114) Sort [codegen id : 92] +Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Arguments: [ws_item_sk#26 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#47] +Output [1]: [ss_item_sk#31] -(116) Sort [codegen id : 106] -Input [1]: [ss_item_sk#47] -Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 +(116) Sort [codegen id : 111] +Input [1]: [ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 (117) SortMergeJoin -Left keys [1]: [ws_item_sk#97] -Right keys [1]: [ss_item_sk#47] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [ss_item_sk#31] Join condition: None (118) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#102] +Output [1]: [d_date_sk#12] -(119) BroadcastHashJoin [codegen id : 128] -Left keys [1]: [ws_sold_date_sk#100] -Right keys [1]: [d_date_sk#102] +(119) BroadcastHashJoin [codegen id : 134] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None -(120) Project [codegen id : 128] -Output [3]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99] -Input [5]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100, d_date_sk#102] +(120) Project [codegen id : 134] +Output [3]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69] +Input [5]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] (121) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] +Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(122) BroadcastHashJoin [codegen id : 128] -Left keys [1]: [ws_item_sk#97] -Right keys [1]: [i_item_sk#103] +(122) BroadcastHashJoin [codegen id : 134] +Left keys [1]: [ws_item_sk#26] +Right keys [1]: [i_item_sk#7] Join condition: None -(123) Project [codegen id : 128] -Output [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] -Input [7]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] +(123) Project [codegen id : 134] +Output [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [7]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(124) HashAggregate [codegen id : 128] -Input [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] -Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#107, isEmpty#108, count#109] -Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +(124) HashAggregate [codegen id : 134] +Input [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#71, isEmpty#72, count#73] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] (125) Exchange -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] -Arguments: hashpartitioning(i_brand_id#104, i_class_id#105, i_category_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#77] -(126) HashAggregate [codegen id : 129] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] -Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114, count(1)#115] -Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sales#116, count(1)#115 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] +(126) HashAggregate [codegen id : 135] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78, count(1)#79] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sales#80, count(1)#79 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] -(127) Filter [codegen id : 129] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(127) Filter [codegen id : 135] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(128) Project [codegen id : 129] -Output [6]: [web AS channel#119, i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117] -Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] +(128) Project [codegen id : 135] +Output [6]: [web AS channel#83, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] (129) Union -(130) HashAggregate [codegen id : 130] -Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69] -Keys [4]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#120, isEmpty#121, sum#122] -Results [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] +(130) HashAggregate [codegen id : 136] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] +Aggregate Attributes [3]: [sum#84, isEmpty#85, sum#86] +Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#87, isEmpty#88, sum#89] (131) Exchange -Input [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] -Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#126] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#87, isEmpty#88, sum#89] +Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#90] -(132) HashAggregate [codegen id : 131] -Input [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] -Keys [4]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#127, sum(number_sales#69)#128] -Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum(sales#68)#127 AS sum_sales#129, sum(number_sales#69)#128 AS number_sales#130] +(132) HashAggregate [codegen id : 137] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#87, isEmpty#88, sum#89] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(sales#46), sum(number_sales#47)] +Aggregate Attributes [2]: [sum(sales#46)#91, sum(number_sales#47)#92] +Results [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum(sales#46)#91 AS sum_sales#93, sum(number_sales#47)#92 AS number_sales#94] (133) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, count#136] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#95, isEmpty#96, count#97] -(134) HashAggregate [codegen id : 174] -Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, count#136] -Keys [3]: [i_brand_id#131, i_class_id#132, i_category_id#133] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139, count(1)#140] -Results [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sales#68, count(1)#140 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] +(134) HashAggregate [codegen id : 182] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#95, isEmpty#96, count#97] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#98, count(1)#99] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#98 AS sales#46, count(1)#99 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#98 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100] -(135) Filter [codegen id : 174] -Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(135) Filter [codegen id : 182] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(136) Project [codegen id : 174] -Output [6]: [store AS channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69] -Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] +(136) Project [codegen id : 182] +Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100] (137) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum#145, isEmpty#146, count#147] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#101, isEmpty#102, count#103] -(138) HashAggregate [codegen id : 217] -Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum#145, isEmpty#146, count#147] -Keys [3]: [i_brand_id#142, i_class_id#143, i_category_id#144] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150, count(1)#151] -Results [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sales#93, count(1)#151 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] +(138) HashAggregate [codegen id : 227] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#101, isEmpty#102, count#103] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#104, count(1)#105] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sales#64, count(1)#105 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106] -(139) Filter [codegen id : 217] -Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(139) Filter [codegen id : 227] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(140) Project [codegen id : 217] -Output [6]: [catalog AS channel#96, i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94] -Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] +(140) Project [codegen id : 227] +Output [6]: [catalog AS channel#107, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106] (141) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum#156, isEmpty#157, count#158] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#108, isEmpty#109, count#110] -(142) HashAggregate [codegen id : 260] -Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum#156, isEmpty#157, count#158] -Keys [3]: [i_brand_id#153, i_class_id#154, i_category_id#155] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161, count(1)#162] -Results [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161 AS sales#116, count(1)#162 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] +(142) HashAggregate [codegen id : 272] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#108, isEmpty#109, count#110] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#111, count(1)#112] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#111 AS sales#80, count(1)#112 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#111 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113] -(143) Filter [codegen id : 260] -Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(143) Filter [codegen id : 272] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(144) Project [codegen id : 260] -Output [6]: [web AS channel#119, i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117] -Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] +(144) Project [codegen id : 272] +Output [6]: [web AS channel#114, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113] (145) Union -(146) HashAggregate [codegen id : 261] -Input [6]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69] -Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#164, isEmpty#165, sum#166] -Results [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] +(146) HashAggregate [codegen id : 273] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] +Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] +Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#118, isEmpty#119, sum#120] (147) Exchange -Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] -Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, 5), ENSURE_REQUIREMENTS, [id=#170] - -(148) HashAggregate [codegen id : 262] -Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] -Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#171, sum(number_sales#69)#172] -Results [5]: [channel#73, i_brand_id#131, i_class_id#132, sum(sales#68)#171 AS sum_sales#129, sum(number_sales#69)#172 AS number_sales#130] - -(149) HashAggregate [codegen id : 262] -Input [5]: [channel#73, i_brand_id#131, i_class_id#132, sum_sales#129, number_sales#130] -Keys [3]: [channel#73, i_brand_id#131, i_class_id#132] -Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#173, isEmpty#174, sum#175] -Results [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#118, isEmpty#119, sum#120] +Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#121] + +(148) HashAggregate [codegen id : 274] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#118, isEmpty#119, sum#120] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(sales#46), sum(number_sales#47)] +Aggregate Attributes [2]: [sum(sales#46)#122, sum(number_sales#47)#123] +Results [5]: [channel#51, i_brand_id#8, i_class_id#9, sum(sales#46)#122 AS sum_sales#93, sum(number_sales#47)#123 AS number_sales#94] + +(149) HashAggregate [codegen id : 274] +Input [5]: [channel#51, i_brand_id#8, i_class_id#9, sum_sales#93, number_sales#94] +Keys [3]: [channel#51, i_brand_id#8, i_class_id#9] +Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] +Aggregate Attributes [3]: [sum#124, isEmpty#125, sum#126] +Results [6]: [channel#51, i_brand_id#8, i_class_id#9, sum#127, isEmpty#128, sum#129] (150) Exchange -Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] -Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, 5), ENSURE_REQUIREMENTS, [id=#179] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, sum#127, isEmpty#128, sum#129] +Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, 5), ENSURE_REQUIREMENTS, [id=#130] -(151) HashAggregate [codegen id : 263] -Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] -Keys [3]: [channel#73, i_brand_id#131, i_class_id#132] -Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#180, sum(number_sales#130)#181] -Results [6]: [channel#73, i_brand_id#131, i_class_id#132, null AS i_category_id#182, sum(sum_sales#129)#180 AS sum(sum_sales)#183, sum(number_sales#130)#181 AS sum(number_sales)#184] +(151) HashAggregate [codegen id : 275] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, sum#127, isEmpty#128, sum#129] +Keys [3]: [channel#51, i_brand_id#8, i_class_id#9] +Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] +Aggregate Attributes [2]: [sum(sum_sales#93)#131, sum(number_sales#94)#132] +Results [6]: [channel#51, i_brand_id#8, i_class_id#9, null AS i_category_id#133, sum(sum_sales#93)#131 AS sum(sum_sales)#134, sum(number_sales#94)#132 AS sum(number_sales)#135] (152) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum#188, isEmpty#189, count#190] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#136, isEmpty#137, count#138] -(153) HashAggregate [codegen id : 306] -Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum#188, isEmpty#189, count#190] -Keys [3]: [i_brand_id#185, i_class_id#186, i_category_id#187] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193, count(1)#194] -Results [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193 AS sales#68, count(1)#194 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] +(153) HashAggregate [codegen id : 320] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#136, isEmpty#137, count#138] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#139, count(1)#140] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sales#46, count(1)#140 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141] -(154) Filter [codegen id : 306] -Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(154) Filter [codegen id : 320] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(155) Project [codegen id : 306] -Output [6]: [store AS channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69] -Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] +(155) Project [codegen id : 320] +Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141] (156) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum#199, isEmpty#200, count#201] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#142, isEmpty#143, count#144] -(157) HashAggregate [codegen id : 349] -Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum#199, isEmpty#200, count#201] -Keys [3]: [i_brand_id#196, i_class_id#197, i_category_id#198] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204, count(1)#205] -Results [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204 AS sales#93, count(1)#205 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] +(157) HashAggregate [codegen id : 365] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#142, isEmpty#143, count#144] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#145, count(1)#146] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sales#64, count(1)#146 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147] -(158) Filter [codegen id : 349] -Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(158) Filter [codegen id : 365] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(159) Project [codegen id : 349] -Output [6]: [catalog AS channel#96, i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94] -Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] +(159) Project [codegen id : 365] +Output [6]: [catalog AS channel#148, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147] (160) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum#210, isEmpty#211, count#212] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#149, isEmpty#150, count#151] -(161) HashAggregate [codegen id : 392] -Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum#210, isEmpty#211, count#212] -Keys [3]: [i_brand_id#207, i_class_id#208, i_category_id#209] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215, count(1)#216] -Results [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sales#116, count(1)#216 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] +(161) HashAggregate [codegen id : 410] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#149, isEmpty#150, count#151] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#152, count(1)#153] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#152 AS sales#80, count(1)#153 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#152 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154] -(162) Filter [codegen id : 392] -Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(162) Filter [codegen id : 410] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(163) Project [codegen id : 392] -Output [6]: [web AS channel#119, i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117] -Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] +(163) Project [codegen id : 410] +Output [6]: [web AS channel#155, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154] (164) Union -(165) HashAggregate [codegen id : 393] -Input [6]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69] -Keys [4]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#218, isEmpty#219, sum#220] -Results [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] +(165) HashAggregate [codegen id : 411] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] +Aggregate Attributes [3]: [sum#156, isEmpty#157, sum#158] +Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#159, isEmpty#160, sum#161] (166) Exchange -Input [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] -Arguments: hashpartitioning(channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, 5), ENSURE_REQUIREMENTS, [id=#224] - -(167) HashAggregate [codegen id : 394] -Input [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] -Keys [4]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#225, sum(number_sales#69)#226] -Results [4]: [channel#73, i_brand_id#185, sum(sales#68)#225 AS sum_sales#129, sum(number_sales#69)#226 AS number_sales#130] - -(168) HashAggregate [codegen id : 394] -Input [4]: [channel#73, i_brand_id#185, sum_sales#129, number_sales#130] -Keys [2]: [channel#73, i_brand_id#185] -Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#227, isEmpty#228, sum#229] -Results [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#159, isEmpty#160, sum#161] +Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#162] + +(167) HashAggregate [codegen id : 412] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#159, isEmpty#160, sum#161] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(sales#46), sum(number_sales#47)] +Aggregate Attributes [2]: [sum(sales#46)#163, sum(number_sales#47)#164] +Results [4]: [channel#51, i_brand_id#8, sum(sales#46)#163 AS sum_sales#93, sum(number_sales#47)#164 AS number_sales#94] + +(168) HashAggregate [codegen id : 412] +Input [4]: [channel#51, i_brand_id#8, sum_sales#93, number_sales#94] +Keys [2]: [channel#51, i_brand_id#8] +Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] +Aggregate Attributes [3]: [sum#165, isEmpty#166, sum#167] +Results [5]: [channel#51, i_brand_id#8, sum#168, isEmpty#169, sum#170] (169) Exchange -Input [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] -Arguments: hashpartitioning(channel#73, i_brand_id#185, 5), ENSURE_REQUIREMENTS, [id=#233] +Input [5]: [channel#51, i_brand_id#8, sum#168, isEmpty#169, sum#170] +Arguments: hashpartitioning(channel#51, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [id=#171] -(170) HashAggregate [codegen id : 395] -Input [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] -Keys [2]: [channel#73, i_brand_id#185] -Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#234, sum(number_sales#130)#235] -Results [6]: [channel#73, i_brand_id#185, null AS i_class_id#236, null AS i_category_id#237, sum(sum_sales#129)#234 AS sum(sum_sales)#238, sum(number_sales#130)#235 AS sum(number_sales)#239] +(170) HashAggregate [codegen id : 413] +Input [5]: [channel#51, i_brand_id#8, sum#168, isEmpty#169, sum#170] +Keys [2]: [channel#51, i_brand_id#8] +Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] +Aggregate Attributes [2]: [sum(sum_sales#93)#172, sum(number_sales#94)#173] +Results [6]: [channel#51, i_brand_id#8, null AS i_class_id#174, null AS i_category_id#175, sum(sum_sales#93)#172 AS sum(sum_sales)#176, sum(number_sales#94)#173 AS sum(number_sales)#177] (171) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum#243, isEmpty#244, count#245] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#178, isEmpty#179, count#180] -(172) HashAggregate [codegen id : 438] -Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum#243, isEmpty#244, count#245] -Keys [3]: [i_brand_id#240, i_class_id#241, i_category_id#242] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248, count(1)#249] -Results [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248 AS sales#68, count(1)#249 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] +(172) HashAggregate [codegen id : 458] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#178, isEmpty#179, count#180] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#181, count(1)#182] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sales#46, count(1)#182 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183] -(173) Filter [codegen id : 438] -Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(173) Filter [codegen id : 458] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(174) Project [codegen id : 438] -Output [6]: [store AS channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69] -Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] +(174) Project [codegen id : 458] +Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183] (175) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum#254, isEmpty#255, count#256] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#184, isEmpty#185, count#186] -(176) HashAggregate [codegen id : 481] -Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum#254, isEmpty#255, count#256] -Keys [3]: [i_brand_id#251, i_class_id#252, i_category_id#253] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259, count(1)#260] -Results [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259 AS sales#93, count(1)#260 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] +(176) HashAggregate [codegen id : 503] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#184, isEmpty#185, count#186] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#187, count(1)#188] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sales#64, count(1)#188 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189] -(177) Filter [codegen id : 481] -Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(177) Filter [codegen id : 503] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(178) Project [codegen id : 481] -Output [6]: [catalog AS channel#96, i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94] -Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] +(178) Project [codegen id : 503] +Output [6]: [catalog AS channel#190, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189] (179) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum#265, isEmpty#266, count#267] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#191, isEmpty#192, count#193] -(180) HashAggregate [codegen id : 524] -Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum#265, isEmpty#266, count#267] -Keys [3]: [i_brand_id#262, i_class_id#263, i_category_id#264] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270, count(1)#271] -Results [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270 AS sales#116, count(1)#271 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] +(180) HashAggregate [codegen id : 548] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#191, isEmpty#192, count#193] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#194, count(1)#195] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#194 AS sales#80, count(1)#195 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#194 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196] -(181) Filter [codegen id : 524] -Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(181) Filter [codegen id : 548] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(182) Project [codegen id : 524] -Output [6]: [web AS channel#119, i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117] -Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] +(182) Project [codegen id : 548] +Output [6]: [web AS channel#197, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196] (183) Union -(184) HashAggregate [codegen id : 525] -Input [6]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69] -Keys [4]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#273, isEmpty#274, sum#275] -Results [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] +(184) HashAggregate [codegen id : 549] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] +Aggregate Attributes [3]: [sum#198, isEmpty#199, sum#200] +Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#201, isEmpty#202, sum#203] (185) Exchange -Input [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] -Arguments: hashpartitioning(channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, 5), ENSURE_REQUIREMENTS, [id=#279] - -(186) HashAggregate [codegen id : 526] -Input [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] -Keys [4]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#280, sum(number_sales#69)#281] -Results [3]: [channel#73, sum(sales#68)#280 AS sum_sales#129, sum(number_sales#69)#281 AS number_sales#130] - -(187) HashAggregate [codegen id : 526] -Input [3]: [channel#73, sum_sales#129, number_sales#130] -Keys [1]: [channel#73] -Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#282, isEmpty#283, sum#284] -Results [4]: [channel#73, sum#285, isEmpty#286, sum#287] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#201, isEmpty#202, sum#203] +Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#204] + +(186) HashAggregate [codegen id : 550] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#201, isEmpty#202, sum#203] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(sales#46), sum(number_sales#47)] +Aggregate Attributes [2]: [sum(sales#46)#205, sum(number_sales#47)#206] +Results [3]: [channel#51, sum(sales#46)#205 AS sum_sales#93, sum(number_sales#47)#206 AS number_sales#94] + +(187) HashAggregate [codegen id : 550] +Input [3]: [channel#51, sum_sales#93, number_sales#94] +Keys [1]: [channel#51] +Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] +Aggregate Attributes [3]: [sum#207, isEmpty#208, sum#209] +Results [4]: [channel#51, sum#210, isEmpty#211, sum#212] (188) Exchange -Input [4]: [channel#73, sum#285, isEmpty#286, sum#287] -Arguments: hashpartitioning(channel#73, 5), ENSURE_REQUIREMENTS, [id=#288] +Input [4]: [channel#51, sum#210, isEmpty#211, sum#212] +Arguments: hashpartitioning(channel#51, 5), ENSURE_REQUIREMENTS, [id=#213] -(189) HashAggregate [codegen id : 527] -Input [4]: [channel#73, sum#285, isEmpty#286, sum#287] -Keys [1]: [channel#73] -Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#289, sum(number_sales#130)#290] -Results [6]: [channel#73, null AS i_brand_id#291, null AS i_class_id#292, null AS i_category_id#293, sum(sum_sales#129)#289 AS sum(sum_sales)#294, sum(number_sales#130)#290 AS sum(number_sales)#295] +(189) HashAggregate [codegen id : 551] +Input [4]: [channel#51, sum#210, isEmpty#211, sum#212] +Keys [1]: [channel#51] +Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] +Aggregate Attributes [2]: [sum(sum_sales#93)#214, sum(number_sales#94)#215] +Results [6]: [channel#51, null AS i_brand_id#216, null AS i_class_id#217, null AS i_category_id#218, sum(sum_sales#93)#214 AS sum(sum_sales)#219, sum(number_sales#94)#215 AS sum(number_sales)#220] (190) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum#299, isEmpty#300, count#301] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#221, isEmpty#222, count#223] -(191) HashAggregate [codegen id : 570] -Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum#299, isEmpty#300, count#301] -Keys [3]: [i_brand_id#296, i_class_id#297, i_category_id#298] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304, count(1)#305] -Results [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304 AS sales#68, count(1)#305 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] +(191) HashAggregate [codegen id : 596] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#221, isEmpty#222, count#223] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#224, count(1)#225] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sales#46, count(1)#225 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226] -(192) Filter [codegen id : 570] -Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(192) Filter [codegen id : 596] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(193) Project [codegen id : 570] -Output [6]: [store AS channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69] -Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] +(193) Project [codegen id : 596] +Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226] (194) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum#310, isEmpty#311, count#312] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#227, isEmpty#228, count#229] -(195) HashAggregate [codegen id : 613] -Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum#310, isEmpty#311, count#312] -Keys [3]: [i_brand_id#307, i_class_id#308, i_category_id#309] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315, count(1)#316] -Results [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315 AS sales#93, count(1)#316 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] +(195) HashAggregate [codegen id : 641] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#227, isEmpty#228, count#229] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#230, count(1)#231] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sales#64, count(1)#231 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232] -(196) Filter [codegen id : 613] -Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(196) Filter [codegen id : 641] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(197) Project [codegen id : 613] -Output [6]: [catalog AS channel#96, i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94] -Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] +(197) Project [codegen id : 641] +Output [6]: [catalog AS channel#233, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232] (198) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum#321, isEmpty#322, count#323] +Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#234, isEmpty#235, count#236] -(199) HashAggregate [codegen id : 656] -Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum#321, isEmpty#322, count#323] -Keys [3]: [i_brand_id#318, i_class_id#319, i_category_id#320] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326, count(1)#327] -Results [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326 AS sales#116, count(1)#327 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] +(199) HashAggregate [codegen id : 686] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#234, isEmpty#235, count#236] +Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#237, count(1)#238] +Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#237 AS sales#80, count(1)#238 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#237 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239] -(200) Filter [codegen id : 656] -Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) +(200) Filter [codegen id : 686] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) -(201) Project [codegen id : 656] -Output [6]: [web AS channel#119, i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117] -Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] +(201) Project [codegen id : 686] +Output [6]: [web AS channel#240, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] +Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239] (202) Union -(203) HashAggregate [codegen id : 657] -Input [6]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69] -Keys [4]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298] -Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] -Aggregate Attributes [3]: [sum#329, isEmpty#330, sum#331] -Results [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] +(203) HashAggregate [codegen id : 687] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] +Aggregate Attributes [3]: [sum#241, isEmpty#242, sum#243] +Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#244, isEmpty#245, sum#246] (204) Exchange -Input [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] -Arguments: hashpartitioning(channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, 5), ENSURE_REQUIREMENTS, [id=#335] - -(205) HashAggregate [codegen id : 658] -Input [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] -Keys [4]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298] -Functions [2]: [sum(sales#68), sum(number_sales#69)] -Aggregate Attributes [2]: [sum(sales#68)#336, sum(number_sales#69)#337] -Results [2]: [sum(sales#68)#336 AS sum_sales#129, sum(number_sales#69)#337 AS number_sales#130] - -(206) HashAggregate [codegen id : 658] -Input [2]: [sum_sales#129, number_sales#130] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#244, isEmpty#245, sum#246] +Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#247] + +(205) HashAggregate [codegen id : 688] +Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#244, isEmpty#245, sum#246] +Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] +Functions [2]: [sum(sales#46), sum(number_sales#47)] +Aggregate Attributes [2]: [sum(sales#46)#248, sum(number_sales#47)#249] +Results [2]: [sum(sales#46)#248 AS sum_sales#93, sum(number_sales#47)#249 AS number_sales#94] + +(206) HashAggregate [codegen id : 688] +Input [2]: [sum_sales#93, number_sales#94] Keys: [] -Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] -Aggregate Attributes [3]: [sum#338, isEmpty#339, sum#340] -Results [3]: [sum#341, isEmpty#342, sum#343] +Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] +Aggregate Attributes [3]: [sum#250, isEmpty#251, sum#252] +Results [3]: [sum#253, isEmpty#254, sum#255] (207) Exchange -Input [3]: [sum#341, isEmpty#342, sum#343] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#344] +Input [3]: [sum#253, isEmpty#254, sum#255] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#256] -(208) HashAggregate [codegen id : 659] -Input [3]: [sum#341, isEmpty#342, sum#343] +(208) HashAggregate [codegen id : 689] +Input [3]: [sum#253, isEmpty#254, sum#255] Keys: [] -Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] -Aggregate Attributes [2]: [sum(sum_sales#129)#345, sum(number_sales#130)#346] -Results [6]: [null AS channel#347, null AS i_brand_id#348, null AS i_class_id#349, null AS i_category_id#350, sum(sum_sales#129)#345 AS sum(sum_sales)#351, sum(number_sales#130)#346 AS sum(number_sales)#352] +Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] +Aggregate Attributes [2]: [sum(sum_sales#93)#257, sum(number_sales#94)#258] +Results [6]: [null AS channel#259, null AS i_brand_id#260, null AS i_class_id#261, null AS i_category_id#262, sum(sum_sales#93)#257 AS sum(sum_sales)#263, sum(number_sales#94)#258 AS sum(number_sales)#264] (209) Union -(210) HashAggregate [codegen id : 660] -Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] -Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +(210) HashAggregate [codegen id : 690] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Keys [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Results [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] (211) Exchange -Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] -Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130, 5), ENSURE_REQUIREMENTS, [id=#353] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94, 5), ENSURE_REQUIREMENTS, [id=#265] -(212) HashAggregate [codegen id : 661] -Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] -Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +(212) HashAggregate [codegen id : 691] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Keys [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Results [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] (213) TakeOrderedAndProject -Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] -Arguments: 100, [channel#73 ASC NULLS FIRST, i_brand_id#54 ASC NULLS FIRST, i_class_id#55 ASC NULLS FIRST, i_category_id#56 ASC NULLS FIRST], [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Arguments: 100, [channel#51 ASC NULLS FIRST, i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#71, [id=#72] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#49, [id=#50] * HashAggregate (240) +- Exchange (239) +- * HashAggregate (238) @@ -1240,193 +1240,193 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (214) Scan parquet default.store_sales -Output [3]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356] +Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#356), dynamicpruningexpression(ss_sold_date_sk#356 IN dynamicpruning#357)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#266)] ReadSchema: struct (215) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356] +Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (216) Scan parquet default.date_dim -Output [2]: [d_date_sk#358, d_year#359] +Output [2]: [d_date_sk#12, d_year#13] 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 (217) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#358, d_year#359] +Input [2]: [d_date_sk#12, d_year#13] (218) Filter [codegen id : 1] -Input [2]: [d_date_sk#358, d_year#359] -Condition : (((isnotnull(d_year#359) AND (d_year#359 >= 1999)) AND (d_year#359 <= 2001)) AND isnotnull(d_date_sk#358)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (219) Project [codegen id : 1] -Output [1]: [d_date_sk#358] -Input [2]: [d_date_sk#358, d_year#359] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (220) BroadcastExchange -Input [1]: [d_date_sk#358] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#360] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#267] (221) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#356] -Right keys [1]: [d_date_sk#358] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (222) Project [codegen id : 2] -Output [2]: [ss_quantity#354 AS quantity#361, ss_list_price#355 AS list_price#362] -Input [4]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356, d_date_sk#358] +Output [2]: [ss_quantity#2 AS quantity#268, ss_list_price#3 AS list_price#269] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (223) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365] +Output [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#365), dynamicpruningexpression(cs_sold_date_sk#365 IN dynamicpruning#366)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#270)] ReadSchema: struct (224) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365] +Input [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] (225) Scan parquet default.date_dim -Output [2]: [d_date_sk#367, d_year#368] +Output [2]: [d_date_sk#12, d_year#13] 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 (226) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#367, d_year#368] +Input [2]: [d_date_sk#12, d_year#13] (227) Filter [codegen id : 3] -Input [2]: [d_date_sk#367, d_year#368] -Condition : (((isnotnull(d_year#368) AND (d_year#368 >= 1998)) AND (d_year#368 <= 2000)) AND isnotnull(d_date_sk#367)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) (228) Project [codegen id : 3] -Output [1]: [d_date_sk#367] -Input [2]: [d_date_sk#367, d_year#368] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (229) BroadcastExchange -Input [1]: [d_date_sk#367] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#369] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#271] (230) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#365] -Right keys [1]: [d_date_sk#367] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#12] Join condition: None (231) Project [codegen id : 4] -Output [2]: [cs_quantity#363 AS quantity#370, cs_list_price#364 AS list_price#371] -Input [4]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365, d_date_sk#367] +Output [2]: [cs_quantity#52 AS quantity#272, cs_list_price#53 AS list_price#273] +Input [4]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] (232) Scan parquet default.web_sales -Output [3]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374] +Output [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#374), dynamicpruningexpression(ws_sold_date_sk#374 IN dynamicpruning#366)] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#270)] ReadSchema: struct (233) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374] +Input [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] (234) ReusedExchange [Reuses operator id: 229] -Output [1]: [d_date_sk#375] +Output [1]: [d_date_sk#12] (235) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#374] -Right keys [1]: [d_date_sk#375] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#12] Join condition: None (236) Project [codegen id : 6] -Output [2]: [ws_quantity#372 AS quantity#376, ws_list_price#373 AS list_price#377] -Input [4]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374, d_date_sk#375] +Output [2]: [ws_quantity#68 AS quantity#274, ws_list_price#69 AS list_price#275] +Input [4]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] (237) Union (238) HashAggregate [codegen id : 7] -Input [2]: [quantity#361, list_price#362] +Input [2]: [quantity#268, list_price#269] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#378, count#379] -Results [2]: [sum#380, count#381] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#276, count#277] +Results [2]: [sum#278, count#279] (239) Exchange -Input [2]: [sum#380, count#381] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#382] +Input [2]: [sum#278, count#279] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#280] (240) HashAggregate [codegen id : 8] -Input [2]: [sum#380, count#381] +Input [2]: [sum#278, count#279] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))#383] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))#383 AS average_sales#384] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#281] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#281 AS average_sales#282] -Subquery:2 Hosting operator id = 214 Hosting Expression = ss_sold_date_sk#356 IN dynamicpruning#357 +Subquery:2 Hosting operator id = 214 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#266 ReusedExchange (241) (241) ReusedExchange [Reuses operator id: 220] -Output [1]: [d_date_sk#358] +Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 223 Hosting Expression = cs_sold_date_sk#365 IN dynamicpruning#366 +Subquery:3 Hosting operator id = 223 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#270 ReusedExchange (242) (242) ReusedExchange [Reuses operator id: 229] -Output [1]: [d_date_sk#367] +Output [1]: [d_date_sk#12] -Subquery:4 Hosting operator id = 232 Hosting Expression = ws_sold_date_sk#374 IN dynamicpruning#366 +Subquery:4 Hosting operator id = 232 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#270 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (243) (243) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#49] +Output [1]: [d_date_sk#12] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 ReusedExchange (244) (244) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 +Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 -Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#77 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#5 -Subquery:13 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:13 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:14 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:14 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:15 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:15 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:16 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:16 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:17 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:17 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:18 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:18 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:19 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:19 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:20 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:20 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:21 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:21 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:22 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:22 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:23 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:23 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] -Subquery:24 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] +Subquery:24 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt index cb186b0ddcd3b..d27eb5a32b387 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt @@ -1,21 +1,21 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - WholeStageCodegen (661) + WholeStageCodegen (691) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (660) + WholeStageCodegen (690) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union - WholeStageCodegen (131) + WholeStageCodegen (137) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (130) + WholeStageCodegen (136) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (43) + WholeStageCodegen (45) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (42) + WholeStageCodegen (44) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -90,11 +90,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - WholeStageCodegen (20) + WholeStageCodegen (21) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #6 - WholeStageCodegen (19) + WholeStageCodegen (20) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -103,82 +103,84 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #7 - WholeStageCodegen (18) + WholeStageCodegen (19) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (17) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (12) + WholeStageCodegen (13) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (11) + WholeStageCodegen (12) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #10 - WholeStageCodegen (10) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #11 + InputAdapter + SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (6) + Sort [brand_id,class_id,category_id] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #12 - SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (5) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #13 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (9) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (8) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (7) - Filter [i_item_sk] + Exchange [brand_id,class_id,category_id] #11 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #12 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (16) + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (10) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #12 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (8) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (17) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (15) + WholeStageCodegen (16) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -189,12 +191,12 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #11 + ReusedExchange [d_date_sk] #12 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 InputAdapter BroadcastExchange #5 - WholeStageCodegen (21) + WholeStageCodegen (22) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow @@ -203,27 +205,27 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter BroadcastExchange #17 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (23) + WholeStageCodegen (24) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #18 - WholeStageCodegen (22) + WholeStageCodegen (23) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (41) + WholeStageCodegen (43) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 - WholeStageCodegen (86) + WholeStageCodegen (90) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #23 - WholeStageCodegen (85) + WholeStageCodegen (89) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -231,17 +233,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [cs_item_sk,ss_item_sk] - WholeStageCodegen (45) + WholeStageCodegen (47) Sort [cs_item_sk] InputAdapter Exchange [cs_item_sk] #24 - WholeStageCodegen (44) + WholeStageCodegen (46) Filter [cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (63) + WholeStageCodegen (66) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 @@ -249,14 +251,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedExchange [d_date_sk] #5 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (129) + WholeStageCodegen (135) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #25 - WholeStageCodegen (128) + WholeStageCodegen (134) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] @@ -264,17 +266,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ws_item_sk,ss_item_sk] - WholeStageCodegen (88) + WholeStageCodegen (92) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #26 - WholeStageCodegen (87) + WholeStageCodegen (91) Filter [ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (106) + WholeStageCodegen (111) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 @@ -282,136 +284,136 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedExchange [d_date_sk] #5 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (263) + WholeStageCodegen (275) HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id] #27 - WholeStageCodegen (262) + WholeStageCodegen (274) HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #28 - WholeStageCodegen (261) + WholeStageCodegen (273) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (174) + WholeStageCodegen (182) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (217) + WholeStageCodegen (227) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (260) + WholeStageCodegen (272) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (395) + WholeStageCodegen (413) HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id] #32 - WholeStageCodegen (394) + WholeStageCodegen (412) HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #33 - WholeStageCodegen (393) + WholeStageCodegen (411) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (306) + WholeStageCodegen (320) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (349) + WholeStageCodegen (365) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (392) + WholeStageCodegen (410) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (527) + WholeStageCodegen (551) HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel] #34 - WholeStageCodegen (526) + WholeStageCodegen (550) HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #35 - WholeStageCodegen (525) + WholeStageCodegen (549) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (438) + WholeStageCodegen (458) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (481) + WholeStageCodegen (503) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (524) + WholeStageCodegen (548) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (659) + WholeStageCodegen (689) HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange #36 - WholeStageCodegen (658) + WholeStageCodegen (688) HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #37 - WholeStageCodegen (657) + WholeStageCodegen (687) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (570) + WholeStageCodegen (596) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (613) + WholeStageCodegen (641) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (656) + WholeStageCodegen (686) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 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 755973f79e493..04fe14cab2c9b 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 @@ -34,36 +34,36 @@ TakeOrderedAndProject (194) : : : : :- * HashAggregate (39) : : : : : +- Exchange (38) : : : : : +- * HashAggregate (37) - : : : : : +- * Project (36) - : : : : : +- * BroadcastHashJoin Inner BuildRight (35) - : : : : : :- * Project (33) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (32) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (31) - : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) - : : : : : : :- * Filter (12) - : : : : : : : +- * ColumnarToRow (11) - : : : : : : : +- Scan parquet default.item (10) - : : : : : : +- BroadcastExchange (29) - : : : : : : +- * Project (28) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) - : : : : : : :- * Project (21) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) - : : : : : : : :- * Filter (15) - : : : : : : : : +- * ColumnarToRow (14) - : : : : : : : : +- Scan parquet default.catalog_sales (13) - : : : : : : : +- BroadcastExchange (19) - : : : : : : : +- * Filter (18) - : : : : : : : +- * ColumnarToRow (17) - : : : : : : : +- Scan parquet default.item (16) - : : : : : : +- BroadcastExchange (26) - : : : : : : +- * Project (25) - : : : : : : +- * Filter (24) - : : : : : : +- * ColumnarToRow (23) - : : : : : : +- Scan parquet default.date_dim (22) - : : : : : +- ReusedExchange (34) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) + : : : : : :- * Project (22) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : : : :- * Project (15) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : : :- * Filter (9) + : : : : : : : : +- * ColumnarToRow (8) + : : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : : +- BroadcastExchange (13) + : : : : : : : +- * Filter (12) + : : : : : : : +- * ColumnarToRow (11) + : : : : : : : +- Scan parquet default.item (10) + : : : : : : +- BroadcastExchange (20) + : : : : : : +- * Project (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet default.date_dim (16) + : : : : : +- BroadcastExchange (35) + : : : : : +- * Project (34) + : : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : : :- * Project (31) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : : : : :- * Filter (25) + : : : : : : : +- * ColumnarToRow (24) + : : : : : : : +- Scan parquet default.catalog_sales (23) + : : : : : : +- BroadcastExchange (29) + : : : : : : +- * Filter (28) + : : : : : : +- * ColumnarToRow (27) + : : : : : : +- Scan parquet default.item (26) + : : : : : +- ReusedExchange (32) : : : : +- BroadcastExchange (49) : : : : +- * Project (48) : : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -225,896 +225,896 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] -Condition : isnotnull(ss_item_sk#10) +Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) (10) Scan parquet default.item -Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +(11) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(12) Filter [codegen id : 4] -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) +(12) Filter [codegen id : 1] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) -(13) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +(13) BroadcastExchange +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] -(14) ColumnarToRow [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +(14) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join condition: None -(15) Filter [codegen id : 3] -Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -Condition : isnotnull(cs_item_sk#17) +(15) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(16) Scan parquet default.item -Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +(16) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_year#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +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 + +(17) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#12, d_year#13] -(17) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +(18) Filter [codegen id : 2] +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) -(18) Filter [codegen id : 1] -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Condition : isnotnull(i_item_sk#19) +(19) Project [codegen id : 2] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] -(19) BroadcastExchange -Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] +(20) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_item_sk#17] -Right keys [1]: [i_item_sk#19] +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None -(21) Project [codegen id : 3] -Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] -Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +(22) Project [codegen id : 6] +Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] +Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] -(22) Scan parquet default.date_dim -Output [2]: [d_date_sk#24, d_year#25] +(23) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] 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 - -(23) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#24, d_year#25] +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(24) Filter [codegen id : 2] -Input [2]: [d_date_sk#24, d_year#25] -Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) +(24) ColumnarToRow [codegen id : 5] +Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -(25) Project [codegen id : 2] -Output [1]: [d_date_sk#24] -Input [2]: [d_date_sk#24, d_year#25] +(25) Filter [codegen id : 5] +Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +Condition : isnotnull(cs_item_sk#18) -(26) BroadcastExchange -Input [1]: [d_date_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +(26) Scan parquet default.item +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(27) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#24] -Join condition: None +(27) ColumnarToRow [codegen id : 3] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(28) Project [codegen id : 3] -Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] +(28) Filter [codegen id : 3] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : isnotnull(i_item_sk#6) (29) BroadcastExchange -Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] -(30) BroadcastHashJoin [codegen id : 4] -Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] -Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] +(30) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#6] Join condition: None -(31) BroadcastExchange -Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] +(31) Project [codegen id : 5] +Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] + +(32) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] -(32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#10] -Right keys [1]: [i_item_sk#13] +(33) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None -(33) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] -Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +(34) Project [codegen id : 5] +Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] -(34) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#29] +(35) BroadcastExchange +Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] -(35) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#11] -Right keys [1]: [d_date_sk#29] +(36) BroadcastHashJoin [codegen id : 6] +Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] +Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None -(36) Project [codegen id : 6] -Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] -Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] - (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#15, class_id#16, category_id#17] (38) Exchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#15, class_id#16, category_id#17] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_item_sk#34) +Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Condition : isnotnull(ws_item_sk#23) -(43) ReusedExchange [Reuses operator id: 19] -Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] +(43) ReusedExchange [Reuses operator id: 29] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#36] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [i_item_sk#6] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] -Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] +Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -(46) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#40] +(46) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#40] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] +Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (49) BroadcastExchange -Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] +Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] -Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] +Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] +Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#15, class_id#16, category_id#17] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#30, class_id#31, category_id#32] -Keys [3]: [brand_id#30, class_id#31, category_id#32] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Keys [3]: [brand_id#15, class_id#16, category_id#17] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#30, class_id#31, category_id#32] +Results [3]: [brand_id#15, class_id#16, category_id#17] (53) BroadcastExchange -Input [3]: [brand_id#30, class_id#31, category_id#32] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] +Input [3]: [brand_id#15, class_id#16, category_id#17] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#30, class_id#31, category_id#32] +Right keys [3]: [brand_id#15, class_id#16, category_id#17] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#43] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] +Output [1]: [i_item_sk#6 AS ss_item_sk#27] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] (56) BroadcastExchange -Input [1]: [ss_item_sk#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] +Input [1]: [ss_item_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#43] +Right keys [1]: [ss_item_sk#27] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] -Condition : isnotnull(i_item_sk#45) +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : isnotnull(i_item_sk#6) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#27] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#45] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [i_item_sk#6] +Right keys [1]: [ss_item_sk#27] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#45] +Right keys [1]: [i_item_sk#6] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (66) Scan parquet default.date_dim -Output [3]: [d_date_sk#50, d_year#51, d_moy#52] +Output [3]: [d_date_sk#12, d_year#13, d_moy#30] 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 (67) ColumnarToRow [codegen id : 24] -Input [3]: [d_date_sk#50, d_year#51, d_moy#52] +Input [3]: [d_date_sk#12, d_year#13, d_moy#30] (68) Filter [codegen id : 24] -Input [3]: [d_date_sk#50, d_year#51, d_moy#52] -Condition : ((((isnotnull(d_year#51) AND isnotnull(d_moy#52)) AND (d_year#51 = 2000)) AND (d_moy#52 = 11)) AND isnotnull(d_date_sk#50)) +Input [3]: [d_date_sk#12, d_year#13, d_moy#30] +Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#30)) AND (d_year#13 = 2000)) AND (d_moy#30 = 11)) AND isnotnull(d_date_sk#12)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#50] -Input [3]: [d_date_sk#50, d_year#51, d_moy#52] +Output [1]: [d_date_sk#12] +Input [3]: [d_date_sk#12, d_year#13, d_moy#30] (70) BroadcastExchange -Input [1]: [d_date_sk#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#50] +Right keys [1]: [d_date_sk#12] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] -Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] +Aggregate Attributes [3]: [sum#32, isEmpty#33, count#34] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] (74) Exchange -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] -Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#38] (75) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] -Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] -Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] (76) Filter [codegen id : 26] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [store AS channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64] -Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] (78) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +Output [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] (80) Filter [codegen id : 51] -Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] -Condition : isnotnull(cs_item_sk#69) +Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Condition : isnotnull(cs_item_sk#18) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#27] (82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#69] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [ss_item_sk#27] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#69] -Right keys [1]: [i_item_sk#73] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#6] Join condition: None (85) Project [codegen id : 51] -Output [6]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76] -Input [8]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] +Output [6]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [8]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#77] +Output [1]: [d_date_sk#12] (87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#72] -Right keys [1]: [d_date_sk#77] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None (88) Project [codegen id : 51] -Output [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] -Input [7]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76, d_date_sk#77] +Output [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] -Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#78, isEmpty#79, count#80] -Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Input [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#49, isEmpty#50, count#51] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] (90) Exchange -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] -Arguments: hashpartitioning(i_brand_id#74, i_class_id#75, i_category_id#76, 5), ENSURE_REQUIREMENTS, [id=#84] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#55] (91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] -Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85, count(1)#86] -Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sales#87, count(1)#86 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56, count(1)#57] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sales#58, count(1)#57 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] (92) Filter [codegen id : 52] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [catalog AS channel#90, i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88] -Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] +Output [6]: [catalog AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] (94) Scan parquet default.web_sales -Output [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Output [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] (96) Filter [codegen id : 77] -Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] -Condition : isnotnull(ws_item_sk#91) +Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Condition : isnotnull(ws_item_sk#23) (97) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#43] +Output [1]: [ss_item_sk#27] (98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#91] -Right keys [1]: [ss_item_sk#43] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [ss_item_sk#27] Join condition: None (99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#91] -Right keys [1]: [i_item_sk#95] +Left keys [1]: [ws_item_sk#23] +Right keys [1]: [i_item_sk#6] Join condition: None (101) Project [codegen id : 77] -Output [6]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98] -Input [8]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] +Output [6]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [8]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] (102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#99] +Output [1]: [d_date_sk#12] (103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#94] -Right keys [1]: [d_date_sk#99] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (104) Project [codegen id : 77] -Output [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] -Input [7]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98, d_date_sk#99] +Output [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [7]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] (105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] -Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#100, isEmpty#101, count#102] -Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Input [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] (106) Exchange -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] -Arguments: hashpartitioning(i_brand_id#96, i_class_id#97, i_category_id#98, 5), ENSURE_REQUIREMENTS, [id=#106] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] +Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#70] (107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] -Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] -Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#109, count(1)#108 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71, count(1)#72] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sales#73, count(1)#72 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] (108) Filter [codegen id : 78] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [web AS channel#112, i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110] -Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] +Output [6]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] (110) Union (111) HashAggregate [codegen id : 79] -Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64] -Keys [4]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#113, isEmpty#114, sum#115] -Results [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] +Aggregate Attributes [3]: [sum#77, isEmpty#78, sum#79] +Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#80, isEmpty#81, sum#82] (112) Exchange -Input [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] -Arguments: hashpartitioning(channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#119] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#80, isEmpty#81, sum#82] +Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#83] (113) HashAggregate [codegen id : 80] -Input [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] -Keys [4]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#120, sum(number_sales#64)#121] -Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum(sales#63)#120 AS sum_sales#122, sum(number_sales#64)#121 AS number_sales#123] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#80, isEmpty#81, sum#82] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#41), sum(number_sales#42)] +Aggregate Attributes [2]: [sum(sales#41)#84, sum(number_sales#42)#85] +Results [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#41)#84 AS sum_sales#86, sum(number_sales#42)#85 AS number_sales#87] (114) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#88, isEmpty#89, count#90] (115) HashAggregate [codegen id : 106] -Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] -Keys [3]: [i_brand_id#124, i_class_id#125, i_category_id#126] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132, count(1)#133] -Results [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sales#63, count(1)#133 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#88, isEmpty#89, count#90] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#41, count(1)#92 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93] (116) Filter [codegen id : 106] -Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (117) Project [codegen id : 106] -Output [6]: [store AS channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64] -Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] +Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93] (118) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#94, isEmpty#95, count#96] (119) HashAggregate [codegen id : 132] -Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] -Keys [3]: [i_brand_id#135, i_class_id#136, i_category_id#137] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143, count(1)#144] -Results [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sales#87, count(1)#144 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#94, isEmpty#95, count#96] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#58, count(1)#98 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99] (120) Filter [codegen id : 132] -Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (121) Project [codegen id : 132] -Output [6]: [catalog AS channel#90, i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88] -Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] +Output [6]: [catalog AS channel#100, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99] (122) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum#149, isEmpty#150, count#151] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#101, isEmpty#102, count#103] (123) HashAggregate [codegen id : 158] -Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum#149, isEmpty#150, count#151] -Keys [3]: [i_brand_id#146, i_class_id#147, i_category_id#148] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154, count(1)#155] -Results [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154 AS sales#109, count(1)#155 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#101, isEmpty#102, count#103] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#104, count(1)#105] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sales#73, count(1)#105 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106] (124) Filter [codegen id : 158] -Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (125) Project [codegen id : 158] -Output [6]: [web AS channel#112, i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110] -Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] +Output [6]: [web AS channel#107, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106] (126) Union (127) HashAggregate [codegen id : 159] -Input [6]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64] -Keys [4]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#157, isEmpty#158, sum#159] -Results [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] +Aggregate Attributes [3]: [sum#108, isEmpty#109, sum#110] +Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#111, isEmpty#112, sum#113] (128) Exchange -Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] -Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, 5), ENSURE_REQUIREMENTS, [id=#163] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#111, isEmpty#112, sum#113] +Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#114] (129) HashAggregate [codegen id : 160] -Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] -Keys [4]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#164, sum(number_sales#64)#165] -Results [5]: [channel#68, i_brand_id#124, i_class_id#125, sum(sales#63)#164 AS sum_sales#122, sum(number_sales#64)#165 AS number_sales#123] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#111, isEmpty#112, sum#113] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#41), sum(number_sales#42)] +Aggregate Attributes [2]: [sum(sales#41)#115, sum(number_sales#42)#116] +Results [5]: [channel#46, i_brand_id#7, i_class_id#8, sum(sales#41)#115 AS sum_sales#86, sum(number_sales#42)#116 AS number_sales#87] (130) HashAggregate [codegen id : 160] -Input [5]: [channel#68, i_brand_id#124, i_class_id#125, sum_sales#122, number_sales#123] -Keys [3]: [channel#68, i_brand_id#124, i_class_id#125] -Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] -Aggregate Attributes [3]: [sum#166, isEmpty#167, sum#168] -Results [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] +Input [5]: [channel#46, i_brand_id#7, i_class_id#8, sum_sales#86, number_sales#87] +Keys [3]: [channel#46, i_brand_id#7, i_class_id#8] +Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] +Aggregate Attributes [3]: [sum#117, isEmpty#118, sum#119] +Results [6]: [channel#46, i_brand_id#7, i_class_id#8, sum#120, isEmpty#121, sum#122] (131) Exchange -Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] -Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, 5), ENSURE_REQUIREMENTS, [id=#172] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, sum#120, isEmpty#121, sum#122] +Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, 5), ENSURE_REQUIREMENTS, [id=#123] (132) HashAggregate [codegen id : 161] -Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] -Keys [3]: [channel#68, i_brand_id#124, i_class_id#125] -Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] -Aggregate Attributes [2]: [sum(sum_sales#122)#173, sum(number_sales#123)#174] -Results [6]: [channel#68, i_brand_id#124, i_class_id#125, null AS i_category_id#175, sum(sum_sales#122)#173 AS sum(sum_sales)#176, sum(number_sales#123)#174 AS sum(number_sales)#177] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, sum#120, isEmpty#121, sum#122] +Keys [3]: [channel#46, i_brand_id#7, i_class_id#8] +Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] +Aggregate Attributes [2]: [sum(sum_sales#86)#124, sum(number_sales#87)#125] +Results [6]: [channel#46, i_brand_id#7, i_class_id#8, null AS i_category_id#126, sum(sum_sales#86)#124 AS sum(sum_sales)#127, sum(number_sales#87)#125 AS sum(number_sales)#128] (133) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#129, isEmpty#130, count#131] (134) HashAggregate [codegen id : 187] -Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] -Keys [3]: [i_brand_id#178, i_class_id#179, i_category_id#180] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186, count(1)#187] -Results [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sales#63, count(1)#187 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#129, isEmpty#130, count#131] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#132, count(1)#133] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sales#41, count(1)#133 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134] (135) Filter [codegen id : 187] -Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (136) Project [codegen id : 187] -Output [6]: [store AS channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64] -Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] +Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134] (137) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#135, isEmpty#136, count#137] (138) HashAggregate [codegen id : 213] -Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] -Keys [3]: [i_brand_id#189, i_class_id#190, i_category_id#191] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197, count(1)#198] -Results [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sales#87, count(1)#198 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#135, isEmpty#136, count#137] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#138, count(1)#139] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sales#58, count(1)#139 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140] (139) Filter [codegen id : 213] -Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (140) Project [codegen id : 213] -Output [6]: [catalog AS channel#90, i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88] -Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] +Output [6]: [catalog AS channel#141, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140] (141) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum#203, isEmpty#204, count#205] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#142, isEmpty#143, count#144] (142) HashAggregate [codegen id : 239] -Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum#203, isEmpty#204, count#205] -Keys [3]: [i_brand_id#200, i_class_id#201, i_category_id#202] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208, count(1)#209] -Results [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208 AS sales#109, count(1)#209 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#142, isEmpty#143, count#144] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#145, count(1)#146] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sales#73, count(1)#146 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147] (143) Filter [codegen id : 239] -Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (144) Project [codegen id : 239] -Output [6]: [web AS channel#112, i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110] -Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] +Output [6]: [web AS channel#148, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147] (145) Union (146) HashAggregate [codegen id : 240] -Input [6]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64] -Keys [4]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#211, isEmpty#212, sum#213] -Results [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] +Aggregate Attributes [3]: [sum#149, isEmpty#150, sum#151] +Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] (147) Exchange -Input [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] -Arguments: hashpartitioning(channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, 5), ENSURE_REQUIREMENTS, [id=#217] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] +Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#155] (148) HashAggregate [codegen id : 241] -Input [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] -Keys [4]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#218, sum(number_sales#64)#219] -Results [4]: [channel#68, i_brand_id#178, sum(sales#63)#218 AS sum_sales#122, sum(number_sales#64)#219 AS number_sales#123] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#41), sum(number_sales#42)] +Aggregate Attributes [2]: [sum(sales#41)#156, sum(number_sales#42)#157] +Results [4]: [channel#46, i_brand_id#7, sum(sales#41)#156 AS sum_sales#86, sum(number_sales#42)#157 AS number_sales#87] (149) HashAggregate [codegen id : 241] -Input [4]: [channel#68, i_brand_id#178, sum_sales#122, number_sales#123] -Keys [2]: [channel#68, i_brand_id#178] -Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] -Aggregate Attributes [3]: [sum#220, isEmpty#221, sum#222] -Results [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] +Input [4]: [channel#46, i_brand_id#7, sum_sales#86, number_sales#87] +Keys [2]: [channel#46, i_brand_id#7] +Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] +Aggregate Attributes [3]: [sum#158, isEmpty#159, sum#160] +Results [5]: [channel#46, i_brand_id#7, sum#161, isEmpty#162, sum#163] (150) Exchange -Input [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] -Arguments: hashpartitioning(channel#68, i_brand_id#178, 5), ENSURE_REQUIREMENTS, [id=#226] +Input [5]: [channel#46, i_brand_id#7, sum#161, isEmpty#162, sum#163] +Arguments: hashpartitioning(channel#46, i_brand_id#7, 5), ENSURE_REQUIREMENTS, [id=#164] (151) HashAggregate [codegen id : 242] -Input [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] -Keys [2]: [channel#68, i_brand_id#178] -Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] -Aggregate Attributes [2]: [sum(sum_sales#122)#227, sum(number_sales#123)#228] -Results [6]: [channel#68, i_brand_id#178, null AS i_class_id#229, null AS i_category_id#230, sum(sum_sales#122)#227 AS sum(sum_sales)#231, sum(number_sales#123)#228 AS sum(number_sales)#232] +Input [5]: [channel#46, i_brand_id#7, sum#161, isEmpty#162, sum#163] +Keys [2]: [channel#46, i_brand_id#7] +Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] +Aggregate Attributes [2]: [sum(sum_sales#86)#165, sum(number_sales#87)#166] +Results [6]: [channel#46, i_brand_id#7, null AS i_class_id#167, null AS i_category_id#168, sum(sum_sales#86)#165 AS sum(sum_sales)#169, sum(number_sales#87)#166 AS sum(number_sales)#170] (152) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#171, isEmpty#172, count#173] (153) HashAggregate [codegen id : 268] -Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] -Keys [3]: [i_brand_id#233, i_class_id#234, i_category_id#235] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241, count(1)#242] -Results [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sales#63, count(1)#242 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#171, isEmpty#172, count#173] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#174, count(1)#175] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sales#41, count(1)#175 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176] (154) Filter [codegen id : 268] -Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (155) Project [codegen id : 268] -Output [6]: [store AS channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64] -Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] +Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176] (156) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#177, isEmpty#178, count#179] (157) HashAggregate [codegen id : 294] -Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] -Keys [3]: [i_brand_id#244, i_class_id#245, i_category_id#246] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252, count(1)#253] -Results [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sales#87, count(1)#253 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#177, isEmpty#178, count#179] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#180, count(1)#181] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sales#58, count(1)#181 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182] (158) Filter [codegen id : 294] -Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (159) Project [codegen id : 294] -Output [6]: [catalog AS channel#90, i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88] -Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] +Output [6]: [catalog AS channel#183, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182] (160) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#260] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#184, isEmpty#185, count#186] (161) HashAggregate [codegen id : 320] -Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#260] -Keys [3]: [i_brand_id#255, i_class_id#256, i_category_id#257] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263, count(1)#264] -Results [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263 AS sales#109, count(1)#264 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#184, isEmpty#185, count#186] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#187, count(1)#188] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sales#73, count(1)#188 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189] (162) Filter [codegen id : 320] -Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (163) Project [codegen id : 320] -Output [6]: [web AS channel#112, i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110] -Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] +Output [6]: [web AS channel#190, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189] (164) Union (165) HashAggregate [codegen id : 321] -Input [6]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64] -Keys [4]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#266, isEmpty#267, sum#268] -Results [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] +Aggregate Attributes [3]: [sum#191, isEmpty#192, sum#193] +Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#194, isEmpty#195, sum#196] (166) Exchange -Input [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] -Arguments: hashpartitioning(channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, 5), ENSURE_REQUIREMENTS, [id=#272] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#194, isEmpty#195, sum#196] +Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#197] (167) HashAggregate [codegen id : 322] -Input [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] -Keys [4]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#273, sum(number_sales#64)#274] -Results [3]: [channel#68, sum(sales#63)#273 AS sum_sales#122, sum(number_sales#64)#274 AS number_sales#123] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#194, isEmpty#195, sum#196] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#41), sum(number_sales#42)] +Aggregate Attributes [2]: [sum(sales#41)#198, sum(number_sales#42)#199] +Results [3]: [channel#46, sum(sales#41)#198 AS sum_sales#86, sum(number_sales#42)#199 AS number_sales#87] (168) HashAggregate [codegen id : 322] -Input [3]: [channel#68, sum_sales#122, number_sales#123] -Keys [1]: [channel#68] -Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] -Aggregate Attributes [3]: [sum#275, isEmpty#276, sum#277] -Results [4]: [channel#68, sum#278, isEmpty#279, sum#280] +Input [3]: [channel#46, sum_sales#86, number_sales#87] +Keys [1]: [channel#46] +Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] +Aggregate Attributes [3]: [sum#200, isEmpty#201, sum#202] +Results [4]: [channel#46, sum#203, isEmpty#204, sum#205] (169) Exchange -Input [4]: [channel#68, sum#278, isEmpty#279, sum#280] -Arguments: hashpartitioning(channel#68, 5), ENSURE_REQUIREMENTS, [id=#281] +Input [4]: [channel#46, sum#203, isEmpty#204, sum#205] +Arguments: hashpartitioning(channel#46, 5), ENSURE_REQUIREMENTS, [id=#206] (170) HashAggregate [codegen id : 323] -Input [4]: [channel#68, sum#278, isEmpty#279, sum#280] -Keys [1]: [channel#68] -Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] -Aggregate Attributes [2]: [sum(sum_sales#122)#282, sum(number_sales#123)#283] -Results [6]: [channel#68, null AS i_brand_id#284, null AS i_class_id#285, null AS i_category_id#286, sum(sum_sales#122)#282 AS sum(sum_sales)#287, sum(number_sales#123)#283 AS sum(number_sales)#288] +Input [4]: [channel#46, sum#203, isEmpty#204, sum#205] +Keys [1]: [channel#46] +Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] +Aggregate Attributes [2]: [sum(sum_sales#86)#207, sum(number_sales#87)#208] +Results [6]: [channel#46, null AS i_brand_id#209, null AS i_class_id#210, null AS i_category_id#211, sum(sum_sales#86)#207 AS sum(sum_sales)#212, sum(number_sales#87)#208 AS sum(number_sales)#213] (171) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#214, isEmpty#215, count#216] (172) HashAggregate [codegen id : 349] -Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] -Keys [3]: [i_brand_id#289, i_class_id#290, i_category_id#291] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297, count(1)#298] -Results [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sales#63, count(1)#298 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#214, isEmpty#215, count#216] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#217, count(1)#218] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#217 AS sales#41, count(1)#218 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#217 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219] (173) Filter [codegen id : 349] -Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (174) Project [codegen id : 349] -Output [6]: [store AS channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64] -Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] +Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219] (175) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] (176) HashAggregate [codegen id : 375] -Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] -Keys [3]: [i_brand_id#300, i_class_id#301, i_category_id#302] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308, count(1)#309] -Results [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sales#87, count(1)#309 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#223, count(1)#224] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sales#58, count(1)#224 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225] (177) Filter [codegen id : 375] -Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (178) Project [codegen id : 375] -Output [6]: [catalog AS channel#90, i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88] -Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] +Output [6]: [catalog AS channel#226, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225] (179) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum#314, isEmpty#315, count#316] +Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#227, isEmpty#228, count#229] (180) HashAggregate [codegen id : 401] -Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum#314, isEmpty#315, count#316] -Keys [3]: [i_brand_id#311, i_class_id#312, i_category_id#313] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319, count(1)#320] -Results [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319 AS sales#109, count(1)#320 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#227, isEmpty#228, count#229] +Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#230, count(1)#231] +Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sales#73, count(1)#231 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232] (181) Filter [codegen id : 401] -Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) (182) Project [codegen id : 401] -Output [6]: [web AS channel#112, i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110] -Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] +Output [6]: [web AS channel#233, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] +Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232] (183) Union (184) HashAggregate [codegen id : 402] -Input [6]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64] -Keys [4]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291] -Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] -Aggregate Attributes [3]: [sum#322, isEmpty#323, sum#324] -Results [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] +Aggregate Attributes [3]: [sum#234, isEmpty#235, sum#236] +Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#237, isEmpty#238, sum#239] (185) Exchange -Input [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] -Arguments: hashpartitioning(channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, 5), ENSURE_REQUIREMENTS, [id=#328] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#237, isEmpty#238, sum#239] +Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#240] (186) HashAggregate [codegen id : 403] -Input [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] -Keys [4]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291] -Functions [2]: [sum(sales#63), sum(number_sales#64)] -Aggregate Attributes [2]: [sum(sales#63)#329, sum(number_sales#64)#330] -Results [2]: [sum(sales#63)#329 AS sum_sales#122, sum(number_sales#64)#330 AS number_sales#123] +Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#237, isEmpty#238, sum#239] +Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] +Functions [2]: [sum(sales#41), sum(number_sales#42)] +Aggregate Attributes [2]: [sum(sales#41)#241, sum(number_sales#42)#242] +Results [2]: [sum(sales#41)#241 AS sum_sales#86, sum(number_sales#42)#242 AS number_sales#87] (187) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#122, number_sales#123] +Input [2]: [sum_sales#86, number_sales#87] Keys: [] -Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] -Aggregate Attributes [3]: [sum#331, isEmpty#332, sum#333] -Results [3]: [sum#334, isEmpty#335, sum#336] +Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] +Aggregate Attributes [3]: [sum#243, isEmpty#244, sum#245] +Results [3]: [sum#246, isEmpty#247, sum#248] (188) Exchange -Input [3]: [sum#334, isEmpty#335, sum#336] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#337] +Input [3]: [sum#246, isEmpty#247, sum#248] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#249] (189) HashAggregate [codegen id : 404] -Input [3]: [sum#334, isEmpty#335, sum#336] +Input [3]: [sum#246, isEmpty#247, sum#248] Keys: [] -Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] -Aggregate Attributes [2]: [sum(sum_sales#122)#338, sum(number_sales#123)#339] -Results [6]: [null AS channel#340, null AS i_brand_id#341, null AS i_class_id#342, null AS i_category_id#343, sum(sum_sales#122)#338 AS sum(sum_sales)#344, sum(number_sales#123)#339 AS sum(number_sales)#345] +Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] +Aggregate Attributes [2]: [sum(sum_sales#86)#250, sum(number_sales#87)#251] +Results [6]: [null AS channel#252, null AS i_brand_id#253, null AS i_class_id#254, null AS i_category_id#255, sum(sum_sales#86)#250 AS sum(sum_sales)#256, sum(number_sales#87)#251 AS sum(number_sales)#257] (190) Union (191) HashAggregate [codegen id : 405] -Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] -Keys [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Keys [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Results [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] (192) Exchange -Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] -Arguments: hashpartitioning(channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123, 5), ENSURE_REQUIREMENTS, [id=#346] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87, 5), ENSURE_REQUIREMENTS, [id=#258] (193) HashAggregate [codegen id : 406] -Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] -Keys [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Keys [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Results [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] (194) TakeOrderedAndProject -Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] -Arguments: 100, [channel#68 ASC NULLS FIRST, i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Arguments: 100, [channel#46 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#66, [id=#67] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] * HashAggregate (221) +- Exchange (220) +- * HashAggregate (219) @@ -1145,193 +1145,193 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (195) Scan parquet default.store_sales -Output [3]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349] +Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#349), dynamicpruningexpression(ss_sold_date_sk#349 IN dynamicpruning#350)] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#259)] ReadSchema: struct (196) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349] +Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] (197) Scan parquet default.date_dim -Output [2]: [d_date_sk#351, d_year#352] +Output [2]: [d_date_sk#12, d_year#13] 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 (198) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#351, d_year#352] +Input [2]: [d_date_sk#12, d_year#13] (199) Filter [codegen id : 1] -Input [2]: [d_date_sk#351, d_year#352] -Condition : (((isnotnull(d_year#352) AND (d_year#352 >= 1999)) AND (d_year#352 <= 2001)) AND isnotnull(d_date_sk#351)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) (200) Project [codegen id : 1] -Output [1]: [d_date_sk#351] -Input [2]: [d_date_sk#351, d_year#352] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (201) BroadcastExchange -Input [1]: [d_date_sk#351] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#353] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#260] (202) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#349] -Right keys [1]: [d_date_sk#351] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#12] Join condition: None (203) Project [codegen id : 2] -Output [2]: [ss_quantity#347 AS quantity#354, ss_list_price#348 AS list_price#355] -Input [4]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349, d_date_sk#351] +Output [2]: [ss_quantity#2 AS quantity#261, ss_list_price#3 AS list_price#262] +Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] (204) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358] +Output [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#358), dynamicpruningexpression(cs_sold_date_sk#358 IN dynamicpruning#359)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#263)] ReadSchema: struct (205) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358] +Input [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] (206) Scan parquet default.date_dim -Output [2]: [d_date_sk#360, d_year#361] +Output [2]: [d_date_sk#12, d_year#13] 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 (207) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#360, d_year#361] +Input [2]: [d_date_sk#12, d_year#13] (208) Filter [codegen id : 3] -Input [2]: [d_date_sk#360, d_year#361] -Condition : (((isnotnull(d_year#361) AND (d_year#361 >= 1998)) AND (d_year#361 <= 2000)) AND isnotnull(d_date_sk#360)) +Input [2]: [d_date_sk#12, d_year#13] +Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) (209) Project [codegen id : 3] -Output [1]: [d_date_sk#360] -Input [2]: [d_date_sk#360, d_year#361] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_year#13] (210) BroadcastExchange -Input [1]: [d_date_sk#360] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#362] +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#264] (211) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#358] -Right keys [1]: [d_date_sk#360] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#12] Join condition: None (212) Project [codegen id : 4] -Output [2]: [cs_quantity#356 AS quantity#363, cs_list_price#357 AS list_price#364] -Input [4]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358, d_date_sk#360] +Output [2]: [cs_quantity#47 AS quantity#265, cs_list_price#48 AS list_price#266] +Input [4]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, d_date_sk#12] (213) Scan parquet default.web_sales -Output [3]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367] +Output [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#367), dynamicpruningexpression(ws_sold_date_sk#367 IN dynamicpruning#359)] +PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#263)] ReadSchema: struct (214) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367] +Input [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] (215) ReusedExchange [Reuses operator id: 210] -Output [1]: [d_date_sk#368] +Output [1]: [d_date_sk#12] (216) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#367] -Right keys [1]: [d_date_sk#368] +Left keys [1]: [ws_sold_date_sk#24] +Right keys [1]: [d_date_sk#12] Join condition: None (217) Project [codegen id : 6] -Output [2]: [ws_quantity#365 AS quantity#369, ws_list_price#366 AS list_price#370] -Input [4]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367, d_date_sk#368] +Output [2]: [ws_quantity#62 AS quantity#267, ws_list_price#63 AS list_price#268] +Input [4]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, d_date_sk#12] (218) Union (219) HashAggregate [codegen id : 7] -Input [2]: [quantity#354, list_price#355] +Input [2]: [quantity#261, list_price#262] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#371, count#372] -Results [2]: [sum#373, count#374] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#269, count#270] +Results [2]: [sum#271, count#272] (220) Exchange -Input [2]: [sum#373, count#374] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#375] +Input [2]: [sum#271, count#272] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#273] (221) HashAggregate [codegen id : 8] -Input [2]: [sum#373, count#374] +Input [2]: [sum#271, count#272] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))#376] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))#376 AS average_sales#377] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#274] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#274 AS average_sales#275] -Subquery:2 Hosting operator id = 195 Hosting Expression = ss_sold_date_sk#349 IN dynamicpruning#350 +Subquery:2 Hosting operator id = 195 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#259 ReusedExchange (222) (222) ReusedExchange [Reuses operator id: 201] -Output [1]: [d_date_sk#351] +Output [1]: [d_date_sk#12] -Subquery:3 Hosting operator id = 204 Hosting Expression = cs_sold_date_sk#358 IN dynamicpruning#359 +Subquery:3 Hosting operator id = 204 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#263 ReusedExchange (223) (223) ReusedExchange [Reuses operator id: 210] -Output [1]: [d_date_sk#360] +Output [1]: [d_date_sk#12] -Subquery:4 Hosting operator id = 213 Hosting Expression = ws_sold_date_sk#367 IN dynamicpruning#359 +Subquery:4 Hosting operator id = 213 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#263 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (224) (224) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#50] +Output [1]: [d_date_sk#12] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 ReusedExchange (225) -(225) ReusedExchange [Reuses operator id: 26] -Output [1]: [d_date_sk#29] +(225) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#12] -Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 +Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 -Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#5 -Subquery:13 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:13 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:14 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:14 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:15 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:15 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:16 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:16 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:17 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:17 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:18 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:18 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:19 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:19 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:20 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:20 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:21 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:21 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:22 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:22 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:23 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:23 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] -Subquery:24 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] +Subquery:24 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index e351f9e687027..1634c1e247a12 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -104,53 +104,53 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Exchange [brand_id,class_id,category_id] #7 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (5) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #10 + BroadcastExchange #11 WholeStageCodegen (3) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (1) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + Filter [i_item_sk] + ColumnarToRow InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - ReusedExchange [d_date_sk] #8 + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter BroadcastExchange #12 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt index 3f2b62a7b8fa4..b2934b6bb8242 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt @@ -81,57 +81,57 @@ Input [3]: [i_item_sk#5, i_current_price#6, i_category#7] Condition : (isnotnull(i_current_price#6) AND isnotnull(i_item_sk#5)) (7) Scan parquet default.item -Output [2]: [i_current_price#8, i_category#9] +Output [2]: [i_current_price#6, i_category#7] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [i_current_price#8, i_category#9] +Input [2]: [i_current_price#6, i_category#7] (9) Filter [codegen id : 1] -Input [2]: [i_current_price#8, i_category#9] -Condition : isnotnull(i_category#9) +Input [2]: [i_current_price#6, i_category#7] +Condition : isnotnull(i_category#7) (10) HashAggregate [codegen id : 1] -Input [2]: [i_current_price#8, i_category#9] -Keys [1]: [i_category#9] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#8))] -Aggregate Attributes [2]: [sum#10, count#11] -Results [3]: [i_category#9, sum#12, count#13] +Input [2]: [i_current_price#6, i_category#7] +Keys [1]: [i_category#7] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#6))] +Aggregate Attributes [2]: [sum#8, count#9] +Results [3]: [i_category#7, sum#10, count#11] (11) Exchange -Input [3]: [i_category#9, sum#12, count#13] -Arguments: hashpartitioning(i_category#9, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [3]: [i_category#7, sum#10, count#11] +Arguments: hashpartitioning(i_category#7, 5), ENSURE_REQUIREMENTS, [id=#12] (12) HashAggregate [codegen id : 2] -Input [3]: [i_category#9, sum#12, count#13] -Keys [1]: [i_category#9] -Functions [1]: [avg(UnscaledValue(i_current_price#8))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#8))#15] -Results [2]: [cast((avg(UnscaledValue(i_current_price#8))#15 / 100.0) as decimal(11,6)) AS avg(i_current_price)#16, i_category#9] +Input [3]: [i_category#7, sum#10, count#11] +Keys [1]: [i_category#7] +Functions [1]: [avg(UnscaledValue(i_current_price#6))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#6))#13] +Results [2]: [cast((avg(UnscaledValue(i_current_price#6))#13 / 100.0) as decimal(11,6)) AS avg(i_current_price)#14, i_category#7 AS i_category#7#15] (13) BroadcastExchange -Input [2]: [avg(i_current_price)#16, i_category#9] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#17] +Input [2]: [avg(i_current_price)#14, i_category#7#15] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#16] (14) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_category#7] -Right keys [1]: [i_category#9] +Right keys [1]: [i_category#7#15] Join condition: None (15) Filter [codegen id : 3] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] -Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#16)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] +Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#14)), DecimalType(14,7), true)) (16) Project [codegen id : 3] Output [1]: [i_item_sk#5] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] (17) BroadcastExchange Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (18) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#1] @@ -143,143 +143,143 @@ Output [2]: [ss_customer_sk#2, ss_sold_date_sk#3] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, i_item_sk#5] (20) Scan parquet default.date_dim -Output [2]: [d_date_sk#19, d_month_seq#20] +Output [2]: [d_date_sk#18, d_month_seq#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#19, d_month_seq#20] +Input [2]: [d_date_sk#18, d_month_seq#19] (22) Filter [codegen id : 4] -Input [2]: [d_date_sk#19, d_month_seq#20] -Condition : ((isnotnull(d_month_seq#20) AND (d_month_seq#20 = Subquery scalar-subquery#21, [id=#22])) AND isnotnull(d_date_sk#19)) +Input [2]: [d_date_sk#18, d_month_seq#19] +Condition : ((isnotnull(d_month_seq#19) AND (d_month_seq#19 = Subquery scalar-subquery#20, [id=#21])) AND isnotnull(d_date_sk#18)) (23) Project [codegen id : 4] -Output [1]: [d_date_sk#19] -Input [2]: [d_date_sk#19, d_month_seq#20] +Output [1]: [d_date_sk#18] +Input [2]: [d_date_sk#18, d_month_seq#19] (24) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [d_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (25) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#19] +Right keys [1]: [d_date_sk#18] Join condition: None (26) Project [codegen id : 5] Output [1]: [ss_customer_sk#2] -Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#19] +Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#18] (27) Exchange Input [1]: [ss_customer_sk#2] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#23] (28) Sort [codegen id : 6] Input [1]: [ss_customer_sk#2] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#25, ca_state#26] +Output [2]: [ca_address_sk#24, ca_state#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#25, ca_state#26] +Input [2]: [ca_address_sk#24, ca_state#25] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#25, ca_state#26] -Condition : isnotnull(ca_address_sk#25) +Input [2]: [ca_address_sk#24, ca_state#25] +Condition : isnotnull(ca_address_sk#24) (32) Exchange -Input [2]: [ca_address_sk#25, ca_state#26] -Arguments: hashpartitioning(ca_address_sk#25, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] (33) Sort [codegen id : 8] -Input [2]: [ca_address_sk#25, ca_state#26] -Arguments: [ca_address_sk#25 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 (34) Scan parquet default.customer -Output [2]: [c_customer_sk#28, c_current_addr_sk#29] +Output [2]: [c_customer_sk#27, c_current_addr_sk#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 9] -Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Input [2]: [c_customer_sk#27, c_current_addr_sk#28] (36) Filter [codegen id : 9] -Input [2]: [c_customer_sk#28, c_current_addr_sk#29] -Condition : (isnotnull(c_current_addr_sk#29) AND isnotnull(c_customer_sk#28)) +Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Condition : (isnotnull(c_current_addr_sk#28) AND isnotnull(c_customer_sk#27)) (37) Exchange -Input [2]: [c_customer_sk#28, c_current_addr_sk#29] -Arguments: hashpartitioning(c_current_addr_sk#29, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Arguments: hashpartitioning(c_current_addr_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] (38) Sort [codegen id : 10] -Input [2]: [c_customer_sk#28, c_current_addr_sk#29] -Arguments: [c_current_addr_sk#29 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Arguments: [c_current_addr_sk#28 ASC NULLS FIRST], false, 0 (39) SortMergeJoin [codegen id : 11] -Left keys [1]: [ca_address_sk#25] -Right keys [1]: [c_current_addr_sk#29] +Left keys [1]: [ca_address_sk#24] +Right keys [1]: [c_current_addr_sk#28] Join condition: None (40) Project [codegen id : 11] -Output [2]: [ca_state#26, c_customer_sk#28] -Input [4]: [ca_address_sk#25, ca_state#26, c_customer_sk#28, c_current_addr_sk#29] +Output [2]: [ca_state#25, c_customer_sk#27] +Input [4]: [ca_address_sk#24, ca_state#25, c_customer_sk#27, c_current_addr_sk#28] (41) Exchange -Input [2]: [ca_state#26, c_customer_sk#28] -Arguments: hashpartitioning(c_customer_sk#28, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [2]: [ca_state#25, c_customer_sk#27] +Arguments: hashpartitioning(c_customer_sk#27, 5), ENSURE_REQUIREMENTS, [id=#30] (42) Sort [codegen id : 12] -Input [2]: [ca_state#26, c_customer_sk#28] -Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 +Input [2]: [ca_state#25, c_customer_sk#27] +Arguments: [c_customer_sk#27 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 13] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#28] +Right keys [1]: [c_customer_sk#27] Join condition: None (44) Project [codegen id : 13] -Output [1]: [ca_state#26] -Input [3]: [ss_customer_sk#2, ca_state#26, c_customer_sk#28] +Output [1]: [ca_state#25] +Input [3]: [ss_customer_sk#2, ca_state#25, c_customer_sk#27] (45) HashAggregate [codegen id : 13] -Input [1]: [ca_state#26] -Keys [1]: [ca_state#26] +Input [1]: [ca_state#25] +Keys [1]: [ca_state#25] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#32] -Results [2]: [ca_state#26, count#33] +Aggregate Attributes [1]: [count#31] +Results [2]: [ca_state#25, count#32] (46) Exchange -Input [2]: [ca_state#26, count#33] -Arguments: hashpartitioning(ca_state#26, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [2]: [ca_state#25, count#32] +Arguments: hashpartitioning(ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#33] (47) HashAggregate [codegen id : 14] -Input [2]: [ca_state#26, count#33] -Keys [1]: [ca_state#26] +Input [2]: [ca_state#25, count#32] +Keys [1]: [ca_state#25] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [4]: [ca_state#26 AS state#36, count(1)#35 AS cnt#37, count(1)#35 AS count(1)#38, ca_state#26] +Aggregate Attributes [1]: [count(1)#34] +Results [4]: [ca_state#25 AS state#35, count(1)#34 AS cnt#36, count(1)#34 AS count(1)#37, ca_state#25] (48) Filter [codegen id : 14] -Input [4]: [state#36, cnt#37, count(1)#38, ca_state#26] -Condition : (count(1)#38 >= 10) +Input [4]: [state#35, cnt#36, count(1)#37, ca_state#25] +Condition : (count(1)#37 >= 10) (49) Project [codegen id : 14] -Output [3]: [state#36, cnt#37, ca_state#26] -Input [4]: [state#36, cnt#37, count(1)#38, ca_state#26] +Output [3]: [state#35, cnt#36, ca_state#25] +Input [4]: [state#35, cnt#36, count(1)#37, ca_state#25] (50) TakeOrderedAndProject -Input [3]: [state#36, cnt#37, ca_state#26] -Arguments: 100, [cnt#37 ASC NULLS FIRST, ca_state#26 ASC NULLS FIRST], [state#36, cnt#37] +Input [3]: [state#35, cnt#36, ca_state#25] +Arguments: 100, [cnt#36 ASC NULLS FIRST, ca_state#25 ASC NULLS FIRST], [state#35, cnt#36] ===== Subqueries ===== @@ -288,9 +288,9 @@ ReusedExchange (51) (51) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#19] +Output [1]: [d_date_sk#18] -Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#21, [id=#22] +Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#20, [id=#21] * HashAggregate (58) +- Exchange (57) +- * HashAggregate (56) @@ -301,39 +301,39 @@ Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquer (52) Scan parquet default.date_dim -Output [3]: [d_month_seq#39, d_year#40, d_moy#41] +Output [3]: [d_month_seq#19, d_year#38, d_moy#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (53) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#39, d_year#40, d_moy#41] +Input [3]: [d_month_seq#19, d_year#38, d_moy#39] (54) Filter [codegen id : 1] -Input [3]: [d_month_seq#39, d_year#40, d_moy#41] -Condition : (((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2000)) AND (d_moy#41 = 1)) +Input [3]: [d_month_seq#19, d_year#38, d_moy#39] +Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) (55) Project [codegen id : 1] -Output [1]: [d_month_seq#39] -Input [3]: [d_month_seq#39, d_year#40, d_moy#41] +Output [1]: [d_month_seq#19] +Input [3]: [d_month_seq#19, d_year#38, d_moy#39] (56) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#39] -Keys [1]: [d_month_seq#39] +Input [1]: [d_month_seq#19] +Keys [1]: [d_month_seq#19] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#39] +Results [1]: [d_month_seq#19] (57) Exchange -Input [1]: [d_month_seq#39] -Arguments: hashpartitioning(d_month_seq#39, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [1]: [d_month_seq#19] +Arguments: hashpartitioning(d_month_seq#19, 5), ENSURE_REQUIREMENTS, [id=#40] (58) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#39] -Keys [1]: [d_month_seq#39] +Input [1]: [d_month_seq#19] +Keys [1]: [d_month_seq#19] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#39] +Results [1]: [d_month_seq#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt index f3badf6efe6b6..23b705e3b97e7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt @@ -38,7 +38,7 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] InputAdapter Exchange [i_category] #6 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt index b37db85388e0f..8598023b0276d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt @@ -160,57 +160,57 @@ Input [3]: [i_item_sk#16, i_current_price#17, i_category#18] Condition : (isnotnull(i_current_price#17) AND isnotnull(i_item_sk#16)) (26) Scan parquet default.item -Output [2]: [i_current_price#19, i_category#20] +Output [2]: [i_current_price#17, i_category#18] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (27) ColumnarToRow [codegen id : 4] -Input [2]: [i_current_price#19, i_category#20] +Input [2]: [i_current_price#17, i_category#18] (28) Filter [codegen id : 4] -Input [2]: [i_current_price#19, i_category#20] -Condition : isnotnull(i_category#20) +Input [2]: [i_current_price#17, i_category#18] +Condition : isnotnull(i_category#18) (29) HashAggregate [codegen id : 4] -Input [2]: [i_current_price#19, i_category#20] -Keys [1]: [i_category#20] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#19))] -Aggregate Attributes [2]: [sum#21, count#22] -Results [3]: [i_category#20, sum#23, count#24] +Input [2]: [i_current_price#17, i_category#18] +Keys [1]: [i_category#18] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] +Aggregate Attributes [2]: [sum#19, count#20] +Results [3]: [i_category#18, sum#21, count#22] (30) Exchange -Input [3]: [i_category#20, sum#23, count#24] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [i_category#18, sum#21, count#22] +Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [id=#23] (31) HashAggregate [codegen id : 5] -Input [3]: [i_category#20, sum#23, count#24] -Keys [1]: [i_category#20] -Functions [1]: [avg(UnscaledValue(i_current_price#19))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#19))#26] -Results [2]: [cast((avg(UnscaledValue(i_current_price#19))#26 / 100.0) as decimal(11,6)) AS avg(i_current_price)#27, i_category#20] +Input [3]: [i_category#18, sum#21, count#22] +Keys [1]: [i_category#18] +Functions [1]: [avg(UnscaledValue(i_current_price#17))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#17))#24] +Results [2]: [cast((avg(UnscaledValue(i_current_price#17))#24 / 100.0) as decimal(11,6)) AS avg(i_current_price)#25, i_category#18 AS i_category#18#26] (32) BroadcastExchange -Input [2]: [avg(i_current_price)#27, i_category#20] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#28] +Input [2]: [avg(i_current_price)#25, i_category#18#26] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#27] (33) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_category#18] -Right keys [1]: [i_category#20] +Right keys [1]: [i_category#18#26] Join condition: None (34) Filter [codegen id : 6] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] -Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#27)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] +Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#25)), DecimalType(14,7), true)) (35) Project [codegen id : 6] Output [1]: [i_item_sk#16] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] (36) BroadcastExchange Input [1]: [i_item_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (37) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#6] @@ -225,31 +225,31 @@ Input [3]: [ca_state#2, ss_item_sk#6, i_item_sk#16] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#30] -Results [2]: [ca_state#2, count#31] +Aggregate Attributes [1]: [count#29] +Results [2]: [ca_state#2, count#30] (40) Exchange -Input [2]: [ca_state#2, count#31] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [2]: [ca_state#2, count#30] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#31] (41) HashAggregate [codegen id : 8] -Input [2]: [ca_state#2, count#31] +Input [2]: [ca_state#2, count#30] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#33] -Results [4]: [ca_state#2 AS state#34, count(1)#33 AS cnt#35, count(1)#33 AS count(1)#36, ca_state#2] +Aggregate Attributes [1]: [count(1)#32] +Results [4]: [ca_state#2 AS state#33, count(1)#32 AS cnt#34, count(1)#32 AS count(1)#35, ca_state#2] (42) Filter [codegen id : 8] -Input [4]: [state#34, cnt#35, count(1)#36, ca_state#2] -Condition : (count(1)#36 >= 10) +Input [4]: [state#33, cnt#34, count(1)#35, ca_state#2] +Condition : (count(1)#35 >= 10) (43) Project [codegen id : 8] -Output [3]: [state#34, cnt#35, ca_state#2] -Input [4]: [state#34, cnt#35, count(1)#36, ca_state#2] +Output [3]: [state#33, cnt#34, ca_state#2] +Input [4]: [state#33, cnt#34, count(1)#35, ca_state#2] (44) TakeOrderedAndProject -Input [3]: [state#34, cnt#35, ca_state#2] -Arguments: 100, [cnt#35 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#34, cnt#35] +Input [3]: [state#33, cnt#34, ca_state#2] +Arguments: 100, [cnt#34 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#33, cnt#34] ===== Subqueries ===== @@ -271,39 +271,39 @@ Subquery:2 Hosting operator id = 18 Hosting Expression = Subquery scalar-subquer (46) Scan parquet default.date_dim -Output [3]: [d_month_seq#37, d_year#38, d_moy#39] +Output [3]: [d_month_seq#12, d_year#36, d_moy#37] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (47) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#38, d_moy#39] +Input [3]: [d_month_seq#12, d_year#36, d_moy#37] (48) Filter [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#38, d_moy#39] -Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) +Input [3]: [d_month_seq#12, d_year#36, d_moy#37] +Condition : (((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2000)) AND (d_moy#37 = 1)) (49) Project [codegen id : 1] -Output [1]: [d_month_seq#37] -Input [3]: [d_month_seq#37, d_year#38, d_moy#39] +Output [1]: [d_month_seq#12] +Input [3]: [d_month_seq#12, d_year#36, d_moy#37] (50) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#37] -Keys [1]: [d_month_seq#37] +Input [1]: [d_month_seq#12] +Keys [1]: [d_month_seq#12] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#37] +Results [1]: [d_month_seq#12] (51) Exchange -Input [1]: [d_month_seq#37] -Arguments: hashpartitioning(d_month_seq#37, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [1]: [d_month_seq#12] +Arguments: hashpartitioning(d_month_seq#12, 5), ENSURE_REQUIREMENTS, [id=#38] (52) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#37] -Keys [1]: [d_month_seq#37] +Input [1]: [d_month_seq#12] +Keys [1]: [d_month_seq#12] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#37] +Results [1]: [d_month_seq#12] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index 4ba09283e73cd..ceddf1b04fc78 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -68,7 +68,7 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] InputAdapter Exchange [i_category] #8 WholeStageCodegen (4) 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 413a918da6a0f..6d9adf1d38106 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 @@ -445,7 +445,7 @@ Arguments: [rank(sumsales#29) windowspecdefinition(i_category#20, sumsales#29 DE (73) Filter [codegen id : 82] Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] -Condition : (rk#148 <= 100) +Condition : (isnotnull(rk#148) AND (rk#148 <= 100)) (74) TakeOrderedAndProject Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] 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 fa822f3ac9ed7..ae6b3ff4d7542 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 @@ -430,7 +430,7 @@ Arguments: [rank(sumsales#28) windowspecdefinition(i_category#19, sumsales#28 DE (70) Filter [codegen id : 55] Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] -Condition : (rk#147 <= 100) +Condition : (isnotnull(rk#147) AND (rk#147 <= 100)) (71) TakeOrderedAndProject Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] 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 e024d06c710a7..3c65529504320 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 @@ -207,7 +207,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (ranking#19 <= 5) +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#16] 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 5ae5fd82839cc..406acb0e0a27f 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 @@ -207,7 +207,7 @@ Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST (32) Filter [codegen id : 7] Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (ranking#19 <= 5) +Condition : (isnotnull(ranking#19) AND (ranking#19 <= 5)) (33) Project [codegen id : 7] Output [1]: [s_state#16] From 39f161576f3c708abeaae302b7f1d4ab8b2e6a04 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 25 Mar 2021 15:55:19 +0800 Subject: [PATCH 30/38] regen golden files --- .../q14a.sf100/explain.txt | 844 +++++----- .../q14a.sf100/simplified.txt | 158 +- .../approved-plans-v1_4/q14a/explain.txt | 678 ++++---- .../approved-plans-v1_4/q14a/simplified.txt | 80 +- .../q14b.sf100/explain.txt | 762 ++++----- .../q14b.sf100/simplified.txt | 146 +- .../approved-plans-v1_4/q14b/explain.txt | 630 +++---- .../approved-plans-v1_4/q14b/simplified.txt | 80 +- .../approved-plans-v1_4/q32.sf100/explain.txt | 44 +- .../q32.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q32/explain.txt | 86 +- .../approved-plans-v1_4/q32/simplified.txt | 2 +- .../approved-plans-v1_4/q41.sf100/explain.txt | 50 +- .../q41.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q41/explain.txt | 50 +- .../approved-plans-v1_4/q41/simplified.txt | 2 +- .../approved-plans-v1_4/q6.sf100/explain.txt | 186 +-- .../q6.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q6/explain.txt | 100 +- .../approved-plans-v1_4/q6/simplified.txt | 2 +- .../approved-plans-v1_4/q92.sf100/explain.txt | 66 +- .../q92.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q92/explain.txt | 100 +- .../approved-plans-v1_4/q92/simplified.txt | 2 +- .../approved-plans-v2_7/q14.sf100/explain.txt | 762 ++++----- .../q14.sf100/simplified.txt | 146 +- .../approved-plans-v2_7/q14/explain.txt | 630 +++---- .../approved-plans-v2_7/q14/simplified.txt | 80 +- .../q14a.sf100/explain.txt | 1446 ++++++++--------- .../q14a.sf100/simplified.txt | 210 ++- .../approved-plans-v2_7/q14a/explain.txt | 1158 ++++++------- .../approved-plans-v2_7/q14a/simplified.txt | 80 +- .../approved-plans-v2_7/q6.sf100/explain.txt | 186 +-- .../q6.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q6/explain.txt | 100 +- .../approved-plans-v2_7/q6/simplified.txt | 2 +- 36 files changed, 4435 insertions(+), 4443 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt index d4ab3f60494d9..e4ec487623d2c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt @@ -37,39 +37,39 @@ TakeOrderedAndProject (134) : : : : +- * HashAggregate (44) : : : : +- Exchange (43) : : : : +- * HashAggregate (42) - : : : : +- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) + : : : : +- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Project (18) + : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : :- * Filter (11) + : : : : : : +- * ColumnarToRow (10) + : : : : : : +- Scan parquet default.store_sales (9) + : : : : : +- BroadcastExchange (16) + : : : : : +- * Project (15) + : : : : : +- * Filter (14) + : : : : : +- * ColumnarToRow (13) + : : : : : +- Scan parquet default.date_dim (12) + : : : : +- BroadcastExchange (39) + : : : : +- SortMergeJoin LeftSemi (38) + : : : : :- * Sort (23) + : : : : : +- Exchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (37) + : : : : +- Exchange (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- Scan parquet default.catalog_sales (24) + : : : : : +- ReusedExchange (27) + : : : : +- BroadcastExchange (33) + : : : : +- * Filter (32) + : : : : +- * ColumnarToRow (31) + : : : : +- Scan parquet default.item (30) : : : +- * Sort (57) : : : +- Exchange (56) : : : +- * Project (55) @@ -165,588 +165,588 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 20] +(7) ColumnarToRow [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 20] +(8) Filter [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 5] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +(10) ColumnarToRow [codegen id : 10] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -(11) Filter [codegen id : 5] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +(11) Filter [codegen id : 10] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_item_sk#11) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#14, d_year#15] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#14, d_year#15] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1999)) AND (d_year#15 <= 2001)) AND isnotnull(d_date_sk#14)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#14] +Input [2]: [d_date_sk#14, d_year#15] (16) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(17) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(17) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#14] Join condition: None -(18) Project [codegen id : 5] -Output [1]: [ss_item_sk#1] -Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] +(18) Project [codegen id : 10] +Output [1]: [ss_item_sk#11] +Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] (19) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) -(22) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] +(22) Exchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join condition: None +(23) Sort [codegen id : 5] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 -(24) Project [codegen id : 5] -Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] -Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] - -(25) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] - -(26) Sort [codegen id : 6] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 - -(27) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] +(24) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +(25) ColumnarToRow [codegen id : 8] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] -(29) Filter [codegen id : 9] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +(26) Filter [codegen id : 8] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#22) -(30) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +(27) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#24] -(31) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +(28) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None -(32) Project [codegen id : 9] -Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] +(29) Project [codegen id : 8] +Output [1]: [cs_item_sk#22] +Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] -(33) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(30) Scan parquet default.item +Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(31) ColumnarToRow [codegen id : 7] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -(35) Filter [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +(32) Filter [codegen id : 7] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Condition : isnotnull(i_item_sk#25) -(36) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +(33) BroadcastExchange +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_item_sk#22] +Right keys [1]: [i_item_sk#25] Join condition: None -(38) Project [codegen id : 9] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(35) Project [codegen id : 8] +Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -(39) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] +(36) Exchange +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] -(40) Sort [codegen id : 10] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(37) Sort [codegen id : 9] +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 -(41) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +(38) SortMergeJoin +Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] Join condition: None -(42) HashAggregate [codegen id : 11] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(39) BroadcastExchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] + +(40) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_item_sk#11] +Right keys [1]: [i_item_sk#17] +Join condition: None + +(41) Project [codegen id : 10] +Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] +Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] + +(42) HashAggregate [codegen id : 10] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (43) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(44) HashAggregate [codegen id : 12] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(44) HashAggregate [codegen id : 11] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (45) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] -(46) Sort [codegen id : 13] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 12] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +(48) ColumnarToRow [codegen id : 15] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -(49) Filter [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(49) Filter [codegen id : 15] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_item_sk#37) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#39] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(51) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#39] Join condition: None -(52) Project [codegen id : 16] -Output [1]: [ws_item_sk#26] -Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] +(52) Project [codegen id : 15] +Output [1]: [ws_item_sk#37] +Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] -(53) ReusedExchange [Reuses operator id: 36] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(53) ReusedExchange [Reuses operator id: 33] +Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(54) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#40] Join condition: None -(55) Project [codegen id : 16] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(55) Project [codegen id : 15] +Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] (56) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] -(57) Sort [codegen id : 17] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 16] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] +Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] Join condition: None -(59) HashAggregate [codegen id : 18] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(59) HashAggregate [codegen id : 17] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (60) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] -(61) HashAggregate [codegen id : 19] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(61) HashAggregate [codegen id : 18] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (62) BroadcastExchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] -(63) BroadcastHashJoin [codegen id : 20] +(63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#16, class_id#17, category_id#18] +Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join condition: None -(64) Project [codegen id : 20] -Output [1]: [i_item_sk#7 AS ss_item_sk#31] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] +(64) Project [codegen id : 19] +Output [1]: [i_item_sk#7 AS ss_item_sk#47] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] (65) Exchange -Input [1]: [ss_item_sk#31] -Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [1]: [ss_item_sk#47] +Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] -(66) Sort [codegen id : 21] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 20] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#47] Join condition: None (68) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#33] +Output [3]: [d_date_sk#49, d_year#50, d_moy#51] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 22] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] +(69) ColumnarToRow [codegen id : 21] +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] -(70) Filter [codegen id : 22] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#33)) AND (d_year#13 = 2001)) AND (d_moy#33 = 11)) AND isnotnull(d_date_sk#12)) +(70) Filter [codegen id : 21] +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] +Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2001)) AND (d_moy#51 = 11)) AND isnotnull(d_date_sk#49)) -(71) Project [codegen id : 22] -Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] +(71) Project [codegen id : 21] +Output [1]: [d_date_sk#49] +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] (72) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] +Input [1]: [d_date_sk#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] -(73) BroadcastHashJoin [codegen id : 44] +(73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#49] Join condition: None -(74) Project [codegen id : 44] +(74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] (75) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(76) ColumnarToRow [codegen id : 22] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(77) Filter [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +(77) Filter [codegen id : 22] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Condition : isnotnull(i_item_sk#53) (78) Exchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: hashpartitioning(i_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#57] -(79) Sort [codegen id : 24] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 +(79) Sort [codegen id : 23] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: [i_item_sk#53 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(81) Sort [codegen id : 43] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 41] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [i_item_sk#53] +Right keys [1]: [ss_item_sk#47] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] -(84) BroadcastHashJoin [codegen id : 44] +(84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#53] Join condition: None -(85) Project [codegen id : 44] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(85) Project [codegen id : 42] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(86) HashAggregate [codegen id : 44] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +(86) HashAggregate [codegen id : 42] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] +Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#37, isEmpty#38, count#39] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] +Aggregate Attributes [3]: [sum#59, isEmpty#60, count#61] +Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] +Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#65] -(88) HashAggregate [codegen id : 45] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +(88) HashAggregate [codegen id : 43] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] +Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44, count(1)#45] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sales#46, count(1)#45 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66, count(1)#67] +Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] -(89) Filter [codegen id : 45] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48 as decimal(32,6)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(89) Filter [codegen id : 43] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(32,6)) > cast(Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(90) Project [codegen id : 45] -Output [6]: [sales#46, number_sales#47, store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] +(90) Project [codegen id : 43] +Output [6]: [sales#68, number_sales#69, store AS channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] (91) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Output [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#77), dynamicpruningexpression(cs_sold_date_sk#77 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 46] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +(92) ColumnarToRow [codegen id : 44] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -(93) Filter [codegen id : 46] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +(93) Filter [codegen id : 44] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Condition : isnotnull(cs_item_sk#74) (94) Exchange -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Arguments: hashpartitioning(cs_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#78] -(95) Sort [codegen id : 47] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -Arguments: [cs_item_sk#20 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 45] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Arguments: [cs_item_sk#74 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(97) Sort [codegen id : 66] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 63] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [cs_item_sk#74] +Right keys [1]: [ss_item_sk#47] Join condition: None (99) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#79] -(100) BroadcastHashJoin [codegen id : 89] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +(100) BroadcastHashJoin [codegen id : 85] +Left keys [1]: [cs_sold_date_sk#77] +Right keys [1]: [d_date_sk#79] Join condition: None -(101) Project [codegen id : 89] -Output [3]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53] -Input [5]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] +(101) Project [codegen id : 85] +Output [3]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76] +Input [5]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77, d_date_sk#79] (102) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -(103) BroadcastHashJoin [codegen id : 89] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +(103) BroadcastHashJoin [codegen id : 85] +Left keys [1]: [cs_item_sk#74] +Right keys [1]: [i_item_sk#80] Join condition: None -(104) Project [codegen id : 89] -Output [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(104) Project [codegen id : 85] +Output [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] +Input [7]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -(105) HashAggregate [codegen id : 89] -Input [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] +(105) HashAggregate [codegen id : 85] +Input [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] +Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] +Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] (106) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, [id=#90] -(107) HashAggregate [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] +(107) HashAggregate [codegen id : 86] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] +Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] -(108) Filter [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(108) Filter [codegen id : 86] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(109) Project [codegen id : 90] -Output [6]: [sales#64, number_sales#65, catalog AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] +(109) Project [codegen id : 86] +Output [6]: [sales#93, number_sales#94, catalog AS channel#96, i_brand_id#81, i_class_id#82, i_category_id#83] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] (110) Scan parquet default.web_sales -Output [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Output [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#100), dynamicpruningexpression(ws_sold_date_sk#100 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 91] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +(111) ColumnarToRow [codegen id : 87] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -(112) Filter [codegen id : 91] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(112) Filter [codegen id : 87] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Condition : isnotnull(ws_item_sk#97) (113) Exchange -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_item_sk#26, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Arguments: hashpartitioning(ws_item_sk#97, 5), ENSURE_REQUIREMENTS, [id=#101] -(114) Sort [codegen id : 92] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Arguments: [ws_item_sk#26 ASC NULLS FIRST], false, 0 +(114) Sort [codegen id : 88] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Arguments: [ws_item_sk#97 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(116) Sort [codegen id : 111] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(116) Sort [codegen id : 106] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (117) SortMergeJoin -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [ws_item_sk#97] +Right keys [1]: [ss_item_sk#47] Join condition: None (118) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#102] -(119) BroadcastHashJoin [codegen id : 134] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(119) BroadcastHashJoin [codegen id : 128] +Left keys [1]: [ws_sold_date_sk#100] +Right keys [1]: [d_date_sk#102] Join condition: None -(120) Project [codegen id : 134] -Output [3]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69] -Input [5]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] +(120) Project [codegen id : 128] +Output [3]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99] +Input [5]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100, d_date_sk#102] (121) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] -(122) BroadcastHashJoin [codegen id : 134] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(122) BroadcastHashJoin [codegen id : 128] +Left keys [1]: [ws_item_sk#97] +Right keys [1]: [i_item_sk#103] Join condition: None -(123) Project [codegen id : 134] -Output [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(123) Project [codegen id : 128] +Output [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] +Input [7]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] -(124) HashAggregate [codegen id : 134] -Input [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#71, isEmpty#72, count#73] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] +(124) HashAggregate [codegen id : 128] +Input [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] +Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#107, isEmpty#108, count#109] +Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] (125) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +Arguments: hashpartitioning(i_brand_id#104, i_class_id#105, i_category_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] -(126) HashAggregate [codegen id : 135] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78, count(1)#79] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sales#80, count(1)#79 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +(126) HashAggregate [codegen id : 129] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114, count(1)#115] +Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sales#116, count(1)#115 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] -(127) Filter [codegen id : 135] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(127) Filter [codegen id : 129] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(128) Project [codegen id : 135] -Output [6]: [sales#80, number_sales#81, web AS channel#83, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +(128) Project [codegen id : 129] +Output [6]: [sales#116, number_sales#117, web AS channel#119, i_brand_id#104, i_class_id#105, i_category_id#106] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] (129) Union -(130) Expand [codegen id : 136] -Input [6]: [sales#46, number_sales#47, channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [List(sales#46, number_sales#47, channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 0), List(sales#46, number_sales#47, channel#51, i_brand_id#8, i_class_id#9, null, 1), List(sales#46, number_sales#47, channel#51, i_brand_id#8, null, null, 3), List(sales#46, number_sales#47, channel#51, null, null, null, 7), List(sales#46, number_sales#47, null, null, null, null, 15)], [sales#46, number_sales#47, channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] +(130) Expand [codegen id : 130] +Input [6]: [sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: [List(sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, 0), List(sales#68, number_sales#69, channel#73, i_brand_id#54, i_class_id#55, null, 1), List(sales#68, number_sales#69, channel#73, i_brand_id#54, null, null, 3), List(sales#68, number_sales#69, channel#73, null, null, null, 7), List(sales#68, number_sales#69, null, null, null, null, 15)], [sales#68, number_sales#69, channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] -(131) HashAggregate [codegen id : 136] -Input [7]: [sales#46, number_sales#47, channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] -Keys [5]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#89, isEmpty#90, sum#91] -Results [8]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, sum#92, isEmpty#93, sum#94] +(131) HashAggregate [codegen id : 130] +Input [7]: [sales#68, number_sales#69, channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] +Keys [5]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#125, isEmpty#126, sum#127] +Results [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] (132) Exchange -Input [8]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, sum#92, isEmpty#93, sum#94] -Arguments: hashpartitioning(channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, 5), ENSURE_REQUIREMENTS, [id=#95] +Input [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] +Arguments: hashpartitioning(channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, 5), ENSURE_REQUIREMENTS, [id=#131] -(133) HashAggregate [codegen id : 137] -Input [8]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88, sum#92, isEmpty#93, sum#94] -Keys [5]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, spark_grouping_id#88] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#96, sum(number_sales#47)#97] -Results [6]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, sum(sales#46)#96 AS sum(sales)#98, sum(number_sales#47)#97 AS sum(number_sales)#99] +(133) HashAggregate [codegen id : 131] +Input [8]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124, sum#128, isEmpty#129, sum#130] +Keys [5]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, spark_grouping_id#124] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#132, sum(number_sales#69)#133] +Results [6]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales#68)#132 AS sum(sales)#134, sum(number_sales#69)#133 AS sum(number_sales)#135] (134) TakeOrderedAndProject -Input [6]: [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, sum(sales)#98, sum(number_sales)#99] -Arguments: 100, [channel#84 ASC NULLS FIRST, i_brand_id#85 ASC NULLS FIRST, i_class_id#86 ASC NULLS FIRST, i_category_id#87 ASC NULLS FIRST], [channel#84, i_brand_id#85, i_class_id#86, i_category_id#87, sum(sales)#98, sum(number_sales)#99] +Input [6]: [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales)#134, sum(number_sales)#135] +Arguments: 100, [channel#120 ASC NULLS FIRST, i_brand_id#121 ASC NULLS FIRST, i_class_id#122 ASC NULLS FIRST, i_category_id#123 ASC NULLS FIRST], [channel#120, i_brand_id#121, i_class_id#122, i_category_id#123, sum(sales)#134, sum(number_sales)#135] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#49, [id=#50] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#71, [id=#72] * HashAggregate (157) +- Exchange (156) +- * HashAggregate (155) @@ -773,145 +773,145 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (135) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#100)] +PartitionFilters: [isnotnull(ss_sold_date_sk#138), dynamicpruningexpression(ss_sold_date_sk#138 IN dynamicpruning#139)] ReadSchema: struct (136) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138] (137) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#140, d_year#141] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (138) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#140, d_year#141] (139) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#140, d_year#141] +Condition : (((isnotnull(d_year#141) AND (d_year#141 >= 1999)) AND (d_year#141 <= 2001)) AND isnotnull(d_date_sk#140)) (140) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#140] +Input [2]: [d_date_sk#140, d_year#141] (141) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#101] +Input [1]: [d_date_sk#140] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#142] (142) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#138] +Right keys [1]: [d_date_sk#140] Join condition: None (143) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#102, ss_list_price#3 AS list_price#103] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#136 AS quantity#143, ss_list_price#137 AS list_price#144] +Input [4]: [ss_quantity#136, ss_list_price#137, ss_sold_date_sk#138, d_date_sk#140] (144) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Output [3]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#100)] +PartitionFilters: [isnotnull(cs_sold_date_sk#147), dynamicpruningexpression(cs_sold_date_sk#147 IN dynamicpruning#139)] ReadSchema: struct (145) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Input [3]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147] (146) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#148] (147) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#147] +Right keys [1]: [d_date_sk#148] Join condition: None (148) Project [codegen id : 4] -Output [2]: [cs_quantity#52 AS quantity#104, cs_list_price#53 AS list_price#105] -Input [4]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] +Output [2]: [cs_quantity#145 AS quantity#149, cs_list_price#146 AS list_price#150] +Input [4]: [cs_quantity#145, cs_list_price#146, cs_sold_date_sk#147, d_date_sk#148] (149) Scan parquet default.web_sales -Output [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Output [3]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#100)] +PartitionFilters: [isnotnull(ws_sold_date_sk#153), dynamicpruningexpression(ws_sold_date_sk#153 IN dynamicpruning#139)] ReadSchema: struct (150) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Input [3]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153] (151) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#154] (152) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#153] +Right keys [1]: [d_date_sk#154] Join condition: None (153) Project [codegen id : 6] -Output [2]: [ws_quantity#68 AS quantity#106, ws_list_price#69 AS list_price#107] -Input [4]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] +Output [2]: [ws_quantity#151 AS quantity#155, ws_list_price#152 AS list_price#156] +Input [4]: [ws_quantity#151, ws_list_price#152, ws_sold_date_sk#153, d_date_sk#154] (154) Union (155) HashAggregate [codegen id : 7] -Input [2]: [quantity#102, list_price#103] +Input [2]: [quantity#143, list_price#144] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#108, count#109] -Results [2]: [sum#110, count#111] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#157, count#158] +Results [2]: [sum#159, count#160] (156) Exchange -Input [2]: [sum#110, count#111] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#112] +Input [2]: [sum#159, count#160] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#161] (157) HashAggregate [codegen id : 8] -Input [2]: [sum#110, count#111] +Input [2]: [sum#159, count#160] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))#113] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#102 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#103 as decimal(12,2)))), DecimalType(18,2), true))#113 AS average_sales#114] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))#162] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#143 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#144 as decimal(12,2)))), DecimalType(18,2), true))#162 AS average_sales#163] -Subquery:2 Hosting operator id = 135 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#100 +Subquery:2 Hosting operator id = 135 Hosting Expression = ss_sold_date_sk#138 IN dynamicpruning#139 ReusedExchange (158) (158) ReusedExchange [Reuses operator id: 141] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#140] -Subquery:3 Hosting operator id = 144 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#100 +Subquery:3 Hosting operator id = 144 Hosting Expression = cs_sold_date_sk#147 IN dynamicpruning#139 -Subquery:4 Hosting operator id = 149 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#100 +Subquery:4 Hosting operator id = 149 Hosting Expression = ws_sold_date_sk#153 IN dynamicpruning#139 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (159) (159) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#49] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 ReusedExchange (160) (160) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#14] -Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 -Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#77 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt index 4ee1ff31e9beb..2134091c43b82 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt @@ -1,14 +1,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] - WholeStageCodegen (137) + WholeStageCodegen (131) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 - WholeStageCodegen (136) + WholeStageCodegen (130) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] Expand [sales,number_sales,channel,i_brand_id,i_class_id,i_category_id] InputAdapter Union - WholeStageCodegen (45) + WholeStageCodegen (43) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #3 @@ -57,7 +57,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (44) + WholeStageCodegen (42) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -76,11 +76,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #4 - WholeStageCodegen (21) + WholeStageCodegen (20) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #5 - WholeStageCodegen (20) + WholeStageCodegen (19) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -89,84 +89,82 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #6 - WholeStageCodegen (19) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (18) + WholeStageCodegen (17) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (13) + WholeStageCodegen (12) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (11) + WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #10 InputAdapter - Exchange [brand_id,class_id,category_id] #10 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] + BroadcastExchange #10 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #11 + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #10 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (7) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #13 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #11 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (17) + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (16) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #15 - WholeStageCodegen (16) + WholeStageCodegen (15) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -177,12 +175,12 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #11 + ReusedExchange [d_date_sk] #10 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #14 InputAdapter BroadcastExchange #4 - WholeStageCodegen (22) + WholeStageCodegen (21) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow @@ -191,27 +189,27 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su InputAdapter BroadcastExchange #16 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #17 - WholeStageCodegen (23) + WholeStageCodegen (22) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 - WholeStageCodegen (90) + WholeStageCodegen (86) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (89) + WholeStageCodegen (85) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -219,17 +217,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [cs_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [cs_item_sk,ss_item_sk] - WholeStageCodegen (47) + WholeStageCodegen (45) Sort [cs_item_sk] InputAdapter Exchange [cs_item_sk] #22 - WholeStageCodegen (46) + WholeStageCodegen (44) Filter [cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (66) + WholeStageCodegen (63) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 @@ -237,14 +235,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su ReusedExchange [d_date_sk] #4 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (135) + WholeStageCodegen (129) Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #23 - WholeStageCodegen (134) + WholeStageCodegen (128) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] @@ -252,17 +250,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ws_item_sk,ss_item_sk] - WholeStageCodegen (92) + WholeStageCodegen (88) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #24 - WholeStageCodegen (91) + WholeStageCodegen (87) Filter [ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (111) + WholeStageCodegen (106) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #18 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index 85a3a63413f92..6f61fc8e96ae1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -31,36 +31,36 @@ TakeOrderedAndProject (115) : : : :- * HashAggregate (39) : : : : +- Exchange (38) : : : : +- * HashAggregate (37) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) + : : : : +- * Project (36) + : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : :- * Project (33) + : : : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet default.store_sales (7) + : : : : : +- BroadcastExchange (31) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Project (28) + : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet default.item (16) + : : : : : +- BroadcastExchange (26) + : : : : : +- * Project (25) + : : : : : +- * Filter (24) + : : : : : +- * ColumnarToRow (23) + : : : : : +- Scan parquet default.date_dim (22) + : : : : +- ReusedExchange (34) : : : +- BroadcastExchange (49) : : : +- * Project (48) : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -146,512 +146,512 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_item_sk#10) (10) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(11) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(12) Filter [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +(12) Filter [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(13) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +(13) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join condition: None +(14) ColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(15) Filter [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Condition : isnotnull(cs_item_sk#17) -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +(16) Scan parquet default.item +Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] 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 - -(17) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(18) Filter [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +(17) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(19) Project [codegen id : 2] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +(18) Filter [codegen id : 1] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Condition : isnotnull(i_item_sk#19) -(20) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +(19) BroadcastExchange +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#19] Join condition: None -(22) Project [codegen id : 6] -Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] -Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(23) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(22) Scan parquet default.date_dim +Output [2]: [d_date_sk#24, d_year#25] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +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 -(24) ColumnarToRow [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(23) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#24, d_year#25] -(25) Filter [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +(24) Filter [codegen id : 2] +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) -(26) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +(25) Project [codegen id : 2] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_year#25] -(27) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(26) BroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(28) Filter [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +(27) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] +Join condition: None + +(28) Project [codegen id : 3] +Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] (29) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] -(30) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +(30) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join condition: None -(31) Project [codegen id : 5] -Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(32) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(31) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +(32) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] Join condition: None -(34) Project [codegen id : 5] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(33) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(35) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] +(34) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#29] -(36) BroadcastHashJoin [codegen id : 6] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#29] Join condition: None +(36) Project [codegen id : 6] +Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] + (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (38) Exchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_item_sk#34) -(43) ReusedExchange [Reuses operator id: 29] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(43) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#36] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] +Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -(46) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(46) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#40] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#40] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] (49) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (53) BroadcastExchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#15, class_id#16, category_id#17] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#27] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] +Output [1]: [i_item_sk#6 AS ss_item_sk#43] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] (56) BroadcastExchange -Input [1]: [ss_item_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [ss_item_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#43] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Condition : isnotnull(i_item_sk#45) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [i_item_sk#45] +Right keys [1]: [ss_item_sk#43] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#45] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (66) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#30] +Output [3]: [d_date_sk#50, d_year#51, d_moy#52] 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,11), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] (68) Filter [codegen id : 24] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#30)) AND (d_year#13 = 2001)) AND (d_moy#30 = 11)) AND isnotnull(d_date_sk#12)) +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] +Condition : ((((isnotnull(d_year#51) AND isnotnull(d_moy#52)) AND (d_year#51 = 2001)) AND (d_moy#52 = 11)) AND isnotnull(d_date_sk#50)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] +Output [1]: [d_date_sk#50] +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] (70) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [1]: [d_date_sk#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#50] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#32, isEmpty#33, count#34] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] +Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] +Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#60] (75) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] (76) Filter [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [sales#41, number_sales#42, store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] +Output [6]: [sales#63, number_sales#64, store AS channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] (78) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Output [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] (80) Filter [codegen id : 51] -Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +Condition : isnotnull(cs_item_sk#69) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [cs_item_sk#69] +Right keys [1]: [ss_item_sk#43] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] (84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [cs_item_sk#69] +Right keys [1]: [i_item_sk#73] Join condition: None (85) Project [codegen id : 51] -Output [6]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [8]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] (86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#77] (87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#72] +Right keys [1]: [d_date_sk#77] Join condition: None (88) Project [codegen id : 51] -Output [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [7]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76, d_date_sk#77] (89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#49, isEmpty#50, count#51] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] +Input [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] +Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#78, isEmpty#79, count#80] +Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] (90) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Arguments: hashpartitioning(i_brand_id#74, i_class_id#75, i_category_id#76, 5), ENSURE_REQUIREMENTS, [id=#84] (91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56, count(1)#57] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sales#58, count(1)#57 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85, count(1)#86] +Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sales#87, count(1)#86 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] (92) Filter [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [sales#58, number_sales#59, catalog AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] +Output [6]: [sales#87, number_sales#88, catalog AS channel#90, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] (94) Scan parquet default.web_sales -Output [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Output [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] (96) Filter [codegen id : 77] -Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Condition : isnotnull(ws_item_sk#91) (97) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [ws_item_sk#91] +Right keys [1]: [ss_item_sk#43] Join condition: None (99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] (100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#91] +Right keys [1]: [i_item_sk#95] Join condition: None (101) Project [codegen id : 77] -Output [6]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [8]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] (102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#99] (103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#94] +Right keys [1]: [d_date_sk#99] Join condition: None (104) Project [codegen id : 77] -Output [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [7]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98, d_date_sk#99] (105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] +Input [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] +Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#100, isEmpty#101, count#102] +Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] (106) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Arguments: hashpartitioning(i_brand_id#96, i_class_id#97, i_category_id#98, 5), ENSURE_REQUIREMENTS, [id=#106] (107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71, count(1)#72] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sales#73, count(1)#72 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] +Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#109, count(1)#108 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] (108) Filter [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [sales#73, number_sales#74, web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] +Output [6]: [sales#109, number_sales#110, web AS channel#112, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] (110) Union (111) Expand [codegen id : 79] -Input [6]: [sales#41, number_sales#42, channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: [List(sales#41, number_sales#42, channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 0), List(sales#41, number_sales#42, channel#46, i_brand_id#7, i_class_id#8, null, 1), List(sales#41, number_sales#42, channel#46, i_brand_id#7, null, null, 3), List(sales#41, number_sales#42, channel#46, null, null, null, 7), List(sales#41, number_sales#42, null, null, null, null, 15)], [sales#41, number_sales#42, channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] +Input [6]: [sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] +Arguments: [List(sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, 0), List(sales#63, number_sales#64, channel#68, i_brand_id#46, i_class_id#47, null, 1), List(sales#63, number_sales#64, channel#68, i_brand_id#46, null, null, 3), List(sales#63, number_sales#64, channel#68, null, null, null, 7), List(sales#63, number_sales#64, null, null, null, null, 15)], [sales#63, number_sales#64, channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] (112) HashAggregate [codegen id : 79] -Input [7]: [sales#41, number_sales#42, channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] -Keys [5]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#82, isEmpty#83, sum#84] -Results [8]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, sum#85, isEmpty#86, sum#87] +Input [7]: [sales#63, number_sales#64, channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] +Keys [5]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#118, isEmpty#119, sum#120] +Results [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] (113) Exchange -Input [8]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, sum#85, isEmpty#86, sum#87] -Arguments: hashpartitioning(channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] +Arguments: hashpartitioning(channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, 5), ENSURE_REQUIREMENTS, [id=#124] (114) HashAggregate [codegen id : 80] -Input [8]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81, sum#85, isEmpty#86, sum#87] -Keys [5]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, spark_grouping_id#81] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#89, sum(number_sales#42)#90] -Results [6]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, sum(sales#41)#89 AS sum(sales)#91, sum(number_sales#42)#90 AS sum(number_sales)#92] +Input [8]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117, sum#121, isEmpty#122, sum#123] +Keys [5]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, spark_grouping_id#117] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#125, sum(number_sales#64)#126] +Results [6]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales#63)#125 AS sum(sales)#127, sum(number_sales#64)#126 AS sum(number_sales)#128] (115) TakeOrderedAndProject -Input [6]: [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, sum(sales)#91, sum(number_sales)#92] -Arguments: 100, [channel#77 ASC NULLS FIRST, i_brand_id#78 ASC NULLS FIRST, i_class_id#79 ASC NULLS FIRST, i_category_id#80 ASC NULLS FIRST], [channel#77, i_brand_id#78, i_class_id#79, i_category_id#80, sum(sales)#91, sum(number_sales)#92] +Input [6]: [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales)#127, sum(number_sales)#128] +Arguments: 100, [channel#113 ASC NULLS FIRST, i_brand_id#114 ASC NULLS FIRST, i_class_id#115 ASC NULLS FIRST, i_category_id#116 ASC NULLS FIRST], [channel#113, i_brand_id#114, i_class_id#115, i_category_id#116, sum(sales)#127, sum(number_sales)#128] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#66, [id=#67] * HashAggregate (138) +- Exchange (137) +- * HashAggregate (136) @@ -678,145 +678,145 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (116) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#93)] +PartitionFilters: [isnotnull(ss_sold_date_sk#131), dynamicpruningexpression(ss_sold_date_sk#131 IN dynamicpruning#132)] ReadSchema: struct (117) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131] (118) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#133, d_year#134] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (119) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#133, d_year#134] (120) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#133, d_year#134] +Condition : (((isnotnull(d_year#134) AND (d_year#134 >= 1999)) AND (d_year#134 <= 2001)) AND isnotnull(d_date_sk#133)) (121) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#133] +Input [2]: [d_date_sk#133, d_year#134] (122) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#94] +Input [1]: [d_date_sk#133] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#135] (123) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#131] +Right keys [1]: [d_date_sk#133] Join condition: None (124) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#95, ss_list_price#3 AS list_price#96] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#129 AS quantity#136, ss_list_price#130 AS list_price#137] +Input [4]: [ss_quantity#129, ss_list_price#130, ss_sold_date_sk#131, d_date_sk#133] (125) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Output [3]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#93)] +PartitionFilters: [isnotnull(cs_sold_date_sk#140), dynamicpruningexpression(cs_sold_date_sk#140 IN dynamicpruning#132)] ReadSchema: struct (126) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Input [3]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140] (127) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#141] (128) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#140] +Right keys [1]: [d_date_sk#141] Join condition: None (129) Project [codegen id : 4] -Output [2]: [cs_quantity#47 AS quantity#97, cs_list_price#48 AS list_price#98] -Input [4]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, d_date_sk#12] +Output [2]: [cs_quantity#138 AS quantity#142, cs_list_price#139 AS list_price#143] +Input [4]: [cs_quantity#138, cs_list_price#139, cs_sold_date_sk#140, d_date_sk#141] (130) Scan parquet default.web_sales -Output [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Output [3]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#93)] +PartitionFilters: [isnotnull(ws_sold_date_sk#146), dynamicpruningexpression(ws_sold_date_sk#146 IN dynamicpruning#132)] ReadSchema: struct (131) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Input [3]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146] (132) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#147] (133) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#146] +Right keys [1]: [d_date_sk#147] Join condition: None (134) Project [codegen id : 6] -Output [2]: [ws_quantity#62 AS quantity#99, ws_list_price#63 AS list_price#100] -Input [4]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, d_date_sk#12] +Output [2]: [ws_quantity#144 AS quantity#148, ws_list_price#145 AS list_price#149] +Input [4]: [ws_quantity#144, ws_list_price#145, ws_sold_date_sk#146, d_date_sk#147] (135) Union (136) HashAggregate [codegen id : 7] -Input [2]: [quantity#95, list_price#96] +Input [2]: [quantity#136, list_price#137] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#101, count#102] -Results [2]: [sum#103, count#104] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#150, count#151] +Results [2]: [sum#152, count#153] (137) Exchange -Input [2]: [sum#103, count#104] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] +Input [2]: [sum#152, count#153] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#154] (138) HashAggregate [codegen id : 8] -Input [2]: [sum#103, count#104] +Input [2]: [sum#152, count#153] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))#106] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#95 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#96 as decimal(12,2)))), DecimalType(18,2), true))#106 AS average_sales#107] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#155] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#136 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#137 as decimal(12,2)))), DecimalType(18,2), true))#155 AS average_sales#156] -Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#93 +Subquery:2 Hosting operator id = 116 Hosting Expression = ss_sold_date_sk#131 IN dynamicpruning#132 ReusedExchange (139) (139) ReusedExchange [Reuses operator id: 122] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#133] -Subquery:3 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#93 +Subquery:3 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#140 IN dynamicpruning#132 -Subquery:4 Hosting operator id = 130 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#93 +Subquery:4 Hosting operator id = 130 Hosting Expression = ws_sold_date_sk#146 IN dynamicpruning#132 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#50] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 ReusedExchange (141) -(141) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(141) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#29] -Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt index f9b76560e363c..9ae1a0e30e90e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt @@ -90,53 +90,53 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su Exchange [brand_id,class_id,category_id] #6 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #7 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (1) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #10 + BroadcastExchange #9 WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (1) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #7 + BroadcastExchange #7 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #7 InputAdapter BroadcastExchange #11 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt index 76f5ac48f96cc..fab231dc80a6c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt @@ -33,39 +33,39 @@ TakeOrderedAndProject (116) : : : : +- * HashAggregate (44) : : : : +- Exchange (43) : : : : +- * HashAggregate (42) - : : : : +- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) + : : : : +- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Project (18) + : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : :- * Filter (11) + : : : : : : +- * ColumnarToRow (10) + : : : : : : +- Scan parquet default.store_sales (9) + : : : : : +- BroadcastExchange (16) + : : : : : +- * Project (15) + : : : : : +- * Filter (14) + : : : : : +- * ColumnarToRow (13) + : : : : : +- Scan parquet default.date_dim (12) + : : : : +- BroadcastExchange (39) + : : : : +- SortMergeJoin LeftSemi (38) + : : : : :- * Sort (23) + : : : : : +- Exchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (37) + : : : : +- Exchange (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- Scan parquet default.catalog_sales (24) + : : : : : +- ReusedExchange (27) + : : : : +- BroadcastExchange (33) + : : : : +- * Filter (32) + : : : : +- * ColumnarToRow (31) + : : : : +- Scan parquet default.item (30) : : : +- * Sort (57) : : : +- Exchange (56) : : : +- * Project (55) @@ -147,507 +147,507 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 20] +(7) ColumnarToRow [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 20] +(8) Filter [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 5] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +(10) ColumnarToRow [codegen id : 10] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -(11) Filter [codegen id : 5] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +(11) Filter [codegen id : 10] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_item_sk#11) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#14, d_year#15] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#14, d_year#15] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1999)) AND (d_year#15 <= 2001)) AND isnotnull(d_date_sk#14)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#14] +Input [2]: [d_date_sk#14, d_year#15] (16) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(17) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(17) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#14] Join condition: None -(18) Project [codegen id : 5] -Output [1]: [ss_item_sk#1] -Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] +(18) Project [codegen id : 10] +Output [1]: [ss_item_sk#11] +Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] (19) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) - -(22) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] - -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join condition: None - -(24) Project [codegen id : 5] -Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] -Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) -(25) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] +(22) Exchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] -(26) Sort [codegen id : 6] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 +(23) Sort [codegen id : 5] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 -(27) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] +(24) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +(25) ColumnarToRow [codegen id : 8] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] -(29) Filter [codegen id : 9] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +(26) Filter [codegen id : 8] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#22) -(30) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +(27) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#24] -(31) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +(28) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None -(32) Project [codegen id : 9] -Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] +(29) Project [codegen id : 8] +Output [1]: [cs_item_sk#22] +Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] -(33) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(30) Scan parquet default.item +Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(31) ColumnarToRow [codegen id : 7] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -(35) Filter [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +(32) Filter [codegen id : 7] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Condition : isnotnull(i_item_sk#25) -(36) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +(33) BroadcastExchange +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_item_sk#22] +Right keys [1]: [i_item_sk#25] Join condition: None -(38) Project [codegen id : 9] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(35) Project [codegen id : 8] +Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] + +(36) Exchange +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] -(39) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] +(37) Sort [codegen id : 9] +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 -(40) Sort [codegen id : 10] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(38) SortMergeJoin +Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] +Join condition: None + +(39) BroadcastExchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] -(41) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +(40) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_item_sk#11] +Right keys [1]: [i_item_sk#17] Join condition: None -(42) HashAggregate [codegen id : 11] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(41) Project [codegen id : 10] +Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] +Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] + +(42) HashAggregate [codegen id : 10] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (43) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(44) HashAggregate [codegen id : 12] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(44) HashAggregate [codegen id : 11] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (45) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] -(46) Sort [codegen id : 13] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 12] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +(48) ColumnarToRow [codegen id : 15] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -(49) Filter [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(49) Filter [codegen id : 15] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_item_sk#37) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#39] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(51) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#39] Join condition: None -(52) Project [codegen id : 16] -Output [1]: [ws_item_sk#26] -Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] +(52) Project [codegen id : 15] +Output [1]: [ws_item_sk#37] +Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] -(53) ReusedExchange [Reuses operator id: 36] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(53) ReusedExchange [Reuses operator id: 33] +Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(54) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#40] Join condition: None -(55) Project [codegen id : 16] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(55) Project [codegen id : 15] +Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] (56) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] -(57) Sort [codegen id : 17] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 16] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] +Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] Join condition: None -(59) HashAggregate [codegen id : 18] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(59) HashAggregate [codegen id : 17] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (60) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] -(61) HashAggregate [codegen id : 19] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(61) HashAggregate [codegen id : 18] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (62) BroadcastExchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] -(63) BroadcastHashJoin [codegen id : 20] +(63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#16, class_id#17, category_id#18] +Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join condition: None -(64) Project [codegen id : 20] -Output [1]: [i_item_sk#7 AS ss_item_sk#31] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] +(64) Project [codegen id : 19] +Output [1]: [i_item_sk#7 AS ss_item_sk#47] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] (65) Exchange -Input [1]: [ss_item_sk#31] -Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [1]: [ss_item_sk#47] +Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] -(66) Sort [codegen id : 21] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 20] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#47] Join condition: None (68) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#33] +Output [2]: [d_date_sk#49, d_week_seq#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 22] -Input [2]: [d_date_sk#12, d_week_seq#33] +(69) ColumnarToRow [codegen id : 21] +Input [2]: [d_date_sk#49, d_week_seq#50] -(70) Filter [codegen id : 22] -Input [2]: [d_date_sk#12, d_week_seq#33] -Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#34, [id=#35])) AND isnotnull(d_date_sk#12)) +(70) Filter [codegen id : 21] +Input [2]: [d_date_sk#49, d_week_seq#50] +Condition : ((isnotnull(d_week_seq#50) AND (d_week_seq#50 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#49)) -(71) Project [codegen id : 22] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#33] +(71) Project [codegen id : 21] +Output [1]: [d_date_sk#49] +Input [2]: [d_date_sk#49, d_week_seq#50] (72) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [1]: [d_date_sk#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] -(73) BroadcastHashJoin [codegen id : 44] +(73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#49] Join condition: None -(74) Project [codegen id : 44] +(74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] (75) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(76) ColumnarToRow [codegen id : 22] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -(77) Filter [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +(77) Filter [codegen id : 22] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Condition : (((isnotnull(i_item_sk#54) AND isnotnull(i_brand_id#55)) AND isnotnull(i_class_id#56)) AND isnotnull(i_category_id#57)) (78) Exchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: hashpartitioning(i_item_sk#54, 5), ENSURE_REQUIREMENTS, [id=#58] -(79) Sort [codegen id : 24] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 +(79) Sort [codegen id : 23] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: [i_item_sk#54 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(81) Sort [codegen id : 43] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 41] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [i_item_sk#54] +Right keys [1]: [ss_item_sk#47] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#59] -(84) BroadcastHashJoin [codegen id : 44] +(84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#54] Join condition: None -(85) Project [codegen id : 44] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(85) Project [codegen id : 42] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -(86) HashAggregate [codegen id : 44] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +(86) HashAggregate [codegen id : 42] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] +Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#39, isEmpty#40, count#41] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] +Aggregate Attributes [3]: [sum#60, isEmpty#61, count#62] +Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] +Arguments: hashpartitioning(i_brand_id#55, i_class_id#56, i_category_id#57, 5), ENSURE_REQUIREMENTS, [id=#66] -(88) HashAggregate [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +(88) HashAggregate [codegen id : 86] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] +Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46, count(1)#47] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sales#48, count(1)#47 AS number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67, count(1)#68] +Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#69, count(1)#68 AS number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] -(89) Filter [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) +(89) Filter [codegen id : 86] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(32,6)) > cast(Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) -(90) Project [codegen id : 90] -Output [6]: [store AS channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] +(90) Project [codegen id : 86] +Output [6]: [store AS channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] (91) Scan parquet default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#54)] +PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 45] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +(92) ColumnarToRow [codegen id : 43] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -(93) Filter [codegen id : 45] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +(93) Filter [codegen id : 43] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Condition : isnotnull(ss_item_sk#75) (94) Exchange -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Arguments: hashpartitioning(ss_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#80] -(95) Sort [codegen id : 46] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 44] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Arguments: [ss_item_sk#75 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(97) Sort [codegen id : 65] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 62] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [ss_item_sk#75] +Right keys [1]: [ss_item_sk#47] Join condition: None (99) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#33] +Output [2]: [d_date_sk#81, d_week_seq#82] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 66] -Input [2]: [d_date_sk#12, d_week_seq#33] +(100) ColumnarToRow [codegen id : 63] +Input [2]: [d_date_sk#81, d_week_seq#82] -(101) Filter [codegen id : 66] -Input [2]: [d_date_sk#12, d_week_seq#33] -Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#56, [id=#57])) AND isnotnull(d_date_sk#12)) +(101) Filter [codegen id : 63] +Input [2]: [d_date_sk#81, d_week_seq#82] +Condition : ((isnotnull(d_week_seq#82) AND (d_week_seq#82 = Subquery scalar-subquery#83, [id=#84])) AND isnotnull(d_date_sk#81)) -(102) Project [codegen id : 66] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#33] +(102) Project [codegen id : 63] +Output [1]: [d_date_sk#81] +Input [2]: [d_date_sk#81, d_week_seq#82] (103) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] +Input [1]: [d_date_sk#81] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#85] -(104) BroadcastHashJoin [codegen id : 88] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(104) BroadcastHashJoin [codegen id : 84] +Left keys [1]: [ss_sold_date_sk#78] +Right keys [1]: [d_date_sk#81] Join condition: None -(105) Project [codegen id : 88] -Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +(105) Project [codegen id : 84] +Output [3]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77] +Input [5]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#81] (106) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +Output [4]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] -(107) BroadcastHashJoin [codegen id : 88] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#59] +(107) BroadcastHashJoin [codegen id : 84] +Left keys [1]: [ss_item_sk#75] +Right keys [1]: [i_item_sk#86] Join condition: None -(108) Project [codegen id : 88] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +(108) Project [codegen id : 84] +Output [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] +Input [7]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] -(109) HashAggregate [codegen id : 88] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] +(109) HashAggregate [codegen id : 84] +Input [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] +Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] +Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] (110) Exchange -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] -Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] +Arguments: hashpartitioning(i_brand_id#87, i_class_id#88, i_category_id#89, 5), ENSURE_REQUIREMENTS, [id=#96] -(111) HashAggregate [codegen id : 89] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70, count(1)#71] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sales#72, count(1)#71 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] +(111) HashAggregate [codegen id : 85] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] +Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] +Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#99, count(1)#98 AS number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] -(112) Filter [codegen id : 89] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) +(112) Filter [codegen id : 85] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) -(113) Project [codegen id : 89] -Output [6]: [store AS channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] +(113) Project [codegen id : 85] +Output [6]: [store AS channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] (114) BroadcastExchange -Input [6]: [channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#76] +Input [6]: [channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#103] -(115) BroadcastHashJoin [codegen id : 90] -Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +(115) BroadcastHashJoin [codegen id : 86] +Left keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] +Right keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] Join condition: None (116) TakeOrderedAndProject -Input [12]: [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Input [12]: [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Arguments: 100, [i_brand_id#55 ASC NULLS FIRST, i_class_id#56 ASC NULLS FIRST, i_category_id#57 ASC NULLS FIRST], [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#72, [id=#73] * HashAggregate (139) +- Exchange (138) +- * HashAggregate (137) @@ -674,140 +674,140 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (117) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ss_sold_date_sk#106), dynamicpruningexpression(ss_sold_date_sk#106 IN dynamicpruning#107)] ReadSchema: struct (118) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] (119) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#108, d_year#109] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (120) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#108, d_year#109] (121) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#108, d_year#109] +Condition : (((isnotnull(d_year#109) AND (d_year#109 >= 1999)) AND (d_year#109 <= 2001)) AND isnotnull(d_date_sk#108)) (122) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#108] +Input [2]: [d_date_sk#108, d_year#109] (123) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] +Input [1]: [d_date_sk#108] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#110] (124) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#106] +Right keys [1]: [d_date_sk#108] Join condition: None (125) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#79, ss_list_price#3 AS list_price#80] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#104 AS quantity#111, ss_list_price#105 AS list_price#112] +Input [4]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106, d_date_sk#108] (126) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] +Output [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(cs_sold_date_sk#115), dynamicpruningexpression(cs_sold_date_sk#115 IN dynamicpruning#107)] ReadSchema: struct (127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] +Input [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] (128) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#116] (129) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#115] +Right keys [1]: [d_date_sk#116] Join condition: None (130) Project [codegen id : 4] -Output [2]: [cs_quantity#81 AS quantity#83, cs_list_price#82 AS list_price#84] -Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21, d_date_sk#12] +Output [2]: [cs_quantity#113 AS quantity#117, cs_list_price#114 AS list_price#118] +Input [4]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115, d_date_sk#116] (131) Scan parquet default.web_sales -Output [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] +Output [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#107)] ReadSchema: struct (132) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] +Input [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] (133) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#122] (134) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#121] +Right keys [1]: [d_date_sk#122] Join condition: None (135) Project [codegen id : 6] -Output [2]: [ws_quantity#85 AS quantity#87, ws_list_price#86 AS list_price#88] -Input [4]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27, d_date_sk#12] +Output [2]: [ws_quantity#119 AS quantity#123, ws_list_price#120 AS list_price#124] +Input [4]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121, d_date_sk#122] (136) Union (137) HashAggregate [codegen id : 7] -Input [2]: [quantity#79, list_price#80] +Input [2]: [quantity#111, list_price#112] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#89, count#90] -Results [2]: [sum#91, count#92] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#125, count#126] +Results [2]: [sum#127, count#128] (138) Exchange -Input [2]: [sum#91, count#92] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] +Input [2]: [sum#127, count#128] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] (139) HashAggregate [codegen id : 8] -Input [2]: [sum#91, count#92] +Input [2]: [sum#127, count#128] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94 AS average_sales#95] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130 AS average_sales#131] -Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#77 +Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#106 IN dynamicpruning#107 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#108] -Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#77 +Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#115 IN dynamicpruning#107 -Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#77 +Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#107 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (141) (141) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#49] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 ReusedExchange (142) (142) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#14] -Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 -Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#34, [id=#35] +Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#51, [id=#52] * Project (146) +- * Filter (145) +- * ColumnarToRow (144) @@ -815,33 +815,33 @@ Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquer (143) Scan parquet default.date_dim -Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] (145) Filter [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 2000)) AND (d_moy#96 = 12)) AND (d_dom#97 = 11)) +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Condition : (((((isnotnull(d_year#133) AND isnotnull(d_moy#134)) AND isnotnull(d_dom#135)) AND (d_year#133 = 2000)) AND (d_moy#134 = 12)) AND (d_dom#135 = 11)) (146) Project [codegen id : 1] -Output [1]: [d_week_seq#33] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [1]: [d_week_seq#132] +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] -Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] +Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] -Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#54 +Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#79 ReusedExchange (147) (147) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#81] -Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#56, [id=#57] +Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#83, [id=#84] * Project (151) +- * Filter (150) +- * ColumnarToRow (149) @@ -849,21 +849,21 @@ Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subqu (148) Scan parquet default.date_dim -Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (149) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] (150) Filter [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 1999)) AND (d_moy#96 = 12)) AND (d_dom#97 = 11)) +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Condition : (((((isnotnull(d_year#137) AND isnotnull(d_moy#138)) AND isnotnull(d_dom#139)) AND (d_year#137 = 1999)) AND (d_moy#138 = 12)) AND (d_dom#139 = 11)) (151) Project [codegen id : 1] -Output [1]: [d_week_seq#33] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [1]: [d_week_seq#136] +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt index 1d5907e319658..1b41abbc97b06 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (90) + WholeStageCodegen (86) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] @@ -49,7 +49,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (44) + WholeStageCodegen (42) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -68,11 +68,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #3 - WholeStageCodegen (21) + WholeStageCodegen (20) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 - WholeStageCodegen (20) + WholeStageCodegen (19) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -81,84 +81,82 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (19) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (18) + WholeStageCodegen (17) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (13) + WholeStageCodegen (12) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (11) + WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #9 InputAdapter - Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] + BroadcastExchange #9 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #10 + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #9 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (7) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (17) + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (16) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (16) + WholeStageCodegen (15) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -169,12 +167,12 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #10 + ReusedExchange [d_date_sk] #9 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter BroadcastExchange #3 - WholeStageCodegen (22) + WholeStageCodegen (21) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #3 @@ -190,29 +188,29 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #15 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #16 - WholeStageCodegen (23) + WholeStageCodegen (22) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #20 - WholeStageCodegen (89) + WholeStageCodegen (85) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (88) + WholeStageCodegen (84) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -220,24 +218,24 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] - WholeStageCodegen (46) + WholeStageCodegen (44) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #22 - WholeStageCodegen (45) + WholeStageCodegen (43) Filter [ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #6 ReusedExchange [d_date_sk] #23 - WholeStageCodegen (65) + WholeStageCodegen (62) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #23 - WholeStageCodegen (66) + WholeStageCodegen (63) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 01bcb83622844..ae653b43971db 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -27,36 +27,36 @@ TakeOrderedAndProject (100) : : : :- * HashAggregate (39) : : : : +- Exchange (38) : : : : +- * HashAggregate (37) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) + : : : : +- * Project (36) + : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : :- * Project (33) + : : : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet default.store_sales (7) + : : : : : +- BroadcastExchange (31) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Project (28) + : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet default.item (16) + : : : : : +- BroadcastExchange (26) + : : : : : +- * Project (25) + : : : : : +- * Filter (24) + : : : : : +- * ColumnarToRow (23) + : : : : : +- Scan parquet default.date_dim (22) + : : : : +- ReusedExchange (34) : : : +- BroadcastExchange (49) : : : +- * Project (48) : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -131,443 +131,443 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_item_sk#10) (10) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(11) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(12) Filter [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +(12) Filter [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(13) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +(13) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join condition: None +(14) ColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(15) Filter [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Condition : isnotnull(cs_item_sk#17) -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +(16) Scan parquet default.item +Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] 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 - -(17) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(18) Filter [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +(17) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(19) Project [codegen id : 2] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +(18) Filter [codegen id : 1] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Condition : isnotnull(i_item_sk#19) -(20) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +(19) BroadcastExchange +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#19] Join condition: None -(22) Project [codegen id : 6] -Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] -Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(23) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(22) Scan parquet default.date_dim +Output [2]: [d_date_sk#24, d_year#25] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +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 -(24) ColumnarToRow [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(23) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#24, d_year#25] -(25) Filter [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +(24) Filter [codegen id : 2] +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) -(26) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +(25) Project [codegen id : 2] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_year#25] -(27) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(26) BroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(28) Filter [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +(27) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] +Join condition: None + +(28) Project [codegen id : 3] +Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] (29) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] -(30) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +(30) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join condition: None -(31) Project [codegen id : 5] -Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(32) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(31) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +(32) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] Join condition: None -(34) Project [codegen id : 5] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(33) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(35) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] +(34) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#29] -(36) BroadcastHashJoin [codegen id : 6] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#29] Join condition: None +(36) Project [codegen id : 6] +Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] + (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (38) Exchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_item_sk#34) -(43) ReusedExchange [Reuses operator id: 29] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(43) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#36] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] +Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -(46) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(46) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#40] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#40] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] (49) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (53) BroadcastExchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#15, class_id#16, category_id#17] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#27] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] +Output [1]: [i_item_sk#6 AS ss_item_sk#43] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] (56) BroadcastExchange -Input [1]: [ss_item_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [ss_item_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#43] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Condition : (((isnotnull(i_item_sk#45) AND isnotnull(i_brand_id#46)) AND isnotnull(i_class_id#47)) AND isnotnull(i_category_id#48)) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [i_item_sk#45] +Right keys [1]: [ss_item_sk#43] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#45] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (66) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#30] +Output [2]: [d_date_sk#50, d_week_seq#51] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [2]: [d_date_sk#12, d_week_seq#30] +Input [2]: [d_date_sk#50, d_week_seq#51] (68) Filter [codegen id : 24] -Input [2]: [d_date_sk#12, d_week_seq#30] -Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#31, [id=#32])) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#50, d_week_seq#51] +Condition : ((isnotnull(d_week_seq#51) AND (d_week_seq#51 = Subquery scalar-subquery#52, [id=#53])) AND isnotnull(d_date_sk#50)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#30] +Output [1]: [d_date_sk#50] +Input [2]: [d_date_sk#50, d_week_seq#51] (70) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] +Input [1]: [d_date_sk#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#50] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#34, isEmpty#35, count#36] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] +Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] +Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#61] (75) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41, count(1)#42] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sales#43, count(1)#42 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] (76) Filter [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45 as decimal(32,6)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) (77) Project [codegen id : 52] -Output [6]: [store AS channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] +Output [6]: [store AS channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] (78) Scan parquet default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#49)] +PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] (80) Filter [codegen id : 50] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +Condition : isnotnull(ss_item_sk#70) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [ss_item_sk#70] +Right keys [1]: [ss_item_sk#43] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] +Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] (84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#50] +Left keys [1]: [ss_item_sk#70] +Right keys [1]: [i_item_sk#75] Join condition: None (85) Project [codegen id : 50] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] +Output [6]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] +Input [8]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#30] +Output [2]: [d_date_sk#79, d_week_seq#80] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#12, d_week_seq#30] +Input [2]: [d_date_sk#79, d_week_seq#80] (88) Filter [codegen id : 49] -Input [2]: [d_date_sk#12, d_week_seq#30] -Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#54, [id=#55])) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#79, d_week_seq#80] +Condition : ((isnotnull(d_week_seq#80) AND (d_week_seq#80 = Subquery scalar-subquery#81, [id=#82])) AND isnotnull(d_date_sk#79)) (89) Project [codegen id : 49] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#30] +Output [1]: [d_date_sk#79] +Input [2]: [d_date_sk#79, d_week_seq#80] (90) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +Input [1]: [d_date_sk#79] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#83] (91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#73] +Right keys [1]: [d_date_sk#79] Join condition: None (92) Project [codegen id : 50] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53, d_date_sk#12] +Output [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] +Input [7]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78, d_date_sk#79] (93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] -Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#57, isEmpty#58, count#59] -Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] +Input [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] +Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] +Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] (94) Exchange -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] -Arguments: hashpartitioning(i_brand_id#51, i_class_id#52, i_category_id#53, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] +Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, [id=#90] (95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] -Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] -Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] +Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] +Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] (96) Filter [codegen id : 51] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) (97) Project [codegen id : 51] -Output [6]: [store AS channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] +Output [6]: [store AS channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] (98) BroadcastExchange -Input [6]: [channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] +Input [6]: [channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#97] (99) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] +Left keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Right keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] Join condition: None (100) TakeOrderedAndProject -Input [12]: [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Input [12]: [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Arguments: 100, [i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#46, [id=#47] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#67, [id=#68] * HashAggregate (123) +- Exchange (122) +- * HashAggregate (121) @@ -594,140 +594,140 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(ss_sold_date_sk#100), dynamicpruningexpression(ss_sold_date_sk#100 IN dynamicpruning#101)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] (103) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#102, d_year#103] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#102, d_year#103] (105) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#102, d_year#103] +Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1999)) AND (d_year#103 <= 2001)) AND isnotnull(d_date_sk#102)) (106) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#102] +Input [2]: [d_date_sk#102, d_year#103] (107) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] +Input [1]: [d_date_sk#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#104] (108) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#100] +Right keys [1]: [d_date_sk#102] Join condition: None (109) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#73, ss_list_price#3 AS list_price#74] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#98 AS quantity#105, ss_list_price#99 AS list_price#106] +Input [4]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100, d_date_sk#102] (110) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] +Output [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(cs_sold_date_sk#109), dynamicpruningexpression(cs_sold_date_sk#109 IN dynamicpruning#101)] ReadSchema: struct (111) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] +Input [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] (112) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#110] (113) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#109] +Right keys [1]: [d_date_sk#110] Join condition: None (114) Project [codegen id : 4] -Output [2]: [cs_quantity#75 AS quantity#77, cs_list_price#76 AS list_price#78] -Input [4]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19, d_date_sk#12] +Output [2]: [cs_quantity#107 AS quantity#111, cs_list_price#108 AS list_price#112] +Input [4]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109, d_date_sk#110] (115) Scan parquet default.web_sales -Output [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] +Output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(ws_sold_date_sk#115), dynamicpruningexpression(ws_sold_date_sk#115 IN dynamicpruning#101)] ReadSchema: struct (116) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] +Input [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] (117) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#116] (118) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#115] +Right keys [1]: [d_date_sk#116] Join condition: None (119) Project [codegen id : 6] -Output [2]: [ws_quantity#79 AS quantity#81, ws_list_price#80 AS list_price#82] -Input [4]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24, d_date_sk#12] +Output [2]: [ws_quantity#113 AS quantity#117, ws_list_price#114 AS list_price#118] +Input [4]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115, d_date_sk#116] (120) Union (121) HashAggregate [codegen id : 7] -Input [2]: [quantity#73, list_price#74] +Input [2]: [quantity#105, list_price#106] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#83, count#84] -Results [2]: [sum#85, count#86] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#119, count#120] +Results [2]: [sum#121, count#122] (122) Exchange -Input [2]: [sum#85, count#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#87] +Input [2]: [sum#121, count#122] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#123] (123) HashAggregate [codegen id : 8] -Input [2]: [sum#85, count#86] +Input [2]: [sum#121, count#122] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88 AS average_sales#89] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124 AS average_sales#125] -Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#71 +Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#100 IN dynamicpruning#101 ReusedExchange (124) (124) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#102] -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#71 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#109 IN dynamicpruning#101 -Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#71 +Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#115 IN dynamicpruning#101 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#50] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 ReusedExchange (126) -(126) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(126) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#29] -Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#31, [id=#32] +Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] * Project (130) +- * Filter (129) +- * ColumnarToRow (128) @@ -735,33 +735,33 @@ Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer (127) Scan parquet default.date_dim -Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (128) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] (129) Filter [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 2000)) AND (d_moy#90 = 12)) AND (d_dom#91 = 11)) +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Condition : (((((isnotnull(d_year#127) AND isnotnull(d_moy#128)) AND isnotnull(d_dom#129)) AND (d_year#127 = 2000)) AND (d_moy#128 = 12)) AND (d_dom#129 = 11)) (130) Project [codegen id : 1] -Output [1]: [d_week_seq#30] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [1]: [d_week_seq#126] +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] -Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] -Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#49 +Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 ReusedExchange (131) (131) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#79] -Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#81, [id=#82] * Project (135) +- * Filter (134) +- * ColumnarToRow (133) @@ -769,21 +769,21 @@ Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subque (132) Scan parquet default.date_dim -Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] ReadSchema: struct (133) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] (134) Filter [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 1999)) AND (d_moy#90 = 12)) AND (d_dom#91 = 11)) +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Condition : (((((isnotnull(d_year#131) AND isnotnull(d_moy#132)) AND isnotnull(d_dom#133)) AND (d_year#131 = 1999)) AND (d_moy#132 = 12)) AND (d_dom#133 = 11)) (135) Project [codegen id : 1] -Output [1]: [d_week_seq#30] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [1]: [d_week_seq#130] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt index 3507d76bcf9b9..cb9b410ea8568 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt @@ -82,53 +82,53 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Exchange [brand_id,class_id,category_id] #5 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #9 + BroadcastExchange #8 WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 + BroadcastExchange #6 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt index b70c36db4bc9d..5d171e5f595b9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt @@ -81,7 +81,7 @@ Input [2]: [d_date_sk#8, d_date#9] (11) Filter [codegen id : 2] Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) (12) Project [codegen id : 2] Output [1]: [d_date_sk#8] @@ -116,63 +116,63 @@ Input [3]: [cs_item_sk#4, sum#13, count#14] Keys [1]: [cs_item_sk#4] Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#5))] Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#5))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4 AS cs_item_sk#4#18] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4] (19) Filter -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4#18] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4] Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) (20) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#4#18] +Right keys [1]: [cs_item_sk#4] Join condition: None (21) Project [codegen id : 4] Output [2]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17] -Input [3]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4#18] +Input [3]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4] (22) BroadcastExchange Input [2]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (23) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] +Output [3]: [cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#6), dynamicpruningexpression(cs_sold_date_sk#6 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#7)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] ReadSchema: struct (24) ColumnarToRow -Input [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] +Input [3]: [cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] (25) Filter -Input [3]: [cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] -Condition : (isnotnull(cs_item_sk#4) AND isnotnull(cs_ext_discount_amt#5)) +Input [3]: [cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] +Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_ext_discount_amt#20)) (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#1] -Right keys [1]: [cs_item_sk#4] -Join condition: (cast(cs_ext_discount_amt#5 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) +Right keys [1]: [cs_item_sk#19] +Join condition: (cast(cs_ext_discount_amt#20 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) (27) Project [codegen id : 6] -Output [1]: [cs_sold_date_sk#6] -Input [5]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#4, cs_ext_discount_amt#5, cs_sold_date_sk#6] +Output [1]: [cs_sold_date_sk#21] +Input [5]: [i_item_sk#1, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#19, cs_ext_discount_amt#20, cs_sold_date_sk#21] (28) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#21] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 6] -Output [1]: [1 AS excess discount amount #20] -Input [2]: [cs_sold_date_sk#6, d_date_sk#8] +Output [1]: [1 AS excess discount amount #23] +Input [2]: [cs_sold_date_sk#21, d_date_sk#22] (31) CollectLimit -Input [1]: [excess discount amount #20] +Input [1]: [excess discount amount #23] Arguments: 100 ===== Subqueries ===== @@ -184,6 +184,6 @@ ReusedExchange (32) (32) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#6 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt index d885ad3178181..303bdf58604bf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/simplified.txt @@ -18,7 +18,7 @@ CollectLimit InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] Filter [(1.3 * avg(cs_ext_discount_amt))] - HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,sum,count] + HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] InputAdapter Exchange [cs_item_sk] #3 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index 9537689459170..ad918310a918a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -79,100 +79,100 @@ Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] (11) Scan parquet default.catalog_sales -Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] +Output [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#4)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] +Input [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] (13) Filter [codegen id : 3] -Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] -Condition : isnotnull(cs_item_sk#1) +Input [3]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10] +Condition : isnotnull(cs_item_sk#8) (14) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#9] +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#8, d_date#9] +Input [2]: [d_date_sk#11, d_date#12] (16) Filter [codegen id : 2] -Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) (17) Project [codegen id : 2] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#9] +Output [1]: [d_date_sk#11] +Input [2]: [d_date_sk#11, d_date#12] (18) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] (19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#10] +Right keys [1]: [d_date_sk#11] Join condition: None (20) Project [codegen id : 3] -Output [2]: [cs_item_sk#1, cs_ext_discount_amt#2] -Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#8] +Output [2]: [cs_item_sk#8, cs_ext_discount_amt#9] +Input [4]: [cs_item_sk#8, cs_ext_discount_amt#9, cs_sold_date_sk#10, d_date_sk#11] (21) HashAggregate [codegen id : 3] -Input [2]: [cs_item_sk#1, cs_ext_discount_amt#2] -Keys [1]: [cs_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [2]: [sum#11, count#12] -Results [3]: [cs_item_sk#1, sum#13, count#14] +Input [2]: [cs_item_sk#8, cs_ext_discount_amt#9] +Keys [1]: [cs_item_sk#8] +Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#9))] +Aggregate Attributes [2]: [sum#14, count#15] +Results [3]: [cs_item_sk#8, sum#16, count#17] (22) Exchange -Input [3]: [cs_item_sk#1, sum#13, count#14] -Arguments: hashpartitioning(cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [3]: [cs_item_sk#8, sum#16, count#17] +Arguments: hashpartitioning(cs_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#18] (23) HashAggregate [codegen id : 4] -Input [3]: [cs_item_sk#1, sum#13, count#14] -Keys [1]: [cs_item_sk#1] -Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#2))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#2))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1 AS cs_item_sk#1#18] +Input [3]: [cs_item_sk#8, sum#16, count#17] +Keys [1]: [cs_item_sk#8] +Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#9))] +Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#9))#19] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#9))#19 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] (24) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] -Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] +Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#20) (25) BroadcastExchange -Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#19] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#21] (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] -Right keys [1]: [cs_item_sk#1#18] -Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) +Right keys [1]: [cs_item_sk#8] +Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#20) (27) Project [codegen id : 6] Output [1]: [cs_sold_date_sk#3] -Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#1#18] +Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#20, cs_item_sk#8] (28) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 6] -Output [1]: [1 AS excess discount amount #20] -Input [2]: [cs_sold_date_sk#3, d_date_sk#8] +Output [1]: [1 AS excess discount amount #23] +Input [2]: [cs_sold_date_sk#3, d_date_sk#22] (31) CollectLimit -Input [1]: [excess discount amount #20] +Input [1]: [excess discount amount #23] Arguments: 100 ===== Subqueries ===== @@ -182,8 +182,8 @@ ReusedExchange (32) (32) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] -Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt index 563281eef37bc..f3dd6d0954046 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/simplified.txt @@ -24,7 +24,7 @@ CollectLimit BroadcastExchange #3 WholeStageCodegen (4) Filter [(1.3 * avg(cs_ext_discount_amt))] - HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),cs_item_sk,sum,count] + HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] InputAdapter Exchange [cs_item_sk] #4 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt index 13d73e61e1443..33c03d2b767dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt @@ -40,61 +40,61 @@ Output [2]: [i_manufact#2, i_product_name#3] Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] (5) Scan parquet default.item -Output [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,powder),EqualTo(i_color,khaki)),Or(EqualTo(i_units,Ounce),EqualTo(i_units,Oz))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,brown),EqualTo(i_color,honeydew)),Or(EqualTo(i_units,Bunch),EqualTo(i_units,Ton))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,floral),EqualTo(i_color,deep)),Or(EqualTo(i_units,N/A),EqualTo(i_units,Dozen))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,light),EqualTo(i_color,cornflower)),Or(EqualTo(i_units,Box),EqualTo(i_units,Pound))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large)))))),Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,midnight),EqualTo(i_color,snow)),Or(EqualTo(i_units,Pallet),EqualTo(i_units,Gross))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,cyan),EqualTo(i_color,papaya)),Or(EqualTo(i_units,Cup),EqualTo(i_units,Dram))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,orange),EqualTo(i_color,frosted)),Or(EqualTo(i_units,Each),EqualTo(i_units,Tbl))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,forest),EqualTo(i_color,ghost)),Or(EqualTo(i_units,Lb),EqualTo(i_units,Bundle))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))))))), IsNotNull(i_manufact)] +PushedFilters: [Or(Or(And(EqualTo(i_category,Women ),Or(And(And(Or(EqualTo(i_color,powder ),EqualTo(i_color,khaki )),Or(EqualTo(i_units,Ounce ),EqualTo(i_units,Oz ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))),And(And(Or(EqualTo(i_color,brown ),EqualTo(i_color,honeydew )),Or(EqualTo(i_units,Bunch ),EqualTo(i_units,Ton ))),Or(EqualTo(i_size,N/A ),EqualTo(i_size,small ))))),And(EqualTo(i_category,Men ),Or(And(And(Or(EqualTo(i_color,floral ),EqualTo(i_color,deep )),Or(EqualTo(i_units,N/A ),EqualTo(i_units,Dozen ))),Or(EqualTo(i_size,petite ),EqualTo(i_size,large ))),And(And(Or(EqualTo(i_color,light ),EqualTo(i_color,cornflower )),Or(EqualTo(i_units,Box ),EqualTo(i_units,Pound ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large )))))),Or(And(EqualTo(i_category,Women ),Or(And(And(Or(EqualTo(i_color,midnight ),EqualTo(i_color,snow )),Or(EqualTo(i_units,Pallet ),EqualTo(i_units,Gross ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))),And(And(Or(EqualTo(i_color,cyan ),EqualTo(i_color,papaya )),Or(EqualTo(i_units,Cup ),EqualTo(i_units,Dram ))),Or(EqualTo(i_size,N/A ),EqualTo(i_size,small ))))),And(EqualTo(i_category,Men ),Or(And(And(Or(EqualTo(i_color,orange ),EqualTo(i_color,frosted )),Or(EqualTo(i_units,Each ),EqualTo(i_units,Tbl ))),Or(EqualTo(i_size,petite ),EqualTo(i_size,large ))),And(And(Or(EqualTo(i_color,forest ),EqualTo(i_color,ghost )),Or(EqualTo(i_units,Lb ),EqualTo(i_units,Bundle ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))))))), IsNotNull(i_manufact)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] (7) Filter [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Condition : (((((i_category#4 = Women) AND (((((i_color#6 = powder) OR (i_color#6 = khaki)) AND ((i_units#7 = Ounce) OR (i_units#7 = Oz))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = brown) OR (i_color#6 = honeydew)) AND ((i_units#7 = Bunch) OR (i_units#7 = Ton))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = floral) OR (i_color#6 = deep)) AND ((i_units#7 = N/A) OR (i_units#7 = Dozen))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = light) OR (i_color#6 = cornflower)) AND ((i_units#7 = Box) OR (i_units#7 = Pound))) AND ((i_size#5 = medium) OR (i_size#5 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#6 = midnight) OR (i_color#6 = snow)) AND ((i_units#7 = Pallet) OR (i_units#7 = Gross))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = cyan) OR (i_color#6 = papaya)) AND ((i_units#7 = Cup) OR (i_units#7 = Dram))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = orange) OR (i_color#6 = frosted)) AND ((i_units#7 = Each) OR (i_units#7 = Tbl))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = forest) OR (i_color#6 = ghost)) AND ((i_units#7 = Lb) OR (i_units#7 = Bundle))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))))))) AND isnotnull(i_manufact#2)) +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Condition : (((((i_category#4 = Women ) AND (((((i_color#7 = powder ) OR (i_color#7 = khaki )) AND ((i_units#8 = Ounce ) OR (i_units#8 = Oz ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))) OR ((((i_color#7 = brown ) OR (i_color#7 = honeydew )) AND ((i_units#8 = Bunch ) OR (i_units#8 = Ton ))) AND ((i_size#6 = N/A ) OR (i_size#6 = small ))))) OR ((i_category#4 = Men ) AND (((((i_color#7 = floral ) OR (i_color#7 = deep )) AND ((i_units#8 = N/A ) OR (i_units#8 = Dozen ))) AND ((i_size#6 = petite ) OR (i_size#6 = large ))) OR ((((i_color#7 = light ) OR (i_color#7 = cornflower )) AND ((i_units#8 = Box ) OR (i_units#8 = Pound ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large )))))) OR (((i_category#4 = Women ) AND (((((i_color#7 = midnight ) OR (i_color#7 = snow )) AND ((i_units#8 = Pallet ) OR (i_units#8 = Gross ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))) OR ((((i_color#7 = cyan ) OR (i_color#7 = papaya )) AND ((i_units#8 = Cup ) OR (i_units#8 = Dram ))) AND ((i_size#6 = N/A ) OR (i_size#6 = small ))))) OR ((i_category#4 = Men ) AND (((((i_color#7 = orange ) OR (i_color#7 = frosted )) AND ((i_units#8 = Each ) OR (i_units#8 = Tbl ))) AND ((i_size#6 = petite ) OR (i_size#6 = large ))) OR ((((i_color#7 = forest ) OR (i_color#7 = ghost )) AND ((i_units#8 = Lb ) OR (i_units#8 = Bundle ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))))))) AND isnotnull(i_manufact#5)) (8) Project [codegen id : 1] -Output [1]: [i_manufact#2] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Output [1]: [i_manufact#5] +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] (9) HashAggregate [codegen id : 1] -Input [1]: [i_manufact#2] -Keys [1]: [i_manufact#2] +Input [1]: [i_manufact#5] +Keys [1]: [i_manufact#5] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#8] -Results [2]: [i_manufact#2, count#9] +Aggregate Attributes [1]: [count#9] +Results [2]: [i_manufact#5, count#10] (10) Exchange -Input [2]: [i_manufact#2, count#9] -Arguments: hashpartitioning(i_manufact#2, 5), true, [id=#10] +Input [2]: [i_manufact#5, count#10] +Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [id=#11] (11) HashAggregate [codegen id : 2] -Input [2]: [i_manufact#2, count#9] -Keys [1]: [i_manufact#2] +Input [2]: [i_manufact#5, count#10] +Keys [1]: [i_manufact#5] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#11] -Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13] +Aggregate Attributes [1]: [count(1)#12] +Results [2]: [count(1)#12 AS item_cnt#13, i_manufact#5] (12) Filter [codegen id : 2] -Input [2]: [item_cnt#12, i_manufact#2#13] -Condition : (item_cnt#12 > 0) +Input [2]: [item_cnt#13, i_manufact#5] +Condition : (item_cnt#13 > 0) (13) Project [codegen id : 2] -Output [1]: [i_manufact#2#13] -Input [2]: [item_cnt#12, i_manufact#2#13] +Output [1]: [i_manufact#5] +Input [2]: [item_cnt#13, i_manufact#5] (14) BroadcastExchange -Input [1]: [i_manufact#2#13] +Input [1]: [i_manufact#5] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14] (15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_manufact#2] -Right keys [1]: [i_manufact#2#13] +Right keys [1]: [i_manufact#5] Join condition: None (16) Project [codegen id : 3] Output [1]: [i_product_name#3] -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#2#13] +Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] (17) HashAggregate [codegen id : 3] Input [1]: [i_product_name#3] @@ -105,7 +105,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] +Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt index 2d14d75ca9362..d36800823bb3f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (2) Project [i_manufact] Filter [item_cnt] - HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,count] + HashAggregate [i_manufact,count] [count(1),item_cnt,count] InputAdapter Exchange [i_manufact] #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index 13d73e61e1443..33c03d2b767dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -40,61 +40,61 @@ Output [2]: [i_manufact#2, i_product_name#3] Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] (5) Scan parquet default.item -Output [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [Or(Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,powder),EqualTo(i_color,khaki)),Or(EqualTo(i_units,Ounce),EqualTo(i_units,Oz))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,brown),EqualTo(i_color,honeydew)),Or(EqualTo(i_units,Bunch),EqualTo(i_units,Ton))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,floral),EqualTo(i_color,deep)),Or(EqualTo(i_units,N/A),EqualTo(i_units,Dozen))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,light),EqualTo(i_color,cornflower)),Or(EqualTo(i_units,Box),EqualTo(i_units,Pound))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large)))))),Or(And(EqualTo(i_category,Women),Or(And(And(Or(EqualTo(i_color,midnight),EqualTo(i_color,snow)),Or(EqualTo(i_units,Pallet),EqualTo(i_units,Gross))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))),And(And(Or(EqualTo(i_color,cyan),EqualTo(i_color,papaya)),Or(EqualTo(i_units,Cup),EqualTo(i_units,Dram))),Or(EqualTo(i_size,N/A),EqualTo(i_size,small))))),And(EqualTo(i_category,Men),Or(And(And(Or(EqualTo(i_color,orange),EqualTo(i_color,frosted)),Or(EqualTo(i_units,Each),EqualTo(i_units,Tbl))),Or(EqualTo(i_size,petite),EqualTo(i_size,large))),And(And(Or(EqualTo(i_color,forest),EqualTo(i_color,ghost)),Or(EqualTo(i_units,Lb),EqualTo(i_units,Bundle))),Or(EqualTo(i_size,medium),EqualTo(i_size,extra large))))))), IsNotNull(i_manufact)] +PushedFilters: [Or(Or(And(EqualTo(i_category,Women ),Or(And(And(Or(EqualTo(i_color,powder ),EqualTo(i_color,khaki )),Or(EqualTo(i_units,Ounce ),EqualTo(i_units,Oz ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))),And(And(Or(EqualTo(i_color,brown ),EqualTo(i_color,honeydew )),Or(EqualTo(i_units,Bunch ),EqualTo(i_units,Ton ))),Or(EqualTo(i_size,N/A ),EqualTo(i_size,small ))))),And(EqualTo(i_category,Men ),Or(And(And(Or(EqualTo(i_color,floral ),EqualTo(i_color,deep )),Or(EqualTo(i_units,N/A ),EqualTo(i_units,Dozen ))),Or(EqualTo(i_size,petite ),EqualTo(i_size,large ))),And(And(Or(EqualTo(i_color,light ),EqualTo(i_color,cornflower )),Or(EqualTo(i_units,Box ),EqualTo(i_units,Pound ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large )))))),Or(And(EqualTo(i_category,Women ),Or(And(And(Or(EqualTo(i_color,midnight ),EqualTo(i_color,snow )),Or(EqualTo(i_units,Pallet ),EqualTo(i_units,Gross ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))),And(And(Or(EqualTo(i_color,cyan ),EqualTo(i_color,papaya )),Or(EqualTo(i_units,Cup ),EqualTo(i_units,Dram ))),Or(EqualTo(i_size,N/A ),EqualTo(i_size,small ))))),And(EqualTo(i_category,Men ),Or(And(And(Or(EqualTo(i_color,orange ),EqualTo(i_color,frosted )),Or(EqualTo(i_units,Each ),EqualTo(i_units,Tbl ))),Or(EqualTo(i_size,petite ),EqualTo(i_size,large ))),And(And(Or(EqualTo(i_color,forest ),EqualTo(i_color,ghost )),Or(EqualTo(i_units,Lb ),EqualTo(i_units,Bundle ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))))))), IsNotNull(i_manufact)] ReadSchema: struct (6) ColumnarToRow [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] (7) Filter [codegen id : 1] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] -Condition : (((((i_category#4 = Women) AND (((((i_color#6 = powder) OR (i_color#6 = khaki)) AND ((i_units#7 = Ounce) OR (i_units#7 = Oz))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = brown) OR (i_color#6 = honeydew)) AND ((i_units#7 = Bunch) OR (i_units#7 = Ton))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = floral) OR (i_color#6 = deep)) AND ((i_units#7 = N/A) OR (i_units#7 = Dozen))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = light) OR (i_color#6 = cornflower)) AND ((i_units#7 = Box) OR (i_units#7 = Pound))) AND ((i_size#5 = medium) OR (i_size#5 = extra large)))))) OR (((i_category#4 = Women) AND (((((i_color#6 = midnight) OR (i_color#6 = snow)) AND ((i_units#7 = Pallet) OR (i_units#7 = Gross))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))) OR ((((i_color#6 = cyan) OR (i_color#6 = papaya)) AND ((i_units#7 = Cup) OR (i_units#7 = Dram))) AND ((i_size#5 = N/A) OR (i_size#5 = small))))) OR ((i_category#4 = Men) AND (((((i_color#6 = orange) OR (i_color#6 = frosted)) AND ((i_units#7 = Each) OR (i_units#7 = Tbl))) AND ((i_size#5 = petite) OR (i_size#5 = large))) OR ((((i_color#6 = forest) OR (i_color#6 = ghost)) AND ((i_units#7 = Lb) OR (i_units#7 = Bundle))) AND ((i_size#5 = medium) OR (i_size#5 = extra large))))))) AND isnotnull(i_manufact#2)) +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Condition : (((((i_category#4 = Women ) AND (((((i_color#7 = powder ) OR (i_color#7 = khaki )) AND ((i_units#8 = Ounce ) OR (i_units#8 = Oz ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))) OR ((((i_color#7 = brown ) OR (i_color#7 = honeydew )) AND ((i_units#8 = Bunch ) OR (i_units#8 = Ton ))) AND ((i_size#6 = N/A ) OR (i_size#6 = small ))))) OR ((i_category#4 = Men ) AND (((((i_color#7 = floral ) OR (i_color#7 = deep )) AND ((i_units#8 = N/A ) OR (i_units#8 = Dozen ))) AND ((i_size#6 = petite ) OR (i_size#6 = large ))) OR ((((i_color#7 = light ) OR (i_color#7 = cornflower )) AND ((i_units#8 = Box ) OR (i_units#8 = Pound ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large )))))) OR (((i_category#4 = Women ) AND (((((i_color#7 = midnight ) OR (i_color#7 = snow )) AND ((i_units#8 = Pallet ) OR (i_units#8 = Gross ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))) OR ((((i_color#7 = cyan ) OR (i_color#7 = papaya )) AND ((i_units#8 = Cup ) OR (i_units#8 = Dram ))) AND ((i_size#6 = N/A ) OR (i_size#6 = small ))))) OR ((i_category#4 = Men ) AND (((((i_color#7 = orange ) OR (i_color#7 = frosted )) AND ((i_units#8 = Each ) OR (i_units#8 = Tbl ))) AND ((i_size#6 = petite ) OR (i_size#6 = large ))) OR ((((i_color#7 = forest ) OR (i_color#7 = ghost )) AND ((i_units#8 = Lb ) OR (i_units#8 = Bundle ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))))))) AND isnotnull(i_manufact#5)) (8) Project [codegen id : 1] -Output [1]: [i_manufact#2] -Input [5]: [i_category#4, i_manufact#2, i_size#5, i_color#6, i_units#7] +Output [1]: [i_manufact#5] +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] (9) HashAggregate [codegen id : 1] -Input [1]: [i_manufact#2] -Keys [1]: [i_manufact#2] +Input [1]: [i_manufact#5] +Keys [1]: [i_manufact#5] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#8] -Results [2]: [i_manufact#2, count#9] +Aggregate Attributes [1]: [count#9] +Results [2]: [i_manufact#5, count#10] (10) Exchange -Input [2]: [i_manufact#2, count#9] -Arguments: hashpartitioning(i_manufact#2, 5), true, [id=#10] +Input [2]: [i_manufact#5, count#10] +Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [id=#11] (11) HashAggregate [codegen id : 2] -Input [2]: [i_manufact#2, count#9] -Keys [1]: [i_manufact#2] +Input [2]: [i_manufact#5, count#10] +Keys [1]: [i_manufact#5] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#11] -Results [2]: [count(1)#11 AS item_cnt#12, i_manufact#2 AS i_manufact#2#13] +Aggregate Attributes [1]: [count(1)#12] +Results [2]: [count(1)#12 AS item_cnt#13, i_manufact#5] (12) Filter [codegen id : 2] -Input [2]: [item_cnt#12, i_manufact#2#13] -Condition : (item_cnt#12 > 0) +Input [2]: [item_cnt#13, i_manufact#5] +Condition : (item_cnt#13 > 0) (13) Project [codegen id : 2] -Output [1]: [i_manufact#2#13] -Input [2]: [item_cnt#12, i_manufact#2#13] +Output [1]: [i_manufact#5] +Input [2]: [item_cnt#13, i_manufact#5] (14) BroadcastExchange -Input [1]: [i_manufact#2#13] +Input [1]: [i_manufact#5] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#14] (15) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_manufact#2] -Right keys [1]: [i_manufact#2#13] +Right keys [1]: [i_manufact#5] Join condition: None (16) Project [codegen id : 3] Output [1]: [i_product_name#3] -Input [3]: [i_manufact#2, i_product_name#3, i_manufact#2#13] +Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] (17) HashAggregate [codegen id : 3] Input [1]: [i_product_name#3] @@ -105,7 +105,7 @@ Results [1]: [i_product_name#3] (18) Exchange Input [1]: [i_product_name#3] -Arguments: hashpartitioning(i_product_name#3, 5), true, [id=#15] +Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [id=#15] (19) HashAggregate [codegen id : 4] Input [1]: [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt index 2d14d75ca9362..d36800823bb3f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/simplified.txt @@ -17,7 +17,7 @@ TakeOrderedAndProject [i_product_name] WholeStageCodegen (2) Project [i_manufact] Filter [item_cnt] - HashAggregate [i_manufact,count] [count(1),item_cnt,i_manufact,count] + HashAggregate [i_manufact,count] [count(1),item_cnt,count] InputAdapter Exchange [i_manufact] #3 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt index 4c5b5f25108e0..7e82b4d5df296 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/explain.txt @@ -81,57 +81,57 @@ Input [3]: [i_item_sk#5, i_current_price#6, i_category#7] Condition : (isnotnull(i_current_price#6) AND isnotnull(i_item_sk#5)) (7) Scan parquet default.item -Output [2]: [i_current_price#6, i_category#7] +Output [2]: [i_current_price#8, i_category#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] +Input [2]: [i_current_price#8, i_category#9] (9) Filter [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] -Condition : isnotnull(i_category#7) +Input [2]: [i_current_price#8, i_category#9] +Condition : isnotnull(i_category#9) (10) HashAggregate [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] -Keys [1]: [i_category#7] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#6))] -Aggregate Attributes [2]: [sum#8, count#9] -Results [3]: [i_category#7, sum#10, count#11] +Input [2]: [i_current_price#8, i_category#9] +Keys [1]: [i_category#9] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#8))] +Aggregate Attributes [2]: [sum#10, count#11] +Results [3]: [i_category#9, sum#12, count#13] (11) Exchange -Input [3]: [i_category#7, sum#10, count#11] -Arguments: hashpartitioning(i_category#7, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [3]: [i_category#9, sum#12, count#13] +Arguments: hashpartitioning(i_category#9, 5), ENSURE_REQUIREMENTS, [id=#14] (12) HashAggregate [codegen id : 2] -Input [3]: [i_category#7, sum#10, count#11] -Keys [1]: [i_category#7] -Functions [1]: [avg(UnscaledValue(i_current_price#6))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#6))#13] -Results [2]: [cast((avg(UnscaledValue(i_current_price#6))#13 / 100.0) as decimal(11,6)) AS avg(i_current_price)#14, i_category#7 AS i_category#7#15] +Input [3]: [i_category#9, sum#12, count#13] +Keys [1]: [i_category#9] +Functions [1]: [avg(UnscaledValue(i_current_price#8))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#8))#15] +Results [2]: [cast((avg(UnscaledValue(i_current_price#8))#15 / 100.0) as decimal(11,6)) AS avg(i_current_price)#16, i_category#9] (13) BroadcastExchange -Input [2]: [avg(i_current_price)#14, i_category#7#15] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#16] +Input [2]: [avg(i_current_price)#16, i_category#9] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#17] (14) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_category#7] -Right keys [1]: [i_category#7#15] +Right keys [1]: [i_category#9] Join condition: None (15) Filter [codegen id : 3] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] -Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#14)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] +Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#16)), DecimalType(14,7), true)) (16) Project [codegen id : 3] Output [1]: [i_item_sk#5] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] (17) BroadcastExchange Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (18) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#1] @@ -143,143 +143,143 @@ Output [2]: [ss_customer_sk#2, ss_sold_date_sk#3] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, i_item_sk#5] (20) Scan parquet default.date_dim -Output [2]: [d_date_sk#18, d_month_seq#19] +Output [2]: [d_date_sk#19, d_month_seq#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#18, d_month_seq#19] +Input [2]: [d_date_sk#19, d_month_seq#20] (22) Filter [codegen id : 4] -Input [2]: [d_date_sk#18, d_month_seq#19] -Condition : ((isnotnull(d_month_seq#19) AND (d_month_seq#19 = Subquery scalar-subquery#20, [id=#21])) AND isnotnull(d_date_sk#18)) +Input [2]: [d_date_sk#19, d_month_seq#20] +Condition : ((isnotnull(d_month_seq#20) AND (d_month_seq#20 = Subquery scalar-subquery#21, [id=#22])) AND isnotnull(d_date_sk#19)) (23) Project [codegen id : 4] -Output [1]: [d_date_sk#18] -Input [2]: [d_date_sk#18, d_month_seq#19] +Output [1]: [d_date_sk#19] +Input [2]: [d_date_sk#19, d_month_seq#20] (24) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (25) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (26) Project [codegen id : 5] Output [1]: [ss_customer_sk#2] -Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#18] +Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#19] (27) Exchange Input [1]: [ss_customer_sk#2] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#23] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] (28) Sort [codegen id : 6] Input [1]: [ss_customer_sk#2] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#25, ca_state#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#24, ca_state#25] +Input [2]: [ca_address_sk#25, ca_state#26] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : isnotnull(ca_address_sk#24) +Input [2]: [ca_address_sk#25, ca_state#26] +Condition : isnotnull(ca_address_sk#25) (32) Exchange -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [ca_address_sk#25, ca_state#26] +Arguments: hashpartitioning(ca_address_sk#25, 5), ENSURE_REQUIREMENTS, [id=#27] (33) Sort [codegen id : 8] -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#25, ca_state#26] +Arguments: [ca_address_sk#25 ASC NULLS FIRST], false, 0 (34) Scan parquet default.customer -Output [2]: [c_customer_sk#27, c_current_addr_sk#28] +Output [2]: [c_customer_sk#28, c_current_addr_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 9] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] (36) Filter [codegen id : 9] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Condition : (isnotnull(c_current_addr_sk#28) AND isnotnull(c_customer_sk#27)) +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Condition : (isnotnull(c_current_addr_sk#29) AND isnotnull(c_customer_sk#28)) (37) Exchange -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Arguments: hashpartitioning(c_current_addr_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Arguments: hashpartitioning(c_current_addr_sk#29, 5), ENSURE_REQUIREMENTS, [id=#30] (38) Sort [codegen id : 10] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Arguments: [c_current_addr_sk#28 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Arguments: [c_current_addr_sk#29 ASC NULLS FIRST], false, 0 (39) SortMergeJoin [codegen id : 11] -Left keys [1]: [ca_address_sk#24] -Right keys [1]: [c_current_addr_sk#28] +Left keys [1]: [ca_address_sk#25] +Right keys [1]: [c_current_addr_sk#29] Join condition: None (40) Project [codegen id : 11] -Output [2]: [ca_state#25, c_customer_sk#27] -Input [4]: [ca_address_sk#24, ca_state#25, c_customer_sk#27, c_current_addr_sk#28] +Output [2]: [ca_state#26, c_customer_sk#28] +Input [4]: [ca_address_sk#25, ca_state#26, c_customer_sk#28, c_current_addr_sk#29] (41) Exchange -Input [2]: [ca_state#25, c_customer_sk#27] -Arguments: hashpartitioning(c_customer_sk#27, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [ca_state#26, c_customer_sk#28] +Arguments: hashpartitioning(c_customer_sk#28, 5), ENSURE_REQUIREMENTS, [id=#31] (42) Sort [codegen id : 12] -Input [2]: [ca_state#25, c_customer_sk#27] -Arguments: [c_customer_sk#27 ASC NULLS FIRST], false, 0 +Input [2]: [ca_state#26, c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 13] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#27] +Right keys [1]: [c_customer_sk#28] Join condition: None (44) Project [codegen id : 13] -Output [1]: [ca_state#25] -Input [3]: [ss_customer_sk#2, ca_state#25, c_customer_sk#27] +Output [1]: [ca_state#26] +Input [3]: [ss_customer_sk#2, ca_state#26, c_customer_sk#28] (45) HashAggregate [codegen id : 13] -Input [1]: [ca_state#25] -Keys [1]: [ca_state#25] +Input [1]: [ca_state#26] +Keys [1]: [ca_state#26] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#31] -Results [2]: [ca_state#25, count#32] +Aggregate Attributes [1]: [count#32] +Results [2]: [ca_state#26, count#33] (46) Exchange -Input [2]: [ca_state#25, count#32] -Arguments: hashpartitioning(ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [2]: [ca_state#26, count#33] +Arguments: hashpartitioning(ca_state#26, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 14] -Input [2]: [ca_state#25, count#32] -Keys [1]: [ca_state#25] +Input [2]: [ca_state#26, count#33] +Keys [1]: [ca_state#26] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#34] -Results [3]: [ca_state#25 AS state#35, count(1)#34 AS cnt#36, count(1)#34 AS count(1)#37] +Aggregate Attributes [1]: [count(1)#35] +Results [3]: [ca_state#26 AS state#36, count(1)#35 AS cnt#37, count(1)#35 AS count(1)#38] (48) Filter [codegen id : 14] -Input [3]: [state#35, cnt#36, count(1)#37] -Condition : (count(1)#37 >= 10) +Input [3]: [state#36, cnt#37, count(1)#38] +Condition : (count(1)#38 >= 10) (49) Project [codegen id : 14] -Output [2]: [state#35, cnt#36] -Input [3]: [state#35, cnt#36, count(1)#37] +Output [2]: [state#36, cnt#37] +Input [3]: [state#36, cnt#37, count(1)#38] (50) TakeOrderedAndProject -Input [2]: [state#35, cnt#36] -Arguments: 100, [cnt#36 ASC NULLS FIRST], [state#35, cnt#36] +Input [2]: [state#36, cnt#37] +Arguments: 100, [cnt#37 ASC NULLS FIRST], [state#36, cnt#37] ===== Subqueries ===== @@ -288,9 +288,9 @@ ReusedExchange (51) (51) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#19] -Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#20, [id=#21] +Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#21, [id=#22] * HashAggregate (58) +- Exchange (57) +- * HashAggregate (56) @@ -301,39 +301,39 @@ Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquer (52) Scan parquet default.date_dim -Output [3]: [d_month_seq#19, d_year#38, d_moy#39] +Output [3]: [d_month_seq#39, d_year#40, d_moy#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (53) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] (54) Filter [codegen id : 1] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] -Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] +Condition : (((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2000)) AND (d_moy#41 = 1)) (55) Project [codegen id : 1] -Output [1]: [d_month_seq#19] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] +Output [1]: [d_month_seq#39] +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] (56) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#19] -Keys [1]: [d_month_seq#19] +Input [1]: [d_month_seq#39] +Keys [1]: [d_month_seq#39] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#19] +Results [1]: [d_month_seq#39] (57) Exchange -Input [1]: [d_month_seq#19] -Arguments: hashpartitioning(d_month_seq#19, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [1]: [d_month_seq#39] +Arguments: hashpartitioning(d_month_seq#39, 5), ENSURE_REQUIREMENTS, [id=#42] (58) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#19] -Keys [1]: [d_month_seq#19] +Input [1]: [d_month_seq#39] +Keys [1]: [d_month_seq#39] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#19] +Results [1]: [d_month_seq#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt index a5097742ec146..1cdb7a5df8e6f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.sf100/simplified.txt @@ -38,7 +38,7 @@ TakeOrderedAndProject [cnt,state] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] InputAdapter Exchange [i_category] #6 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt index 58fabaa8e44be..0f98039fc0f7f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/explain.txt @@ -160,57 +160,57 @@ Input [3]: [i_item_sk#16, i_current_price#17, i_category#18] Condition : (isnotnull(i_current_price#17) AND isnotnull(i_item_sk#16)) (26) Scan parquet default.item -Output [2]: [i_current_price#17, i_category#18] +Output [2]: [i_current_price#19, i_category#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (27) ColumnarToRow [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] +Input [2]: [i_current_price#19, i_category#20] (28) Filter [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] -Condition : isnotnull(i_category#18) +Input [2]: [i_current_price#19, i_category#20] +Condition : isnotnull(i_category#20) (29) HashAggregate [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] -Keys [1]: [i_category#18] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] -Aggregate Attributes [2]: [sum#19, count#20] -Results [3]: [i_category#18, sum#21, count#22] +Input [2]: [i_current_price#19, i_category#20] +Keys [1]: [i_category#20] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#19))] +Aggregate Attributes [2]: [sum#21, count#22] +Results [3]: [i_category#20, sum#23, count#24] (30) Exchange -Input [3]: [i_category#18, sum#21, count#22] -Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [i_category#20, sum#23, count#24] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#25] (31) HashAggregate [codegen id : 5] -Input [3]: [i_category#18, sum#21, count#22] -Keys [1]: [i_category#18] -Functions [1]: [avg(UnscaledValue(i_current_price#17))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#17))#24] -Results [2]: [cast((avg(UnscaledValue(i_current_price#17))#24 / 100.0) as decimal(11,6)) AS avg(i_current_price)#25, i_category#18 AS i_category#18#26] +Input [3]: [i_category#20, sum#23, count#24] +Keys [1]: [i_category#20] +Functions [1]: [avg(UnscaledValue(i_current_price#19))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#19))#26] +Results [2]: [cast((avg(UnscaledValue(i_current_price#19))#26 / 100.0) as decimal(11,6)) AS avg(i_current_price)#27, i_category#20] (32) BroadcastExchange -Input [2]: [avg(i_current_price)#25, i_category#18#26] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#27] +Input [2]: [avg(i_current_price)#27, i_category#20] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#28] (33) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_category#18] -Right keys [1]: [i_category#18#26] +Right keys [1]: [i_category#20] Join condition: None (34) Filter [codegen id : 6] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] -Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#25)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] +Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#27)), DecimalType(14,7), true)) (35) Project [codegen id : 6] Output [1]: [i_item_sk#16] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] (36) BroadcastExchange Input [1]: [i_item_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (37) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#6] @@ -225,31 +225,31 @@ Input [3]: [ca_state#2, ss_item_sk#6, i_item_sk#16] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [2]: [ca_state#2, count#30] +Aggregate Attributes [1]: [count#30] +Results [2]: [ca_state#2, count#31] (40) Exchange -Input [2]: [ca_state#2, count#30] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [2]: [ca_state#2, count#31] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#32] (41) HashAggregate [codegen id : 8] -Input [2]: [ca_state#2, count#30] +Input [2]: [ca_state#2, count#31] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [3]: [ca_state#2 AS state#33, count(1)#32 AS cnt#34, count(1)#32 AS count(1)#35] +Aggregate Attributes [1]: [count(1)#33] +Results [3]: [ca_state#2 AS state#34, count(1)#33 AS cnt#35, count(1)#33 AS count(1)#36] (42) Filter [codegen id : 8] -Input [3]: [state#33, cnt#34, count(1)#35] -Condition : (count(1)#35 >= 10) +Input [3]: [state#34, cnt#35, count(1)#36] +Condition : (count(1)#36 >= 10) (43) Project [codegen id : 8] -Output [2]: [state#33, cnt#34] -Input [3]: [state#33, cnt#34, count(1)#35] +Output [2]: [state#34, cnt#35] +Input [3]: [state#34, cnt#35, count(1)#36] (44) TakeOrderedAndProject -Input [2]: [state#33, cnt#34] -Arguments: 100, [cnt#34 ASC NULLS FIRST], [state#33, cnt#34] +Input [2]: [state#34, cnt#35] +Arguments: 100, [cnt#35 ASC NULLS FIRST], [state#34, cnt#35] ===== Subqueries ===== @@ -271,39 +271,39 @@ Subquery:2 Hosting operator id = 18 Hosting Expression = Subquery scalar-subquer (46) Scan parquet default.date_dim -Output [3]: [d_month_seq#12, d_year#36, d_moy#37] +Output [3]: [d_month_seq#37, d_year#38, d_moy#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (47) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] (48) Filter [codegen id : 1] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] -Condition : (((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2000)) AND (d_moy#37 = 1)) +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] +Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) (49) Project [codegen id : 1] -Output [1]: [d_month_seq#12] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] +Output [1]: [d_month_seq#37] +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] (50) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#12] -Keys [1]: [d_month_seq#12] +Input [1]: [d_month_seq#37] +Keys [1]: [d_month_seq#37] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#12] +Results [1]: [d_month_seq#37] (51) Exchange -Input [1]: [d_month_seq#12] -Arguments: hashpartitioning(d_month_seq#12, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [1]: [d_month_seq#37] +Arguments: hashpartitioning(d_month_seq#37, 5), ENSURE_REQUIREMENTS, [id=#40] (52) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#12] -Keys [1]: [d_month_seq#12] +Input [1]: [d_month_seq#37] +Keys [1]: [d_month_seq#37] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#12] +Results [1]: [d_month_seq#37] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt index 8b60c2f86a280..d7fad5948f64b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/simplified.txt @@ -68,7 +68,7 @@ TakeOrderedAndProject [cnt,state] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] InputAdapter Exchange [i_category] #8 WholeStageCodegen (4) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt index 8297114e1e9ab..94e76ced527a0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt @@ -84,7 +84,7 @@ Input [2]: [d_date_sk#8, d_date#9] (11) Filter [codegen id : 2] Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 2000-01-27)) AND (d_date#9 <= 2000-04-26)) AND isnotnull(d_date_sk#8)) (12) Project [codegen id : 2] Output [1]: [d_date_sk#8] @@ -119,82 +119,82 @@ Input [3]: [ws_item_sk#4, sum#13, count#14] Keys [1]: [ws_item_sk#4] Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#5))] Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#5))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4 AS ws_item_sk#4#18] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#5))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4] (19) Filter -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4#18] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4] Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) (20) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_sk#1] -Right keys [1]: [ws_item_sk#4#18] +Right keys [1]: [ws_item_sk#4] Join condition: None (21) Project [codegen id : 4] Output [2]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17] -Input [3]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4#18] +Input [3]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4] (22) BroadcastExchange Input [2]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (23) Scan parquet default.web_sales -Output [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] +Output [3]: [ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ws_sold_date_sk#21), dynamicpruningexpression(ws_sold_date_sk#21 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_ext_discount_amt)] ReadSchema: struct (24) ColumnarToRow -Input [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] +Input [3]: [ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] (25) Filter -Input [3]: [ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] -Condition : (isnotnull(ws_item_sk#4) AND isnotnull(ws_ext_discount_amt#5)) +Input [3]: [ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] +Condition : (isnotnull(ws_item_sk#19) AND isnotnull(ws_ext_discount_amt#20)) (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#1] -Right keys [1]: [ws_item_sk#4] -Join condition: (cast(ws_ext_discount_amt#5 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) +Right keys [1]: [ws_item_sk#19] +Join condition: (cast(ws_ext_discount_amt#20 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) (27) Project [codegen id : 6] -Output [2]: [ws_ext_discount_amt#5, ws_sold_date_sk#6] -Input [5]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#4, ws_ext_discount_amt#5, ws_sold_date_sk#6] +Output [2]: [ws_ext_discount_amt#20, ws_sold_date_sk#21] +Input [5]: [i_item_sk#1, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#19, ws_ext_discount_amt#20, ws_sold_date_sk#21] (28) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ws_sold_date_sk#21] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 6] -Output [1]: [ws_ext_discount_amt#5] -Input [3]: [ws_ext_discount_amt#5, ws_sold_date_sk#6, d_date_sk#8] +Output [1]: [ws_ext_discount_amt#20] +Input [3]: [ws_ext_discount_amt#20, ws_sold_date_sk#21, d_date_sk#22] (31) HashAggregate [codegen id : 6] -Input [1]: [ws_ext_discount_amt#5] +Input [1]: [ws_ext_discount_amt#20] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#5))] -Aggregate Attributes [1]: [sum#20] -Results [1]: [sum#21] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#20))] +Aggregate Attributes [1]: [sum#23] +Results [1]: [sum#24] (32) Exchange -Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [sum#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#25] (33) HashAggregate [codegen id : 7] -Input [1]: [sum#21] +Input [1]: [sum#24] Keys: [] -Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#5))#23] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#5))#23,17,2) AS Excess Discount Amount #24] +Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#20))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#20))#26] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#20))#26,17,2) AS Excess Discount Amount #27] (34) Sort [codegen id : 7] -Input [1]: [Excess Discount Amount #24] -Arguments: [Excess Discount Amount #24 ASC NULLS FIRST], true, 0 +Input [1]: [Excess Discount Amount #27] +Arguments: [Excess Discount Amount #27 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -205,6 +205,6 @@ ReusedExchange (35) (35) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 23 Hosting Expression = ws_sold_date_sk#21 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt index cc3ffa0de4bfd..e83a3e67e5c6f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/simplified.txt @@ -23,7 +23,7 @@ WholeStageCodegen (7) InputAdapter Scan parquet default.item [i_item_sk,i_manufact_id] Filter [(1.3 * avg(ws_ext_discount_amt))] - HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,sum,count] + HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] InputAdapter Exchange [ws_item_sk] #4 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt index faf82026138f6..46953fe184f28 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt @@ -82,119 +82,119 @@ Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] (11) Scan parquet default.web_sales -Output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] +Output [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] +Input [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] (13) Filter [codegen id : 3] -Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] -Condition : isnotnull(ws_item_sk#1) +Input [3]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10] +Condition : isnotnull(ws_item_sk#8) (14) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_date#9] +Output [2]: [d_date_sk#11, d_date#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#8, d_date#9] +Input [2]: [d_date_sk#11, d_date#12] (16) Filter [codegen id : 2] -Input [2]: [d_date_sk#8, d_date#9] -Condition : (((isnotnull(d_date#9) AND (d_date#9 >= 10983)) AND (d_date#9 <= 11073)) AND isnotnull(d_date_sk#8)) +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-01-27)) AND (d_date#12 <= 2000-04-26)) AND isnotnull(d_date_sk#11)) (17) Project [codegen id : 2] -Output [1]: [d_date_sk#8] -Input [2]: [d_date_sk#8, d_date#9] +Output [1]: [d_date_sk#11] +Input [2]: [d_date_sk#11, d_date#12] (18) BroadcastExchange -Input [1]: [d_date_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] (19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ws_sold_date_sk#10] +Right keys [1]: [d_date_sk#11] Join condition: None (20) Project [codegen id : 3] -Output [2]: [ws_item_sk#1, ws_ext_discount_amt#2] -Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#8] +Output [2]: [ws_item_sk#8, ws_ext_discount_amt#9] +Input [4]: [ws_item_sk#8, ws_ext_discount_amt#9, ws_sold_date_sk#10, d_date_sk#11] (21) HashAggregate [codegen id : 3] -Input [2]: [ws_item_sk#1, ws_ext_discount_amt#2] -Keys [1]: [ws_item_sk#1] -Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [2]: [sum#11, count#12] -Results [3]: [ws_item_sk#1, sum#13, count#14] +Input [2]: [ws_item_sk#8, ws_ext_discount_amt#9] +Keys [1]: [ws_item_sk#8] +Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#9))] +Aggregate Attributes [2]: [sum#14, count#15] +Results [3]: [ws_item_sk#8, sum#16, count#17] (22) Exchange -Input [3]: [ws_item_sk#1, sum#13, count#14] -Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [3]: [ws_item_sk#8, sum#16, count#17] +Arguments: hashpartitioning(ws_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#18] (23) HashAggregate [codegen id : 4] -Input [3]: [ws_item_sk#1, sum#13, count#14] -Keys [1]: [ws_item_sk#1] -Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#2))#16] -Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#2))#16 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1 AS ws_item_sk#1#18] +Input [3]: [ws_item_sk#8, sum#16, count#17] +Keys [1]: [ws_item_sk#8] +Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#9))] +Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#9))#19] +Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#9))#19 / 100.0) as decimal(11,6)))), DecimalType(14,7), true) AS (1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] (24) Filter [codegen id : 4] -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] -Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] +Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#20) (25) BroadcastExchange -Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#19] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#21] (26) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_sk#5] -Right keys [1]: [ws_item_sk#1#18] -Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) +Right keys [1]: [ws_item_sk#8] +Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#20) (27) Project [codegen id : 6] Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] -Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#1#18] +Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#20, ws_item_sk#8] (28) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#3] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 6] Output [1]: [ws_ext_discount_amt#2] -Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#8] +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#22] (31) HashAggregate [codegen id : 6] Input [1]: [ws_ext_discount_amt#2] Keys: [] Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum#20] -Results [1]: [sum#21] +Aggregate Attributes [1]: [sum#23] +Results [1]: [sum#24] (32) Exchange -Input [1]: [sum#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [sum#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#25] (33) HashAggregate [codegen id : 7] -Input [1]: [sum#21] +Input [1]: [sum#24] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#23] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#23,17,2) AS Excess Discount Amount #24] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#26] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#26,17,2) AS Excess Discount Amount #27] (34) Sort [codegen id : 7] -Input [1]: [Excess Discount Amount #24] -Arguments: [Excess Discount Amount #24 ASC NULLS FIRST], true, 0 +Input [1]: [Excess Discount Amount #27] +Arguments: [Excess Discount Amount #27 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -203,8 +203,8 @@ ReusedExchange (35) (35) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#22] -Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt index 3108c3bc82c7c..0df713c0e075e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/simplified.txt @@ -29,7 +29,7 @@ WholeStageCodegen (7) BroadcastExchange #4 WholeStageCodegen (4) Filter [(1.3 * avg(ws_ext_discount_amt))] - HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),ws_item_sk,sum,count] + HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] InputAdapter Exchange [ws_item_sk] #5 WholeStageCodegen (3) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt index 312aba94bd9c6..25baf22f2b4b5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt @@ -33,39 +33,39 @@ TakeOrderedAndProject (116) : : : : +- * HashAggregate (44) : : : : +- Exchange (43) : : : : +- * HashAggregate (42) - : : : : +- SortMergeJoin LeftSemi (41) - : : : : :- * Sort (26) - : : : : : +- Exchange (25) - : : : : : +- * Project (24) - : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : :- * Project (18) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : :- * Filter (11) - : : : : : : : +- * ColumnarToRow (10) - : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : +- BroadcastExchange (16) - : : : : : : +- * Project (15) - : : : : : : +- * Filter (14) - : : : : : : +- * ColumnarToRow (13) - : : : : : : +- Scan parquet default.date_dim (12) - : : : : : +- BroadcastExchange (22) - : : : : : +- * Filter (21) - : : : : : +- * ColumnarToRow (20) - : : : : : +- Scan parquet default.item (19) - : : : : +- * Sort (40) - : : : : +- Exchange (39) - : : : : +- * Project (38) - : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : :- * Project (32) - : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : :- * Filter (29) - : : : : : : +- * ColumnarToRow (28) - : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : +- ReusedExchange (30) - : : : : +- BroadcastExchange (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.item (33) + : : : : +- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * Project (18) + : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : :- * Filter (11) + : : : : : : +- * ColumnarToRow (10) + : : : : : : +- Scan parquet default.store_sales (9) + : : : : : +- BroadcastExchange (16) + : : : : : +- * Project (15) + : : : : : +- * Filter (14) + : : : : : +- * ColumnarToRow (13) + : : : : : +- Scan parquet default.date_dim (12) + : : : : +- BroadcastExchange (39) + : : : : +- SortMergeJoin LeftSemi (38) + : : : : :- * Sort (23) + : : : : : +- Exchange (22) + : : : : : +- * Filter (21) + : : : : : +- * ColumnarToRow (20) + : : : : : +- Scan parquet default.item (19) + : : : : +- * Sort (37) + : : : : +- Exchange (36) + : : : : +- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * Filter (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- Scan parquet default.catalog_sales (24) + : : : : : +- ReusedExchange (27) + : : : : +- BroadcastExchange (33) + : : : : +- * Filter (32) + : : : : +- * ColumnarToRow (31) + : : : : +- Scan parquet default.item (30) : : : +- * Sort (57) : : : +- Exchange (56) : : : +- * Project (55) @@ -147,507 +147,507 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 20] +(7) ColumnarToRow [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 20] +(8) Filter [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 5] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +(10) ColumnarToRow [codegen id : 10] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -(11) Filter [codegen id : 5] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +(11) Filter [codegen id : 10] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_item_sk#11) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#14, d_year#15] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#14, d_year#15] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1998)) AND (d_year#15 <= 2000)) AND isnotnull(d_date_sk#14)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#14] +Input [2]: [d_date_sk#14, d_year#15] (16) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(17) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(17) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#14] Join condition: None -(18) Project [codegen id : 5] -Output [1]: [ss_item_sk#1] -Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] +(18) Project [codegen id : 10] +Output [1]: [ss_item_sk#11] +Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] (19) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) - -(22) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] - -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join condition: None - -(24) Project [codegen id : 5] -Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] -Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) -(25) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] +(22) Exchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] -(26) Sort [codegen id : 6] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 +(23) Sort [codegen id : 5] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 -(27) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] +(24) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +(25) ColumnarToRow [codegen id : 8] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] -(29) Filter [codegen id : 9] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +(26) Filter [codegen id : 8] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#22) -(30) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +(27) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#24] -(31) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +(28) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None -(32) Project [codegen id : 9] -Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] +(29) Project [codegen id : 8] +Output [1]: [cs_item_sk#22] +Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] -(33) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(30) Scan parquet default.item +Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(31) ColumnarToRow [codegen id : 7] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -(35) Filter [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +(32) Filter [codegen id : 7] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Condition : isnotnull(i_item_sk#25) -(36) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +(33) BroadcastExchange +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_item_sk#22] +Right keys [1]: [i_item_sk#25] Join condition: None -(38) Project [codegen id : 9] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(35) Project [codegen id : 8] +Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] + +(36) Exchange +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] -(39) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] +(37) Sort [codegen id : 9] +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 -(40) Sort [codegen id : 10] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(38) SortMergeJoin +Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] +Join condition: None + +(39) BroadcastExchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] -(41) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +(40) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_item_sk#11] +Right keys [1]: [i_item_sk#17] Join condition: None -(42) HashAggregate [codegen id : 11] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(41) Project [codegen id : 10] +Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] +Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] + +(42) HashAggregate [codegen id : 10] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (43) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(44) HashAggregate [codegen id : 12] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(44) HashAggregate [codegen id : 11] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (45) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] -(46) Sort [codegen id : 13] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 12] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +(48) ColumnarToRow [codegen id : 15] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -(49) Filter [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(49) Filter [codegen id : 15] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_item_sk#37) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#39] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(51) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#39] Join condition: None -(52) Project [codegen id : 16] -Output [1]: [ws_item_sk#26] -Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] +(52) Project [codegen id : 15] +Output [1]: [ws_item_sk#37] +Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] -(53) ReusedExchange [Reuses operator id: 36] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(53) ReusedExchange [Reuses operator id: 33] +Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(54) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#40] Join condition: None -(55) Project [codegen id : 16] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(55) Project [codegen id : 15] +Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] (56) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] -(57) Sort [codegen id : 17] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 16] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] +Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] Join condition: None -(59) HashAggregate [codegen id : 18] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(59) HashAggregate [codegen id : 17] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (60) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] -(61) HashAggregate [codegen id : 19] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(61) HashAggregate [codegen id : 18] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (62) BroadcastExchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] -(63) BroadcastHashJoin [codegen id : 20] +(63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#16, class_id#17, category_id#18] +Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join condition: None -(64) Project [codegen id : 20] -Output [1]: [i_item_sk#7 AS ss_item_sk#31] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] +(64) Project [codegen id : 19] +Output [1]: [i_item_sk#7 AS ss_item_sk#47] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] (65) Exchange -Input [1]: [ss_item_sk#31] -Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [1]: [ss_item_sk#47] +Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] -(66) Sort [codegen id : 21] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 20] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#47] Join condition: None (68) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#33] +Output [2]: [d_date_sk#49, d_week_seq#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(69) ColumnarToRow [codegen id : 22] -Input [2]: [d_date_sk#12, d_week_seq#33] +(69) ColumnarToRow [codegen id : 21] +Input [2]: [d_date_sk#49, d_week_seq#50] -(70) Filter [codegen id : 22] -Input [2]: [d_date_sk#12, d_week_seq#33] -Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#34, [id=#35])) AND isnotnull(d_date_sk#12)) +(70) Filter [codegen id : 21] +Input [2]: [d_date_sk#49, d_week_seq#50] +Condition : ((isnotnull(d_week_seq#50) AND (d_week_seq#50 = Subquery scalar-subquery#51, [id=#52])) AND isnotnull(d_date_sk#49)) -(71) Project [codegen id : 22] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#33] +(71) Project [codegen id : 21] +Output [1]: [d_date_sk#49] +Input [2]: [d_date_sk#49, d_week_seq#50] (72) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [1]: [d_date_sk#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] -(73) BroadcastHashJoin [codegen id : 44] +(73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#49] Join condition: None -(74) Project [codegen id : 44] +(74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] (75) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(76) ColumnarToRow [codegen id : 22] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -(77) Filter [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +(77) Filter [codegen id : 22] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Condition : (((isnotnull(i_item_sk#54) AND isnotnull(i_brand_id#55)) AND isnotnull(i_class_id#56)) AND isnotnull(i_category_id#57)) (78) Exchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: hashpartitioning(i_item_sk#54, 5), ENSURE_REQUIREMENTS, [id=#58] -(79) Sort [codegen id : 24] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 +(79) Sort [codegen id : 23] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: [i_item_sk#54 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(81) Sort [codegen id : 43] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 41] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [i_item_sk#54] +Right keys [1]: [ss_item_sk#47] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] +Input [4]: [i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#59] -(84) BroadcastHashJoin [codegen id : 44] +(84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#54] Join condition: None -(85) Project [codegen id : 44] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(85) Project [codegen id : 42] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#54, i_brand_id#55, i_class_id#56, i_category_id#57] -(86) HashAggregate [codegen id : 44] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +(86) HashAggregate [codegen id : 42] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#55, i_class_id#56, i_category_id#57] +Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#39, isEmpty#40, count#41] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] +Aggregate Attributes [3]: [sum#60, isEmpty#61, count#62] +Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] +Arguments: hashpartitioning(i_brand_id#55, i_class_id#56, i_category_id#57, 5), ENSURE_REQUIREMENTS, [id=#66] -(88) HashAggregate [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#42, isEmpty#43, count#44] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +(88) HashAggregate [codegen id : 86] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum#63, isEmpty#64, count#65] +Keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46, count(1)#47] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sales#48, count(1)#47 AS number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#46 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67, count(1)#68] +Results [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sales#69, count(1)#68 AS number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#67 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] -(89) Filter [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50 as decimal(32,6)) > cast(Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) +(89) Filter [codegen id : 86] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(32,6)) > cast(Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) -(90) Project [codegen id : 90] -Output [6]: [store AS channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#50] +(90) Project [codegen id : 86] +Output [6]: [store AS channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70] +Input [6]: [i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#71] (91) Scan parquet default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#54)] +PartitionFilters: [isnotnull(ss_sold_date_sk#78), dynamicpruningexpression(ss_sold_date_sk#78 IN dynamicpruning#79)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 45] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +(92) ColumnarToRow [codegen id : 43] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] -(93) Filter [codegen id : 45] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +(93) Filter [codegen id : 43] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Condition : isnotnull(ss_item_sk#75) (94) Exchange -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Arguments: hashpartitioning(ss_item_sk#75, 5), ENSURE_REQUIREMENTS, [id=#80] -(95) Sort [codegen id : 46] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 44] +Input [4]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78] +Arguments: [ss_item_sk#75 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(97) Sort [codegen id : 65] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 62] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [ss_item_sk#75] +Right keys [1]: [ss_item_sk#47] Join condition: None (99) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#33] +Output [2]: [d_date_sk#81, d_week_seq#82] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(100) ColumnarToRow [codegen id : 66] -Input [2]: [d_date_sk#12, d_week_seq#33] +(100) ColumnarToRow [codegen id : 63] +Input [2]: [d_date_sk#81, d_week_seq#82] -(101) Filter [codegen id : 66] -Input [2]: [d_date_sk#12, d_week_seq#33] -Condition : ((isnotnull(d_week_seq#33) AND (d_week_seq#33 = Subquery scalar-subquery#56, [id=#57])) AND isnotnull(d_date_sk#12)) +(101) Filter [codegen id : 63] +Input [2]: [d_date_sk#81, d_week_seq#82] +Condition : ((isnotnull(d_week_seq#82) AND (d_week_seq#82 = Subquery scalar-subquery#83, [id=#84])) AND isnotnull(d_date_sk#81)) -(102) Project [codegen id : 66] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#33] +(102) Project [codegen id : 63] +Output [1]: [d_date_sk#81] +Input [2]: [d_date_sk#81, d_week_seq#82] (103) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] +Input [1]: [d_date_sk#81] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#85] -(104) BroadcastHashJoin [codegen id : 88] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(104) BroadcastHashJoin [codegen id : 84] +Left keys [1]: [ss_sold_date_sk#78] +Right keys [1]: [d_date_sk#81] Join condition: None -(105) Project [codegen id : 88] -Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +(105) Project [codegen id : 84] +Output [3]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77] +Input [5]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, ss_sold_date_sk#78, d_date_sk#81] (106) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +Output [4]: [i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] -(107) BroadcastHashJoin [codegen id : 88] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#59] +(107) BroadcastHashJoin [codegen id : 84] +Left keys [1]: [ss_item_sk#75] +Right keys [1]: [i_item_sk#86] Join condition: None -(108) Project [codegen id : 88] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#59, i_brand_id#60, i_class_id#61, i_category_id#62] +(108) Project [codegen id : 84] +Output [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] +Input [7]: [ss_item_sk#75, ss_quantity#76, ss_list_price#77, i_item_sk#86, i_brand_id#87, i_class_id#88, i_category_id#89] -(109) HashAggregate [codegen id : 88] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#60, i_class_id#61, i_category_id#62] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] +(109) HashAggregate [codegen id : 84] +Input [5]: [ss_quantity#76, ss_list_price#77, i_brand_id#87, i_class_id#88, i_category_id#89] +Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#90, isEmpty#91, count#92] +Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] (110) Exchange -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] -Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] +Arguments: hashpartitioning(i_brand_id#87, i_class_id#88, i_category_id#89, 5), ENSURE_REQUIREMENTS, [id=#96] -(111) HashAggregate [codegen id : 89] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68] -Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70, count(1)#71] -Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sales#72, count(1)#71 AS number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] +(111) HashAggregate [codegen id : 85] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum#93, isEmpty#94, count#95] +Keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] +Results [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#99, count(1)#98 AS number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] -(112) Filter [codegen id : 89] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#51, [id=#52] as decimal(32,6)))) +(112) Filter [codegen id : 85] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#72, [id=#73] as decimal(32,6)))) -(113) Project [codegen id : 89] -Output [6]: [store AS channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#74] +(113) Project [codegen id : 85] +Output [6]: [store AS channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Input [6]: [i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#76 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#77 as decimal(12,2)))), DecimalType(18,2), true))#101] (114) BroadcastExchange -Input [6]: [channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#76] +Input [6]: [channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#103] -(115) BroadcastHashJoin [codegen id : 90] -Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62] +(115) BroadcastHashJoin [codegen id : 86] +Left keys [3]: [i_brand_id#55, i_class_id#56, i_category_id#57] +Right keys [3]: [i_brand_id#87, i_class_id#88, i_category_id#89] Join condition: None (116) TakeOrderedAndProject -Input [12]: [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] -Arguments: 100, [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#53, i_brand_id#8, i_class_id#9, i_category_id#10, sales#48, number_sales#49, channel#75, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73] +Input [12]: [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] +Arguments: 100, [i_brand_id#55 ASC NULLS FIRST, i_class_id#56 ASC NULLS FIRST, i_category_id#57 ASC NULLS FIRST], [channel#74, i_brand_id#55, i_class_id#56, i_category_id#57, sales#69, number_sales#70, channel#102, i_brand_id#87, i_class_id#88, i_category_id#89, sales#99, number_sales#100] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#51, [id=#52] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#72, [id=#73] * HashAggregate (139) +- Exchange (138) +- * HashAggregate (137) @@ -674,140 +674,140 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (117) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ss_sold_date_sk#106), dynamicpruningexpression(ss_sold_date_sk#106 IN dynamicpruning#107)] ReadSchema: struct (118) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106] (119) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#108, d_year#109] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (120) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#108, d_year#109] (121) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#108, d_year#109] +Condition : (((isnotnull(d_year#109) AND (d_year#109 >= 1998)) AND (d_year#109 <= 2000)) AND isnotnull(d_date_sk#108)) (122) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#108] +Input [2]: [d_date_sk#108, d_year#109] (123) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#78] +Input [1]: [d_date_sk#108] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#110] (124) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#106] +Right keys [1]: [d_date_sk#108] Join condition: None (125) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#79, ss_list_price#3 AS list_price#80] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#104 AS quantity#111, ss_list_price#105 AS list_price#112] +Input [4]: [ss_quantity#104, ss_list_price#105, ss_sold_date_sk#106, d_date_sk#108] (126) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] +Output [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(cs_sold_date_sk#115), dynamicpruningexpression(cs_sold_date_sk#115 IN dynamicpruning#107)] ReadSchema: struct (127) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21] +Input [3]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115] (128) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#116] (129) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#115] +Right keys [1]: [d_date_sk#116] Join condition: None (130) Project [codegen id : 4] -Output [2]: [cs_quantity#81 AS quantity#83, cs_list_price#82 AS list_price#84] -Input [4]: [cs_quantity#81, cs_list_price#82, cs_sold_date_sk#21, d_date_sk#12] +Output [2]: [cs_quantity#113 AS quantity#117, cs_list_price#114 AS list_price#118] +Input [4]: [cs_quantity#113, cs_list_price#114, cs_sold_date_sk#115, d_date_sk#116] (131) Scan parquet default.web_sales -Output [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] +Output [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#77)] +PartitionFilters: [isnotnull(ws_sold_date_sk#121), dynamicpruningexpression(ws_sold_date_sk#121 IN dynamicpruning#107)] ReadSchema: struct (132) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27] +Input [3]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121] (133) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#122] (134) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#121] +Right keys [1]: [d_date_sk#122] Join condition: None (135) Project [codegen id : 6] -Output [2]: [ws_quantity#85 AS quantity#87, ws_list_price#86 AS list_price#88] -Input [4]: [ws_quantity#85, ws_list_price#86, ws_sold_date_sk#27, d_date_sk#12] +Output [2]: [ws_quantity#119 AS quantity#123, ws_list_price#120 AS list_price#124] +Input [4]: [ws_quantity#119, ws_list_price#120, ws_sold_date_sk#121, d_date_sk#122] (136) Union (137) HashAggregate [codegen id : 7] -Input [2]: [quantity#79, list_price#80] +Input [2]: [quantity#111, list_price#112] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#89, count#90] -Results [2]: [sum#91, count#92] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#125, count#126] +Results [2]: [sum#127, count#128] (138) Exchange -Input [2]: [sum#91, count#92] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] +Input [2]: [sum#127, count#128] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] (139) HashAggregate [codegen id : 8] -Input [2]: [sum#91, count#92] +Input [2]: [sum#127, count#128] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#79 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2), true))#94 AS average_sales#95] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#111 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#112 as decimal(12,2)))), DecimalType(18,2), true))#130 AS average_sales#131] -Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#77 +Subquery:2 Hosting operator id = 117 Hosting Expression = ss_sold_date_sk#106 IN dynamicpruning#107 ReusedExchange (140) (140) ReusedExchange [Reuses operator id: 123] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#108] -Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#77 +Subquery:3 Hosting operator id = 126 Hosting Expression = cs_sold_date_sk#115 IN dynamicpruning#107 -Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#77 +Subquery:4 Hosting operator id = 131 Hosting Expression = ws_sold_date_sk#121 IN dynamicpruning#107 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (141) (141) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#49] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 ReusedExchange (142) (142) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#14] -Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 -Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#34, [id=#35] +Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquery#51, [id=#52] * Project (146) +- * Filter (145) +- * ColumnarToRow (144) @@ -815,33 +815,33 @@ Subquery:9 Hosting operator id = 70 Hosting Expression = Subquery scalar-subquer (143) Scan parquet default.date_dim -Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] (145) Filter [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 1999)) AND (d_moy#96 = 12)) AND (d_dom#97 = 16)) +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] +Condition : (((((isnotnull(d_year#133) AND isnotnull(d_moy#134)) AND isnotnull(d_dom#135)) AND (d_year#133 = 1999)) AND (d_moy#134 = 12)) AND (d_dom#135 = 16)) (146) Project [codegen id : 1] -Output [1]: [d_week_seq#33] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [1]: [d_week_seq#132] +Input [4]: [d_week_seq#132, d_year#133, d_moy#134, d_dom#135] -Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#51, [id=#52] +Subquery:10 Hosting operator id = 112 Hosting Expression = ReusedSubquery Subquery scalar-subquery#72, [id=#73] -Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#54 +Subquery:11 Hosting operator id = 91 Hosting Expression = ss_sold_date_sk#78 IN dynamicpruning#79 ReusedExchange (147) (147) ReusedExchange [Reuses operator id: 103] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#81] -Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#56, [id=#57] +Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subquery#83, [id=#84] * Project (151) +- * Filter (150) +- * ColumnarToRow (149) @@ -849,21 +849,21 @@ Subquery:12 Hosting operator id = 101 Hosting Expression = Subquery scalar-subqu (148) Scan parquet default.date_dim -Output [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (149) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] (150) Filter [codegen id : 1] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#96)) AND isnotnull(d_dom#97)) AND (d_year#13 = 1998)) AND (d_moy#96 = 12)) AND (d_dom#97 = 16)) +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] +Condition : (((((isnotnull(d_year#137) AND isnotnull(d_moy#138)) AND isnotnull(d_dom#139)) AND (d_year#137 = 1998)) AND (d_moy#138 = 12)) AND (d_dom#139 = 16)) (151) Project [codegen id : 1] -Output [1]: [d_week_seq#33] -Input [4]: [d_week_seq#33, d_year#13, d_moy#96, d_dom#97] +Output [1]: [d_week_seq#136] +Input [4]: [d_week_seq#136, d_year#137, d_moy#138, d_dom#139] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt index 1d5907e319658..1b41abbc97b06 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_sales,channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] - WholeStageCodegen (90) + WholeStageCodegen (86) BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] @@ -49,7 +49,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #1 - WholeStageCodegen (44) + WholeStageCodegen (42) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -68,11 +68,11 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #3 - WholeStageCodegen (21) + WholeStageCodegen (20) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #4 - WholeStageCodegen (20) + WholeStageCodegen (19) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -81,84 +81,82 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #5 - WholeStageCodegen (19) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #6 - WholeStageCodegen (18) + WholeStageCodegen (17) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (13) + WholeStageCodegen (12) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #7 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (11) + WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #9 InputAdapter - Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] + BroadcastExchange #9 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #10 + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #11 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #12 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #9 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (7) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #12 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #10 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (17) + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (16) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (16) + WholeStageCodegen (15) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -169,12 +167,12 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #10 + ReusedExchange [d_date_sk] #9 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 InputAdapter BroadcastExchange #3 - WholeStageCodegen (22) + WholeStageCodegen (21) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #3 @@ -190,29 +188,29 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ InputAdapter BroadcastExchange #15 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #16 - WholeStageCodegen (23) + WholeStageCodegen (22) Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #20 - WholeStageCodegen (89) + WholeStageCodegen (85) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #4 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #21 - WholeStageCodegen (88) + WholeStageCodegen (84) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -220,24 +218,24 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ BroadcastHashJoin [ss_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ss_item_sk,ss_item_sk] - WholeStageCodegen (46) + WholeStageCodegen (44) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #22 - WholeStageCodegen (45) + WholeStageCodegen (43) Filter [ss_item_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #6 ReusedExchange [d_date_sk] #23 - WholeStageCodegen (65) + WholeStageCodegen (62) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #17 InputAdapter BroadcastExchange #23 - WholeStageCodegen (66) + WholeStageCodegen (63) Project [d_date_sk] Filter [d_week_seq,d_date_sk] Subquery #7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 258224e574cfb..bdafb17d69d45 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -27,36 +27,36 @@ TakeOrderedAndProject (100) : : : :- * HashAggregate (39) : : : : +- Exchange (38) : : : : +- * HashAggregate (37) - : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : :- * Project (22) - : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : :- * Project (15) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : :- * Filter (9) - : : : : : : : +- * ColumnarToRow (8) - : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : +- BroadcastExchange (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (20) - : : : : : +- * Project (19) - : : : : : +- * Filter (18) - : : : : : +- * ColumnarToRow (17) - : : : : : +- Scan parquet default.date_dim (16) - : : : : +- BroadcastExchange (35) - : : : : +- * Project (34) - : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : :- * Project (31) - : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : :- * Filter (25) - : : : : : : +- * ColumnarToRow (24) - : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : +- BroadcastExchange (29) - : : : : : +- * Filter (28) - : : : : : +- * ColumnarToRow (27) - : : : : : +- Scan parquet default.item (26) - : : : : +- ReusedExchange (32) + : : : : +- * Project (36) + : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : :- * Project (33) + : : : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : : : :- * Filter (9) + : : : : : : +- * ColumnarToRow (8) + : : : : : : +- Scan parquet default.store_sales (7) + : : : : : +- BroadcastExchange (31) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : : : : :- * Filter (12) + : : : : : : +- * ColumnarToRow (11) + : : : : : : +- Scan parquet default.item (10) + : : : : : +- BroadcastExchange (29) + : : : : : +- * Project (28) + : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * Filter (15) + : : : : : : : +- * ColumnarToRow (14) + : : : : : : : +- Scan parquet default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * Filter (18) + : : : : : : +- * ColumnarToRow (17) + : : : : : : +- Scan parquet default.item (16) + : : : : : +- BroadcastExchange (26) + : : : : : +- * Project (25) + : : : : : +- * Filter (24) + : : : : : +- * ColumnarToRow (23) + : : : : : +- Scan parquet default.date_dim (22) + : : : : +- ReusedExchange (34) : : : +- BroadcastExchange (49) : : : +- * Project (48) : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -131,443 +131,443 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_item_sk#10) (10) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(11) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(12) Filter [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +(12) Filter [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(13) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +(13) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join condition: None +(14) ColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(15) Filter [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Condition : isnotnull(cs_item_sk#17) -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +(16) Scan parquet default.item +Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] 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 - -(17) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(18) Filter [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +(17) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(19) Project [codegen id : 2] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +(18) Filter [codegen id : 1] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Condition : isnotnull(i_item_sk#19) -(20) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +(19) BroadcastExchange +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#19] Join condition: None -(22) Project [codegen id : 6] -Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] -Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(23) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(22) Scan parquet default.date_dim +Output [2]: [d_date_sk#24, d_year#25] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +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 -(24) ColumnarToRow [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(23) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#24, d_year#25] -(25) Filter [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +(24) Filter [codegen id : 2] +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1998)) AND (d_year#25 <= 2000)) AND isnotnull(d_date_sk#24)) -(26) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +(25) Project [codegen id : 2] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_year#25] -(27) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(26) BroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(28) Filter [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +(27) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] +Join condition: None + +(28) Project [codegen id : 3] +Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] (29) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] -(30) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +(30) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join condition: None -(31) Project [codegen id : 5] -Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(32) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(31) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +(32) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] Join condition: None -(34) Project [codegen id : 5] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(33) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(35) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] +(34) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#29] -(36) BroadcastHashJoin [codegen id : 6] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#29] Join condition: None +(36) Project [codegen id : 6] +Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] + (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (38) Exchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_item_sk#34) -(43) ReusedExchange [Reuses operator id: 29] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(43) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#36] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] +Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -(46) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(46) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#40] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#40] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] (49) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (53) BroadcastExchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#15, class_id#16, category_id#17] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#27] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] +Output [1]: [i_item_sk#6 AS ss_item_sk#43] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] (56) BroadcastExchange -Input [1]: [ss_item_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [ss_item_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#43] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Condition : (((isnotnull(i_item_sk#45) AND isnotnull(i_brand_id#46)) AND isnotnull(i_class_id#47)) AND isnotnull(i_category_id#48)) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [i_item_sk#45] +Right keys [1]: [ss_item_sk#43] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#45] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (66) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#30] +Output [2]: [d_date_sk#50, d_week_seq#51] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 24] -Input [2]: [d_date_sk#12, d_week_seq#30] +Input [2]: [d_date_sk#50, d_week_seq#51] (68) Filter [codegen id : 24] -Input [2]: [d_date_sk#12, d_week_seq#30] -Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#31, [id=#32])) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#50, d_week_seq#51] +Condition : ((isnotnull(d_week_seq#51) AND (d_week_seq#51 = Subquery scalar-subquery#52, [id=#53])) AND isnotnull(d_date_sk#50)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#30] +Output [1]: [d_date_sk#50] +Input [2]: [d_date_sk#50, d_week_seq#51] (70) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#33] +Input [1]: [d_date_sk#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#50] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#34, isEmpty#35, count#36] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] +Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] +Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#61] (75) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#37, isEmpty#38, count#39] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#58, isEmpty#59, count#60] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41, count(1)#42] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sales#43, count(1)#42 AS number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#41 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] (76) Filter [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45 as decimal(32,6)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) (77) Project [codegen id : 52] -Output [6]: [store AS channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#45] +Output [6]: [store AS channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66] (78) Scan parquet default.store_sales -Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#49)] +PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 50] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] (80) Filter [codegen id : 50] -Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [4]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73] +Condition : isnotnull(ss_item_sk#70) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (82) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [ss_item_sk#70] +Right keys [1]: [ss_item_sk#43] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] +Output [4]: [i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] (84) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#50] +Left keys [1]: [ss_item_sk#70] +Right keys [1]: [i_item_sk#75] Join condition: None (85) Project [codegen id : 50] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#50, i_brand_id#51, i_class_id#52, i_category_id#53] +Output [6]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78] +Input [8]: [ss_item_sk#70, ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_item_sk#75, i_brand_id#76, i_class_id#77, i_category_id#78] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_week_seq#30] +Output [2]: [d_date_sk#79, d_week_seq#80] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 49] -Input [2]: [d_date_sk#12, d_week_seq#30] +Input [2]: [d_date_sk#79, d_week_seq#80] (88) Filter [codegen id : 49] -Input [2]: [d_date_sk#12, d_week_seq#30] -Condition : ((isnotnull(d_week_seq#30) AND (d_week_seq#30 = Subquery scalar-subquery#54, [id=#55])) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#79, d_week_seq#80] +Condition : ((isnotnull(d_week_seq#80) AND (d_week_seq#80 = Subquery scalar-subquery#81, [id=#82])) AND isnotnull(d_date_sk#79)) (89) Project [codegen id : 49] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_week_seq#30] +Output [1]: [d_date_sk#79] +Input [2]: [d_date_sk#79, d_week_seq#80] (90) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +Input [1]: [d_date_sk#79] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#83] (91) BroadcastHashJoin [codegen id : 50] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#73] +Right keys [1]: [d_date_sk#79] Join condition: None (92) Project [codegen id : 50] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#51, i_class_id#52, i_category_id#53, d_date_sk#12] +Output [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] +Input [7]: [ss_quantity#71, ss_list_price#72, ss_sold_date_sk#73, i_brand_id#76, i_class_id#77, i_category_id#78, d_date_sk#79] (93) HashAggregate [codegen id : 50] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#51, i_class_id#52, i_category_id#53] -Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#57, isEmpty#58, count#59] -Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] +Input [5]: [ss_quantity#71, ss_list_price#72, i_brand_id#76, i_class_id#77, i_category_id#78] +Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] +Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] (94) Exchange -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] -Arguments: hashpartitioning(i_brand_id#51, i_class_id#52, i_category_id#53, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] +Arguments: hashpartitioning(i_brand_id#76, i_class_id#77, i_category_id#78, 5), ENSURE_REQUIREMENTS, [id=#90] (95) HashAggregate [codegen id : 51] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum#60, isEmpty#61, count#62] -Keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64, count(1)#65] -Results [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sales#66, count(1)#65 AS number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#64 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum#87, isEmpty#88, count#89] +Keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] +Results [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] (96) Filter [codegen id : 51] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#46, [id=#47] as decimal(32,6)))) +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#67, [id=#68] as decimal(32,6)))) (97) Project [codegen id : 51] -Output [6]: [store AS channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Input [6]: [i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#68] +Output [6]: [store AS channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Input [6]: [i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#72 as decimal(12,2)))), DecimalType(18,2), true))#95] (98) BroadcastExchange -Input [6]: [channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#70] +Input [6]: [channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [id=#97] (99) BroadcastHashJoin [codegen id : 52] -Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [i_brand_id#51, i_class_id#52, i_category_id#53] +Left keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] +Right keys [3]: [i_brand_id#76, i_class_id#77, i_category_id#78] Join condition: None (100) TakeOrderedAndProject -Input [12]: [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] -Arguments: 100, [i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#48, i_brand_id#7, i_class_id#8, i_category_id#9, sales#43, number_sales#44, channel#69, i_brand_id#51, i_class_id#52, i_category_id#53, sales#66, number_sales#67] +Input [12]: [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] +Arguments: 100, [i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#69, i_brand_id#46, i_class_id#47, i_category_id#48, sales#64, number_sales#65, channel#96, i_brand_id#76, i_class_id#77, i_category_id#78, sales#93, number_sales#94] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#46, [id=#47] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#67, [id=#68] * HashAggregate (123) +- Exchange (122) +- * HashAggregate (121) @@ -594,140 +594,140 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(ss_sold_date_sk#100), dynamicpruningexpression(ss_sold_date_sk#100 IN dynamicpruning#101)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100] (103) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#102, d_year#103] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#102, d_year#103] (105) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#102, d_year#103] +Condition : (((isnotnull(d_year#103) AND (d_year#103 >= 1998)) AND (d_year#103 <= 2000)) AND isnotnull(d_date_sk#102)) (106) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#102] +Input [2]: [d_date_sk#102, d_year#103] (107) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#72] +Input [1]: [d_date_sk#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#104] (108) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#100] +Right keys [1]: [d_date_sk#102] Join condition: None (109) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#73, ss_list_price#3 AS list_price#74] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#98 AS quantity#105, ss_list_price#99 AS list_price#106] +Input [4]: [ss_quantity#98, ss_list_price#99, ss_sold_date_sk#100, d_date_sk#102] (110) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] +Output [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(cs_sold_date_sk#109), dynamicpruningexpression(cs_sold_date_sk#109 IN dynamicpruning#101)] ReadSchema: struct (111) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19] +Input [3]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109] (112) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#110] (113) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#109] +Right keys [1]: [d_date_sk#110] Join condition: None (114) Project [codegen id : 4] -Output [2]: [cs_quantity#75 AS quantity#77, cs_list_price#76 AS list_price#78] -Input [4]: [cs_quantity#75, cs_list_price#76, cs_sold_date_sk#19, d_date_sk#12] +Output [2]: [cs_quantity#107 AS quantity#111, cs_list_price#108 AS list_price#112] +Input [4]: [cs_quantity#107, cs_list_price#108, cs_sold_date_sk#109, d_date_sk#110] (115) Scan parquet default.web_sales -Output [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] +Output [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(ws_sold_date_sk#115), dynamicpruningexpression(ws_sold_date_sk#115 IN dynamicpruning#101)] ReadSchema: struct (116) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24] +Input [3]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115] (117) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#116] (118) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#115] +Right keys [1]: [d_date_sk#116] Join condition: None (119) Project [codegen id : 6] -Output [2]: [ws_quantity#79 AS quantity#81, ws_list_price#80 AS list_price#82] -Input [4]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#24, d_date_sk#12] +Output [2]: [ws_quantity#113 AS quantity#117, ws_list_price#114 AS list_price#118] +Input [4]: [ws_quantity#113, ws_list_price#114, ws_sold_date_sk#115, d_date_sk#116] (120) Union (121) HashAggregate [codegen id : 7] -Input [2]: [quantity#73, list_price#74] +Input [2]: [quantity#105, list_price#106] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#83, count#84] -Results [2]: [sum#85, count#86] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#119, count#120] +Results [2]: [sum#121, count#122] (122) Exchange -Input [2]: [sum#85, count#86] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#87] +Input [2]: [sum#121, count#122] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#123] (123) HashAggregate [codegen id : 8] -Input [2]: [sum#85, count#86] +Input [2]: [sum#121, count#122] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#73 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#74 as decimal(12,2)))), DecimalType(18,2), true))#88 AS average_sales#89] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#105 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#106 as decimal(12,2)))), DecimalType(18,2), true))#124 AS average_sales#125] -Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#71 +Subquery:2 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#100 IN dynamicpruning#101 ReusedExchange (124) (124) ReusedExchange [Reuses operator id: 107] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#102] -Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#71 +Subquery:3 Hosting operator id = 110 Hosting Expression = cs_sold_date_sk#109 IN dynamicpruning#101 -Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#71 +Subquery:4 Hosting operator id = 115 Hosting Expression = ws_sold_date_sk#115 IN dynamicpruning#101 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#50] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 ReusedExchange (126) -(126) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(126) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#29] -Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#31, [id=#32] +Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquery#52, [id=#53] * Project (130) +- * Filter (129) +- * ColumnarToRow (128) @@ -735,33 +735,33 @@ Subquery:9 Hosting operator id = 68 Hosting Expression = Subquery scalar-subquer (127) Scan parquet default.date_dim -Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (128) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] (129) Filter [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 1999)) AND (d_moy#90 = 12)) AND (d_dom#91 = 16)) +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] +Condition : (((((isnotnull(d_year#127) AND isnotnull(d_moy#128)) AND isnotnull(d_dom#129)) AND (d_year#127 = 1999)) AND (d_moy#128 = 12)) AND (d_dom#129 = 16)) (130) Project [codegen id : 1] -Output [1]: [d_week_seq#30] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [1]: [d_week_seq#126] +Input [4]: [d_week_seq#126, d_year#127, d_moy#128, d_dom#129] -Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#46, [id=#47] +Subquery:10 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#67, [id=#68] -Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#49 +Subquery:11 Hosting operator id = 78 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 ReusedExchange (131) (131) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#79] -Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subquery#81, [id=#82] * Project (135) +- * Filter (134) +- * ColumnarToRow (133) @@ -769,21 +769,21 @@ Subquery:12 Hosting operator id = 88 Hosting Expression = Subquery scalar-subque (132) Scan parquet default.date_dim -Output [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] ReadSchema: struct (133) ColumnarToRow [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] (134) Filter [codegen id : 1] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] -Condition : (((((isnotnull(d_year#13) AND isnotnull(d_moy#90)) AND isnotnull(d_dom#91)) AND (d_year#13 = 1998)) AND (d_moy#90 = 12)) AND (d_dom#91 = 16)) +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] +Condition : (((((isnotnull(d_year#131) AND isnotnull(d_moy#132)) AND isnotnull(d_dom#133)) AND (d_year#131 = 1998)) AND (d_moy#132 = 12)) AND (d_dom#133 = 16)) (135) Project [codegen id : 1] -Output [1]: [d_week_seq#30] -Input [4]: [d_week_seq#30, d_year#13, d_moy#90, d_dom#91] +Output [1]: [d_week_seq#130] +Input [4]: [d_week_seq#130, d_year#131, d_moy#132, d_dom#133] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt index 3507d76bcf9b9..cb9b410ea8568 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt @@ -82,53 +82,53 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_ Exchange [brand_id,class_id,category_id] #5 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #6 - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #9 + BroadcastExchange #8 WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #6 + BroadcastExchange #6 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #6 InputAdapter BroadcastExchange #10 WholeStageCodegen (9) 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 9b47400d3b780..43ebe6e0b4d9b 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 @@ -40,39 +40,39 @@ TakeOrderedAndProject (213) : : : : : +- * HashAggregate (44) : : : : : +- Exchange (43) : : : : : +- * HashAggregate (42) - : : : : : +- SortMergeJoin LeftSemi (41) - : : : : : :- * Sort (26) - : : : : : : +- Exchange (25) - : : : : : : +- * Project (24) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) - : : : : : : :- * Project (18) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) - : : : : : : : :- * Filter (11) - : : : : : : : : +- * ColumnarToRow (10) - : : : : : : : : +- Scan parquet default.store_sales (9) - : : : : : : : +- BroadcastExchange (16) - : : : : : : : +- * Project (15) - : : : : : : : +- * Filter (14) - : : : : : : : +- * ColumnarToRow (13) - : : : : : : : +- Scan parquet default.date_dim (12) - : : : : : : +- BroadcastExchange (22) - : : : : : : +- * Filter (21) - : : : : : : +- * ColumnarToRow (20) - : : : : : : +- Scan parquet default.item (19) - : : : : : +- * Sort (40) - : : : : : +- Exchange (39) - : : : : : +- * Project (38) - : : : : : +- * BroadcastHashJoin Inner BuildRight (37) - : : : : : :- * Project (32) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (31) - : : : : : : :- * Filter (29) - : : : : : : : +- * ColumnarToRow (28) - : : : : : : : +- Scan parquet default.catalog_sales (27) - : : : : : : +- ReusedExchange (30) - : : : : : +- BroadcastExchange (36) - : : : : : +- * Filter (35) - : : : : : +- * ColumnarToRow (34) - : : : : : +- Scan parquet default.item (33) + : : : : : +- * Project (41) + : : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : : :- * Project (18) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : : :- * Filter (11) + : : : : : : : +- * ColumnarToRow (10) + : : : : : : : +- Scan parquet default.store_sales (9) + : : : : : : +- BroadcastExchange (16) + : : : : : : +- * Project (15) + : : : : : : +- * Filter (14) + : : : : : : +- * ColumnarToRow (13) + : : : : : : +- Scan parquet default.date_dim (12) + : : : : : +- BroadcastExchange (39) + : : : : : +- SortMergeJoin LeftSemi (38) + : : : : : :- * Sort (23) + : : : : : : +- Exchange (22) + : : : : : : +- * Filter (21) + : : : : : : +- * ColumnarToRow (20) + : : : : : : +- Scan parquet default.item (19) + : : : : : +- * Sort (37) + : : : : : +- Exchange (36) + : : : : : +- * Project (35) + : : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : : :- * Project (29) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : : :- * Filter (26) + : : : : : : : +- * ColumnarToRow (25) + : : : : : : : +- Scan parquet default.catalog_sales (24) + : : : : : : +- ReusedExchange (27) + : : : : : +- BroadcastExchange (33) + : : : : : +- * Filter (32) + : : : : : +- * ColumnarToRow (31) + : : : : : +- Scan parquet default.item (30) : : : : +- * Sort (57) : : : : +- Exchange (56) : : : : +- * Project (55) @@ -244,972 +244,972 @@ Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(7) ColumnarToRow [codegen id : 20] +(7) ColumnarToRow [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -(8) Filter [codegen id : 20] +(8) Filter [codegen id : 19] Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] Condition : ((isnotnull(i_brand_id#8) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) (9) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(10) ColumnarToRow [codegen id : 5] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +(10) ColumnarToRow [codegen id : 10] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] -(11) Filter [codegen id : 5] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +(11) Filter [codegen id : 10] +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_item_sk#11) (12) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#14, d_year#15] 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 (13) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#14, d_year#15] (14) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#14, d_year#15] +Condition : (((isnotnull(d_year#15) AND (d_year#15 >= 1999)) AND (d_year#15 <= 2001)) AND isnotnull(d_date_sk#14)) (15) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#14] +Input [2]: [d_date_sk#14, d_year#15] (16) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(17) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(17) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#14] Join condition: None -(18) Project [codegen id : 5] -Output [1]: [ss_item_sk#1] -Input [3]: [ss_item_sk#1, ss_sold_date_sk#4, d_date_sk#12] +(18) Project [codegen id : 10] +Output [1]: [ss_item_sk#11] +Input [3]: [ss_item_sk#11, ss_sold_date_sk#12, d_date_sk#14] (19) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct (20) ColumnarToRow [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] (21) Filter [codegen id : 4] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : (((isnotnull(i_item_sk#7) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Condition : (((isnotnull(i_item_sk#17) AND isnotnull(i_brand_id#18)) AND isnotnull(i_class_id#19)) AND isnotnull(i_category_id#20)) -(22) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] - -(23) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] -Join condition: None +(22) Exchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: hashpartitioning(coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20), 5), ENSURE_REQUIREMENTS, [id=#21] -(24) Project [codegen id : 5] -Output [3]: [i_brand_id#8 AS brand_id#16, i_class_id#9 AS class_id#17, i_category_id#10 AS category_id#18] -Input [5]: [ss_item_sk#1, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(23) Sort [codegen id : 5] +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: [coalesce(i_brand_id#18, 0) ASC NULLS FIRST, isnull(i_brand_id#18) ASC NULLS FIRST, coalesce(i_class_id#19, 0) ASC NULLS FIRST, isnull(i_class_id#19) ASC NULLS FIRST, coalesce(i_category_id#20, 0) ASC NULLS FIRST, isnull(i_category_id#20) ASC NULLS FIRST], false, 0 -(25) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#19] - -(26) Sort [codegen id : 6] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 - -(27) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#20, cs_sold_date_sk#21] +(24) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#22, cs_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(cs_sold_date_sk#23), dynamicpruningexpression(cs_sold_date_sk#23 IN dynamicpruning#13)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(28) ColumnarToRow [codegen id : 9] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] +(25) ColumnarToRow [codegen id : 8] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] -(29) Filter [codegen id : 9] -Input [2]: [cs_item_sk#20, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +(26) Filter [codegen id : 8] +Input [2]: [cs_item_sk#22, cs_sold_date_sk#23] +Condition : isnotnull(cs_item_sk#22) -(30) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +(27) ReusedExchange [Reuses operator id: 16] +Output [1]: [d_date_sk#24] -(31) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +(28) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None -(32) Project [codegen id : 9] -Output [1]: [cs_item_sk#20] -Input [3]: [cs_item_sk#20, cs_sold_date_sk#21, d_date_sk#12] +(29) Project [codegen id : 8] +Output [1]: [cs_item_sk#22] +Input [3]: [cs_item_sk#22, cs_sold_date_sk#23, d_date_sk#24] -(33) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(30) Scan parquet default.item +Output [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(31) ColumnarToRow [codegen id : 7] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -(35) Filter [codegen id : 8] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +(32) Filter [codegen id : 7] +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Condition : isnotnull(i_item_sk#25) -(36) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +(33) BroadcastExchange +Input [4]: [i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] -(37) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_item_sk#22] +Right keys [1]: [i_item_sk#25] Join condition: None -(38) Project [codegen id : 9] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [cs_item_sk#20, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(35) Project [codegen id : 8] +Output [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Input [5]: [cs_item_sk#22, i_item_sk#25, i_brand_id#26, i_class_id#27, i_category_id#28] -(39) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#23] +(36) Exchange +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: hashpartitioning(coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28), 5), ENSURE_REQUIREMENTS, [id=#30] -(40) Sort [codegen id : 10] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(37) Sort [codegen id : 9] +Input [3]: [i_brand_id#26, i_class_id#27, i_category_id#28] +Arguments: [coalesce(i_brand_id#26, 0) ASC NULLS FIRST, isnull(i_brand_id#26) ASC NULLS FIRST, coalesce(i_class_id#27, 0) ASC NULLS FIRST, isnull(i_class_id#27) ASC NULLS FIRST, coalesce(i_category_id#28, 0) ASC NULLS FIRST, isnull(i_category_id#28) ASC NULLS FIRST], false, 0 -(41) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +(38) SortMergeJoin +Left keys [6]: [coalesce(i_brand_id#18, 0), isnull(i_brand_id#18), coalesce(i_class_id#19, 0), isnull(i_class_id#19), coalesce(i_category_id#20, 0), isnull(i_category_id#20)] +Right keys [6]: [coalesce(i_brand_id#26, 0), isnull(i_brand_id#26), coalesce(i_class_id#27, 0), isnull(i_class_id#27), coalesce(i_category_id#28, 0), isnull(i_category_id#28)] Join condition: None -(42) HashAggregate [codegen id : 11] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(39) BroadcastExchange +Input [4]: [i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] + +(40) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_item_sk#11] +Right keys [1]: [i_item_sk#17] +Join condition: None + +(41) Project [codegen id : 10] +Output [3]: [i_brand_id#18 AS brand_id#32, i_class_id#19 AS class_id#33, i_category_id#20 AS category_id#34] +Input [5]: [ss_item_sk#11, i_item_sk#17, i_brand_id#18, i_class_id#19, i_category_id#20] + +(42) HashAggregate [codegen id : 10] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (43) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(44) HashAggregate [codegen id : 12] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(44) HashAggregate [codegen id : 11] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (45) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18), 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34), 5), ENSURE_REQUIREMENTS, [id=#36] -(46) Sort [codegen id : 13] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: [coalesce(brand_id#16, 0) ASC NULLS FIRST, isnull(brand_id#16) ASC NULLS FIRST, coalesce(class_id#17, 0) ASC NULLS FIRST, isnull(class_id#17) ASC NULLS FIRST, coalesce(category_id#18, 0) ASC NULLS FIRST, isnull(category_id#18) ASC NULLS FIRST], false, 0 +(46) Sort [codegen id : 12] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: [coalesce(brand_id#32, 0) ASC NULLS FIRST, isnull(brand_id#32) ASC NULLS FIRST, coalesce(class_id#33, 0) ASC NULLS FIRST, isnull(class_id#33) ASC NULLS FIRST, coalesce(category_id#34, 0) ASC NULLS FIRST, isnull(category_id#34) ASC NULLS FIRST], false, 0 (47) Scan parquet default.web_sales -Output [2]: [ws_item_sk#26, ws_sold_date_sk#27] +Output [2]: [ws_item_sk#37, ws_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#11)] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(48) ColumnarToRow [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] +(48) ColumnarToRow [codegen id : 15] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] -(49) Filter [codegen id : 16] -Input [2]: [ws_item_sk#26, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(49) Filter [codegen id : 15] +Input [2]: [ws_item_sk#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_item_sk#37) (50) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#39] -(51) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(51) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#39] Join condition: None -(52) Project [codegen id : 16] -Output [1]: [ws_item_sk#26] -Input [3]: [ws_item_sk#26, ws_sold_date_sk#27, d_date_sk#12] +(52) Project [codegen id : 15] +Output [1]: [ws_item_sk#37] +Input [3]: [ws_item_sk#37, ws_sold_date_sk#38, d_date_sk#39] -(53) ReusedExchange [Reuses operator id: 36] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(53) ReusedExchange [Reuses operator id: 33] +Output [4]: [i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] -(54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(54) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#40] Join condition: None -(55) Project [codegen id : 16] -Output [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Input [5]: [ws_item_sk#26, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(55) Project [codegen id : 15] +Output [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Input [5]: [ws_item_sk#37, i_item_sk#40, i_brand_id#41, i_class_id#42, i_category_id#43] (56) Exchange -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10), 5), ENSURE_REQUIREMENTS, [id=#28] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: hashpartitioning(coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43), 5), ENSURE_REQUIREMENTS, [id=#44] -(57) Sort [codegen id : 17] -Input [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [coalesce(i_brand_id#8, 0) ASC NULLS FIRST, isnull(i_brand_id#8) ASC NULLS FIRST, coalesce(i_class_id#9, 0) ASC NULLS FIRST, isnull(i_class_id#9) ASC NULLS FIRST, coalesce(i_category_id#10, 0) ASC NULLS FIRST, isnull(i_category_id#10) ASC NULLS FIRST], false, 0 +(57) Sort [codegen id : 16] +Input [3]: [i_brand_id#41, i_class_id#42, i_category_id#43] +Arguments: [coalesce(i_brand_id#41, 0) ASC NULLS FIRST, isnull(i_brand_id#41) ASC NULLS FIRST, coalesce(i_class_id#42, 0) ASC NULLS FIRST, isnull(i_class_id#42) ASC NULLS FIRST, coalesce(i_category_id#43, 0) ASC NULLS FIRST, isnull(i_category_id#43) ASC NULLS FIRST], false, 0 (58) SortMergeJoin -Left keys [6]: [coalesce(brand_id#16, 0), isnull(brand_id#16), coalesce(class_id#17, 0), isnull(class_id#17), coalesce(category_id#18, 0), isnull(category_id#18)] -Right keys [6]: [coalesce(i_brand_id#8, 0), isnull(i_brand_id#8), coalesce(i_class_id#9, 0), isnull(i_class_id#9), coalesce(i_category_id#10, 0), isnull(i_category_id#10)] +Left keys [6]: [coalesce(brand_id#32, 0), isnull(brand_id#32), coalesce(class_id#33, 0), isnull(class_id#33), coalesce(category_id#34, 0), isnull(category_id#34)] +Right keys [6]: [coalesce(i_brand_id#41, 0), isnull(i_brand_id#41), coalesce(i_class_id#42, 0), isnull(i_class_id#42), coalesce(i_category_id#43, 0), isnull(i_category_id#43)] Join condition: None -(59) HashAggregate [codegen id : 18] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(59) HashAggregate [codegen id : 17] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (60) Exchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: hashpartitioning(brand_id#16, class_id#17, category_id#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: hashpartitioning(brand_id#32, class_id#33, category_id#34, 5), ENSURE_REQUIREMENTS, [id=#45] -(61) HashAggregate [codegen id : 19] -Input [3]: [brand_id#16, class_id#17, category_id#18] -Keys [3]: [brand_id#16, class_id#17, category_id#18] +(61) HashAggregate [codegen id : 18] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Keys [3]: [brand_id#32, class_id#33, category_id#34] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#16, class_id#17, category_id#18] +Results [3]: [brand_id#32, class_id#33, category_id#34] (62) BroadcastExchange -Input [3]: [brand_id#16, class_id#17, category_id#18] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#30] +Input [3]: [brand_id#32, class_id#33, category_id#34] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#46] -(63) BroadcastHashJoin [codegen id : 20] +(63) BroadcastHashJoin [codegen id : 19] Left keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Right keys [3]: [brand_id#16, class_id#17, category_id#18] +Right keys [3]: [brand_id#32, class_id#33, category_id#34] Join condition: None -(64) Project [codegen id : 20] -Output [1]: [i_item_sk#7 AS ss_item_sk#31] -Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#16, class_id#17, category_id#18] +(64) Project [codegen id : 19] +Output [1]: [i_item_sk#7 AS ss_item_sk#47] +Input [7]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, brand_id#32, class_id#33, category_id#34] (65) Exchange -Input [1]: [ss_item_sk#31] -Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [1]: [ss_item_sk#47] +Arguments: hashpartitioning(ss_item_sk#47, 5), ENSURE_REQUIREMENTS, [id=#48] -(66) Sort [codegen id : 21] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(66) Sort [codegen id : 20] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (67) SortMergeJoin Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#31] +Right keys [1]: [ss_item_sk#47] Join condition: None (68) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#33] +Output [3]: [d_date_sk#49, d_year#50, d_moy#51] 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 -(69) ColumnarToRow [codegen id : 22] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] +(69) ColumnarToRow [codegen id : 21] +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] -(70) Filter [codegen id : 22] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#33)) AND (d_year#13 = 2000)) AND (d_moy#33 = 11)) AND isnotnull(d_date_sk#12)) +(70) Filter [codegen id : 21] +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] +Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2000)) AND (d_moy#51 = 11)) AND isnotnull(d_date_sk#49)) -(71) Project [codegen id : 22] -Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#33] +(71) Project [codegen id : 21] +Output [1]: [d_date_sk#49] +Input [3]: [d_date_sk#49, d_year#50, d_moy#51] (72) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] +Input [1]: [d_date_sk#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#52] -(73) BroadcastHashJoin [codegen id : 44] +(73) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#49] Join condition: None -(74) Project [codegen id : 44] +(74) Project [codegen id : 42] Output [3]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3] -Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Input [5]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#49] (75) Scan parquet default.item -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(76) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(76) ColumnarToRow [codegen id : 22] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(77) Filter [codegen id : 23] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Condition : isnotnull(i_item_sk#7) +(77) Filter [codegen id : 22] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Condition : isnotnull(i_item_sk#53) (78) Exchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: hashpartitioning(i_item_sk#7, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: hashpartitioning(i_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#57] -(79) Sort [codegen id : 24] -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: [i_item_sk#7 ASC NULLS FIRST], false, 0 +(79) Sort [codegen id : 23] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: [i_item_sk#53 ASC NULLS FIRST], false, 0 (80) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(81) Sort [codegen id : 43] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(81) Sort [codegen id : 41] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (82) SortMergeJoin -Left keys [1]: [i_item_sk#7] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [i_item_sk#53] +Right keys [1]: [ss_item_sk#47] Join condition: None (83) BroadcastExchange -Input [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] +Input [4]: [i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] -(84) BroadcastHashJoin [codegen id : 44] +(84) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#53] Join condition: None -(85) Project [codegen id : 44] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(85) Project [codegen id : 42] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] +Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#53, i_brand_id#54, i_class_id#55, i_category_id#56] -(86) HashAggregate [codegen id : 44] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +(86) HashAggregate [codegen id : 42] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#54, i_class_id#55, i_category_id#56] +Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#37, isEmpty#38, count#39] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] +Aggregate Attributes [3]: [sum#59, isEmpty#60, count#61] +Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] (87) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] +Arguments: hashpartitioning(i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#65] -(88) HashAggregate [codegen id : 45] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#40, isEmpty#41, count#42] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] +(88) HashAggregate [codegen id : 43] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum#62, isEmpty#63, count#64] +Keys [3]: [i_brand_id#54, i_class_id#55, i_category_id#56] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44, count(1)#45] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sales#46, count(1)#45 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#44 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66, count(1)#67] +Results [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sales#68, count(1)#67 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#66 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] -(89) Filter [codegen id : 45] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48 as decimal(32,6)) > cast(Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(89) Filter [codegen id : 43] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(32,6)) > cast(Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(90) Project [codegen id : 45] -Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#48] +(90) Project [codegen id : 43] +Output [6]: [store AS channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69] +Input [6]: [i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#70] (91) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Output [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#77), dynamicpruningexpression(cs_sold_date_sk#77 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(92) ColumnarToRow [codegen id : 46] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +(92) ColumnarToRow [codegen id : 44] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] -(93) Filter [codegen id : 46] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -Condition : isnotnull(cs_item_sk#20) +(93) Filter [codegen id : 44] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Condition : isnotnull(cs_item_sk#74) (94) Exchange -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -Arguments: hashpartitioning(cs_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Arguments: hashpartitioning(cs_item_sk#74, 5), ENSURE_REQUIREMENTS, [id=#78] -(95) Sort [codegen id : 47] -Input [4]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] -Arguments: [cs_item_sk#20 ASC NULLS FIRST], false, 0 +(95) Sort [codegen id : 45] +Input [4]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77] +Arguments: [cs_item_sk#74 ASC NULLS FIRST], false, 0 (96) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(97) Sort [codegen id : 66] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(97) Sort [codegen id : 63] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [cs_item_sk#74] +Right keys [1]: [ss_item_sk#47] Join condition: None (99) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#79] -(100) BroadcastHashJoin [codegen id : 89] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +(100) BroadcastHashJoin [codegen id : 85] +Left keys [1]: [cs_sold_date_sk#77] +Right keys [1]: [d_date_sk#79] Join condition: None -(101) Project [codegen id : 89] -Output [3]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53] -Input [5]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] +(101) Project [codegen id : 85] +Output [3]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76] +Input [5]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, cs_sold_date_sk#77, d_date_sk#79] (102) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -(103) BroadcastHashJoin [codegen id : 89] -Left keys [1]: [cs_item_sk#20] -Right keys [1]: [i_item_sk#7] +(103) BroadcastHashJoin [codegen id : 85] +Left keys [1]: [cs_item_sk#74] +Right keys [1]: [i_item_sk#80] Join condition: None -(104) Project [codegen id : 89] -Output [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [cs_item_sk#20, cs_quantity#52, cs_list_price#53, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(104) Project [codegen id : 85] +Output [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] +Input [7]: [cs_item_sk#74, cs_quantity#75, cs_list_price#76, i_item_sk#80, i_brand_id#81, i_class_id#82, i_category_id#83] -(105) HashAggregate [codegen id : 89] -Input [5]: [cs_quantity#52, cs_list_price#53, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#55, isEmpty#56, count#57] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] +(105) HashAggregate [codegen id : 85] +Input [5]: [cs_quantity#75, cs_list_price#76, i_brand_id#81, i_class_id#82, i_category_id#83] +Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#84, isEmpty#85, count#86] +Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] (106) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +Arguments: hashpartitioning(i_brand_id#81, i_class_id#82, i_category_id#83, 5), ENSURE_REQUIREMENTS, [id=#90] -(107) HashAggregate [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#58, isEmpty#59, count#60] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62, count(1)#63] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sales#64, count(1)#63 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] +(107) HashAggregate [codegen id : 86] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum#87, isEmpty#88, count#89] +Keys [3]: [i_brand_id#81, i_class_id#82, i_category_id#83] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] +Results [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#93, count(1)#92 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] -(108) Filter [codegen id : 90] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(108) Filter [codegen id : 86] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(109) Project [codegen id : 90] -Output [6]: [catalog AS channel#67, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#66] +(109) Project [codegen id : 86] +Output [6]: [catalog AS channel#96, i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94] +Input [6]: [i_brand_id#81, i_class_id#82, i_category_id#83, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#76 as decimal(12,2)))), DecimalType(18,2), true))#95] (110) Scan parquet default.web_sales -Output [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Output [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#100), dynamicpruningexpression(ws_sold_date_sk#100 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(111) ColumnarToRow [codegen id : 91] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +(111) ColumnarToRow [codegen id : 87] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] -(112) Filter [codegen id : 91] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Condition : isnotnull(ws_item_sk#26) +(112) Filter [codegen id : 87] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Condition : isnotnull(ws_item_sk#97) (113) Exchange -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Arguments: hashpartitioning(ws_item_sk#26, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Arguments: hashpartitioning(ws_item_sk#97, 5), ENSURE_REQUIREMENTS, [id=#101] -(114) Sort [codegen id : 92] -Input [4]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] -Arguments: [ws_item_sk#26 ASC NULLS FIRST], false, 0 +(114) Sort [codegen id : 88] +Input [4]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100] +Arguments: [ws_item_sk#97 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#31] +Output [1]: [ss_item_sk#47] -(116) Sort [codegen id : 111] -Input [1]: [ss_item_sk#31] -Arguments: [ss_item_sk#31 ASC NULLS FIRST], false, 0 +(116) Sort [codegen id : 106] +Input [1]: [ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST], false, 0 (117) SortMergeJoin -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [ss_item_sk#31] +Left keys [1]: [ws_item_sk#97] +Right keys [1]: [ss_item_sk#47] Join condition: None (118) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#102] -(119) BroadcastHashJoin [codegen id : 134] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +(119) BroadcastHashJoin [codegen id : 128] +Left keys [1]: [ws_sold_date_sk#100] +Right keys [1]: [d_date_sk#102] Join condition: None -(120) Project [codegen id : 134] -Output [3]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69] -Input [5]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] +(120) Project [codegen id : 128] +Output [3]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99] +Input [5]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, ws_sold_date_sk#100, d_date_sk#102] (121) ReusedExchange [Reuses operator id: 83] -Output [4]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +Output [4]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] -(122) BroadcastHashJoin [codegen id : 134] -Left keys [1]: [ws_item_sk#26] -Right keys [1]: [i_item_sk#7] +(122) BroadcastHashJoin [codegen id : 128] +Left keys [1]: [ws_item_sk#97] +Right keys [1]: [i_item_sk#103] Join condition: None -(123) Project [codegen id : 134] -Output [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] -Input [7]: [ws_item_sk#26, ws_quantity#68, ws_list_price#69, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10] +(123) Project [codegen id : 128] +Output [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] +Input [7]: [ws_item_sk#97, ws_quantity#98, ws_list_price#99, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106] -(124) HashAggregate [codegen id : 134] -Input [5]: [ws_quantity#68, ws_list_price#69, i_brand_id#8, i_class_id#9, i_category_id#10] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#71, isEmpty#72, count#73] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] +(124) HashAggregate [codegen id : 128] +Input [5]: [ws_quantity#98, ws_list_price#99, i_brand_id#104, i_class_id#105, i_category_id#106] +Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#107, isEmpty#108, count#109] +Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] (125) Exchange -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#77] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +Arguments: hashpartitioning(i_brand_id#104, i_class_id#105, i_category_id#106, 5), ENSURE_REQUIREMENTS, [id=#113] -(126) HashAggregate [codegen id : 135] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#74, isEmpty#75, count#76] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78, count(1)#79] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sales#80, count(1)#79 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#78 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +(126) HashAggregate [codegen id : 129] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum#110, isEmpty#111, count#112] +Keys [3]: [i_brand_id#104, i_class_id#105, i_category_id#106] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114, count(1)#115] +Results [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sales#116, count(1)#115 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#114 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] -(127) Filter [codegen id : 135] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(127) Filter [codegen id : 129] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(128) Project [codegen id : 135] -Output [6]: [web AS channel#83, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#82] +(128) Project [codegen id : 129] +Output [6]: [web AS channel#119, i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117] +Input [6]: [i_brand_id#104, i_class_id#105, i_category_id#106, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#98 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#99 as decimal(12,2)))), DecimalType(18,2), true))#118] (129) Union -(130) HashAggregate [codegen id : 136] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#84, isEmpty#85, sum#86] -Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#87, isEmpty#88, sum#89] +(130) HashAggregate [codegen id : 130] +Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sales#68, number_sales#69] +Keys [4]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#120, isEmpty#121, sum#122] +Results [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] (131) Exchange -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#87, isEmpty#88, sum#89] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] +Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, 5), ENSURE_REQUIREMENTS, [id=#126] -(132) HashAggregate [codegen id : 137] -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#87, isEmpty#88, sum#89] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#91, sum(number_sales#47)#92] -Results [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum(sales#46)#91 AS sum_sales#93, sum(number_sales#47)#92 AS number_sales#94] +(132) HashAggregate [codegen id : 131] +Input [7]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum#123, isEmpty#124, sum#125] +Keys [4]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#127, sum(number_sales#69)#128] +Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum(sales#68)#127 AS sum_sales#129, sum(number_sales#69)#128 AS number_sales#130] (133) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#95, isEmpty#96, count#97] +Output [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, count#136] -(134) HashAggregate [codegen id : 182] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#95, isEmpty#96, count#97] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#98, count(1)#99] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#98 AS sales#46, count(1)#99 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#98 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100] +(134) HashAggregate [codegen id : 174] +Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum#134, isEmpty#135, count#136] +Keys [3]: [i_brand_id#131, i_class_id#132, i_category_id#133] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139, count(1)#140] +Results [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sales#68, count(1)#140 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] -(135) Filter [codegen id : 182] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(135) Filter [codegen id : 174] +Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(136) Project [codegen id : 182] -Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#100] +(136) Project [codegen id : 174] +Output [6]: [store AS channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69] +Input [6]: [i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#137 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#138 as decimal(12,2)))), DecimalType(18,2), true))#141] (137) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#101, isEmpty#102, count#103] +Output [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum#145, isEmpty#146, count#147] -(138) HashAggregate [codegen id : 227] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#101, isEmpty#102, count#103] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#104, count(1)#105] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sales#64, count(1)#105 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106] +(138) HashAggregate [codegen id : 217] +Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum#145, isEmpty#146, count#147] +Keys [3]: [i_brand_id#142, i_class_id#143, i_category_id#144] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150, count(1)#151] +Results [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sales#93, count(1)#151 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#150 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] -(139) Filter [codegen id : 227] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(139) Filter [codegen id : 217] +Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(140) Project [codegen id : 227] -Output [6]: [catalog AS channel#107, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#106] +(140) Project [codegen id : 217] +Output [6]: [catalog AS channel#96, i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94] +Input [6]: [i_brand_id#142, i_class_id#143, i_category_id#144, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#148 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#149 as decimal(12,2)))), DecimalType(18,2), true))#152] (141) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#108, isEmpty#109, count#110] +Output [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum#156, isEmpty#157, count#158] -(142) HashAggregate [codegen id : 272] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#108, isEmpty#109, count#110] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#111, count(1)#112] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#111 AS sales#80, count(1)#112 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#111 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113] +(142) HashAggregate [codegen id : 260] +Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum#156, isEmpty#157, count#158] +Keys [3]: [i_brand_id#153, i_class_id#154, i_category_id#155] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161, count(1)#162] +Results [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161 AS sales#116, count(1)#162 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#161 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] -(143) Filter [codegen id : 272] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(143) Filter [codegen id : 260] +Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(144) Project [codegen id : 272] -Output [6]: [web AS channel#114, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#113] +(144) Project [codegen id : 260] +Output [6]: [web AS channel#119, i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117] +Input [6]: [i_brand_id#153, i_class_id#154, i_category_id#155, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#159 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#160 as decimal(12,2)))), DecimalType(18,2), true))#163] (145) Union -(146) HashAggregate [codegen id : 273] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#115, isEmpty#116, sum#117] -Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#118, isEmpty#119, sum#120] +(146) HashAggregate [codegen id : 261] +Input [6]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sales#68, number_sales#69] +Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#164, isEmpty#165, sum#166] +Results [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] (147) Exchange -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#118, isEmpty#119, sum#120] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#121] - -(148) HashAggregate [codegen id : 274] -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#118, isEmpty#119, sum#120] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#122, sum(number_sales#47)#123] -Results [5]: [channel#51, i_brand_id#8, i_class_id#9, sum(sales#46)#122 AS sum_sales#93, sum(number_sales#47)#123 AS number_sales#94] - -(149) HashAggregate [codegen id : 274] -Input [5]: [channel#51, i_brand_id#8, i_class_id#9, sum_sales#93, number_sales#94] -Keys [3]: [channel#51, i_brand_id#8, i_class_id#9] -Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] -Aggregate Attributes [3]: [sum#124, isEmpty#125, sum#126] -Results [6]: [channel#51, i_brand_id#8, i_class_id#9, sum#127, isEmpty#128, sum#129] +Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] +Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, 5), ENSURE_REQUIREMENTS, [id=#170] + +(148) HashAggregate [codegen id : 262] +Input [7]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133, sum#167, isEmpty#168, sum#169] +Keys [4]: [channel#73, i_brand_id#131, i_class_id#132, i_category_id#133] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#171, sum(number_sales#69)#172] +Results [5]: [channel#73, i_brand_id#131, i_class_id#132, sum(sales#68)#171 AS sum_sales#129, sum(number_sales#69)#172 AS number_sales#130] + +(149) HashAggregate [codegen id : 262] +Input [5]: [channel#73, i_brand_id#131, i_class_id#132, sum_sales#129, number_sales#130] +Keys [3]: [channel#73, i_brand_id#131, i_class_id#132] +Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] +Aggregate Attributes [3]: [sum#173, isEmpty#174, sum#175] +Results [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] (150) Exchange -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, sum#127, isEmpty#128, sum#129] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, 5), ENSURE_REQUIREMENTS, [id=#130] +Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] +Arguments: hashpartitioning(channel#73, i_brand_id#131, i_class_id#132, 5), ENSURE_REQUIREMENTS, [id=#179] -(151) HashAggregate [codegen id : 275] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, sum#127, isEmpty#128, sum#129] -Keys [3]: [channel#51, i_brand_id#8, i_class_id#9] -Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] -Aggregate Attributes [2]: [sum(sum_sales#93)#131, sum(number_sales#94)#132] -Results [6]: [channel#51, i_brand_id#8, i_class_id#9, null AS i_category_id#133, sum(sum_sales#93)#131 AS sum(sum_sales)#134, sum(number_sales#94)#132 AS sum(number_sales)#135] +(151) HashAggregate [codegen id : 263] +Input [6]: [channel#73, i_brand_id#131, i_class_id#132, sum#176, isEmpty#177, sum#178] +Keys [3]: [channel#73, i_brand_id#131, i_class_id#132] +Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] +Aggregate Attributes [2]: [sum(sum_sales#129)#180, sum(number_sales#130)#181] +Results [6]: [channel#73, i_brand_id#131, i_class_id#132, null AS i_category_id#182, sum(sum_sales#129)#180 AS sum(sum_sales)#183, sum(number_sales#130)#181 AS sum(number_sales)#184] (152) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#136, isEmpty#137, count#138] +Output [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum#188, isEmpty#189, count#190] -(153) HashAggregate [codegen id : 320] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#136, isEmpty#137, count#138] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#139, count(1)#140] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sales#46, count(1)#140 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#139 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141] +(153) HashAggregate [codegen id : 306] +Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum#188, isEmpty#189, count#190] +Keys [3]: [i_brand_id#185, i_class_id#186, i_category_id#187] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193, count(1)#194] +Results [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193 AS sales#68, count(1)#194 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#193 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] -(154) Filter [codegen id : 320] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(154) Filter [codegen id : 306] +Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(155) Project [codegen id : 320] -Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#141] +(155) Project [codegen id : 306] +Output [6]: [store AS channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69] +Input [6]: [i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#191 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#192 as decimal(12,2)))), DecimalType(18,2), true))#195] (156) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#142, isEmpty#143, count#144] +Output [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum#199, isEmpty#200, count#201] -(157) HashAggregate [codegen id : 365] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#142, isEmpty#143, count#144] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#145, count(1)#146] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sales#64, count(1)#146 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147] +(157) HashAggregate [codegen id : 349] +Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum#199, isEmpty#200, count#201] +Keys [3]: [i_brand_id#196, i_class_id#197, i_category_id#198] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204, count(1)#205] +Results [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204 AS sales#93, count(1)#205 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#204 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] -(158) Filter [codegen id : 365] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(158) Filter [codegen id : 349] +Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(159) Project [codegen id : 365] -Output [6]: [catalog AS channel#148, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#147] +(159) Project [codegen id : 349] +Output [6]: [catalog AS channel#96, i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94] +Input [6]: [i_brand_id#196, i_class_id#197, i_category_id#198, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#202 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#203 as decimal(12,2)))), DecimalType(18,2), true))#206] (160) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#149, isEmpty#150, count#151] +Output [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum#210, isEmpty#211, count#212] -(161) HashAggregate [codegen id : 410] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#149, isEmpty#150, count#151] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#152, count(1)#153] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#152 AS sales#80, count(1)#153 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#152 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154] +(161) HashAggregate [codegen id : 392] +Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum#210, isEmpty#211, count#212] +Keys [3]: [i_brand_id#207, i_class_id#208, i_category_id#209] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215, count(1)#216] +Results [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sales#116, count(1)#216 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#215 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] -(162) Filter [codegen id : 410] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(162) Filter [codegen id : 392] +Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(163) Project [codegen id : 410] -Output [6]: [web AS channel#155, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#154] +(163) Project [codegen id : 392] +Output [6]: [web AS channel#119, i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117] +Input [6]: [i_brand_id#207, i_class_id#208, i_category_id#209, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#213 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#214 as decimal(12,2)))), DecimalType(18,2), true))#217] (164) Union -(165) HashAggregate [codegen id : 411] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#156, isEmpty#157, sum#158] -Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#159, isEmpty#160, sum#161] +(165) HashAggregate [codegen id : 393] +Input [6]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sales#68, number_sales#69] +Keys [4]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#218, isEmpty#219, sum#220] +Results [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] (166) Exchange -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#159, isEmpty#160, sum#161] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#162] - -(167) HashAggregate [codegen id : 412] -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#159, isEmpty#160, sum#161] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#163, sum(number_sales#47)#164] -Results [4]: [channel#51, i_brand_id#8, sum(sales#46)#163 AS sum_sales#93, sum(number_sales#47)#164 AS number_sales#94] - -(168) HashAggregate [codegen id : 412] -Input [4]: [channel#51, i_brand_id#8, sum_sales#93, number_sales#94] -Keys [2]: [channel#51, i_brand_id#8] -Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] -Aggregate Attributes [3]: [sum#165, isEmpty#166, sum#167] -Results [5]: [channel#51, i_brand_id#8, sum#168, isEmpty#169, sum#170] +Input [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] +Arguments: hashpartitioning(channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, 5), ENSURE_REQUIREMENTS, [id=#224] + +(167) HashAggregate [codegen id : 394] +Input [7]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187, sum#221, isEmpty#222, sum#223] +Keys [4]: [channel#73, i_brand_id#185, i_class_id#186, i_category_id#187] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#225, sum(number_sales#69)#226] +Results [4]: [channel#73, i_brand_id#185, sum(sales#68)#225 AS sum_sales#129, sum(number_sales#69)#226 AS number_sales#130] + +(168) HashAggregate [codegen id : 394] +Input [4]: [channel#73, i_brand_id#185, sum_sales#129, number_sales#130] +Keys [2]: [channel#73, i_brand_id#185] +Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] +Aggregate Attributes [3]: [sum#227, isEmpty#228, sum#229] +Results [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] (169) Exchange -Input [5]: [channel#51, i_brand_id#8, sum#168, isEmpty#169, sum#170] -Arguments: hashpartitioning(channel#51, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [id=#171] +Input [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] +Arguments: hashpartitioning(channel#73, i_brand_id#185, 5), ENSURE_REQUIREMENTS, [id=#233] -(170) HashAggregate [codegen id : 413] -Input [5]: [channel#51, i_brand_id#8, sum#168, isEmpty#169, sum#170] -Keys [2]: [channel#51, i_brand_id#8] -Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] -Aggregate Attributes [2]: [sum(sum_sales#93)#172, sum(number_sales#94)#173] -Results [6]: [channel#51, i_brand_id#8, null AS i_class_id#174, null AS i_category_id#175, sum(sum_sales#93)#172 AS sum(sum_sales)#176, sum(number_sales#94)#173 AS sum(number_sales)#177] +(170) HashAggregate [codegen id : 395] +Input [5]: [channel#73, i_brand_id#185, sum#230, isEmpty#231, sum#232] +Keys [2]: [channel#73, i_brand_id#185] +Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] +Aggregate Attributes [2]: [sum(sum_sales#129)#234, sum(number_sales#130)#235] +Results [6]: [channel#73, i_brand_id#185, null AS i_class_id#236, null AS i_category_id#237, sum(sum_sales#129)#234 AS sum(sum_sales)#238, sum(number_sales#130)#235 AS sum(number_sales)#239] (171) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#178, isEmpty#179, count#180] +Output [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum#243, isEmpty#244, count#245] -(172) HashAggregate [codegen id : 458] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#178, isEmpty#179, count#180] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#181, count(1)#182] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sales#46, count(1)#182 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#181 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183] +(172) HashAggregate [codegen id : 438] +Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum#243, isEmpty#244, count#245] +Keys [3]: [i_brand_id#240, i_class_id#241, i_category_id#242] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248, count(1)#249] +Results [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248 AS sales#68, count(1)#249 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#248 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] -(173) Filter [codegen id : 458] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(173) Filter [codegen id : 438] +Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(174) Project [codegen id : 458] -Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#183] +(174) Project [codegen id : 438] +Output [6]: [store AS channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69] +Input [6]: [i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#246 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#247 as decimal(12,2)))), DecimalType(18,2), true))#250] (175) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#184, isEmpty#185, count#186] +Output [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum#254, isEmpty#255, count#256] -(176) HashAggregate [codegen id : 503] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#184, isEmpty#185, count#186] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#187, count(1)#188] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sales#64, count(1)#188 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189] +(176) HashAggregate [codegen id : 481] +Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum#254, isEmpty#255, count#256] +Keys [3]: [i_brand_id#251, i_class_id#252, i_category_id#253] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259, count(1)#260] +Results [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259 AS sales#93, count(1)#260 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#259 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] -(177) Filter [codegen id : 503] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(177) Filter [codegen id : 481] +Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(178) Project [codegen id : 503] -Output [6]: [catalog AS channel#190, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#189] +(178) Project [codegen id : 481] +Output [6]: [catalog AS channel#96, i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94] +Input [6]: [i_brand_id#251, i_class_id#252, i_category_id#253, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#257 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#258 as decimal(12,2)))), DecimalType(18,2), true))#261] (179) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#191, isEmpty#192, count#193] +Output [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum#265, isEmpty#266, count#267] -(180) HashAggregate [codegen id : 548] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#191, isEmpty#192, count#193] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#194, count(1)#195] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#194 AS sales#80, count(1)#195 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#194 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196] +(180) HashAggregate [codegen id : 524] +Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum#265, isEmpty#266, count#267] +Keys [3]: [i_brand_id#262, i_class_id#263, i_category_id#264] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270, count(1)#271] +Results [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270 AS sales#116, count(1)#271 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#270 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] -(181) Filter [codegen id : 548] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(181) Filter [codegen id : 524] +Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(182) Project [codegen id : 548] -Output [6]: [web AS channel#197, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#196] +(182) Project [codegen id : 524] +Output [6]: [web AS channel#119, i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117] +Input [6]: [i_brand_id#262, i_class_id#263, i_category_id#264, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#272] (183) Union -(184) HashAggregate [codegen id : 549] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#198, isEmpty#199, sum#200] -Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#201, isEmpty#202, sum#203] +(184) HashAggregate [codegen id : 525] +Input [6]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sales#68, number_sales#69] +Keys [4]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#273, isEmpty#274, sum#275] +Results [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] (185) Exchange -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#201, isEmpty#202, sum#203] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#204] - -(186) HashAggregate [codegen id : 550] -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#201, isEmpty#202, sum#203] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#205, sum(number_sales#47)#206] -Results [3]: [channel#51, sum(sales#46)#205 AS sum_sales#93, sum(number_sales#47)#206 AS number_sales#94] - -(187) HashAggregate [codegen id : 550] -Input [3]: [channel#51, sum_sales#93, number_sales#94] -Keys [1]: [channel#51] -Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] -Aggregate Attributes [3]: [sum#207, isEmpty#208, sum#209] -Results [4]: [channel#51, sum#210, isEmpty#211, sum#212] +Input [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] +Arguments: hashpartitioning(channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, 5), ENSURE_REQUIREMENTS, [id=#279] + +(186) HashAggregate [codegen id : 526] +Input [7]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242, sum#276, isEmpty#277, sum#278] +Keys [4]: [channel#73, i_brand_id#240, i_class_id#241, i_category_id#242] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#280, sum(number_sales#69)#281] +Results [3]: [channel#73, sum(sales#68)#280 AS sum_sales#129, sum(number_sales#69)#281 AS number_sales#130] + +(187) HashAggregate [codegen id : 526] +Input [3]: [channel#73, sum_sales#129, number_sales#130] +Keys [1]: [channel#73] +Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] +Aggregate Attributes [3]: [sum#282, isEmpty#283, sum#284] +Results [4]: [channel#73, sum#285, isEmpty#286, sum#287] (188) Exchange -Input [4]: [channel#51, sum#210, isEmpty#211, sum#212] -Arguments: hashpartitioning(channel#51, 5), ENSURE_REQUIREMENTS, [id=#213] +Input [4]: [channel#73, sum#285, isEmpty#286, sum#287] +Arguments: hashpartitioning(channel#73, 5), ENSURE_REQUIREMENTS, [id=#288] -(189) HashAggregate [codegen id : 551] -Input [4]: [channel#51, sum#210, isEmpty#211, sum#212] -Keys [1]: [channel#51] -Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] -Aggregate Attributes [2]: [sum(sum_sales#93)#214, sum(number_sales#94)#215] -Results [6]: [channel#51, null AS i_brand_id#216, null AS i_class_id#217, null AS i_category_id#218, sum(sum_sales#93)#214 AS sum(sum_sales)#219, sum(number_sales#94)#215 AS sum(number_sales)#220] +(189) HashAggregate [codegen id : 527] +Input [4]: [channel#73, sum#285, isEmpty#286, sum#287] +Keys [1]: [channel#73] +Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] +Aggregate Attributes [2]: [sum(sum_sales#129)#289, sum(number_sales#130)#290] +Results [6]: [channel#73, null AS i_brand_id#291, null AS i_class_id#292, null AS i_category_id#293, sum(sum_sales#129)#289 AS sum(sum_sales)#294, sum(number_sales#130)#290 AS sum(number_sales)#295] (190) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#221, isEmpty#222, count#223] +Output [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum#299, isEmpty#300, count#301] -(191) HashAggregate [codegen id : 596] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#221, isEmpty#222, count#223] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#224, count(1)#225] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sales#46, count(1)#225 AS number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#224 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226] +(191) HashAggregate [codegen id : 570] +Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum#299, isEmpty#300, count#301] +Keys [3]: [i_brand_id#296, i_class_id#297, i_category_id#298] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304, count(1)#305] +Results [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304 AS sales#68, count(1)#305 AS number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#304 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] -(192) Filter [codegen id : 596] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(192) Filter [codegen id : 570] +Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(193) Project [codegen id : 596] -Output [6]: [store AS channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#226] +(193) Project [codegen id : 570] +Output [6]: [store AS channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69] +Input [6]: [i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#302 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#303 as decimal(12,2)))), DecimalType(18,2), true))#306] (194) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#227, isEmpty#228, count#229] +Output [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum#310, isEmpty#311, count#312] -(195) HashAggregate [codegen id : 641] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#227, isEmpty#228, count#229] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#230, count(1)#231] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sales#64, count(1)#231 AS number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232] +(195) HashAggregate [codegen id : 613] +Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum#310, isEmpty#311, count#312] +Keys [3]: [i_brand_id#307, i_class_id#308, i_category_id#309] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315, count(1)#316] +Results [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315 AS sales#93, count(1)#316 AS number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#315 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] -(196) Filter [codegen id : 641] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(196) Filter [codegen id : 613] +Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(197) Project [codegen id : 641] -Output [6]: [catalog AS channel#233, i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#64, number_sales#65, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#52 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#53 as decimal(12,2)))), DecimalType(18,2), true))#232] +(197) Project [codegen id : 613] +Output [6]: [catalog AS channel#96, i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94] +Input [6]: [i_brand_id#307, i_class_id#308, i_category_id#309, sales#93, number_sales#94, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#313 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#314 as decimal(12,2)))), DecimalType(18,2), true))#317] (198) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#234, isEmpty#235, count#236] +Output [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum#321, isEmpty#322, count#323] -(199) HashAggregate [codegen id : 686] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum#234, isEmpty#235, count#236] -Keys [3]: [i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#237, count(1)#238] -Results [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#237 AS sales#80, count(1)#238 AS number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#237 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239] +(199) HashAggregate [codegen id : 656] +Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum#321, isEmpty#322, count#323] +Keys [3]: [i_brand_id#318, i_class_id#319, i_category_id#320] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326, count(1)#327] +Results [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326 AS sales#116, count(1)#327 AS number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#326 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] -(200) Filter [codegen id : 686] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#49, [id=#50] as decimal(32,6)))) +(200) Filter [codegen id : 656] +Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#71, [id=#72] as decimal(32,6)))) -(201) Project [codegen id : 686] -Output [6]: [web AS channel#240, i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81] -Input [6]: [i_brand_id#8, i_class_id#9, i_category_id#10, sales#80, number_sales#81, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#69 as decimal(12,2)))), DecimalType(18,2), true))#239] +(201) Project [codegen id : 656] +Output [6]: [web AS channel#119, i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117] +Input [6]: [i_brand_id#318, i_class_id#319, i_category_id#320, sales#116, number_sales#117, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#324 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#325 as decimal(12,2)))), DecimalType(18,2), true))#328] (202) Union -(203) HashAggregate [codegen id : 687] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sales#46, number_sales#47] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [partial_sum(sales#46), partial_sum(number_sales#47)] -Aggregate Attributes [3]: [sum#241, isEmpty#242, sum#243] -Results [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#244, isEmpty#245, sum#246] +(203) HashAggregate [codegen id : 657] +Input [6]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sales#68, number_sales#69] +Keys [4]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298] +Functions [2]: [partial_sum(sales#68), partial_sum(number_sales#69)] +Aggregate Attributes [3]: [sum#329, isEmpty#330, sum#331] +Results [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] (204) Exchange -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#244, isEmpty#245, sum#246] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, 5), ENSURE_REQUIREMENTS, [id=#247] - -(205) HashAggregate [codegen id : 688] -Input [7]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum#244, isEmpty#245, sum#246] -Keys [4]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10] -Functions [2]: [sum(sales#46), sum(number_sales#47)] -Aggregate Attributes [2]: [sum(sales#46)#248, sum(number_sales#47)#249] -Results [2]: [sum(sales#46)#248 AS sum_sales#93, sum(number_sales#47)#249 AS number_sales#94] - -(206) HashAggregate [codegen id : 688] -Input [2]: [sum_sales#93, number_sales#94] +Input [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] +Arguments: hashpartitioning(channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, 5), ENSURE_REQUIREMENTS, [id=#335] + +(205) HashAggregate [codegen id : 658] +Input [7]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298, sum#332, isEmpty#333, sum#334] +Keys [4]: [channel#73, i_brand_id#296, i_class_id#297, i_category_id#298] +Functions [2]: [sum(sales#68), sum(number_sales#69)] +Aggregate Attributes [2]: [sum(sales#68)#336, sum(number_sales#69)#337] +Results [2]: [sum(sales#68)#336 AS sum_sales#129, sum(number_sales#69)#337 AS number_sales#130] + +(206) HashAggregate [codegen id : 658] +Input [2]: [sum_sales#129, number_sales#130] Keys: [] -Functions [2]: [partial_sum(sum_sales#93), partial_sum(number_sales#94)] -Aggregate Attributes [3]: [sum#250, isEmpty#251, sum#252] -Results [3]: [sum#253, isEmpty#254, sum#255] +Functions [2]: [partial_sum(sum_sales#129), partial_sum(number_sales#130)] +Aggregate Attributes [3]: [sum#338, isEmpty#339, sum#340] +Results [3]: [sum#341, isEmpty#342, sum#343] (207) Exchange -Input [3]: [sum#253, isEmpty#254, sum#255] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#256] +Input [3]: [sum#341, isEmpty#342, sum#343] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#344] -(208) HashAggregate [codegen id : 689] -Input [3]: [sum#253, isEmpty#254, sum#255] +(208) HashAggregate [codegen id : 659] +Input [3]: [sum#341, isEmpty#342, sum#343] Keys: [] -Functions [2]: [sum(sum_sales#93), sum(number_sales#94)] -Aggregate Attributes [2]: [sum(sum_sales#93)#257, sum(number_sales#94)#258] -Results [6]: [null AS channel#259, null AS i_brand_id#260, null AS i_class_id#261, null AS i_category_id#262, sum(sum_sales#93)#257 AS sum(sum_sales)#263, sum(number_sales#94)#258 AS sum(number_sales)#264] +Functions [2]: [sum(sum_sales#129), sum(number_sales#130)] +Aggregate Attributes [2]: [sum(sum_sales#129)#345, sum(number_sales#130)#346] +Results [6]: [null AS channel#347, null AS i_brand_id#348, null AS i_class_id#349, null AS i_category_id#350, sum(sum_sales#129)#345 AS sum(sum_sales)#351, sum(number_sales#130)#346 AS sum(number_sales)#352] (209) Union -(210) HashAggregate [codegen id : 690] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] -Keys [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +(210) HashAggregate [codegen id : 660] +Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] (211) Exchange -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] -Arguments: hashpartitioning(channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94, 5), ENSURE_REQUIREMENTS, [id=#265] +Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Arguments: hashpartitioning(channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130, 5), ENSURE_REQUIREMENTS, [id=#353] -(212) HashAggregate [codegen id : 691] -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] -Keys [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +(212) HashAggregate [codegen id : 661] +Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Keys [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Results [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] (213) TakeOrderedAndProject -Input [6]: [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] -Arguments: 100, [channel#51 ASC NULLS FIRST, i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST], [channel#51, i_brand_id#8, i_class_id#9, i_category_id#10, sum_sales#93, number_sales#94] +Input [6]: [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] +Arguments: 100, [channel#73 ASC NULLS FIRST, i_brand_id#54 ASC NULLS FIRST, i_class_id#55 ASC NULLS FIRST, i_category_id#56 ASC NULLS FIRST], [channel#73, i_brand_id#54, i_class_id#55, i_category_id#56, sum_sales#129, number_sales#130] ===== Subqueries ===== -Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#49, [id=#50] +Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquery#71, [id=#72] * HashAggregate (240) +- Exchange (239) +- * HashAggregate (238) @@ -1240,193 +1240,193 @@ Subquery:1 Hosting operator id = 89 Hosting Expression = Subquery scalar-subquer (214) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#266)] +PartitionFilters: [isnotnull(ss_sold_date_sk#356), dynamicpruningexpression(ss_sold_date_sk#356 IN dynamicpruning#357)] ReadSchema: struct (215) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356] (216) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#358, d_year#359] 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 (217) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#358, d_year#359] (218) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#358, d_year#359] +Condition : (((isnotnull(d_year#359) AND (d_year#359 >= 1999)) AND (d_year#359 <= 2001)) AND isnotnull(d_date_sk#358)) (219) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#358] +Input [2]: [d_date_sk#358, d_year#359] (220) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#267] +Input [1]: [d_date_sk#358] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#360] (221) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#356] +Right keys [1]: [d_date_sk#358] Join condition: None (222) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#268, ss_list_price#3 AS list_price#269] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#354 AS quantity#361, ss_list_price#355 AS list_price#362] +Input [4]: [ss_quantity#354, ss_list_price#355, ss_sold_date_sk#356, d_date_sk#358] (223) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Output [3]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#270)] +PartitionFilters: [isnotnull(cs_sold_date_sk#365), dynamicpruningexpression(cs_sold_date_sk#365 IN dynamicpruning#366)] ReadSchema: struct (224) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21] +Input [3]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365] (225) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#367, d_year#368] 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 (226) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#367, d_year#368] (227) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#367, d_year#368] +Condition : (((isnotnull(d_year#368) AND (d_year#368 >= 1998)) AND (d_year#368 <= 2000)) AND isnotnull(d_date_sk#367)) (228) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#367] +Input [2]: [d_date_sk#367, d_year#368] (229) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#271] +Input [1]: [d_date_sk#367] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#369] (230) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#365] +Right keys [1]: [d_date_sk#367] Join condition: None (231) Project [codegen id : 4] -Output [2]: [cs_quantity#52 AS quantity#272, cs_list_price#53 AS list_price#273] -Input [4]: [cs_quantity#52, cs_list_price#53, cs_sold_date_sk#21, d_date_sk#12] +Output [2]: [cs_quantity#363 AS quantity#370, cs_list_price#364 AS list_price#371] +Input [4]: [cs_quantity#363, cs_list_price#364, cs_sold_date_sk#365, d_date_sk#367] (232) Scan parquet default.web_sales -Output [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Output [3]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#270)] +PartitionFilters: [isnotnull(ws_sold_date_sk#374), dynamicpruningexpression(ws_sold_date_sk#374 IN dynamicpruning#366)] ReadSchema: struct (233) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27] +Input [3]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374] (234) ReusedExchange [Reuses operator id: 229] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#375] (235) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#27] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#374] +Right keys [1]: [d_date_sk#375] Join condition: None (236) Project [codegen id : 6] -Output [2]: [ws_quantity#68 AS quantity#274, ws_list_price#69 AS list_price#275] -Input [4]: [ws_quantity#68, ws_list_price#69, ws_sold_date_sk#27, d_date_sk#12] +Output [2]: [ws_quantity#372 AS quantity#376, ws_list_price#373 AS list_price#377] +Input [4]: [ws_quantity#372, ws_list_price#373, ws_sold_date_sk#374, d_date_sk#375] (237) Union (238) HashAggregate [codegen id : 7] -Input [2]: [quantity#268, list_price#269] +Input [2]: [quantity#361, list_price#362] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#276, count#277] -Results [2]: [sum#278, count#279] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#378, count#379] +Results [2]: [sum#380, count#381] (239) Exchange -Input [2]: [sum#278, count#279] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#280] +Input [2]: [sum#380, count#381] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#382] (240) HashAggregate [codegen id : 8] -Input [2]: [sum#278, count#279] +Input [2]: [sum#380, count#381] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#281] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#268 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#269 as decimal(12,2)))), DecimalType(18,2), true))#281 AS average_sales#282] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))#383] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#361 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#362 as decimal(12,2)))), DecimalType(18,2), true))#383 AS average_sales#384] -Subquery:2 Hosting operator id = 214 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#266 +Subquery:2 Hosting operator id = 214 Hosting Expression = ss_sold_date_sk#356 IN dynamicpruning#357 ReusedExchange (241) (241) ReusedExchange [Reuses operator id: 220] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#358] -Subquery:3 Hosting operator id = 223 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#270 +Subquery:3 Hosting operator id = 223 Hosting Expression = cs_sold_date_sk#365 IN dynamicpruning#366 ReusedExchange (242) (242) ReusedExchange [Reuses operator id: 229] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#367] -Subquery:4 Hosting operator id = 232 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#270 +Subquery:4 Hosting operator id = 232 Hosting Expression = ws_sold_date_sk#374 IN dynamicpruning#366 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (243) (243) ReusedExchange [Reuses operator id: 72] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#49] -Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#11 +Subquery:6 Hosting operator id = 9 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 ReusedExchange (244) (244) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#14] -Subquery:7 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#11 +Subquery:7 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#23 IN dynamicpruning#13 -Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#11 +Subquery:8 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 -Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:9 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 91 Hosting Expression = cs_sold_date_sk#77 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:11 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 110 Hosting Expression = ws_sold_date_sk#100 IN dynamicpruning#5 -Subquery:13 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:13 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:14 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:14 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:15 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:15 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:16 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:16 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:17 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:17 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:18 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:18 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:19 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:19 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:20 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:20 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:21 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:21 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:22 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:22 Hosting operator id = 192 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:23 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:23 Hosting operator id = 196 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] -Subquery:24 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#49, [id=#50] +Subquery:24 Hosting operator id = 200 Hosting Expression = ReusedSubquery Subquery scalar-subquery#71, [id=#72] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt index d27eb5a32b387..cb186b0ddcd3b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt @@ -1,21 +1,21 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] - WholeStageCodegen (691) + WholeStageCodegen (661) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 - WholeStageCodegen (690) + WholeStageCodegen (660) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] InputAdapter Union - WholeStageCodegen (137) + WholeStageCodegen (131) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #2 - WholeStageCodegen (136) + WholeStageCodegen (130) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (45) + WholeStageCodegen (43) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] Subquery #3 @@ -71,7 +71,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #3 - WholeStageCodegen (44) + WholeStageCodegen (42) HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ss_item_sk,i_item_sk] @@ -90,11 +90,11 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] SubqueryBroadcast [d_date_sk] #1 ReusedExchange [d_date_sk] #5 - WholeStageCodegen (21) + WholeStageCodegen (20) Sort [ss_item_sk] InputAdapter Exchange [ss_item_sk] #6 - WholeStageCodegen (20) + WholeStageCodegen (19) Project [i_item_sk] BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] Filter [i_brand_id,i_class_id,i_category_id] @@ -103,84 +103,82 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter BroadcastExchange #7 - WholeStageCodegen (19) + WholeStageCodegen (18) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #8 - WholeStageCodegen (18) + WholeStageCodegen (17) HashAggregate [brand_id,class_id,category_id] InputAdapter SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (13) + WholeStageCodegen (12) Sort [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #9 - WholeStageCodegen (12) + WholeStageCodegen (11) HashAggregate [brand_id,class_id,category_id] InputAdapter Exchange [brand_id,class_id,category_id] #10 - WholeStageCodegen (11) + WholeStageCodegen (10) HashAggregate [brand_id,class_id,category_id] - InputAdapter - SortMergeJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (6) - Sort [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #11 InputAdapter - Exchange [brand_id,class_id,category_id] #11 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [ss_item_sk] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #12 - InputAdapter - BroadcastExchange #12 - WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] + BroadcastExchange #11 + WholeStageCodegen (3) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #12 + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (5) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #13 + WholeStageCodegen (4) + Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (9) + Sort [i_brand_id,i_class_id,i_category_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (8) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #11 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (7) + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (4) - Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (10) - Sort [i_brand_id,i_class_id,i_category_id] - InputAdapter - Exchange [i_brand_id,i_class_id,i_category_id] #14 - WholeStageCodegen (9) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Project [cs_item_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - InputAdapter - ReusedExchange [d_date_sk] #12 - InputAdapter - BroadcastExchange #15 - WholeStageCodegen (8) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (17) + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + WholeStageCodegen (16) Sort [i_brand_id,i_class_id,i_category_id] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #16 - WholeStageCodegen (16) + WholeStageCodegen (15) Project [i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] Project [ws_item_sk] @@ -191,12 +189,12 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Scan parquet default.web_sales [ws_item_sk,ws_sold_date_sk] ReusedSubquery [d_date_sk] #2 InputAdapter - ReusedExchange [d_date_sk] #12 + ReusedExchange [d_date_sk] #11 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #15 InputAdapter BroadcastExchange #5 - WholeStageCodegen (22) + WholeStageCodegen (21) Project [d_date_sk] Filter [d_year,d_moy,d_date_sk] ColumnarToRow @@ -205,27 +203,27 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num InputAdapter BroadcastExchange #17 SortMergeJoin [i_item_sk,ss_item_sk] - WholeStageCodegen (24) + WholeStageCodegen (23) Sort [i_item_sk] InputAdapter Exchange [i_item_sk] #18 - WholeStageCodegen (23) + WholeStageCodegen (22) Filter [i_item_sk] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - WholeStageCodegen (43) + WholeStageCodegen (41) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 - WholeStageCodegen (90) + WholeStageCodegen (86) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #23 - WholeStageCodegen (89) + WholeStageCodegen (85) HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [cs_item_sk,i_item_sk] @@ -233,17 +231,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [cs_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [cs_item_sk,ss_item_sk] - WholeStageCodegen (47) + WholeStageCodegen (45) Sort [cs_item_sk] InputAdapter Exchange [cs_item_sk] #24 - WholeStageCodegen (46) + WholeStageCodegen (44) Filter [cs_item_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (66) + WholeStageCodegen (63) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 @@ -251,14 +249,14 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedExchange [d_date_sk] #5 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (135) + WholeStageCodegen (129) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter Exchange [i_brand_id,i_class_id,i_category_id] #25 - WholeStageCodegen (134) + WholeStageCodegen (128) HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] BroadcastHashJoin [ws_item_sk,i_item_sk] @@ -266,17 +264,17 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num BroadcastHashJoin [ws_sold_date_sk,d_date_sk] InputAdapter SortMergeJoin [ws_item_sk,ss_item_sk] - WholeStageCodegen (92) + WholeStageCodegen (88) Sort [ws_item_sk] InputAdapter Exchange [ws_item_sk] #26 - WholeStageCodegen (91) + WholeStageCodegen (87) Filter [ws_item_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] ReusedSubquery [d_date_sk] #1 - WholeStageCodegen (111) + WholeStageCodegen (106) Sort [ss_item_sk] InputAdapter ReusedExchange [ss_item_sk] #19 @@ -284,136 +282,136 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num ReusedExchange [d_date_sk] #5 InputAdapter ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #17 - WholeStageCodegen (275) + WholeStageCodegen (263) HashAggregate [channel,i_brand_id,i_class_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id] #27 - WholeStageCodegen (274) + WholeStageCodegen (262) HashAggregate [channel,i_brand_id,i_class_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #28 - WholeStageCodegen (273) + WholeStageCodegen (261) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (182) + WholeStageCodegen (174) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (227) + WholeStageCodegen (217) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (272) + WholeStageCodegen (260) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (413) + WholeStageCodegen (395) HashAggregate [channel,i_brand_id,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id] #32 - WholeStageCodegen (412) + WholeStageCodegen (394) HashAggregate [channel,i_brand_id,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #33 - WholeStageCodegen (411) + WholeStageCodegen (393) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (320) + WholeStageCodegen (306) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (365) + WholeStageCodegen (349) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (410) + WholeStageCodegen (392) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (551) + WholeStageCodegen (527) HashAggregate [channel,sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange [channel] #34 - WholeStageCodegen (550) + WholeStageCodegen (526) HashAggregate [channel,sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #35 - WholeStageCodegen (549) + WholeStageCodegen (525) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (458) + WholeStageCodegen (438) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (503) + WholeStageCodegen (481) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (548) + WholeStageCodegen (524) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #31 - WholeStageCodegen (689) + WholeStageCodegen (659) HashAggregate [sum,isEmpty,sum] [sum(sum_sales),sum(number_salesL),channel,i_brand_id,i_class_id,i_category_id,sum(sum_sales),sum(number_sales),sum,isEmpty,sum] InputAdapter Exchange #36 - WholeStageCodegen (688) + WholeStageCodegen (658) HashAggregate [sum_sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum_sales,number_sales,sum,isEmpty,sum] InputAdapter Exchange [channel,i_brand_id,i_class_id,i_category_id] #37 - WholeStageCodegen (687) + WholeStageCodegen (657) HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sales,number_sales] [sum,isEmpty,sum,sum,isEmpty,sum] InputAdapter Union - WholeStageCodegen (596) + WholeStageCodegen (570) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(ss_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #29 - WholeStageCodegen (641) + WholeStageCodegen (613) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),count(1),sales,number_sales,sum(CheckOverflow((promote_precision(cast(cast(cs_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2), true)),sum,isEmpty,count] InputAdapter ReusedExchange [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] #30 - WholeStageCodegen (686) + WholeStageCodegen (656) Project [i_brand_id,i_class_id,i_category_id,sales,number_sales] Filter [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2), true))] ReusedSubquery [average_sales] #3 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 04fe14cab2c9b..755973f79e493 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 @@ -34,36 +34,36 @@ TakeOrderedAndProject (194) : : : : :- * HashAggregate (39) : : : : : +- Exchange (38) : : : : : +- * HashAggregate (37) - : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (36) - : : : : : :- * Project (22) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (21) - : : : : : : :- * Project (15) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) - : : : : : : : :- * Filter (9) - : : : : : : : : +- * ColumnarToRow (8) - : : : : : : : : +- Scan parquet default.store_sales (7) - : : : : : : : +- BroadcastExchange (13) - : : : : : : : +- * Filter (12) - : : : : : : : +- * ColumnarToRow (11) - : : : : : : : +- Scan parquet default.item (10) - : : : : : : +- BroadcastExchange (20) - : : : : : : +- * Project (19) - : : : : : : +- * Filter (18) - : : : : : : +- * ColumnarToRow (17) - : : : : : : +- Scan parquet default.date_dim (16) - : : : : : +- BroadcastExchange (35) - : : : : : +- * Project (34) - : : : : : +- * BroadcastHashJoin Inner BuildRight (33) - : : : : : :- * Project (31) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (30) - : : : : : : :- * Filter (25) - : : : : : : : +- * ColumnarToRow (24) - : : : : : : : +- Scan parquet default.catalog_sales (23) - : : : : : : +- BroadcastExchange (29) - : : : : : : +- * Filter (28) - : : : : : : +- * ColumnarToRow (27) - : : : : : : +- Scan parquet default.item (26) - : : : : : +- ReusedExchange (32) + : : : : : +- * Project (36) + : : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : : :- * Project (33) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : : : : :- * Filter (9) + : : : : : : : +- * ColumnarToRow (8) + : : : : : : : +- Scan parquet default.store_sales (7) + : : : : : : +- BroadcastExchange (31) + : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (30) + : : : : : : :- * Filter (12) + : : : : : : : +- * ColumnarToRow (11) + : : : : : : : +- Scan parquet default.item (10) + : : : : : : +- BroadcastExchange (29) + : : : : : : +- * Project (28) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * Filter (15) + : : : : : : : : +- * ColumnarToRow (14) + : : : : : : : : +- Scan parquet default.catalog_sales (13) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * Filter (18) + : : : : : : : +- * ColumnarToRow (17) + : : : : : : : +- Scan parquet default.item (16) + : : : : : : +- BroadcastExchange (26) + : : : : : : +- * Project (25) + : : : : : : +- * Filter (24) + : : : : : : +- * ColumnarToRow (23) + : : : : : : +- Scan parquet default.date_dim (22) + : : : : : +- ReusedExchange (34) : : : : +- BroadcastExchange (49) : : : : +- * Project (48) : : : : +- * BroadcastHashJoin Inner BuildRight (47) @@ -225,896 +225,896 @@ Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) (7) Scan parquet default.store_sales -Output [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] (9) Filter [codegen id : 6] -Input [2]: [ss_item_sk#1, ss_sold_date_sk#4] -Condition : isnotnull(ss_item_sk#1) +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_item_sk#10) (10) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] ReadSchema: struct -(11) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(11) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(12) Filter [codegen id : 1] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : (((isnotnull(i_item_sk#6) AND isnotnull(i_brand_id#7)) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) +(12) Filter [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) -(13) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +(13) Scan parquet default.catalog_sales +Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct -(14) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] -Join condition: None +(14) ColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] -(15) Project [codegen id : 6] -Output [4]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ss_item_sk#1, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(15) Filter [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Condition : isnotnull(cs_item_sk#17) -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +(16) Scan parquet default.item +Output [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] 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 - -(17) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct -(18) Filter [codegen id : 2] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +(17) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(19) Project [codegen id : 2] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +(18) Filter [codegen id : 1] +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Condition : isnotnull(i_item_sk#19) -(20) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +(19) BroadcastExchange +Input [4]: [i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] -(21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#19] Join condition: None -(22) Project [codegen id : 6] -Output [3]: [i_brand_id#7 AS brand_id#15, i_class_id#8 AS class_id#16, i_category_id#9 AS category_id#17] -Input [5]: [ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#19, i_brand_id#20, i_class_id#21, i_category_id#22] -(23) Scan parquet default.catalog_sales -Output [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(22) Scan parquet default.date_dim +Output [2]: [d_date_sk#24, d_year#25] Batched: true -Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#10)] -PushedFilters: [IsNotNull(cs_item_sk)] -ReadSchema: struct +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 -(24) ColumnarToRow [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] +(23) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#24, d_year#25] -(25) Filter [codegen id : 5] -Input [2]: [cs_item_sk#18, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +(24) Filter [codegen id : 2] +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 >= 1999)) AND (d_year#25 <= 2001)) AND isnotnull(d_date_sk#24)) -(26) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +(25) Project [codegen id : 2] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_year#25] -(27) ColumnarToRow [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(26) BroadcastExchange +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] -(28) Filter [codegen id : 3] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +(27) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] +Join condition: None + +(28) Project [codegen id : 3] +Output [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Input [5]: [cs_sold_date_sk#18, i_brand_id#20, i_class_id#21, i_category_id#22, d_date_sk#24] (29) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [i_brand_id#20, i_class_id#21, i_category_id#22] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#27] -(30) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +(30) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#20, 0), isnull(i_brand_id#20), coalesce(i_class_id#21, 0), isnull(i_class_id#21), coalesce(i_category_id#22, 0), isnull(i_category_id#22)] Join condition: None -(31) Project [codegen id : 5] -Output [4]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [cs_item_sk#18, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] - -(32) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(31) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] -(33) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +(32) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] Join condition: None -(34) Project [codegen id : 5] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +(33) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] -(35) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#21] +(34) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#29] -(36) BroadcastHashJoin [codegen id : 6] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#29] Join condition: None +(36) Project [codegen id : 6] +Output [3]: [i_brand_id#14 AS brand_id#30, i_class_id#15 AS class_id#31, i_category_id#16 AS category_id#32] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#29] + (37) HashAggregate [codegen id : 6] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (38) Exchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: hashpartitioning(brand_id#15, class_id#16, category_id#17, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: hashpartitioning(brand_id#30, class_id#31, category_id#32, 5), ENSURE_REQUIREMENTS, [id=#33] (39) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (40) Scan parquet default.web_sales -Output [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Output [2]: [ws_item_sk#34, ws_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#12)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] (42) Filter [codegen id : 9] -Input [2]: [ws_item_sk#23, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [2]: [ws_item_sk#34, ws_sold_date_sk#35] +Condition : isnotnull(ws_item_sk#34) -(43) ReusedExchange [Reuses operator id: 29] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +(43) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] (44) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#34] +Right keys [1]: [i_item_sk#36] Join condition: None (45) Project [codegen id : 9] -Output [4]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [6]: [ws_item_sk#23, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39] +Input [6]: [ws_item_sk#34, ws_sold_date_sk#35, i_item_sk#36, i_brand_id#37, i_class_id#38, i_category_id#39] -(46) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(46) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#40] (47) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#35] +Right keys [1]: [d_date_sk#40] Join condition: None (48) Project [codegen id : 9] -Output [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Input [5]: [ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Input [5]: [ws_sold_date_sk#35, i_brand_id#37, i_class_id#38, i_category_id#39, d_date_sk#40] (49) BroadcastExchange -Input [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#25] +Input [3]: [i_brand_id#37, i_class_id#38, i_category_id#39] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [id=#41] (50) BroadcastHashJoin [codegen id : 10] -Left keys [6]: [coalesce(brand_id#15, 0), isnull(brand_id#15), coalesce(class_id#16, 0), isnull(class_id#16), coalesce(category_id#17, 0), isnull(category_id#17)] -Right keys [6]: [coalesce(i_brand_id#7, 0), isnull(i_brand_id#7), coalesce(i_class_id#8, 0), isnull(i_class_id#8), coalesce(i_category_id#9, 0), isnull(i_category_id#9)] +Left keys [6]: [coalesce(brand_id#30, 0), isnull(brand_id#30), coalesce(class_id#31, 0), isnull(class_id#31), coalesce(category_id#32, 0), isnull(category_id#32)] +Right keys [6]: [coalesce(i_brand_id#37, 0), isnull(i_brand_id#37), coalesce(i_class_id#38, 0), isnull(i_class_id#38), coalesce(i_category_id#39, 0), isnull(i_category_id#39)] Join condition: None (51) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (52) HashAggregate [codegen id : 10] -Input [3]: [brand_id#15, class_id#16, category_id#17] -Keys [3]: [brand_id#15, class_id#16, category_id#17] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Keys [3]: [brand_id#30, class_id#31, category_id#32] Functions: [] Aggregate Attributes: [] -Results [3]: [brand_id#15, class_id#16, category_id#17] +Results [3]: [brand_id#30, class_id#31, category_id#32] (53) BroadcastExchange -Input [3]: [brand_id#15, class_id#16, category_id#17] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#26] +Input [3]: [brand_id#30, class_id#31, category_id#32] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [id=#42] (54) BroadcastHashJoin [codegen id : 11] Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Right keys [3]: [brand_id#15, class_id#16, category_id#17] +Right keys [3]: [brand_id#30, class_id#31, category_id#32] Join condition: None (55) Project [codegen id : 11] -Output [1]: [i_item_sk#6 AS ss_item_sk#27] -Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#15, class_id#16, category_id#17] +Output [1]: [i_item_sk#6 AS ss_item_sk#43] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#30, class_id#31, category_id#32] (56) BroadcastExchange -Input [1]: [ss_item_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [ss_item_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (57) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [ss_item_sk#27] +Right keys [1]: [ss_item_sk#43] Join condition: None (58) Scan parquet default.item -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (60) Filter [codegen id : 23] -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Condition : isnotnull(i_item_sk#6) +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Condition : isnotnull(i_item_sk#45) (61) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (62) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [i_item_sk#6] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [i_item_sk#45] +Right keys [1]: [ss_item_sk#43] Join condition: None (63) BroadcastExchange -Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [4]: [i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] (64) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#45] Join condition: None (65) Project [codegen id : 25] -Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#45, i_brand_id#46, i_class_id#47, i_category_id#48] (66) Scan parquet default.date_dim -Output [3]: [d_date_sk#12, d_year#13, d_moy#30] +Output [3]: [d_date_sk#50, d_year#51, d_moy#52] 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 (67) ColumnarToRow [codegen id : 24] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] (68) Filter [codegen id : 24] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#30)) AND (d_year#13 = 2000)) AND (d_moy#30 = 11)) AND isnotnull(d_date_sk#12)) +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] +Condition : ((((isnotnull(d_year#51) AND isnotnull(d_moy#52)) AND (d_year#51 = 2000)) AND (d_moy#52 = 11)) AND isnotnull(d_date_sk#50)) (69) Project [codegen id : 24] -Output [1]: [d_date_sk#12] -Input [3]: [d_date_sk#12, d_year#13, d_moy#30] +Output [1]: [d_date_sk#50] +Input [3]: [d_date_sk#50, d_year#51, d_moy#52] (70) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [1]: [d_date_sk#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#53] (71) BroadcastHashJoin [codegen id : 25] Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Right keys [1]: [d_date_sk#50] Join condition: None (72) Project [codegen id : 25] -Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#46, i_class_id#47, i_category_id#48, d_date_sk#50] (73) HashAggregate [codegen id : 25] -Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#46, i_class_id#47, i_category_id#48] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#32, isEmpty#33, count#34] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] +Aggregate Attributes [3]: [sum#54, isEmpty#55, count#56] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] (74) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] +Arguments: hashpartitioning(i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#60] (75) HashAggregate [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#35, isEmpty#36, count#37] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum#57, isEmpty#58, count#59] +Keys [3]: [i_brand_id#46, i_class_id#47, i_category_id#48] Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39, count(1)#40] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sales#41, count(1)#40 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#39 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61, count(1)#62] +Results [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sales#63, count(1)#62 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#61 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] (76) Filter [codegen id : 26] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43 as decimal(32,6)) > cast(Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65 as decimal(32,6)) > cast(Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (77) Project [codegen id : 26] -Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#43] +Output [6]: [store AS channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64] +Input [6]: [i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#65] (78) Scan parquet default.catalog_sales -Output [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Output [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (79) ColumnarToRow [codegen id : 51] -Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] (80) Filter [codegen id : 51] -Input [4]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] -Condition : isnotnull(cs_item_sk#18) +Input [4]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72] +Condition : isnotnull(cs_item_sk#69) (81) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (82) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [cs_item_sk#69] +Right keys [1]: [ss_item_sk#43] Join condition: None (83) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] (84) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_item_sk#18] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [cs_item_sk#69] +Right keys [1]: [i_item_sk#73] Join condition: None (85) Project [codegen id : 51] -Output [6]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [cs_item_sk#18, cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [8]: [cs_item_sk#69, cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_item_sk#73, i_brand_id#74, i_class_id#75, i_category_id#76] (86) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#77] (87) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#72] +Right keys [1]: [d_date_sk#77] Join condition: None (88) Project [codegen id : 51] -Output [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] +Input [7]: [cs_quantity#70, cs_list_price#71, cs_sold_date_sk#72, i_brand_id#74, i_class_id#75, i_category_id#76, d_date_sk#77] (89) HashAggregate [codegen id : 51] -Input [5]: [cs_quantity#47, cs_list_price#48, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#49, isEmpty#50, count#51] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] +Input [5]: [cs_quantity#70, cs_list_price#71, i_brand_id#74, i_class_id#75, i_category_id#76] +Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#78, isEmpty#79, count#80] +Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] (90) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Arguments: hashpartitioning(i_brand_id#74, i_class_id#75, i_category_id#76, 5), ENSURE_REQUIREMENTS, [id=#84] (91) HashAggregate [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#52, isEmpty#53, count#54] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56, count(1)#57] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sales#58, count(1)#57 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum#81, isEmpty#82, count#83] +Keys [3]: [i_brand_id#74, i_class_id#75, i_category_id#76] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85, count(1)#86] +Results [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sales#87, count(1)#86 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#85 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] (92) Filter [codegen id : 52] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (93) Project [codegen id : 52] -Output [6]: [catalog AS channel#61, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#60] +Output [6]: [catalog AS channel#90, i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88] +Input [6]: [i_brand_id#74, i_class_id#75, i_category_id#76, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#70 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#71 as decimal(12,2)))), DecimalType(18,2), true))#89] (94) Scan parquet default.web_sales -Output [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Output [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (95) ColumnarToRow [codegen id : 77] -Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] (96) Filter [codegen id : 77] -Input [4]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] -Condition : isnotnull(ws_item_sk#23) +Input [4]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Condition : isnotnull(ws_item_sk#91) (97) ReusedExchange [Reuses operator id: unknown] -Output [1]: [ss_item_sk#27] +Output [1]: [ss_item_sk#43] (98) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [ss_item_sk#27] +Left keys [1]: [ws_item_sk#91] +Right keys [1]: [ss_item_sk#43] Join condition: None (99) ReusedExchange [Reuses operator id: 63] -Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [4]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] (100) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_item_sk#23] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ws_item_sk#91] +Right keys [1]: [i_item_sk#95] Join condition: None (101) Project [codegen id : 77] -Output [6]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [8]: [ws_item_sk#23, ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Output [6]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [8]: [ws_item_sk#91, ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98] (102) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#99] (103) BroadcastHashJoin [codegen id : 77] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#94] +Right keys [1]: [d_date_sk#99] Join condition: None (104) Project [codegen id : 77] -Output [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] -Input [7]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, i_brand_id#7, i_class_id#8, i_category_id#9, d_date_sk#12] +Output [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] +Input [7]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, i_brand_id#96, i_class_id#97, i_category_id#98, d_date_sk#99] (105) HashAggregate [codegen id : 77] -Input [5]: [ws_quantity#62, ws_list_price#63, i_brand_id#7, i_class_id#8, i_category_id#9] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] -Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] +Input [5]: [ws_quantity#92, ws_list_price#93, i_brand_id#96, i_class_id#97, i_category_id#98] +Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] +Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), partial_count(1)] +Aggregate Attributes [3]: [sum#100, isEmpty#101, count#102] +Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] (106) Exchange -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] -Arguments: hashpartitioning(i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Arguments: hashpartitioning(i_brand_id#96, i_class_id#97, i_category_id#98, 5), ENSURE_REQUIREMENTS, [id=#106] (107) HashAggregate [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#67, isEmpty#68, count#69] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71, count(1)#72] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sales#73, count(1)#72 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum#103, isEmpty#104, count#105] +Keys [3]: [i_brand_id#96, i_class_id#97, i_category_id#98] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107, count(1)#108] +Results [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sales#109, count(1)#108 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#107 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] (108) Filter [codegen id : 78] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (109) Project [codegen id : 78] -Output [6]: [web AS channel#76, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#75] +Output [6]: [web AS channel#112, i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110] +Input [6]: [i_brand_id#96, i_class_id#97, i_category_id#98, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#92 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#93 as decimal(12,2)))), DecimalType(18,2), true))#111] (110) Union (111) HashAggregate [codegen id : 79] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#77, isEmpty#78, sum#79] -Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#80, isEmpty#81, sum#82] +Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sales#63, number_sales#64] +Keys [4]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#113, isEmpty#114, sum#115] +Results [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] (112) Exchange -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#80, isEmpty#81, sum#82] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] +Arguments: hashpartitioning(channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, 5), ENSURE_REQUIREMENTS, [id=#119] (113) HashAggregate [codegen id : 80] -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#80, isEmpty#81, sum#82] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#84, sum(number_sales#42)#85] -Results [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum(sales#41)#84 AS sum_sales#86, sum(number_sales#42)#85 AS number_sales#87] +Input [7]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum#116, isEmpty#117, sum#118] +Keys [4]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#120, sum(number_sales#64)#121] +Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum(sales#63)#120 AS sum_sales#122, sum(number_sales#64)#121 AS number_sales#123] (114) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#88, isEmpty#89, count#90] +Output [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] (115) HashAggregate [codegen id : 106] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#88, isEmpty#89, count#90] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#91, count(1)#92] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sales#41, count(1)#92 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#91 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93] +Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum#127, isEmpty#128, count#129] +Keys [3]: [i_brand_id#124, i_class_id#125, i_category_id#126] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132, count(1)#133] +Results [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sales#63, count(1)#133 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] (116) Filter [codegen id : 106] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (117) Project [codegen id : 106] -Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#93] +Output [6]: [store AS channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64] +Input [6]: [i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#130 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#131 as decimal(12,2)))), DecimalType(18,2), true))#134] (118) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#94, isEmpty#95, count#96] +Output [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] (119) HashAggregate [codegen id : 132] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#94, isEmpty#95, count#96] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#97, count(1)#98] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sales#58, count(1)#98 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#97 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99] +Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum#138, isEmpty#139, count#140] +Keys [3]: [i_brand_id#135, i_class_id#136, i_category_id#137] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143, count(1)#144] +Results [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sales#87, count(1)#144 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#143 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] (120) Filter [codegen id : 132] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (121) Project [codegen id : 132] -Output [6]: [catalog AS channel#100, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#99] +Output [6]: [catalog AS channel#90, i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88] +Input [6]: [i_brand_id#135, i_class_id#136, i_category_id#137, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#141 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#142 as decimal(12,2)))), DecimalType(18,2), true))#145] (122) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#101, isEmpty#102, count#103] +Output [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum#149, isEmpty#150, count#151] (123) HashAggregate [codegen id : 158] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#101, isEmpty#102, count#103] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#104, count(1)#105] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sales#73, count(1)#105 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#104 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106] +Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum#149, isEmpty#150, count#151] +Keys [3]: [i_brand_id#146, i_class_id#147, i_category_id#148] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154, count(1)#155] +Results [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154 AS sales#109, count(1)#155 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#154 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] (124) Filter [codegen id : 158] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (125) Project [codegen id : 158] -Output [6]: [web AS channel#107, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#106] +Output [6]: [web AS channel#112, i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110] +Input [6]: [i_brand_id#146, i_class_id#147, i_category_id#148, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#152 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#153 as decimal(12,2)))), DecimalType(18,2), true))#156] (126) Union (127) HashAggregate [codegen id : 159] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#108, isEmpty#109, sum#110] -Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#111, isEmpty#112, sum#113] +Input [6]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sales#63, number_sales#64] +Keys [4]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#157, isEmpty#158, sum#159] +Results [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] (128) Exchange -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#111, isEmpty#112, sum#113] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#114] +Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] +Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, 5), ENSURE_REQUIREMENTS, [id=#163] (129) HashAggregate [codegen id : 160] -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#111, isEmpty#112, sum#113] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#115, sum(number_sales#42)#116] -Results [5]: [channel#46, i_brand_id#7, i_class_id#8, sum(sales#41)#115 AS sum_sales#86, sum(number_sales#42)#116 AS number_sales#87] +Input [7]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126, sum#160, isEmpty#161, sum#162] +Keys [4]: [channel#68, i_brand_id#124, i_class_id#125, i_category_id#126] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#164, sum(number_sales#64)#165] +Results [5]: [channel#68, i_brand_id#124, i_class_id#125, sum(sales#63)#164 AS sum_sales#122, sum(number_sales#64)#165 AS number_sales#123] (130) HashAggregate [codegen id : 160] -Input [5]: [channel#46, i_brand_id#7, i_class_id#8, sum_sales#86, number_sales#87] -Keys [3]: [channel#46, i_brand_id#7, i_class_id#8] -Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] -Aggregate Attributes [3]: [sum#117, isEmpty#118, sum#119] -Results [6]: [channel#46, i_brand_id#7, i_class_id#8, sum#120, isEmpty#121, sum#122] +Input [5]: [channel#68, i_brand_id#124, i_class_id#125, sum_sales#122, number_sales#123] +Keys [3]: [channel#68, i_brand_id#124, i_class_id#125] +Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] +Aggregate Attributes [3]: [sum#166, isEmpty#167, sum#168] +Results [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] (131) Exchange -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, sum#120, isEmpty#121, sum#122] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, 5), ENSURE_REQUIREMENTS, [id=#123] +Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] +Arguments: hashpartitioning(channel#68, i_brand_id#124, i_class_id#125, 5), ENSURE_REQUIREMENTS, [id=#172] (132) HashAggregate [codegen id : 161] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, sum#120, isEmpty#121, sum#122] -Keys [3]: [channel#46, i_brand_id#7, i_class_id#8] -Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] -Aggregate Attributes [2]: [sum(sum_sales#86)#124, sum(number_sales#87)#125] -Results [6]: [channel#46, i_brand_id#7, i_class_id#8, null AS i_category_id#126, sum(sum_sales#86)#124 AS sum(sum_sales)#127, sum(number_sales#87)#125 AS sum(number_sales)#128] +Input [6]: [channel#68, i_brand_id#124, i_class_id#125, sum#169, isEmpty#170, sum#171] +Keys [3]: [channel#68, i_brand_id#124, i_class_id#125] +Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] +Aggregate Attributes [2]: [sum(sum_sales#122)#173, sum(number_sales#123)#174] +Results [6]: [channel#68, i_brand_id#124, i_class_id#125, null AS i_category_id#175, sum(sum_sales#122)#173 AS sum(sum_sales)#176, sum(number_sales#123)#174 AS sum(number_sales)#177] (133) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#129, isEmpty#130, count#131] +Output [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] (134) HashAggregate [codegen id : 187] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#129, isEmpty#130, count#131] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#132, count(1)#133] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sales#41, count(1)#133 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#132 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134] +Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum#181, isEmpty#182, count#183] +Keys [3]: [i_brand_id#178, i_class_id#179, i_category_id#180] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186, count(1)#187] +Results [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sales#63, count(1)#187 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#186 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] (135) Filter [codegen id : 187] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (136) Project [codegen id : 187] -Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#134] +Output [6]: [store AS channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64] +Input [6]: [i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#184 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#185 as decimal(12,2)))), DecimalType(18,2), true))#188] (137) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#135, isEmpty#136, count#137] +Output [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] (138) HashAggregate [codegen id : 213] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#135, isEmpty#136, count#137] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#138, count(1)#139] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sales#58, count(1)#139 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#138 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140] +Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum#192, isEmpty#193, count#194] +Keys [3]: [i_brand_id#189, i_class_id#190, i_category_id#191] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197, count(1)#198] +Results [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sales#87, count(1)#198 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#197 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] (139) Filter [codegen id : 213] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (140) Project [codegen id : 213] -Output [6]: [catalog AS channel#141, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#140] +Output [6]: [catalog AS channel#90, i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88] +Input [6]: [i_brand_id#189, i_class_id#190, i_category_id#191, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#195 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#196 as decimal(12,2)))), DecimalType(18,2), true))#199] (141) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#142, isEmpty#143, count#144] +Output [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum#203, isEmpty#204, count#205] (142) HashAggregate [codegen id : 239] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#142, isEmpty#143, count#144] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#145, count(1)#146] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sales#73, count(1)#146 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#145 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147] +Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum#203, isEmpty#204, count#205] +Keys [3]: [i_brand_id#200, i_class_id#201, i_category_id#202] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208, count(1)#209] +Results [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208 AS sales#109, count(1)#209 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#208 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] (143) Filter [codegen id : 239] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (144) Project [codegen id : 239] -Output [6]: [web AS channel#148, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#147] +Output [6]: [web AS channel#112, i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110] +Input [6]: [i_brand_id#200, i_class_id#201, i_category_id#202, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#206 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#207 as decimal(12,2)))), DecimalType(18,2), true))#210] (145) Union (146) HashAggregate [codegen id : 240] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#149, isEmpty#150, sum#151] -Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] +Input [6]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sales#63, number_sales#64] +Keys [4]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#211, isEmpty#212, sum#213] +Results [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] (147) Exchange -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#155] +Input [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] +Arguments: hashpartitioning(channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, 5), ENSURE_REQUIREMENTS, [id=#217] (148) HashAggregate [codegen id : 241] -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#152, isEmpty#153, sum#154] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#156, sum(number_sales#42)#157] -Results [4]: [channel#46, i_brand_id#7, sum(sales#41)#156 AS sum_sales#86, sum(number_sales#42)#157 AS number_sales#87] +Input [7]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180, sum#214, isEmpty#215, sum#216] +Keys [4]: [channel#68, i_brand_id#178, i_class_id#179, i_category_id#180] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#218, sum(number_sales#64)#219] +Results [4]: [channel#68, i_brand_id#178, sum(sales#63)#218 AS sum_sales#122, sum(number_sales#64)#219 AS number_sales#123] (149) HashAggregate [codegen id : 241] -Input [4]: [channel#46, i_brand_id#7, sum_sales#86, number_sales#87] -Keys [2]: [channel#46, i_brand_id#7] -Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] -Aggregate Attributes [3]: [sum#158, isEmpty#159, sum#160] -Results [5]: [channel#46, i_brand_id#7, sum#161, isEmpty#162, sum#163] +Input [4]: [channel#68, i_brand_id#178, sum_sales#122, number_sales#123] +Keys [2]: [channel#68, i_brand_id#178] +Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] +Aggregate Attributes [3]: [sum#220, isEmpty#221, sum#222] +Results [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] (150) Exchange -Input [5]: [channel#46, i_brand_id#7, sum#161, isEmpty#162, sum#163] -Arguments: hashpartitioning(channel#46, i_brand_id#7, 5), ENSURE_REQUIREMENTS, [id=#164] +Input [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] +Arguments: hashpartitioning(channel#68, i_brand_id#178, 5), ENSURE_REQUIREMENTS, [id=#226] (151) HashAggregate [codegen id : 242] -Input [5]: [channel#46, i_brand_id#7, sum#161, isEmpty#162, sum#163] -Keys [2]: [channel#46, i_brand_id#7] -Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] -Aggregate Attributes [2]: [sum(sum_sales#86)#165, sum(number_sales#87)#166] -Results [6]: [channel#46, i_brand_id#7, null AS i_class_id#167, null AS i_category_id#168, sum(sum_sales#86)#165 AS sum(sum_sales)#169, sum(number_sales#87)#166 AS sum(number_sales)#170] +Input [5]: [channel#68, i_brand_id#178, sum#223, isEmpty#224, sum#225] +Keys [2]: [channel#68, i_brand_id#178] +Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] +Aggregate Attributes [2]: [sum(sum_sales#122)#227, sum(number_sales#123)#228] +Results [6]: [channel#68, i_brand_id#178, null AS i_class_id#229, null AS i_category_id#230, sum(sum_sales#122)#227 AS sum(sum_sales)#231, sum(number_sales#123)#228 AS sum(number_sales)#232] (152) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#171, isEmpty#172, count#173] +Output [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] (153) HashAggregate [codegen id : 268] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#171, isEmpty#172, count#173] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#174, count(1)#175] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sales#41, count(1)#175 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#174 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176] +Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum#236, isEmpty#237, count#238] +Keys [3]: [i_brand_id#233, i_class_id#234, i_category_id#235] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241, count(1)#242] +Results [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sales#63, count(1)#242 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#241 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] (154) Filter [codegen id : 268] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (155) Project [codegen id : 268] -Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#176] +Output [6]: [store AS channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64] +Input [6]: [i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#239 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#240 as decimal(12,2)))), DecimalType(18,2), true))#243] (156) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#177, isEmpty#178, count#179] +Output [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] (157) HashAggregate [codegen id : 294] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#177, isEmpty#178, count#179] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#180, count(1)#181] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sales#58, count(1)#181 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#180 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182] +Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum#247, isEmpty#248, count#249] +Keys [3]: [i_brand_id#244, i_class_id#245, i_category_id#246] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252, count(1)#253] +Results [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sales#87, count(1)#253 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#252 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] (158) Filter [codegen id : 294] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (159) Project [codegen id : 294] -Output [6]: [catalog AS channel#183, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#182] +Output [6]: [catalog AS channel#90, i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88] +Input [6]: [i_brand_id#244, i_class_id#245, i_category_id#246, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#250 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#251 as decimal(12,2)))), DecimalType(18,2), true))#254] (160) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#184, isEmpty#185, count#186] +Output [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#260] (161) HashAggregate [codegen id : 320] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#184, isEmpty#185, count#186] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#187, count(1)#188] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sales#73, count(1)#188 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#187 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189] +Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum#258, isEmpty#259, count#260] +Keys [3]: [i_brand_id#255, i_class_id#256, i_category_id#257] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263, count(1)#264] +Results [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263 AS sales#109, count(1)#264 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#263 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] (162) Filter [codegen id : 320] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (163) Project [codegen id : 320] -Output [6]: [web AS channel#190, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#189] +Output [6]: [web AS channel#112, i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110] +Input [6]: [i_brand_id#255, i_class_id#256, i_category_id#257, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#265] (164) Union (165) HashAggregate [codegen id : 321] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#191, isEmpty#192, sum#193] -Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#194, isEmpty#195, sum#196] +Input [6]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sales#63, number_sales#64] +Keys [4]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#266, isEmpty#267, sum#268] +Results [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] (166) Exchange -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#194, isEmpty#195, sum#196] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#197] +Input [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] +Arguments: hashpartitioning(channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, 5), ENSURE_REQUIREMENTS, [id=#272] (167) HashAggregate [codegen id : 322] -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#194, isEmpty#195, sum#196] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#198, sum(number_sales#42)#199] -Results [3]: [channel#46, sum(sales#41)#198 AS sum_sales#86, sum(number_sales#42)#199 AS number_sales#87] +Input [7]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235, sum#269, isEmpty#270, sum#271] +Keys [4]: [channel#68, i_brand_id#233, i_class_id#234, i_category_id#235] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#273, sum(number_sales#64)#274] +Results [3]: [channel#68, sum(sales#63)#273 AS sum_sales#122, sum(number_sales#64)#274 AS number_sales#123] (168) HashAggregate [codegen id : 322] -Input [3]: [channel#46, sum_sales#86, number_sales#87] -Keys [1]: [channel#46] -Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] -Aggregate Attributes [3]: [sum#200, isEmpty#201, sum#202] -Results [4]: [channel#46, sum#203, isEmpty#204, sum#205] +Input [3]: [channel#68, sum_sales#122, number_sales#123] +Keys [1]: [channel#68] +Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] +Aggregate Attributes [3]: [sum#275, isEmpty#276, sum#277] +Results [4]: [channel#68, sum#278, isEmpty#279, sum#280] (169) Exchange -Input [4]: [channel#46, sum#203, isEmpty#204, sum#205] -Arguments: hashpartitioning(channel#46, 5), ENSURE_REQUIREMENTS, [id=#206] +Input [4]: [channel#68, sum#278, isEmpty#279, sum#280] +Arguments: hashpartitioning(channel#68, 5), ENSURE_REQUIREMENTS, [id=#281] (170) HashAggregate [codegen id : 323] -Input [4]: [channel#46, sum#203, isEmpty#204, sum#205] -Keys [1]: [channel#46] -Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] -Aggregate Attributes [2]: [sum(sum_sales#86)#207, sum(number_sales#87)#208] -Results [6]: [channel#46, null AS i_brand_id#209, null AS i_class_id#210, null AS i_category_id#211, sum(sum_sales#86)#207 AS sum(sum_sales)#212, sum(number_sales#87)#208 AS sum(number_sales)#213] +Input [4]: [channel#68, sum#278, isEmpty#279, sum#280] +Keys [1]: [channel#68] +Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] +Aggregate Attributes [2]: [sum(sum_sales#122)#282, sum(number_sales#123)#283] +Results [6]: [channel#68, null AS i_brand_id#284, null AS i_class_id#285, null AS i_category_id#286, sum(sum_sales#122)#282 AS sum(sum_sales)#287, sum(number_sales#123)#283 AS sum(number_sales)#288] (171) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#214, isEmpty#215, count#216] +Output [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] (172) HashAggregate [codegen id : 349] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#214, isEmpty#215, count#216] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#217, count(1)#218] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#217 AS sales#41, count(1)#218 AS number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#217 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219] +Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum#292, isEmpty#293, count#294] +Keys [3]: [i_brand_id#289, i_class_id#290, i_category_id#291] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297, count(1)#298] +Results [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sales#63, count(1)#298 AS number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#297 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] (173) Filter [codegen id : 349] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (174) Project [codegen id : 349] -Output [6]: [store AS channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#2 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2), true))#219] +Output [6]: [store AS channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64] +Input [6]: [i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#295 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_list_price#296 as decimal(12,2)))), DecimalType(18,2), true))#299] (175) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] +Output [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] (176) HashAggregate [codegen id : 375] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#220, isEmpty#221, count#222] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#223, count(1)#224] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sales#58, count(1)#224 AS number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#223 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225] +Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum#303, isEmpty#304, count#305] +Keys [3]: [i_brand_id#300, i_class_id#301, i_category_id#302] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308, count(1)#309] +Results [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sales#87, count(1)#309 AS number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#308 AS sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] (177) Filter [codegen id : 375] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (178) Project [codegen id : 375] -Output [6]: [catalog AS channel#226, i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#58, number_sales#59, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#47 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#48 as decimal(12,2)))), DecimalType(18,2), true))#225] +Output [6]: [catalog AS channel#90, i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88] +Input [6]: [i_brand_id#300, i_class_id#301, i_category_id#302, sales#87, number_sales#88, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#306 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#307 as decimal(12,2)))), DecimalType(18,2), true))#310] (179) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#227, isEmpty#228, count#229] +Output [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum#314, isEmpty#315, count#316] (180) HashAggregate [codegen id : 401] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum#227, isEmpty#228, count#229] -Keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] -Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#230, count(1)#231] -Results [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sales#73, count(1)#231 AS number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#230 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232] +Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum#314, isEmpty#315, count#316] +Keys [3]: [i_brand_id#311, i_class_id#312, i_category_id#313] +Functions [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true)), count(1)] +Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319, count(1)#320] +Results [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319 AS sales#109, count(1)#320 AS number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#319 AS sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] (181) Filter [codegen id : 401] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#44, [id=#45] as decimal(32,6)))) +Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#66, [id=#67] as decimal(32,6)))) (182) Project [codegen id : 401] -Output [6]: [web AS channel#233, i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74] -Input [6]: [i_brand_id#7, i_class_id#8, i_category_id#9, sales#73, number_sales#74, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#62 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#63 as decimal(12,2)))), DecimalType(18,2), true))#232] +Output [6]: [web AS channel#112, i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110] +Input [6]: [i_brand_id#311, i_class_id#312, i_category_id#313, sales#109, number_sales#110, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#317 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#318 as decimal(12,2)))), DecimalType(18,2), true))#321] (183) Union (184) HashAggregate [codegen id : 402] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sales#41, number_sales#42] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [partial_sum(sales#41), partial_sum(number_sales#42)] -Aggregate Attributes [3]: [sum#234, isEmpty#235, sum#236] -Results [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#237, isEmpty#238, sum#239] +Input [6]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sales#63, number_sales#64] +Keys [4]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291] +Functions [2]: [partial_sum(sales#63), partial_sum(number_sales#64)] +Aggregate Attributes [3]: [sum#322, isEmpty#323, sum#324] +Results [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] (185) Exchange -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#237, isEmpty#238, sum#239] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, 5), ENSURE_REQUIREMENTS, [id=#240] +Input [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] +Arguments: hashpartitioning(channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, 5), ENSURE_REQUIREMENTS, [id=#328] (186) HashAggregate [codegen id : 403] -Input [7]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum#237, isEmpty#238, sum#239] -Keys [4]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9] -Functions [2]: [sum(sales#41), sum(number_sales#42)] -Aggregate Attributes [2]: [sum(sales#41)#241, sum(number_sales#42)#242] -Results [2]: [sum(sales#41)#241 AS sum_sales#86, sum(number_sales#42)#242 AS number_sales#87] +Input [7]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291, sum#325, isEmpty#326, sum#327] +Keys [4]: [channel#68, i_brand_id#289, i_class_id#290, i_category_id#291] +Functions [2]: [sum(sales#63), sum(number_sales#64)] +Aggregate Attributes [2]: [sum(sales#63)#329, sum(number_sales#64)#330] +Results [2]: [sum(sales#63)#329 AS sum_sales#122, sum(number_sales#64)#330 AS number_sales#123] (187) HashAggregate [codegen id : 403] -Input [2]: [sum_sales#86, number_sales#87] +Input [2]: [sum_sales#122, number_sales#123] Keys: [] -Functions [2]: [partial_sum(sum_sales#86), partial_sum(number_sales#87)] -Aggregate Attributes [3]: [sum#243, isEmpty#244, sum#245] -Results [3]: [sum#246, isEmpty#247, sum#248] +Functions [2]: [partial_sum(sum_sales#122), partial_sum(number_sales#123)] +Aggregate Attributes [3]: [sum#331, isEmpty#332, sum#333] +Results [3]: [sum#334, isEmpty#335, sum#336] (188) Exchange -Input [3]: [sum#246, isEmpty#247, sum#248] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#249] +Input [3]: [sum#334, isEmpty#335, sum#336] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#337] (189) HashAggregate [codegen id : 404] -Input [3]: [sum#246, isEmpty#247, sum#248] +Input [3]: [sum#334, isEmpty#335, sum#336] Keys: [] -Functions [2]: [sum(sum_sales#86), sum(number_sales#87)] -Aggregate Attributes [2]: [sum(sum_sales#86)#250, sum(number_sales#87)#251] -Results [6]: [null AS channel#252, null AS i_brand_id#253, null AS i_class_id#254, null AS i_category_id#255, sum(sum_sales#86)#250 AS sum(sum_sales)#256, sum(number_sales#87)#251 AS sum(number_sales)#257] +Functions [2]: [sum(sum_sales#122), sum(number_sales#123)] +Aggregate Attributes [2]: [sum(sum_sales#122)#338, sum(number_sales#123)#339] +Results [6]: [null AS channel#340, null AS i_brand_id#341, null AS i_class_id#342, null AS i_category_id#343, sum(sum_sales#122)#338 AS sum(sum_sales)#344, sum(number_sales#123)#339 AS sum(number_sales)#345] (190) Union (191) HashAggregate [codegen id : 405] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] -Keys [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Keys [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] (192) Exchange -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] -Arguments: hashpartitioning(channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87, 5), ENSURE_REQUIREMENTS, [id=#258] +Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Arguments: hashpartitioning(channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123, 5), ENSURE_REQUIREMENTS, [id=#346] (193) HashAggregate [codegen id : 406] -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] -Keys [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Keys [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] Functions: [] Aggregate Attributes: [] -Results [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Results [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] (194) TakeOrderedAndProject -Input [6]: [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] -Arguments: 100, [channel#46 ASC NULLS FIRST, i_brand_id#7 ASC NULLS FIRST, i_class_id#8 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST], [channel#46, i_brand_id#7, i_class_id#8, i_category_id#9, sum_sales#86, number_sales#87] +Input [6]: [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] +Arguments: 100, [channel#68 ASC NULLS FIRST, i_brand_id#46 ASC NULLS FIRST, i_class_id#47 ASC NULLS FIRST, i_category_id#48 ASC NULLS FIRST], [channel#68, i_brand_id#46, i_class_id#47, i_category_id#48, sum_sales#122, number_sales#123] ===== Subqueries ===== -Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquery#66, [id=#67] * HashAggregate (221) +- Exchange (220) +- * HashAggregate (219) @@ -1145,193 +1145,193 @@ Subquery:1 Hosting operator id = 76 Hosting Expression = Subquery scalar-subquer (195) Scan parquet default.store_sales -Output [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Output [3]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#259)] +PartitionFilters: [isnotnull(ss_sold_date_sk#349), dynamicpruningexpression(ss_sold_date_sk#349 IN dynamicpruning#350)] ReadSchema: struct (196) ColumnarToRow [codegen id : 2] -Input [3]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Input [3]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349] (197) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#351, d_year#352] 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 (198) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#351, d_year#352] (199) Filter [codegen id : 1] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1999)) AND (d_year#13 <= 2001)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#351, d_year#352] +Condition : (((isnotnull(d_year#352) AND (d_year#352 >= 1999)) AND (d_year#352 <= 2001)) AND isnotnull(d_date_sk#351)) (200) Project [codegen id : 1] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#351] +Input [2]: [d_date_sk#351, d_year#352] (201) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#260] +Input [1]: [d_date_sk#351] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#353] (202) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ss_sold_date_sk#349] +Right keys [1]: [d_date_sk#351] Join condition: None (203) Project [codegen id : 2] -Output [2]: [ss_quantity#2 AS quantity#261, ss_list_price#3 AS list_price#262] -Input [4]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, d_date_sk#12] +Output [2]: [ss_quantity#347 AS quantity#354, ss_list_price#348 AS list_price#355] +Input [4]: [ss_quantity#347, ss_list_price#348, ss_sold_date_sk#349, d_date_sk#351] (204) Scan parquet default.catalog_sales -Output [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Output [3]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#263)] +PartitionFilters: [isnotnull(cs_sold_date_sk#358), dynamicpruningexpression(cs_sold_date_sk#358 IN dynamicpruning#359)] ReadSchema: struct (205) ColumnarToRow [codegen id : 4] -Input [3]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19] +Input [3]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358] (206) Scan parquet default.date_dim -Output [2]: [d_date_sk#12, d_year#13] +Output [2]: [d_date_sk#360, d_year#361] 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 (207) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] +Input [2]: [d_date_sk#360, d_year#361] (208) Filter [codegen id : 3] -Input [2]: [d_date_sk#12, d_year#13] -Condition : (((isnotnull(d_year#13) AND (d_year#13 >= 1998)) AND (d_year#13 <= 2000)) AND isnotnull(d_date_sk#12)) +Input [2]: [d_date_sk#360, d_year#361] +Condition : (((isnotnull(d_year#361) AND (d_year#361 >= 1998)) AND (d_year#361 <= 2000)) AND isnotnull(d_date_sk#360)) (209) Project [codegen id : 3] -Output [1]: [d_date_sk#12] -Input [2]: [d_date_sk#12, d_year#13] +Output [1]: [d_date_sk#360] +Input [2]: [d_date_sk#360, d_year#361] (210) BroadcastExchange -Input [1]: [d_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#264] +Input [1]: [d_date_sk#360] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#362] (211) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [cs_sold_date_sk#358] +Right keys [1]: [d_date_sk#360] Join condition: None (212) Project [codegen id : 4] -Output [2]: [cs_quantity#47 AS quantity#265, cs_list_price#48 AS list_price#266] -Input [4]: [cs_quantity#47, cs_list_price#48, cs_sold_date_sk#19, d_date_sk#12] +Output [2]: [cs_quantity#356 AS quantity#363, cs_list_price#357 AS list_price#364] +Input [4]: [cs_quantity#356, cs_list_price#357, cs_sold_date_sk#358, d_date_sk#360] (213) Scan parquet default.web_sales -Output [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Output [3]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#24), dynamicpruningexpression(ws_sold_date_sk#24 IN dynamicpruning#263)] +PartitionFilters: [isnotnull(ws_sold_date_sk#367), dynamicpruningexpression(ws_sold_date_sk#367 IN dynamicpruning#359)] ReadSchema: struct (214) ColumnarToRow [codegen id : 6] -Input [3]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24] +Input [3]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367] (215) ReusedExchange [Reuses operator id: 210] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#368] (216) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ws_sold_date_sk#24] -Right keys [1]: [d_date_sk#12] +Left keys [1]: [ws_sold_date_sk#367] +Right keys [1]: [d_date_sk#368] Join condition: None (217) Project [codegen id : 6] -Output [2]: [ws_quantity#62 AS quantity#267, ws_list_price#63 AS list_price#268] -Input [4]: [ws_quantity#62, ws_list_price#63, ws_sold_date_sk#24, d_date_sk#12] +Output [2]: [ws_quantity#365 AS quantity#369, ws_list_price#366 AS list_price#370] +Input [4]: [ws_quantity#365, ws_list_price#366, ws_sold_date_sk#367, d_date_sk#368] (218) Union (219) HashAggregate [codegen id : 7] -Input [2]: [quantity#261, list_price#262] +Input [2]: [quantity#354, list_price#355] Keys: [] -Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#269, count#270] -Results [2]: [sum#271, count#272] +Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#371, count#372] +Results [2]: [sum#373, count#374] (220) Exchange -Input [2]: [sum#271, count#272] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#273] +Input [2]: [sum#373, count#374] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#375] (221) HashAggregate [codegen id : 8] -Input [2]: [sum#271, count#272] +Input [2]: [sum#373, count#374] Keys: [] -Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#274] -Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#261 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#262 as decimal(12,2)))), DecimalType(18,2), true))#274 AS average_sales#275] +Functions [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))#376] +Results [1]: [avg(CheckOverflow((promote_precision(cast(cast(quantity#354 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(list_price#355 as decimal(12,2)))), DecimalType(18,2), true))#376 AS average_sales#377] -Subquery:2 Hosting operator id = 195 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#259 +Subquery:2 Hosting operator id = 195 Hosting Expression = ss_sold_date_sk#349 IN dynamicpruning#350 ReusedExchange (222) (222) ReusedExchange [Reuses operator id: 201] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#351] -Subquery:3 Hosting operator id = 204 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#263 +Subquery:3 Hosting operator id = 204 Hosting Expression = cs_sold_date_sk#358 IN dynamicpruning#359 ReusedExchange (223) (223) ReusedExchange [Reuses operator id: 210] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#360] -Subquery:4 Hosting operator id = 213 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#263 +Subquery:4 Hosting operator id = 213 Hosting Expression = ws_sold_date_sk#367 IN dynamicpruning#359 Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 ReusedExchange (224) (224) ReusedExchange [Reuses operator id: 70] -Output [1]: [d_date_sk#12] +Output [1]: [d_date_sk#50] -Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#10 +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 ReusedExchange (225) -(225) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#12] +(225) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#29] -Subquery:7 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#10 +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 -Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#10 +Subquery:8 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#12 -Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:9 Hosting operator id = 92 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 +Subquery:10 Hosting operator id = 78 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#5 -Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:11 Hosting operator id = 108 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#24 IN dynamicpruning#5 +Subquery:12 Hosting operator id = 94 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#5 -Subquery:13 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:13 Hosting operator id = 116 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:14 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:14 Hosting operator id = 120 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:15 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:15 Hosting operator id = 124 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:16 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:16 Hosting operator id = 135 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:17 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:17 Hosting operator id = 139 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:18 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:18 Hosting operator id = 143 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:19 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:19 Hosting operator id = 154 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:20 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:20 Hosting operator id = 158 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:21 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:21 Hosting operator id = 162 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:22 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:22 Hosting operator id = 173 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:23 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:23 Hosting operator id = 177 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] -Subquery:24 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] +Subquery:24 Hosting operator id = 181 Hosting Expression = ReusedSubquery Subquery scalar-subquery#66, [id=#67] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt index 1634c1e247a12..e351f9e687027 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt @@ -104,53 +104,53 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num Exchange [brand_id,class_id,category_id] #7 WholeStageCodegen (6) HashAggregate [brand_id,class_id,category_id] - BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - ReusedExchange [d_date_sk] #8 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (1) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Filter [ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] Filter [i_item_sk,i_brand_id,i_class_id,i_category_id] ColumnarToRow InputAdapter Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (5) - Project [i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] - Filter [cs_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #2 InputAdapter - BroadcastExchange #11 + BroadcastExchange #10 WholeStageCodegen (3) - Filter [i_item_sk] - ColumnarToRow + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Filter [cs_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (1) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] - InputAdapter - ReusedExchange [d_date_sk] #8 + BroadcastExchange #8 + WholeStageCodegen (2) + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #8 InputAdapter BroadcastExchange #12 WholeStageCodegen (9) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt index b2934b6bb8242..3f2b62a7b8fa4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt @@ -81,57 +81,57 @@ Input [3]: [i_item_sk#5, i_current_price#6, i_category#7] Condition : (isnotnull(i_current_price#6) AND isnotnull(i_item_sk#5)) (7) Scan parquet default.item -Output [2]: [i_current_price#6, i_category#7] +Output [2]: [i_current_price#8, i_category#9] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] +Input [2]: [i_current_price#8, i_category#9] (9) Filter [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] -Condition : isnotnull(i_category#7) +Input [2]: [i_current_price#8, i_category#9] +Condition : isnotnull(i_category#9) (10) HashAggregate [codegen id : 1] -Input [2]: [i_current_price#6, i_category#7] -Keys [1]: [i_category#7] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#6))] -Aggregate Attributes [2]: [sum#8, count#9] -Results [3]: [i_category#7, sum#10, count#11] +Input [2]: [i_current_price#8, i_category#9] +Keys [1]: [i_category#9] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#8))] +Aggregate Attributes [2]: [sum#10, count#11] +Results [3]: [i_category#9, sum#12, count#13] (11) Exchange -Input [3]: [i_category#7, sum#10, count#11] -Arguments: hashpartitioning(i_category#7, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [3]: [i_category#9, sum#12, count#13] +Arguments: hashpartitioning(i_category#9, 5), ENSURE_REQUIREMENTS, [id=#14] (12) HashAggregate [codegen id : 2] -Input [3]: [i_category#7, sum#10, count#11] -Keys [1]: [i_category#7] -Functions [1]: [avg(UnscaledValue(i_current_price#6))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#6))#13] -Results [2]: [cast((avg(UnscaledValue(i_current_price#6))#13 / 100.0) as decimal(11,6)) AS avg(i_current_price)#14, i_category#7 AS i_category#7#15] +Input [3]: [i_category#9, sum#12, count#13] +Keys [1]: [i_category#9] +Functions [1]: [avg(UnscaledValue(i_current_price#8))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#8))#15] +Results [2]: [cast((avg(UnscaledValue(i_current_price#8))#15 / 100.0) as decimal(11,6)) AS avg(i_current_price)#16, i_category#9] (13) BroadcastExchange -Input [2]: [avg(i_current_price)#14, i_category#7#15] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#16] +Input [2]: [avg(i_current_price)#16, i_category#9] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#17] (14) BroadcastHashJoin [codegen id : 3] Left keys [1]: [i_category#7] -Right keys [1]: [i_category#7#15] +Right keys [1]: [i_category#9] Join condition: None (15) Filter [codegen id : 3] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] -Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#14)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] +Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#16)), DecimalType(14,7), true)) (16) Project [codegen id : 3] Output [1]: [i_item_sk#5] -Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#14, i_category#7#15] +Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#16, i_category#9] (17) BroadcastExchange Input [1]: [i_item_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (18) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_item_sk#1] @@ -143,143 +143,143 @@ Output [2]: [ss_customer_sk#2, ss_sold_date_sk#3] Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, i_item_sk#5] (20) Scan parquet default.date_dim -Output [2]: [d_date_sk#18, d_month_seq#19] +Output [2]: [d_date_sk#19, d_month_seq#20] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#18, d_month_seq#19] +Input [2]: [d_date_sk#19, d_month_seq#20] (22) Filter [codegen id : 4] -Input [2]: [d_date_sk#18, d_month_seq#19] -Condition : ((isnotnull(d_month_seq#19) AND (d_month_seq#19 = Subquery scalar-subquery#20, [id=#21])) AND isnotnull(d_date_sk#18)) +Input [2]: [d_date_sk#19, d_month_seq#20] +Condition : ((isnotnull(d_month_seq#20) AND (d_month_seq#20 = Subquery scalar-subquery#21, [id=#22])) AND isnotnull(d_date_sk#19)) (23) Project [codegen id : 4] -Output [1]: [d_date_sk#18] -Input [2]: [d_date_sk#18, d_month_seq#19] +Output [1]: [d_date_sk#19] +Input [2]: [d_date_sk#19, d_month_seq#20] (24) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (25) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (26) Project [codegen id : 5] Output [1]: [ss_customer_sk#2] -Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#18] +Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#19] (27) Exchange Input [1]: [ss_customer_sk#2] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#23] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] (28) Sort [codegen id : 6] Input [1]: [ss_customer_sk#2] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#25, ca_state#26] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#24, ca_state#25] +Input [2]: [ca_address_sk#25, ca_state#26] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : isnotnull(ca_address_sk#24) +Input [2]: [ca_address_sk#25, ca_state#26] +Condition : isnotnull(ca_address_sk#25) (32) Exchange -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [ca_address_sk#25, ca_state#26] +Arguments: hashpartitioning(ca_address_sk#25, 5), ENSURE_REQUIREMENTS, [id=#27] (33) Sort [codegen id : 8] -Input [2]: [ca_address_sk#24, ca_state#25] -Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#25, ca_state#26] +Arguments: [ca_address_sk#25 ASC NULLS FIRST], false, 0 (34) Scan parquet default.customer -Output [2]: [c_customer_sk#27, c_current_addr_sk#28] +Output [2]: [c_customer_sk#28, c_current_addr_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 9] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] (36) Filter [codegen id : 9] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Condition : (isnotnull(c_current_addr_sk#28) AND isnotnull(c_customer_sk#27)) +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Condition : (isnotnull(c_current_addr_sk#29) AND isnotnull(c_customer_sk#28)) (37) Exchange -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Arguments: hashpartitioning(c_current_addr_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Arguments: hashpartitioning(c_current_addr_sk#29, 5), ENSURE_REQUIREMENTS, [id=#30] (38) Sort [codegen id : 10] -Input [2]: [c_customer_sk#27, c_current_addr_sk#28] -Arguments: [c_current_addr_sk#28 ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#28, c_current_addr_sk#29] +Arguments: [c_current_addr_sk#29 ASC NULLS FIRST], false, 0 (39) SortMergeJoin [codegen id : 11] -Left keys [1]: [ca_address_sk#24] -Right keys [1]: [c_current_addr_sk#28] +Left keys [1]: [ca_address_sk#25] +Right keys [1]: [c_current_addr_sk#29] Join condition: None (40) Project [codegen id : 11] -Output [2]: [ca_state#25, c_customer_sk#27] -Input [4]: [ca_address_sk#24, ca_state#25, c_customer_sk#27, c_current_addr_sk#28] +Output [2]: [ca_state#26, c_customer_sk#28] +Input [4]: [ca_address_sk#25, ca_state#26, c_customer_sk#28, c_current_addr_sk#29] (41) Exchange -Input [2]: [ca_state#25, c_customer_sk#27] -Arguments: hashpartitioning(c_customer_sk#27, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [ca_state#26, c_customer_sk#28] +Arguments: hashpartitioning(c_customer_sk#28, 5), ENSURE_REQUIREMENTS, [id=#31] (42) Sort [codegen id : 12] -Input [2]: [ca_state#25, c_customer_sk#27] -Arguments: [c_customer_sk#27 ASC NULLS FIRST], false, 0 +Input [2]: [ca_state#26, c_customer_sk#28] +Arguments: [c_customer_sk#28 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 13] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#27] +Right keys [1]: [c_customer_sk#28] Join condition: None (44) Project [codegen id : 13] -Output [1]: [ca_state#25] -Input [3]: [ss_customer_sk#2, ca_state#25, c_customer_sk#27] +Output [1]: [ca_state#26] +Input [3]: [ss_customer_sk#2, ca_state#26, c_customer_sk#28] (45) HashAggregate [codegen id : 13] -Input [1]: [ca_state#25] -Keys [1]: [ca_state#25] +Input [1]: [ca_state#26] +Keys [1]: [ca_state#26] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#31] -Results [2]: [ca_state#25, count#32] +Aggregate Attributes [1]: [count#32] +Results [2]: [ca_state#26, count#33] (46) Exchange -Input [2]: [ca_state#25, count#32] -Arguments: hashpartitioning(ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [2]: [ca_state#26, count#33] +Arguments: hashpartitioning(ca_state#26, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 14] -Input [2]: [ca_state#25, count#32] -Keys [1]: [ca_state#25] +Input [2]: [ca_state#26, count#33] +Keys [1]: [ca_state#26] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#34] -Results [4]: [ca_state#25 AS state#35, count(1)#34 AS cnt#36, count(1)#34 AS count(1)#37, ca_state#25] +Aggregate Attributes [1]: [count(1)#35] +Results [4]: [ca_state#26 AS state#36, count(1)#35 AS cnt#37, count(1)#35 AS count(1)#38, ca_state#26] (48) Filter [codegen id : 14] -Input [4]: [state#35, cnt#36, count(1)#37, ca_state#25] -Condition : (count(1)#37 >= 10) +Input [4]: [state#36, cnt#37, count(1)#38, ca_state#26] +Condition : (count(1)#38 >= 10) (49) Project [codegen id : 14] -Output [3]: [state#35, cnt#36, ca_state#25] -Input [4]: [state#35, cnt#36, count(1)#37, ca_state#25] +Output [3]: [state#36, cnt#37, ca_state#26] +Input [4]: [state#36, cnt#37, count(1)#38, ca_state#26] (50) TakeOrderedAndProject -Input [3]: [state#35, cnt#36, ca_state#25] -Arguments: 100, [cnt#36 ASC NULLS FIRST, ca_state#25 ASC NULLS FIRST], [state#35, cnt#36] +Input [3]: [state#36, cnt#37, ca_state#26] +Arguments: 100, [cnt#37 ASC NULLS FIRST, ca_state#26 ASC NULLS FIRST], [state#36, cnt#37] ===== Subqueries ===== @@ -288,9 +288,9 @@ ReusedExchange (51) (51) ReusedExchange [Reuses operator id: 24] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#19] -Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#20, [id=#21] +Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquery#21, [id=#22] * HashAggregate (58) +- Exchange (57) +- * HashAggregate (56) @@ -301,39 +301,39 @@ Subquery:2 Hosting operator id = 22 Hosting Expression = Subquery scalar-subquer (52) Scan parquet default.date_dim -Output [3]: [d_month_seq#19, d_year#38, d_moy#39] +Output [3]: [d_month_seq#39, d_year#40, d_moy#41] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (53) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] (54) Filter [codegen id : 1] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] -Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] +Condition : (((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2000)) AND (d_moy#41 = 1)) (55) Project [codegen id : 1] -Output [1]: [d_month_seq#19] -Input [3]: [d_month_seq#19, d_year#38, d_moy#39] +Output [1]: [d_month_seq#39] +Input [3]: [d_month_seq#39, d_year#40, d_moy#41] (56) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#19] -Keys [1]: [d_month_seq#19] +Input [1]: [d_month_seq#39] +Keys [1]: [d_month_seq#39] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#19] +Results [1]: [d_month_seq#39] (57) Exchange -Input [1]: [d_month_seq#19] -Arguments: hashpartitioning(d_month_seq#19, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [1]: [d_month_seq#39] +Arguments: hashpartitioning(d_month_seq#39, 5), ENSURE_REQUIREMENTS, [id=#42] (58) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#19] -Keys [1]: [d_month_seq#19] +Input [1]: [d_month_seq#39] +Keys [1]: [d_month_seq#39] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#19] +Results [1]: [d_month_seq#39] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt index 23b705e3b97e7..f3badf6efe6b6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt @@ -38,7 +38,7 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter BroadcastExchange #5 WholeStageCodegen (2) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] InputAdapter Exchange [i_category] #6 WholeStageCodegen (1) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt index 8598023b0276d..b37db85388e0f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt @@ -160,57 +160,57 @@ Input [3]: [i_item_sk#16, i_current_price#17, i_category#18] Condition : (isnotnull(i_current_price#17) AND isnotnull(i_item_sk#16)) (26) Scan parquet default.item -Output [2]: [i_current_price#17, i_category#18] +Output [2]: [i_current_price#19, i_category#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category)] ReadSchema: struct (27) ColumnarToRow [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] +Input [2]: [i_current_price#19, i_category#20] (28) Filter [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] -Condition : isnotnull(i_category#18) +Input [2]: [i_current_price#19, i_category#20] +Condition : isnotnull(i_category#20) (29) HashAggregate [codegen id : 4] -Input [2]: [i_current_price#17, i_category#18] -Keys [1]: [i_category#18] -Functions [1]: [partial_avg(UnscaledValue(i_current_price#17))] -Aggregate Attributes [2]: [sum#19, count#20] -Results [3]: [i_category#18, sum#21, count#22] +Input [2]: [i_current_price#19, i_category#20] +Keys [1]: [i_category#20] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#19))] +Aggregate Attributes [2]: [sum#21, count#22] +Results [3]: [i_category#20, sum#23, count#24] (30) Exchange -Input [3]: [i_category#18, sum#21, count#22] -Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [i_category#20, sum#23, count#24] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#25] (31) HashAggregate [codegen id : 5] -Input [3]: [i_category#18, sum#21, count#22] -Keys [1]: [i_category#18] -Functions [1]: [avg(UnscaledValue(i_current_price#17))] -Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#17))#24] -Results [2]: [cast((avg(UnscaledValue(i_current_price#17))#24 / 100.0) as decimal(11,6)) AS avg(i_current_price)#25, i_category#18 AS i_category#18#26] +Input [3]: [i_category#20, sum#23, count#24] +Keys [1]: [i_category#20] +Functions [1]: [avg(UnscaledValue(i_current_price#19))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#19))#26] +Results [2]: [cast((avg(UnscaledValue(i_current_price#19))#26 / 100.0) as decimal(11,6)) AS avg(i_current_price)#27, i_category#20] (32) BroadcastExchange -Input [2]: [avg(i_current_price)#25, i_category#18#26] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#27] +Input [2]: [avg(i_current_price)#27, i_category#20] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#28] (33) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_category#18] -Right keys [1]: [i_category#18#26] +Right keys [1]: [i_category#20] Join condition: None (34) Filter [codegen id : 6] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] -Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#25)), DecimalType(14,7), true)) +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] +Condition : (cast(i_current_price#17 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#27)), DecimalType(14,7), true)) (35) Project [codegen id : 6] Output [1]: [i_item_sk#16] -Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#25, i_category#18#26] +Input [5]: [i_item_sk#16, i_current_price#17, i_category#18, avg(i_current_price)#27, i_category#20] (36) BroadcastExchange Input [1]: [i_item_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (37) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_item_sk#6] @@ -225,31 +225,31 @@ Input [3]: [ca_state#2, ss_item_sk#6, i_item_sk#16] Input [1]: [ca_state#2] Keys [1]: [ca_state#2] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [2]: [ca_state#2, count#30] +Aggregate Attributes [1]: [count#30] +Results [2]: [ca_state#2, count#31] (40) Exchange -Input [2]: [ca_state#2, count#30] -Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [2]: [ca_state#2, count#31] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [id=#32] (41) HashAggregate [codegen id : 8] -Input [2]: [ca_state#2, count#30] +Input [2]: [ca_state#2, count#31] Keys [1]: [ca_state#2] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [4]: [ca_state#2 AS state#33, count(1)#32 AS cnt#34, count(1)#32 AS count(1)#35, ca_state#2] +Aggregate Attributes [1]: [count(1)#33] +Results [4]: [ca_state#2 AS state#34, count(1)#33 AS cnt#35, count(1)#33 AS count(1)#36, ca_state#2] (42) Filter [codegen id : 8] -Input [4]: [state#33, cnt#34, count(1)#35, ca_state#2] -Condition : (count(1)#35 >= 10) +Input [4]: [state#34, cnt#35, count(1)#36, ca_state#2] +Condition : (count(1)#36 >= 10) (43) Project [codegen id : 8] -Output [3]: [state#33, cnt#34, ca_state#2] -Input [4]: [state#33, cnt#34, count(1)#35, ca_state#2] +Output [3]: [state#34, cnt#35, ca_state#2] +Input [4]: [state#34, cnt#35, count(1)#36, ca_state#2] (44) TakeOrderedAndProject -Input [3]: [state#33, cnt#34, ca_state#2] -Arguments: 100, [cnt#34 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#33, cnt#34] +Input [3]: [state#34, cnt#35, ca_state#2] +Arguments: 100, [cnt#35 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#34, cnt#35] ===== Subqueries ===== @@ -271,39 +271,39 @@ Subquery:2 Hosting operator id = 18 Hosting Expression = Subquery scalar-subquer (46) Scan parquet default.date_dim -Output [3]: [d_month_seq#12, d_year#36, d_moy#37] +Output [3]: [d_month_seq#37, d_year#38, d_moy#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] ReadSchema: struct (47) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] (48) Filter [codegen id : 1] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] -Condition : (((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2000)) AND (d_moy#37 = 1)) +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] +Condition : (((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2000)) AND (d_moy#39 = 1)) (49) Project [codegen id : 1] -Output [1]: [d_month_seq#12] -Input [3]: [d_month_seq#12, d_year#36, d_moy#37] +Output [1]: [d_month_seq#37] +Input [3]: [d_month_seq#37, d_year#38, d_moy#39] (50) HashAggregate [codegen id : 1] -Input [1]: [d_month_seq#12] -Keys [1]: [d_month_seq#12] +Input [1]: [d_month_seq#37] +Keys [1]: [d_month_seq#37] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#12] +Results [1]: [d_month_seq#37] (51) Exchange -Input [1]: [d_month_seq#12] -Arguments: hashpartitioning(d_month_seq#12, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [1]: [d_month_seq#37] +Arguments: hashpartitioning(d_month_seq#37, 5), ENSURE_REQUIREMENTS, [id=#40] (52) HashAggregate [codegen id : 2] -Input [1]: [d_month_seq#12] -Keys [1]: [d_month_seq#12] +Input [1]: [d_month_seq#37] +Keys [1]: [d_month_seq#37] Functions: [] Aggregate Attributes: [] -Results [1]: [d_month_seq#12] +Results [1]: [d_month_seq#37] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt index ceddf1b04fc78..4ba09283e73cd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt @@ -68,7 +68,7 @@ TakeOrderedAndProject [cnt,ca_state,state] InputAdapter BroadcastExchange #7 WholeStageCodegen (5) - HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),i_category,sum,count] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] InputAdapter Exchange [i_category] #8 WholeStageCodegen (4) From 36a74ad764cf0ee0a04c878d0ce18e6a30916abb Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Fri, 26 Mar 2021 15:00:42 +0800 Subject: [PATCH 31/38] fix hasConflictingAttrs --- .../apache/spark/sql/catalyst/analysis/Analyzer.scala | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) 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 a44370094fed0..462f8a6e86d95 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 @@ -1437,9 +1437,14 @@ class Analyzer(override val catalogManager: CatalogManager) // Note that duplicated attributes are allowed within a single node, // e.g., df.select($"a", $"a"), so we should only check conflicting // attributes between nodes. - p.children.tail.foldLeft(p.children.head.outputSet) { - case (conflictAttrs, child) => conflictAttrs.intersect(child.outputSet) - }.nonEmpty + val uniqueAttrs = mutable.HashSet[Attribute]() + p.children.head.outputSet.foreach(uniqueAttrs.add) + p.children.tail.exists { child => + val uniqueSize = uniqueAttrs.size + val childSize = child.outputSet.size + child.outputSet.foreach(uniqueAttrs.add) + uniqueSize + childSize > uniqueAttrs.size + } } } From 8bbacf646466bc6ffb37ecf52c50312b5478c81d Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Fri, 26 Mar 2021 16:07:55 +0800 Subject: [PATCH 32/38] compare exprId --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 462f8a6e86d95..491490486fd7b 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 @@ -1437,12 +1437,12 @@ class Analyzer(override val catalogManager: CatalogManager) // Note that duplicated attributes are allowed within a single node, // e.g., df.select($"a", $"a"), so we should only check conflicting // attributes between nodes. - val uniqueAttrs = mutable.HashSet[Attribute]() - p.children.head.outputSet.foreach(uniqueAttrs.add) + val uniqueAttrs = mutable.HashSet[ExprId]() + p.children.head.outputSet.foreach(a => uniqueAttrs.add(a.exprId)) p.children.tail.exists { child => val uniqueSize = uniqueAttrs.size val childSize = child.outputSet.size - child.outputSet.foreach(uniqueAttrs.add) + child.outputSet.foreach(a => uniqueAttrs.add(a.exprId)) uniqueSize + childSize > uniqueAttrs.size } } From a04c4e7b79f9ad6ae23a422e53c0f3789c6195ba Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Fri, 26 Mar 2021 16:22:24 +0800 Subject: [PATCH 33/38] skip DeduplicateRelations if no conflicting attributes --- .../sql/catalyst/analysis/Analyzer.scala | 17 +--- .../analysis/DeduplicateRelations.scala | 81 ++++++++++++------- 2 files changed, 51 insertions(+), 47 deletions(-) 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 491490486fd7b..78797e90a314b 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 @@ -27,6 +27,7 @@ import scala.util.Random import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst._ +import org.apache.spark.sql.catalyst.analysis.DeduplicateRelations.hasConflictingAttrs import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders.OuterScopes import org.apache.spark.sql.catalyst.expressions.{FrameLessOffsetWindowFunction, _} @@ -1432,22 +1433,6 @@ class Analyzer(override val catalogManager: CatalogManager) */ object ResolveReferences extends Rule[LogicalPlan] { - private def hasConflictingAttrs(p: LogicalPlan): Boolean = { - p.children.length > 1 && { - // Note that duplicated attributes are allowed within a single node, - // e.g., df.select($"a", $"a"), so we should only check conflicting - // attributes between nodes. - val uniqueAttrs = mutable.HashSet[ExprId]() - p.children.head.outputSet.foreach(a => uniqueAttrs.add(a.exprId)) - p.children.tail.exists { child => - val uniqueSize = uniqueAttrs.size - val childSize = child.outputSet.size - child.outputSet.foreach(a => uniqueAttrs.add(a.exprId)) - uniqueSize + childSize > uniqueAttrs.size - } - } - } - def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { case p: LogicalPlan if !p.childrenResolved => p diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index 306c63010d49c..22ef9fc65bd6e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -17,45 +17,64 @@ package org.apache.spark.sql.catalyst.analysis -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, HashSet} -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, AttributeSet, NamedExpression, PlanExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, AttributeSet, ExprId, NamedExpression, PlanExpression} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule object DeduplicateRelations extends Rule[LogicalPlan] { + /** Return true if there're conflicting attributes among children of a plan */ + def hasConflictingAttrs(p: LogicalPlan): Boolean = { + p.children.length > 1 && { + // Note that duplicated attributes are allowed within a single node, + // e.g., df.select($"a", $"a"), so we should only check conflicting + // attributes between nodes. + val uniqueAttrs = HashSet[ExprId]() + p.children.head.outputSet.foreach(a => uniqueAttrs.add(a.exprId)) + p.children.tail.exists { child => + val uniqueSize = uniqueAttrs.size + val childSize = child.outputSet.size + child.outputSet.foreach(a => uniqueAttrs.add(a.exprId)) + uniqueSize + childSize > uniqueAttrs.size + } + } + } + override def apply(plan: LogicalPlan): LogicalPlan = { - renewDuplicatedRelations(Nil, plan)._1.resolveOperatorsUp { - case p: LogicalPlan if !p.childrenResolved => p - // To resolve duplicate expression IDs for Join. - case j @ Join(left, right, _, _, _) if !j.duplicateResolved => - j.copy(right = dedupRight(left, right)) - // intersect/except will be rewritten to join at the beginning of optimizer. Here we need to - // deduplicate the right side plan, so that we won't produce an invalid self-join later. - case i @ Intersect(left, right, _) if !i.duplicateResolved => - i.copy(right = dedupRight(left, right)) - case e @ Except(left, right, _) if !e.duplicateResolved => - e.copy(right = dedupRight(left, right)) - // Only after we finish by-name resolution for Union - case u: Union if !u.byName && !u.duplicateResolved => - // Use projection-based de-duplication for Union to avoid breaking the checkpoint sharing - // feature in streaming. - val newChildren = u.children.foldRight(Seq.empty[LogicalPlan]) { (head, tail) => - head +: tail.map { - case child if head.outputSet.intersect(child.outputSet).isEmpty => - child - case child => - val projectList = child.output.map { attr => - Alias(attr, attr.name)() - } - Project(projectList, child) + case p: LogicalPlan if !hasConflictingAttrs(p) => p + case _ => + renewDuplicatedRelations(Nil, plan)._1.resolveOperatorsUp { + case p: LogicalPlan if !p.childrenResolved => p + // To resolve duplicate expression IDs for Join. + case j @ Join(left, right, _, _, _) if !j.duplicateResolved => + j.copy(right = dedupRight(left, right)) + // intersect/except will be rewritten to join at the beginning of optimizer. Here we need to + // deduplicate the right side plan, so that we won't produce an invalid self-join later. + case i @ Intersect(left, right, _) if !i.duplicateResolved => + i.copy(right = dedupRight(left, right)) + case e @ Except(left, right, _) if !e.duplicateResolved => + e.copy(right = dedupRight(left, right)) + // Only after we finish by-name resolution for Union + case u: Union if !u.byName && !u.duplicateResolved => + // Use projection-based de-duplication for Union to avoid breaking the checkpoint sharing + // feature in streaming. + val newChildren = u.children.foldRight(Seq.empty[LogicalPlan]) { (head, tail) => + head +: tail.map { + case child if head.outputSet.intersect(child.outputSet).isEmpty => + child + case child => + val projectList = child.output.map { attr => + Alias(attr, attr.name)() + } + Project(projectList, child) + } } - } - u.copy(children = newChildren) - case m @ MergeIntoTable(targetTable, sourceTable, _, _, _) if !m.duplicateResolved => - m.copy(sourceTable = dedupRight(targetTable, sourceTable)) - } + u.copy(children = newChildren) + case m @ MergeIntoTable(targetTable, sourceTable, _, _, _) if !m.duplicateResolved => + m.copy(sourceTable = dedupRight(targetTable, sourceTable)) + } } /** From b27d4d3b350fb46477ba6092d59a2d7ebd68d8a0 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Fri, 26 Mar 2021 16:22:52 +0800 Subject: [PATCH 34/38] remove uncessary pattern match --- .../spark/sql/catalyst/analysis/DeduplicateRelations.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index 22ef9fc65bd6e..f4a50a609a9dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -131,8 +131,6 @@ object DeduplicateRelations extends Rule[LogicalPlan] { subquery.withNewPlan(renewed) } (planWithNewSubquery, relations) - - case _ => (plan, Nil) } private def isDuplicated( From 6243b43160c62b0f4eb3b2ee767be11b42f5f0b1 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Fri, 26 Mar 2021 16:58:44 +0800 Subject: [PATCH 35/38] revert skip if no conflicting attrs --- .../sql/catalyst/analysis/Analyzer.scala | 18 +++- .../analysis/DeduplicateRelations.scala | 82 +++++++------------ 2 files changed, 48 insertions(+), 52 deletions(-) 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 0f238bbadea8e..f306d0f116358 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 @@ -27,7 +27,6 @@ import scala.util.Random import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst._ -import org.apache.spark.sql.catalyst.analysis.DeduplicateRelations.hasConflictingAttrs import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders.OuterScopes import org.apache.spark.sql.catalyst.expressions.{FrameLessOffsetWindowFunction, _} @@ -1436,6 +1435,23 @@ class Analyzer(override val catalogManager: CatalogManager) */ object ResolveReferences extends Rule[LogicalPlan] { + /** Return true if there're conflicting attributes among children's outputs of a plan */ + def hasConflictingAttrs(p: LogicalPlan): Boolean = { + p.children.length > 1 && { + // Note that duplicated attributes are allowed within a single node, + // e.g., df.select($"a", $"a"), so we should only check conflicting + // attributes between nodes. + val uniqueAttrs = mutable.HashSet[ExprId]() + p.children.head.outputSet.foreach(a => uniqueAttrs.add(a.exprId)) + p.children.tail.exists { child => + val uniqueSize = uniqueAttrs.size + val childSize = child.outputSet.size + child.outputSet.foreach(a => uniqueAttrs.add(a.exprId)) + uniqueSize + childSize > uniqueAttrs.size + } + } + } + def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { case p: LogicalPlan if !p.childrenResolved => p diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index f4a50a609a9dd..e4a709c1cdf21 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -17,64 +17,44 @@ package org.apache.spark.sql.catalyst.analysis -import scala.collection.mutable.{ArrayBuffer, HashSet} +import scala.collection.mutable.ArrayBuffer -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, AttributeSet, ExprId, NamedExpression, PlanExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, AttributeSet, NamedExpression, PlanExpression} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule object DeduplicateRelations extends Rule[LogicalPlan] { - - /** Return true if there're conflicting attributes among children of a plan */ - def hasConflictingAttrs(p: LogicalPlan): Boolean = { - p.children.length > 1 && { - // Note that duplicated attributes are allowed within a single node, - // e.g., df.select($"a", $"a"), so we should only check conflicting - // attributes between nodes. - val uniqueAttrs = HashSet[ExprId]() - p.children.head.outputSet.foreach(a => uniqueAttrs.add(a.exprId)) - p.children.tail.exists { child => - val uniqueSize = uniqueAttrs.size - val childSize = child.outputSet.size - child.outputSet.foreach(a => uniqueAttrs.add(a.exprId)) - uniqueSize + childSize > uniqueAttrs.size - } - } - } - override def apply(plan: LogicalPlan): LogicalPlan = { - case p: LogicalPlan if !hasConflictingAttrs(p) => p - case _ => - renewDuplicatedRelations(Nil, plan)._1.resolveOperatorsUp { - case p: LogicalPlan if !p.childrenResolved => p - // To resolve duplicate expression IDs for Join. - case j @ Join(left, right, _, _, _) if !j.duplicateResolved => - j.copy(right = dedupRight(left, right)) - // intersect/except will be rewritten to join at the beginning of optimizer. Here we need to - // deduplicate the right side plan, so that we won't produce an invalid self-join later. - case i @ Intersect(left, right, _) if !i.duplicateResolved => - i.copy(right = dedupRight(left, right)) - case e @ Except(left, right, _) if !e.duplicateResolved => - e.copy(right = dedupRight(left, right)) - // Only after we finish by-name resolution for Union - case u: Union if !u.byName && !u.duplicateResolved => - // Use projection-based de-duplication for Union to avoid breaking the checkpoint sharing - // feature in streaming. - val newChildren = u.children.foldRight(Seq.empty[LogicalPlan]) { (head, tail) => - head +: tail.map { - case child if head.outputSet.intersect(child.outputSet).isEmpty => - child - case child => - val projectList = child.output.map { attr => - Alias(attr, attr.name)() - } - Project(projectList, child) - } + renewDuplicatedRelations(Nil, plan)._1.resolveOperatorsUp { + case p: LogicalPlan if !p.childrenResolved => p + // To resolve duplicate expression IDs for Join. + case j @ Join(left, right, _, _, _) if !j.duplicateResolved => + j.copy(right = dedupRight(left, right)) + // intersect/except will be rewritten to join at the beginning of optimizer. Here we need to + // deduplicate the right side plan, so that we won't produce an invalid self-join later. + case i @ Intersect(left, right, _) if !i.duplicateResolved => + i.copy(right = dedupRight(left, right)) + case e @ Except(left, right, _) if !e.duplicateResolved => + e.copy(right = dedupRight(left, right)) + // Only after we finish by-name resolution for Union + case u: Union if !u.byName && !u.duplicateResolved => + // Use projection-based de-duplication for Union to avoid breaking the checkpoint sharing + // feature in streaming. + val newChildren = u.children.foldRight(Seq.empty[LogicalPlan]) { (head, tail) => + head +: tail.map { + case child if head.outputSet.intersect(child.outputSet).isEmpty => + child + case child => + val projectList = child.output.map { attr => + Alias(attr, attr.name)() + } + Project(projectList, child) } - u.copy(children = newChildren) - case m @ MergeIntoTable(targetTable, sourceTable, _, _, _) if !m.duplicateResolved => - m.copy(sourceTable = dedupRight(targetTable, sourceTable)) - } + } + u.copy(children = newChildren) + case m @ MergeIntoTable(targetTable, sourceTable, _, _, _) if !m.duplicateResolved => + m.copy(sourceTable = dedupRight(targetTable, sourceTable)) + } } /** From c6f9714bc1711708739af5398e0fbe56064037cc Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Fri, 26 Mar 2021 17:12:13 +0800 Subject: [PATCH 36/38] gen golden file --- .../q10.sf100/explain.txt | 102 ++-- .../approved-plans-modified/q10/explain.txt | 100 ++-- .../q27.sf100/explain.txt | 162 +++--- .../approved-plans-modified/q27/explain.txt | 162 +++--- .../q46.sf100/explain.txt | 18 +- .../approved-plans-modified/q46/explain.txt | 14 +- .../q59.sf100/explain.txt | 102 ++-- .../approved-plans-modified/q59/explain.txt | 102 ++-- .../q65.sf100/explain.txt | 86 +-- .../approved-plans-modified/q65/explain.txt | 48 +- .../q68.sf100/explain.txt | 20 +- .../approved-plans-modified/q68/explain.txt | 14 +- .../approved-plans-v1_4/q1.sf100/explain.txt | 122 ++-- .../approved-plans-v1_4/q1/explain.txt | 116 ++-- .../approved-plans-v1_4/q10.sf100/explain.txt | 104 ++-- .../approved-plans-v1_4/q10/explain.txt | 94 ++-- .../approved-plans-v1_4/q11.sf100/explain.txt | 272 ++++----- .../approved-plans-v1_4/q11/explain.txt | 254 ++++----- .../approved-plans-v1_4/q16.sf100/explain.txt | 128 ++--- .../approved-plans-v1_4/q16/explain.txt | 128 ++--- .../approved-plans-v1_4/q2.sf100/explain.txt | 52 +- .../approved-plans-v1_4/q2/explain.txt | 52 +- .../q23a.sf100/explain.txt | 362 ++++++------ .../approved-plans-v1_4/q23a/explain.txt | 274 ++++----- .../q23b.sf100/explain.txt | 528 +++++++++--------- .../approved-plans-v1_4/q23b/explain.txt | 404 +++++++------- .../q24a.sf100/explain.txt | 206 +++---- .../approved-plans-v1_4/q24a/explain.txt | 170 +++--- .../q24b.sf100/explain.txt | 206 +++---- .../approved-plans-v1_4/q24b/explain.txt | 170 +++--- .../approved-plans-v1_4/q28.sf100/explain.txt | 300 +++++----- .../approved-plans-v1_4/q28/explain.txt | 300 +++++----- .../approved-plans-v1_4/q30.sf100/explain.txt | 142 ++--- .../approved-plans-v1_4/q30/explain.txt | 130 ++--- .../approved-plans-v1_4/q31.sf100/explain.txt | 430 +++++++------- .../approved-plans-v1_4/q31/explain.txt | 362 ++++++------ .../approved-plans-v1_4/q33.sf100/explain.txt | 138 ++--- .../approved-plans-v1_4/q33/explain.txt | 138 ++--- .../approved-plans-v1_4/q35.sf100/explain.txt | 116 ++-- .../approved-plans-v1_4/q35/explain.txt | 96 ++-- .../q39a.sf100/explain.txt | 108 ++-- .../approved-plans-v1_4/q39a/explain.txt | 104 ++-- .../q39b.sf100/explain.txt | 108 ++-- .../approved-plans-v1_4/q39b/explain.txt | 104 ++-- .../approved-plans-v1_4/q4.sf100/explain.txt | 442 +++++++-------- .../approved-plans-v1_4/q4/explain.txt | 412 +++++++------- .../approved-plans-v1_4/q44.sf100/explain.txt | 118 ++-- .../approved-plans-v1_4/q44/explain.txt | 118 ++-- .../approved-plans-v1_4/q45.sf100/explain.txt | 36 +- .../approved-plans-v1_4/q45/explain.txt | 36 +- .../approved-plans-v1_4/q46.sf100/explain.txt | 44 +- .../approved-plans-v1_4/q46/explain.txt | 14 +- .../approved-plans-v1_4/q47.sf100/explain.txt | 66 +-- .../approved-plans-v1_4/q47/explain.txt | 58 +- .../approved-plans-v1_4/q49.sf100/explain.txt | 198 +++---- .../approved-plans-v1_4/q49/explain.txt | 152 ++--- .../approved-plans-v1_4/q5.sf100/explain.txt | 162 +++--- .../approved-plans-v1_4/q5/explain.txt | 174 +++--- .../approved-plans-v1_4/q54.sf100/explain.txt | 118 ++-- .../approved-plans-v1_4/q54/explain.txt | 120 ++-- .../approved-plans-v1_4/q56.sf100/explain.txt | 140 ++--- .../approved-plans-v1_4/q56/explain.txt | 140 ++--- .../approved-plans-v1_4/q57.sf100/explain.txt | 66 +-- .../approved-plans-v1_4/q57/explain.txt | 58 +- .../approved-plans-v1_4/q58.sf100/explain.txt | 226 ++++---- .../approved-plans-v1_4/q58/explain.txt | 228 ++++---- .../approved-plans-v1_4/q59.sf100/explain.txt | 70 +-- .../approved-plans-v1_4/q59/explain.txt | 70 +-- .../approved-plans-v1_4/q60.sf100/explain.txt | 140 ++--- .../approved-plans-v1_4/q60/explain.txt | 140 ++--- .../approved-plans-v1_4/q61.sf100/explain.txt | 84 +-- .../approved-plans-v1_4/q61/explain.txt | 94 ++-- .../approved-plans-v1_4/q64.sf100/explain.txt | 316 +++++------ .../approved-plans-v1_4/q64/explain.txt | 284 +++++----- .../approved-plans-v1_4/q65.sf100/explain.txt | 98 ++-- .../approved-plans-v1_4/q65/explain.txt | 48 +- .../approved-plans-v1_4/q66.sf100/explain.txt | 70 +-- .../approved-plans-v1_4/q66/explain.txt | 74 +-- .../approved-plans-v1_4/q68.sf100/explain.txt | 44 +- .../approved-plans-v1_4/q68/explain.txt | 14 +- .../approved-plans-v1_4/q69.sf100/explain.txt | 96 ++-- .../approved-plans-v1_4/q69/explain.txt | 94 ++-- .../approved-plans-v1_4/q70.sf100/explain.txt | 136 ++--- .../approved-plans-v1_4/q70/explain.txt | 136 ++--- .../approved-plans-v1_4/q71.sf100/explain.txt | 80 +-- .../approved-plans-v1_4/q71/explain.txt | 80 +-- .../approved-plans-v1_4/q74.sf100/explain.txt | 268 ++++----- .../approved-plans-v1_4/q74/explain.txt | 250 ++++----- .../approved-plans-v1_4/q75.sf100/explain.txt | 368 ++++++------ .../approved-plans-v1_4/q75/explain.txt | 368 ++++++------ .../approved-plans-v1_4/q76.sf100/explain.txt | 80 +-- .../approved-plans-v1_4/q76/explain.txt | 64 +-- .../approved-plans-v1_4/q77.sf100/explain.txt | 284 +++++----- .../approved-plans-v1_4/q77/explain.txt | 300 +++++----- .../approved-plans-v1_4/q78.sf100/explain.txt | 146 ++--- .../approved-plans-v1_4/q78/explain.txt | 146 ++--- .../approved-plans-v1_4/q8.sf100/explain.txt | 120 ++-- .../approved-plans-v1_4/q8/explain.txt | 106 ++-- .../approved-plans-v1_4/q80.sf100/explain.txt | 204 +++---- .../approved-plans-v1_4/q80/explain.txt | 208 +++---- .../approved-plans-v1_4/q81.sf100/explain.txt | 144 ++--- .../approved-plans-v1_4/q81/explain.txt | 126 ++--- .../approved-plans-v1_4/q83.sf100/explain.txt | 210 +++---- .../approved-plans-v1_4/q83/explain.txt | 186 +++--- .../approved-plans-v1_4/q88.sf100/explain.txt | 504 ++++++++--------- .../approved-plans-v1_4/q88/explain.txt | 504 ++++++++--------- .../approved-plans-v1_4/q9.sf100/explain.txt | 416 +++++++------- .../approved-plans-v1_4/q9/explain.txt | 416 +++++++------- .../approved-plans-v1_4/q90.sf100/explain.txt | 80 +-- .../approved-plans-v1_4/q90/explain.txt | 80 +-- .../approved-plans-v1_4/q94.sf100/explain.txt | 134 ++--- .../approved-plans-v1_4/q94/explain.txt | 134 ++--- .../approved-plans-v1_4/q95.sf100/explain.txt | 194 +++---- .../approved-plans-v1_4/q95/explain.txt | 194 +++---- .../approved-plans-v1_4/q97.sf100/explain.txt | 40 +- .../approved-plans-v1_4/q97/explain.txt | 40 +- .../q10a.sf100/explain.txt | 96 ++-- .../approved-plans-v2_7/q10a/explain.txt | 94 ++-- .../approved-plans-v2_7/q11.sf100/explain.txt | 268 ++++----- .../approved-plans-v2_7/q11/explain.txt | 250 ++++----- .../q18a.sf100/explain.txt | 402 ++++++------- .../approved-plans-v2_7/q18a/explain.txt | 410 +++++++------- .../q22a.sf100/explain.txt | 122 ++-- .../approved-plans-v2_7/q22a/explain.txt | 122 ++-- .../approved-plans-v2_7/q24.sf100/explain.txt | 182 +++--- .../approved-plans-v2_7/q24/explain.txt | 170 +++--- .../q27a.sf100/explain.txt | 162 +++--- .../approved-plans-v2_7/q27a/explain.txt | 162 +++--- .../approved-plans-v2_7/q35.sf100/explain.txt | 116 ++-- .../approved-plans-v2_7/q35/explain.txt | 96 ++-- .../q35a.sf100/explain.txt | 116 ++-- .../approved-plans-v2_7/q35a/explain.txt | 96 ++-- .../q36a.sf100/explain.txt | 92 +-- .../approved-plans-v2_7/q36a/explain.txt | 92 +-- .../approved-plans-v2_7/q47.sf100/explain.txt | 66 +-- .../approved-plans-v2_7/q47/explain.txt | 58 +- .../approved-plans-v2_7/q49.sf100/explain.txt | 198 +++---- .../approved-plans-v2_7/q49/explain.txt | 152 ++--- .../q51a.sf100/explain.txt | 244 ++++---- .../approved-plans-v2_7/q51a/explain.txt | 242 ++++---- .../approved-plans-v2_7/q57.sf100/explain.txt | 66 +-- .../approved-plans-v2_7/q57/explain.txt | 58 +- .../approved-plans-v2_7/q5a.sf100/explain.txt | 230 ++++---- .../approved-plans-v2_7/q5a/explain.txt | 242 ++++---- .../approved-plans-v2_7/q64.sf100/explain.txt | 316 +++++------ .../approved-plans-v2_7/q64/explain.txt | 284 +++++----- .../q67a.sf100/explain.txt | 296 +++++----- .../approved-plans-v2_7/q67a/explain.txt | 296 +++++----- .../q70a.sf100/explain.txt | 212 +++---- .../approved-plans-v2_7/q70a/explain.txt | 212 +++---- .../approved-plans-v2_7/q74.sf100/explain.txt | 268 ++++----- .../approved-plans-v2_7/q74/explain.txt | 250 ++++----- .../approved-plans-v2_7/q75.sf100/explain.txt | 368 ++++++------ .../approved-plans-v2_7/q75/explain.txt | 368 ++++++------ .../q77a.sf100/explain.txt | 364 ++++++------ .../approved-plans-v2_7/q77a/explain.txt | 380 ++++++------- .../approved-plans-v2_7/q78.sf100/explain.txt | 146 ++--- .../approved-plans-v2_7/q78/explain.txt | 146 ++--- .../q80a.sf100/explain.txt | 268 ++++----- .../approved-plans-v2_7/q80a/explain.txt | 272 ++++----- .../q86a.sf100/explain.txt | 92 +-- .../approved-plans-v2_7/q86a/explain.txt | 92 +-- 162 files changed, 13995 insertions(+), 13995 deletions(-) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt index 8a4b341ac925b..1fdcba1ed4de4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt @@ -137,22 +137,22 @@ Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] Condition : isnotnull(cs_ship_customer_sk#13) (19) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#15] (20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [cs_sold_date_sk#14] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (21) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#13 AS customer_sk#15] -Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#8] +Output [1]: [cs_ship_customer_sk#13 AS customer_sk#16] +Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#15] (22) Union (23) Exchange Input [1]: [customer_sk#12] -Arguments: hashpartitioning(customer_sk#12, 5), ENSURE_REQUIREMENTS, [id=#16] +Arguments: hashpartitioning(customer_sk#12, 5), ENSURE_REQUIREMENTS, [id=#17] (24) Sort [codegen id : 7] Input [1]: [customer_sk#12] @@ -164,43 +164,43 @@ Right keys [1]: [customer_sk#12] Join condition: None (26) Scan parquet default.store_sales -Output [2]: [ss_customer_sk#17, ss_sold_date_sk#18] +Output [2]: [ss_customer_sk#18, ss_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#19), dynamicpruningexpression(ss_sold_date_sk#19 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 9] -Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] +Input [2]: [ss_customer_sk#18, ss_sold_date_sk#19] (28) Filter [codegen id : 9] -Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] -Condition : isnotnull(ss_customer_sk#17) +Input [2]: [ss_customer_sk#18, ss_sold_date_sk#19] +Condition : isnotnull(ss_customer_sk#18) (29) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#20] (30) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_sold_date_sk#18] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ss_sold_date_sk#19] +Right keys [1]: [d_date_sk#20] Join condition: None (31) Project [codegen id : 9] -Output [1]: [ss_customer_sk#17 AS customer_sk#19] -Input [3]: [ss_customer_sk#17, ss_sold_date_sk#18, d_date_sk#8] +Output [1]: [ss_customer_sk#18 AS customer_sk#21] +Input [3]: [ss_customer_sk#18, ss_sold_date_sk#19, d_date_sk#20] (32) Exchange -Input [1]: [customer_sk#19] -Arguments: hashpartitioning(customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [customer_sk#21] +Arguments: hashpartitioning(customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#22] (33) Sort [codegen id : 10] -Input [1]: [customer_sk#19] -Arguments: [customer_sk#19 ASC NULLS FIRST], false, 0 +Input [1]: [customer_sk#21] +Arguments: [customer_sk#21 ASC NULLS FIRST], false, 0 (34) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#19] +Right keys [1]: [customer_sk#21] Join condition: None (35) Project [codegen id : 12] @@ -208,84 +208,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (36) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_county#22] +Output [2]: [ca_address_sk#23, ca_county#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 11] -Input [2]: [ca_address_sk#21, ca_county#22] +Input [2]: [ca_address_sk#23, ca_county#24] (38) Filter [codegen id : 11] -Input [2]: [ca_address_sk#21, ca_county#22] -Condition : (ca_county#22 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#21)) +Input [2]: [ca_address_sk#23, ca_county#24] +Condition : (ca_county#24 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#23)) (39) Project [codegen id : 11] -Output [1]: [ca_address_sk#21] -Input [2]: [ca_address_sk#21, ca_county#22] +Output [1]: [ca_address_sk#23] +Input [2]: [ca_address_sk#23, ca_county#24] (40) BroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [ca_address_sk#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#23] Join condition: None (42) Project [codegen id : 12] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#23] (43) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (44) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (45) ColumnarToRow -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] (46) Filter -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Condition : isnotnull(cd_demo_sk#25) +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Condition : isnotnull(cd_demo_sk#27) (47) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#27] Join condition: None (48) Project [codegen id : 13] -Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] (49) HashAggregate [codegen id : 13] -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#34] -Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#35] +Aggregate Attributes [1]: [count#36] +Results [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] (50) Exchange -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#35] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] +Arguments: hashpartitioning(cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, 5), ENSURE_REQUIREMENTS, [id=#38] (51) HashAggregate [codegen id : 14] -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#35] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#37] +Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#37] -Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#37 AS cnt1#38, cd_purchase_estimate#29, count(1)#37 AS cnt2#39, cd_credit_rating#30, count(1)#37 AS cnt3#40, cd_dep_count#31, count(1)#37 AS cnt4#41, cd_dep_employed_count#32, count(1)#37 AS cnt5#42, cd_dep_college_count#33, count(1)#37 AS cnt6#43] +Aggregate Attributes [1]: [count(1)#39] +Results [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, count(1)#39 AS cnt1#40, cd_purchase_estimate#31, count(1)#39 AS cnt2#41, cd_credit_rating#32, count(1)#39 AS cnt3#42, cd_dep_count#33, count(1)#39 AS cnt4#43, cd_dep_employed_count#34, count(1)#39 AS cnt5#44, cd_dep_college_count#35, count(1)#39 AS cnt6#45] (52) TakeOrderedAndProject -Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#38, cd_purchase_estimate#29, cnt2#39, cd_credit_rating#30, cnt3#40, cd_dep_count#31, cnt4#41, cd_dep_employed_count#32, cnt5#42, cd_dep_college_count#33, cnt6#43] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#38, cd_purchase_estimate#29, cnt2#39, cd_credit_rating#30, cnt3#40, cd_dep_count#31, cnt4#41, cd_dep_employed_count#32, cnt5#42, cd_dep_college_count#33, cnt6#43] +Input [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#40, cd_purchase_estimate#31, cnt2#41, cd_credit_rating#32, cnt3#42, cd_dep_count#33, cnt4#43, cd_dep_employed_count#34, cnt5#44, cd_dep_college_count#35, cnt6#45] +Arguments: 100, [cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_education_status#30 ASC NULLS FIRST, cd_purchase_estimate#31 ASC NULLS FIRST, cd_credit_rating#32 ASC NULLS FIRST, cd_dep_count#33 ASC NULLS FIRST, cd_dep_employed_count#34 ASC NULLS FIRST, cd_dep_college_count#35 ASC NULLS FIRST], [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#40, cd_purchase_estimate#31, cnt2#41, cd_credit_rating#32, cnt3#42, cd_dep_count#33, cnt4#43, cd_dep_employed_count#34, cnt5#44, cd_dep_college_count#35, cnt6#45] ===== Subqueries ===== @@ -298,6 +298,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#19 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt index 6f64579d09c78..ff55b1c511456 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt @@ -125,22 +125,22 @@ Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] Condition : isnotnull(cs_ship_customer_sk#12) (17) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] -Output [1]: [cs_ship_customer_sk#12 AS customer_sk#14] -Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#7] +Output [1]: [cs_ship_customer_sk#12 AS customer_sk#15] +Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#14] (20) Union (21) BroadcastExchange Input [1]: [customer_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (22) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] @@ -148,39 +148,39 @@ Right keys [1]: [customer_sk#11] Join condition: None (23) Scan parquet default.store_sales -Output [2]: [ss_customer_sk#16, ss_sold_date_sk#17] +Output [2]: [ss_customer_sk#17, ss_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#17), dynamicpruningexpression(ss_sold_date_sk#17 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ss_sold_date_sk#18), dynamicpruningexpression(ss_sold_date_sk#18 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [2]: [ss_customer_sk#16, ss_sold_date_sk#17] +Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] (25) Filter [codegen id : 6] -Input [2]: [ss_customer_sk#16, ss_sold_date_sk#17] -Condition : isnotnull(ss_customer_sk#16) +Input [2]: [ss_customer_sk#17, ss_sold_date_sk#18] +Condition : isnotnull(ss_customer_sk#17) (26) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#19] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#17] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [ss_sold_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (28) Project [codegen id : 6] -Output [1]: [ss_customer_sk#16 AS customer_sk#18] -Input [3]: [ss_customer_sk#16, ss_sold_date_sk#17, d_date_sk#7] +Output [1]: [ss_customer_sk#17 AS customer_sk#20] +Input [3]: [ss_customer_sk#17, ss_sold_date_sk#18, d_date_sk#19] (29) BroadcastExchange -Input [1]: [customer_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [customer_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (30) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#18] +Right keys [1]: [customer_sk#20] Join condition: None (31) Project [codegen id : 9] @@ -188,84 +188,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (32) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] +Output [2]: [ca_address_sk#22, ca_county#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (33) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_county#21] +Input [2]: [ca_address_sk#22, ca_county#23] (34) Filter [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) +Input [2]: [ca_address_sk#22, ca_county#23] +Condition : (ca_county#23 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#22)) (35) Project [codegen id : 7] -Output [1]: [ca_address_sk#20] -Input [2]: [ca_address_sk#20, ca_county#21] +Output [1]: [ca_address_sk#22] +Input [2]: [ca_address_sk#22, ca_county#23] (36) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [ca_address_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#22] Join condition: None (38) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#22] (39) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (41) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Condition : isnotnull(cd_demo_sk#23) +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Condition : isnotnull(cd_demo_sk#25) (42) BroadcastExchange -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (43) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#25] Join condition: None (44) Project [codegen id : 9] -Output [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (45) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] +Aggregate Attributes [1]: [count#35] +Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] (46) Exchange -Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] -Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] (47) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] -Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#36] -Results [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#36 AS cnt1#37, cd_purchase_estimate#27, count(1)#36 AS cnt2#38, cd_credit_rating#28, count(1)#36 AS cnt3#39, cd_dep_count#29, count(1)#36 AS cnt4#40, cd_dep_employed_count#30, count(1)#36 AS cnt5#41, cd_dep_college_count#31, count(1)#36 AS cnt6#42] +Aggregate Attributes [1]: [count(1)#38] +Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] (48) TakeOrderedAndProject -Input [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] -Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] +Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] +Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] ===== Subqueries ===== @@ -278,6 +278,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 14 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#17 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#6 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 baed833e42a47..8f7cbc7d0ffbc 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -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)] +PartitionFilters: [isnotnull(ss_sold_date_sk#61), (ss_sold_date_sk#61 >= 2451545), (ss_sold_date_sk#61 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -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] +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] (35) Filter [codegen id : 11] -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)) +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#62] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#61] +Right keys [1]: [d_date_sk#62] Join condition: None (38) Project [codegen id : 11] -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] +Output [7]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#62] (39) Scan parquet default.store -Output [2]: [s_store_sk#18, s_state#19] +Output [2]: [s_store_sk#63, s_state#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_state, [TN,AL,SD]), IsNotNull(s_store_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 8] -Input [2]: [s_store_sk#18, s_state#19] +Input [2]: [s_store_sk#63, s_state#64] (41) Filter [codegen id : 8] -Input [2]: [s_store_sk#18, s_state#19] -Condition : (s_state#19 IN (TN,AL,SD) AND isnotnull(s_store_sk#18)) +Input [2]: [s_store_sk#63, s_state#64] +Condition : (s_state#64 IN (TN,AL,SD) AND isnotnull(s_store_sk#63)) (42) Project [codegen id : 8] -Output [1]: [s_store_sk#18] -Input [2]: [s_store_sk#18, s_state#19] +Output [1]: [s_store_sk#63] +Input [2]: [s_store_sk#63, s_state#64] (43) BroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [s_store_sk#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] (44) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#56] +Right keys [1]: [s_store_sk#63] Join condition: None (45) Project [codegen id : 11] -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#18] +Output [6]: [ss_item_sk#54, ss_cdemo_sk#55, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, s_store_sk#63] (46) ReusedExchange [Reuses operator id: 15] -Output [1]: [cd_demo_sk#13] +Output [1]: [cd_demo_sk#66] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#13] +Left keys [1]: [ss_cdemo_sk#55] +Right keys [1]: [cd_demo_sk#66] Join condition: None (48) Project [codegen id : 11] -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#13] +Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [7]: [ss_item_sk#54, ss_cdemo_sk#55, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, cd_demo_sk#66] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#21, i_item_id#22] +Output [2]: [i_item_sk#67, i_item_id#68] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#67] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] +Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] +Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#22] +Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#68] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] +Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] (53) Exchange -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [1]: [i_item_id#22] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Keys [1]: [i_item_id#68] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] -Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] +Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] +Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] (55) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -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)] +PartitionFilters: [isnotnull(ss_sold_date_sk#103), (ss_sold_date_sk#103 >= 2451545), (ss_sold_date_sk#103 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 17] -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] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] (57) Filter [codegen id : 17] -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)) +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#104] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#103] +Right keys [1]: [d_date_sk#104] Join condition: None (60) Project [codegen id : 17] -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] +Output [7]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#104] (61) ReusedExchange [Reuses operator id: 43] -Output [1]: [s_store_sk#18] +Output [1]: [s_store_sk#105] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#98] +Right keys [1]: [s_store_sk#105] Join condition: None (63) Project [codegen id : 17] -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#18] +Output [6]: [ss_item_sk#96, ss_cdemo_sk#97, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#105] (64) ReusedExchange [Reuses operator id: 15] -Output [1]: [cd_demo_sk#13] +Output [1]: [cd_demo_sk#106] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#13] +Left keys [1]: [ss_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#106] Join condition: None (66) Project [codegen id : 17] -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#13] +Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [7]: [ss_item_sk#96, ss_cdemo_sk#97, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, cd_demo_sk#106] (67) Scan parquet default.item -Output [1]: [i_item_sk#21] +Output [1]: [i_item_sk#107] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#21] +Input [1]: [i_item_sk#107] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#21] -Condition : isnotnull(i_item_sk#21) +Input [1]: [i_item_sk#107] +Condition : isnotnull(i_item_sk#107) (70) BroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] +Input [1]: [i_item_sk#107] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#96] +Right keys [1]: [i_item_sk#107] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] +Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] +Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] -Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] +Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] (74) Exchange -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] -Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] +Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] +Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#10] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 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 735f763ae5d20..bfb9e8384817d 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -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)] +PartitionFilters: [isnotnull(ss_sold_date_sk#61), (ss_sold_date_sk#61 >= 2451545), (ss_sold_date_sk#61 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -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] +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] (35) Filter [codegen id : 11] -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)) +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#62] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#55] +Right keys [1]: [cd_demo_sk#62] Join condition: None (38) Project [codegen id : 11] -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] +Output [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, cd_demo_sk#62] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#63] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#61] +Right keys [1]: [d_date_sk#63] Join condition: None (41) Project [codegen id : 11] -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] +Output [6]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [8]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#63] (42) Scan parquet default.store -Output [2]: [s_store_sk#18, s_state#19] +Output [2]: [s_store_sk#64, s_state#65] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [In(s_state, [TN,AL,SD]), IsNotNull(s_store_sk)] ReadSchema: struct (43) ColumnarToRow [codegen id : 9] -Input [2]: [s_store_sk#18, s_state#19] +Input [2]: [s_store_sk#64, s_state#65] (44) Filter [codegen id : 9] -Input [2]: [s_store_sk#18, s_state#19] -Condition : (s_state#19 IN (TN,AL,SD) AND isnotnull(s_store_sk#18)) +Input [2]: [s_store_sk#64, s_state#65] +Condition : (s_state#65 IN (TN,AL,SD) AND isnotnull(s_store_sk#64)) (45) Project [codegen id : 9] -Output [1]: [s_store_sk#18] -Input [2]: [s_store_sk#18, s_state#19] +Output [1]: [s_store_sk#64] +Input [2]: [s_store_sk#64, s_state#65] (46) BroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [s_store_sk#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#66] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#56] +Right keys [1]: [s_store_sk#64] Join condition: None (48) Project [codegen id : 11] -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#18] +Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, s_store_sk#64] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#21, i_item_id#22] +Output [2]: [i_item_sk#67, i_item_id#68] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#67] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] +Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] +Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#22] +Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#68] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] +Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] (53) Exchange -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [1]: [i_item_id#22] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Keys [1]: [i_item_id#68] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] -Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] +Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] +Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] (55) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -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)] +PartitionFilters: [isnotnull(ss_sold_date_sk#103), (ss_sold_date_sk#103 >= 2451545), (ss_sold_date_sk#103 <= 2451910), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 17] -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] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] (57) Filter [codegen id : 17] -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)) +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#104] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#104] Join condition: None (60) Project [codegen id : 17] -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] +Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] (61) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#105] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#103] +Right keys [1]: [d_date_sk#105] Join condition: None (63) Project [codegen id : 17] -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] +Output [6]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#105] (64) ReusedExchange [Reuses operator id: 46] -Output [1]: [s_store_sk#18] +Output [1]: [s_store_sk#106] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#98] +Right keys [1]: [s_store_sk#106] Join condition: None (66) Project [codegen id : 17] -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#18] +Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#106] (67) Scan parquet default.item -Output [1]: [i_item_sk#21] +Output [1]: [i_item_sk#107] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#21] +Input [1]: [i_item_sk#107] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#21] -Condition : isnotnull(i_item_sk#21) +Input [1]: [i_item_sk#107] +Condition : isnotnull(i_item_sk#107) (70) BroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] +Input [1]: [i_item_sk#107] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#96] +Right keys [1]: [i_item_sk#107] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] +Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] +Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] -Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] +Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] (74) Exchange -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] -Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] +Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] +Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#15] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt index 4eb212c336669..e62003f82faec 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt @@ -261,24 +261,24 @@ Input [7]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_add Arguments: [c_current_addr_sk#36 ASC NULLS FIRST], false, 0 (47) ReusedExchange [Reuses operator id: 30] -Output [2]: [ca_address_sk#22, ca_city#23] +Output [2]: [ca_address_sk#41, ca_city#42] (48) Sort [codegen id : 15] -Input [2]: [ca_address_sk#22, ca_city#23] -Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#41, ca_city#42] +Arguments: [ca_address_sk#41 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#36] -Right keys [1]: [ca_address_sk#22] -Join condition: NOT (ca_city#23 = bought_city#31) +Right keys [1]: [ca_address_sk#41] +Join condition: NOT (ca_city#42 = bought_city#31) (50) Project [codegen id : 16] -Output [7]: [c_last_name#38, c_first_name#37, ca_city#23, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#36, c_first_name#37, c_last_name#38, ca_address_sk#22, ca_city#23] +Output [7]: [c_last_name#38, c_first_name#37, ca_city#42, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#36, c_first_name#37, c_last_name#38, ca_address_sk#41, ca_city#42] (51) TakeOrderedAndProject -Input [7]: [c_last_name#38, c_first_name#37, ca_city#23, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Arguments: 100, [c_last_name#38 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, ca_city#23 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#38, c_first_name#37, ca_city#23, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [7]: [c_last_name#38, c_first_name#37, ca_city#42, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Arguments: 100, [c_last_name#38 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, ca_city#42 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#38, c_first_name#37, ca_city#42, bought_city#31, ss_ticket_number#5, amt#32, profit#33] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt index 951558ca1b130..a00e2eeac49b7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt @@ -225,20 +225,20 @@ Output [7]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_ad Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#31, amt#32, profit#33, c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#21, ca_city#22] +Output [2]: [ca_address_sk#39, ca_city#40] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#21] -Join condition: NOT (ca_city#22 = bought_city#31) +Right keys [1]: [ca_address_sk#39] +Join condition: NOT (ca_city#40 = bought_city#31) (42) Project [codegen id : 8] -Output [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#21, ca_city#22] +Output [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#39, ca_city#40] (43) TakeOrderedAndProject -Input [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#22 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#40 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] ===== Subqueries ===== 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 4cdcd9685acbe..d20f7cf4a2857 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#1, ss_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3)] +PartitionFilters: [isnotnull(ss_sold_date_sk#56)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 6] -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] (28) Filter [codegen id : 6] -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] +Condition : isnotnull(ss_store_sk#54) (29) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] +Output [3]: [d_date_sk#57, d_week_seq#58, d_day_name#59] (30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#4] +Left keys [1]: [ss_sold_date_sk#56] +Right keys [1]: [d_date_sk#57] Join condition: None (31) Project [codegen id : 6] -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] +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] (32) HashAggregate [codegen id : 6] -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] +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] (33) Exchange -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] +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] (34) HashAggregate [codegen id : 9] -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))#67, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72] -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))#67,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72,17,2) AS sat_sales#36] +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] (35) Scan parquet default.store -Output [2]: [s_store_sk#37, s_store_id#38] +Output [2]: [s_store_sk#79, s_store_id#80] 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#37, s_store_id#38] +Input [2]: [s_store_sk#79, s_store_id#80] (37) Filter [codegen id : 7] -Input [2]: [s_store_sk#37, s_store_id#38] -Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) +Input [2]: [s_store_sk#79, s_store_id#80] +Condition : (isnotnull(s_store_sk#79) AND isnotnull(s_store_id#80)) (38) BroadcastExchange -Input [2]: [s_store_sk#37, s_store_id#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [s_store_sk#79, s_store_id#80] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] (39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#37] +Left keys [1]: [ss_store_sk#54] +Right keys [1]: [s_store_sk#79] Join condition: None (40) Project [codegen id : 9] -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#38] -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#37, s_store_id#38] +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] (41) Scan parquet default.date_dim -Output [2]: [d_month_seq#74, d_week_seq#75] +Output [2]: [d_month_seq#82, d_week_seq#83] 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#74, d_week_seq#75] +Input [2]: [d_month_seq#82, d_week_seq#83] (43) Filter [codegen id : 8] -Input [2]: [d_month_seq#74, d_week_seq#75] -Condition : (((isnotnull(d_month_seq#74) AND (d_month_seq#74 >= 1197)) AND (d_month_seq#74 <= 1208)) AND isnotnull(d_week_seq#75)) +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)) (44) Project [codegen id : 8] -Output [1]: [d_week_seq#75] -Input [2]: [d_month_seq#74, d_week_seq#75] +Output [1]: [d_week_seq#83] +Input [2]: [d_month_seq#82, d_week_seq#83] (45) BroadcastExchange -Input [1]: [d_week_seq#75] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] +Input [1]: [d_week_seq#83] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#84] (46) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#75] +Left keys [1]: [d_week_seq#58] +Right keys [1]: [d_week_seq#83] Join condition: None (47) Project [codegen id : 9] -Output [8]: [d_week_seq#5 AS d_week_seq2#77, s_store_id#38 AS s_store_id2#78, sun_sales#30 AS sun_sales2#79, mon_sales#31 AS mon_sales2#80, wed_sales#33 AS wed_sales2#81, thu_sales#34 AS thu_sales2#82, fri_sales#35 AS fri_sales2#83, sat_sales#36 AS sat_sales2#84] -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#38, d_week_seq#75] +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] (48) BroadcastExchange -Input [8]: [d_week_seq2#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#85] +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] (49) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#78, (d_week_seq2#77 - 52)] +Right keys [2]: [s_store_id2#86, (d_week_seq2#85 - 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#79)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#86, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#80)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#87, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#88, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#81)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#89, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#82)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#90, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#83)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#91, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#84)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#92] -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#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] +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] (51) TakeOrderedAndProject -Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] -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)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] +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] 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 4cdcd9685acbe..d20f7cf4a2857 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#1, ss_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3)] +PartitionFilters: [isnotnull(ss_sold_date_sk#56)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (27) ColumnarToRow [codegen id : 6] -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] (28) Filter [codegen id : 6] -Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#54, ss_sales_price#55, ss_sold_date_sk#56] +Condition : isnotnull(ss_store_sk#54) (29) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] +Output [3]: [d_date_sk#57, d_week_seq#58, d_day_name#59] (30) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#4] +Left keys [1]: [ss_sold_date_sk#56] +Right keys [1]: [d_date_sk#57] Join condition: None (31) Project [codegen id : 6] -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] +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] (32) HashAggregate [codegen id : 6] -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] +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] (33) Exchange -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] +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] (34) HashAggregate [codegen id : 9] -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))#67, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72] -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))#67,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#68,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#69,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#70,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#71,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#72,17,2) AS sat_sales#36] +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] (35) Scan parquet default.store -Output [2]: [s_store_sk#37, s_store_id#38] +Output [2]: [s_store_sk#79, s_store_id#80] 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#37, s_store_id#38] +Input [2]: [s_store_sk#79, s_store_id#80] (37) Filter [codegen id : 7] -Input [2]: [s_store_sk#37, s_store_id#38] -Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) +Input [2]: [s_store_sk#79, s_store_id#80] +Condition : (isnotnull(s_store_sk#79) AND isnotnull(s_store_id#80)) (38) BroadcastExchange -Input [2]: [s_store_sk#37, s_store_id#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [s_store_sk#79, s_store_id#80] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] (39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#37] +Left keys [1]: [ss_store_sk#54] +Right keys [1]: [s_store_sk#79] Join condition: None (40) Project [codegen id : 9] -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#38] -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#37, s_store_id#38] +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] (41) Scan parquet default.date_dim -Output [2]: [d_month_seq#74, d_week_seq#75] +Output [2]: [d_month_seq#82, d_week_seq#83] 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#74, d_week_seq#75] +Input [2]: [d_month_seq#82, d_week_seq#83] (43) Filter [codegen id : 8] -Input [2]: [d_month_seq#74, d_week_seq#75] -Condition : (((isnotnull(d_month_seq#74) AND (d_month_seq#74 >= 1197)) AND (d_month_seq#74 <= 1208)) AND isnotnull(d_week_seq#75)) +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)) (44) Project [codegen id : 8] -Output [1]: [d_week_seq#75] -Input [2]: [d_month_seq#74, d_week_seq#75] +Output [1]: [d_week_seq#83] +Input [2]: [d_month_seq#82, d_week_seq#83] (45) BroadcastExchange -Input [1]: [d_week_seq#75] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] +Input [1]: [d_week_seq#83] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#84] (46) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#75] +Left keys [1]: [d_week_seq#58] +Right keys [1]: [d_week_seq#83] Join condition: None (47) Project [codegen id : 9] -Output [8]: [d_week_seq#5 AS d_week_seq2#77, s_store_id#38 AS s_store_id2#78, sun_sales#30 AS sun_sales2#79, mon_sales#31 AS mon_sales2#80, wed_sales#33 AS wed_sales2#81, thu_sales#34 AS thu_sales2#82, fri_sales#35 AS fri_sales2#83, sat_sales#36 AS sat_sales2#84] -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#38, d_week_seq#75] +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] (48) BroadcastExchange -Input [8]: [d_week_seq2#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [id=#85] +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] (49) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#78, (d_week_seq2#77 - 52)] +Right keys [2]: [s_store_id2#86, (d_week_seq2#85 - 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#79)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#86, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#80)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#87, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales1#49)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales1)#88, CheckOverflow((promote_precision(wed_sales1#50) / promote_precision(wed_sales2#81)), DecimalType(37,20), true) AS (wed_sales1 / wed_sales2)#89, CheckOverflow((promote_precision(thu_sales1#51) / promote_precision(thu_sales2#82)), DecimalType(37,20), true) AS (thu_sales1 / thu_sales2)#90, CheckOverflow((promote_precision(fri_sales1#52) / promote_precision(fri_sales2#83)), DecimalType(37,20), true) AS (fri_sales1 / fri_sales2)#91, CheckOverflow((promote_precision(sat_sales1#53) / promote_precision(sat_sales2#84)), DecimalType(37,20), true) AS (sat_sales1 / sat_sales2)#92] -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#77, s_store_id2#78, sun_sales2#79, mon_sales2#80, wed_sales2#81, thu_sales2#82, fri_sales2#83, sat_sales2#84] +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] (51) TakeOrderedAndProject -Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] -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)#86, (mon_sales1 / mon_sales2)#87, (tue_sales1 / tue_sales1)#88, (wed_sales1 / wed_sales2)#89, (thu_sales1 / thu_sales2)#90, (fri_sales1 / fri_sales2)#91, (sat_sales1 / sat_sales2)#92] +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] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt index 741a644c4fdbc..240f579025038 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt @@ -127,123 +127,123 @@ Input [4]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17 Condition : isnotnull(ss_store_sk#15) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#18] (19) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#17] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#18] Join condition: None (20) Project [codegen id : 4] Output [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] -Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#6] +Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#18] (21) HashAggregate [codegen id : 4] Input [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] +Aggregate Attributes [1]: [sum#19] +Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] (22) Exchange -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] -Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] +Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#21] (23) HashAggregate [codegen id : 5] -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#21] -Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#21,17,2) AS revenue#22] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#22] +Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#22,17,2) AS revenue#23] (24) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#15, revenue#22] +Input [2]: [ss_store_sk#15, revenue#23] Keys [1]: [ss_store_sk#15] -Functions [1]: [partial_avg(revenue#22)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ss_store_sk#15, sum#25, count#26] +Functions [1]: [partial_avg(revenue#23)] +Aggregate Attributes [2]: [sum#24, count#25] +Results [3]: [ss_store_sk#15, sum#26, count#27] (25) Exchange -Input [3]: [ss_store_sk#15, sum#25, count#26] -Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ss_store_sk#15, sum#26, count#27] +Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#28] (26) HashAggregate [codegen id : 6] -Input [3]: [ss_store_sk#15, sum#25, count#26] +Input [3]: [ss_store_sk#15, sum#26, count#27] Keys [1]: [ss_store_sk#15] -Functions [1]: [avg(revenue#22)] -Aggregate Attributes [1]: [avg(revenue#22)#28] -Results [2]: [ss_store_sk#15, avg(revenue#22)#28 AS ave#29] +Functions [1]: [avg(revenue#23)] +Aggregate Attributes [1]: [avg(revenue#23)#29] +Results [2]: [ss_store_sk#15, avg(revenue#23)#29 AS ave#30] (27) BroadcastExchange -Input [2]: [ss_store_sk#15, ave#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] +Input [2]: [ss_store_sk#15, ave#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (28) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ss_store_sk#2] Right keys [1]: [ss_store_sk#15] -Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#29)), DecimalType(23,7), true)) +Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#30)), DecimalType(23,7), true)) (29) Project [codegen id : 7] Output [3]: [ss_store_sk#2, ss_item_sk#1, revenue#13] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#29] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#30] (30) BroadcastExchange Input [3]: [ss_store_sk#2, ss_item_sk#1, revenue#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] (31) Scan parquet default.store -Output [2]: [s_store_sk#32, s_store_name#33] +Output [2]: [s_store_sk#33, s_store_name#34] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (32) ColumnarToRow -Input [2]: [s_store_sk#32, s_store_name#33] +Input [2]: [s_store_sk#33, s_store_name#34] (33) Filter -Input [2]: [s_store_sk#32, s_store_name#33] -Condition : isnotnull(s_store_sk#32) +Input [2]: [s_store_sk#33, s_store_name#34] +Condition : isnotnull(s_store_sk#33) (34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#32] +Right keys [1]: [s_store_sk#33] Join condition: None (35) Project [codegen id : 8] -Output [3]: [ss_item_sk#1, revenue#13, s_store_name#33] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#32, s_store_name#33] +Output [3]: [ss_item_sk#1, revenue#13, s_store_name#34] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#33, s_store_name#34] (36) BroadcastExchange -Input [3]: [ss_item_sk#1, revenue#13, s_store_name#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#34] +Input [3]: [ss_item_sk#1, revenue#13, s_store_name#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] (37) Scan parquet default.item -Output [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Output [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (38) ColumnarToRow -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] (39) Filter -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Condition : isnotnull(i_item_sk#35) +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Condition : isnotnull(i_item_sk#36) (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#35] +Right keys [1]: [i_item_sk#36] Join condition: None (41) Project [codegen id : 9] -Output [6]: [s_store_name#33, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Input [8]: [ss_item_sk#1, revenue#13, s_store_name#33, i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Output [6]: [s_store_name#34, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Input [8]: [ss_item_sk#1, revenue#13, s_store_name#34, i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] (42) TakeOrderedAndProject -Input [6]: [s_store_name#33, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Arguments: 100, [s_store_name#33 ASC NULLS FIRST, i_item_desc#36 ASC NULLS FIRST], [s_store_name#33, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [6]: [s_store_name#34, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Arguments: 100, [s_store_name#34 ASC NULLS FIRST, i_item_desc#37 ASC NULLS FIRST], [s_store_name#34, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt index 0851190e54fe0..43ef320ed235f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt @@ -181,65 +181,65 @@ Input [4]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26 Condition : isnotnull(ss_store_sk#24) (30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#27] (31) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#27] Join condition: None (32) Project [codegen id : 6] Output [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] -Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#8] +Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#27] (33) HashAggregate [codegen id : 6] Input [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#27] -Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] +Aggregate Attributes [1]: [sum#28] +Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] (34) Exchange -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] -Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] +Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#30] (35) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#30] -Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#30,17,2) AS revenue#31] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] +Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS revenue#32] (36) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#24, revenue#31] +Input [2]: [ss_store_sk#24, revenue#32] Keys [1]: [ss_store_sk#24] -Functions [1]: [partial_avg(revenue#31)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [3]: [ss_store_sk#24, sum#34, count#35] +Functions [1]: [partial_avg(revenue#32)] +Aggregate Attributes [2]: [sum#33, count#34] +Results [3]: [ss_store_sk#24, sum#35, count#36] (37) Exchange -Input [3]: [ss_store_sk#24, sum#34, count#35] -Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ss_store_sk#24, sum#35, count#36] +Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#37] (38) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#24, sum#34, count#35] +Input [3]: [ss_store_sk#24, sum#35, count#36] Keys [1]: [ss_store_sk#24] -Functions [1]: [avg(revenue#31)] -Aggregate Attributes [1]: [avg(revenue#31)#37] -Results [2]: [ss_store_sk#24, avg(revenue#31)#37 AS ave#38] +Functions [1]: [avg(revenue#32)] +Aggregate Attributes [1]: [avg(revenue#32)#38] +Results [2]: [ss_store_sk#24, avg(revenue#32)#38 AS ave#39] (39) BroadcastExchange -Input [2]: [ss_store_sk#24, ave#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Input [2]: [ss_store_sk#24, ave#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#24] -Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#38)), DecimalType(23,7), true)) +Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#39)), DecimalType(23,7), true)) (41) Project [codegen id : 9] Output [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#38] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#39] (42) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt index 10ac72a6b44de..0b96b76a0ffc4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt @@ -232,31 +232,31 @@ Input [8]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37 Arguments: HashedRelationBroadcastMode(List(cast(input[5, int, true] as bigint)),false), [id=#44] (41) Scan parquet default.customer_address -Output [2]: [ca_address_sk#23, ca_city#24] +Output [2]: [ca_address_sk#45, ca_city#46] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] ReadSchema: struct (42) ColumnarToRow -Input [2]: [ca_address_sk#23, ca_city#24] +Input [2]: [ca_address_sk#45, ca_city#46] (43) Filter -Input [2]: [ca_address_sk#23, ca_city#24] -Condition : (isnotnull(ca_address_sk#23) AND isnotnull(ca_city#24)) +Input [2]: [ca_address_sk#45, ca_city#46] +Condition : (isnotnull(ca_address_sk#45) AND isnotnull(ca_city#46)) (44) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#41] -Right keys [1]: [ca_address_sk#23] -Join condition: NOT (ca_city#24 = bought_city#35) +Right keys [1]: [ca_address_sk#45] +Join condition: NOT (ca_city#46 = bought_city#35) (45) Project [codegen id : 8] -Output [8]: [c_last_name#43, c_first_name#42, ca_city#24, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#41, c_first_name#42, c_last_name#43, ca_address_sk#23, ca_city#24] +Output [8]: [c_last_name#43, c_first_name#42, ca_city#46, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#41, c_first_name#42, c_last_name#43, ca_address_sk#45, ca_city#46] (46) TakeOrderedAndProject -Input [8]: [c_last_name#43, c_first_name#42, ca_city#24, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Arguments: 100, [c_last_name#43 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#43, c_first_name#42, ca_city#24, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [8]: [c_last_name#43, c_first_name#42, ca_city#46, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Arguments: 100, [c_last_name#43 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#43, c_first_name#42, ca_city#46, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt index 81faa98299f82..f50329db7e05e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt @@ -225,20 +225,20 @@ Output [8]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#3 Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_customer_sk#39, c_current_addr_sk#40, c_first_name#41, c_last_name#42] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#22, ca_city#23] +Output [2]: [ca_address_sk#44, ca_city#45] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#40] -Right keys [1]: [ca_address_sk#22] -Join condition: NOT (ca_city#23 = bought_city#35) +Right keys [1]: [ca_address_sk#44] +Join condition: NOT (ca_city#45 = bought_city#35) (42) Project [codegen id : 8] -Output [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#22, ca_city#23] +Output [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#44, ca_city#45] (43) TakeOrderedAndProject -Input [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] ===== Subqueries ===== 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 9c68e55238aee..a950e7b9e8639 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 @@ -117,158 +117,158 @@ Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) (15) Scan parquet default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Output [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#5)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct (16) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] (17) Filter [codegen id : 4] -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) +Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] +Condition : isnotnull(sr_store_sk#17) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#20] (19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#4] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [sr_returned_date_sk#19] +Right keys [1]: [cast(d_date_sk#20 as bigint)] Join condition: None (20) Project [codegen id : 4] -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] +Output [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] +Input [5]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19, d_date_sk#20] (21) HashAggregate [codegen id : 4] -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#16] -Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] +Keys [2]: [sr_customer_sk#16, sr_store_sk#17] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#18))] +Aggregate Attributes [1]: [sum#21] +Results [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] (22) Exchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] +Arguments: hashpartitioning(sr_customer_sk#16, sr_store_sk#17, 5), ENSURE_REQUIREMENTS, [id=#23] (23) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] -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))#19] -Results [2]: [sr_store_sk#2 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#19,17,2) AS ctr_total_return#15] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] +Keys [2]: [sr_customer_sk#16, sr_store_sk#17] +Functions [1]: [sum(UnscaledValue(sr_return_amt#18))] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#18))#24] +Results [2]: [sr_store_sk#17 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#18))#24,17,2) AS ctr_total_return#15] (24) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#14, ctr_total_return#15] Keys [1]: [ctr_store_sk#14] Functions [1]: [partial_avg(ctr_total_return#15)] -Aggregate Attributes [2]: [sum#20, count#21] -Results [3]: [ctr_store_sk#14, sum#22, count#23] +Aggregate Attributes [2]: [sum#25, count#26] +Results [3]: [ctr_store_sk#14, sum#27, count#28] (25) Exchange -Input [3]: [ctr_store_sk#14, sum#22, count#23] -Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [ctr_store_sk#14, sum#27, count#28] +Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#29] (26) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#14, sum#22, count#23] +Input [3]: [ctr_store_sk#14, sum#27, count#28] Keys [1]: [ctr_store_sk#14] Functions [1]: [avg(ctr_total_return#15)] -Aggregate Attributes [1]: [avg(ctr_total_return#15)#25] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#25) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14 AS ctr_store_sk#14#27] +Aggregate Attributes [1]: [avg(ctr_total_return#15)#30] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14 AS ctr_store_sk#14#32] (27) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#26) +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) (28) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#28] +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] +Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#33] (29) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [ctr_store_sk#14#27] -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#26) +Right keys [1]: [ctr_store_sk#14#32] +Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) (30) Project [codegen id : 8] Output [2]: [ctr_customer_sk#13, ctr_store_sk#14] -Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] +Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] (31) Scan parquet default.store -Output [2]: [s_store_sk#29, s_state#30] +Output [2]: [s_store_sk#34, s_state#35] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#29, s_state#30] +Input [2]: [s_store_sk#34, s_state#35] (33) Filter [codegen id : 7] -Input [2]: [s_store_sk#29, s_state#30] -Condition : ((isnotnull(s_state#30) AND (s_state#30 = TN)) AND isnotnull(s_store_sk#29)) +Input [2]: [s_store_sk#34, s_state#35] +Condition : ((isnotnull(s_state#35) AND (s_state#35 = TN)) AND isnotnull(s_store_sk#34)) (34) Project [codegen id : 7] -Output [1]: [s_store_sk#29] -Input [2]: [s_store_sk#29, s_state#30] +Output [1]: [s_store_sk#34] +Input [2]: [s_store_sk#34, s_state#35] (35) BroadcastExchange -Input [1]: [s_store_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [1]: [s_store_sk#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (36) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [cast(s_store_sk#29 as bigint)] +Right keys [1]: [cast(s_store_sk#34 as bigint)] Join condition: None (37) Project [codegen id : 8] Output [1]: [ctr_customer_sk#13] -Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#29] +Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#34] (38) Exchange Input [1]: [ctr_customer_sk#13] -Arguments: hashpartitioning(ctr_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#32] +Arguments: hashpartitioning(ctr_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#37] (39) Sort [codegen id : 9] Input [1]: [ctr_customer_sk#13] Arguments: [ctr_customer_sk#13 ASC NULLS FIRST], false, 0 (40) Scan parquet default.customer -Output [2]: [c_customer_sk#33, c_customer_id#34] +Output [2]: [c_customer_sk#38, c_customer_id#39] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [2]: [c_customer_sk#33, c_customer_id#34] +Input [2]: [c_customer_sk#38, c_customer_id#39] (42) Filter [codegen id : 10] -Input [2]: [c_customer_sk#33, c_customer_id#34] -Condition : isnotnull(c_customer_sk#33) +Input [2]: [c_customer_sk#38, c_customer_id#39] +Condition : isnotnull(c_customer_sk#38) (43) Exchange -Input [2]: [c_customer_sk#33, c_customer_id#34] -Arguments: hashpartitioning(cast(c_customer_sk#33 as bigint), 5), ENSURE_REQUIREMENTS, [id=#35] +Input [2]: [c_customer_sk#38, c_customer_id#39] +Arguments: hashpartitioning(cast(c_customer_sk#38 as bigint), 5), ENSURE_REQUIREMENTS, [id=#40] (44) Sort [codegen id : 11] -Input [2]: [c_customer_sk#33, c_customer_id#34] -Arguments: [cast(c_customer_sk#33 as bigint) ASC NULLS FIRST], false, 0 +Input [2]: [c_customer_sk#38, c_customer_id#39] +Arguments: [cast(c_customer_sk#38 as bigint) ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 12] Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [cast(c_customer_sk#33 as bigint)] +Right keys [1]: [cast(c_customer_sk#38 as bigint)] Join condition: None (46) Project [codegen id : 12] -Output [1]: [c_customer_id#34] -Input [3]: [ctr_customer_sk#13, c_customer_sk#33, c_customer_id#34] +Output [1]: [c_customer_id#39] +Input [3]: [ctr_customer_sk#13, c_customer_sk#38, c_customer_id#39] (47) TakeOrderedAndProject -Input [1]: [c_customer_id#34] -Arguments: 100, [c_customer_id#34 ASC NULLS FIRST], [c_customer_id#34] +Input [1]: [c_customer_id#39] +Arguments: 100, [c_customer_id#39 ASC NULLS FIRST], [c_customer_id#39] ===== Subqueries ===== @@ -279,6 +279,6 @@ ReusedExchange (48) (48) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 15 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 15 Hosting Expression = sr_returned_date_sk#19 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 4487f04c8293d..015228686d39b 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 @@ -114,146 +114,146 @@ Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15] Condition : isnotnull(ctr_total_return#15) (15) Scan parquet default.store_returns -Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Output [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#5)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct (16) ColumnarToRow [codegen id : 4] -Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] (17) Filter [codegen id : 4] -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) +Input [4]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19] +Condition : isnotnull(sr_store_sk#17) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#20] (19) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [sr_returned_date_sk#4] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [sr_returned_date_sk#19] +Right keys [1]: [cast(d_date_sk#20 as bigint)] Join condition: None (20) Project [codegen id : 4] -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] +Output [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] +Input [5]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18, sr_returned_date_sk#19, d_date_sk#20] (21) HashAggregate [codegen id : 4] -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#16] -Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sr_return_amt#18] +Keys [2]: [sr_customer_sk#16, sr_store_sk#17] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#18))] +Aggregate Attributes [1]: [sum#21] +Results [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] (22) Exchange -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] -Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] +Arguments: hashpartitioning(sr_customer_sk#16, sr_store_sk#17, 5), ENSURE_REQUIREMENTS, [id=#23] (23) HashAggregate [codegen id : 5] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#17] -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))#19] -Results [2]: [sr_store_sk#2 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#19,17,2) AS ctr_total_return#15] +Input [3]: [sr_customer_sk#16, sr_store_sk#17, sum#22] +Keys [2]: [sr_customer_sk#16, sr_store_sk#17] +Functions [1]: [sum(UnscaledValue(sr_return_amt#18))] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#18))#24] +Results [2]: [sr_store_sk#17 AS ctr_store_sk#14, MakeDecimal(sum(UnscaledValue(sr_return_amt#18))#24,17,2) AS ctr_total_return#15] (24) HashAggregate [codegen id : 5] Input [2]: [ctr_store_sk#14, ctr_total_return#15] Keys [1]: [ctr_store_sk#14] Functions [1]: [partial_avg(ctr_total_return#15)] -Aggregate Attributes [2]: [sum#20, count#21] -Results [3]: [ctr_store_sk#14, sum#22, count#23] +Aggregate Attributes [2]: [sum#25, count#26] +Results [3]: [ctr_store_sk#14, sum#27, count#28] (25) Exchange -Input [3]: [ctr_store_sk#14, sum#22, count#23] -Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [ctr_store_sk#14, sum#27, count#28] +Arguments: hashpartitioning(ctr_store_sk#14, 5), ENSURE_REQUIREMENTS, [id=#29] (26) HashAggregate [codegen id : 6] -Input [3]: [ctr_store_sk#14, sum#22, count#23] +Input [3]: [ctr_store_sk#14, sum#27, count#28] Keys [1]: [ctr_store_sk#14] Functions [1]: [avg(ctr_total_return#15)] -Aggregate Attributes [1]: [avg(ctr_total_return#15)#25] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#25) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14 AS ctr_store_sk#14#27] +Aggregate Attributes [1]: [avg(ctr_total_return#15)#30] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#15)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14 AS ctr_store_sk#14#32] (27) Filter [codegen id : 6] -Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#26) +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) (28) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] -Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#28] +Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] +Arguments: HashedRelationBroadcastMode(List(input[1, bigint, true]),false), [id=#33] (29) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [ctr_store_sk#14#27] -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#26) +Right keys [1]: [ctr_store_sk#14#32] +Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) (30) Project [codegen id : 9] Output [2]: [ctr_customer_sk#13, ctr_store_sk#14] -Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#26, ctr_store_sk#14#27] +Input [5]: [ctr_customer_sk#13, ctr_store_sk#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#31, ctr_store_sk#14#32] (31) Scan parquet default.store -Output [2]: [s_store_sk#29, s_state#30] +Output [2]: [s_store_sk#34, s_state#35] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#29, s_state#30] +Input [2]: [s_store_sk#34, s_state#35] (33) Filter [codegen id : 7] -Input [2]: [s_store_sk#29, s_state#30] -Condition : ((isnotnull(s_state#30) AND (s_state#30 = TN)) AND isnotnull(s_store_sk#29)) +Input [2]: [s_store_sk#34, s_state#35] +Condition : ((isnotnull(s_state#35) AND (s_state#35 = TN)) AND isnotnull(s_store_sk#34)) (34) Project [codegen id : 7] -Output [1]: [s_store_sk#29] -Input [2]: [s_store_sk#29, s_state#30] +Output [1]: [s_store_sk#34] +Input [2]: [s_store_sk#34, s_state#35] (35) BroadcastExchange -Input [1]: [s_store_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] +Input [1]: [s_store_sk#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_store_sk#14] -Right keys [1]: [cast(s_store_sk#29 as bigint)] +Right keys [1]: [cast(s_store_sk#34 as bigint)] Join condition: None (37) Project [codegen id : 9] Output [1]: [ctr_customer_sk#13] -Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#29] +Input [3]: [ctr_customer_sk#13, ctr_store_sk#14, s_store_sk#34] (38) Scan parquet default.customer -Output [2]: [c_customer_sk#32, c_customer_id#33] +Output [2]: [c_customer_sk#37, c_customer_id#38] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 8] -Input [2]: [c_customer_sk#32, c_customer_id#33] +Input [2]: [c_customer_sk#37, c_customer_id#38] (40) Filter [codegen id : 8] -Input [2]: [c_customer_sk#32, c_customer_id#33] -Condition : isnotnull(c_customer_sk#32) +Input [2]: [c_customer_sk#37, c_customer_id#38] +Condition : isnotnull(c_customer_sk#37) (41) BroadcastExchange -Input [2]: [c_customer_sk#32, c_customer_id#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [2]: [c_customer_sk#37, c_customer_id#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#39] (42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [cast(c_customer_sk#32 as bigint)] +Right keys [1]: [cast(c_customer_sk#37 as bigint)] Join condition: None (43) Project [codegen id : 9] -Output [1]: [c_customer_id#33] -Input [3]: [ctr_customer_sk#13, c_customer_sk#32, c_customer_id#33] +Output [1]: [c_customer_id#38] +Input [3]: [ctr_customer_sk#13, c_customer_sk#37, c_customer_id#38] (44) TakeOrderedAndProject -Input [1]: [c_customer_id#33] -Arguments: 100, [c_customer_id#33 ASC NULLS FIRST], [c_customer_id#33] +Input [1]: [c_customer_id#38] +Arguments: 100, [c_customer_id#38 ASC NULLS FIRST], [c_customer_id#38] ===== Subqueries ===== @@ -264,6 +264,6 @@ ReusedExchange (45) (45) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 15 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 15 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt index c25973f527384..2dd0a13e74a74 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt @@ -143,20 +143,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#17] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#10] +Right keys [1]: [d_date_sk#17] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] -Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] +Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] (23) Exchange Input [1]: [ws_bill_customer_sk#15] -Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] @@ -168,38 +168,38 @@ Right keys [1]: [ws_bill_customer_sk#15] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Output [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#21] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [cs_sold_date_sk#20] +Right keys [1]: [d_date_sk#21] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#18] -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] +Output [1]: [cs_ship_customer_sk#19] +Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] (31) Exchange -Input [1]: [cs_ship_customer_sk#18] -Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [cs_ship_customer_sk#19] +Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#18] -Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#19] +Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#19] Join condition: None (34) Filter [codegen id : 13] @@ -211,96 +211,96 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (36) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_county#22] +Output [2]: [ca_address_sk#23, ca_county#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 12] -Input [2]: [ca_address_sk#21, ca_county#22] +Input [2]: [ca_address_sk#23, ca_county#24] (38) Filter [codegen id : 12] -Input [2]: [ca_address_sk#21, ca_county#22] -Condition : (ca_county#22 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#21)) +Input [2]: [ca_address_sk#23, ca_county#24] +Condition : (ca_county#24 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#23)) (39) Project [codegen id : 12] -Output [1]: [ca_address_sk#21] -Input [2]: [ca_address_sk#21, ca_county#22] +Output [1]: [ca_address_sk#23] +Input [2]: [ca_address_sk#23, ca_county#24] (40) BroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [ca_address_sk#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (41) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#23] Join condition: None (42) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#21] +Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#23] (43) Exchange Input [1]: [c_current_cdemo_sk#4] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#24] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#26] (44) Sort [codegen id : 14] Input [1]: [c_current_cdemo_sk#4] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 (45) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (46) ColumnarToRow [codegen id : 15] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] (47) Filter [codegen id : 15] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Condition : isnotnull(cd_demo_sk#25) +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Condition : isnotnull(cd_demo_sk#27) (48) Exchange -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: hashpartitioning(cd_demo_sk#25, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Arguments: hashpartitioning(cd_demo_sk#27, 5), ENSURE_REQUIREMENTS, [id=#36] (49) Sort [codegen id : 16] -Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Arguments: [cd_demo_sk#25 ASC NULLS FIRST], false, 0 +Input [9]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Arguments: [cd_demo_sk#27 ASC NULLS FIRST], false, 0 (50) SortMergeJoin [codegen id : 17] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#27] Join condition: None (51) Project [codegen id : 17] -Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Output [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] (52) HashAggregate [codegen id : 17] -Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] +Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#35] -Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Aggregate Attributes [1]: [count#37] +Results [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] (53) Exchange -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] +Arguments: hashpartitioning(cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, 5), ENSURE_REQUIREMENTS, [id=#39] (54) HashAggregate [codegen id : 18] -Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] -Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [9]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35, count#38] +Keys [8]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cd_purchase_estimate#31, cd_credit_rating#32, cd_dep_count#33, cd_dep_employed_count#34, cd_dep_college_count#35] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#38] -Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] +Aggregate Attributes [1]: [count(1)#40] +Results [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, count(1)#40 AS cnt1#41, cd_purchase_estimate#31, count(1)#40 AS cnt2#42, cd_credit_rating#32, count(1)#40 AS cnt3#43, cd_dep_count#33, count(1)#40 AS cnt4#44, cd_dep_employed_count#34, count(1)#40 AS cnt5#45, cd_dep_college_count#35, count(1)#40 AS cnt6#46] (55) TakeOrderedAndProject -Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] -Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] +Input [14]: [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#41, cd_purchase_estimate#31, cnt2#42, cd_credit_rating#32, cnt3#43, cd_dep_count#33, cnt4#44, cd_dep_employed_count#34, cnt5#45, cd_dep_college_count#35, cnt6#46] +Arguments: 100, [cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_education_status#30 ASC NULLS FIRST, cd_purchase_estimate#31 ASC NULLS FIRST, cd_credit_rating#32 ASC NULLS FIRST, cd_dep_count#33 ASC NULLS FIRST, cd_dep_employed_count#34 ASC NULLS FIRST, cd_dep_college_count#35 ASC NULLS FIRST], [cd_gender#28, cd_marital_status#29, cd_education_status#30, cnt1#41, cd_purchase_estimate#31, cnt2#42, cd_credit_rating#32, cnt3#43, cd_dep_count#33, cnt4#44, cd_dep_employed_count#34, cnt5#45, cd_dep_college_count#35, cnt6#46] ===== Subqueries ===== @@ -313,6 +313,6 @@ Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index 07bcd309e9ce6..370551a2de652 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -123,20 +123,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#16] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] -Right keys [1]: [d_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] +Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#16] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] @@ -144,34 +144,34 @@ Right keys [1]: [ws_bill_customer_sk#14] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#8)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#20] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#9] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#20] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#17] -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] +Output [1]: [cs_ship_customer_sk#18] +Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#20] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [cs_ship_customer_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#17] +Right keys [1]: [cs_ship_customer_sk#18] Join condition: None (29) Filter [codegen id : 9] @@ -183,84 +183,84 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] +Output [2]: [ca_address_sk#22, ca_county#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County]), IsNotNull(ca_address_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_county#21] +Input [2]: [ca_address_sk#22, ca_county#23] (33) Filter [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#20)) +Input [2]: [ca_address_sk#22, ca_county#23] +Condition : (ca_county#23 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#22)) (34) Project [codegen id : 7] -Output [1]: [ca_address_sk#20] -Input [2]: [ca_address_sk#20, ca_county#21] +Output [1]: [ca_address_sk#22] +Input [2]: [ca_address_sk#22, ca_county#23] (35) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [ca_address_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#22] Join condition: None (37) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#4] -Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20] +Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22] (38) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (40) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Condition : isnotnull(cd_demo_sk#23) +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Condition : isnotnull(cd_demo_sk#25) (41) BroadcastExchange -Input [9]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] +Input [9]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (42) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#25] Join condition: None (43) Project [codegen id : 9] -Output [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (44) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] +Aggregate Attributes [1]: [count#35] +Results [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] (45) Exchange -Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] -Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#37] (46) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#34] -Keys [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [9]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#36] +Keys [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#36] -Results [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#36 AS cnt1#37, cd_purchase_estimate#27, count(1)#36 AS cnt2#38, cd_credit_rating#28, count(1)#36 AS cnt3#39, cd_dep_count#29, count(1)#36 AS cnt4#40, cd_dep_employed_count#30, count(1)#36 AS cnt5#41, cd_dep_college_count#31, count(1)#36 AS cnt6#42] +Aggregate Attributes [1]: [count(1)#38] +Results [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#38 AS cnt1#39, cd_purchase_estimate#29, count(1)#38 AS cnt2#40, cd_credit_rating#30, count(1)#38 AS cnt3#41, cd_dep_count#31, count(1)#38 AS cnt4#42, cd_dep_employed_count#32, count(1)#38 AS cnt5#43, cd_dep_college_count#33, count(1)#38 AS cnt6#44] (47) TakeOrderedAndProject -Input [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] -Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#37, cd_purchase_estimate#27, cnt2#38, cd_credit_rating#28, cnt3#39, cd_dep_count#29, cnt4#40, cd_dep_employed_count#30, cnt5#41, cd_dep_college_count#31, cnt6#42] +Input [14]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] +Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#39, cd_purchase_estimate#29, cnt2#40, cd_credit_rating#30, cnt3#41, cd_dep_count#31, cnt4#42, cd_dep_employed_count#32, cnt5#43, cd_dep_college_count#33, cnt6#44] ===== Subqueries ===== @@ -273,6 +273,6 @@ Output [1]: [d_date_sk#9] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 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 da2143dc20a90..92e8e38634ea6 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 @@ -200,289 +200,289 @@ Input [2]: [customer_id#23, year_total#24] Arguments: [customer_id#23 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] (27) Filter [codegen id : 10] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_customer_sk#1) +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] +Condition : isnotnull(ss_customer_sk#26) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#31, d_year#32] 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 (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#6, d_year#7] +Input [2]: [d_date_sk#31, d_year#32] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#31, d_year#32] +Condition : ((isnotnull(d_year#32) AND (d_year#32 = 2002)) AND isnotnull(d_date_sk#31)) (31) BroadcastExchange -Input [2]: [d_date_sk#6, d_year#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [2]: [d_date_sk#31, d_year#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ss_sold_date_sk#29] +Right keys [1]: [d_date_sk#31] Join condition: None (33) Project [codegen id : 10] -Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4, d_date_sk#6, d_year#7] +Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Input [6]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29, d_date_sk#31, d_year#32] (34) Exchange -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#34] (35) Sort [codegen id : 11] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] (37) Sort [codegen id : 13] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Arguments: [c_customer_sk#35 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#35] Join condition: None (39) Project [codegen id : 14] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Input [12]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Input [12]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32, c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] (40) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#29] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] +Input [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#43] +Results [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] (41) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] +Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, 5), ENSURE_REQUIREMENTS, [id=#45] (42) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32] -Results [3]: [c_customer_id#11 AS customer_id#33, c_preferred_cust_flag#14 AS customer_preferred_cust_flag#34, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32,18,2) AS year_total#35] +Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] +Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46] +Results [3]: [c_customer_id#36 AS customer_id#47, c_preferred_cust_flag#39 AS customer_preferred_cust_flag#48, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46,18,2) AS year_total#49] (43) Exchange -Input [3]: [customer_id#33, customer_preferred_cust_flag#34, year_total#35] -Arguments: hashpartitioning(customer_id#33, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] +Arguments: hashpartitioning(customer_id#47, 5), ENSURE_REQUIREMENTS, [id=#50] (44) Sort [codegen id : 16] -Input [3]: [customer_id#33, customer_preferred_cust_flag#34, year_total#35] -Arguments: [customer_id#33 ASC NULLS FIRST], false, 0 +Input [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] +Arguments: [customer_id#47 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#33] +Right keys [1]: [customer_id#47] Join condition: None (46) Project [codegen id : 17] -Output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35] -Input [5]: [customer_id#23, year_total#24, customer_id#33, customer_preferred_cust_flag#34, year_total#35] +Output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49] +Input [5]: [customer_id#23, year_total#24, customer_id#47, customer_preferred_cust_flag#48, year_total#49] (47) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 19] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] (49) Filter [codegen id : 19] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_customer_sk#37) +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) (50) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#55, d_year#56] (51) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#54] +Right keys [1]: [d_date_sk#55] Join condition: None (52) Project [codegen id : 19] -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Input [6]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#6, d_year#7] +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] (53) Exchange -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Arguments: hashpartitioning(ws_bill_customer_sk#37, 5), ENSURE_REQUIREMENTS, [id=#41] +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] (54) Sort [codegen id : 20] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Arguments: [ws_bill_customer_sk#37 ASC NULLS FIRST], false, 0 +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 (55) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +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] (56) Sort [codegen id : 22] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +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 (57) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#37] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ws_bill_customer_sk#51] +Right keys [1]: [c_customer_sk#58] Join condition: None (58) Project [codegen id : 23] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Input [12]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +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] (59) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#42] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#43] +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] (60) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#43] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#44] +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] (61) HashAggregate [codegen id : 24] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#43] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45] -Results [2]: [c_customer_id#11 AS customer_id#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45,18,2) AS year_total#47] +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] (62) Filter [codegen id : 24] -Input [2]: [customer_id#46, year_total#47] -Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) +Input [2]: [customer_id#70, year_total#71] +Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.00)) (63) Project [codegen id : 24] -Output [2]: [customer_id#46 AS customer_id#48, year_total#47 AS year_total#49] -Input [2]: [customer_id#46, year_total#47] +Output [2]: [customer_id#70 AS customer_id#72, year_total#71 AS year_total#73] +Input [2]: [customer_id#70, year_total#71] (64) Exchange -Input [2]: [customer_id#48, year_total#49] -Arguments: hashpartitioning(customer_id#48, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [customer_id#72, year_total#73] +Arguments: hashpartitioning(customer_id#72, 5), ENSURE_REQUIREMENTS, [id=#74] (65) Sort [codegen id : 25] -Input [2]: [customer_id#48, year_total#49] -Arguments: [customer_id#48 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#72, year_total#73] +Arguments: [customer_id#72 ASC NULLS FIRST], false, 0 (66) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#48] +Right keys [1]: [customer_id#72] Join condition: None (67) Project [codegen id : 26] -Output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35, year_total#49] -Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35, customer_id#48, year_total#49] +Output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73] +Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, customer_id#72, year_total#73] (68) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ws_sold_date_sk#78), dynamicpruningexpression(ws_sold_date_sk#78 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] (70) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_customer_sk#37) +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78] +Condition : isnotnull(ws_bill_customer_sk#75) (71) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#79, d_year#80] (72) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#78] +Right keys [1]: [d_date_sk#79] Join condition: None (73) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Input [6]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#6, d_year#7] +Output [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Input [6]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, ws_sold_date_sk#78, d_date_sk#79, d_year#80] (74) Exchange -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Arguments: hashpartitioning(ws_bill_customer_sk#37, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Arguments: hashpartitioning(ws_bill_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#81] (75) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Arguments: [ws_bill_customer_sk#37 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Arguments: [ws_bill_customer_sk#75 ASC NULLS FIRST], false, 0 (76) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] (77) Sort [codegen id : 31] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] +Arguments: [c_customer_sk#82 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#37] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ws_bill_customer_sk#75] +Right keys [1]: [c_customer_sk#82] Join condition: None (79) Project [codegen id : 32] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Input [12]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Input [12]: [ws_bill_customer_sk#75, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80, c_customer_sk#82, c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89] (80) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#52] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#53] +Input [10]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, ws_ext_discount_amt#76, ws_ext_list_price#77, d_year#80] +Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#90] +Results [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] (81) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#53] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] +Arguments: hashpartitioning(c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, 5), ENSURE_REQUIREMENTS, [id=#92] (82) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#53] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55] -Results [2]: [c_customer_id#11 AS customer_id#56, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55,18,2) AS year_total#57] +Input [9]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80, sum#91] +Keys [8]: [c_customer_id#83, c_first_name#84, c_last_name#85, c_preferred_cust_flag#86, c_birth_country#87, c_login#88, c_email_address#89, d_year#80] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93] +Results [2]: [c_customer_id#83 AS customer_id#94, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#77 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#76 as decimal(8,2)))), DecimalType(8,2), true)))#93,18,2) AS year_total#95] (83) Exchange -Input [2]: [customer_id#56, year_total#57] -Arguments: hashpartitioning(customer_id#56, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [customer_id#94, year_total#95] +Arguments: hashpartitioning(customer_id#94, 5), ENSURE_REQUIREMENTS, [id=#96] (84) Sort [codegen id : 34] -Input [2]: [customer_id#56, year_total#57] -Arguments: [customer_id#56 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#94, year_total#95] +Arguments: [customer_id#94 ASC NULLS FIRST], false, 0 (85) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#56] -Join condition: (CASE WHEN (year_total#49 > 0.00) THEN CheckOverflow((promote_precision(year_total#57) / promote_precision(year_total#49)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#35) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE null END) +Right keys [1]: [customer_id#94] +Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#95) / promote_precision(year_total#73)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#49) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE null END) (86) Project [codegen id : 35] -Output [1]: [customer_preferred_cust_flag#34] -Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#34, year_total#35, year_total#49, customer_id#56, year_total#57] +Output [1]: [customer_preferred_cust_flag#48] +Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73, customer_id#94, year_total#95] (87) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#34] -Arguments: 100, [customer_preferred_cust_flag#34 ASC NULLS FIRST], [customer_preferred_cust_flag#34] +Input [1]: [customer_preferred_cust_flag#48] +Arguments: 100, [customer_preferred_cust_flag#48 ASC NULLS FIRST], [customer_preferred_cust_flag#48] ===== Subqueries ===== @@ -493,15 +493,15 @@ ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#6, d_year#7] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#26 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#30 ReusedExchange (89) (89) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#31, d_year#32] -Subquery:3 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#26 +Subquery:4 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#78 IN dynamicpruning#30 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 5a5249a3aa81b..4081f69732168 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 @@ -172,286 +172,286 @@ Input [2]: [customer_id#22, year_total#23] Condition : (isnotnull(year_total#23) AND (year_total#23 > 0.00)) (20) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] (22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_customer_id#25)) (23) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Output [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ss_sold_date_sk#35), dynamicpruningexpression(ss_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] (25) Filter [codegen id : 4] -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_customer_sk#9) +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Condition : isnotnull(ss_customer_sk#32) (26) BroadcastExchange -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#9] +Left keys [1]: [c_customer_sk#24] +Right keys [1]: [ss_customer_sk#32] Join condition: None (28) Project [codegen id : 6] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Input [12]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#38, d_year#39] 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 (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#15, d_year#16] +Input [2]: [d_date_sk#38, d_year#39] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#15, d_year#16] -Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2002)) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#38, d_year#39] +Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) (32) BroadcastExchange -Input [2]: [d_date_sk#15, d_year#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +Input [2]: [d_date_sk#38, d_year#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#35] +Right keys [1]: [d_date_sk#38] Join condition: None (34) Project [codegen id : 6] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#15, d_year#16] +Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] +Input [12]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#38, d_year#39] (35) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#27] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] +Input [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] +Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#41] +Results [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] (36) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] +Arguments: hashpartitioning(c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, 5), ENSURE_REQUIREMENTS, [id=#43] (37) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30] -Results [3]: [c_customer_id#2 AS customer_id#31, c_preferred_cust_flag#5 AS customer_preferred_cust_flag#32, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30,18,2) AS year_total#33] +Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] +Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44] +Results [3]: [c_customer_id#25 AS customer_id#45, c_preferred_cust_flag#28 AS customer_preferred_cust_flag#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44,18,2) AS year_total#47] (38) BroadcastExchange -Input [3]: [customer_id#31, customer_preferred_cust_flag#32, year_total#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] +Input [3]: [customer_id#45, customer_preferred_cust_flag#46, year_total#47] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#31] +Right keys [1]: [customer_id#45] Join condition: None (40) Project [codegen id : 16] -Output [4]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33] -Input [5]: [customer_id#22, year_total#23, customer_id#31, customer_preferred_cust_flag#32, year_total#33] +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] (41) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +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] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (42) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +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] (43) Filter [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +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)) (44) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Output [4]: [ws_bill_customer_sk#57, ws_ext_discount_amt#58, ws_ext_list_price#59, ws_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (45) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Input [4]: [ws_bill_customer_sk#57, ws_ext_discount_amt#58, ws_ext_list_price#59, ws_sold_date_sk#60] (46) Filter [codegen id : 8] -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_bill_customer_sk#35) +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) (47) BroadcastExchange -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#39] +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] (48) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#35] +Left keys [1]: [c_customer_sk#49] +Right keys [1]: [ws_bill_customer_sk#57] Join condition: None (49) Project [codegen id : 10] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +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] (50) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#62, d_year#63] (51) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ws_sold_date_sk#60] +Right keys [1]: [d_date_sk#62] Join condition: None (52) Project [codegen id : 10] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38, d_date_sk#15, d_year#16] +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] (53) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#40] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#41] +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] (54) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#41] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#42] +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] (55) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#41] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#43] -Results [2]: [c_customer_id#2 AS customer_id#44, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#43,18,2) AS year_total#45] +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] (56) Filter [codegen id : 11] -Input [2]: [customer_id#44, year_total#45] -Condition : (isnotnull(year_total#45) AND (year_total#45 > 0.00)) +Input [2]: [customer_id#68, year_total#69] +Condition : (isnotnull(year_total#69) AND (year_total#69 > 0.00)) (57) Project [codegen id : 11] -Output [2]: [customer_id#44 AS customer_id#46, year_total#45 AS year_total#47] -Input [2]: [customer_id#44, year_total#45] +Output [2]: [customer_id#68 AS customer_id#70, year_total#69 AS year_total#71] +Input [2]: [customer_id#68, year_total#69] (58) BroadcastExchange -Input [2]: [customer_id#46, year_total#47] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] +Input [2]: [customer_id#70, year_total#71] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#72] (59) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#46] +Right keys [1]: [customer_id#70] Join condition: None (60) Project [codegen id : 16] -Output [5]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33, year_total#47] -Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33, customer_id#46, year_total#47] +Output [5]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, year_total#71] +Input [6]: [customer_id#22, year_total#23, customer_preferred_cust_flag#46, year_total#47, customer_id#70, year_total#71] (61) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (62) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] (63) Filter [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_customer_id#74)) (64) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Output [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ws_sold_date_sk#84), dynamicpruningexpression(ws_sold_date_sk#84 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] (66) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Condition : isnotnull(ws_bill_customer_sk#35) +Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Condition : isnotnull(ws_bill_customer_sk#81) (67) BroadcastExchange -Input [4]: [ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Input [4]: [ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#85] (68) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#35] +Left keys [1]: [c_customer_sk#73] +Right keys [1]: [ws_bill_customer_sk#81] Join condition: None (69) Project [codegen id : 14] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#35, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38] +Output [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] +Input [12]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_bill_customer_sk#81, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84] (70) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#86, d_year#87] (71) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#38] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ws_sold_date_sk#84] +Right keys [1]: [d_date_sk#86] Join condition: None (72) Project [codegen id : 14] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, ws_sold_date_sk#38, d_date_sk#15, d_year#16] +Output [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, d_year#87] +Input [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, ws_sold_date_sk#84, d_date_sk#86, d_year#87] (73) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#36, ws_ext_list_price#37, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#50] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#51] +Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, ws_ext_discount_amt#82, ws_ext_list_price#83, d_year#87] +Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#88] +Results [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] (74) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#51] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] +Arguments: hashpartitioning(c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, 5), ENSURE_REQUIREMENTS, [id=#90] (75) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#51] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#53] -Results [2]: [c_customer_id#2 AS customer_id#54, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#36 as decimal(8,2)))), DecimalType(8,2), true)))#53,18,2) AS year_total#55] +Input [9]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87, sum#89] +Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#87] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))#91] +Results [2]: [c_customer_id#74 AS customer_id#92, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#83 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#82 as decimal(8,2)))), DecimalType(8,2), true)))#91,18,2) AS year_total#93] (76) BroadcastExchange -Input [2]: [customer_id#54, year_total#55] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] +Input [2]: [customer_id#92, year_total#93] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#94] (77) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#54] -Join condition: (CASE WHEN (year_total#47 > 0.00) THEN CheckOverflow((promote_precision(year_total#55) / promote_precision(year_total#47)), DecimalType(38,20), true) ELSE null END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#33) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE null END) +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 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) (78) Project [codegen id : 16] -Output [1]: [customer_preferred_cust_flag#32] -Input [7]: [customer_id#22, year_total#23, customer_preferred_cust_flag#32, year_total#33, year_total#47, customer_id#54, year_total#55] +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#71, customer_id#92, year_total#93] (79) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#32] -Arguments: 100, [customer_preferred_cust_flag#32 ASC NULLS FIRST], [customer_preferred_cust_flag#32] +Input [1]: [customer_preferred_cust_flag#46] +Arguments: 100, [customer_preferred_cust_flag#46 ASC NULLS FIRST], [customer_preferred_cust_flag#46] ===== Subqueries ===== @@ -462,15 +462,15 @@ ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#15, d_year#16] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#24 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 ReusedExchange (81) (81) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#38, d_year#39] -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#24 +Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#84 IN dynamicpruning#36 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt index 9c56438870f64..7604ea1e786d8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.sf100/explain.txt @@ -74,187 +74,187 @@ Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet default.catalog_sales -Output [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Output [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Input [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] (9) Project [codegen id : 3] -Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#10, cs_order_number#5 AS cs_order_number#5#11] -Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Output [2]: [cs_warehouse_sk#10, cs_order_number#11] +Input [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] (10) Exchange -Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] -Arguments: hashpartitioning(cs_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [2]: [cs_warehouse_sk#10, cs_order_number#11] +Arguments: hashpartitioning(cs_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 4] -Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] -Arguments: [cs_order_number#5#11 ASC NULLS FIRST], false, 0 +Input [2]: [cs_warehouse_sk#10, cs_order_number#11] +Arguments: [cs_order_number#11 ASC NULLS FIRST], false, 0 (12) SortMergeJoin Left keys [1]: [cs_order_number#5] -Right keys [1]: [cs_order_number#5#11] -Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#10) +Right keys [1]: [cs_order_number#11] +Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#10) (13) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] (14) Scan parquet default.catalog_returns -Output [2]: [cr_order_number#13, cr_returned_date_sk#14] +Output [2]: [cr_order_number#14, cr_returned_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] ReadSchema: struct (15) ColumnarToRow [codegen id : 6] -Input [2]: [cr_order_number#13, cr_returned_date_sk#14] +Input [2]: [cr_order_number#14, cr_returned_date_sk#15] (16) Project [codegen id : 6] -Output [1]: [cr_order_number#13] -Input [2]: [cr_order_number#13, cr_returned_date_sk#14] +Output [1]: [cr_order_number#14] +Input [2]: [cr_order_number#14, cr_returned_date_sk#15] (17) Exchange -Input [1]: [cr_order_number#13] -Arguments: hashpartitioning(cr_order_number#13, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [1]: [cr_order_number#14] +Arguments: hashpartitioning(cr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] (18) Sort [codegen id : 7] -Input [1]: [cr_order_number#13] -Arguments: [cr_order_number#13 ASC NULLS FIRST], false, 0 +Input [1]: [cr_order_number#14] +Arguments: [cr_order_number#14 ASC NULLS FIRST], false, 0 (19) SortMergeJoin Left keys [1]: [cs_order_number#5] -Right keys [1]: [cr_order_number#13] +Right keys [1]: [cr_order_number#14] Join condition: None (20) Scan parquet default.customer_address -Output [2]: [ca_address_sk#16, ca_state#17] +Output [2]: [ca_address_sk#17, ca_state#18] 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 (21) ColumnarToRow [codegen id : 8] -Input [2]: [ca_address_sk#16, ca_state#17] +Input [2]: [ca_address_sk#17, ca_state#18] (22) Filter [codegen id : 8] -Input [2]: [ca_address_sk#16, ca_state#17] -Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_address_sk#16)) +Input [2]: [ca_address_sk#17, ca_state#18] +Condition : ((isnotnull(ca_state#18) AND (ca_state#18 = GA)) AND isnotnull(ca_address_sk#17)) (23) Project [codegen id : 8] -Output [1]: [ca_address_sk#16] -Input [2]: [ca_address_sk#16, ca_state#17] +Output [1]: [ca_address_sk#17] +Input [2]: [ca_address_sk#17, ca_state#18] (24) BroadcastExchange -Input [1]: [ca_address_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Input [1]: [ca_address_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#16] +Right keys [1]: [ca_address_sk#17] Join condition: None (26) Project [codegen id : 11] Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] +Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#17] (27) Scan parquet default.call_center -Output [2]: [cc_call_center_sk#19, cc_county#20] +Output [2]: [cc_call_center_sk#20, cc_county#21] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 9] -Input [2]: [cc_call_center_sk#19, cc_county#20] +Input [2]: [cc_call_center_sk#20, cc_county#21] (29) Filter [codegen id : 9] -Input [2]: [cc_call_center_sk#19, cc_county#20] -Condition : ((isnotnull(cc_county#20) AND (cc_county#20 = Williamson County)) AND isnotnull(cc_call_center_sk#19)) +Input [2]: [cc_call_center_sk#20, cc_county#21] +Condition : ((isnotnull(cc_county#21) AND (cc_county#21 = Williamson County)) AND isnotnull(cc_call_center_sk#20)) (30) Project [codegen id : 9] -Output [1]: [cc_call_center_sk#19] -Input [2]: [cc_call_center_sk#19, cc_county#20] +Output [1]: [cc_call_center_sk#20] +Input [2]: [cc_call_center_sk#20, cc_county#21] (31) BroadcastExchange -Input [1]: [cc_call_center_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [cc_call_center_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] -Right keys [1]: [cc_call_center_sk#19] +Right keys [1]: [cc_call_center_sk#20] Join condition: None (33) Project [codegen id : 11] Output [4]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [6]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#19] +Input [6]: [cs_ship_date_sk#1, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#20] (34) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_date#23] +Output [2]: [d_date_sk#23, d_date#24] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 10] -Input [2]: [d_date_sk#22, d_date#23] +Input [2]: [d_date_sk#23, d_date#24] (36) Filter [codegen id : 10] -Input [2]: [d_date_sk#22, d_date#23] -Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2002-02-01)) AND (d_date#23 <= 2002-04-02)) AND isnotnull(d_date_sk#22)) +Input [2]: [d_date_sk#23, d_date#24] +Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 2002-02-01)) AND (d_date#24 <= 2002-04-02)) AND isnotnull(d_date_sk#23)) (37) Project [codegen id : 10] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_date#23] +Output [1]: [d_date_sk#23] +Input [2]: [d_date_sk#23, d_date#24] (38) 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#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#23] Join condition: None (40) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [5]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#22] +Input [5]: [cs_ship_date_sk#1, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#23] (41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] -Results [3]: [cs_order_number#5, sum#27, sum#28] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] +Results [3]: [cs_order_number#5, sum#28, sum#29] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_order_number#5, sum#27, sum#28] +Input [3]: [cs_order_number#5, sum#28, sum#29] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] -Results [3]: [cs_order_number#5, sum#27, sum#28] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] +Results [3]: [cs_order_number#5, sum#28, sum#29] (43) HashAggregate [codegen id : 11] -Input [3]: [cs_order_number#5, sum#27, sum#28] +Input [3]: [cs_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] -Results [3]: [sum#27, sum#28, count#30] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] +Results [3]: [sum#28, sum#29, count#31] (44) Exchange -Input [3]: [sum#27, sum#28, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [sum#28, sum#29, count#31] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] (45) HashAggregate [codegen id : 12] -Input [3]: [sum#27, sum#28, count#30] +Input [3]: [sum#28, sum#29, count#31] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] -Results [3]: [count(cs_order_number#5)#29 AS order count #32, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#25,17,2) AS total shipping cost #33, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#26,17,2) AS total net profit #34] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] +Results [3]: [count(cs_order_number#5)#30 AS order count #33, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#26,17,2) AS total shipping cost #34, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#27,17,2) AS total net profit #35] (46) Sort [codegen id : 12] -Input [3]: [order count #32, total shipping cost #33, total net profit #34] -Arguments: [order count #32 ASC NULLS FIRST], true, 0 +Input [3]: [order count #33, total shipping cost #34, total net profit #35] +Arguments: [order count #33 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt index dde6a9f564859..683d83235cce5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/explain.txt @@ -74,187 +74,187 @@ Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_wareho Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet default.catalog_sales -Output [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Output [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Input [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] (9) Project [codegen id : 3] -Output [2]: [cs_warehouse_sk#4 AS cs_warehouse_sk#4#10, cs_order_number#5 AS cs_order_number#5#11] -Input [3]: [cs_warehouse_sk#4, cs_order_number#5, cs_sold_date_sk#8] +Output [2]: [cs_warehouse_sk#10, cs_order_number#11] +Input [3]: [cs_warehouse_sk#10, cs_order_number#11, cs_sold_date_sk#12] (10) Exchange -Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] -Arguments: hashpartitioning(cs_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [2]: [cs_warehouse_sk#10, cs_order_number#11] +Arguments: hashpartitioning(cs_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 4] -Input [2]: [cs_warehouse_sk#4#10, cs_order_number#5#11] -Arguments: [cs_order_number#5#11 ASC NULLS FIRST], false, 0 +Input [2]: [cs_warehouse_sk#10, cs_order_number#11] +Arguments: [cs_order_number#11 ASC NULLS FIRST], false, 0 (12) SortMergeJoin Left keys [1]: [cs_order_number#5] -Right keys [1]: [cs_order_number#5#11] -Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#4#10) +Right keys [1]: [cs_order_number#11] +Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#10) (13) Project [codegen id : 5] Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] (14) Scan parquet default.catalog_returns -Output [2]: [cr_order_number#13, cr_returned_date_sk#14] +Output [2]: [cr_order_number#14, cr_returned_date_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] ReadSchema: struct (15) ColumnarToRow [codegen id : 6] -Input [2]: [cr_order_number#13, cr_returned_date_sk#14] +Input [2]: [cr_order_number#14, cr_returned_date_sk#15] (16) Project [codegen id : 6] -Output [1]: [cr_order_number#13] -Input [2]: [cr_order_number#13, cr_returned_date_sk#14] +Output [1]: [cr_order_number#14] +Input [2]: [cr_order_number#14, cr_returned_date_sk#15] (17) Exchange -Input [1]: [cr_order_number#13] -Arguments: hashpartitioning(cr_order_number#13, 5), ENSURE_REQUIREMENTS, [id=#15] +Input [1]: [cr_order_number#14] +Arguments: hashpartitioning(cr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] (18) Sort [codegen id : 7] -Input [1]: [cr_order_number#13] -Arguments: [cr_order_number#13 ASC NULLS FIRST], false, 0 +Input [1]: [cr_order_number#14] +Arguments: [cr_order_number#14 ASC NULLS FIRST], false, 0 (19) SortMergeJoin Left keys [1]: [cs_order_number#5] -Right keys [1]: [cr_order_number#13] +Right keys [1]: [cr_order_number#14] Join condition: None (20) Scan parquet default.date_dim -Output [2]: [d_date_sk#16, d_date#17] +Output [2]: [d_date_sk#17, d_date#18] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 8] -Input [2]: [d_date_sk#16, d_date#17] +Input [2]: [d_date_sk#17, d_date#18] (22) Filter [codegen id : 8] -Input [2]: [d_date_sk#16, d_date#17] -Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2002-02-01)) AND (d_date#17 <= 2002-04-02)) AND isnotnull(d_date_sk#16)) +Input [2]: [d_date_sk#17, d_date#18] +Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 2002-02-01)) AND (d_date#18 <= 2002-04-02)) AND isnotnull(d_date_sk#17)) (23) Project [codegen id : 8] -Output [1]: [d_date_sk#16] -Input [2]: [d_date_sk#16, d_date#17] +Output [1]: [d_date_sk#17] +Input [2]: [d_date_sk#17, d_date#18] (24) BroadcastExchange -Input [1]: [d_date_sk#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Input [1]: [d_date_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (25) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_date_sk#1] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#17] Join condition: None (26) Project [codegen id : 11] Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#16] +Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#17] (27) Scan parquet default.customer_address -Output [2]: [ca_address_sk#19, ca_state#20] +Output [2]: [ca_address_sk#20, ca_state#21] 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 (28) ColumnarToRow [codegen id : 9] -Input [2]: [ca_address_sk#19, ca_state#20] +Input [2]: [ca_address_sk#20, ca_state#21] (29) Filter [codegen id : 9] -Input [2]: [ca_address_sk#19, ca_state#20] -Condition : ((isnotnull(ca_state#20) AND (ca_state#20 = GA)) AND isnotnull(ca_address_sk#19)) +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : ((isnotnull(ca_state#21) AND (ca_state#21 = GA)) AND isnotnull(ca_address_sk#20)) (30) Project [codegen id : 9] -Output [1]: [ca_address_sk#19] -Input [2]: [ca_address_sk#19, ca_state#20] +Output [1]: [ca_address_sk#20] +Input [2]: [ca_address_sk#20, ca_state#21] (31) BroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [ca_address_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#19] +Right keys [1]: [ca_address_sk#20] Join condition: None (33) Project [codegen id : 11] Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#19] +Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#20] (34) Scan parquet default.call_center -Output [2]: [cc_call_center_sk#22, cc_county#23] +Output [2]: [cc_call_center_sk#23, cc_county#24] Batched: true Location [not included in comparison]/{warehouse_dir}/call_center] PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 10] -Input [2]: [cc_call_center_sk#22, cc_county#23] +Input [2]: [cc_call_center_sk#23, cc_county#24] (36) Filter [codegen id : 10] -Input [2]: [cc_call_center_sk#22, cc_county#23] -Condition : ((isnotnull(cc_county#23) AND (cc_county#23 = Williamson County)) AND isnotnull(cc_call_center_sk#22)) +Input [2]: [cc_call_center_sk#23, cc_county#24] +Condition : ((isnotnull(cc_county#24) AND (cc_county#24 = Williamson County)) AND isnotnull(cc_call_center_sk#23)) (37) Project [codegen id : 10] -Output [1]: [cc_call_center_sk#22] -Input [2]: [cc_call_center_sk#22, cc_county#23] +Output [1]: [cc_call_center_sk#23] +Input [2]: [cc_call_center_sk#23, cc_county#24] (38) BroadcastExchange -Input [1]: [cc_call_center_sk#22] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] +Input [1]: [cc_call_center_sk#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_call_center_sk#3] -Right keys [1]: [cc_call_center_sk#22] +Right keys [1]: [cc_call_center_sk#23] Join condition: None (40) Project [codegen id : 11] Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] -Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#22] +Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#23] (41) HashAggregate [codegen id : 11] Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] Keys [1]: [cs_order_number#5] Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] -Results [3]: [cs_order_number#5, sum#27, sum#28] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] +Results [3]: [cs_order_number#5, sum#28, sum#29] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_order_number#5, sum#27, sum#28] +Input [3]: [cs_order_number#5, sum#28, sum#29] Keys [1]: [cs_order_number#5] Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26] -Results [3]: [cs_order_number#5, sum#27, sum#28] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27] +Results [3]: [cs_order_number#5, sum#28, sum#29] (43) HashAggregate [codegen id : 11] -Input [3]: [cs_order_number#5, sum#27, sum#28] +Input [3]: [cs_order_number#5, sum#28, sum#29] Keys: [] Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] -Results [3]: [sum#27, sum#28, count#30] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] +Results [3]: [sum#28, sum#29, count#31] (44) Exchange -Input [3]: [sum#27, sum#28, count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] +Input [3]: [sum#28, sum#29, count#31] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#32] (45) HashAggregate [codegen id : 12] -Input [3]: [sum#27, sum#28, count#30] +Input [3]: [sum#28, sum#29, count#31] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#25, sum(UnscaledValue(cs_net_profit#7))#26, count(cs_order_number#5)#29] -Results [3]: [count(cs_order_number#5)#29 AS order count #32, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#25,17,2) AS total shipping cost #33, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#26,17,2) AS total net profit #34] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#26, sum(UnscaledValue(cs_net_profit#7))#27, count(cs_order_number#5)#30] +Results [3]: [count(cs_order_number#5)#30 AS order count #33, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#26,17,2) AS total shipping cost #34, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#27,17,2) AS total net profit #35] (46) Sort [codegen id : 12] -Input [3]: [order count #32, total shipping cost #33, total net profit #34] -Arguments: [order count #32 ASC NULLS FIRST], true, 0 +Input [3]: [order count #33, total shipping cost #34, total net profit #35] +Arguments: [order count #33 ASC NULLS FIRST], true, 0 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 f540e067bbc92..c31bb7470648f 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#10, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] +Output [8]: [d_week_seq#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] (25) HashAggregate [codegen id : 11] -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))#60, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#60,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66,17,2) AS sat_sales#41] +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] (26) Scan parquet default.date_dim -Output [2]: [d_week_seq#67, d_year#68] +Output [2]: [d_week_seq#69, d_year#70] 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#67, d_year#68] +Input [2]: [d_week_seq#69, d_year#70] (28) Filter [codegen id : 10] -Input [2]: [d_week_seq#67, d_year#68] -Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) +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)) (29) Project [codegen id : 10] -Output [1]: [d_week_seq#67] -Input [2]: [d_week_seq#67, d_year#68] +Output [1]: [d_week_seq#69] +Input [2]: [d_week_seq#69, d_year#70] (30) BroadcastExchange -Input [1]: [d_week_seq#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] +Input [1]: [d_week_seq#69] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] (31) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#67] +Left keys [1]: [d_week_seq#53] +Right keys [1]: [d_week_seq#69] Join condition: None (32) Project [codegen id : 11] -Output [8]: [d_week_seq#10 AS d_week_seq2#70, sun_sales#35 AS sun_sales2#71, mon_sales#36 AS mon_sales2#72, tue_sales#37 AS tue_sales2#73, wed_sales#38 AS wed_sales2#74, thu_sales#39 AS thu_sales2#75, fri_sales#40 AS fri_sales2#76, sat_sales#41 AS sat_sales2#77] -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#67] +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] (33) BroadcastExchange -Input [8]: [d_week_seq2#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#78] +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] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq1#45] -Right keys [1]: [(d_week_seq2#70 - 53)] +Right keys [1]: [(d_week_seq2#72 - 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#71)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#79, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#72)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#80, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#73)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#81, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#74)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#82, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#75)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#83, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#76)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#84, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#77)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#85] -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#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] +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] (36) Exchange -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] -Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#86] +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] (37) Sort [codegen id : 13] -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] +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: [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 f540e067bbc92..c31bb7470648f 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#10, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] +Output [8]: [d_week_seq#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] (25) HashAggregate [codegen id : 11] -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))#60, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 ELSE null END))#60,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 ELSE null END))#61,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 ELSE null END))#62,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 ELSE null END))#64,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 ELSE null END))#65,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 ELSE null END))#66,17,2) AS sat_sales#41] +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] (26) Scan parquet default.date_dim -Output [2]: [d_week_seq#67, d_year#68] +Output [2]: [d_week_seq#69, d_year#70] 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#67, d_year#68] +Input [2]: [d_week_seq#69, d_year#70] (28) Filter [codegen id : 10] -Input [2]: [d_week_seq#67, d_year#68] -Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) +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)) (29) Project [codegen id : 10] -Output [1]: [d_week_seq#67] -Input [2]: [d_week_seq#67, d_year#68] +Output [1]: [d_week_seq#69] +Input [2]: [d_week_seq#69, d_year#70] (30) BroadcastExchange -Input [1]: [d_week_seq#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] +Input [1]: [d_week_seq#69] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] (31) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#67] +Left keys [1]: [d_week_seq#53] +Right keys [1]: [d_week_seq#69] Join condition: None (32) Project [codegen id : 11] -Output [8]: [d_week_seq#10 AS d_week_seq2#70, sun_sales#35 AS sun_sales2#71, mon_sales#36 AS mon_sales2#72, tue_sales#37 AS tue_sales2#73, wed_sales#38 AS wed_sales2#74, thu_sales#39 AS thu_sales2#75, fri_sales#40 AS fri_sales2#76, sat_sales#41 AS sat_sales2#77] -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#67] +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] (33) BroadcastExchange -Input [8]: [d_week_seq2#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#78] +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] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq1#45] -Right keys [1]: [(d_week_seq2#70 - 53)] +Right keys [1]: [(d_week_seq2#72 - 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#71)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#79, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#72)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#80, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#73)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#81, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#74)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#82, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#75)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#83, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#76)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#84, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#77)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#85] -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#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] +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] (36) Exchange -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] -Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#86] +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] (37) Sort [codegen id : 13] -Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] +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: [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 aa96edd4c7fba..f42a7615324c2 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 @@ -246,91 +246,91 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (33) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (35) Filter [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Condition : isnotnull(ss_customer_sk#26) (36) Project [codegen id : 11] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (37) Exchange Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#29] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#30] (38) Sort [codegen id : 12] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (39) Scan parquet default.customer -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 13] -Input [1]: [c_customer_sk#30] +Input [1]: [c_customer_sk#31] (41) Filter [codegen id : 13] -Input [1]: [c_customer_sk#30] -Condition : isnotnull(c_customer_sk#30) +Input [1]: [c_customer_sk#31] +Condition : isnotnull(c_customer_sk#31) (42) Exchange -Input [1]: [c_customer_sk#30] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [1]: [c_customer_sk#31] +Arguments: hashpartitioning(c_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] (43) Sort [codegen id : 14] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (44) SortMergeJoin [codegen id : 15] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#31] Join condition: None (45) Project [codegen id : 15] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#31] (46) HashAggregate [codegen id : 15] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Keys [1]: [c_customer_sk#31] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Aggregate Attributes [2]: [sum#33, isEmpty#34] +Results [3]: [c_customer_sk#31, sum#35, isEmpty#36] (47) HashAggregate [codegen id : 15] -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Keys [1]: [c_customer_sk#30] +Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] +Keys [1]: [c_customer_sk#31] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (48) Filter [codegen id : 15] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (49) Project [codegen id : 15] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#31] +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (50) Sort [codegen id : 15] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (51) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#31] Join condition: None (52) Project [codegen id : 17] @@ -338,209 +338,209 @@ Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (53) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_year#13, d_moy#40] +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 16] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (55) Filter [codegen id : 16] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#40)) AND (d_year#13 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#11)) +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) (56) Project [codegen id : 16] -Output [1]: [d_date_sk#11] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] +Output [1]: [d_date_sk#43] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (57) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] (58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#11] +Right keys [1]: [d_date_sk#43] Join condition: None (59) Project [codegen id : 17] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#42] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#11] +Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#47] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] (60) Scan parquet default.web_sales -Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#6)] ReadSchema: struct (61) ColumnarToRow [codegen id : 18] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (62) Exchange -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: hashpartitioning(ws_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: hashpartitioning(ws_item_sk#48, 5), ENSURE_REQUIREMENTS, [id=#53] (63) Sort [codegen id : 19] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: [ws_item_sk#43 ASC NULLS FIRST], false, 0 +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: [ws_item_sk#48 ASC NULLS FIRST], false, 0 (64) ReusedExchange [Reuses operator id: unknown] -Output [2]: [ss_item_sk#8, d_date#12] +Output [2]: [ss_item_sk#54, d_date#55] (65) Sort [codegen id : 22] -Input [2]: [ss_item_sk#8, d_date#12] -Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 +Input [2]: [ss_item_sk#54, d_date#55] +Arguments: [ss_item_sk#54 ASC NULLS FIRST], false, 0 (66) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#16, i_item_desc#17] +Output [2]: [i_item_sk#56, i_item_desc#57] (67) Sort [codegen id : 24] -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16 ASC NULLS FIRST], false, 0 +Input [2]: [i_item_sk#56, i_item_desc#57] +Arguments: [i_item_sk#56 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 25] -Left keys [1]: [ss_item_sk#8] -Right keys [1]: [i_item_sk#16] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#56] Join condition: None (69) Project [codegen id : 25] -Output [3]: [d_date#12, i_item_sk#16, i_item_desc#17] -Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#16, i_item_desc#17] +Output [3]: [d_date#55, i_item_sk#56, i_item_desc#57] +Input [4]: [ss_item_sk#54, d_date#55, i_item_sk#56, i_item_desc#57] (70) HashAggregate [codegen id : 25] -Input [3]: [d_date#12, i_item_sk#16, i_item_desc#17] -Keys [3]: [substr(i_item_desc#17, 1, 30) AS substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12] +Input [3]: [d_date#55, i_item_sk#56, i_item_desc#57] +Keys [3]: [substr(i_item_desc#57, 1, 30) AS substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#50] -Results [4]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12, count#51] +Aggregate Attributes [1]: [count#59] +Results [4]: [substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55, count#60] (71) HashAggregate [codegen id : 25] -Input [4]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12, count#51] -Keys [3]: [substr(i_item_desc#17, 1, 30)#49, i_item_sk#16, d_date#12] +Input [4]: [substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55, count#60] +Keys [3]: [substr(i_item_desc#57, 1, 30)#58, i_item_sk#56, d_date#55] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#52] -Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#52 AS count(1)#53] +Aggregate Attributes [1]: [count(1)#61] +Results [2]: [i_item_sk#56 AS item_sk#23, count(1)#61 AS count(1)#62] (72) Filter [codegen id : 25] -Input [2]: [item_sk#23, count(1)#53] -Condition : (count(1)#53 > 4) +Input [2]: [item_sk#23, count(1)#62] +Condition : (count(1)#62 > 4) (73) Project [codegen id : 25] Output [1]: [item_sk#23] -Input [2]: [item_sk#23, count(1)#53] +Input [2]: [item_sk#23, count(1)#62] (74) Sort [codegen id : 25] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 (75) SortMergeJoin -Left keys [1]: [ws_item_sk#43] +Left keys [1]: [ws_item_sk#48] Right keys [1]: [item_sk#23] Join condition: None (76) Project [codegen id : 26] -Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (77) Exchange -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: hashpartitioning(ws_bill_customer_sk#49, 5), ENSURE_REQUIREMENTS, [id=#63] (78) Sort [codegen id : 27] -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: [ws_bill_customer_sk#49 ASC NULLS FIRST], false, 0 (79) ReusedExchange [Reuses operator id: 37] -Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Output [3]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66] (80) Sort [codegen id : 29] -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66] +Arguments: [ss_customer_sk#64 ASC NULLS FIRST], false, 0 (81) ReusedExchange [Reuses operator id: 42] -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#67] (82) Sort [codegen id : 31] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#67] +Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 (83) SortMergeJoin [codegen id : 32] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ss_customer_sk#64] +Right keys [1]: [c_customer_sk#67] Join condition: None (84) Project [codegen id : 32] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#65, ss_sales_price#66, c_customer_sk#67] +Input [4]: [ss_customer_sk#64, ss_quantity#65, ss_sales_price#66, c_customer_sk#67] (85) HashAggregate [codegen id : 32] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#55, isEmpty#56] -Results [3]: [c_customer_sk#30, sum#57, isEmpty#58] +Input [3]: [ss_quantity#65, ss_sales_price#66, c_customer_sk#67] +Keys [1]: [c_customer_sk#67] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#68, isEmpty#69] +Results [3]: [c_customer_sk#67, sum#70, isEmpty#71] (86) HashAggregate [codegen id : 32] -Input [3]: [c_customer_sk#30, sum#57, isEmpty#58] -Keys [1]: [c_customer_sk#30] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#59] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#59 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] +Input [3]: [c_customer_sk#67, sum#70, isEmpty#71] +Keys [1]: [c_customer_sk#67] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))#72] +Results [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#65 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#66 as decimal(12,2)))), DecimalType(18,2), true))#72 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] (87) Filter [codegen id : 32] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (88) Project [codegen id : 32] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#60] +Output [1]: [c_customer_sk#67] +Input [2]: [c_customer_sk#67, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#73] (89) Sort [codegen id : 32] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#67] +Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#44] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ws_bill_customer_sk#49] +Right keys [1]: [c_customer_sk#67] Join condition: None (91) Project [codegen id : 34] -Output [3]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [3]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (92) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#74] (93) BroadcastHashJoin [codegen id : 34] -Left keys [1]: [ws_sold_date_sk#47] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ws_sold_date_sk#52] +Right keys [1]: [d_date_sk#74] Join condition: None (94) Project [codegen id : 34] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2), true) AS sales#61] -Input [4]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#11] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#51 as decimal(12,2)))), DecimalType(18,2), true) AS sales#75] +Input [4]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52, d_date_sk#74] (95) Union (96) HashAggregate [codegen id : 35] -Input [1]: [sales#42] +Input [1]: [sales#47] Keys: [] -Functions [1]: [partial_sum(sales#42)] -Aggregate Attributes [2]: [sum#62, isEmpty#63] -Results [2]: [sum#64, isEmpty#65] +Functions [1]: [partial_sum(sales#47)] +Aggregate Attributes [2]: [sum#76, isEmpty#77] +Results [2]: [sum#78, isEmpty#79] (97) Exchange -Input [2]: [sum#64, isEmpty#65] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] +Input [2]: [sum#78, isEmpty#79] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#80] (98) HashAggregate [codegen id : 36] -Input [2]: [sum#64, isEmpty#65] +Input [2]: [sum#78, isEmpty#79] Keys: [] -Functions [1]: [sum(sales#42)] -Aggregate Attributes [1]: [sum(sales#42)#67] -Results [1]: [sum(sales#42)#67 AS sum(sales)#68] +Functions [1]: [sum(sales#47)] +Aggregate Attributes [1]: [sum(sales#47)#81] +Results [1]: [sum(sales#47)#81 AS sum(sales)#82] ===== Subqueries ===== @@ -549,7 +549,7 @@ ReusedExchange (99) (99) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#43] Subquery:2 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 ReusedExchange (100) @@ -558,7 +558,7 @@ ReusedExchange (100) (100) ReusedExchange [Reuses operator id: 12] Output [2]: [d_date_sk#11, d_date#12] -Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (124) +- Exchange (123) +- * HashAggregate (122) @@ -586,131 +586,131 @@ Subquery:3 Hosting operator id = 48 Hosting Expression = Subquery scalar-subquer (101) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Output [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#69)] +PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#87)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] (103) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] -Condition : isnotnull(ss_customer_sk#26) +Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86] +Condition : isnotnull(ss_customer_sk#83) (104) Scan parquet default.date_dim -Output [2]: [d_date_sk#11, d_year#13] +Output [2]: [d_date_sk#88, d_year#89] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (105) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#11, d_year#13] +Input [2]: [d_date_sk#88, d_year#89] (106) Filter [codegen id : 1] -Input [2]: [d_date_sk#11, d_year#13] -Condition : (d_year#13 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) +Input [2]: [d_date_sk#88, d_year#89] +Condition : (d_year#89 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#88)) (107) Project [codegen id : 1] -Output [1]: [d_date_sk#11] -Input [2]: [d_date_sk#11, d_year#13] +Output [1]: [d_date_sk#88] +Input [2]: [d_date_sk#88, d_year#89] (108) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#70] +Input [1]: [d_date_sk#88] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#90] (109) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ss_sold_date_sk#86] +Right keys [1]: [d_date_sk#88] Join condition: None (110) Project [codegen id : 2] -Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9, d_date_sk#11] +Output [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] +Input [5]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, ss_sold_date_sk#86, d_date_sk#88] (111) Exchange -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] +Arguments: hashpartitioning(ss_customer_sk#83, 5), ENSURE_REQUIREMENTS, [id=#91] (112) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85] +Arguments: [ss_customer_sk#83 ASC NULLS FIRST], false, 0 (113) Scan parquet default.customer -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#92] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (114) ColumnarToRow [codegen id : 4] -Input [1]: [c_customer_sk#30] +Input [1]: [c_customer_sk#92] (115) Filter [codegen id : 4] -Input [1]: [c_customer_sk#30] -Condition : isnotnull(c_customer_sk#30) +Input [1]: [c_customer_sk#92] +Condition : isnotnull(c_customer_sk#92) (116) Exchange -Input [1]: [c_customer_sk#30] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#72] +Input [1]: [c_customer_sk#92] +Arguments: hashpartitioning(c_customer_sk#92, 5), ENSURE_REQUIREMENTS, [id=#93] (117) Sort [codegen id : 5] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#92] +Arguments: [c_customer_sk#92 ASC NULLS FIRST], false, 0 (118) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ss_customer_sk#83] +Right keys [1]: [c_customer_sk#92] Join condition: None (119) Project [codegen id : 6] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#92] +Input [4]: [ss_customer_sk#83, ss_quantity#84, ss_sales_price#85, c_customer_sk#92] (120) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#73, isEmpty#74] -Results [3]: [c_customer_sk#30, sum#75, isEmpty#76] +Input [3]: [ss_quantity#84, ss_sales_price#85, c_customer_sk#92] +Keys [1]: [c_customer_sk#92] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#94, isEmpty#95] +Results [3]: [c_customer_sk#92, sum#96, isEmpty#97] (121) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#30, sum#75, isEmpty#76] -Keys [1]: [c_customer_sk#30] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#77] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#77 AS csales#78] +Input [3]: [c_customer_sk#92, sum#96, isEmpty#97] +Keys [1]: [c_customer_sk#92] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#85 as decimal(12,2)))), DecimalType(18,2), true))#98 AS csales#99] (122) HashAggregate [codegen id : 6] -Input [1]: [csales#78] +Input [1]: [csales#99] Keys: [] -Functions [1]: [partial_max(csales#78)] -Aggregate Attributes [1]: [max#79] -Results [1]: [max#80] +Functions [1]: [partial_max(csales#99)] +Aggregate Attributes [1]: [max#100] +Results [1]: [max#101] (123) Exchange -Input [1]: [max#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +Input [1]: [max#101] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#102] (124) HashAggregate [codegen id : 7] -Input [1]: [max#80] +Input [1]: [max#101] Keys: [] -Functions [1]: [max(csales#78)] -Aggregate Attributes [1]: [max(csales#78)#82] -Results [1]: [max(csales#78)#82 AS tpcds_cmax#83] +Functions [1]: [max(csales#99)] +Aggregate Attributes [1]: [max(csales#99)#103] +Results [1]: [max(csales#99)#103 AS tpcds_cmax#104] -Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#69 +Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#87 ReusedExchange (125) (125) ReusedExchange [Reuses operator id: 108] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#88] -Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:6 Hosting operator id = 87 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] 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 692671c9f287d..8c2aed03ce0cc 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 @@ -206,83 +206,83 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (29) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (31) Filter [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Condition : isnotnull(ss_customer_sk#26) (32) Project [codegen id : 8] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (33) Scan parquet default.customer -Output [1]: [c_customer_sk#29] +Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 7] -Input [1]: [c_customer_sk#29] +Input [1]: [c_customer_sk#30] (35) Filter [codegen id : 7] -Input [1]: [c_customer_sk#29] -Condition : isnotnull(c_customer_sk#29) +Input [1]: [c_customer_sk#30] +Condition : isnotnull(c_customer_sk#30) (36) BroadcastExchange -Input [1]: [c_customer_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [1]: [c_customer_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (37) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#30] Join condition: None (38) Project [codegen id : 8] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (39) HashAggregate [codegen id : 8] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#31, isEmpty#32] -Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] (40) Exchange -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#36] (41) HashAggregate [codegen id : 9] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (42) Filter [codegen id : 9] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (43) Project [codegen id : 9] -Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (44) Sort [codegen id : 9] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (45) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#30] Join condition: None (46) Project [codegen id : 11] @@ -290,128 +290,128 @@ Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] (47) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_year#12, d_moy#40] +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 10] -Input [3]: [d_date_sk#10, d_year#12, d_moy#40] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (49) Filter [codegen id : 10] -Input [3]: [d_date_sk#10, d_year#12, d_moy#40] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#40)) AND (d_year#12 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#10)) +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) (50) Project [codegen id : 10] -Output [1]: [d_date_sk#10] -Input [3]: [d_date_sk#10, d_year#12, d_moy#40] +Output [1]: [d_date_sk#43] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (51) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] (52) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#10] +Right keys [1]: [d_date_sk#43] Join condition: None (53) Project [codegen id : 11] -Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#42] -Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#10] +Output [1]: [CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true) AS sales#47] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] (54) Scan parquet default.web_sales -Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#6)] ReadSchema: struct (55) ColumnarToRow [codegen id : 16] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (56) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] (57) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ws_item_sk#43] +Left keys [1]: [ws_item_sk#48] Right keys [1]: [item_sk#22] Join condition: None (58) Project [codegen id : 16] -Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [5]: [ws_item_sk#48, ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (59) Exchange -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: hashpartitioning(ws_bill_customer_sk#49, 5), ENSURE_REQUIREMENTS, [id=#53] (60) Sort [codegen id : 17] -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Arguments: [ws_bill_customer_sk#49 ASC NULLS FIRST], false, 0 (61) ReusedExchange [Reuses operator id: 40] -Output [3]: [c_customer_sk#29, sum#49, isEmpty#50] +Output [3]: [c_customer_sk#54, sum#55, isEmpty#56] (62) HashAggregate [codegen id : 20] -Input [3]: [c_customer_sk#29, sum#49, isEmpty#50] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#51] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#51 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] +Input [3]: [c_customer_sk#54, sum#55, isEmpty#56] +Keys [1]: [c_customer_sk#54] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))#59] +Results [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#57 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#58 as decimal(12,2)))), DecimalType(18,2), true))#59 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] (63) Filter [codegen id : 20] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (64) Project [codegen id : 20] -Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#52] +Output [1]: [c_customer_sk#54] +Input [2]: [c_customer_sk#54, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#60] (65) Sort [codegen id : 20] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#54] +Arguments: [c_customer_sk#54 ASC NULLS FIRST], false, 0 (66) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#44] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ws_bill_customer_sk#49] +Right keys [1]: [c_customer_sk#54] Join condition: None (67) Project [codegen id : 22] -Output [3]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] -Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Output [3]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] +Input [4]: [ws_bill_customer_sk#49, ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52] (68) ReusedExchange [Reuses operator id: 51] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#61] (69) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#47] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#52] +Right keys [1]: [d_date_sk#61] Join condition: None (70) Project [codegen id : 22] -Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#45 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2), true) AS sales#53] -Input [4]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#10] +Output [1]: [CheckOverflow((promote_precision(cast(cast(ws_quantity#50 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#51 as decimal(12,2)))), DecimalType(18,2), true) AS sales#62] +Input [4]: [ws_quantity#50, ws_list_price#51, ws_sold_date_sk#52, d_date_sk#61] (71) Union (72) HashAggregate [codegen id : 23] -Input [1]: [sales#42] +Input [1]: [sales#47] Keys: [] -Functions [1]: [partial_sum(sales#42)] -Aggregate Attributes [2]: [sum#54, isEmpty#55] -Results [2]: [sum#56, isEmpty#57] +Functions [1]: [partial_sum(sales#47)] +Aggregate Attributes [2]: [sum#63, isEmpty#64] +Results [2]: [sum#65, isEmpty#66] (73) Exchange -Input [2]: [sum#56, isEmpty#57] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [sum#65, isEmpty#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] (74) HashAggregate [codegen id : 24] -Input [2]: [sum#56, isEmpty#57] +Input [2]: [sum#65, isEmpty#66] Keys: [] -Functions [1]: [sum(sales#42)] -Aggregate Attributes [1]: [sum(sales#42)#59] -Results [1]: [sum(sales#42)#59 AS sum(sales)#60] +Functions [1]: [sum(sales#47)] +Aggregate Attributes [1]: [sum(sales#47)#68] +Results [1]: [sum(sales#47)#68 AS sum(sales)#69] ===== Subqueries ===== @@ -420,7 +420,7 @@ ReusedExchange (75) (75) ReusedExchange [Reuses operator id: 51] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#43] Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 ReusedExchange (76) @@ -429,7 +429,7 @@ ReusedExchange (76) (76) ReusedExchange [Reuses operator id: 10] Output [2]: [d_date_sk#10, d_date#11] -Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (98) +- Exchange (97) +- * HashAggregate (96) @@ -455,123 +455,123 @@ Subquery:3 Hosting operator id = 42 Hosting Expression = Subquery scalar-subquer (77) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Output [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(ss_sold_date_sk#73), dynamicpruningexpression(ss_sold_date_sk#73 IN dynamicpruning#74)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (78) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] (79) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] -Condition : isnotnull(ss_customer_sk#26) +Input [4]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73] +Condition : isnotnull(ss_customer_sk#70) (80) Scan parquet default.customer -Output [1]: [c_customer_sk#29] +Output [1]: [c_customer_sk#75] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (81) ColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#29] +Input [1]: [c_customer_sk#75] (82) Filter [codegen id : 1] -Input [1]: [c_customer_sk#29] -Condition : isnotnull(c_customer_sk#29) +Input [1]: [c_customer_sk#75] +Condition : isnotnull(c_customer_sk#75) (83) BroadcastExchange -Input [1]: [c_customer_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#62] +Input [1]: [c_customer_sk#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] (84) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ss_customer_sk#70] +Right keys [1]: [c_customer_sk#75] Join condition: None (85) Project [codegen id : 3] -Output [4]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] -Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] +Output [4]: [ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75] +Input [5]: [ss_customer_sk#70, ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75] (86) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#12] +Output [2]: [d_date_sk#77, d_year#78] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#10, d_year#12] +Input [2]: [d_date_sk#77, d_year#78] (88) Filter [codegen id : 2] -Input [2]: [d_date_sk#10, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +Input [2]: [d_date_sk#77, d_year#78] +Condition : (d_year#78 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#77)) (89) Project [codegen id : 2] -Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_year#12] +Output [1]: [d_date_sk#77] +Input [2]: [d_date_sk#77, d_year#78] (90) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] +Input [1]: [d_date_sk#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#79] (91) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#73] +Right keys [1]: [d_date_sk#77] Join condition: None (92) Project [codegen id : 3] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Input [5]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29, d_date_sk#10] +Output [3]: [ss_quantity#71, ss_sales_price#72, c_customer_sk#75] +Input [5]: [ss_quantity#71, ss_sales_price#72, ss_sold_date_sk#73, c_customer_sk#75, d_date_sk#77] (93) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#64, isEmpty#65] -Results [3]: [c_customer_sk#29, sum#66, isEmpty#67] +Input [3]: [ss_quantity#71, ss_sales_price#72, c_customer_sk#75] +Keys [1]: [c_customer_sk#75] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#80, isEmpty#81] +Results [3]: [c_customer_sk#75, sum#82, isEmpty#83] (94) Exchange -Input [3]: [c_customer_sk#29, sum#66, isEmpty#67] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [3]: [c_customer_sk#75, sum#82, isEmpty#83] +Arguments: hashpartitioning(c_customer_sk#75, 5), ENSURE_REQUIREMENTS, [id=#84] (95) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#29, sum#66, isEmpty#67] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS csales#70] +Input [3]: [c_customer_sk#75, sum#82, isEmpty#83] +Keys [1]: [c_customer_sk#75] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))#85] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#71 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#72 as decimal(12,2)))), DecimalType(18,2), true))#85 AS csales#86] (96) HashAggregate [codegen id : 4] -Input [1]: [csales#70] +Input [1]: [csales#86] Keys: [] -Functions [1]: [partial_max(csales#70)] -Aggregate Attributes [1]: [max#71] -Results [1]: [max#72] +Functions [1]: [partial_max(csales#86)] +Aggregate Attributes [1]: [max#87] +Results [1]: [max#88] (97) Exchange -Input [1]: [max#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#73] +Input [1]: [max#88] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#89] (98) HashAggregate [codegen id : 5] -Input [1]: [max#72] +Input [1]: [max#88] Keys: [] -Functions [1]: [max(csales#70)] -Aggregate Attributes [1]: [max(csales#70)#74] -Results [1]: [max(csales#70)#74 AS tpcds_cmax#75] +Functions [1]: [max(csales#86)] +Aggregate Attributes [1]: [max(csales#86)#90] +Results [1]: [max(csales#86)#90 AS tpcds_cmax#91] -Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#61 +Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#73 IN dynamicpruning#74 ReusedExchange (99) (99) ReusedExchange [Reuses operator id: 90] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#77] -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#6 -Subquery:6 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:6 Hosting operator id = 63 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] 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 35a42fa256325..e75b223ea599a 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 @@ -290,145 +290,145 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (34) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (36) Filter [codegen id : 11] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Condition : isnotnull(ss_customer_sk#26) (37) Project [codegen id : 11] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (38) Exchange Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#29] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#30] (39) Sort [codegen id : 12] Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (40) Scan parquet default.customer -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 13] -Input [1]: [c_customer_sk#30] +Input [1]: [c_customer_sk#31] (42) Filter [codegen id : 13] -Input [1]: [c_customer_sk#30] -Condition : isnotnull(c_customer_sk#30) +Input [1]: [c_customer_sk#31] +Condition : isnotnull(c_customer_sk#31) (43) Exchange -Input [1]: [c_customer_sk#30] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [1]: [c_customer_sk#31] +Arguments: hashpartitioning(c_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#32] (44) Sort [codegen id : 14] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 15] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#31] Join condition: None (46) Project [codegen id : 15] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#31] (47) HashAggregate [codegen id : 15] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Keys [1]: [c_customer_sk#31] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Aggregate Attributes [2]: [sum#33, isEmpty#34] +Results [3]: [c_customer_sk#31, sum#35, isEmpty#36] (48) HashAggregate [codegen id : 15] -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Keys [1]: [c_customer_sk#30] +Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] +Keys [1]: [c_customer_sk#31] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (49) Filter [codegen id : 15] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (50) Project [codegen id : 15] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#31] +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (51) Sort [codegen id : 15] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (52) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#31] Join condition: None (53) Scan parquet default.date_dim -Output [3]: [d_date_sk#11, d_year#13, d_moy#40] +Output [3]: [d_date_sk#43, d_year#44, d_moy#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 16] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (55) Filter [codegen id : 16] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] -Condition : ((((isnotnull(d_year#13) AND isnotnull(d_moy#40)) AND (d_year#13 = 2000)) AND (d_moy#40 = 2)) AND isnotnull(d_date_sk#11)) +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#43)) (56) Project [codegen id : 16] -Output [1]: [d_date_sk#11] -Input [3]: [d_date_sk#11, d_year#13, d_moy#40] +Output [1]: [d_date_sk#43] +Input [3]: [d_date_sk#43, d_year#44, d_moy#45] (57) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#41] +Input [1]: [d_date_sk#43] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] (58) BroadcastHashJoin [codegen id : 17] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#11] +Right keys [1]: [d_date_sk#43] Join condition: None (59) Project [codegen id : 17] Output [3]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4] -Input [5]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#11] +Input [5]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#43] (60) Scan parquet default.customer -Output [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Output [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 18] -Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] (62) Filter [codegen id : 18] -Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] -Condition : isnotnull(c_customer_sk#30) +Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] +Condition : isnotnull(c_customer_sk#47) (63) Exchange -Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] +Arguments: hashpartitioning(c_customer_sk#47, 5), ENSURE_REQUIREMENTS, [id=#50] (64) Sort [codegen id : 19] -Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#47, c_first_name#48, c_last_name#49] +Arguments: [c_customer_sk#47 ASC NULLS FIRST], false, 0 (65) ReusedExchange [Reuses operator id: 38] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] @@ -438,327 +438,327 @@ Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (67) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#31] (68) Sort [codegen id : 23] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (69) SortMergeJoin [codegen id : 24] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#31] Join condition: None (70) Project [codegen id : 24] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#31] (71) HashAggregate [codegen id : 24] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#31] +Keys [1]: [c_customer_sk#31] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#32, isEmpty#33] -Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Aggregate Attributes [2]: [sum#33, isEmpty#34] +Results [3]: [c_customer_sk#31, sum#35, isEmpty#36] (72) HashAggregate [codegen id : 24] -Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] -Keys [1]: [c_customer_sk#30] +Input [3]: [c_customer_sk#31, sum#35, isEmpty#36] +Keys [1]: [c_customer_sk#31] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (73) Filter [codegen id : 24] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (74) Project [codegen id : 24] -Output [1]: [c_customer_sk#30 AS c_customer_sk#30#45] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#31] +Input [2]: [c_customer_sk#31, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (75) Sort [codegen id : 24] -Input [1]: [c_customer_sk#30#45] -Arguments: [c_customer_sk#30#45 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#31] +Arguments: [c_customer_sk#31 ASC NULLS FIRST], false, 0 (76) SortMergeJoin -Left keys [1]: [c_customer_sk#30] -Right keys [1]: [c_customer_sk#30#45] +Left keys [1]: [c_customer_sk#47] +Right keys [1]: [c_customer_sk#31] Join condition: None (77) SortMergeJoin [codegen id : 25] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#30] +Right keys [1]: [c_customer_sk#47] Join condition: None (78) Project [codegen id : 25] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#42, c_last_name#43] -Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#30, c_first_name#42, c_last_name#43] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#48, c_last_name#49] +Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk#47, c_first_name#48, c_last_name#49] (79) HashAggregate [codegen id : 25] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#42, c_last_name#43] -Keys [2]: [c_last_name#43, c_first_name#42] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#48, c_last_name#49] +Keys [2]: [c_last_name#49, c_first_name#48] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#46, isEmpty#47] -Results [4]: [c_last_name#43, c_first_name#42, sum#48, isEmpty#49] +Aggregate Attributes [2]: [sum#51, isEmpty#52] +Results [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] (80) Exchange -Input [4]: [c_last_name#43, c_first_name#42, sum#48, isEmpty#49] -Arguments: hashpartitioning(c_last_name#43, c_first_name#42, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] +Arguments: hashpartitioning(c_last_name#49, c_first_name#48, 5), ENSURE_REQUIREMENTS, [id=#55] (81) HashAggregate [codegen id : 26] -Input [4]: [c_last_name#43, c_first_name#42, sum#48, isEmpty#49] -Keys [2]: [c_last_name#43, c_first_name#42] +Input [4]: [c_last_name#49, c_first_name#48, sum#53, isEmpty#54] +Keys [2]: [c_last_name#49, c_first_name#48] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#51] -Results [3]: [c_last_name#43, c_first_name#42, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#51 AS sales#52] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#56] +Results [3]: [c_last_name#49, c_first_name#48, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#56 AS sales#57] (82) Scan parquet default.web_sales -Output [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Output [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#57), dynamicpruningexpression(ws_sold_date_sk#57 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (83) ColumnarToRow [codegen id : 27] -Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] (84) Filter [codegen id : 27] -Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Condition : isnotnull(ws_bill_customer_sk#54) +Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Condition : isnotnull(ws_bill_customer_sk#59) (85) Exchange -Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Arguments: hashpartitioning(ws_item_sk#53, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Arguments: hashpartitioning(ws_item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#63] (86) Sort [codegen id : 28] -Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Arguments: [ws_item_sk#53 ASC NULLS FIRST], false, 0 +Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Arguments: [ws_item_sk#58 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: unknown] -Output [2]: [ss_item_sk#8, d_date#12] +Output [2]: [ss_item_sk#64, d_date#65] (88) Sort [codegen id : 31] -Input [2]: [ss_item_sk#8, d_date#12] -Arguments: [ss_item_sk#8 ASC NULLS FIRST], false, 0 +Input [2]: [ss_item_sk#64, d_date#65] +Arguments: [ss_item_sk#64 ASC NULLS FIRST], false, 0 (89) ReusedExchange [Reuses operator id: 21] -Output [2]: [i_item_sk#16, i_item_desc#17] +Output [2]: [i_item_sk#66, i_item_desc#67] (90) Sort [codegen id : 33] -Input [2]: [i_item_sk#16, i_item_desc#17] -Arguments: [i_item_sk#16 ASC NULLS FIRST], false, 0 +Input [2]: [i_item_sk#66, i_item_desc#67] +Arguments: [i_item_sk#66 ASC NULLS FIRST], false, 0 (91) SortMergeJoin [codegen id : 34] -Left keys [1]: [ss_item_sk#8] -Right keys [1]: [i_item_sk#16] +Left keys [1]: [ss_item_sk#64] +Right keys [1]: [i_item_sk#66] Join condition: None (92) Project [codegen id : 34] -Output [3]: [d_date#12, i_item_sk#16, i_item_desc#17] -Input [4]: [ss_item_sk#8, d_date#12, i_item_sk#16, i_item_desc#17] +Output [3]: [d_date#65, i_item_sk#66, i_item_desc#67] +Input [4]: [ss_item_sk#64, d_date#65, i_item_sk#66, i_item_desc#67] (93) HashAggregate [codegen id : 34] -Input [3]: [d_date#12, i_item_sk#16, i_item_desc#17] -Keys [3]: [substr(i_item_desc#17, 1, 30) AS substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12] +Input [3]: [d_date#65, i_item_sk#66, i_item_desc#67] +Keys [3]: [substr(i_item_desc#67, 1, 30) AS substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#60] -Results [4]: [substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12, count#61] +Aggregate Attributes [1]: [count#69] +Results [4]: [substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65, count#70] (94) HashAggregate [codegen id : 34] -Input [4]: [substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12, count#61] -Keys [3]: [substr(i_item_desc#17, 1, 30)#59, i_item_sk#16, d_date#12] +Input [4]: [substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65, count#70] +Keys [3]: [substr(i_item_desc#67, 1, 30)#68, i_item_sk#66, d_date#65] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#62] -Results [2]: [i_item_sk#16 AS item_sk#23, count(1)#62 AS count(1)#63] +Aggregate Attributes [1]: [count(1)#71] +Results [2]: [i_item_sk#66 AS item_sk#23, count(1)#71 AS count(1)#72] (95) Filter [codegen id : 34] -Input [2]: [item_sk#23, count(1)#63] -Condition : (count(1)#63 > 4) +Input [2]: [item_sk#23, count(1)#72] +Condition : (count(1)#72 > 4) (96) Project [codegen id : 34] Output [1]: [item_sk#23] -Input [2]: [item_sk#23, count(1)#63] +Input [2]: [item_sk#23, count(1)#72] (97) Sort [codegen id : 34] Input [1]: [item_sk#23] Arguments: [item_sk#23 ASC NULLS FIRST], false, 0 (98) SortMergeJoin -Left keys [1]: [ws_item_sk#53] +Left keys [1]: [ws_item_sk#58] Right keys [1]: [item_sk#23] Join condition: None (99) Project [codegen id : 35] -Output [4]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Input [5]: [ws_item_sk#53, ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] +Output [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Input [5]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] (100) Exchange -Input [4]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Arguments: hashpartitioning(ws_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Arguments: hashpartitioning(ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#73] (101) Sort [codegen id : 36] -Input [4]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57] -Arguments: [ws_bill_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62] +Arguments: [ws_bill_customer_sk#59 ASC NULLS FIRST], false, 0 (102) ReusedExchange [Reuses operator id: 38] -Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Output [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] (103) Sort [codegen id : 38] -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] +Arguments: [ss_customer_sk#74 ASC NULLS FIRST], false, 0 (104) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#77] (105) Sort [codegen id : 40] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#77] +Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 (106) SortMergeJoin [codegen id : 41] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ss_customer_sk#74] +Right keys [1]: [c_customer_sk#77] Join condition: None (107) Project [codegen id : 41] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, c_customer_sk#77] (108) HashAggregate [codegen id : 41] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#65, isEmpty#66] -Results [3]: [c_customer_sk#30, sum#67, isEmpty#68] +Input [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Keys [1]: [c_customer_sk#77] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#78, isEmpty#79] +Results [3]: [c_customer_sk#77, sum#80, isEmpty#81] (109) HashAggregate [codegen id : 41] -Input [3]: [c_customer_sk#30, sum#67, isEmpty#68] -Keys [1]: [c_customer_sk#30] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] +Input [3]: [c_customer_sk#77, sum#80, isEmpty#81] +Keys [1]: [c_customer_sk#77] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82] +Results [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] (110) Filter [codegen id : 41] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (111) Project [codegen id : 41] -Output [1]: [c_customer_sk#30] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] +Output [1]: [c_customer_sk#77] +Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] (112) Sort [codegen id : 41] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#77] +Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 (113) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#54] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ws_bill_customer_sk#59] +Right keys [1]: [c_customer_sk#77] Join condition: None (114) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#84] (115) BroadcastHashJoin [codegen id : 43] -Left keys [1]: [ws_sold_date_sk#57] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ws_sold_date_sk#62] +Right keys [1]: [d_date_sk#84] Join condition: None (116) Project [codegen id : 43] -Output [3]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56] -Input [5]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, ws_sold_date_sk#57, d_date_sk#11] +Output [3]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61] +Input [5]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, ws_sold_date_sk#62, d_date_sk#84] (117) ReusedExchange [Reuses operator id: 63] -Output [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] +Output [3]: [c_customer_sk#85, c_first_name#86, c_last_name#87] (118) Sort [codegen id : 45] -Input [3]: [c_customer_sk#30, c_first_name#42, c_last_name#43] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#85, c_first_name#86, c_last_name#87] +Arguments: [c_customer_sk#85 ASC NULLS FIRST], false, 0 (119) ReusedExchange [Reuses operator id: 38] -Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] +Output [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] (120) Sort [codegen id : 47] -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76] +Arguments: [ss_customer_sk#74 ASC NULLS FIRST], false, 0 (121) ReusedExchange [Reuses operator id: 43] -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#77] (122) Sort [codegen id : 49] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#77] +Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 (123) SortMergeJoin [codegen id : 50] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ss_customer_sk#74] +Right keys [1]: [c_customer_sk#77] Join condition: None (124) Project [codegen id : 50] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Input [4]: [ss_customer_sk#74, ss_quantity#75, ss_sales_price#76, c_customer_sk#77] (125) HashAggregate [codegen id : 50] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#65, isEmpty#66] -Results [3]: [c_customer_sk#30, sum#67, isEmpty#68] +Input [3]: [ss_quantity#75, ss_sales_price#76, c_customer_sk#77] +Keys [1]: [c_customer_sk#77] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#78, isEmpty#79] +Results [3]: [c_customer_sk#77, sum#80, isEmpty#81] (126) HashAggregate [codegen id : 50] -Input [3]: [c_customer_sk#30, sum#67, isEmpty#68] -Keys [1]: [c_customer_sk#30] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69] -Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#69 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] +Input [3]: [c_customer_sk#77, sum#80, isEmpty#81] +Keys [1]: [c_customer_sk#77] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82] +Results [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#75 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#76 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] (127) Filter [codegen id : 50] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (128) Project [codegen id : 50] -Output [1]: [c_customer_sk#30 AS c_customer_sk#30#71] -Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#70] +Output [1]: [c_customer_sk#77] +Input [2]: [c_customer_sk#77, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#83] (129) Sort [codegen id : 50] -Input [1]: [c_customer_sk#30#71] -Arguments: [c_customer_sk#30#71 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#77] +Arguments: [c_customer_sk#77 ASC NULLS FIRST], false, 0 (130) SortMergeJoin -Left keys [1]: [c_customer_sk#30] -Right keys [1]: [c_customer_sk#30#71] +Left keys [1]: [c_customer_sk#85] +Right keys [1]: [c_customer_sk#77] Join condition: None (131) SortMergeJoin [codegen id : 51] -Left keys [1]: [ws_bill_customer_sk#54] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ws_bill_customer_sk#59] +Right keys [1]: [c_customer_sk#85] Join condition: None (132) Project [codegen id : 51] -Output [4]: [ws_quantity#55, ws_list_price#56, c_first_name#42, c_last_name#43] -Input [6]: [ws_bill_customer_sk#54, ws_quantity#55, ws_list_price#56, c_customer_sk#30, c_first_name#42, c_last_name#43] +Output [4]: [ws_quantity#60, ws_list_price#61, c_first_name#86, c_last_name#87] +Input [6]: [ws_bill_customer_sk#59, ws_quantity#60, ws_list_price#61, c_customer_sk#85, c_first_name#86, c_last_name#87] (133) HashAggregate [codegen id : 51] -Input [4]: [ws_quantity#55, ws_list_price#56, c_first_name#42, c_last_name#43] -Keys [2]: [c_last_name#43, c_first_name#42] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#72, isEmpty#73] -Results [4]: [c_last_name#43, c_first_name#42, sum#74, isEmpty#75] +Input [4]: [ws_quantity#60, ws_list_price#61, c_first_name#86, c_last_name#87] +Keys [2]: [c_last_name#87, c_first_name#86] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#88, isEmpty#89] +Results [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] (134) Exchange -Input [4]: [c_last_name#43, c_first_name#42, sum#74, isEmpty#75] -Arguments: hashpartitioning(c_last_name#43, c_first_name#42, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] +Arguments: hashpartitioning(c_last_name#87, c_first_name#86, 5), ENSURE_REQUIREMENTS, [id=#92] (135) HashAggregate [codegen id : 52] -Input [4]: [c_last_name#43, c_first_name#42, sum#74, isEmpty#75] -Keys [2]: [c_last_name#43, c_first_name#42] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))#77] -Results [3]: [c_last_name#43, c_first_name#42, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#55 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#56 as decimal(12,2)))), DecimalType(18,2), true))#77 AS sales#78] +Input [4]: [c_last_name#87, c_first_name#86, sum#90, isEmpty#91] +Keys [2]: [c_last_name#87, c_first_name#86] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#93] +Results [3]: [c_last_name#87, c_first_name#86, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#60 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#61 as decimal(12,2)))), DecimalType(18,2), true))#93 AS sales#94] (136) Union (137) TakeOrderedAndProject -Input [3]: [c_last_name#43, c_first_name#42, sales#52] -Arguments: 100, [c_last_name#43 ASC NULLS FIRST, c_first_name#42 ASC NULLS FIRST, sales#52 ASC NULLS FIRST], [c_last_name#43, c_first_name#42, sales#52] +Input [3]: [c_last_name#49, c_first_name#48, sales#57] +Arguments: 100, [c_last_name#49 ASC NULLS FIRST, c_first_name#48 ASC NULLS FIRST, sales#57 ASC NULLS FIRST], [c_last_name#49, c_first_name#48, sales#57] ===== Subqueries ===== @@ -767,7 +767,7 @@ ReusedExchange (138) (138) ReusedExchange [Reuses operator id: 57] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#43] Subquery:2 Hosting operator id = 6 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 ReusedExchange (139) @@ -776,7 +776,7 @@ ReusedExchange (139) (139) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#11, d_date#12] -Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (163) +- Exchange (162) +- * HashAggregate (161) @@ -804,135 +804,135 @@ Subquery:3 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (140) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Output [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#79)] +PartitionFilters: [isnotnull(ss_sold_date_sk#98), dynamicpruningexpression(ss_sold_date_sk#98 IN dynamicpruning#99)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (141) ColumnarToRow [codegen id : 2] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] +Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] (142) Filter [codegen id : 2] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9] -Condition : isnotnull(ss_customer_sk#26) +Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98] +Condition : isnotnull(ss_customer_sk#95) (143) Scan parquet default.date_dim -Output [2]: [d_date_sk#11, d_year#13] +Output [2]: [d_date_sk#100, d_year#101] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] ReadSchema: struct (144) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#11, d_year#13] +Input [2]: [d_date_sk#100, d_year#101] (145) Filter [codegen id : 1] -Input [2]: [d_date_sk#11, d_year#13] -Condition : (d_year#13 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#11)) +Input [2]: [d_date_sk#100, d_year#101] +Condition : (d_year#101 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#100)) (146) Project [codegen id : 1] -Output [1]: [d_date_sk#11] -Input [2]: [d_date_sk#11, d_year#13] +Output [1]: [d_date_sk#100] +Input [2]: [d_date_sk#100, d_year#101] (147) BroadcastExchange -Input [1]: [d_date_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#80] +Input [1]: [d_date_sk#100] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#102] (148) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#9] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ss_sold_date_sk#98] +Right keys [1]: [d_date_sk#100] Join condition: None (149) Project [codegen id : 2] -Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#9, d_date_sk#11] +Output [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] +Input [5]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, ss_sold_date_sk#98, d_date_sk#100] (150) Exchange -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] +Arguments: hashpartitioning(ss_customer_sk#95, 5), ENSURE_REQUIREMENTS, [id=#103] (151) Sort [codegen id : 3] -Input [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97] +Arguments: [ss_customer_sk#95 ASC NULLS FIRST], false, 0 (152) Scan parquet default.customer -Output [1]: [c_customer_sk#30] +Output [1]: [c_customer_sk#104] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (153) ColumnarToRow [codegen id : 4] -Input [1]: [c_customer_sk#30] +Input [1]: [c_customer_sk#104] (154) Filter [codegen id : 4] -Input [1]: [c_customer_sk#30] -Condition : isnotnull(c_customer_sk#30) +Input [1]: [c_customer_sk#104] +Condition : isnotnull(c_customer_sk#104) (155) Exchange -Input [1]: [c_customer_sk#30] -Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#82] +Input [1]: [c_customer_sk#104] +Arguments: hashpartitioning(c_customer_sk#104, 5), ENSURE_REQUIREMENTS, [id=#105] (156) Sort [codegen id : 5] -Input [1]: [c_customer_sk#30] -Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#104] +Arguments: [c_customer_sk#104 ASC NULLS FIRST], false, 0 (157) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#30] +Left keys [1]: [ss_customer_sk#95] +Right keys [1]: [c_customer_sk#104] Join condition: None (158) Project [codegen id : 6] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Output [3]: [ss_quantity#96, ss_sales_price#97, c_customer_sk#104] +Input [4]: [ss_customer_sk#95, ss_quantity#96, ss_sales_price#97, c_customer_sk#104] (159) HashAggregate [codegen id : 6] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] -Keys [1]: [c_customer_sk#30] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#83, isEmpty#84] -Results [3]: [c_customer_sk#30, sum#85, isEmpty#86] +Input [3]: [ss_quantity#96, ss_sales_price#97, c_customer_sk#104] +Keys [1]: [c_customer_sk#104] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#106, isEmpty#107] +Results [3]: [c_customer_sk#104, sum#108, isEmpty#109] (160) HashAggregate [codegen id : 6] -Input [3]: [c_customer_sk#30, sum#85, isEmpty#86] -Keys [1]: [c_customer_sk#30] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#87] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#87 AS csales#88] +Input [3]: [c_customer_sk#104, sum#108, isEmpty#109] +Keys [1]: [c_customer_sk#104] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))#110] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#96 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#97 as decimal(12,2)))), DecimalType(18,2), true))#110 AS csales#111] (161) HashAggregate [codegen id : 6] -Input [1]: [csales#88] +Input [1]: [csales#111] Keys: [] -Functions [1]: [partial_max(csales#88)] -Aggregate Attributes [1]: [max#89] -Results [1]: [max#90] +Functions [1]: [partial_max(csales#111)] +Aggregate Attributes [1]: [max#112] +Results [1]: [max#113] (162) Exchange -Input [1]: [max#90] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#91] +Input [1]: [max#113] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] (163) HashAggregate [codegen id : 7] -Input [1]: [max#90] +Input [1]: [max#113] Keys: [] -Functions [1]: [max(csales#88)] -Aggregate Attributes [1]: [max(csales#88)#92] -Results [1]: [max(csales#88)#92 AS tpcds_cmax#93] +Functions [1]: [max(csales#111)] +Aggregate Attributes [1]: [max(csales#111)#115] +Results [1]: [max(csales#111)#115 AS tpcds_cmax#116] -Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#79 +Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#98 IN dynamicpruning#99 ReusedExchange (164) (164) ReusedExchange [Reuses operator id: 147] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#100] -Subquery:5 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:5 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] -Subquery:6 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#57 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 110 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:7 Hosting operator id = 110 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] -Subquery:8 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:8 Hosting operator id = 127 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] 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 385cf017e58c3..f5c64908c8fb9 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 @@ -240,340 +240,340 @@ Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_ Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (30) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (32) Filter [codegen id : 8] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] Condition : isnotnull(ss_customer_sk#26) (33) Project [codegen id : 8] Output [3]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#29] (34) Scan parquet default.customer -Output [1]: [c_customer_sk#29] +Output [1]: [c_customer_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (35) ColumnarToRow [codegen id : 7] -Input [1]: [c_customer_sk#29] +Input [1]: [c_customer_sk#30] (36) Filter [codegen id : 7] -Input [1]: [c_customer_sk#29] -Condition : isnotnull(c_customer_sk#29) +Input [1]: [c_customer_sk#30] +Condition : isnotnull(c_customer_sk#30) (37) BroadcastExchange -Input [1]: [c_customer_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [1]: [c_customer_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (38) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#30] Join condition: None (39) Project [codegen id : 8] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#29] +Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, c_customer_sk#30] (40) HashAggregate [codegen id : 8] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] +Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#30] +Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#31, isEmpty#32] -Results [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [3]: [c_customer_sk#30, sum#34, isEmpty#35] (41) Exchange -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Arguments: hashpartitioning(c_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#36] (42) HashAggregate [codegen id : 9] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (43) Filter [codegen id : 9] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (44) Project [codegen id : 9] -Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (45) Sort [codegen id : 9] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (46) SortMergeJoin Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#30] Join condition: None (47) Scan parquet default.customer -Output [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Output [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 10] -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] (49) Filter [codegen id : 10] -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Condition : isnotnull(c_customer_sk#29) +Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Condition : isnotnull(c_customer_sk#43) (50) Exchange -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Arguments: hashpartitioning(c_customer_sk#43, 5), ENSURE_REQUIREMENTS, [id=#46] (51) Sort [codegen id : 11] -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Arguments: [c_customer_sk#43 ASC NULLS FIRST], false, 0 (52) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#29, sum#33, isEmpty#34] +Output [3]: [c_customer_sk#30, sum#34, isEmpty#35] (53) HashAggregate [codegen id : 14] -Input [3]: [c_customer_sk#29, sum#33, isEmpty#34] -Keys [1]: [c_customer_sk#29] +Input [3]: [c_customer_sk#30, sum#34, isEmpty#35] +Keys [1]: [c_customer_sk#30] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#36 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Results [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (54) Filter [codegen id : 14] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (55) Project [codegen id : 14] -Output [1]: [c_customer_sk#29 AS c_customer_sk#29#43] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#37] +Output [1]: [c_customer_sk#30] +Input [2]: [c_customer_sk#30, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#40] (56) Sort [codegen id : 14] -Input [1]: [c_customer_sk#29#43] -Arguments: [c_customer_sk#29#43 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#30] +Arguments: [c_customer_sk#30 ASC NULLS FIRST], false, 0 (57) SortMergeJoin -Left keys [1]: [c_customer_sk#29] -Right keys [1]: [c_customer_sk#29#43] +Left keys [1]: [c_customer_sk#43] +Right keys [1]: [c_customer_sk#30] Join condition: None (58) BroadcastExchange -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] +Input [3]: [c_customer_sk#43, c_first_name#44, c_last_name#45] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] (59) BroadcastHashJoin [codegen id : 16] Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#29] +Right keys [1]: [c_customer_sk#43] Join condition: None (60) Project [codegen id : 16] -Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#40, c_last_name#41] -Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#29, c_first_name#40, c_last_name#41] +Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#44, c_last_name#45] +Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#43, c_first_name#44, c_last_name#45] (61) Scan parquet default.date_dim -Output [3]: [d_date_sk#10, d_year#12, d_moy#45] +Output [3]: [d_date_sk#48, d_year#49, d_moy#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] ReadSchema: struct (62) ColumnarToRow [codegen id : 15] -Input [3]: [d_date_sk#10, d_year#12, d_moy#45] +Input [3]: [d_date_sk#48, d_year#49, d_moy#50] (63) Filter [codegen id : 15] -Input [3]: [d_date_sk#10, d_year#12, d_moy#45] -Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#45)) AND (d_year#12 = 2000)) AND (d_moy#45 = 2)) AND isnotnull(d_date_sk#10)) +Input [3]: [d_date_sk#48, d_year#49, d_moy#50] +Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2000)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#48)) (64) Project [codegen id : 15] -Output [1]: [d_date_sk#10] -Input [3]: [d_date_sk#10, d_year#12, d_moy#45] +Output [1]: [d_date_sk#48] +Input [3]: [d_date_sk#48, d_year#49, d_moy#50] (65) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#46] +Input [1]: [d_date_sk#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] (66) BroadcastHashJoin [codegen id : 16] Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#10] +Right keys [1]: [d_date_sk#48] Join condition: None (67) Project [codegen id : 16] -Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#40, c_last_name#41] -Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#40, c_last_name#41, d_date_sk#10] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#44, c_last_name#45] +Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#44, c_last_name#45, d_date_sk#48] (68) HashAggregate [codegen id : 16] -Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#40, c_last_name#41] -Keys [2]: [c_last_name#41, c_first_name#40] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#44, c_last_name#45] +Keys [2]: [c_last_name#45, c_first_name#44] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#47, isEmpty#48] -Results [4]: [c_last_name#41, c_first_name#40, sum#49, isEmpty#50] +Aggregate Attributes [2]: [sum#52, isEmpty#53] +Results [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] (69) Exchange -Input [4]: [c_last_name#41, c_first_name#40, sum#49, isEmpty#50] -Arguments: hashpartitioning(c_last_name#41, c_first_name#40, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] +Arguments: hashpartitioning(c_last_name#45, c_first_name#44, 5), ENSURE_REQUIREMENTS, [id=#56] (70) HashAggregate [codegen id : 17] -Input [4]: [c_last_name#41, c_first_name#40, sum#49, isEmpty#50] -Keys [2]: [c_last_name#41, c_first_name#40] +Input [4]: [c_last_name#45, c_first_name#44, sum#54, isEmpty#55] +Keys [2]: [c_last_name#45, c_first_name#44] Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#52] -Results [3]: [c_last_name#41, c_first_name#40, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#52 AS sales#53] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#57] +Results [3]: [c_last_name#45, c_first_name#44, sum(CheckOverflow((promote_precision(cast(cast(cs_quantity#3 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2), true))#57 AS sales#58] (71) Scan parquet default.web_sales -Output [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Output [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 22] -Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] (73) Filter [codegen id : 22] -Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] -Condition : isnotnull(ws_bill_customer_sk#55) +Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_bill_customer_sk#60) (74) ReusedExchange [Reuses operator id: unknown] Output [1]: [item_sk#22] (75) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_item_sk#54] +Left keys [1]: [ws_item_sk#59] Right keys [1]: [item_sk#22] Join condition: None (76) Project [codegen id : 22] -Output [4]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] -Input [5]: [ws_item_sk#54, ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] +Output [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Input [5]: [ws_item_sk#59, ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] (77) Exchange -Input [4]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] -Arguments: hashpartitioning(ws_bill_customer_sk#55, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Arguments: hashpartitioning(ws_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#64] (78) Sort [codegen id : 23] -Input [4]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58] -Arguments: [ws_bill_customer_sk#55 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63] +Arguments: [ws_bill_customer_sk#60 ASC NULLS FIRST], false, 0 (79) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#29, sum#60, isEmpty#61] +Output [3]: [c_customer_sk#65, sum#66, isEmpty#67] (80) HashAggregate [codegen id : 26] -Input [3]: [c_customer_sk#29, sum#60, isEmpty#61] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] +Input [3]: [c_customer_sk#65, sum#66, isEmpty#67] +Keys [1]: [c_customer_sk#65] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70] +Results [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] (81) Filter [codegen id : 26] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (82) Project [codegen id : 26] -Output [1]: [c_customer_sk#29] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] +Output [1]: [c_customer_sk#65] +Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] (83) Sort [codegen id : 26] -Input [1]: [c_customer_sk#29] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#65] +Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 (84) SortMergeJoin -Left keys [1]: [ws_bill_customer_sk#55] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ws_bill_customer_sk#60] +Right keys [1]: [c_customer_sk#65] Join condition: None (85) ReusedExchange [Reuses operator id: 50] -Output [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] +Output [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] (86) Sort [codegen id : 28] -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] +Arguments: [c_customer_sk#72 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: 41] -Output [3]: [c_customer_sk#29, sum#60, isEmpty#61] +Output [3]: [c_customer_sk#65, sum#66, isEmpty#67] (88) HashAggregate [codegen id : 31] -Input [3]: [c_customer_sk#29, sum#60, isEmpty#61] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62] -Results [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#62 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] +Input [3]: [c_customer_sk#65, sum#66, isEmpty#67] +Keys [1]: [c_customer_sk#65] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70] +Results [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#68 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#69 as decimal(12,2)))), DecimalType(18,2), true))#70 AS sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] (89) Filter [codegen id : 31] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] -Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#38, [id=#39] as decimal(32,6)))), DecimalType(38,8), true))) +Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] +Condition : (isnotnull(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71) AND (cast(sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#41, [id=#42] as decimal(32,6)))), DecimalType(38,8), true))) (90) Project [codegen id : 31] -Output [1]: [c_customer_sk#29 AS c_customer_sk#29#64] -Input [2]: [c_customer_sk#29, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#63] +Output [1]: [c_customer_sk#65] +Input [2]: [c_customer_sk#65, sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#38 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#39 as decimal(12,2)))), DecimalType(18,2), true))#71] (91) Sort [codegen id : 31] -Input [1]: [c_customer_sk#29#64] -Arguments: [c_customer_sk#29#64 ASC NULLS FIRST], false, 0 +Input [1]: [c_customer_sk#65] +Arguments: [c_customer_sk#65 ASC NULLS FIRST], false, 0 (92) SortMergeJoin -Left keys [1]: [c_customer_sk#29] -Right keys [1]: [c_customer_sk#29#64] +Left keys [1]: [c_customer_sk#72] +Right keys [1]: [c_customer_sk#65] Join condition: None (93) BroadcastExchange -Input [3]: [c_customer_sk#29, c_first_name#40, c_last_name#41] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#65] +Input [3]: [c_customer_sk#72, c_first_name#73, c_last_name#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] (94) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ws_bill_customer_sk#55] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ws_bill_customer_sk#60] +Right keys [1]: [c_customer_sk#72] Join condition: None (95) Project [codegen id : 33] -Output [5]: [ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58, c_first_name#40, c_last_name#41] -Input [7]: [ws_bill_customer_sk#55, ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58, c_customer_sk#29, c_first_name#40, c_last_name#41] +Output [5]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74] +Input [7]: [ws_bill_customer_sk#60, ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_customer_sk#72, c_first_name#73, c_last_name#74] (96) ReusedExchange [Reuses operator id: 65] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#76] (97) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ws_sold_date_sk#58] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#63] +Right keys [1]: [d_date_sk#76] Join condition: None (98) Project [codegen id : 33] -Output [4]: [ws_quantity#56, ws_list_price#57, c_first_name#40, c_last_name#41] -Input [6]: [ws_quantity#56, ws_list_price#57, ws_sold_date_sk#58, c_first_name#40, c_last_name#41, d_date_sk#10] +Output [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74] +Input [6]: [ws_quantity#61, ws_list_price#62, ws_sold_date_sk#63, c_first_name#73, c_last_name#74, d_date_sk#76] (99) HashAggregate [codegen id : 33] -Input [4]: [ws_quantity#56, ws_list_price#57, c_first_name#40, c_last_name#41] -Keys [2]: [c_last_name#41, c_first_name#40] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#66, isEmpty#67] -Results [4]: [c_last_name#41, c_first_name#40, sum#68, isEmpty#69] +Input [4]: [ws_quantity#61, ws_list_price#62, c_first_name#73, c_last_name#74] +Keys [2]: [c_last_name#74, c_first_name#73] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#77, isEmpty#78] +Results [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] (100) Exchange -Input [4]: [c_last_name#41, c_first_name#40, sum#68, isEmpty#69] -Arguments: hashpartitioning(c_last_name#41, c_first_name#40, 5), ENSURE_REQUIREMENTS, [id=#70] +Input [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] +Arguments: hashpartitioning(c_last_name#74, c_first_name#73, 5), ENSURE_REQUIREMENTS, [id=#81] (101) HashAggregate [codegen id : 34] -Input [4]: [c_last_name#41, c_first_name#40, sum#68, isEmpty#69] -Keys [2]: [c_last_name#41, c_first_name#40] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))#71] -Results [3]: [c_last_name#41, c_first_name#40, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#56 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#57 as decimal(12,2)))), DecimalType(18,2), true))#71 AS sales#72] +Input [4]: [c_last_name#74, c_first_name#73, sum#79, isEmpty#80] +Keys [2]: [c_last_name#74, c_first_name#73] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#82] +Results [3]: [c_last_name#74, c_first_name#73, sum(CheckOverflow((promote_precision(cast(cast(ws_quantity#61 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ws_list_price#62 as decimal(12,2)))), DecimalType(18,2), true))#82 AS sales#83] (102) Union (103) TakeOrderedAndProject -Input [3]: [c_last_name#41, c_first_name#40, sales#53] -Arguments: 100, [c_last_name#41 ASC NULLS FIRST, c_first_name#40 ASC NULLS FIRST, sales#53 ASC NULLS FIRST], [c_last_name#41, c_first_name#40, sales#53] +Input [3]: [c_last_name#45, c_first_name#44, sales#58] +Arguments: 100, [c_last_name#45 ASC NULLS FIRST, c_first_name#44 ASC NULLS FIRST, sales#58 ASC NULLS FIRST], [c_last_name#45, c_first_name#44, sales#58] ===== Subqueries ===== @@ -582,7 +582,7 @@ ReusedExchange (104) (104) ReusedExchange [Reuses operator id: 65] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#48] Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 ReusedExchange (105) @@ -591,7 +591,7 @@ ReusedExchange (105) (105) ReusedExchange [Reuses operator id: 11] Output [2]: [d_date_sk#10, d_date#11] -Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (127) +- Exchange (126) +- * HashAggregate (125) @@ -617,127 +617,127 @@ Subquery:3 Hosting operator id = 43 Hosting Expression = Subquery scalar-subquer (106) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Output [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(ss_sold_date_sk#87), dynamicpruningexpression(ss_sold_date_sk#87 IN dynamicpruning#88)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (107) ColumnarToRow [codegen id : 3] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] +Input [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] (108) Filter [codegen id : 3] -Input [4]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8] -Condition : isnotnull(ss_customer_sk#26) +Input [4]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87] +Condition : isnotnull(ss_customer_sk#84) (109) Scan parquet default.customer -Output [1]: [c_customer_sk#29] +Output [1]: [c_customer_sk#89] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct (110) ColumnarToRow [codegen id : 1] -Input [1]: [c_customer_sk#29] +Input [1]: [c_customer_sk#89] (111) Filter [codegen id : 1] -Input [1]: [c_customer_sk#29] -Condition : isnotnull(c_customer_sk#29) +Input [1]: [c_customer_sk#89] +Condition : isnotnull(c_customer_sk#89) (112) BroadcastExchange -Input [1]: [c_customer_sk#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [1]: [c_customer_sk#89] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#90] (113) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#26] -Right keys [1]: [c_customer_sk#29] +Left keys [1]: [ss_customer_sk#84] +Right keys [1]: [c_customer_sk#89] Join condition: None (114) Project [codegen id : 3] -Output [4]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] -Input [5]: [ss_customer_sk#26, ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29] +Output [4]: [ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89] +Input [5]: [ss_customer_sk#84, ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89] (115) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#12] +Output [2]: [d_date_sk#91, d_year#92] 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 (116) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#10, d_year#12] +Input [2]: [d_date_sk#91, d_year#92] (117) Filter [codegen id : 2] -Input [2]: [d_date_sk#10, d_year#12] -Condition : (d_year#12 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) +Input [2]: [d_date_sk#91, d_year#92] +Condition : (d_year#92 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#91)) (118) Project [codegen id : 2] -Output [1]: [d_date_sk#10] -Input [2]: [d_date_sk#10, d_year#12] +Output [1]: [d_date_sk#91] +Input [2]: [d_date_sk#91, d_year#92] (119) BroadcastExchange -Input [1]: [d_date_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#75] +Input [1]: [d_date_sk#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] (120) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#87] +Right keys [1]: [d_date_sk#91] Join condition: None (121) Project [codegen id : 3] -Output [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Input [5]: [ss_quantity#27, ss_sales_price#28, ss_sold_date_sk#8, c_customer_sk#29, d_date_sk#10] +Output [3]: [ss_quantity#85, ss_sales_price#86, c_customer_sk#89] +Input [5]: [ss_quantity#85, ss_sales_price#86, ss_sold_date_sk#87, c_customer_sk#89, d_date_sk#91] (122) HashAggregate [codegen id : 3] -Input [3]: [ss_quantity#27, ss_sales_price#28, c_customer_sk#29] -Keys [1]: [c_customer_sk#29] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [2]: [sum#76, isEmpty#77] -Results [3]: [c_customer_sk#29, sum#78, isEmpty#79] +Input [3]: [ss_quantity#85, ss_sales_price#86, c_customer_sk#89] +Keys [1]: [c_customer_sk#89] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [2]: [sum#94, isEmpty#95] +Results [3]: [c_customer_sk#89, sum#96, isEmpty#97] (123) Exchange -Input [3]: [c_customer_sk#29, sum#78, isEmpty#79] -Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [3]: [c_customer_sk#89, sum#96, isEmpty#97] +Arguments: hashpartitioning(c_customer_sk#89, 5), ENSURE_REQUIREMENTS, [id=#98] (124) HashAggregate [codegen id : 4] -Input [3]: [c_customer_sk#29, sum#78, isEmpty#79] -Keys [1]: [c_customer_sk#29] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))] -Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#81] -Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#27 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#28 as decimal(12,2)))), DecimalType(18,2), true))#81 AS csales#82] +Input [3]: [c_customer_sk#89, sum#96, isEmpty#97] +Keys [1]: [c_customer_sk#89] +Functions [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))] +Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))#99] +Results [1]: [sum(CheckOverflow((promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2))) * promote_precision(cast(ss_sales_price#86 as decimal(12,2)))), DecimalType(18,2), true))#99 AS csales#100] (125) HashAggregate [codegen id : 4] -Input [1]: [csales#82] +Input [1]: [csales#100] Keys: [] -Functions [1]: [partial_max(csales#82)] -Aggregate Attributes [1]: [max#83] -Results [1]: [max#84] +Functions [1]: [partial_max(csales#100)] +Aggregate Attributes [1]: [max#101] +Results [1]: [max#102] (126) Exchange -Input [1]: [max#84] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#85] +Input [1]: [max#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103] (127) HashAggregate [codegen id : 5] -Input [1]: [max#84] +Input [1]: [max#102] Keys: [] -Functions [1]: [max(csales#82)] -Aggregate Attributes [1]: [max(csales#82)#86] -Results [1]: [max(csales#82)#86 AS tpcds_cmax#87] +Functions [1]: [max(csales#100)] +Aggregate Attributes [1]: [max(csales#100)#104] +Results [1]: [max(csales#100)#104 AS tpcds_cmax#105] -Subquery:4 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#73 +Subquery:4 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#87 IN dynamicpruning#88 ReusedExchange (128) (128) ReusedExchange [Reuses operator id: 119] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#91] -Subquery:5 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:5 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] -Subquery:6 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#6 +Subquery:6 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#6 -Subquery:7 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:7 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] -Subquery:8 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#38, [id=#39] +Subquery:8 Hosting operator id = 89 Hosting Expression = ReusedSubquery Subquery scalar-subquery#41, [id=#42] 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 431cf3ef92deb..7895ee1838ad7 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 @@ -340,248 +340,248 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (51) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] (53) Filter [codegen id : 2] -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)) +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) (54) Project [codegen id : 2] -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] +Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] (55) Scan parquet default.store -Output [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (56) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] (57) Filter [codegen id : 1] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -Condition : (((isnotnull(s_market_id#22) AND (s_market_id#22 = 8)) AND isnotnull(s_store_sk#20)) AND isnotnull(s_zip#24)) +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) (58) Project [codegen id : 1] -Output [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] (59) BroadcastExchange -Input [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] +Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] (60) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#20] +Left keys [1]: [ss_store_sk#52] +Right keys [1]: [s_store_sk#56] Join condition: None (61) Project [codegen id : 2] -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -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#20, s_store_name#21, s_state#23, s_zip#24] +Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] (62) Exchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#62] (63) Sort [codegen id : 3] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 (64) Scan parquet default.customer -Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Output [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (65) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] (66) Filter [codegen id : 4] -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_birth_country#18)) +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_birth_country#66)) (67) Exchange -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, [id=#67] (68) Sort [codegen id : 5] -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 +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: [c_customer_sk#63 ASC NULLS FIRST], false, 0 (69) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#15] +Left keys [1]: [ss_customer_sk#51] +Right keys [1]: [c_customer_sk#63] Join condition: None (70) Project [codegen id : 6] -Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Output [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Input [11]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] (71) Exchange -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#68] (72) Sort [codegen id : 7] -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 (73) Scan parquet default.item -Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (75) Filter [codegen id : 8] -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) +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Condition : isnotnull(i_item_sk#69) (76) Exchange -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=#54] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: hashpartitioning(i_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] (77) Sort [codegen id : 9] -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 +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: [i_item_sk#69 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ss_item_sk#50] +Right keys [1]: [i_item_sk#69] Join condition: None (79) Project [codegen id : 10] -Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (80) Exchange -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Arguments: hashpartitioning(c_birth_country#18, s_zip#24, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: hashpartitioning(c_birth_country#66, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] (81) Sort [codegen id : 11] -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Arguments: [c_birth_country#18 ASC NULLS FIRST, s_zip#24 ASC NULLS FIRST], false, 0 +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: [c_birth_country#66 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 (82) Scan parquet default.customer_address -Output [3]: [ca_state#26, ca_zip#27, ca_country#28] +Output [3]: [ca_state#77, ca_zip#78, ca_country#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (83) ColumnarToRow [codegen id : 12] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] (84) Filter [codegen id : 12] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Condition : (isnotnull(ca_country#28) AND isnotnull(ca_zip#27)) +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) (85) Exchange -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Arguments: hashpartitioning(upper(ca_country#28), ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: hashpartitioning(upper(ca_country#79), ca_zip#78, 5), ENSURE_REQUIREMENTS, [id=#80] (86) Sort [codegen id : 13] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Arguments: [upper(ca_country#28) ASC NULLS FIRST, ca_zip#27 ASC NULLS FIRST], false, 0 +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: [upper(ca_country#79) ASC NULLS FIRST, ca_zip#78 ASC NULLS FIRST], false, 0 (87) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#18, s_zip#24] -Right keys [2]: [upper(ca_country#28), ca_zip#27] +Left keys [2]: [c_birth_country#66, s_zip#60] +Right keys [2]: [upper(ca_country#79), ca_zip#78] Join condition: None (88) Project [codegen id : 14] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, ca_zip#27, ca_country#28] +Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, ca_zip#78, ca_country#79] (89) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#57] +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Arguments: hashpartitioning(cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint), 5), ENSURE_REQUIREMENTS, [id=#81] (90) Sort [codegen id : 15] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Arguments: [cast(ss_ticket_number#53 as bigint) ASC NULLS FIRST, cast(ss_item_sk#50 as bigint) ASC NULLS FIRST], false, 0 (91) Scan parquet default.store_returns -Output [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Output [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 16] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] (93) Filter [codegen id : 16] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] -Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Condition : (isnotnull(sr_ticket_number#83) AND isnotnull(sr_item_sk#82)) (94) Project [codegen id : 16] -Output [2]: [sr_item_sk#31, sr_ticket_number#32] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Output [2]: [sr_item_sk#82, sr_ticket_number#83] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] (95) Exchange -Input [2]: [sr_item_sk#31, sr_ticket_number#32] -Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: hashpartitioning(sr_ticket_number#83, sr_item_sk#82, 5), ENSURE_REQUIREMENTS, [id=#85] (96) Sort [codegen id : 17] -Input [2]: [sr_item_sk#31, sr_ticket_number#32] -Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 (97) SortMergeJoin [codegen id : 18] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] +Left keys [2]: [cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint)] +Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] Join condition: None (98) Project [codegen id : 18] -Output [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, sr_item_sk#31, sr_ticket_number#32] +Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, sr_item_sk#82, sr_ticket_number#83] (99) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] -Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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#59] -Results [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] +Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] +Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum#86] +Results [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] (100) Exchange -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Arguments: hashpartitioning(c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#88] (101) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] -Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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))#62] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#62,17,2) AS netpaid#39] +Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] (102) HashAggregate [codegen id : 19] Input [1]: [netpaid#39] Keys: [] Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#63, count#64] -Results [2]: [sum#65, count#66] +Aggregate Attributes [2]: [sum#90, count#91] +Results [2]: [sum#92, count#93] (103) Exchange -Input [2]: [sum#65, count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] +Input [2]: [sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] (104) HashAggregate [codegen id : 20] -Input [2]: [sum#65, count#66] +Input [2]: [sum#92, count#93] Keys: [] Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#68] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#68)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#69] +Aggregate Attributes [1]: [avg(netpaid#39)#95] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] 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 f73a5a5e052e3..0565ae76a440d 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 @@ -316,212 +316,212 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (48) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 1] -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] +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] (50) Filter [codegen id : 1] -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)) +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)) (51) Project [codegen id : 1] -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] +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] (52) Exchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#49] +Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Arguments: hashpartitioning(cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] (53) Sort [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Arguments: [cast(ss_ticket_number#52 as bigint) ASC NULLS FIRST, cast(ss_item_sk#49 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.store_returns -Output [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] (56) Filter [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] -Condition : (isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#8)) +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Condition : (isnotnull(sr_ticket_number#57) AND isnotnull(sr_item_sk#56)) (57) Project [codegen id : 3] -Output [2]: [sr_item_sk#8, sr_ticket_number#9] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [2]: [sr_item_sk#56, sr_ticket_number#57] +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] (58) Exchange -Input [2]: [sr_item_sk#8, sr_ticket_number#9] -Arguments: hashpartitioning(sr_ticket_number#9, sr_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [sr_item_sk#56, sr_ticket_number#57] +Arguments: hashpartitioning(sr_ticket_number#57, sr_item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#59] (59) Sort [codegen id : 4] -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 +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 (60) SortMergeJoin [codegen id : 9] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] +Left keys [2]: [cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint)] +Right keys [2]: [sr_ticket_number#57, sr_item_sk#56] Join condition: None (61) Project [codegen id : 9] -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] +Output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] +Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#56, sr_ticket_number#57] (62) Scan parquet default.store -Output [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (63) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] (64) Filter [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] -Condition : (((isnotnull(s_market_id#14) AND (s_market_id#14 = 8)) AND isnotnull(s_store_sk#12)) AND isnotnull(s_zip#16)) +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Condition : (((isnotnull(s_market_id#62) AND (s_market_id#62 = 8)) AND isnotnull(s_store_sk#60)) AND isnotnull(s_zip#64)) (65) Project [codegen id : 5] -Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] (66) BroadcastExchange -Input [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] +Input [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] (67) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] +Left keys [1]: [ss_store_sk#51] +Right keys [1]: [s_store_sk#60] Join condition: None (68) Project [codegen id : 9] -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] +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64] +Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] (69) Scan parquet default.item -Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Output [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] (71) Filter [codegen id : 6] -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) +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Condition : isnotnull(i_item_sk#66) (72) BroadcastExchange -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=#52] +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#72] (73) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [ss_item_sk#49] +Right keys [1]: [i_item_sk#66] Join condition: None (74) Project [codegen id : 9] -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] +Output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] (75) Scan parquet default.customer -Output [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Output [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (76) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] (77) Filter [codegen id : 7] -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_birth_country#28)) +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_birth_country#76)) (78) BroadcastExchange -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] (79) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#25] +Left keys [1]: [ss_customer_sk#50] +Right keys [1]: [c_customer_sk#73] Join condition: None (80) Project [codegen id : 9] -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] +Output [12]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76] +Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] (81) Scan parquet default.customer_address -Output [3]: [ca_state#30, ca_zip#31, ca_country#32] +Output [3]: [ca_state#78, ca_zip#79, ca_country#80] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (82) ColumnarToRow [codegen id : 8] -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] (83) Filter [codegen id : 8] -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] -Condition : (isnotnull(ca_country#32) AND isnotnull(ca_zip#31)) +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) (84) BroadcastExchange -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#54] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#81] (85) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#28, s_zip#16] -Right keys [2]: [upper(ca_country#32), ca_zip#31] +Left keys [2]: [c_birth_country#76, s_zip#64] +Right keys [2]: [upper(ca_country#80), ca_zip#79] Join condition: None (86) Project [codegen id : 9] -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] +Output [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] +Input [15]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76, ca_state#78, ca_zip#79, ca_country#80] (87) HashAggregate [codegen id : 9] -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#55] -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#56] +Input [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] +Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum#82] +Results [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] (88) Exchange -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#56] -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=#57] +Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +Arguments: hashpartitioning(c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, [id=#84] (89) HashAggregate [codegen id : 10] -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#56] -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))#58] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#58,17,2) AS netpaid#38] +Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] +Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#85] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#38] (90) HashAggregate [codegen id : 10] Input [1]: [netpaid#38] Keys: [] Functions [1]: [partial_avg(netpaid#38)] -Aggregate Attributes [2]: [sum#59, count#60] -Results [2]: [sum#61, count#62] +Aggregate Attributes [2]: [sum#86, count#87] +Results [2]: [sum#88, count#89] (91) Exchange -Input [2]: [sum#61, count#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] +Input [2]: [sum#88, count#89] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#90] (92) HashAggregate [codegen id : 11] -Input [2]: [sum#61, count#62] +Input [2]: [sum#88, count#89] Keys: [] Functions [1]: [avg(netpaid#38)] -Aggregate Attributes [1]: [avg(netpaid#38)#64] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#64)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#65] +Aggregate Attributes [1]: [avg(netpaid#38)#91] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] 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 163a4e7e3ddad..e45459d529104 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 @@ -340,248 +340,248 @@ Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquer (51) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 2] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] (53) Filter [codegen id : 2] -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)) +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) (54) Project [codegen id : 2] -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] +Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] (55) Scan parquet default.store -Output [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (56) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] (57) Filter [codegen id : 1] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -Condition : (((isnotnull(s_market_id#22) AND (s_market_id#22 = 8)) AND isnotnull(s_store_sk#20)) AND isnotnull(s_zip#24)) +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) (58) Project [codegen id : 1] -Output [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] -Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] (59) BroadcastExchange -Input [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#50] +Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] (60) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#20] +Left keys [1]: [ss_store_sk#52] +Right keys [1]: [s_store_sk#56] Join condition: None (61) Project [codegen id : 2] -Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -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#20, s_store_name#21, s_state#23, s_zip#24] +Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] (62) Exchange -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#62] (63) Sort [codegen id : 3] -Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 (64) Scan parquet default.customer -Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Output [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (65) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] (66) Filter [codegen id : 4] -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_birth_country#18)) +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_birth_country#66)) (67) Exchange -Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: hashpartitioning(c_customer_sk#63, 5), ENSURE_REQUIREMENTS, [id=#67] (68) Sort [codegen id : 5] -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 +Input [4]: [c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: [c_customer_sk#63 ASC NULLS FIRST], false, 0 (69) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#15] +Left keys [1]: [ss_customer_sk#51] +Right keys [1]: [c_customer_sk#63] Join condition: None (70) Project [codegen id : 6] -Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Output [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Input [11]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_customer_sk#63, c_first_name#64, c_last_name#65, c_birth_country#66] (71) Exchange -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#68] (72) Sort [codegen id : 7] -Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66] +Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 (73) Scan parquet default.item -Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (75) Filter [codegen id : 8] -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) +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Condition : isnotnull(i_item_sk#69) (76) Exchange -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=#54] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: hashpartitioning(i_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] (77) Sort [codegen id : 9] -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 +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: [i_item_sk#69 ASC NULLS FIRST], false, 0 (78) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ss_item_sk#50] +Right keys [1]: [i_item_sk#69] Join condition: None (79) Project [codegen id : 10] -Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (80) Exchange -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Arguments: hashpartitioning(c_birth_country#18, s_zip#24, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: hashpartitioning(c_birth_country#66, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] (81) Sort [codegen id : 11] -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -Arguments: [c_birth_country#18 ASC NULLS FIRST, s_zip#24 ASC NULLS FIRST], false, 0 +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: [c_birth_country#66 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 (82) Scan parquet default.customer_address -Output [3]: [ca_state#26, ca_zip#27, ca_country#28] +Output [3]: [ca_state#77, ca_zip#78, ca_country#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (83) ColumnarToRow [codegen id : 12] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] (84) Filter [codegen id : 12] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Condition : (isnotnull(ca_country#28) AND isnotnull(ca_zip#27)) +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) (85) Exchange -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Arguments: hashpartitioning(upper(ca_country#28), ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: hashpartitioning(upper(ca_country#79), ca_zip#78, 5), ENSURE_REQUIREMENTS, [id=#80] (86) Sort [codegen id : 13] -Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -Arguments: [upper(ca_country#28) ASC NULLS FIRST, ca_zip#27 ASC NULLS FIRST], false, 0 +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: [upper(ca_country#79) ASC NULLS FIRST, ca_zip#78 ASC NULLS FIRST], false, 0 (87) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#18, s_zip#24] -Right keys [2]: [upper(ca_country#28), ca_zip#27] +Left keys [2]: [c_birth_country#66, s_zip#60] +Right keys [2]: [upper(ca_country#79), ca_zip#78] Join condition: None (88) Project [codegen id : 14] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, s_zip#24, c_first_name#16, c_last_name#17, c_birth_country#18, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, ca_zip#27, ca_country#28] +Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, c_first_name#64, c_last_name#65, c_birth_country#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, ca_zip#78, ca_country#79] (89) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#57] +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Arguments: hashpartitioning(cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint), 5), ENSURE_REQUIREMENTS, [id=#81] (90) Sort [codegen id : 15] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77] +Arguments: [cast(ss_ticket_number#53 as bigint) ASC NULLS FIRST, cast(ss_item_sk#50 as bigint) ASC NULLS FIRST], false, 0 (91) Scan parquet default.store_returns -Output [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Output [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 16] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] (93) Filter [codegen id : 16] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] -Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Condition : (isnotnull(sr_ticket_number#83) AND isnotnull(sr_item_sk#82)) (94) Project [codegen id : 16] -Output [2]: [sr_item_sk#31, sr_ticket_number#32] -Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Output [2]: [sr_item_sk#82, sr_ticket_number#83] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] (95) Exchange -Input [2]: [sr_item_sk#31, sr_ticket_number#32] -Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: hashpartitioning(sr_ticket_number#83, sr_item_sk#82, 5), ENSURE_REQUIREMENTS, [id=#85] (96) Sort [codegen id : 17] -Input [2]: [sr_item_sk#31, sr_ticket_number#32] -Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 (97) SortMergeJoin [codegen id : 18] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] +Left keys [2]: [cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint)] +Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] Join condition: None (98) Project [codegen id : 18] -Output [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#21, s_state#23, c_first_name#16, c_last_name#17, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, ca_state#26, sr_item_sk#31, sr_ticket_number#32] +Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, c_first_name#64, c_last_name#65, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, ca_state#77, sr_item_sk#82, sr_ticket_number#83] (99) HashAggregate [codegen id : 18] -Input [11]: [ss_net_paid#5, s_store_name#21, s_state#23, 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#26] -Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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#59] -Results [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] +Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#64, c_last_name#65, ca_state#77] +Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum#86] +Results [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] (100) Exchange -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] -Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Arguments: hashpartitioning(c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#88] (101) HashAggregate [codegen id : 19] -Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#60] -Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, 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))#62] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#62,17,2) AS netpaid#39] +Input [11]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#87] +Keys [10]: [c_last_name#65, c_first_name#64, s_store_name#57, ca_state#77, s_state#59, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] (102) HashAggregate [codegen id : 19] Input [1]: [netpaid#39] Keys: [] Functions [1]: [partial_avg(netpaid#39)] -Aggregate Attributes [2]: [sum#63, count#64] -Results [2]: [sum#65, count#66] +Aggregate Attributes [2]: [sum#90, count#91] +Results [2]: [sum#92, count#93] (103) Exchange -Input [2]: [sum#65, count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] +Input [2]: [sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] (104) HashAggregate [codegen id : 20] -Input [2]: [sum#65, count#66] +Input [2]: [sum#92, count#93] Keys: [] Functions [1]: [avg(netpaid#39)] -Aggregate Attributes [1]: [avg(netpaid#39)#68] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#68)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#69] +Aggregate Attributes [1]: [avg(netpaid#39)#95] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] 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 09830b1be656e..aeaf3f56b9e13 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 @@ -316,212 +316,212 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (48) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 1] -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] +Input [6]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, ss_sold_date_sk#54] (50) Filter [codegen id : 1] -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)) +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)) (51) Project [codegen id : 1] -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] +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] (52) Exchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#49] +Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Arguments: hashpartitioning(cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] (53) Sort [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +Input [5]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53] +Arguments: [cast(ss_ticket_number#52 as bigint) ASC NULLS FIRST, cast(ss_item_sk#49 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.store_returns -Output [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] (56) Filter [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] -Condition : (isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#8)) +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] +Condition : (isnotnull(sr_ticket_number#57) AND isnotnull(sr_item_sk#56)) (57) Project [codegen id : 3] -Output [2]: [sr_item_sk#8, sr_ticket_number#9] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [2]: [sr_item_sk#56, sr_ticket_number#57] +Input [3]: [sr_item_sk#56, sr_ticket_number#57, sr_returned_date_sk#58] (58) Exchange -Input [2]: [sr_item_sk#8, sr_ticket_number#9] -Arguments: hashpartitioning(sr_ticket_number#9, sr_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [sr_item_sk#56, sr_ticket_number#57] +Arguments: hashpartitioning(sr_ticket_number#57, sr_item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#59] (59) Sort [codegen id : 4] -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 +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 (60) SortMergeJoin [codegen id : 9] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] +Left keys [2]: [cast(ss_ticket_number#52 as bigint), cast(ss_item_sk#49 as bigint)] +Right keys [2]: [sr_ticket_number#57, sr_item_sk#56] Join condition: None (61) Project [codegen id : 9] -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] +Output [4]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53] +Input [7]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_ticket_number#52, ss_net_paid#53, sr_item_sk#56, sr_ticket_number#57] (62) Scan parquet default.store -Output [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (63) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] (64) Filter [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] -Condition : (((isnotnull(s_market_id#14) AND (s_market_id#14 = 8)) AND isnotnull(s_store_sk#12)) AND isnotnull(s_zip#16)) +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] +Condition : (((isnotnull(s_market_id#62) AND (s_market_id#62 = 8)) AND isnotnull(s_store_sk#60)) AND isnotnull(s_zip#64)) (65) Project [codegen id : 5] -Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +Input [5]: [s_store_sk#60, s_store_name#61, s_market_id#62, s_state#63, s_zip#64] (66) BroadcastExchange -Input [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#51] +Input [4]: [s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] (67) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] +Left keys [1]: [ss_store_sk#51] +Right keys [1]: [s_store_sk#60] Join condition: None (68) Project [codegen id : 9] -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] +Output [6]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64] +Input [8]: [ss_item_sk#49, ss_customer_sk#50, ss_store_sk#51, ss_net_paid#53, s_store_sk#60, s_store_name#61, s_state#63, s_zip#64] (69) Scan parquet default.item -Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Output [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] (71) Filter [codegen id : 6] -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) +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Condition : isnotnull(i_item_sk#66) (72) BroadcastExchange -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=#52] +Input [6]: [i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#72] (73) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [ss_item_sk#49] +Right keys [1]: [i_item_sk#66] Join condition: None (74) Project [codegen id : 9] -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] +Output [10]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] +Input [12]: [ss_item_sk#49, ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_item_sk#66, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71] (75) Scan parquet default.customer -Output [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Output [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct (76) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] (77) Filter [codegen id : 7] -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] -Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_birth_country#28)) +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Condition : (isnotnull(c_customer_sk#73) AND isnotnull(c_birth_country#76)) (78) BroadcastExchange -Input [4]: [c_customer_sk#25, c_first_name#26, c_last_name#27, c_birth_country#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] +Input [4]: [c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#77] (79) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#25] +Left keys [1]: [ss_customer_sk#50] +Right keys [1]: [c_customer_sk#73] Join condition: None (80) Project [codegen id : 9] -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] +Output [12]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76] +Input [14]: [ss_customer_sk#50, ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_customer_sk#73, c_first_name#74, c_last_name#75, c_birth_country#76] (81) Scan parquet default.customer_address -Output [3]: [ca_state#30, ca_zip#31, ca_country#32] +Output [3]: [ca_state#78, ca_zip#79, ca_country#80] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (82) ColumnarToRow [codegen id : 8] -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] (83) Filter [codegen id : 8] -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] -Condition : (isnotnull(ca_country#32) AND isnotnull(ca_zip#31)) +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Condition : (isnotnull(ca_country#80) AND isnotnull(ca_zip#79)) (84) BroadcastExchange -Input [3]: [ca_state#30, ca_zip#31, ca_country#32] -Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#54] +Input [3]: [ca_state#78, ca_zip#79, ca_country#80] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [id=#81] (85) BroadcastHashJoin [codegen id : 9] -Left keys [2]: [c_birth_country#28, s_zip#16] -Right keys [2]: [upper(ca_country#32), ca_zip#31] +Left keys [2]: [c_birth_country#76, s_zip#64] +Right keys [2]: [upper(ca_country#80), ca_zip#79] Join condition: None (86) Project [codegen id : 9] -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] +Output [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] +Input [15]: [ss_net_paid#53, s_store_name#61, s_state#63, s_zip#64, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, c_birth_country#76, ca_state#78, ca_zip#79, ca_country#80] (87) HashAggregate [codegen id : 9] -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#55] -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#56] +Input [11]: [ss_net_paid#53, s_store_name#61, s_state#63, i_current_price#67, i_size#68, i_color#69, i_units#70, i_manager_id#71, c_first_name#74, c_last_name#75, ca_state#78] +Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum#82] +Results [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] (88) Exchange -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#56] -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=#57] +Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +Arguments: hashpartitioning(c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, 5), ENSURE_REQUIREMENTS, [id=#84] (89) HashAggregate [codegen id : 10] -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#56] -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))#58] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#58,17,2) AS netpaid#38] +Input [11]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68, sum#83] +Keys [10]: [c_last_name#75, c_first_name#74, s_store_name#61, ca_state#78, s_state#63, i_color#69, i_current_price#67, i_manager_id#71, i_units#70, i_size#68] +Functions [1]: [sum(UnscaledValue(ss_net_paid#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#53))#85] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#53))#85,17,2) AS netpaid#38] (90) HashAggregate [codegen id : 10] Input [1]: [netpaid#38] Keys: [] Functions [1]: [partial_avg(netpaid#38)] -Aggregate Attributes [2]: [sum#59, count#60] -Results [2]: [sum#61, count#62] +Aggregate Attributes [2]: [sum#86, count#87] +Results [2]: [sum#88, count#89] (91) Exchange -Input [2]: [sum#61, count#62] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] +Input [2]: [sum#88, count#89] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#90] (92) HashAggregate [codegen id : 11] -Input [2]: [sum#61, count#62] +Input [2]: [sum#88, count#89] Keys: [] Functions [1]: [avg(netpaid#38)] -Aggregate Attributes [1]: [avg(netpaid#38)#64] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#64)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#65] +Aggregate Attributes [1]: [avg(netpaid#38)#91] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#38)#91)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#92] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt index 9f15199d8c8b7..eec45ea549531 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.sf100/explain.txt @@ -126,306 +126,306 @@ Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#6 / 100.0) as decimal(11,6)) AS B1_LP#15, count(ss_list_price#3)#7 AS B1_CNT#16, count(ss_list_price#3)#12 AS B1_CNTD#17] (11) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] (13) Filter [codegen id : 3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 6)) AND (ss_quantity#1 <= 10)) AND ((((ss_list_price#3 >= 90.00) AND (ss_list_price#3 <= 100.00)) OR ((ss_coupon_amt#4 >= 2323.00) AND (ss_coupon_amt#4 <= 3323.00))) OR ((ss_wholesale_cost#2 >= 31.00) AND (ss_wholesale_cost#2 <= 51.00)))) +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Condition : (((isnotnull(ss_quantity#18) AND (ss_quantity#18 >= 6)) AND (ss_quantity#18 <= 10)) AND ((((ss_list_price#20 >= 90.00) AND (ss_list_price#20 <= 100.00)) OR ((ss_coupon_amt#21 >= 2323.00) AND (ss_coupon_amt#21 <= 3323.00))) OR ((ss_wholesale_cost#19 >= 31.00) AND (ss_wholesale_cost#19 <= 51.00)))) (14) Project [codegen id : 3] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#20] +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] (15) HashAggregate [codegen id : 3] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] -Results [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [1]: [ss_list_price#20] +Keys [1]: [ss_list_price#20] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#20)), partial_count(ss_list_price#20)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] +Results [4]: [ss_list_price#20, sum#25, count#26, count#27] (16) Exchange -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] +Arguments: hashpartitioning(ss_list_price#20, 5), ENSURE_REQUIREMENTS, [id=#28] (17) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] -Results [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] +Keys [1]: [ss_list_price#20] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] +Results [4]: [ss_list_price#20, sum#25, count#26, count#27] (18) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] -Results [4]: [sum#20, count#21, count#22, count#25] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20), partial_count(distinct ss_list_price#20)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] +Results [4]: [sum#25, count#26, count#27, count#30] (19) Exchange -Input [4]: [sum#20, count#21, count#22, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] +Input [4]: [sum#25, count#26, count#27, count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] (20) HashAggregate [codegen id : 5] -Input [4]: [sum#20, count#21, count#22, count#25] +Input [4]: [sum#25, count#26, count#27, count#30] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#18 / 100.0) as decimal(11,6)) AS B2_LP#27, count(ss_list_price#3)#19 AS B2_CNT#28, count(ss_list_price#3)#24 AS B2_CNTD#29] +Functions [3]: [avg(UnscaledValue(ss_list_price#20)), count(ss_list_price#20), count(distinct ss_list_price#20)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#20))#23 / 100.0) as decimal(11,6)) AS B2_LP#32, count(ss_list_price#20)#24 AS B2_CNT#33, count(ss_list_price#20)#29 AS B2_CNTD#34] (21) BroadcastExchange -Input [3]: [B2_LP#27, B2_CNT#28, B2_CNTD#29] -Arguments: IdentityBroadcastMode, [id=#30] +Input [3]: [B2_LP#32, B2_CNT#33, B2_CNTD#34] +Arguments: IdentityBroadcastMode, [id=#35] (22) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (23) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] (25) Filter [codegen id : 6] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 11)) AND (ss_quantity#1 <= 15)) AND ((((ss_list_price#3 >= 142.00) AND (ss_list_price#3 <= 152.00)) OR ((ss_coupon_amt#4 >= 12214.00) AND (ss_coupon_amt#4 <= 13214.00))) OR ((ss_wholesale_cost#2 >= 79.00) AND (ss_wholesale_cost#2 <= 99.00)))) +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Condition : (((isnotnull(ss_quantity#36) AND (ss_quantity#36 >= 11)) AND (ss_quantity#36 <= 15)) AND ((((ss_list_price#38 >= 142.00) AND (ss_list_price#38 <= 152.00)) OR ((ss_coupon_amt#39 >= 12214.00) AND (ss_coupon_amt#39 <= 13214.00))) OR ((ss_wholesale_cost#37 >= 79.00) AND (ss_wholesale_cost#37 <= 99.00)))) (26) Project [codegen id : 6] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#38] +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] (27) HashAggregate [codegen id : 6] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] -Results [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [1]: [ss_list_price#38] +Keys [1]: [ss_list_price#38] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#38)), partial_count(ss_list_price#38)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] +Results [4]: [ss_list_price#38, sum#43, count#44, count#45] (28) Exchange -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] +Arguments: hashpartitioning(ss_list_price#38, 5), ENSURE_REQUIREMENTS, [id=#46] (29) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] -Results [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] +Keys [1]: [ss_list_price#38] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] +Results [4]: [ss_list_price#38, sum#43, count#44, count#45] (30) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] -Results [4]: [sum#33, count#34, count#35, count#38] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38), partial_count(distinct ss_list_price#38)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] +Results [4]: [sum#43, count#44, count#45, count#48] (31) Exchange -Input [4]: [sum#33, count#34, count#35, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#39] +Input [4]: [sum#43, count#44, count#45, count#48] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] (32) HashAggregate [codegen id : 8] -Input [4]: [sum#33, count#34, count#35, count#38] +Input [4]: [sum#43, count#44, count#45, count#48] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#31 / 100.0) as decimal(11,6)) AS B3_LP#40, count(ss_list_price#3)#32 AS B3_CNT#41, count(ss_list_price#3)#37 AS B3_CNTD#42] +Functions [3]: [avg(UnscaledValue(ss_list_price#38)), count(ss_list_price#38), count(distinct ss_list_price#38)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#38))#41 / 100.0) as decimal(11,6)) AS B3_LP#50, count(ss_list_price#38)#42 AS B3_CNT#51, count(ss_list_price#38)#47 AS B3_CNTD#52] (33) BroadcastExchange -Input [3]: [B3_LP#40, B3_CNT#41, B3_CNTD#42] -Arguments: IdentityBroadcastMode, [id=#43] +Input [3]: [B3_LP#50, B3_CNT#51, B3_CNTD#52] +Arguments: IdentityBroadcastMode, [id=#53] (34) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (35) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] ReadSchema: struct (36) ColumnarToRow [codegen id : 9] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] (37) Filter [codegen id : 9] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 16)) AND (ss_quantity#1 <= 20)) AND ((((ss_list_price#3 >= 135.00) AND (ss_list_price#3 <= 145.00)) OR ((ss_coupon_amt#4 >= 6071.00) AND (ss_coupon_amt#4 <= 7071.00))) OR ((ss_wholesale_cost#2 >= 38.00) AND (ss_wholesale_cost#2 <= 58.00)))) +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Condition : (((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 16)) AND (ss_quantity#54 <= 20)) AND ((((ss_list_price#56 >= 135.00) AND (ss_list_price#56 <= 145.00)) OR ((ss_coupon_amt#57 >= 6071.00) AND (ss_coupon_amt#57 <= 7071.00))) OR ((ss_wholesale_cost#55 >= 38.00) AND (ss_wholesale_cost#55 <= 58.00)))) (38) Project [codegen id : 9] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#56] +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] (39) HashAggregate [codegen id : 9] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] -Results [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [1]: [ss_list_price#56] +Keys [1]: [ss_list_price#56] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#56)), partial_count(ss_list_price#56)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] +Results [4]: [ss_list_price#56, sum#61, count#62, count#63] (40) Exchange -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] +Arguments: hashpartitioning(ss_list_price#56, 5), ENSURE_REQUIREMENTS, [id=#64] (41) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] -Results [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] +Keys [1]: [ss_list_price#56] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] +Results [4]: [ss_list_price#56, sum#61, count#62, count#63] (42) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] -Results [4]: [sum#46, count#47, count#48, count#51] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56), partial_count(distinct ss_list_price#56)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] +Results [4]: [sum#61, count#62, count#63, count#66] (43) Exchange -Input [4]: [sum#46, count#47, count#48, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] +Input [4]: [sum#61, count#62, count#63, count#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] (44) HashAggregate [codegen id : 11] -Input [4]: [sum#46, count#47, count#48, count#51] +Input [4]: [sum#61, count#62, count#63, count#66] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#44 / 100.0) as decimal(11,6)) AS B4_LP#53, count(ss_list_price#3)#45 AS B4_CNT#54, count(ss_list_price#3)#50 AS B4_CNTD#55] +Functions [3]: [avg(UnscaledValue(ss_list_price#56)), count(ss_list_price#56), count(distinct ss_list_price#56)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#56))#59 / 100.0) as decimal(11,6)) AS B4_LP#68, count(ss_list_price#56)#60 AS B4_CNT#69, count(ss_list_price#56)#65 AS B4_CNTD#70] (45) BroadcastExchange -Input [3]: [B4_LP#53, B4_CNT#54, B4_CNTD#55] -Arguments: IdentityBroadcastMode, [id=#56] +Input [3]: [B4_LP#68, B4_CNT#69, B4_CNTD#70] +Arguments: IdentityBroadcastMode, [id=#71] (46) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (47) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] ReadSchema: struct (48) ColumnarToRow [codegen id : 12] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] (49) Filter [codegen id : 12] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 21)) AND (ss_quantity#1 <= 25)) AND ((((ss_list_price#3 >= 122.00) AND (ss_list_price#3 <= 132.00)) OR ((ss_coupon_amt#4 >= 836.00) AND (ss_coupon_amt#4 <= 1836.00))) OR ((ss_wholesale_cost#2 >= 17.00) AND (ss_wholesale_cost#2 <= 37.00)))) +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Condition : (((isnotnull(ss_quantity#72) AND (ss_quantity#72 >= 21)) AND (ss_quantity#72 <= 25)) AND ((((ss_list_price#74 >= 122.00) AND (ss_list_price#74 <= 132.00)) OR ((ss_coupon_amt#75 >= 836.00) AND (ss_coupon_amt#75 <= 1836.00))) OR ((ss_wholesale_cost#73 >= 17.00) AND (ss_wholesale_cost#73 <= 37.00)))) (50) Project [codegen id : 12] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#74] +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] (51) HashAggregate [codegen id : 12] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] -Results [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [1]: [ss_list_price#74] +Keys [1]: [ss_list_price#74] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#74)), partial_count(ss_list_price#74)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] +Results [4]: [ss_list_price#74, sum#79, count#80, count#81] (52) Exchange -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] +Arguments: hashpartitioning(ss_list_price#74, 5), ENSURE_REQUIREMENTS, [id=#82] (53) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] -Results [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] +Keys [1]: [ss_list_price#74] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] +Results [4]: [ss_list_price#74, sum#79, count#80, count#81] (54) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] -Results [4]: [sum#59, count#60, count#61, count#64] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74), partial_count(distinct ss_list_price#74)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] +Results [4]: [sum#79, count#80, count#81, count#84] (55) Exchange -Input [4]: [sum#59, count#60, count#61, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] +Input [4]: [sum#79, count#80, count#81, count#84] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#85] (56) HashAggregate [codegen id : 14] -Input [4]: [sum#59, count#60, count#61, count#64] +Input [4]: [sum#79, count#80, count#81, count#84] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#57 / 100.0) as decimal(11,6)) AS B5_LP#66, count(ss_list_price#3)#58 AS B5_CNT#67, count(ss_list_price#3)#63 AS B5_CNTD#68] +Functions [3]: [avg(UnscaledValue(ss_list_price#74)), count(ss_list_price#74), count(distinct ss_list_price#74)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#74))#77 / 100.0) as decimal(11,6)) AS B5_LP#86, count(ss_list_price#74)#78 AS B5_CNT#87, count(ss_list_price#74)#83 AS B5_CNTD#88] (57) BroadcastExchange -Input [3]: [B5_LP#66, B5_CNT#67, B5_CNTD#68] -Arguments: IdentityBroadcastMode, [id=#69] +Input [3]: [B5_LP#86, B5_CNT#87, B5_CNTD#88] +Arguments: IdentityBroadcastMode, [id=#89] (58) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (59) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] ReadSchema: struct (60) ColumnarToRow [codegen id : 15] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] (61) Filter [codegen id : 15] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 26)) AND (ss_quantity#1 <= 30)) AND ((((ss_list_price#3 >= 154.00) AND (ss_list_price#3 <= 164.00)) OR ((ss_coupon_amt#4 >= 7326.00) AND (ss_coupon_amt#4 <= 8326.00))) OR ((ss_wholesale_cost#2 >= 7.00) AND (ss_wholesale_cost#2 <= 27.00)))) +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Condition : (((isnotnull(ss_quantity#90) AND (ss_quantity#90 >= 26)) AND (ss_quantity#90 <= 30)) AND ((((ss_list_price#92 >= 154.00) AND (ss_list_price#92 <= 164.00)) OR ((ss_coupon_amt#93 >= 7326.00) AND (ss_coupon_amt#93 <= 8326.00))) OR ((ss_wholesale_cost#91 >= 7.00) AND (ss_wholesale_cost#91 <= 27.00)))) (62) Project [codegen id : 15] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#92] +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] (63) HashAggregate [codegen id : 15] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] -Results [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [1]: [ss_list_price#92] +Keys [1]: [ss_list_price#92] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#92)), partial_count(ss_list_price#92)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] +Results [4]: [ss_list_price#92, sum#97, count#98, count#99] (64) Exchange -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] +Arguments: hashpartitioning(ss_list_price#92, 5), ENSURE_REQUIREMENTS, [id=#100] (65) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] -Results [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] +Keys [1]: [ss_list_price#92] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] +Results [4]: [ss_list_price#92, sum#97, count#98, count#99] (66) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] -Results [4]: [sum#72, count#73, count#74, count#77] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92), partial_count(distinct ss_list_price#92)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] +Results [4]: [sum#97, count#98, count#99, count#102] (67) Exchange -Input [4]: [sum#72, count#73, count#74, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] +Input [4]: [sum#97, count#98, count#99, count#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103] (68) HashAggregate [codegen id : 17] -Input [4]: [sum#72, count#73, count#74, count#77] +Input [4]: [sum#97, count#98, count#99, count#102] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#70 / 100.0) as decimal(11,6)) AS B6_LP#79, count(ss_list_price#3)#71 AS B6_CNT#80, count(ss_list_price#3)#76 AS B6_CNTD#81] +Functions [3]: [avg(UnscaledValue(ss_list_price#92)), count(ss_list_price#92), count(distinct ss_list_price#92)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#92))#95 / 100.0) as decimal(11,6)) AS B6_LP#104, count(ss_list_price#92)#96 AS B6_CNT#105, count(ss_list_price#92)#101 AS B6_CNTD#106] (69) BroadcastExchange -Input [3]: [B6_LP#79, B6_CNT#80, B6_CNTD#81] -Arguments: IdentityBroadcastMode, [id=#82] +Input [3]: [B6_LP#104, B6_CNT#105, B6_CNTD#106] +Arguments: IdentityBroadcastMode, [id=#107] (70) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt index 9f15199d8c8b7..eec45ea549531 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/explain.txt @@ -126,306 +126,306 @@ Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_ Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#6 / 100.0) as decimal(11,6)) AS B1_LP#15, count(ss_list_price#3)#7 AS B1_CNT#16, count(ss_list_price#3)#12 AS B1_CNTD#17] (11) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] ReadSchema: struct (12) ColumnarToRow [codegen id : 3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] (13) Filter [codegen id : 3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 6)) AND (ss_quantity#1 <= 10)) AND ((((ss_list_price#3 >= 90.00) AND (ss_list_price#3 <= 100.00)) OR ((ss_coupon_amt#4 >= 2323.00) AND (ss_coupon_amt#4 <= 3323.00))) OR ((ss_wholesale_cost#2 >= 31.00) AND (ss_wholesale_cost#2 <= 51.00)))) +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] +Condition : (((isnotnull(ss_quantity#18) AND (ss_quantity#18 >= 6)) AND (ss_quantity#18 <= 10)) AND ((((ss_list_price#20 >= 90.00) AND (ss_list_price#20 <= 100.00)) OR ((ss_coupon_amt#21 >= 2323.00) AND (ss_coupon_amt#21 <= 3323.00))) OR ((ss_wholesale_cost#19 >= 31.00) AND (ss_wholesale_cost#19 <= 51.00)))) (14) Project [codegen id : 3] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#20] +Input [5]: [ss_quantity#18, ss_wholesale_cost#19, ss_list_price#20, ss_coupon_amt#21, ss_sold_date_sk#22] (15) HashAggregate [codegen id : 3] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] -Results [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [1]: [ss_list_price#20] +Keys [1]: [ss_list_price#20] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#20)), partial_count(ss_list_price#20)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] +Results [4]: [ss_list_price#20, sum#25, count#26, count#27] (16) Exchange -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] +Arguments: hashpartitioning(ss_list_price#20, 5), ENSURE_REQUIREMENTS, [id=#28] (17) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19] -Results [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] +Keys [1]: [ss_list_price#20] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24] +Results [4]: [ss_list_price#20, sum#25, count#26, count#27] (18) HashAggregate [codegen id : 4] -Input [4]: [ss_list_price#3, sum#20, count#21, count#22] +Input [4]: [ss_list_price#20, sum#25, count#26, count#27] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] -Results [4]: [sum#20, count#21, count#22, count#25] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#20)), merge_count(ss_list_price#20), partial_count(distinct ss_list_price#20)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] +Results [4]: [sum#25, count#26, count#27, count#30] (19) Exchange -Input [4]: [sum#20, count#21, count#22, count#25] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] +Input [4]: [sum#25, count#26, count#27, count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] (20) HashAggregate [codegen id : 5] -Input [4]: [sum#20, count#21, count#22, count#25] +Input [4]: [sum#25, count#26, count#27, count#30] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#18, count(ss_list_price#3)#19, count(ss_list_price#3)#24] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#18 / 100.0) as decimal(11,6)) AS B2_LP#27, count(ss_list_price#3)#19 AS B2_CNT#28, count(ss_list_price#3)#24 AS B2_CNTD#29] +Functions [3]: [avg(UnscaledValue(ss_list_price#20)), count(ss_list_price#20), count(distinct ss_list_price#20)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#20))#23, count(ss_list_price#20)#24, count(ss_list_price#20)#29] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#20))#23 / 100.0) as decimal(11,6)) AS B2_LP#32, count(ss_list_price#20)#24 AS B2_CNT#33, count(ss_list_price#20)#29 AS B2_CNTD#34] (21) BroadcastExchange -Input [3]: [B2_LP#27, B2_CNT#28, B2_CNTD#29] -Arguments: IdentityBroadcastMode, [id=#30] +Input [3]: [B2_LP#32, B2_CNT#33, B2_CNTD#34] +Arguments: IdentityBroadcastMode, [id=#35] (22) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (23) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] ReadSchema: struct (24) ColumnarToRow [codegen id : 6] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] (25) Filter [codegen id : 6] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 11)) AND (ss_quantity#1 <= 15)) AND ((((ss_list_price#3 >= 142.00) AND (ss_list_price#3 <= 152.00)) OR ((ss_coupon_amt#4 >= 12214.00) AND (ss_coupon_amt#4 <= 13214.00))) OR ((ss_wholesale_cost#2 >= 79.00) AND (ss_wholesale_cost#2 <= 99.00)))) +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] +Condition : (((isnotnull(ss_quantity#36) AND (ss_quantity#36 >= 11)) AND (ss_quantity#36 <= 15)) AND ((((ss_list_price#38 >= 142.00) AND (ss_list_price#38 <= 152.00)) OR ((ss_coupon_amt#39 >= 12214.00) AND (ss_coupon_amt#39 <= 13214.00))) OR ((ss_wholesale_cost#37 >= 79.00) AND (ss_wholesale_cost#37 <= 99.00)))) (26) Project [codegen id : 6] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#38] +Input [5]: [ss_quantity#36, ss_wholesale_cost#37, ss_list_price#38, ss_coupon_amt#39, ss_sold_date_sk#40] (27) HashAggregate [codegen id : 6] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] -Results [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [1]: [ss_list_price#38] +Keys [1]: [ss_list_price#38] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#38)), partial_count(ss_list_price#38)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] +Results [4]: [ss_list_price#38, sum#43, count#44, count#45] (28) Exchange -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] +Arguments: hashpartitioning(ss_list_price#38, 5), ENSURE_REQUIREMENTS, [id=#46] (29) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32] -Results [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] +Keys [1]: [ss_list_price#38] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42] +Results [4]: [ss_list_price#38, sum#43, count#44, count#45] (30) HashAggregate [codegen id : 7] -Input [4]: [ss_list_price#3, sum#33, count#34, count#35] +Input [4]: [ss_list_price#38, sum#43, count#44, count#45] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] -Results [4]: [sum#33, count#34, count#35, count#38] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#38)), merge_count(ss_list_price#38), partial_count(distinct ss_list_price#38)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] +Results [4]: [sum#43, count#44, count#45, count#48] (31) Exchange -Input [4]: [sum#33, count#34, count#35, count#38] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#39] +Input [4]: [sum#43, count#44, count#45, count#48] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] (32) HashAggregate [codegen id : 8] -Input [4]: [sum#33, count#34, count#35, count#38] +Input [4]: [sum#43, count#44, count#45, count#48] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#31, count(ss_list_price#3)#32, count(ss_list_price#3)#37] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#31 / 100.0) as decimal(11,6)) AS B3_LP#40, count(ss_list_price#3)#32 AS B3_CNT#41, count(ss_list_price#3)#37 AS B3_CNTD#42] +Functions [3]: [avg(UnscaledValue(ss_list_price#38)), count(ss_list_price#38), count(distinct ss_list_price#38)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#38))#41, count(ss_list_price#38)#42, count(ss_list_price#38)#47] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#38))#41 / 100.0) as decimal(11,6)) AS B3_LP#50, count(ss_list_price#38)#42 AS B3_CNT#51, count(ss_list_price#38)#47 AS B3_CNTD#52] (33) BroadcastExchange -Input [3]: [B3_LP#40, B3_CNT#41, B3_CNTD#42] -Arguments: IdentityBroadcastMode, [id=#43] +Input [3]: [B3_LP#50, B3_CNT#51, B3_CNTD#52] +Arguments: IdentityBroadcastMode, [id=#53] (34) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (35) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] ReadSchema: struct (36) ColumnarToRow [codegen id : 9] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] (37) Filter [codegen id : 9] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 16)) AND (ss_quantity#1 <= 20)) AND ((((ss_list_price#3 >= 135.00) AND (ss_list_price#3 <= 145.00)) OR ((ss_coupon_amt#4 >= 6071.00) AND (ss_coupon_amt#4 <= 7071.00))) OR ((ss_wholesale_cost#2 >= 38.00) AND (ss_wholesale_cost#2 <= 58.00)))) +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] +Condition : (((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 16)) AND (ss_quantity#54 <= 20)) AND ((((ss_list_price#56 >= 135.00) AND (ss_list_price#56 <= 145.00)) OR ((ss_coupon_amt#57 >= 6071.00) AND (ss_coupon_amt#57 <= 7071.00))) OR ((ss_wholesale_cost#55 >= 38.00) AND (ss_wholesale_cost#55 <= 58.00)))) (38) Project [codegen id : 9] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#56] +Input [5]: [ss_quantity#54, ss_wholesale_cost#55, ss_list_price#56, ss_coupon_amt#57, ss_sold_date_sk#58] (39) HashAggregate [codegen id : 9] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] -Results [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [1]: [ss_list_price#56] +Keys [1]: [ss_list_price#56] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#56)), partial_count(ss_list_price#56)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] +Results [4]: [ss_list_price#56, sum#61, count#62, count#63] (40) Exchange -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] +Arguments: hashpartitioning(ss_list_price#56, 5), ENSURE_REQUIREMENTS, [id=#64] (41) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45] -Results [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] +Keys [1]: [ss_list_price#56] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60] +Results [4]: [ss_list_price#56, sum#61, count#62, count#63] (42) HashAggregate [codegen id : 10] -Input [4]: [ss_list_price#3, sum#46, count#47, count#48] +Input [4]: [ss_list_price#56, sum#61, count#62, count#63] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] -Results [4]: [sum#46, count#47, count#48, count#51] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#56)), merge_count(ss_list_price#56), partial_count(distinct ss_list_price#56)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] +Results [4]: [sum#61, count#62, count#63, count#66] (43) Exchange -Input [4]: [sum#46, count#47, count#48, count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] +Input [4]: [sum#61, count#62, count#63, count#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] (44) HashAggregate [codegen id : 11] -Input [4]: [sum#46, count#47, count#48, count#51] +Input [4]: [sum#61, count#62, count#63, count#66] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#44, count(ss_list_price#3)#45, count(ss_list_price#3)#50] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#44 / 100.0) as decimal(11,6)) AS B4_LP#53, count(ss_list_price#3)#45 AS B4_CNT#54, count(ss_list_price#3)#50 AS B4_CNTD#55] +Functions [3]: [avg(UnscaledValue(ss_list_price#56)), count(ss_list_price#56), count(distinct ss_list_price#56)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#56))#59, count(ss_list_price#56)#60, count(ss_list_price#56)#65] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#56))#59 / 100.0) as decimal(11,6)) AS B4_LP#68, count(ss_list_price#56)#60 AS B4_CNT#69, count(ss_list_price#56)#65 AS B4_CNTD#70] (45) BroadcastExchange -Input [3]: [B4_LP#53, B4_CNT#54, B4_CNTD#55] -Arguments: IdentityBroadcastMode, [id=#56] +Input [3]: [B4_LP#68, B4_CNT#69, B4_CNTD#70] +Arguments: IdentityBroadcastMode, [id=#71] (46) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (47) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] ReadSchema: struct (48) ColumnarToRow [codegen id : 12] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] (49) Filter [codegen id : 12] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 21)) AND (ss_quantity#1 <= 25)) AND ((((ss_list_price#3 >= 122.00) AND (ss_list_price#3 <= 132.00)) OR ((ss_coupon_amt#4 >= 836.00) AND (ss_coupon_amt#4 <= 1836.00))) OR ((ss_wholesale_cost#2 >= 17.00) AND (ss_wholesale_cost#2 <= 37.00)))) +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] +Condition : (((isnotnull(ss_quantity#72) AND (ss_quantity#72 >= 21)) AND (ss_quantity#72 <= 25)) AND ((((ss_list_price#74 >= 122.00) AND (ss_list_price#74 <= 132.00)) OR ((ss_coupon_amt#75 >= 836.00) AND (ss_coupon_amt#75 <= 1836.00))) OR ((ss_wholesale_cost#73 >= 17.00) AND (ss_wholesale_cost#73 <= 37.00)))) (50) Project [codegen id : 12] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#74] +Input [5]: [ss_quantity#72, ss_wholesale_cost#73, ss_list_price#74, ss_coupon_amt#75, ss_sold_date_sk#76] (51) HashAggregate [codegen id : 12] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] -Results [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [1]: [ss_list_price#74] +Keys [1]: [ss_list_price#74] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#74)), partial_count(ss_list_price#74)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] +Results [4]: [ss_list_price#74, sum#79, count#80, count#81] (52) Exchange -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#62] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] +Arguments: hashpartitioning(ss_list_price#74, 5), ENSURE_REQUIREMENTS, [id=#82] (53) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58] -Results [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] +Keys [1]: [ss_list_price#74] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78] +Results [4]: [ss_list_price#74, sum#79, count#80, count#81] (54) HashAggregate [codegen id : 13] -Input [4]: [ss_list_price#3, sum#59, count#60, count#61] +Input [4]: [ss_list_price#74, sum#79, count#80, count#81] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] -Results [4]: [sum#59, count#60, count#61, count#64] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#74)), merge_count(ss_list_price#74), partial_count(distinct ss_list_price#74)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] +Results [4]: [sum#79, count#80, count#81, count#84] (55) Exchange -Input [4]: [sum#59, count#60, count#61, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] +Input [4]: [sum#79, count#80, count#81, count#84] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#85] (56) HashAggregate [codegen id : 14] -Input [4]: [sum#59, count#60, count#61, count#64] +Input [4]: [sum#79, count#80, count#81, count#84] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#57, count(ss_list_price#3)#58, count(ss_list_price#3)#63] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#57 / 100.0) as decimal(11,6)) AS B5_LP#66, count(ss_list_price#3)#58 AS B5_CNT#67, count(ss_list_price#3)#63 AS B5_CNTD#68] +Functions [3]: [avg(UnscaledValue(ss_list_price#74)), count(ss_list_price#74), count(distinct ss_list_price#74)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#74))#77, count(ss_list_price#74)#78, count(ss_list_price#74)#83] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#74))#77 / 100.0) as decimal(11,6)) AS B5_LP#86, count(ss_list_price#74)#78 AS B5_CNT#87, count(ss_list_price#74)#83 AS B5_CNTD#88] (57) BroadcastExchange -Input [3]: [B5_LP#66, B5_CNT#67, B5_CNTD#68] -Arguments: IdentityBroadcastMode, [id=#69] +Input [3]: [B5_LP#86, B5_CNT#87, B5_CNTD#88] +Arguments: IdentityBroadcastMode, [id=#89] (58) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None (59) Scan parquet default.store_sales -Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] ReadSchema: struct (60) ColumnarToRow [codegen id : 15] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] (61) Filter [codegen id : 15] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] -Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 26)) AND (ss_quantity#1 <= 30)) AND ((((ss_list_price#3 >= 154.00) AND (ss_list_price#3 <= 164.00)) OR ((ss_coupon_amt#4 >= 7326.00) AND (ss_coupon_amt#4 <= 8326.00))) OR ((ss_wholesale_cost#2 >= 7.00) AND (ss_wholesale_cost#2 <= 27.00)))) +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] +Condition : (((isnotnull(ss_quantity#90) AND (ss_quantity#90 >= 26)) AND (ss_quantity#90 <= 30)) AND ((((ss_list_price#92 >= 154.00) AND (ss_list_price#92 <= 164.00)) OR ((ss_coupon_amt#93 >= 7326.00) AND (ss_coupon_amt#93 <= 8326.00))) OR ((ss_wholesale_cost#91 >= 7.00) AND (ss_wholesale_cost#91 <= 27.00)))) (62) Project [codegen id : 15] -Output [1]: [ss_list_price#3] -Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Output [1]: [ss_list_price#92] +Input [5]: [ss_quantity#90, ss_wholesale_cost#91, ss_list_price#92, ss_coupon_amt#93, ss_sold_date_sk#94] (63) HashAggregate [codegen id : 15] -Input [1]: [ss_list_price#3] -Keys [1]: [ss_list_price#3] -Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] -Results [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [1]: [ss_list_price#92] +Keys [1]: [ss_list_price#92] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#92)), partial_count(ss_list_price#92)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] +Results [4]: [ss_list_price#92, sum#97, count#98, count#99] (64) Exchange -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] -Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] +Arguments: hashpartitioning(ss_list_price#92, 5), ENSURE_REQUIREMENTS, [id=#100] (65) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] -Keys [1]: [ss_list_price#3] -Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] -Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71] -Results [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] +Keys [1]: [ss_list_price#92] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96] +Results [4]: [ss_list_price#92, sum#97, count#98, count#99] (66) HashAggregate [codegen id : 16] -Input [4]: [ss_list_price#3, sum#72, count#73, count#74] +Input [4]: [ss_list_price#92, sum#97, count#98, count#99] Keys: [] -Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] -Results [4]: [sum#72, count#73, count#74, count#77] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#92)), merge_count(ss_list_price#92), partial_count(distinct ss_list_price#92)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] +Results [4]: [sum#97, count#98, count#99, count#102] (67) Exchange -Input [4]: [sum#72, count#73, count#74, count#77] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] +Input [4]: [sum#97, count#98, count#99, count#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#103] (68) HashAggregate [codegen id : 17] -Input [4]: [sum#72, count#73, count#74, count#77] +Input [4]: [sum#97, count#98, count#99, count#102] Keys: [] -Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] -Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#70, count(ss_list_price#3)#71, count(ss_list_price#3)#76] -Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#70 / 100.0) as decimal(11,6)) AS B6_LP#79, count(ss_list_price#3)#71 AS B6_CNT#80, count(ss_list_price#3)#76 AS B6_CNTD#81] +Functions [3]: [avg(UnscaledValue(ss_list_price#92)), count(ss_list_price#92), count(distinct ss_list_price#92)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#92))#95, count(ss_list_price#92)#96, count(ss_list_price#92)#101] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#92))#95 / 100.0) as decimal(11,6)) AS B6_LP#104, count(ss_list_price#92)#96 AS B6_CNT#105, count(ss_list_price#92)#101 AS B6_CNTD#106] (69) BroadcastExchange -Input [3]: [B6_LP#79, B6_CNT#80, B6_CNTD#81] -Arguments: IdentityBroadcastMode, [id=#82] +Input [3]: [B6_LP#104, B6_CNT#105, B6_CNTD#106] +Arguments: IdentityBroadcastMode, [id=#107] (70) BroadcastNestedLoopJoin [codegen id : 18] Join condition: None 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 ca09ec6ea1a42..9e84e385be302 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 @@ -168,170 +168,170 @@ Input [3]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19] Condition : isnotnull(ctr_total_return#19) (24) Scan parquet default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +Output [4]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#23), dynamicpruningexpression(wr_returned_date_sk#23 IN dynamicpruning#5)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 8] -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +Input [4]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23] (26) Filter [codegen id : 8] -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) +Input [4]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23] +Condition : isnotnull(wr_returning_addr_sk#21) (27) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#24] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [wr_returned_date_sk#4] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#23] +Right keys [1]: [cast(d_date_sk#24 as bigint)] Join condition: None (29) Project [codegen id : 8] -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] +Output [3]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22] +Input [5]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, wr_returned_date_sk#23, d_date_sk#24] (30) Exchange -Input [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Arguments: hashpartitioning(wr_returning_addr_sk#2, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [3]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22] +Arguments: hashpartitioning(wr_returning_addr_sk#21, 5), ENSURE_REQUIREMENTS, [id=#25] (31) Sort [codegen id : 9] -Input [3]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3] -Arguments: [wr_returning_addr_sk#2 ASC NULLS FIRST], false, 0 +Input [3]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22] +Arguments: [wr_returning_addr_sk#21 ASC NULLS FIRST], false, 0 (32) ReusedExchange [Reuses operator id: 16] -Output [2]: [ca_address_sk#10, ca_state#11] +Output [2]: [ca_address_sk#26, ca_state#27] (33) Sort [codegen id : 11] -Input [2]: [ca_address_sk#10, ca_state#11] -Arguments: [cast(ca_address_sk#10 as bigint) ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#26, ca_state#27] +Arguments: [cast(ca_address_sk#26 as bigint) ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 12] -Left keys [1]: [wr_returning_addr_sk#2] -Right keys [1]: [cast(ca_address_sk#10 as bigint)] +Left keys [1]: [wr_returning_addr_sk#21] +Right keys [1]: [cast(ca_address_sk#26 as bigint)] Join condition: None (35) Project [codegen id : 12] -Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#11] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#10, ca_state#11] +Output [3]: [wr_returning_customer_sk#20, wr_return_amt#22, ca_state#27] +Input [5]: [wr_returning_customer_sk#20, wr_returning_addr_sk#21, wr_return_amt#22, ca_address_sk#26, ca_state#27] (36) HashAggregate [codegen id : 12] -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#11] -Keys [2]: [wr_returning_customer_sk#1, ca_state#11] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum#21] -Results [3]: [wr_returning_customer_sk#1, ca_state#11, sum#22] +Input [3]: [wr_returning_customer_sk#20, wr_return_amt#22, ca_state#27] +Keys [2]: [wr_returning_customer_sk#20, ca_state#27] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#22))] +Aggregate Attributes [1]: [sum#28] +Results [3]: [wr_returning_customer_sk#20, ca_state#27, sum#29] (37) Exchange -Input [3]: [wr_returning_customer_sk#1, ca_state#11, sum#22] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#11, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [wr_returning_customer_sk#20, ca_state#27, sum#29] +Arguments: hashpartitioning(wr_returning_customer_sk#20, ca_state#27, 5), ENSURE_REQUIREMENTS, [id=#30] (38) HashAggregate [codegen id : 13] -Input [3]: [wr_returning_customer_sk#1, ca_state#11, sum#22] -Keys [2]: [wr_returning_customer_sk#1, ca_state#11] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#24] -Results [2]: [ca_state#11 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#24,17,2) AS ctr_total_return#19] +Input [3]: [wr_returning_customer_sk#20, ca_state#27, sum#29] +Keys [2]: [wr_returning_customer_sk#20, ca_state#27] +Functions [1]: [sum(UnscaledValue(wr_return_amt#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#22))#31] +Results [2]: [ca_state#27 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(wr_return_amt#22))#31,17,2) AS ctr_total_return#19] (39) HashAggregate [codegen id : 13] Input [2]: [ctr_state#18, ctr_total_return#19] Keys [1]: [ctr_state#18] Functions [1]: [partial_avg(ctr_total_return#19)] -Aggregate Attributes [2]: [sum#25, count#26] -Results [3]: [ctr_state#18, sum#27, count#28] +Aggregate Attributes [2]: [sum#32, count#33] +Results [3]: [ctr_state#18, sum#34, count#35] (40) Exchange -Input [3]: [ctr_state#18, sum#27, count#28] -Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ctr_state#18, sum#34, count#35] +Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#36] (41) HashAggregate [codegen id : 14] -Input [3]: [ctr_state#18, sum#27, count#28] +Input [3]: [ctr_state#18, sum#34, count#35] Keys [1]: [ctr_state#18] Functions [1]: [avg(ctr_total_return#19)] -Aggregate Attributes [1]: [avg(ctr_total_return#19)#30] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#18 AS ctr_state#18#32] +Aggregate Attributes [1]: [avg(ctr_total_return#19)#37] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#37) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#38, ctr_state#18 AS ctr_state#18#39] (42) Filter [codegen id : 14] -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) +Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#38) (43) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#33] +Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#40] (44) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ctr_state#18] -Right keys [1]: [ctr_state#18#32] -Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) +Right keys [1]: [ctr_state#18#39] +Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#38) (45) Project [codegen id : 17] Output [2]: [ctr_customer_sk#17, ctr_total_return#19] -Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] +Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] (46) Scan parquet default.customer -Output [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Output [14]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 16] -Input [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Input [14]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] (48) Filter [codegen id : 16] -Input [14]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] -Condition : (isnotnull(c_customer_sk#34) AND isnotnull(c_current_addr_sk#36)) +Input [14]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] +Condition : (isnotnull(c_customer_sk#41) AND isnotnull(c_current_addr_sk#43)) (49) Scan parquet default.customer_address -Output [2]: [ca_address_sk#10, ca_state#11] +Output [2]: [ca_address_sk#55, ca_state#56] 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 (50) ColumnarToRow [codegen id : 15] -Input [2]: [ca_address_sk#10, ca_state#11] +Input [2]: [ca_address_sk#55, ca_state#56] (51) Filter [codegen id : 15] -Input [2]: [ca_address_sk#10, ca_state#11] -Condition : ((isnotnull(ca_state#11) AND (ca_state#11 = GA)) AND isnotnull(ca_address_sk#10)) +Input [2]: [ca_address_sk#55, ca_state#56] +Condition : ((isnotnull(ca_state#56) AND (ca_state#56 = GA)) AND isnotnull(ca_address_sk#55)) (52) Project [codegen id : 15] -Output [1]: [ca_address_sk#10] -Input [2]: [ca_address_sk#10, ca_state#11] +Output [1]: [ca_address_sk#55] +Input [2]: [ca_address_sk#55, ca_state#56] (53) BroadcastExchange -Input [1]: [ca_address_sk#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] +Input [1]: [ca_address_sk#55] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#57] (54) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [c_current_addr_sk#36] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [c_current_addr_sk#43] +Right keys [1]: [ca_address_sk#55] Join condition: None (55) Project [codegen id : 16] -Output [13]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] -Input [15]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ca_address_sk#10] +Output [13]: [c_customer_sk#41, c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] +Input [15]: [c_customer_sk#41, c_customer_id#42, c_current_addr_sk#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ca_address_sk#55] (56) BroadcastExchange -Input [13]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] +Input [13]: [c_customer_sk#41, c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#58] (57) BroadcastHashJoin [codegen id : 17] Left keys [1]: [ctr_customer_sk#17] -Right keys [1]: [cast(c_customer_sk#34 as bigint)] +Right keys [1]: [cast(c_customer_sk#41 as bigint)] Join condition: None (58) Project [codegen id : 17] -Output [13]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#19] -Input [15]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47] +Output [13]: [c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ctr_total_return#19] +Input [15]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#41, c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54] (59) TakeOrderedAndProject -Input [13]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#19] -Arguments: 100, [c_customer_id#35 ASC NULLS FIRST, c_salutation#37 ASC NULLS FIRST, c_first_name#38 ASC NULLS FIRST, c_last_name#39 ASC NULLS FIRST, c_preferred_cust_flag#40 ASC NULLS FIRST, c_birth_day#41 ASC NULLS FIRST, c_birth_month#42 ASC NULLS FIRST, c_birth_year#43 ASC NULLS FIRST, c_birth_country#44 ASC NULLS FIRST, c_login#45 ASC NULLS FIRST, c_email_address#46 ASC NULLS FIRST, c_last_review_date#47 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, c_preferred_cust_flag#40, c_birth_day#41, c_birth_month#42, c_birth_year#43, c_birth_country#44, c_login#45, c_email_address#46, c_last_review_date#47, ctr_total_return#19] +Input [13]: [c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ctr_total_return#19] +Arguments: 100, [c_customer_id#42 ASC NULLS FIRST, c_salutation#44 ASC NULLS FIRST, c_first_name#45 ASC NULLS FIRST, c_last_name#46 ASC NULLS FIRST, c_preferred_cust_flag#47 ASC NULLS FIRST, c_birth_day#48 ASC NULLS FIRST, c_birth_month#49 ASC NULLS FIRST, c_birth_year#50 ASC NULLS FIRST, c_birth_country#51 ASC NULLS FIRST, c_login#52 ASC NULLS FIRST, c_email_address#53 ASC NULLS FIRST, c_last_review_date#54 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#42, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#48, c_birth_month#49, c_birth_year#50, c_birth_country#51, c_login#52, c_email_address#53, c_last_review_date#54, ctr_total_return#19] ===== Subqueries ===== @@ -342,6 +342,6 @@ ReusedExchange (60) (60) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 24 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 24 Hosting Expression = wr_returned_date_sk#23 IN dynamicpruning#5 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 57b85bc9db340..1aaebad9f1920 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 @@ -150,158 +150,158 @@ Input [3]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18] Condition : isnotnull(ctr_total_return#18) (21) Scan parquet default.web_returns -Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +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#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#22), dynamicpruningexpression(wr_returned_date_sk#22 IN dynamicpruning#5)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct (22) ColumnarToRow [codegen id : 6] -Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +Input [4]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, wr_returned_date_sk#22] (23) Filter [codegen id : 6] -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) +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) (24) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#23] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returned_date_sk#4] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#22] +Right keys [1]: [cast(d_date_sk#23 as bigint)] Join condition: None (26) Project [codegen id : 6] -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] +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#23] (27) ReusedExchange [Reuses operator id: 14] -Output [2]: [ca_address_sk#9, ca_state#10] +Output [2]: [ca_address_sk#24, ca_state#25] (28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wr_returning_addr_sk#2] -Right keys [1]: [cast(ca_address_sk#9 as bigint)] +Left keys [1]: [wr_returning_addr_sk#20] +Right keys [1]: [cast(ca_address_sk#24 as bigint)] Join condition: None (29) Project [codegen id : 6] -Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] -Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#9, ca_state#10] +Output [3]: [wr_returning_customer_sk#19, wr_return_amt#21, ca_state#25] +Input [5]: [wr_returning_customer_sk#19, wr_returning_addr_sk#20, wr_return_amt#21, ca_address_sk#24, ca_state#25] (30) HashAggregate [codegen id : 6] -Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#10] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [wr_returning_customer_sk#1, ca_state#10, sum#20] +Input [3]: [wr_returning_customer_sk#19, wr_return_amt#21, ca_state#25] +Keys [2]: [wr_returning_customer_sk#19, ca_state#25] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#21))] +Aggregate Attributes [1]: [sum#26] +Results [3]: [wr_returning_customer_sk#19, ca_state#25, sum#27] (31) Exchange -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#20] -Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [3]: [wr_returning_customer_sk#19, ca_state#25, sum#27] +Arguments: hashpartitioning(wr_returning_customer_sk#19, ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#28] (32) HashAggregate [codegen id : 7] -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#20] -Keys [2]: [wr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#22] -Results [2]: [ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#22,17,2) AS ctr_total_return#18] +Input [3]: [wr_returning_customer_sk#19, ca_state#25, sum#27] +Keys [2]: [wr_returning_customer_sk#19, ca_state#25] +Functions [1]: [sum(UnscaledValue(wr_return_amt#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#21))#29] +Results [2]: [ca_state#25 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(wr_return_amt#21))#29,17,2) AS ctr_total_return#18] (33) HashAggregate [codegen id : 7] Input [2]: [ctr_state#17, ctr_total_return#18] Keys [1]: [ctr_state#17] Functions [1]: [partial_avg(ctr_total_return#18)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ctr_state#17, sum#25, count#26] +Aggregate Attributes [2]: [sum#30, count#31] +Results [3]: [ctr_state#17, sum#32, count#33] (34) Exchange -Input [3]: [ctr_state#17, sum#25, count#26] -Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ctr_state#17, sum#32, count#33] +Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#34] (35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#17, sum#25, count#26] +Input [3]: [ctr_state#17, sum#32, count#33] Keys [1]: [ctr_state#17] Functions [1]: [avg(ctr_total_return#18)] -Aggregate Attributes [1]: [avg(ctr_total_return#18)#28] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#28) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#29, ctr_state#17 AS ctr_state#17#30] +Aggregate Attributes [1]: [avg(ctr_total_return#18)#35] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#35) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#36, ctr_state#17 AS ctr_state#17#37] (36) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) +Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#36) (37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#31] +Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#38] (38) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#17] -Right keys [1]: [ctr_state#17#30] -Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) +Right keys [1]: [ctr_state#17#37] +Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#36) (39) Project [codegen id : 11] Output [2]: [ctr_customer_sk#16, ctr_total_return#18] -Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] +Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] (40) Scan parquet default.customer -Output [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +Output [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +Input [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] (42) Filter [codegen id : 9] -Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34)) +Input [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] +Condition : (isnotnull(c_customer_sk#39) AND isnotnull(c_current_addr_sk#41)) (43) BroadcastExchange -Input [14]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#46] +Input [14]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#53] (44) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#16] -Right keys [1]: [cast(c_customer_sk#32 as bigint)] +Right keys [1]: [cast(c_customer_sk#39 as bigint)] Join condition: None (45) Project [codegen id : 11] -Output [14]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] -Input [16]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45] +Output [14]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] +Input [16]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52] (46) Scan parquet default.customer_address -Output [2]: [ca_address_sk#9, ca_state#10] +Output [2]: [ca_address_sk#54, ca_state#55] 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 (47) ColumnarToRow [codegen id : 10] -Input [2]: [ca_address_sk#9, ca_state#10] +Input [2]: [ca_address_sk#54, ca_state#55] (48) Filter [codegen id : 10] -Input [2]: [ca_address_sk#9, ca_state#10] -Condition : ((isnotnull(ca_state#10) AND (ca_state#10 = GA)) AND isnotnull(ca_address_sk#9)) +Input [2]: [ca_address_sk#54, ca_state#55] +Condition : ((isnotnull(ca_state#55) AND (ca_state#55 = GA)) AND isnotnull(ca_address_sk#54)) (49) Project [codegen id : 10] -Output [1]: [ca_address_sk#9] -Input [2]: [ca_address_sk#9, ca_state#10] +Output [1]: [ca_address_sk#54] +Input [2]: [ca_address_sk#54, ca_state#55] (50) BroadcastExchange -Input [1]: [ca_address_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] +Input [1]: [ca_address_sk#54] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] (51) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#34] -Right keys [1]: [ca_address_sk#9] +Left keys [1]: [c_current_addr_sk#41] +Right keys [1]: [ca_address_sk#54] Join condition: None (52) Project [codegen id : 11] -Output [13]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#18] -Input [15]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ca_address_sk#9] +Output [13]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ctr_total_return#18] +Input [15]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ca_address_sk#54] (53) TakeOrderedAndProject -Input [13]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#18] -Arguments: 100, [c_customer_id#33 ASC NULLS FIRST, c_salutation#35 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, c_last_name#37 ASC NULLS FIRST, c_preferred_cust_flag#38 ASC NULLS FIRST, c_birth_day#39 ASC NULLS FIRST, c_birth_month#40 ASC NULLS FIRST, c_birth_year#41 ASC NULLS FIRST, c_birth_country#42 ASC NULLS FIRST, c_login#43 ASC NULLS FIRST, c_email_address#44 ASC NULLS FIRST, c_last_review_date#45 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_day#39, c_birth_month#40, c_birth_year#41, c_birth_country#42, c_login#43, c_email_address#44, c_last_review_date#45, ctr_total_return#18] +Input [13]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ctr_total_return#18] +Arguments: 100, [c_customer_id#40 ASC NULLS FIRST, c_salutation#42 ASC NULLS FIRST, c_first_name#43 ASC NULLS FIRST, c_last_name#44 ASC NULLS FIRST, c_preferred_cust_flag#45 ASC NULLS FIRST, c_birth_day#46 ASC NULLS FIRST, c_birth_month#47 ASC NULLS FIRST, c_birth_year#48 ASC NULLS FIRST, c_birth_country#49 ASC NULLS FIRST, c_login#50 ASC NULLS FIRST, c_email_address#51 ASC NULLS FIRST, c_last_review_date#52 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_day#46, c_birth_month#47, c_birth_year#48, c_birth_country#49, c_login#50, c_email_address#51, c_last_review_date#52, ctr_total_return#18] ===== Subqueries ===== @@ -312,6 +312,6 @@ ReusedExchange (54) (54) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 21 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 21 Hosting Expression = wr_returned_date_sk#22 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 61aa5ea5e7e5c..1703da84e7914 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 @@ -220,452 +220,452 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#16] Results [2]: [ca_county#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#16,17,2) AS store_sales#17] (22) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#18)] +PartitionFilters: [isnotnull(ss_sold_date_sk#20), dynamicpruningexpression(ss_sold_date_sk#20 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 8] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20] (24) Filter [codegen id : 8] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) +Input [3]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20] +Condition : isnotnull(ss_addr_sk#18) (25) Scan parquet default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_qoy#21] +Output [3]: [d_date_sk#22, d_year#23, d_qoy#24] 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 (26) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] +Input [3]: [d_date_sk#22, d_year#23, d_qoy#24] (27) Filter [codegen id : 7] -Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] -Condition : ((((isnotnull(d_qoy#21) AND isnotnull(d_year#20)) AND (d_qoy#21 = 3)) AND (d_year#20 = 2000)) AND isnotnull(d_date_sk#19)) +Input [3]: [d_date_sk#22, d_year#23, d_qoy#24] +Condition : ((((isnotnull(d_qoy#24) AND isnotnull(d_year#23)) AND (d_qoy#24 = 3)) AND (d_year#23 = 2000)) AND isnotnull(d_date_sk#22)) (28) BroadcastExchange -Input [3]: [d_date_sk#19, d_year#20, d_qoy#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +Input [3]: [d_date_sk#22, d_year#23, d_qoy#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] (29) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#19] +Left keys [1]: [ss_sold_date_sk#20] +Right keys [1]: [d_date_sk#22] Join condition: None (30) Project [codegen id : 8] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#19, d_year#20, d_qoy#21] +Output [4]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24] +Input [6]: [ss_addr_sk#18, ss_ext_sales_price#19, ss_sold_date_sk#20, d_date_sk#22, d_year#23, d_qoy#24] (31) Exchange -Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21] -Arguments: hashpartitioning(ss_addr_sk#1, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [4]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24] +Arguments: hashpartitioning(ss_addr_sk#18, 5), ENSURE_REQUIREMENTS, [id=#26] (32) Sort [codegen id : 9] -Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21] -Arguments: [ss_addr_sk#1 ASC NULLS FIRST], false, 0 +Input [4]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24] +Arguments: [ss_addr_sk#18 ASC NULLS FIRST], false, 0 (33) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#24, ca_county#25] +Output [2]: [ca_address_sk#27, ca_county#28] (34) Sort [codegen id : 11] -Input [2]: [ca_address_sk#24, ca_county#25] -Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#27, ca_county#28] +Arguments: [ca_address_sk#27 ASC NULLS FIRST], false, 0 (35) SortMergeJoin [codegen id : 12] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [ss_addr_sk#18] +Right keys [1]: [ca_address_sk#27] Join condition: None (36) Project [codegen id : 12] -Output [4]: [ss_ext_sales_price#2, d_year#20, d_qoy#21, ca_county#25] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#20, d_qoy#21, ca_address_sk#24, ca_county#25] +Output [4]: [ss_ext_sales_price#19, d_year#23, d_qoy#24, ca_county#28] +Input [6]: [ss_addr_sk#18, ss_ext_sales_price#19, d_year#23, d_qoy#24, ca_address_sk#27, ca_county#28] (37) HashAggregate [codegen id : 12] -Input [4]: [ss_ext_sales_price#2, d_year#20, d_qoy#21, ca_county#25] -Keys [3]: [ca_county#25, d_qoy#21, d_year#20] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#26] -Results [4]: [ca_county#25, d_qoy#21, d_year#20, sum#27] +Input [4]: [ss_ext_sales_price#19, d_year#23, d_qoy#24, ca_county#28] +Keys [3]: [ca_county#28, d_qoy#24, d_year#23] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#19))] +Aggregate Attributes [1]: [sum#29] +Results [4]: [ca_county#28, d_qoy#24, d_year#23, sum#30] (38) Exchange -Input [4]: [ca_county#25, d_qoy#21, d_year#20, sum#27] -Arguments: hashpartitioning(ca_county#25, d_qoy#21, d_year#20, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [4]: [ca_county#28, d_qoy#24, d_year#23, sum#30] +Arguments: hashpartitioning(ca_county#28, d_qoy#24, d_year#23, 5), ENSURE_REQUIREMENTS, [id=#31] (39) HashAggregate [codegen id : 13] -Input [4]: [ca_county#25, d_qoy#21, d_year#20, sum#27] -Keys [3]: [ca_county#25, d_qoy#21, d_year#20] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#29] -Results [2]: [ca_county#25, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#29,17,2) AS store_sales#30] +Input [4]: [ca_county#28, d_qoy#24, d_year#23, sum#30] +Keys [3]: [ca_county#28, d_qoy#24, d_year#23] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#19))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#19))#32] +Results [2]: [ca_county#28, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#19))#32,17,2) AS store_sales#33] (40) BroadcastExchange -Input [2]: [ca_county#25, store_sales#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#31] +Input [2]: [ca_county#28, store_sales#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] (41) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ca_county#11] -Right keys [1]: [ca_county#25] +Right keys [1]: [ca_county#28] Join condition: None (42) Project [codegen id : 42] -Output [3]: [ca_county#11, store_sales#17, store_sales#30] -Input [4]: [ca_county#11, store_sales#17, ca_county#25, store_sales#30] +Output [3]: [ca_county#11, store_sales#17, store_sales#33] +Input [4]: [ca_county#11, store_sales#17, ca_county#28, store_sales#33] (43) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(ss_sold_date_sk#37), dynamicpruningexpression(ss_sold_date_sk#37 IN dynamicpruning#38)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 15] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37] (45) Filter [codegen id : 15] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) +Input [3]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37] +Condition : isnotnull(ss_addr_sk#35) (46) Scan parquet default.date_dim -Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Output [3]: [d_date_sk#39, d_year#40, d_qoy#41] 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 (47) ColumnarToRow [codegen id : 14] -Input [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] (48) Filter [codegen id : 14] -Input [3]: [d_date_sk#33, d_year#34, d_qoy#35] -Condition : ((((isnotnull(d_qoy#35) AND isnotnull(d_year#34)) AND (d_qoy#35 = 1)) AND (d_year#34 = 2000)) AND isnotnull(d_date_sk#33)) +Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] +Condition : ((((isnotnull(d_qoy#41) AND isnotnull(d_year#40)) AND (d_qoy#41 = 1)) AND (d_year#40 = 2000)) AND isnotnull(d_date_sk#39)) (49) BroadcastExchange -Input [3]: [d_date_sk#33, d_year#34, d_qoy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#36] +Input [3]: [d_date_sk#39, d_year#40, d_qoy#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#42] (50) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [ss_sold_date_sk#37] +Right keys [1]: [d_date_sk#39] Join condition: None (51) Project [codegen id : 15] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#33, d_year#34, d_qoy#35] +Output [4]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41] +Input [6]: [ss_addr_sk#35, ss_ext_sales_price#36, ss_sold_date_sk#37, d_date_sk#39, d_year#40, d_qoy#41] (52) Exchange -Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35] -Arguments: hashpartitioning(ss_addr_sk#1, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41] +Arguments: hashpartitioning(ss_addr_sk#35, 5), ENSURE_REQUIREMENTS, [id=#43] (53) Sort [codegen id : 16] -Input [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35] -Arguments: [ss_addr_sk#1 ASC NULLS FIRST], false, 0 +Input [4]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41] +Arguments: [ss_addr_sk#35 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#38, ca_county#39] +Output [2]: [ca_address_sk#44, ca_county#45] (55) Sort [codegen id : 18] -Input [2]: [ca_address_sk#38, ca_county#39] -Arguments: [ca_address_sk#38 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#44, ca_county#45] +Arguments: [ca_address_sk#44 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 19] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#38] +Left keys [1]: [ss_addr_sk#35] +Right keys [1]: [ca_address_sk#44] Join condition: None (57) Project [codegen id : 19] -Output [4]: [ss_ext_sales_price#2, d_year#34, d_qoy#35, ca_county#39] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#34, d_qoy#35, ca_address_sk#38, ca_county#39] +Output [4]: [ss_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#45] +Input [6]: [ss_addr_sk#35, ss_ext_sales_price#36, d_year#40, d_qoy#41, ca_address_sk#44, ca_county#45] (58) HashAggregate [codegen id : 19] -Input [4]: [ss_ext_sales_price#2, d_year#34, d_qoy#35, ca_county#39] -Keys [3]: [ca_county#39, d_qoy#35, d_year#34] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#40] -Results [4]: [ca_county#39, d_qoy#35, d_year#34, sum#41] +Input [4]: [ss_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#45] +Keys [3]: [ca_county#45, d_qoy#41, d_year#40] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#36))] +Aggregate Attributes [1]: [sum#46] +Results [4]: [ca_county#45, d_qoy#41, d_year#40, sum#47] (59) Exchange -Input [4]: [ca_county#39, d_qoy#35, d_year#34, sum#41] -Arguments: hashpartitioning(ca_county#39, d_qoy#35, d_year#34, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [4]: [ca_county#45, d_qoy#41, d_year#40, sum#47] +Arguments: hashpartitioning(ca_county#45, d_qoy#41, d_year#40, 5), ENSURE_REQUIREMENTS, [id=#48] (60) HashAggregate [codegen id : 20] -Input [4]: [ca_county#39, d_qoy#35, d_year#34, sum#41] -Keys [3]: [ca_county#39, d_qoy#35, d_year#34] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#43] -Results [3]: [ca_county#39, d_year#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#43,17,2) AS store_sales#44] +Input [4]: [ca_county#45, d_qoy#41, d_year#40, sum#47] +Keys [3]: [ca_county#45, d_qoy#41, d_year#40] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#36))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#36))#49] +Results [3]: [ca_county#45, d_year#40, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#36))#49,17,2) AS store_sales#50] (61) BroadcastExchange -Input [3]: [ca_county#39, d_year#34, store_sales#44] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#45] +Input [3]: [ca_county#45, d_year#40, store_sales#50] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#51] (62) BroadcastHashJoin [codegen id : 42] Left keys [1]: [ca_county#11] -Right keys [1]: [ca_county#39] +Right keys [1]: [ca_county#45] Join condition: None (63) Project [codegen id : 42] -Output [5]: [store_sales#17, store_sales#30, ca_county#39, d_year#34, store_sales#44] -Input [6]: [ca_county#11, store_sales#17, store_sales#30, ca_county#39, d_year#34, store_sales#44] +Output [5]: [store_sales#17, store_sales#33, ca_county#45, d_year#40, store_sales#50] +Input [6]: [ca_county#11, store_sales#17, store_sales#33, ca_county#45, d_year#40, store_sales#50] (64) 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#52, ws_ext_sales_price#53, ws_sold_date_sk#54] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#38)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Input [3]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, ws_sold_date_sk#54] (66) Filter [codegen id : 22] -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#52, ws_ext_sales_price#53, ws_sold_date_sk#54] +Condition : isnotnull(ws_bill_addr_sk#52) (67) ReusedExchange [Reuses operator id: 49] -Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] +Output [3]: [d_date_sk#55, d_year#56, d_qoy#57] (68) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#48] -Right keys [1]: [d_date_sk#49] +Left keys [1]: [ws_sold_date_sk#54] +Right keys [1]: [d_date_sk#55] Join condition: None (69) Project [codegen id : 22] -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#52, ws_ext_sales_price#53, d_year#56, d_qoy#57] +Input [6]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, ws_sold_date_sk#54, d_date_sk#55, d_year#56, d_qoy#57] (70) Exchange -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51] -Arguments: hashpartitioning(ws_bill_addr_sk#46, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [4]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57] +Arguments: hashpartitioning(ws_bill_addr_sk#52, 5), ENSURE_REQUIREMENTS, [id=#58] (71) Sort [codegen id : 23] -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51] -Arguments: [ws_bill_addr_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57] +Arguments: [ws_bill_addr_sk#52 ASC NULLS FIRST], false, 0 (72) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#53, ca_county#54] +Output [2]: [ca_address_sk#59, ca_county#60] (73) Sort [codegen id : 25] -Input [2]: [ca_address_sk#53, ca_county#54] -Arguments: [ca_address_sk#53 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#59, ca_county#60] +Arguments: [ca_address_sk#59 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 26] -Left keys [1]: [ws_bill_addr_sk#46] -Right keys [1]: [ca_address_sk#53] +Left keys [1]: [ws_bill_addr_sk#52] +Right keys [1]: [ca_address_sk#59] Join condition: None (75) Project [codegen id : 26] -Output [4]: [ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_county#54] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_address_sk#53, ca_county#54] +Output [4]: [ws_ext_sales_price#53, d_year#56, d_qoy#57, ca_county#60] +Input [6]: [ws_bill_addr_sk#52, ws_ext_sales_price#53, d_year#56, d_qoy#57, ca_address_sk#59, ca_county#60] (76) HashAggregate [codegen id : 26] -Input [4]: [ws_ext_sales_price#47, d_year#50, d_qoy#51, ca_county#54] -Keys [3]: [ca_county#54, d_qoy#51, d_year#50] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum#55] -Results [4]: [ca_county#54, d_qoy#51, d_year#50, sum#56] +Input [4]: [ws_ext_sales_price#53, d_year#56, d_qoy#57, ca_county#60] +Keys [3]: [ca_county#60, d_qoy#57, d_year#56] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#53))] +Aggregate Attributes [1]: [sum#61] +Results [4]: [ca_county#60, d_qoy#57, d_year#56, sum#62] (77) Exchange -Input [4]: [ca_county#54, d_qoy#51, d_year#50, sum#56] -Arguments: hashpartitioning(ca_county#54, d_qoy#51, d_year#50, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [4]: [ca_county#60, d_qoy#57, d_year#56, sum#62] +Arguments: hashpartitioning(ca_county#60, d_qoy#57, d_year#56, 5), ENSURE_REQUIREMENTS, [id=#63] (78) HashAggregate [codegen id : 41] -Input [4]: [ca_county#54, d_qoy#51, d_year#50, sum#56] -Keys [3]: [ca_county#54, 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))#58] -Results [2]: [ca_county#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#47))#58,17,2) AS web_sales#59] +Input [4]: [ca_county#60, d_qoy#57, d_year#56, sum#62] +Keys [3]: [ca_county#60, d_qoy#57, d_year#56] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#53))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#53))#64] +Results [2]: [ca_county#60, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#53))#64,17,2) AS web_sales#65] (79) 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#66, ws_ext_sales_price#67, ws_sold_date_sk#68] 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#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (80) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Input [3]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, ws_sold_date_sk#68] (81) Filter [codegen id : 28] -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#66, ws_ext_sales_price#67, ws_sold_date_sk#68] +Condition : isnotnull(ws_bill_addr_sk#66) (82) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#60, d_year#61, d_qoy#62] +Output [3]: [d_date_sk#69, d_year#70, d_qoy#71] (83) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#48] -Right keys [1]: [d_date_sk#60] +Left keys [1]: [ws_sold_date_sk#68] +Right keys [1]: [d_date_sk#69] Join condition: None (84) Project [codegen id : 28] -Output [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48, d_date_sk#60, d_year#61, d_qoy#62] +Output [4]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71] +Input [6]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, ws_sold_date_sk#68, d_date_sk#69, d_year#70, d_qoy#71] (85) Exchange -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62] -Arguments: hashpartitioning(ws_bill_addr_sk#46, 5), ENSURE_REQUIREMENTS, [id=#63] +Input [4]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71] +Arguments: hashpartitioning(ws_bill_addr_sk#66, 5), ENSURE_REQUIREMENTS, [id=#72] (86) Sort [codegen id : 29] -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62] -Arguments: [ws_bill_addr_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71] +Arguments: [ws_bill_addr_sk#66 ASC NULLS FIRST], false, 0 (87) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#64, ca_county#65] +Output [2]: [ca_address_sk#73, ca_county#74] (88) Sort [codegen id : 31] -Input [2]: [ca_address_sk#64, ca_county#65] -Arguments: [ca_address_sk#64 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#73, ca_county#74] +Arguments: [ca_address_sk#73 ASC NULLS FIRST], false, 0 (89) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_addr_sk#46] -Right keys [1]: [ca_address_sk#64] +Left keys [1]: [ws_bill_addr_sk#66] +Right keys [1]: [ca_address_sk#73] Join condition: None (90) Project [codegen id : 32] -Output [4]: [ws_ext_sales_price#47, d_year#61, d_qoy#62, ca_county#65] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#61, d_qoy#62, ca_address_sk#64, ca_county#65] +Output [4]: [ws_ext_sales_price#67, d_year#70, d_qoy#71, ca_county#74] +Input [6]: [ws_bill_addr_sk#66, ws_ext_sales_price#67, d_year#70, d_qoy#71, ca_address_sk#73, ca_county#74] (91) HashAggregate [codegen id : 32] -Input [4]: [ws_ext_sales_price#47, d_year#61, d_qoy#62, ca_county#65] -Keys [3]: [ca_county#65, d_qoy#62, d_year#61] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum#66] -Results [4]: [ca_county#65, d_qoy#62, d_year#61, sum#67] +Input [4]: [ws_ext_sales_price#67, d_year#70, d_qoy#71, ca_county#74] +Keys [3]: [ca_county#74, d_qoy#71, d_year#70] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#67))] +Aggregate Attributes [1]: [sum#75] +Results [4]: [ca_county#74, d_qoy#71, d_year#70, sum#76] (92) Exchange -Input [4]: [ca_county#65, d_qoy#62, d_year#61, sum#67] -Arguments: hashpartitioning(ca_county#65, d_qoy#62, d_year#61, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [4]: [ca_county#74, d_qoy#71, d_year#70, sum#76] +Arguments: hashpartitioning(ca_county#74, d_qoy#71, d_year#70, 5), ENSURE_REQUIREMENTS, [id=#77] (93) HashAggregate [codegen id : 33] -Input [4]: [ca_county#65, d_qoy#62, d_year#61, sum#67] -Keys [3]: [ca_county#65, d_qoy#62, d_year#61] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#47))#69] -Results [2]: [ca_county#65, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#47))#69,17,2) AS web_sales#70] +Input [4]: [ca_county#74, d_qoy#71, d_year#70, sum#76] +Keys [3]: [ca_county#74, d_qoy#71, d_year#70] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#67))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#67))#78] +Results [2]: [ca_county#74, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#67))#78,17,2) AS web_sales#79] (94) BroadcastExchange -Input [2]: [ca_county#65, web_sales#70] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#71] +Input [2]: [ca_county#74, web_sales#79] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#80] (95) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#54] -Right keys [1]: [ca_county#65] +Left keys [1]: [ca_county#60] +Right keys [1]: [ca_county#74] Join condition: None (96) Project [codegen id : 41] -Output [3]: [ca_county#54, web_sales#59, web_sales#70] -Input [4]: [ca_county#54, web_sales#59, ca_county#65, web_sales#70] +Output [3]: [ca_county#60, web_sales#65, web_sales#79] +Input [4]: [ca_county#60, web_sales#65, ca_county#74, web_sales#79] (97) 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#81, ws_ext_sales_price#82, ws_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#48), dynamicpruningexpression(ws_sold_date_sk#48 IN dynamicpruning#18)] +PartitionFilters: [isnotnull(ws_sold_date_sk#83), dynamicpruningexpression(ws_sold_date_sk#83 IN dynamicpruning#21)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (98) ColumnarToRow [codegen id : 35] -Input [3]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48] +Input [3]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, ws_sold_date_sk#83] (99) Filter [codegen id : 35] -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#81, ws_ext_sales_price#82, ws_sold_date_sk#83] +Condition : isnotnull(ws_bill_addr_sk#81) (100) ReusedExchange [Reuses operator id: 28] -Output [3]: [d_date_sk#72, d_year#73, d_qoy#74] +Output [3]: [d_date_sk#84, d_year#85, d_qoy#86] (101) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ws_sold_date_sk#48] -Right keys [1]: [d_date_sk#72] +Left keys [1]: [ws_sold_date_sk#83] +Right keys [1]: [d_date_sk#84] Join condition: None (102) Project [codegen id : 35] -Output [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, ws_sold_date_sk#48, d_date_sk#72, d_year#73, d_qoy#74] +Output [4]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86] +Input [6]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, ws_sold_date_sk#83, d_date_sk#84, d_year#85, d_qoy#86] (103) Exchange -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74] -Arguments: hashpartitioning(ws_bill_addr_sk#46, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [4]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86] +Arguments: hashpartitioning(ws_bill_addr_sk#81, 5), ENSURE_REQUIREMENTS, [id=#87] (104) Sort [codegen id : 36] -Input [4]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74] -Arguments: [ws_bill_addr_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86] +Arguments: [ws_bill_addr_sk#81 ASC NULLS FIRST], false, 0 (105) ReusedExchange [Reuses operator id: 15] -Output [2]: [ca_address_sk#76, ca_county#77] +Output [2]: [ca_address_sk#88, ca_county#89] (106) Sort [codegen id : 38] -Input [2]: [ca_address_sk#76, ca_county#77] -Arguments: [ca_address_sk#76 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#88, ca_county#89] +Arguments: [ca_address_sk#88 ASC NULLS FIRST], false, 0 (107) SortMergeJoin [codegen id : 39] -Left keys [1]: [ws_bill_addr_sk#46] -Right keys [1]: [ca_address_sk#76] +Left keys [1]: [ws_bill_addr_sk#81] +Right keys [1]: [ca_address_sk#88] Join condition: None (108) Project [codegen id : 39] -Output [4]: [ws_ext_sales_price#47, d_year#73, d_qoy#74, ca_county#77] -Input [6]: [ws_bill_addr_sk#46, ws_ext_sales_price#47, d_year#73, d_qoy#74, ca_address_sk#76, ca_county#77] +Output [4]: [ws_ext_sales_price#82, d_year#85, d_qoy#86, ca_county#89] +Input [6]: [ws_bill_addr_sk#81, ws_ext_sales_price#82, d_year#85, d_qoy#86, ca_address_sk#88, ca_county#89] (109) HashAggregate [codegen id : 39] -Input [4]: [ws_ext_sales_price#47, d_year#73, d_qoy#74, ca_county#77] -Keys [3]: [ca_county#77, d_qoy#74, d_year#73] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum#78] -Results [4]: [ca_county#77, d_qoy#74, d_year#73, sum#79] +Input [4]: [ws_ext_sales_price#82, d_year#85, d_qoy#86, ca_county#89] +Keys [3]: [ca_county#89, d_qoy#86, d_year#85] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#82))] +Aggregate Attributes [1]: [sum#90] +Results [4]: [ca_county#89, d_qoy#86, d_year#85, sum#91] (110) Exchange -Input [4]: [ca_county#77, d_qoy#74, d_year#73, sum#79] -Arguments: hashpartitioning(ca_county#77, d_qoy#74, d_year#73, 5), ENSURE_REQUIREMENTS, [id=#80] +Input [4]: [ca_county#89, d_qoy#86, d_year#85, sum#91] +Arguments: hashpartitioning(ca_county#89, d_qoy#86, d_year#85, 5), ENSURE_REQUIREMENTS, [id=#92] (111) HashAggregate [codegen id : 40] -Input [4]: [ca_county#77, d_qoy#74, d_year#73, sum#79] -Keys [3]: [ca_county#77, d_qoy#74, d_year#73] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#47))#81] -Results [2]: [ca_county#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#47))#81,17,2) AS web_sales#82] +Input [4]: [ca_county#89, d_qoy#86, d_year#85, sum#91] +Keys [3]: [ca_county#89, d_qoy#86, d_year#85] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#82))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#82))#93] +Results [2]: [ca_county#89, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#82))#93,17,2) AS web_sales#94] (112) BroadcastExchange -Input [2]: [ca_county#77, web_sales#82] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#83] +Input [2]: [ca_county#89, web_sales#94] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#95] (113) BroadcastHashJoin [codegen id : 41] -Left keys [1]: [ca_county#54] -Right keys [1]: [ca_county#77] +Left keys [1]: [ca_county#60] +Right keys [1]: [ca_county#89] Join condition: None (114) Project [codegen id : 41] -Output [4]: [ca_county#54, web_sales#59, web_sales#70, web_sales#82] -Input [5]: [ca_county#54, web_sales#59, web_sales#70, ca_county#77, web_sales#82] +Output [4]: [ca_county#60, web_sales#65, web_sales#79, web_sales#94] +Input [5]: [ca_county#60, web_sales#65, web_sales#79, ca_county#89, web_sales#94] (115) BroadcastExchange -Input [4]: [ca_county#54, web_sales#59, web_sales#70, web_sales#82] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#84] +Input [4]: [ca_county#60, web_sales#65, web_sales#79, web_sales#94] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#96] (116) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ca_county#39] -Right keys [1]: [ca_county#54] -Join condition: ((CASE WHEN (web_sales#59 > 0.00) THEN CheckOverflow((promote_precision(web_sales#70) / promote_precision(web_sales#59)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#44 > 0.00) THEN CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#44)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#70 > 0.00) THEN CheckOverflow((promote_precision(web_sales#82) / promote_precision(web_sales#70)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#17 > 0.00) THEN CheckOverflow((promote_precision(store_sales#30) / promote_precision(store_sales#17)), DecimalType(37,20), true) ELSE null END)) +Left keys [1]: [ca_county#45] +Right keys [1]: [ca_county#60] +Join condition: ((CASE WHEN (web_sales#65 > 0.00) THEN CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#65)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#50 > 0.00) THEN CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#50)), DecimalType(37,20), true) ELSE null END) AND (CASE WHEN (web_sales#79 > 0.00) THEN CheckOverflow((promote_precision(web_sales#94) / promote_precision(web_sales#79)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#17 > 0.00) THEN CheckOverflow((promote_precision(store_sales#33) / promote_precision(store_sales#17)), DecimalType(37,20), true) ELSE null END)) (117) Project [codegen id : 42] -Output [6]: [ca_county#39, d_year#34, CheckOverflow((promote_precision(web_sales#70) / promote_precision(web_sales#59)), DecimalType(37,20), true) AS web_q1_q2_increase#85, CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#44)), DecimalType(37,20), true) AS store_q1_q2_increase#86, CheckOverflow((promote_precision(web_sales#82) / promote_precision(web_sales#70)), DecimalType(37,20), true) AS web_q2_q3_increase#87, CheckOverflow((promote_precision(store_sales#30) / promote_precision(store_sales#17)), DecimalType(37,20), true) AS store_q2_q3_increase#88] -Input [9]: [store_sales#17, store_sales#30, ca_county#39, d_year#34, store_sales#44, ca_county#54, web_sales#59, web_sales#70, web_sales#82] +Output [6]: [ca_county#45, d_year#40, CheckOverflow((promote_precision(web_sales#79) / promote_precision(web_sales#65)), DecimalType(37,20), true) AS web_q1_q2_increase#97, CheckOverflow((promote_precision(store_sales#17) / promote_precision(store_sales#50)), DecimalType(37,20), true) AS store_q1_q2_increase#98, CheckOverflow((promote_precision(web_sales#94) / promote_precision(web_sales#79)), DecimalType(37,20), true) AS web_q2_q3_increase#99, CheckOverflow((promote_precision(store_sales#33) / promote_precision(store_sales#17)), DecimalType(37,20), true) AS store_q2_q3_increase#100] +Input [9]: [store_sales#17, store_sales#33, ca_county#45, d_year#40, store_sales#50, ca_county#60, web_sales#65, web_sales#79, web_sales#94] (118) Exchange -Input [6]: [ca_county#39, d_year#34, web_q1_q2_increase#85, store_q1_q2_increase#86, web_q2_q3_increase#87, store_q2_q3_increase#88] -Arguments: rangepartitioning(ca_county#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#89] +Input [6]: [ca_county#45, d_year#40, web_q1_q2_increase#97, store_q1_q2_increase#98, web_q2_q3_increase#99, store_q2_q3_increase#100] +Arguments: rangepartitioning(ca_county#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#101] (119) Sort [codegen id : 43] -Input [6]: [ca_county#39, d_year#34, web_q1_q2_increase#85, store_q1_q2_increase#86, web_q2_q3_increase#87, store_q2_q3_increase#88] -Arguments: [ca_county#39 ASC NULLS FIRST], true, 0 +Input [6]: [ca_county#45, d_year#40, web_q1_q2_increase#97, store_q1_q2_increase#98, web_q2_q3_increase#99, store_q2_q3_increase#100] +Arguments: [ca_county#45 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -676,24 +676,24 @@ ReusedExchange (120) (120) ReusedExchange [Reuses operator id: 7] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#18 +Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#20 IN dynamicpruning#21 ReusedExchange (121) (121) ReusedExchange [Reuses operator id: 28] -Output [3]: [d_date_sk#19, d_year#20, d_qoy#21] +Output [3]: [d_date_sk#22, d_year#23, d_qoy#24] -Subquery:3 Hosting operator id = 43 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#32 +Subquery:3 Hosting operator id = 43 Hosting Expression = ss_sold_date_sk#37 IN dynamicpruning#38 ReusedExchange (122) (122) ReusedExchange [Reuses operator id: 49] -Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] +Output [3]: [d_date_sk#39, d_year#40, d_qoy#41] -Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#32 +Subquery:4 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#38 -Subquery:5 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#4 +Subquery:5 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#4 -Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#48 IN dynamicpruning#18 +Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#83 IN dynamicpruning#21 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 81a4a7fed99b2..f47a7af5a15e3 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 @@ -188,383 +188,383 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#15] Results [3]: [ca_county#10, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#15,17,2) AS store_sales#16] (19) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(ss_sold_date_sk#19), dynamicpruningexpression(ss_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (20) ColumnarToRow [codegen id : 6] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19] (21) Filter [codegen id : 6] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) +Input [3]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19] +Condition : isnotnull(ss_addr_sk#17) (22) Scan parquet default.date_dim -Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 4] -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] (24) Filter [codegen id : 4] -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) +Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] +Condition : ((((isnotnull(d_qoy#23) AND isnotnull(d_year#22)) AND (d_qoy#23 = 2)) AND (d_year#22 = 2000)) AND isnotnull(d_date_sk#21)) (25) BroadcastExchange -Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Input [3]: [d_date_sk#21, d_year#22, d_qoy#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] (26) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#18] +Left keys [1]: [ss_sold_date_sk#19] +Right keys [1]: [d_date_sk#21] Join condition: None (27) Project [codegen id : 6] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#19, d_qoy#20] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#18, d_year#19, d_qoy#20] +Output [4]: [ss_addr_sk#17, ss_ext_sales_price#18, d_year#22, d_qoy#23] +Input [6]: [ss_addr_sk#17, ss_ext_sales_price#18, ss_sold_date_sk#19, d_date_sk#21, d_year#22, d_qoy#23] (28) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#22, ca_county#23] +Output [2]: [ca_address_sk#25, ca_county#26] (29) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#22] +Left keys [1]: [ss_addr_sk#17] +Right keys [1]: [ca_address_sk#25] Join condition: None (30) Project [codegen id : 6] -Output [4]: [ss_ext_sales_price#2, d_year#19, d_qoy#20, ca_county#23] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#19, d_qoy#20, ca_address_sk#22, ca_county#23] +Output [4]: [ss_ext_sales_price#18, d_year#22, d_qoy#23, ca_county#26] +Input [6]: [ss_addr_sk#17, ss_ext_sales_price#18, d_year#22, d_qoy#23, ca_address_sk#25, ca_county#26] (31) HashAggregate [codegen id : 6] -Input [4]: [ss_ext_sales_price#2, d_year#19, d_qoy#20, ca_county#23] -Keys [3]: [ca_county#23, d_qoy#20, d_year#19] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#24] -Results [4]: [ca_county#23, d_qoy#20, d_year#19, sum#25] +Input [4]: [ss_ext_sales_price#18, d_year#22, d_qoy#23, ca_county#26] +Keys [3]: [ca_county#26, d_qoy#23, d_year#22] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#18))] +Aggregate Attributes [1]: [sum#27] +Results [4]: [ca_county#26, d_qoy#23, d_year#22, sum#28] (32) Exchange -Input [4]: [ca_county#23, d_qoy#20, d_year#19, sum#25] -Arguments: hashpartitioning(ca_county#23, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [4]: [ca_county#26, d_qoy#23, d_year#22, sum#28] +Arguments: hashpartitioning(ca_county#26, d_qoy#23, d_year#22, 5), ENSURE_REQUIREMENTS, [id=#29] (33) HashAggregate [codegen id : 7] -Input [4]: [ca_county#23, d_qoy#20, d_year#19, sum#25] -Keys [3]: [ca_county#23, d_qoy#20, d_year#19] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#27] -Results [2]: [ca_county#23, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#27,17,2) AS store_sales#28] +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] (34) BroadcastExchange -Input [2]: [ca_county#23, store_sales#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#29] +Input [2]: [ca_county#26, store_sales#31] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#32] (35) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#10] -Right keys [1]: [ca_county#23] +Right keys [1]: [ca_county#26] Join condition: None (36) Scan parquet default.store_sales -Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Output [3]: [ss_addr_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ss_sold_date_sk#35), dynamicpruningexpression(ss_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 10] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Input [3]: [ss_addr_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] (38) Filter [codegen id : 10] -Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_addr_sk#1) +Input [3]: [ss_addr_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] +Condition : isnotnull(ss_addr_sk#33) (39) Scan parquet default.date_dim -Output [3]: [d_date_sk#31, d_year#32, d_qoy#33] +Output [3]: [d_date_sk#37, d_year#38, d_qoy#39] 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 (40) ColumnarToRow [codegen id : 8] -Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] +Input [3]: [d_date_sk#37, d_year#38, d_qoy#39] (41) Filter [codegen id : 8] -Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] -Condition : ((((isnotnull(d_qoy#33) AND isnotnull(d_year#32)) AND (d_qoy#33 = 3)) AND (d_year#32 = 2000)) AND isnotnull(d_date_sk#31)) +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 = 3)) AND (d_year#38 = 2000)) AND isnotnull(d_date_sk#37)) (42) BroadcastExchange -Input [3]: [d_date_sk#31, d_year#32, d_qoy#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [3]: [d_date_sk#37, d_year#38, d_qoy#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] (43) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#31] +Left keys [1]: [ss_sold_date_sk#35] +Right keys [1]: [d_date_sk#37] Join condition: None (44) Project [codegen id : 10] -Output [4]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#32, d_qoy#33] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, d_date_sk#31, d_year#32, d_qoy#33] +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] (45) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#35, ca_county#36] +Output [2]: [ca_address_sk#41, ca_county#42] (46) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_addr_sk#1] -Right keys [1]: [ca_address_sk#35] +Left keys [1]: [ss_addr_sk#33] +Right keys [1]: [ca_address_sk#41] Join condition: None (47) Project [codegen id : 10] -Output [4]: [ss_ext_sales_price#2, d_year#32, d_qoy#33, ca_county#36] -Input [6]: [ss_addr_sk#1, ss_ext_sales_price#2, d_year#32, d_qoy#33, ca_address_sk#35, ca_county#36] +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] (48) HashAggregate [codegen id : 10] -Input [4]: [ss_ext_sales_price#2, d_year#32, d_qoy#33, ca_county#36] -Keys [3]: [ca_county#36, d_qoy#33, d_year#32] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum#37] -Results [4]: [ca_county#36, d_qoy#33, d_year#32, sum#38] +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] (49) Exchange -Input [4]: [ca_county#36, d_qoy#33, d_year#32, sum#38] -Arguments: hashpartitioning(ca_county#36, d_qoy#33, d_year#32, 5), ENSURE_REQUIREMENTS, [id=#39] +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] (50) HashAggregate [codegen id : 11] -Input [4]: [ca_county#36, d_qoy#33, d_year#32, sum#38] -Keys [3]: [ca_county#36, d_qoy#33, d_year#32] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#40] -Results [2]: [ca_county#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#40,17,2) AS store_sales#41] +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 [2]: [ca_county#42, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#46,17,2) AS store_sales#47] (51) BroadcastExchange -Input [2]: [ca_county#36, store_sales#41] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#42] +Input [2]: [ca_county#42, store_sales#47] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#48] (52) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#23] -Right keys [1]: [ca_county#36] +Left keys [1]: [ca_county#26] +Right keys [1]: [ca_county#42] Join condition: None (53) Project [codegen id : 24] -Output [5]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41] -Input [7]: [ca_county#10, d_year#6, store_sales#16, ca_county#23, store_sales#28, ca_county#36, store_sales#41] +Output [5]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47] +Input [7]: [ca_county#10, d_year#6, store_sales#16, ca_county#26, store_sales#31, ca_county#42, store_sales#47] (54) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Output [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 14] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] (56) Filter [codegen id : 14] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_bill_addr_sk#43) +Input [3]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_bill_addr_sk#49) (57) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#46, d_year#47, d_qoy#48] +Output [3]: [d_date_sk#52, d_year#53, d_qoy#54] (58) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#46] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#52] Join condition: None (59) Project [codegen id : 14] -Output [4]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#47, d_qoy#48] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45, d_date_sk#46, d_year#47, d_qoy#48] +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] (60) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#49, ca_county#50] +Output [2]: [ca_address_sk#55, ca_county#56] (61) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_bill_addr_sk#43] -Right keys [1]: [ca_address_sk#49] +Left keys [1]: [ws_bill_addr_sk#49] +Right keys [1]: [ca_address_sk#55] Join condition: None (62) Project [codegen id : 14] -Output [4]: [ws_ext_sales_price#44, d_year#47, d_qoy#48, ca_county#50] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#47, d_qoy#48, ca_address_sk#49, ca_county#50] +Output [4]: [ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_county#56] +Input [6]: [ws_bill_addr_sk#49, ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_address_sk#55, ca_county#56] (63) HashAggregate [codegen id : 14] -Input [4]: [ws_ext_sales_price#44, d_year#47, d_qoy#48, ca_county#50] -Keys [3]: [ca_county#50, d_qoy#48, d_year#47] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum#51] -Results [4]: [ca_county#50, d_qoy#48, d_year#47, sum#52] +Input [4]: [ws_ext_sales_price#50, d_year#53, d_qoy#54, ca_county#56] +Keys [3]: [ca_county#56, d_qoy#54, d_year#53] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#50))] +Aggregate Attributes [1]: [sum#57] +Results [4]: [ca_county#56, d_qoy#54, d_year#53, sum#58] (64) Exchange -Input [4]: [ca_county#50, d_qoy#48, d_year#47, sum#52] -Arguments: hashpartitioning(ca_county#50, d_qoy#48, d_year#47, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [4]: [ca_county#56, d_qoy#54, d_year#53, sum#58] +Arguments: hashpartitioning(ca_county#56, d_qoy#54, d_year#53, 5), ENSURE_REQUIREMENTS, [id=#59] (65) HashAggregate [codegen id : 15] -Input [4]: [ca_county#50, d_qoy#48, d_year#47, sum#52] -Keys [3]: [ca_county#50, d_qoy#48, d_year#47] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#44))#54] -Results [2]: [ca_county#50, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#44))#54,17,2) AS web_sales#55] +Input [4]: [ca_county#56, d_qoy#54, d_year#53, sum#58] +Keys [3]: [ca_county#56, 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))#60] +Results [2]: [ca_county#56, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#50))#60,17,2) AS web_sales#61] (66) BroadcastExchange -Input [2]: [ca_county#50, web_sales#55] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] +Input [2]: [ca_county#56, web_sales#61] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#62] (67) BroadcastHashJoin [codegen id : 24] Left keys [1]: [ca_county#10] -Right keys [1]: [ca_county#50] +Right keys [1]: [ca_county#56] Join condition: None (68) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Output [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#20)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 18] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] (70) Filter [codegen id : 18] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_bill_addr_sk#43) +Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +Condition : isnotnull(ws_bill_addr_sk#63) (71) ReusedExchange [Reuses operator id: 25] -Output [3]: [d_date_sk#57, d_year#58, d_qoy#59] +Output [3]: [d_date_sk#66, d_year#67, d_qoy#68] (72) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#57] +Left keys [1]: [ws_sold_date_sk#65] +Right keys [1]: [d_date_sk#66] Join condition: None (73) Project [codegen id : 18] -Output [4]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#58, d_qoy#59] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45, d_date_sk#57, d_year#58, d_qoy#59] +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] (74) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#60, ca_county#61] +Output [2]: [ca_address_sk#69, ca_county#70] (75) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_bill_addr_sk#43] -Right keys [1]: [ca_address_sk#60] +Left keys [1]: [ws_bill_addr_sk#63] +Right keys [1]: [ca_address_sk#69] Join condition: None (76) Project [codegen id : 18] -Output [4]: [ws_ext_sales_price#44, d_year#58, d_qoy#59, ca_county#61] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#58, d_qoy#59, ca_address_sk#60, ca_county#61] +Output [4]: [ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_county#70] +Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_address_sk#69, ca_county#70] (77) HashAggregate [codegen id : 18] -Input [4]: [ws_ext_sales_price#44, d_year#58, d_qoy#59, ca_county#61] -Keys [3]: [ca_county#61, d_qoy#59, d_year#58] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum#62] -Results [4]: [ca_county#61, d_qoy#59, d_year#58, sum#63] +Input [4]: [ws_ext_sales_price#64, d_year#67, d_qoy#68, ca_county#70] +Keys [3]: [ca_county#70, d_qoy#68, d_year#67] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#64))] +Aggregate Attributes [1]: [sum#71] +Results [4]: [ca_county#70, d_qoy#68, d_year#67, sum#72] (78) Exchange -Input [4]: [ca_county#61, d_qoy#59, d_year#58, sum#63] -Arguments: hashpartitioning(ca_county#61, d_qoy#59, d_year#58, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [4]: [ca_county#70, d_qoy#68, d_year#67, sum#72] +Arguments: hashpartitioning(ca_county#70, d_qoy#68, d_year#67, 5), ENSURE_REQUIREMENTS, [id=#73] (79) HashAggregate [codegen id : 19] -Input [4]: [ca_county#61, d_qoy#59, d_year#58, sum#63] -Keys [3]: [ca_county#61, d_qoy#59, d_year#58] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#44))#65] -Results [2]: [ca_county#61, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#44))#65,17,2) AS web_sales#66] +Input [4]: [ca_county#70, d_qoy#68, d_year#67, sum#72] +Keys [3]: [ca_county#70, 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))#74] +Results [2]: [ca_county#70, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#64))#74,17,2) AS web_sales#75] (80) BroadcastExchange -Input [2]: [ca_county#61, web_sales#66] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#67] +Input [2]: [ca_county#70, web_sales#75] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#76] (81) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#50] -Right keys [1]: [ca_county#61] -Join condition: (CASE WHEN (web_sales#55 > 0.00) THEN CheckOverflow((promote_precision(web_sales#66) / promote_precision(web_sales#55)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#16 > 0.00) THEN CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#16)), DecimalType(37,20), true) ELSE null END) +Left keys [1]: [ca_county#56] +Right keys [1]: [ca_county#70] +Join condition: (CASE WHEN (web_sales#61 > 0.00) THEN CheckOverflow((promote_precision(web_sales#75) / promote_precision(web_sales#61)), 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) (82) Project [codegen id : 24] -Output [8]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41, ca_county#50, web_sales#55, web_sales#66] -Input [9]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41, ca_county#50, web_sales#55, ca_county#61, web_sales#66] +Output [8]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47, ca_county#56, web_sales#61, web_sales#75] +Input [9]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47, ca_county#56, web_sales#61, ca_county#70, web_sales#75] (83) Scan parquet default.web_sales -Output [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Output [3]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct (84) ColumnarToRow [codegen id : 22] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] +Input [3]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79] (85) Filter [codegen id : 22] -Input [3]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_bill_addr_sk#43) +Input [3]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79] +Condition : isnotnull(ws_bill_addr_sk#77) (86) ReusedExchange [Reuses operator id: 42] -Output [3]: [d_date_sk#68, d_year#69, d_qoy#70] +Output [3]: [d_date_sk#80, d_year#81, d_qoy#82] (87) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#45] -Right keys [1]: [d_date_sk#68] +Left keys [1]: [ws_sold_date_sk#79] +Right keys [1]: [d_date_sk#80] Join condition: None (88) Project [codegen id : 22] -Output [4]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#69, d_qoy#70] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, ws_sold_date_sk#45, d_date_sk#68, d_year#69, d_qoy#70] +Output [4]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, d_year#81, d_qoy#82] +Input [6]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, ws_sold_date_sk#79, d_date_sk#80, d_year#81, d_qoy#82] (89) ReusedExchange [Reuses operator id: 13] -Output [2]: [ca_address_sk#71, ca_county#72] +Output [2]: [ca_address_sk#83, ca_county#84] (90) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_bill_addr_sk#43] -Right keys [1]: [ca_address_sk#71] +Left keys [1]: [ws_bill_addr_sk#77] +Right keys [1]: [ca_address_sk#83] Join condition: None (91) Project [codegen id : 22] -Output [4]: [ws_ext_sales_price#44, d_year#69, d_qoy#70, ca_county#72] -Input [6]: [ws_bill_addr_sk#43, ws_ext_sales_price#44, d_year#69, d_qoy#70, ca_address_sk#71, ca_county#72] +Output [4]: [ws_ext_sales_price#78, d_year#81, d_qoy#82, ca_county#84] +Input [6]: [ws_bill_addr_sk#77, ws_ext_sales_price#78, d_year#81, d_qoy#82, ca_address_sk#83, ca_county#84] (92) HashAggregate [codegen id : 22] -Input [4]: [ws_ext_sales_price#44, d_year#69, d_qoy#70, ca_county#72] -Keys [3]: [ca_county#72, d_qoy#70, d_year#69] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum#73] -Results [4]: [ca_county#72, d_qoy#70, d_year#69, sum#74] +Input [4]: [ws_ext_sales_price#78, d_year#81, d_qoy#82, ca_county#84] +Keys [3]: [ca_county#84, d_qoy#82, d_year#81] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#78))] +Aggregate Attributes [1]: [sum#85] +Results [4]: [ca_county#84, d_qoy#82, d_year#81, sum#86] (93) Exchange -Input [4]: [ca_county#72, d_qoy#70, d_year#69, sum#74] -Arguments: hashpartitioning(ca_county#72, d_qoy#70, d_year#69, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [4]: [ca_county#84, d_qoy#82, d_year#81, sum#86] +Arguments: hashpartitioning(ca_county#84, d_qoy#82, d_year#81, 5), ENSURE_REQUIREMENTS, [id=#87] (94) HashAggregate [codegen id : 23] -Input [4]: [ca_county#72, d_qoy#70, d_year#69, sum#74] -Keys [3]: [ca_county#72, d_qoy#70, d_year#69] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#44))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#44))#76] -Results [2]: [ca_county#72, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#44))#76,17,2) AS web_sales#77] +Input [4]: [ca_county#84, d_qoy#82, d_year#81, sum#86] +Keys [3]: [ca_county#84, d_qoy#82, d_year#81] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#78))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#78))#88] +Results [2]: [ca_county#84, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#78))#88,17,2) AS web_sales#89] (95) BroadcastExchange -Input [2]: [ca_county#72, web_sales#77] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#78] +Input [2]: [ca_county#84, web_sales#89] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#90] (96) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [ca_county#50] -Right keys [1]: [ca_county#72] -Join condition: (CASE WHEN (web_sales#66 > 0.00) THEN CheckOverflow((promote_precision(web_sales#77) / promote_precision(web_sales#66)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#28 > 0.00) THEN CheckOverflow((promote_precision(store_sales#41) / promote_precision(store_sales#28)), DecimalType(37,20), true) ELSE null END) +Left keys [1]: [ca_county#56] +Right keys [1]: [ca_county#84] +Join condition: (CASE WHEN (web_sales#75 > 0.00) THEN CheckOverflow((promote_precision(web_sales#89) / promote_precision(web_sales#75)), DecimalType(37,20), true) ELSE null END > CASE WHEN (store_sales#31 > 0.00) THEN CheckOverflow((promote_precision(store_sales#47) / promote_precision(store_sales#31)), DecimalType(37,20), true) ELSE null END) (97) Project [codegen id : 24] -Output [6]: [ca_county#10, d_year#6, CheckOverflow((promote_precision(web_sales#66) / promote_precision(web_sales#55)), DecimalType(37,20), true) AS web_q1_q2_increase#79, CheckOverflow((promote_precision(store_sales#28) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q1_q2_increase#80, CheckOverflow((promote_precision(web_sales#77) / promote_precision(web_sales#66)), DecimalType(37,20), true) AS web_q2_q3_increase#81, CheckOverflow((promote_precision(store_sales#41) / promote_precision(store_sales#28)), DecimalType(37,20), true) AS store_q2_q3_increase#82] -Input [10]: [ca_county#10, d_year#6, store_sales#16, store_sales#28, store_sales#41, ca_county#50, web_sales#55, web_sales#66, ca_county#72, web_sales#77] +Output [6]: [ca_county#10, d_year#6, CheckOverflow((promote_precision(web_sales#75) / promote_precision(web_sales#61)), DecimalType(37,20), true) AS web_q1_q2_increase#91, CheckOverflow((promote_precision(store_sales#31) / promote_precision(store_sales#16)), DecimalType(37,20), true) AS store_q1_q2_increase#92, CheckOverflow((promote_precision(web_sales#89) / promote_precision(web_sales#75)), DecimalType(37,20), true) AS web_q2_q3_increase#93, CheckOverflow((promote_precision(store_sales#47) / promote_precision(store_sales#31)), DecimalType(37,20), true) AS store_q2_q3_increase#94] +Input [10]: [ca_county#10, d_year#6, store_sales#16, store_sales#31, store_sales#47, ca_county#56, web_sales#61, web_sales#75, ca_county#84, web_sales#89] (98) Exchange -Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#79, store_q1_q2_increase#80, web_q2_q3_increase#81, store_q2_q3_increase#82] -Arguments: rangepartitioning(ca_county#10 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#83] +Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#91, store_q1_q2_increase#92, web_q2_q3_increase#93, store_q2_q3_increase#94] +Arguments: rangepartitioning(ca_county#10 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#95] (99) Sort [codegen id : 25] -Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#79, store_q1_q2_increase#80, web_q2_q3_increase#81, store_q2_q3_increase#82] +Input [6]: [ca_county#10, d_year#6, web_q1_q2_increase#91, store_q1_q2_increase#92, web_q2_q3_increase#93, store_q2_q3_increase#94] Arguments: [ca_county#10 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -576,24 +576,24 @@ ReusedExchange (100) (100) ReusedExchange [Reuses operator id: 7] Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#17 +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#19 IN dynamicpruning#20 ReusedExchange (101) (101) ReusedExchange [Reuses operator id: 25] -Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] -Subquery:3 Hosting operator id = 36 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#30 +Subquery:3 Hosting operator id = 36 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 ReusedExchange (102) (102) ReusedExchange [Reuses operator id: 42] -Output [3]: [d_date_sk#31, d_year#32, d_qoy#33] +Output [3]: [d_date_sk#37, d_year#38, d_qoy#39] -Subquery:4 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#17 +Subquery:5 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#20 -Subquery:6 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#30 +Subquery:6 Hosting operator id = 83 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#36 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index 1349aa54f3cc2..eeda8611876d6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -160,30 +160,30 @@ Input [2]: [i_item_sk#13, i_manufact_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_category#15, i_manufact_id#14] +Output [2]: [i_category#15, i_manufact_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#14] +Input [2]: [i_category#15, i_manufact_id#16] (23) Filter [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#14] +Input [2]: [i_category#15, i_manufact_id#16] Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] -Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] -Input [2]: [i_category#15, i_manufact_id#14] +Output [1]: [i_manufact_id#16] +Input [2]: [i_category#15, i_manufact_id#16] (25) BroadcastExchange -Input [1]: [i_manufact_id#14#16] +Input [1]: [i_manufact_id#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_manufact_id#14] -Right keys [1]: [i_manufact_id#14#16] +Right keys [1]: [i_manufact_id#16] Join condition: None (27) BroadcastExchange @@ -233,127 +233,127 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_manufact_id#14] +Output [2]: [i_item_sk#30, i_manufact_id#31] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_manufact_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_manufact_id#14] +Output [2]: [cs_ext_sales_price#26, i_manufact_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_manufact_id#31] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_manufact_id#14] -Keys [1]: [i_manufact_id#14] +Input [2]: [cs_ext_sales_price#26, i_manufact_id#31] +Keys [1]: [i_manufact_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_manufact_id#14, sum#29] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_manufact_id#31, sum#33] (46) Exchange -Input [2]: [i_manufact_id#14, sum#29] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [i_manufact_id#31, sum#33] +Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#14, sum#29] -Keys [1]: [i_manufact_id#14] +Input [2]: [i_manufact_id#31, sum#33] +Keys [1]: [i_manufact_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] (48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] (50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#41] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#42] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_manufact_id#14] +Output [2]: [i_item_sk#43, i_manufact_id#44] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_manufact_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_manufact_id#14] +Output [2]: [ws_ext_sales_price#39, i_manufact_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_manufact_id#44] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_manufact_id#14] -Keys [1]: [i_manufact_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_manufact_id#14, sum#38] +Input [2]: [ws_ext_sales_price#39, i_manufact_id#44] +Keys [1]: [i_manufact_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_manufact_id#44, sum#46] (61) Exchange -Input [2]: [i_manufact_id#14, sum#38] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [2]: [i_manufact_id#44, sum#46] +Arguments: hashpartitioning(i_manufact_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] (62) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#14, sum#38] -Keys [1]: [i_manufact_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +Input [2]: [i_manufact_id#44, sum#46] +Keys [1]: [i_manufact_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_manufact_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] (63) Union @@ -361,23 +361,23 @@ Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price Input [2]: [i_manufact_id#14, total_sales#23] Keys [1]: [i_manufact_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_manufact_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_manufact_id#14, sum#52, isEmpty#53] (65) Exchange -Input [3]: [i_manufact_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] (66) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#14, sum#44, isEmpty#45] +Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] Keys [1]: [i_manufact_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_manufact_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_manufact_id#14, sum(total_sales#23)#55 AS total_sales#56] (67) TakeOrderedAndProject -Input [2]: [i_manufact_id#14, total_sales#48] -Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_manufact_id#14, total_sales#48] +Input [2]: [i_manufact_id#14, total_sales#56] +Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_manufact_id#14, total_sales#56] ===== Subqueries ===== @@ -390,6 +390,6 @@ Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index 1349aa54f3cc2..eeda8611876d6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -160,30 +160,30 @@ Input [2]: [i_item_sk#13, i_manufact_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_category#15, i_manufact_id#14] +Output [2]: [i_category#15, i_manufact_id#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#14] +Input [2]: [i_category#15, i_manufact_id#16] (23) Filter [codegen id : 3] -Input [2]: [i_category#15, i_manufact_id#14] +Input [2]: [i_category#15, i_manufact_id#16] Condition : (isnotnull(i_category#15) AND (i_category#15 = Electronics )) (24) Project [codegen id : 3] -Output [1]: [i_manufact_id#14 AS i_manufact_id#14#16] -Input [2]: [i_category#15, i_manufact_id#14] +Output [1]: [i_manufact_id#16] +Input [2]: [i_category#15, i_manufact_id#16] (25) BroadcastExchange -Input [1]: [i_manufact_id#14#16] +Input [1]: [i_manufact_id#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_manufact_id#14] -Right keys [1]: [i_manufact_id#14#16] +Right keys [1]: [i_manufact_id#16] Join condition: None (27) BroadcastExchange @@ -233,127 +233,127 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_manufact_id#14] +Output [2]: [i_item_sk#30, i_manufact_id#31] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_manufact_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_manufact_id#14] +Output [2]: [cs_ext_sales_price#26, i_manufact_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_manufact_id#31] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_manufact_id#14] -Keys [1]: [i_manufact_id#14] +Input [2]: [cs_ext_sales_price#26, i_manufact_id#31] +Keys [1]: [i_manufact_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_manufact_id#14, sum#29] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_manufact_id#31, sum#33] (46) Exchange -Input [2]: [i_manufact_id#14, sum#29] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [i_manufact_id#31, sum#33] +Arguments: hashpartitioning(i_manufact_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#14, sum#29] -Keys [1]: [i_manufact_id#14] +Input [2]: [i_manufact_id#31, sum#33] +Keys [1]: [i_manufact_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_manufact_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] (48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] (50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#41] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#42] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_manufact_id#14] +Output [2]: [i_item_sk#43, i_manufact_id#44] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_manufact_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_manufact_id#14] +Output [2]: [ws_ext_sales_price#39, i_manufact_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_manufact_id#44] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_manufact_id#14] -Keys [1]: [i_manufact_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_manufact_id#14, sum#38] +Input [2]: [ws_ext_sales_price#39, i_manufact_id#44] +Keys [1]: [i_manufact_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_manufact_id#44, sum#46] (61) Exchange -Input [2]: [i_manufact_id#14, sum#38] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [2]: [i_manufact_id#44, sum#46] +Arguments: hashpartitioning(i_manufact_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] (62) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#14, sum#38] -Keys [1]: [i_manufact_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +Input [2]: [i_manufact_id#44, sum#46] +Keys [1]: [i_manufact_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_manufact_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] (63) Union @@ -361,23 +361,23 @@ Results [2]: [i_manufact_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price Input [2]: [i_manufact_id#14, total_sales#23] Keys [1]: [i_manufact_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_manufact_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_manufact_id#14, sum#52, isEmpty#53] (65) Exchange -Input [3]: [i_manufact_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_manufact_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] (66) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#14, sum#44, isEmpty#45] +Input [3]: [i_manufact_id#14, sum#52, isEmpty#53] Keys [1]: [i_manufact_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_manufact_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_manufact_id#14, sum(total_sales#23)#55 AS total_sales#56] (67) TakeOrderedAndProject -Input [2]: [i_manufact_id#14, total_sales#48] -Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_manufact_id#14, total_sales#48] +Input [2]: [i_manufact_id#14, total_sales#56] +Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_manufact_id#14, total_sales#56] ===== Subqueries ===== @@ -390,6 +390,6 @@ Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt index 7098389df6776..9da5f36eb0553 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt @@ -145,20 +145,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#17] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#10] +Right keys [1]: [d_date_sk#17] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] -Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] +Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] (23) Exchange Input [1]: [ws_bill_customer_sk#15] -Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] @@ -170,38 +170,38 @@ Right keys [1]: [ws_bill_customer_sk#15] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Output [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#21] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [cs_sold_date_sk#20] +Right keys [1]: [d_date_sk#21] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#18] -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] +Output [1]: [cs_ship_customer_sk#19] +Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] (31) Exchange -Input [1]: [cs_ship_customer_sk#18] -Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [cs_ship_customer_sk#19] +Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#18] -Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#19] +Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#19] Join condition: None (34) Filter [codegen id : 12] @@ -214,103 +214,103 @@ Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2 (36) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#21] +Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#23] (37) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 (38) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_state#23] +Output [2]: [ca_address_sk#24, ca_state#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 14] -Input [2]: [ca_address_sk#22, ca_state#23] +Input [2]: [ca_address_sk#24, ca_state#25] (40) Filter [codegen id : 14] -Input [2]: [ca_address_sk#22, ca_state#23] -Condition : isnotnull(ca_address_sk#22) +Input [2]: [ca_address_sk#24, ca_state#25] +Condition : isnotnull(ca_address_sk#24) (41) Exchange -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: hashpartitioning(ca_address_sk#22, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] (42) Sort [codegen id : 15] -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#24] Join condition: None (44) Project [codegen id : 16] -Output [2]: [c_current_cdemo_sk#4, ca_state#23] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] +Output [2]: [c_current_cdemo_sk#4, ca_state#25] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24, ca_state#25] (45) Exchange -Input [2]: [c_current_cdemo_sk#4, ca_state#23] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [2]: [c_current_cdemo_sk#4, ca_state#25] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#27] (46) Sort [codegen id : 17] -Input [2]: [c_current_cdemo_sk#4, ca_state#23] +Input [2]: [c_current_cdemo_sk#4, ca_state#25] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 (47) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 18] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (49) Filter [codegen id : 18] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Condition : isnotnull(cd_demo_sk#26) +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Condition : isnotnull(cd_demo_sk#28) (50) Exchange -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: hashpartitioning(cd_demo_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#34] (51) Sort [codegen id : 19] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: [cd_demo_sk#26 ASC NULLS FIRST], false, 0 +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#26] +Right keys [1]: [cd_demo_sk#28] Join condition: None (53) Project [codegen id : 20] -Output [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [8]: [c_current_cdemo_sk#4, ca_state#25, cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (54) HashAggregate [codegen id : 20] -Input [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#29), partial_max(cd_dep_count#29), partial_avg(cd_dep_count#29), partial_min(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_avg(cd_dep_employed_count#30), partial_min(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_avg(cd_dep_college_count#31)] -Aggregate Attributes [13]: [count#33, min#34, max#35, sum#36, count#37, min#38, max#39, sum#40, count#41, min#42, max#43, sum#44, count#45] -Results [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] +Input [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Functions [10]: [partial_count(1), partial_min(cd_dep_count#31), partial_max(cd_dep_count#31), partial_avg(cd_dep_count#31), partial_min(cd_dep_employed_count#32), partial_max(cd_dep_employed_count#32), partial_avg(cd_dep_employed_count#32), partial_min(cd_dep_college_count#33), partial_max(cd_dep_college_count#33), partial_avg(cd_dep_college_count#33)] +Aggregate Attributes [13]: [count#35, min#36, max#37, sum#38, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47] +Results [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] (55) Exchange -Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] -Arguments: hashpartitioning(ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] +Arguments: hashpartitioning(ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#61] (56) HashAggregate [codegen id : 21] -Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, min#47, max#48, sum#49, count#50, min#51, max#52, sum#53, count#54, min#55, max#56, sum#57, count#58] -Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [count(1), min(cd_dep_count#29), max(cd_dep_count#29), avg(cd_dep_count#29), min(cd_dep_employed_count#30), max(cd_dep_employed_count#30), avg(cd_dep_employed_count#30), min(cd_dep_college_count#31), max(cd_dep_college_count#31), avg(cd_dep_college_count#31)] -Aggregate Attributes [10]: [count(1)#60, min(cd_dep_count#29)#61, max(cd_dep_count#29)#62, avg(cd_dep_count#29)#63, min(cd_dep_employed_count#30)#64, max(cd_dep_employed_count#30)#65, avg(cd_dep_employed_count#30)#66, min(cd_dep_college_count#31)#67, max(cd_dep_college_count#31)#68, avg(cd_dep_college_count#31)#69] -Results [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, count(1)#60 AS cnt1#70, min(cd_dep_count#29)#61 AS min(cd_dep_count)#71, max(cd_dep_count#29)#62 AS max(cd_dep_count)#72, avg(cd_dep_count#29)#63 AS avg(cd_dep_count)#73, cd_dep_employed_count#30, count(1)#60 AS cnt2#74, min(cd_dep_employed_count#30)#64 AS min(cd_dep_employed_count)#75, max(cd_dep_employed_count#30)#65 AS max(cd_dep_employed_count)#76, avg(cd_dep_employed_count#30)#66 AS avg(cd_dep_employed_count)#77, cd_dep_college_count#31, count(1)#60 AS cnt3#78, min(cd_dep_college_count#31)#67 AS min(cd_dep_college_count)#79, max(cd_dep_college_count#31)#68 AS max(cd_dep_college_count)#80, avg(cd_dep_college_count#31)#69 AS avg(cd_dep_college_count)#81, cd_dep_count#29 AS aggOrder#82] +Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, min#49, max#50, sum#51, count#52, min#53, max#54, sum#55, count#56, min#57, max#58, sum#59, count#60] +Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Functions [10]: [count(1), min(cd_dep_count#31), max(cd_dep_count#31), avg(cd_dep_count#31), min(cd_dep_employed_count#32), max(cd_dep_employed_count#32), avg(cd_dep_employed_count#32), min(cd_dep_college_count#33), max(cd_dep_college_count#33), avg(cd_dep_college_count#33)] +Aggregate Attributes [10]: [count(1)#62, min(cd_dep_count#31)#63, max(cd_dep_count#31)#64, avg(cd_dep_count#31)#65, min(cd_dep_employed_count#32)#66, max(cd_dep_employed_count#32)#67, avg(cd_dep_employed_count#32)#68, min(cd_dep_college_count#33)#69, max(cd_dep_college_count#33)#70, avg(cd_dep_college_count#33)#71] +Results [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, count(1)#62 AS cnt1#72, min(cd_dep_count#31)#63 AS min(cd_dep_count)#73, max(cd_dep_count#31)#64 AS max(cd_dep_count)#74, avg(cd_dep_count#31)#65 AS avg(cd_dep_count)#75, cd_dep_employed_count#32, count(1)#62 AS cnt2#76, min(cd_dep_employed_count#32)#66 AS min(cd_dep_employed_count)#77, max(cd_dep_employed_count#32)#67 AS max(cd_dep_employed_count)#78, avg(cd_dep_employed_count#32)#68 AS avg(cd_dep_employed_count)#79, cd_dep_college_count#33, count(1)#62 AS cnt3#80, min(cd_dep_college_count#33)#69 AS min(cd_dep_college_count)#81, max(cd_dep_college_count#33)#70 AS max(cd_dep_college_count)#82, avg(cd_dep_college_count#33)#71 AS avg(cd_dep_college_count)#83, cd_dep_count#31 AS aggOrder#84] (57) TakeOrderedAndProject -Input [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cnt1#70, min(cd_dep_count)#71, max(cd_dep_count)#72, avg(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, min(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, avg(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, min(cd_dep_college_count)#79, max(cd_dep_college_count)#80, avg(cd_dep_college_count)#81, aggOrder#82] -Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, aggOrder#82 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#23, cd_gender#27, cd_marital_status#28, cnt1#70, min(cd_dep_count)#71, max(cd_dep_count)#72, avg(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, min(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, avg(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, min(cd_dep_college_count)#79, max(cd_dep_college_count)#80, avg(cd_dep_college_count)#81] +Input [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cnt1#72, min(cd_dep_count)#73, max(cd_dep_count)#74, avg(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, min(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, avg(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, min(cd_dep_college_count)#81, max(cd_dep_college_count)#82, avg(cd_dep_college_count)#83, aggOrder#84] +Arguments: 100, [ca_state#25 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, aggOrder#84 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [ca_state#25, cd_gender#29, cd_marital_status#30, cnt1#72, min(cd_dep_count)#73, max(cd_dep_count)#74, avg(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, min(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, avg(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, min(cd_dep_college_count)#81, max(cd_dep_college_count)#82, avg(cd_dep_college_count)#83] ===== Subqueries ===== @@ -323,6 +323,6 @@ Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index 4dab13491784a..48ae824834450 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -122,20 +122,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#16] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] -Right keys [1]: [d_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] +Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#16] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] @@ -143,34 +143,34 @@ Right keys [1]: [ws_bill_customer_sk#14] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#8)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#20] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#9] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#20] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#17] -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] +Output [1]: [cs_ship_customer_sk#18] +Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#20] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [cs_ship_customer_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#17] +Right keys [1]: [cs_ship_customer_sk#18] Join condition: None (29) Filter [codegen id : 9] @@ -182,80 +182,80 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +Output [2]: [ca_address_sk#22, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_state#21] +Input [2]: [ca_address_sk#22, ca_state#23] (33) Filter [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) +Input [2]: [ca_address_sk#22, ca_state#23] +Condition : isnotnull(ca_address_sk#22) (34) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +Input [2]: [ca_address_sk#22, ca_state#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] (35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#22] Join condition: None (36) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#21] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] +Output [2]: [c_current_cdemo_sk#4, ca_state#23] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] (37) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (39) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#25) (40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#25] Join condition: None (42) Project [codegen id : 9] -Output [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Output [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (43) HashAggregate [codegen id : 9] -Input [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_min(cd_dep_count#26), partial_max(cd_dep_count#26), partial_avg(cd_dep_count#26), partial_min(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_avg(cd_dep_employed_count#27), partial_min(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_avg(cd_dep_college_count#28)] -Aggregate Attributes [13]: [count#30, min#31, max#32, sum#33, count#34, min#35, max#36, sum#37, count#38, min#39, max#40, sum#41, count#42] -Results [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] +Input [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Functions [10]: [partial_count(1), partial_min(cd_dep_count#28), partial_max(cd_dep_count#28), partial_avg(cd_dep_count#28), partial_min(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_avg(cd_dep_employed_count#29), partial_min(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_avg(cd_dep_college_count#30)] +Aggregate Attributes [13]: [count#32, min#33, max#34, sum#35, count#36, min#37, max#38, sum#39, count#40, min#41, max#42, sum#43, count#44] +Results [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] (44) Exchange -Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] -Arguments: hashpartitioning(ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] +Arguments: hashpartitioning(ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] (45) HashAggregate [codegen id : 10] -Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51, min#52, max#53, sum#54, count#55] -Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), min(cd_dep_count#26), max(cd_dep_count#26), avg(cd_dep_count#26), min(cd_dep_employed_count#27), max(cd_dep_employed_count#27), avg(cd_dep_employed_count#27), min(cd_dep_college_count#28), max(cd_dep_college_count#28), avg(cd_dep_college_count#28)] -Aggregate Attributes [10]: [count(1)#57, min(cd_dep_count#26)#58, max(cd_dep_count#26)#59, avg(cd_dep_count#26)#60, min(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, avg(cd_dep_employed_count#27)#63, min(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, avg(cd_dep_college_count#28)#66] -Results [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, count(1)#57 AS cnt1#67, min(cd_dep_count#26)#58 AS min(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, avg(cd_dep_count#26)#60 AS avg(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, min(cd_dep_employed_count#27)#61 AS min(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, avg(cd_dep_employed_count#27)#63 AS avg(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, min(cd_dep_college_count#28)#64 AS min(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, avg(cd_dep_college_count#28)#66 AS avg(cd_dep_college_count)#78, cd_dep_count#26 AS aggOrder#79] +Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, min#46, max#47, sum#48, count#49, min#50, max#51, sum#52, count#53, min#54, max#55, sum#56, count#57] +Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Functions [10]: [count(1), min(cd_dep_count#28), max(cd_dep_count#28), avg(cd_dep_count#28), min(cd_dep_employed_count#29), max(cd_dep_employed_count#29), avg(cd_dep_employed_count#29), min(cd_dep_college_count#30), max(cd_dep_college_count#30), avg(cd_dep_college_count#30)] +Aggregate Attributes [10]: [count(1)#59, min(cd_dep_count#28)#60, max(cd_dep_count#28)#61, avg(cd_dep_count#28)#62, min(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, avg(cd_dep_employed_count#29)#65, min(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, avg(cd_dep_college_count#30)#68] +Results [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, count(1)#59 AS cnt1#69, min(cd_dep_count#28)#60 AS min(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, avg(cd_dep_count#28)#62 AS avg(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, min(cd_dep_employed_count#29)#63 AS min(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, avg(cd_dep_employed_count#29)#65 AS avg(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, min(cd_dep_college_count#30)#66 AS min(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, avg(cd_dep_college_count#30)#68 AS avg(cd_dep_college_count)#80, cd_dep_count#28 AS aggOrder#81] (46) TakeOrderedAndProject -Input [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78, aggOrder#79] -Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, aggOrder#79 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#21, cd_gender#24, cd_marital_status#25, cnt1#67, min(cd_dep_count)#68, max(cd_dep_count)#69, avg(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, min(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, avg(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, min(cd_dep_college_count)#76, max(cd_dep_college_count)#77, avg(cd_dep_college_count)#78] +Input [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80, aggOrder#81] +Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, aggOrder#81 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#23, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80] ===== Subqueries ===== @@ -268,6 +268,6 @@ Output [1]: [d_date_sk#9] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 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 68ab7a8bbecab..5eb63f2a046cf 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 @@ -191,121 +191,121 @@ Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30] Arguments: [i_item_sk#10 ASC NULLS FIRST, w_warehouse_sk#12 ASC NULLS FIRST], false, 0 (30) Scan parquet default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(inv_date_sk#35), dynamicpruningexpression(inv_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 10] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] (32) Filter [codegen id : 10] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] +Condition : (isnotnull(inv_item_sk#32) AND isnotnull(inv_warehouse_sk#33)) (33) Scan parquet default.date_dim -Output [3]: [d_date_sk#33, d_year#34, d_moy#35] +Output [3]: [d_date_sk#37, d_year#38, d_moy#39] 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 (34) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] (35) Filter [codegen id : 7] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] -Condition : ((((isnotnull(d_year#34) AND isnotnull(d_moy#35)) AND (d_year#34 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#33)) +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] +Condition : ((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2001)) AND (d_moy#39 = 2)) AND isnotnull(d_date_sk#37)) (36) Project [codegen id : 7] -Output [2]: [d_date_sk#33, d_moy#35] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] +Output [2]: [d_date_sk#37, d_moy#39] +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] (37) BroadcastExchange -Input [2]: [d_date_sk#33, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [2]: [d_date_sk#37, d_moy#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [inv_date_sk#35] +Right keys [1]: [d_date_sk#37] Join condition: None (39) Project [codegen id : 10] -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35] -Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#33, d_moy#35] +Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39] +Input [6]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35, d_date_sk#37, d_moy#39] (40) ReusedExchange [Reuses operator id: 14] -Output [1]: [i_item_sk#37] +Output [1]: [i_item_sk#41] (41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#37] +Left keys [1]: [inv_item_sk#32] +Right keys [1]: [i_item_sk#41] Join condition: None (42) Project [codegen id : 10] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37] +Output [4]: [inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41] +Input [5]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41] (43) ReusedExchange [Reuses operator id: 20] -Output [2]: [w_warehouse_sk#38, w_warehouse_name#39] +Output [2]: [w_warehouse_sk#42, w_warehouse_name#43] (44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#38] +Left keys [1]: [inv_warehouse_sk#33] +Right keys [1]: [w_warehouse_sk#42] Join condition: None (45) Project [codegen id : 10] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39, d_moy#35] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39] +Output [5]: [inv_quantity_on_hand#34, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43, d_moy#39] +Input [6]: [inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43] (46) HashAggregate [codegen id : 10] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39, d_moy#35] -Keys [4]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#40, avg#41, m2#42, sum#43, count#44] -Results [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, n#45, avg#46, m2#47, sum#48, count#49] +Input [5]: [inv_quantity_on_hand#34, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43, d_moy#39] +Keys [4]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#34 as double)), partial_avg(inv_quantity_on_hand#34)] +Aggregate Attributes [5]: [n#44, avg#45, m2#46, sum#47, count#48] +Results [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] (47) Exchange -Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, n#45, avg#46, m2#47, sum#48, count#49] -Arguments: hashpartitioning(w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] +Arguments: hashpartitioning(w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, 5), ENSURE_REQUIREMENTS, [id=#54] (48) HashAggregate [codegen id : 11] -Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#51, avg(inv_quantity_on_hand#3)#52] -Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#3 as double))#51 AS stdev#53, avg(inv_quantity_on_hand#3)#52 AS mean#54] +Input [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] +Keys [4]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#34 as double)), avg(inv_quantity_on_hand#34)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#34 as double))#55, avg(inv_quantity_on_hand#34)#56] +Results [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stddev_samp(cast(inv_quantity_on_hand#34 as double))#55 AS stdev#57, avg(inv_quantity_on_hand#34)#56 AS mean#58] (49) Filter [codegen id : 11] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#42, i_item_sk#41, d_moy#39, stdev#57, mean#58] +Condition : ((isnotnull(mean#58) AND isnotnull(stdev#57)) AND (NOT (mean#58 = 0.0) AND ((stdev#57 / mean#58) > 1.0))) (50) Project [codegen id : 11] -Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#35, stdev#53, mean#54] +Output [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, CASE WHEN (mean#58 = 0.0) THEN null ELSE (stdev#57 / mean#58) END AS cov#59] +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stdev#57, mean#58] (51) Exchange -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: hashpartitioning(i_item_sk#37, w_warehouse_sk#38, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: hashpartitioning(i_item_sk#41, w_warehouse_sk#42, 5), ENSURE_REQUIREMENTS, [id=#60] (52) Sort [codegen id : 12] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: [i_item_sk#37 ASC NULLS FIRST, w_warehouse_sk#38 ASC NULLS FIRST], false, 0 +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: [i_item_sk#41 ASC NULLS FIRST, w_warehouse_sk#42 ASC NULLS FIRST], false, 0 (53) SortMergeJoin [codegen id : 13] Left keys [2]: [i_item_sk#10, w_warehouse_sk#12] -Right keys [2]: [i_item_sk#37, w_warehouse_sk#38] +Right keys [2]: [i_item_sk#41, w_warehouse_sk#42] Join condition: None (54) Exchange -Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: rangepartitioning(w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: rangepartitioning(w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, mean#58 ASC NULLS FIRST, cov#59 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#61] (55) Sort [codegen id : 14] -Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: [w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: [w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, mean#58 ASC NULLS FIRST, cov#59 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -316,11 +316,11 @@ ReusedExchange (56) (56) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#6, d_moy#8] -Subquery:2 Hosting operator id = 30 Hosting Expression = inv_date_sk#4 IN dynamicpruning#32 +Subquery:2 Hosting operator id = 30 Hosting Expression = inv_date_sk#35 IN dynamicpruning#36 ReusedExchange (57) (57) ReusedExchange [Reuses operator id: 37] -Output [2]: [d_date_sk#33, d_moy#35] +Output [2]: [d_date_sk#37, d_moy#39] 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 7abd3bb1acb11..e77de53c5adcb 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 @@ -180,117 +180,117 @@ Output [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, CASE WHEN (mean#2 Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] (28) Scan parquet default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Output [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(inv_date_sk#34), dynamicpruningexpression(inv_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] (30) Filter [codegen id : 8] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) +Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] +Condition : (isnotnull(inv_item_sk#31) AND isnotnull(inv_warehouse_sk#32)) (31) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#32] +Output [1]: [i_item_sk#36] (32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [inv_item_sk#31] +Right keys [1]: [i_item_sk#36] Join condition: None (33) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32] +Output [4]: [inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36] +Input [5]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36] (34) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#33, w_warehouse_name#34] +Output [2]: [w_warehouse_sk#37, w_warehouse_name#38] (35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#33] +Left keys [1]: [inv_warehouse_sk#32] +Right keys [1]: [w_warehouse_sk#37] Join condition: None (36) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] +Output [5]: [inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38] +Input [6]: [inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38] (37) Scan parquet default.date_dim -Output [3]: [d_date_sk#35, d_year#36, d_moy#37] +Output [3]: [d_date_sk#39, d_year#40, d_moy#41] 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 (38) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] (39) Filter [codegen id : 7] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2001)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] +Condition : ((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2001)) AND (d_moy#41 = 2)) AND isnotnull(d_date_sk#39)) (40) Project [codegen id : 7] -Output [2]: [d_date_sk#35, d_moy#37] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Output [2]: [d_date_sk#39, d_moy#41] +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] (41) BroadcastExchange -Input [2]: [d_date_sk#35, d_moy#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] +Input [2]: [d_date_sk#39, d_moy#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (42) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#35] +Left keys [1]: [inv_date_sk#34] +Right keys [1]: [d_date_sk#39] Join condition: None (43) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_date_sk#35, d_moy#37] +Output [5]: [inv_quantity_on_hand#33, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_moy#41] +Input [7]: [inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_date_sk#39, d_moy#41] (44) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] -Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#39, avg#40, m2#41, sum#42, count#43] -Results [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] +Input [5]: [inv_quantity_on_hand#33, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_moy#41] +Keys [4]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#33 as double)), partial_avg(inv_quantity_on_hand#33)] +Aggregate Attributes [5]: [n#43, avg#44, m2#45, sum#46, count#47] +Results [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] (45) Exchange -Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] -Arguments: hashpartitioning(w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] +Arguments: hashpartitioning(w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, 5), ENSURE_REQUIREMENTS, [id=#53] (46) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] -Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#50, avg(inv_quantity_on_hand#3)#51] -Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_quantity_on_hand#3 as double))#50 AS stdev#52, avg(inv_quantity_on_hand#3)#51 AS mean#53] +Input [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] +Keys [4]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#33 as double)), avg(inv_quantity_on_hand#33)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#33 as double))#54, avg(inv_quantity_on_hand#33)#55] +Results [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stddev_samp(cast(inv_quantity_on_hand#33 as double))#54 AS stdev#56, avg(inv_quantity_on_hand#33)#55 AS mean#57] (47) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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#37, i_item_sk#36, d_moy#41, stdev#56, mean#57] +Condition : ((isnotnull(mean#57) AND isnotnull(stdev#56)) AND (NOT (mean#57 = 0.0) AND ((stdev#56 / mean#57) > 1.0))) (48) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#52, mean#53] +Output [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, CASE WHEN (mean#57 = 0.0) THEN null ELSE (stdev#56 / mean#57) END AS cov#58] +Input [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stdev#56, mean#57] (49) BroadcastExchange -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [id=#59] (50) BroadcastHashJoin [codegen id : 10] Left keys [2]: [i_item_sk#6, w_warehouse_sk#8] -Right keys [2]: [i_item_sk#32, w_warehouse_sk#33] +Right keys [2]: [i_item_sk#36, w_warehouse_sk#37] Join condition: None (51) Exchange -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, cov#54] -Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#37 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#13, mean#29, cov#30, w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, mean#57 ASC NULLS FIRST, cov#58 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#60] (52) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, cov#54] -Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#37 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#13, mean#29, cov#30, w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, mean#57 ASC NULLS FIRST, cov#58 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -301,11 +301,11 @@ ReusedExchange (53) (53) ReusedExchange [Reuses operator id: 20] Output [2]: [d_date_sk#11, d_moy#13] -Subquery:2 Hosting operator id = 28 Hosting Expression = inv_date_sk#4 IN dynamicpruning#31 +Subquery:2 Hosting operator id = 28 Hosting Expression = inv_date_sk#34 IN dynamicpruning#35 ReusedExchange (54) (54) ReusedExchange [Reuses operator id: 41] -Output [2]: [d_date_sk#35, d_moy#37] +Output [2]: [d_date_sk#39, d_moy#41] 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 f0dfbac4f3a0f..e2e3760fe2d03 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 @@ -191,121 +191,121 @@ Input [5]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30] Arguments: [i_item_sk#10 ASC NULLS FIRST, w_warehouse_sk#12 ASC NULLS FIRST], false, 0 (30) Scan parquet default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#32)] +PartitionFilters: [isnotnull(inv_date_sk#35), dynamicpruningexpression(inv_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 10] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] (32) Filter [codegen id : 10] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) +Input [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35] +Condition : (isnotnull(inv_item_sk#32) AND isnotnull(inv_warehouse_sk#33)) (33) Scan parquet default.date_dim -Output [3]: [d_date_sk#33, d_year#34, d_moy#35] +Output [3]: [d_date_sk#37, d_year#38, d_moy#39] 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 (34) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] (35) Filter [codegen id : 7] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] -Condition : ((((isnotnull(d_year#34) AND isnotnull(d_moy#35)) AND (d_year#34 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#33)) +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] +Condition : ((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2001)) AND (d_moy#39 = 2)) AND isnotnull(d_date_sk#37)) (36) Project [codegen id : 7] -Output [2]: [d_date_sk#33, d_moy#35] -Input [3]: [d_date_sk#33, d_year#34, d_moy#35] +Output [2]: [d_date_sk#37, d_moy#39] +Input [3]: [d_date_sk#37, d_year#38, d_moy#39] (37) BroadcastExchange -Input [2]: [d_date_sk#33, d_moy#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [2]: [d_date_sk#37, d_moy#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [inv_date_sk#35] +Right keys [1]: [d_date_sk#37] Join condition: None (39) Project [codegen id : 10] -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35] -Input [6]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, d_date_sk#33, d_moy#35] +Output [4]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39] +Input [6]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, inv_date_sk#35, d_date_sk#37, d_moy#39] (40) ReusedExchange [Reuses operator id: 14] -Output [1]: [i_item_sk#37] +Output [1]: [i_item_sk#41] (41) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#37] +Left keys [1]: [inv_item_sk#32] +Right keys [1]: [i_item_sk#41] Join condition: None (42) Project [codegen id : 10] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37] +Output [4]: [inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41] +Input [5]: [inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41] (43) ReusedExchange [Reuses operator id: 20] -Output [2]: [w_warehouse_sk#38, w_warehouse_name#39] +Output [2]: [w_warehouse_sk#42, w_warehouse_name#43] (44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#38] +Left keys [1]: [inv_warehouse_sk#33] +Right keys [1]: [w_warehouse_sk#42] Join condition: None (45) Project [codegen id : 10] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39, d_moy#35] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, d_moy#35, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39] +Output [5]: [inv_quantity_on_hand#34, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43, d_moy#39] +Input [6]: [inv_warehouse_sk#33, inv_quantity_on_hand#34, d_moy#39, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43] (46) HashAggregate [codegen id : 10] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#37, w_warehouse_sk#38, w_warehouse_name#39, d_moy#35] -Keys [4]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#40, avg#41, m2#42, sum#43, count#44] -Results [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, n#45, avg#46, m2#47, sum#48, count#49] +Input [5]: [inv_quantity_on_hand#34, i_item_sk#41, w_warehouse_sk#42, w_warehouse_name#43, d_moy#39] +Keys [4]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#34 as double)), partial_avg(inv_quantity_on_hand#34)] +Aggregate Attributes [5]: [n#44, avg#45, m2#46, sum#47, count#48] +Results [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] (47) Exchange -Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, n#45, avg#46, m2#47, sum#48, count#49] -Arguments: hashpartitioning(w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] +Arguments: hashpartitioning(w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, 5), ENSURE_REQUIREMENTS, [id=#54] (48) HashAggregate [codegen id : 11] -Input [9]: [w_warehouse_name#39, w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#51, avg(inv_quantity_on_hand#3)#52] -Results [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#3 as double))#51 AS stdev#53, avg(inv_quantity_on_hand#3)#52 AS mean#54] +Input [9]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39, n#49, avg#50, m2#51, sum#52, count#53] +Keys [4]: [w_warehouse_name#43, w_warehouse_sk#42, i_item_sk#41, d_moy#39] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#34 as double)), avg(inv_quantity_on_hand#34)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#34 as double))#55, avg(inv_quantity_on_hand#34)#56] +Results [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stddev_samp(cast(inv_quantity_on_hand#34 as double))#55 AS stdev#57, avg(inv_quantity_on_hand#34)#56 AS mean#58] (49) Filter [codegen id : 11] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#42, i_item_sk#41, d_moy#39, stdev#57, mean#58] +Condition : ((isnotnull(mean#58) AND isnotnull(stdev#57)) AND (NOT (mean#58 = 0.0) AND ((stdev#57 / mean#58) > 1.0))) (50) Project [codegen id : 11] -Output [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, 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#35, stdev#53, mean#54] +Output [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, CASE WHEN (mean#58 = 0.0) THEN null ELSE (stdev#57 / mean#58) END AS cov#59] +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, stdev#57, mean#58] (51) Exchange -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: hashpartitioning(i_item_sk#37, w_warehouse_sk#38, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: hashpartitioning(i_item_sk#41, w_warehouse_sk#42, 5), ENSURE_REQUIREMENTS, [id=#60] (52) Sort [codegen id : 12] -Input [5]: [w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: [i_item_sk#37 ASC NULLS FIRST, w_warehouse_sk#38 ASC NULLS FIRST], false, 0 +Input [5]: [w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: [i_item_sk#41 ASC NULLS FIRST, w_warehouse_sk#42 ASC NULLS FIRST], false, 0 (53) SortMergeJoin [codegen id : 13] Left keys [2]: [i_item_sk#10, w_warehouse_sk#12] -Right keys [2]: [i_item_sk#37, w_warehouse_sk#38] +Right keys [2]: [i_item_sk#41, w_warehouse_sk#42] Join condition: None (54) Exchange -Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: rangepartitioning(w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: rangepartitioning(w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, mean#58 ASC NULLS FIRST, cov#59 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#61] (55) Sort [codegen id : 14] -Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#38, i_item_sk#37, d_moy#35, mean#54, cov#55] -Arguments: [w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#54 ASC NULLS FIRST, cov#55 ASC NULLS FIRST], true, 0 +Input [10]: [w_warehouse_sk#12, i_item_sk#10, d_moy#8, mean#29, cov#30, w_warehouse_sk#42, i_item_sk#41, d_moy#39, mean#58, cov#59] +Arguments: [w_warehouse_sk#12 ASC NULLS FIRST, i_item_sk#10 ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#39 ASC NULLS FIRST, mean#58 ASC NULLS FIRST, cov#59 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -316,11 +316,11 @@ ReusedExchange (56) (56) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#6, d_moy#8] -Subquery:2 Hosting operator id = 30 Hosting Expression = inv_date_sk#4 IN dynamicpruning#32 +Subquery:2 Hosting operator id = 30 Hosting Expression = inv_date_sk#35 IN dynamicpruning#36 ReusedExchange (57) (57) ReusedExchange [Reuses operator id: 37] -Output [2]: [d_date_sk#33, d_moy#35] +Output [2]: [d_date_sk#37, d_moy#39] 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 d1a5ecaa6a4d8..d60cd37ce7bf1 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 @@ -180,117 +180,117 @@ Output [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, CASE WHEN (mean#2 Input [5]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, stdev#28, mean#29] (28) Scan parquet default.inventory -Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Output [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(inv_date_sk#34), dynamicpruningexpression(inv_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 8] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] (30) Filter [codegen id : 8] -Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] -Condition : (isnotnull(inv_item_sk#1) AND isnotnull(inv_warehouse_sk#2)) +Input [4]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34] +Condition : (isnotnull(inv_item_sk#31) AND isnotnull(inv_warehouse_sk#32)) (31) ReusedExchange [Reuses operator id: 7] -Output [1]: [i_item_sk#32] +Output [1]: [i_item_sk#36] (32) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_item_sk#1] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [inv_item_sk#31] +Right keys [1]: [i_item_sk#36] Join condition: None (33) Project [codegen id : 8] -Output [4]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32] -Input [5]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32] +Output [4]: [inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36] +Input [5]: [inv_item_sk#31, inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36] (34) ReusedExchange [Reuses operator id: 13] -Output [2]: [w_warehouse_sk#33, w_warehouse_name#34] +Output [2]: [w_warehouse_sk#37, w_warehouse_name#38] (35) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_warehouse_sk#2] -Right keys [1]: [w_warehouse_sk#33] +Left keys [1]: [inv_warehouse_sk#32] +Right keys [1]: [w_warehouse_sk#37] Join condition: None (36) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] -Input [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34] +Output [5]: [inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38] +Input [6]: [inv_warehouse_sk#32, inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38] (37) Scan parquet default.date_dim -Output [3]: [d_date_sk#35, d_year#36, d_moy#37] +Output [3]: [d_date_sk#39, d_year#40, d_moy#41] 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 (38) ColumnarToRow [codegen id : 7] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] (39) Filter [codegen id : 7] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] -Condition : ((((isnotnull(d_year#36) AND isnotnull(d_moy#37)) AND (d_year#36 = 2001)) AND (d_moy#37 = 2)) AND isnotnull(d_date_sk#35)) +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] +Condition : ((((isnotnull(d_year#40) AND isnotnull(d_moy#41)) AND (d_year#40 = 2001)) AND (d_moy#41 = 2)) AND isnotnull(d_date_sk#39)) (40) Project [codegen id : 7] -Output [2]: [d_date_sk#35, d_moy#37] -Input [3]: [d_date_sk#35, d_year#36, d_moy#37] +Output [2]: [d_date_sk#39, d_moy#41] +Input [3]: [d_date_sk#39, d_year#40, d_moy#41] (41) BroadcastExchange -Input [2]: [d_date_sk#35, d_moy#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] +Input [2]: [d_date_sk#39, d_moy#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] (42) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [inv_date_sk#4] -Right keys [1]: [d_date_sk#35] +Left keys [1]: [inv_date_sk#34] +Right keys [1]: [d_date_sk#39] Join condition: None (43) Project [codegen id : 8] -Output [5]: [inv_quantity_on_hand#3, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] -Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_date_sk#35, d_moy#37] +Output [5]: [inv_quantity_on_hand#33, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_moy#41] +Input [7]: [inv_quantity_on_hand#33, inv_date_sk#34, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_date_sk#39, d_moy#41] (44) HashAggregate [codegen id : 8] -Input [5]: [inv_quantity_on_hand#3, i_item_sk#32, w_warehouse_sk#33, w_warehouse_name#34, d_moy#37] -Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] -Aggregate Attributes [5]: [n#39, avg#40, m2#41, sum#42, count#43] -Results [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] +Input [5]: [inv_quantity_on_hand#33, i_item_sk#36, w_warehouse_sk#37, w_warehouse_name#38, d_moy#41] +Keys [4]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#33 as double)), partial_avg(inv_quantity_on_hand#33)] +Aggregate Attributes [5]: [n#43, avg#44, m2#45, sum#46, count#47] +Results [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] (45) Exchange -Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] -Arguments: hashpartitioning(w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] +Arguments: hashpartitioning(w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, 5), ENSURE_REQUIREMENTS, [id=#53] (46) HashAggregate [codegen id : 9] -Input [9]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37, n#44, avg#45, m2#46, sum#47, count#48] -Keys [4]: [w_warehouse_name#34, w_warehouse_sk#33, i_item_sk#32, d_moy#37] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double)), avg(inv_quantity_on_hand#3)] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#3 as double))#50, avg(inv_quantity_on_hand#3)#51] -Results [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stddev_samp(cast(inv_quantity_on_hand#3 as double))#50 AS stdev#52, avg(inv_quantity_on_hand#3)#51 AS mean#53] +Input [9]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41, n#48, avg#49, m2#50, sum#51, count#52] +Keys [4]: [w_warehouse_name#38, w_warehouse_sk#37, i_item_sk#36, d_moy#41] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#33 as double)), avg(inv_quantity_on_hand#33)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#33 as double))#54, avg(inv_quantity_on_hand#33)#55] +Results [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stddev_samp(cast(inv_quantity_on_hand#33 as double))#54 AS stdev#56, avg(inv_quantity_on_hand#33)#55 AS mean#57] (47) Filter [codegen id : 9] -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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#37, i_item_sk#36, d_moy#41, stdev#56, mean#57] +Condition : ((isnotnull(mean#57) AND isnotnull(stdev#56)) AND (NOT (mean#57 = 0.0) AND ((stdev#56 / mean#57) > 1.0))) (48) Project [codegen id : 9] -Output [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, CASE WHEN (mean#53 = 0.0) THEN null ELSE (stdev#52 / mean#53) END AS cov#54] -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, stdev#52, mean#53] +Output [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, CASE WHEN (mean#57 = 0.0) THEN null ELSE (stdev#56 / mean#57) END AS cov#58] +Input [5]: [w_warehouse_sk#37, i_item_sk#36, d_moy#41, stdev#56, mean#57] (49) BroadcastExchange -Input [5]: [w_warehouse_sk#33, i_item_sk#32, d_moy#37, 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#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [id=#59] (50) BroadcastHashJoin [codegen id : 10] Left keys [2]: [i_item_sk#6, w_warehouse_sk#8] -Right keys [2]: [i_item_sk#32, w_warehouse_sk#33] +Right keys [2]: [i_item_sk#36, w_warehouse_sk#37] Join condition: None (51) Exchange -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, cov#54] -Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#37 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#13, mean#29, cov#30, w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: rangepartitioning(w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, mean#57 ASC NULLS FIRST, cov#58 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#60] (52) Sort [codegen id : 11] -Input [10]: [w_warehouse_sk#8, i_item_sk#6, d_moy#13, mean#29, cov#30, w_warehouse_sk#33, i_item_sk#32, d_moy#37, mean#53, cov#54] -Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#37 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#13, mean#29, cov#30, w_warehouse_sk#37, i_item_sk#36, d_moy#41, mean#57, cov#58] +Arguments: [w_warehouse_sk#8 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, mean#29 ASC NULLS FIRST, cov#30 ASC NULLS FIRST, d_moy#41 ASC NULLS FIRST, mean#57 ASC NULLS FIRST, cov#58 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -301,11 +301,11 @@ ReusedExchange (53) (53) ReusedExchange [Reuses operator id: 20] Output [2]: [d_date_sk#11, d_moy#13] -Subquery:2 Hosting operator id = 28 Hosting Expression = inv_date_sk#4 IN dynamicpruning#31 +Subquery:2 Hosting operator id = 28 Hosting Expression = inv_date_sk#34 IN dynamicpruning#35 ReusedExchange (54) (54) ReusedExchange [Reuses operator id: 41] -Output [2]: [d_date_sk#35, d_moy#37] +Output [2]: [d_date_sk#39, d_moy#41] 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 7b6bd35bfe180..ce8b8bed5a26e 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 @@ -239,465 +239,465 @@ Input [2]: [customer_id#27, year_total#28] Arguments: [customer_id#27 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6] +Output [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ss_sold_date_sk#35), dynamicpruningexpression(ss_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6] +Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] (27) Filter [codegen id : 10] -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6] -Condition : isnotnull(ss_customer_sk#1) +Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Condition : isnotnull(ss_customer_sk#30) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#37, d_year#38] 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 (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#8, d_year#9] +Input [2]: [d_date_sk#37, d_year#38] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#8, d_year#9] -Condition : ((isnotnull(d_year#9) AND (d_year#9 = 2002)) AND isnotnull(d_date_sk#8)) +Input [2]: [d_date_sk#37, d_year#38] +Condition : ((isnotnull(d_year#38) AND (d_year#38 = 2002)) AND isnotnull(d_date_sk#37)) (31) BroadcastExchange -Input [2]: [d_date_sk#8, d_year#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [2]: [d_date_sk#37, d_year#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#39] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ss_sold_date_sk#35] +Right keys [1]: [d_date_sk#37] Join condition: None (33) Project [codegen id : 10] -Output [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] -Input [8]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, ss_sold_date_sk#6, d_date_sk#8, d_year#9] +Output [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Input [8]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#37, d_year#38] (34) Exchange -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] -Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Arguments: hashpartitioning(ss_customer_sk#30, 5), ENSURE_REQUIREMENTS, [id=#40] (35) Sort [codegen id : 11] -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] -Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [6]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Arguments: [ss_customer_sk#30 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] (37) Sort [codegen id : 13] -Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] -Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +Arguments: [c_customer_sk#41 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#12] +Left keys [1]: [ss_customer_sk#30] +Right keys [1]: [c_customer_sk#41] Join condition: None (39) Project [codegen id : 14] -Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] -Input [14]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Input [14]: [ss_customer_sk#30, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38, c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] (40) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#35, isEmpty#36] +Input [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ss_ext_discount_amt#31, ss_ext_sales_price#32, ss_ext_wholesale_cost#33, ss_ext_list_price#34, d_year#38] +Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#49, isEmpty#50] +Results [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, sum#51, isEmpty#52] (41) Exchange -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#35, isEmpty#36] -Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, sum#51, isEmpty#52] +Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, 5), ENSURE_REQUIREMENTS, [id=#53] (42) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#35, isEmpty#36] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 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#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#38] -Results [8]: [c_customer_id#13 AS customer_id#39, c_first_name#14 AS customer_first_name#40, c_last_name#15 AS customer_last_name#41, c_preferred_cust_flag#16 AS customer_preferred_cust_flag#42, c_birth_country#17 AS customer_birth_country#43, c_login#18 AS customer_login#44, c_email_address#19 AS customer_email_address#45, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#38 AS year_total#46] +Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38, sum#51, isEmpty#52] +Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#38] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 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#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#54] +Results [8]: [c_customer_id#42 AS customer_id#55, c_first_name#43 AS customer_first_name#56, c_last_name#44 AS customer_last_name#57, c_preferred_cust_flag#45 AS customer_preferred_cust_flag#58, c_birth_country#46 AS customer_birth_country#59, c_login#47 AS customer_login#60, c_email_address#48 AS customer_email_address#61, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#33 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#31 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#32 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#54 AS year_total#62] (43) Exchange -Input [8]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46] -Arguments: hashpartitioning(customer_id#39, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [8]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62] +Arguments: hashpartitioning(customer_id#55, 5), ENSURE_REQUIREMENTS, [id=#63] (44) Sort [codegen id : 16] -Input [8]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46] -Arguments: [customer_id#39 ASC NULLS FIRST], false, 0 +Input [8]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62] +Arguments: [customer_id#55 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#39] +Right keys [1]: [customer_id#55] Join condition: None (46) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] +Output [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#53), dynamicpruningexpression(cs_sold_date_sk#53 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#69), dynamicpruningexpression(cs_sold_date_sk#69 IN dynamicpruning#7)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] +Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69] (48) Filter [codegen id : 19] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] -Condition : isnotnull(cs_bill_customer_sk#48) +Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69] +Condition : isnotnull(cs_bill_customer_sk#64) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#70, d_year#71] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#69] +Right keys [1]: [d_date_sk#70] Join condition: None (51) Project [codegen id : 19] -Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Input [8]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53, d_date_sk#8, d_year#9] +Output [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] +Input [8]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, cs_sold_date_sk#69, d_date_sk#70, d_year#71] (52) Exchange -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Arguments: hashpartitioning(cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] +Arguments: hashpartitioning(cs_bill_customer_sk#64, 5), ENSURE_REQUIREMENTS, [id=#72] (53) Sort [codegen id : 20] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Arguments: [cs_bill_customer_sk#48 ASC NULLS FIRST], false, 0 +Input [6]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] +Arguments: [cs_bill_customer_sk#64 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] (55) Sort [codegen id : 22] -Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] -Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] +Arguments: [c_customer_sk#73 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#48] -Right keys [1]: [c_customer_sk#12] +Left keys [1]: [cs_bill_customer_sk#64] +Right keys [1]: [c_customer_sk#73] Join condition: None (57) Project [codegen id : 23] -Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Input [14]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] +Input [14]: [cs_bill_customer_sk#64, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71, c_customer_sk#73, c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80] (58) HashAggregate [codegen id : 23] -Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#55, isEmpty#56] -Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#57, isEmpty#58] +Input [12]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, cs_ext_discount_amt#65, cs_ext_sales_price#66, cs_ext_wholesale_cost#67, cs_ext_list_price#68, d_year#71] +Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#81, isEmpty#82] +Results [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, sum#83, isEmpty#84] (59) Exchange -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#57, isEmpty#58] -Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, sum#83, isEmpty#84] +Arguments: hashpartitioning(c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, 5), ENSURE_REQUIREMENTS, [id=#85] (60) HashAggregate [codegen id : 24] -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#57, isEmpty#58] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 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#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#60] -Results [2]: [c_customer_id#13 AS customer_id#61, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#60 AS year_total#62] +Input [10]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71, sum#83, isEmpty#84] +Keys [8]: [c_customer_id#74, c_first_name#75, c_last_name#76, c_preferred_cust_flag#77, c_birth_country#78, c_login#79, c_email_address#80, d_year#71] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 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#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86] +Results [2]: [c_customer_id#74 AS customer_id#87, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#68 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#67 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#65 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#66 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86 AS year_total#88] (61) Filter [codegen id : 24] -Input [2]: [customer_id#61, year_total#62] -Condition : (isnotnull(year_total#62) AND (year_total#62 > 0.000000)) +Input [2]: [customer_id#87, year_total#88] +Condition : (isnotnull(year_total#88) AND (year_total#88 > 0.000000)) (62) Project [codegen id : 24] -Output [2]: [customer_id#61 AS customer_id#63, year_total#62 AS year_total#64] -Input [2]: [customer_id#61, year_total#62] +Output [2]: [customer_id#87 AS customer_id#89, year_total#88 AS year_total#90] +Input [2]: [customer_id#87, year_total#88] (63) Exchange -Input [2]: [customer_id#63, year_total#64] -Arguments: hashpartitioning(customer_id#63, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [2]: [customer_id#89, year_total#90] +Arguments: hashpartitioning(customer_id#89, 5), ENSURE_REQUIREMENTS, [id=#91] (64) Sort [codegen id : 25] -Input [2]: [customer_id#63, year_total#64] -Arguments: [customer_id#63 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#89, year_total#90] +Arguments: [customer_id#89 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#63] +Right keys [1]: [customer_id#89] Join condition: None (66) Project [codegen id : 26] -Output [11]: [customer_id#27, year_total#28, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46, year_total#64] -Input [12]: [customer_id#27, year_total#28, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46, customer_id#63, year_total#64] +Output [11]: [customer_id#27, year_total#28, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62, year_total#90] +Input [12]: [customer_id#27, year_total#28, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62, customer_id#89, year_total#90] (67) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] +Output [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#53), dynamicpruningexpression(cs_sold_date_sk#53 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(cs_sold_date_sk#97), dynamicpruningexpression(cs_sold_date_sk#97 IN dynamicpruning#36)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] +Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97] (69) Filter [codegen id : 28] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53] -Condition : isnotnull(cs_bill_customer_sk#48) +Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97] +Condition : isnotnull(cs_bill_customer_sk#92) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#98, d_year#99] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#97] +Right keys [1]: [d_date_sk#98] Join condition: None (72) Project [codegen id : 28] -Output [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Input [8]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, cs_sold_date_sk#53, d_date_sk#8, d_year#9] +Output [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] +Input [8]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, cs_sold_date_sk#97, d_date_sk#98, d_year#99] (73) Exchange -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Arguments: hashpartitioning(cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] +Arguments: hashpartitioning(cs_bill_customer_sk#92, 5), ENSURE_REQUIREMENTS, [id=#100] (74) Sort [codegen id : 29] -Input [6]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Arguments: [cs_bill_customer_sk#48 ASC NULLS FIRST], false, 0 +Input [6]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] +Arguments: [cs_bill_customer_sk#92 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] (76) Sort [codegen id : 31] -Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] -Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] +Arguments: [c_customer_sk#101 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [cs_bill_customer_sk#48] -Right keys [1]: [c_customer_sk#12] +Left keys [1]: [cs_bill_customer_sk#92] +Right keys [1]: [c_customer_sk#101] Join condition: None (78) Project [codegen id : 32] -Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Input [14]: [cs_bill_customer_sk#48, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] +Input [14]: [cs_bill_customer_sk#92, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99, c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] (79) HashAggregate [codegen id : 32] -Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, cs_ext_discount_amt#49, cs_ext_sales_price#50, cs_ext_wholesale_cost#51, cs_ext_list_price#52, d_year#9] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#67, isEmpty#68] -Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#69, isEmpty#70] +Input [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, cs_ext_discount_amt#93, cs_ext_sales_price#94, cs_ext_wholesale_cost#95, cs_ext_list_price#96, d_year#99] +Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#109, isEmpty#110] +Results [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, sum#111, isEmpty#112] (80) Exchange -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#69, isEmpty#70] -Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, sum#111, isEmpty#112] +Arguments: hashpartitioning(c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, 5), ENSURE_REQUIREMENTS, [id=#113] (81) HashAggregate [codegen id : 33] -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#69, isEmpty#70] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 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#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#72] -Results [2]: [c_customer_id#13 AS customer_id#73, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#52 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#51 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#49 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#50 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#72 AS year_total#74] +Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99, sum#111, isEmpty#112] +Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#99] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 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#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#114] +Results [2]: [c_customer_id#102 AS customer_id#115, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#96 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#95 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#93 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#94 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#114 AS year_total#116] (82) Exchange -Input [2]: [customer_id#73, year_total#74] -Arguments: hashpartitioning(customer_id#73, 5), ENSURE_REQUIREMENTS, [id=#75] +Input [2]: [customer_id#115, year_total#116] +Arguments: hashpartitioning(customer_id#115, 5), ENSURE_REQUIREMENTS, [id=#117] (83) Sort [codegen id : 34] -Input [2]: [customer_id#73, year_total#74] -Arguments: [customer_id#73 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#115, year_total#116] +Arguments: [customer_id#115 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#73] -Join condition: (CASE WHEN (year_total#64 > 0.000000) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#64)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#28 > 0.000000) THEN CheckOverflow((promote_precision(year_total#46) / promote_precision(year_total#28)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#115] +Join condition: (CASE WHEN (year_total#90 > 0.000000) THEN CheckOverflow((promote_precision(year_total#116) / promote_precision(year_total#90)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#28 > 0.000000) THEN CheckOverflow((promote_precision(year_total#62) / promote_precision(year_total#28)), DecimalType(38,14), true) ELSE null END) (85) Project [codegen id : 35] -Output [10]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74] -Input [13]: [customer_id#27, year_total#28, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#46, year_total#64, customer_id#73, year_total#74] +Output [10]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116] +Input [13]: [customer_id#27, year_total#28, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#62, year_total#90, customer_id#115, year_total#116] (86) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] +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#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ws_sold_date_sk#123), dynamicpruningexpression(ws_sold_date_sk#123 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (87) ColumnarToRow [codegen id : 37] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] +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] (88) Filter [codegen id : 37] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_bill_customer_sk#76) +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) (89) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#124, d_year#125] (90) BroadcastHashJoin [codegen id : 37] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ws_sold_date_sk#123] +Right keys [1]: [d_date_sk#124] Join condition: None (91) Project [codegen id : 37] -Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Input [8]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#8, d_year#9] +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, d_year#125] +Input [8]: [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, d_date_sk#124, d_year#125] (92) Exchange -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Arguments: hashpartitioning(ws_bill_customer_sk#76, 5), ENSURE_REQUIREMENTS, [id=#82] +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, d_year#125] +Arguments: hashpartitioning(ws_bill_customer_sk#118, 5), ENSURE_REQUIREMENTS, [id=#126] (93) Sort [codegen id : 38] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Arguments: [ws_bill_customer_sk#76 ASC NULLS FIRST], false, 0 +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, d_year#125] +Arguments: [ws_bill_customer_sk#118 ASC NULLS FIRST], false, 0 (94) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [8]: [c_customer_sk#127, c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134] (95) Sort [codegen id : 40] -Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] -Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#127, c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134] +Arguments: [c_customer_sk#127 ASC NULLS FIRST], false, 0 (96) SortMergeJoin [codegen id : 41] -Left keys [1]: [ws_bill_customer_sk#76] -Right keys [1]: [c_customer_sk#12] +Left keys [1]: [ws_bill_customer_sk#118] +Right keys [1]: [c_customer_sk#127] Join condition: None (97) Project [codegen id : 41] -Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Input [14]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [12]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, d_year#125] +Input [14]: [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, d_year#125, c_customer_sk#127, c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134] (98) HashAggregate [codegen id : 41] -Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#83, isEmpty#84] -Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#85, isEmpty#86] +Input [12]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, ws_ext_discount_amt#119, ws_ext_sales_price#120, ws_ext_wholesale_cost#121, ws_ext_list_price#122, d_year#125] +Keys [8]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125] +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#135, isEmpty#136] +Results [10]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, sum#137, isEmpty#138] (99) Exchange -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#85, isEmpty#86] -Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#87] +Input [10]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, sum#137, isEmpty#138] +Arguments: hashpartitioning(c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, 5), ENSURE_REQUIREMENTS, [id=#139] (100) HashAggregate [codegen id : 42] -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#85, isEmpty#86] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 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#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#88] -Results [2]: [c_customer_id#13 AS customer_id#89, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#88 AS year_total#90] +Input [10]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125, sum#137, isEmpty#138] +Keys [8]: [c_customer_id#128, c_first_name#129, c_last_name#130, c_preferred_cust_flag#131, c_birth_country#132, c_login#133, c_email_address#134, d_year#125] +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))#140] +Results [2]: [c_customer_id#128 AS customer_id#141, 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))#140 AS year_total#142] (101) Filter [codegen id : 42] -Input [2]: [customer_id#89, year_total#90] -Condition : (isnotnull(year_total#90) AND (year_total#90 > 0.000000)) +Input [2]: [customer_id#141, year_total#142] +Condition : (isnotnull(year_total#142) AND (year_total#142 > 0.000000)) (102) Project [codegen id : 42] -Output [2]: [customer_id#89 AS customer_id#91, year_total#90 AS year_total#92] -Input [2]: [customer_id#89, year_total#90] +Output [2]: [customer_id#141 AS customer_id#143, year_total#142 AS year_total#144] +Input [2]: [customer_id#141, year_total#142] (103) Exchange -Input [2]: [customer_id#91, year_total#92] -Arguments: hashpartitioning(customer_id#91, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [2]: [customer_id#143, year_total#144] +Arguments: hashpartitioning(customer_id#143, 5), ENSURE_REQUIREMENTS, [id=#145] (104) Sort [codegen id : 43] -Input [2]: [customer_id#91, year_total#92] -Arguments: [customer_id#91 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#143, year_total#144] +Arguments: [customer_id#143 ASC NULLS FIRST], false, 0 (105) SortMergeJoin [codegen id : 44] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#91] +Right keys [1]: [customer_id#143] Join condition: None (106) Project [codegen id : 44] -Output [11]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74, year_total#92] -Input [12]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74, customer_id#91, year_total#92] +Output [11]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116, year_total#144] +Input [12]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116, customer_id#143, year_total#144] (107) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] +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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#30)] +PartitionFilters: [isnotnull(ws_sold_date_sk#151), dynamicpruningexpression(ws_sold_date_sk#151 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 46] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] +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] (109) Filter [codegen id : 46] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_bill_customer_sk#76) +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) (110) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#152, d_year#153] (111) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [ws_sold_date_sk#81] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ws_sold_date_sk#151] +Right keys [1]: [d_date_sk#152] Join condition: None (112) Project [codegen id : 46] -Output [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Input [8]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, ws_sold_date_sk#81, d_date_sk#8, d_year#9] +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, d_year#153] +Input [8]: [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, d_date_sk#152, d_year#153] (113) Exchange -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Arguments: hashpartitioning(ws_bill_customer_sk#76, 5), ENSURE_REQUIREMENTS, [id=#94] +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, d_year#153] +Arguments: hashpartitioning(ws_bill_customer_sk#146, 5), ENSURE_REQUIREMENTS, [id=#154] (114) Sort [codegen id : 47] -Input [6]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Arguments: [ws_bill_customer_sk#76 ASC NULLS FIRST], false, 0 +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, d_year#153] +Arguments: [ws_bill_customer_sk#146 ASC NULLS FIRST], false, 0 (115) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] (116) Sort [codegen id : 49] -Input [8]: [c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] -Arguments: [c_customer_sk#12 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] +Arguments: [c_customer_sk#155 ASC NULLS FIRST], false, 0 (117) SortMergeJoin [codegen id : 50] -Left keys [1]: [ws_bill_customer_sk#76] -Right keys [1]: [c_customer_sk#12] +Left keys [1]: [ws_bill_customer_sk#146] +Right keys [1]: [c_customer_sk#155] Join condition: None (118) Project [codegen id : 50] -Output [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Input [14]: [ws_bill_customer_sk#76, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9, c_customer_sk#12, c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19] +Output [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] +Input [14]: [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, d_year#153, c_customer_sk#155, c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162] (119) HashAggregate [codegen id : 50] -Input [12]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, ws_ext_discount_amt#77, ws_ext_sales_price#78, ws_ext_wholesale_cost#79, ws_ext_list_price#80, d_year#9] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#95, isEmpty#96] -Results [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#97, isEmpty#98] +Input [12]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, ws_ext_discount_amt#147, ws_ext_sales_price#148, ws_ext_wholesale_cost#149, ws_ext_list_price#150, d_year#153] +Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] +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#163, isEmpty#164] +Results [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] (120) Exchange -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#97, isEmpty#98] -Arguments: hashpartitioning(c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#99] +Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] +Arguments: hashpartitioning(c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, 5), ENSURE_REQUIREMENTS, [id=#167] (121) HashAggregate [codegen id : 51] -Input [10]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9, sum#97, isEmpty#98] -Keys [8]: [c_customer_id#13, c_first_name#14, c_last_name#15, c_preferred_cust_flag#16, c_birth_country#17, c_login#18, c_email_address#19, d_year#9] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 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#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#100] -Results [2]: [c_customer_id#13 AS customer_id#101, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#80 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#79 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#77 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#78 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#100 AS year_total#102] +Input [10]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153, sum#165, isEmpty#166] +Keys [8]: [c_customer_id#156, c_first_name#157, c_last_name#158, c_preferred_cust_flag#159, c_birth_country#160, c_login#161, c_email_address#162, d_year#153] +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))#168] +Results [2]: [c_customer_id#156 AS customer_id#169, 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))#168 AS year_total#170] (122) Exchange -Input [2]: [customer_id#101, year_total#102] -Arguments: hashpartitioning(customer_id#101, 5), ENSURE_REQUIREMENTS, [id=#103] +Input [2]: [customer_id#169, year_total#170] +Arguments: hashpartitioning(customer_id#169, 5), ENSURE_REQUIREMENTS, [id=#171] (123) Sort [codegen id : 52] -Input [2]: [customer_id#101, year_total#102] -Arguments: [customer_id#101 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#169, year_total#170] +Arguments: [customer_id#169 ASC NULLS FIRST], false, 0 (124) SortMergeJoin [codegen id : 53] Left keys [1]: [customer_id#27] -Right keys [1]: [customer_id#101] -Join condition: (CASE WHEN (year_total#64 > 0.000000) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#64)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#92 > 0.000000) THEN CheckOverflow((promote_precision(year_total#102) / promote_precision(year_total#92)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#169] +Join condition: (CASE WHEN (year_total#90 > 0.000000) THEN CheckOverflow((promote_precision(year_total#116) / promote_precision(year_total#90)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#144 > 0.000000) THEN CheckOverflow((promote_precision(year_total#170) / promote_precision(year_total#144)), DecimalType(38,14), true) ELSE null END) (125) Project [codegen id : 53] -Output [7]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45] -Input [13]: [customer_id#27, customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45, year_total#64, year_total#74, year_total#92, customer_id#101, year_total#102] +Output [7]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61] +Input [13]: [customer_id#27, customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61, year_total#90, year_total#116, year_total#144, customer_id#169, year_total#170] (126) TakeOrderedAndProject -Input [7]: [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45] -Arguments: 100, [customer_id#39 ASC NULLS FIRST, customer_first_name#40 ASC NULLS FIRST, customer_last_name#41 ASC NULLS FIRST, customer_preferred_cust_flag#42 ASC NULLS FIRST, customer_birth_country#43 ASC NULLS FIRST, customer_login#44 ASC NULLS FIRST, customer_email_address#45 ASC NULLS FIRST], [customer_id#39, customer_first_name#40, customer_last_name#41, customer_preferred_cust_flag#42, customer_birth_country#43, customer_login#44, customer_email_address#45] +Input [7]: [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61] +Arguments: 100, [customer_id#55 ASC NULLS FIRST, customer_first_name#56 ASC NULLS FIRST, customer_last_name#57 ASC NULLS FIRST, customer_preferred_cust_flag#58 ASC NULLS FIRST, customer_birth_country#59 ASC NULLS FIRST, customer_login#60 ASC NULLS FIRST, customer_email_address#61 ASC NULLS FIRST], [customer_id#55, customer_first_name#56, customer_last_name#57, customer_preferred_cust_flag#58, customer_birth_country#59, customer_login#60, customer_email_address#61] ===== Subqueries ===== @@ -708,19 +708,19 @@ ReusedExchange (127) (127) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#8, d_year#9] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#30 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 ReusedExchange (128) (128) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#8, d_year#9] +Output [2]: [d_date_sk#37, d_year#38] -Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#69 IN dynamicpruning#7 -Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#30 +Subquery:4 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#97 IN dynamicpruning#36 -Subquery:5 Hosting operator id = 86 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#7 +Subquery:5 Hosting operator id = 86 Hosting Expression = ws_sold_date_sk#123 IN dynamicpruning#7 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#30 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#151 IN dynamicpruning#36 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 80dee3d164511..1cf27630d2994 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 @@ -209,460 +209,460 @@ Input [2]: [customer_id#26, year_total#27] Condition : (isnotnull(year_total#27) AND (year_total#27 > 0.000000)) (20) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] (22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35] +Condition : (isnotnull(c_customer_sk#28) AND isnotnull(c_customer_id#29)) (23) Scan parquet default.store_sales -Output [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Output [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(ss_sold_date_sk#41), dynamicpruningexpression(ss_sold_date_sk#41 IN dynamicpruning#42)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] (25) Filter [codegen id : 4] -Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -Condition : isnotnull(ss_customer_sk#9) +Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] +Condition : isnotnull(ss_customer_sk#36) (26) BroadcastExchange -Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [6]: [ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#43] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#9] +Left keys [1]: [c_customer_sk#28] +Right keys [1]: [ss_customer_sk#36] Join condition: None (28) Project [codegen id : 6] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Output [12]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] +Input [14]: [c_customer_sk#28, c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_customer_sk#36, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#44, d_year#45] 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 (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#17, d_year#18] +Input [2]: [d_date_sk#44, d_year#45] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#17, d_year#18] -Condition : ((isnotnull(d_year#18) AND (d_year#18 = 2002)) AND isnotnull(d_date_sk#17)) +Input [2]: [d_date_sk#44, d_year#45] +Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) (32) BroadcastExchange -Input [2]: [d_date_sk#17, d_year#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] +Input [2]: [d_date_sk#44, d_year#45] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#46] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#14] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ss_sold_date_sk#41] +Right keys [1]: [d_date_sk#44] Join condition: None (34) Project [codegen id : 6] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#18] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14, d_date_sk#17, d_year#18] +Output [12]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#45] +Input [14]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, ss_sold_date_sk#41, d_date_sk#44, d_year#45] (35) HashAggregate [codegen id : 6] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#18] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#31, isEmpty#32] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#33, isEmpty#34] +Input [12]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#37, ss_ext_sales_price#38, ss_ext_wholesale_cost#39, ss_ext_list_price#40, d_year#45] +Keys [8]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#47, isEmpty#48] +Results [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, sum#49, isEmpty#50] (36) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#33, isEmpty#34] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, sum#49, isEmpty#50] +Arguments: hashpartitioning(c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, 5), ENSURE_REQUIREMENTS, [id=#51] (37) HashAggregate [codegen id : 7] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#33, isEmpty#34] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 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#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#36] -Results [8]: [c_customer_id#2 AS customer_id#37, c_first_name#3 AS customer_first_name#38, c_last_name#4 AS customer_last_name#39, c_preferred_cust_flag#5 AS customer_preferred_cust_flag#40, c_birth_country#6 AS customer_birth_country#41, c_login#7 AS customer_login#42, c_email_address#8 AS customer_email_address#43, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#36 AS year_total#44] +Input [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45, sum#49, isEmpty#50] +Keys [8]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, d_year#45] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 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#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#52] +Results [8]: [c_customer_id#29 AS customer_id#53, c_first_name#30 AS customer_first_name#54, c_last_name#31 AS customer_last_name#55, c_preferred_cust_flag#32 AS customer_preferred_cust_flag#56, c_birth_country#33 AS customer_birth_country#57, c_login#34 AS customer_login#58, c_email_address#35 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#40 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#39 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#37 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#38 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#52 AS year_total#60] (38) BroadcastExchange -Input [8]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#45] +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: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#61] (39) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#37] +Right keys [1]: [customer_id#53] Join condition: None (40) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] (42) Filter [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69] +Condition : (isnotnull(c_customer_sk#62) AND isnotnull(c_customer_id#63)) (43) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Output [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#51), dynamicpruningexpression(cs_sold_date_sk#51 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(cs_sold_date_sk#75), dynamicpruningexpression(cs_sold_date_sk#75 IN dynamicpruning#15)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Input [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] (45) Filter [codegen id : 8] -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Condition : isnotnull(cs_bill_customer_sk#46) +Input [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] +Condition : isnotnull(cs_bill_customer_sk#70) (46) BroadcastExchange -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#52] +Input [6]: [cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#76] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_bill_customer_sk#46] +Left keys [1]: [c_customer_sk#62] +Right keys [1]: [cs_bill_customer_sk#70] Join condition: None (48) Project [codegen id : 10] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] +Input [14]: [c_customer_sk#62, c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_bill_customer_sk#70, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#77, d_year#78] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#51] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [cs_sold_date_sk#75] +Right keys [1]: [d_date_sk#77] Join condition: None (51) Project [codegen id : 10] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51, d_date_sk#17, d_year#18] +Output [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, d_year#78] +Input [14]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, cs_sold_date_sk#75, d_date_sk#77, d_year#78] (52) HashAggregate [codegen id : 10] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#53, isEmpty#54] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#55, isEmpty#56] +Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#71, cs_ext_sales_price#72, cs_ext_wholesale_cost#73, cs_ext_list_price#74, d_year#78] +Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 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#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, sum#81, isEmpty#82] (53) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#55, isEmpty#56] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, sum#81, isEmpty#82] +Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#83] (54) HashAggregate [codegen id : 11] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#55, isEmpty#56] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 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#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#58] -Results [2]: [c_customer_id#2 AS customer_id#59, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#58 AS year_total#60] +Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78, sum#81, isEmpty#82] +Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#78] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 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#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#84] +Results [2]: [c_customer_id#63 AS customer_id#85, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#74 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#73 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#71 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#72 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#84 AS year_total#86] (55) Filter [codegen id : 11] -Input [2]: [customer_id#59, year_total#60] -Condition : (isnotnull(year_total#60) AND (year_total#60 > 0.000000)) +Input [2]: [customer_id#85, year_total#86] +Condition : (isnotnull(year_total#86) AND (year_total#86 > 0.000000)) (56) Project [codegen id : 11] -Output [2]: [customer_id#59 AS customer_id#61, year_total#60 AS year_total#62] -Input [2]: [customer_id#59, year_total#60] +Output [2]: [customer_id#85 AS customer_id#87, year_total#86 AS year_total#88] +Input [2]: [customer_id#85, year_total#86] (57) BroadcastExchange -Input [2]: [customer_id#61, year_total#62] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#63] +Input [2]: [customer_id#87, year_total#88] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#89] (58) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#61] +Right keys [1]: [customer_id#87] Join condition: None (59) Project [codegen id : 24] -Output [11]: [customer_id#26, year_total#27, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44, year_total#62] -Input [12]: [customer_id#26, year_total#27, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44, customer_id#61, year_total#62] +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#88] +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#87, year_total#88] (60) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97] (62) Filter [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97] +Condition : (isnotnull(c_customer_sk#90) AND isnotnull(c_customer_id#91)) (63) Scan parquet default.catalog_sales -Output [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Output [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#51), dynamicpruningexpression(cs_sold_date_sk#51 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(cs_sold_date_sk#103), dynamicpruningexpression(cs_sold_date_sk#103 IN dynamicpruning#42)] PushedFilters: [IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Input [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] (65) Filter [codegen id : 12] -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Condition : isnotnull(cs_bill_customer_sk#46) +Input [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] +Condition : isnotnull(cs_bill_customer_sk#98) (66) BroadcastExchange -Input [6]: [cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] +Input [6]: [cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#104] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_bill_customer_sk#46] +Left keys [1]: [c_customer_sk#90] +Right keys [1]: [cs_bill_customer_sk#98] Join condition: None (68) Project [codegen id : 14] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_bill_customer_sk#46, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51] +Output [12]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] +Input [14]: [c_customer_sk#90, c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_bill_customer_sk#98, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#105, d_year#106] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [cs_sold_date_sk#51] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [cs_sold_date_sk#103] +Right keys [1]: [d_date_sk#105] Join condition: None (71) Project [codegen id : 14] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, cs_sold_date_sk#51, d_date_sk#17, d_year#18] +Output [12]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, d_year#106] +Input [14]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, cs_sold_date_sk#103, d_date_sk#105, d_year#106] (72) HashAggregate [codegen id : 14] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, cs_ext_discount_amt#47, cs_ext_sales_price#48, cs_ext_wholesale_cost#49, cs_ext_list_price#50, d_year#18] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#65, isEmpty#66] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#67, isEmpty#68] +Input [12]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, cs_ext_discount_amt#99, cs_ext_sales_price#100, cs_ext_wholesale_cost#101, cs_ext_list_price#102, d_year#106] +Keys [8]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#107, isEmpty#108] +Results [10]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, sum#109, isEmpty#110] (73) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#67, isEmpty#68] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [10]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, sum#109, isEmpty#110] +Arguments: hashpartitioning(c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, 5), ENSURE_REQUIREMENTS, [id=#111] (74) HashAggregate [codegen id : 15] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#67, isEmpty#68] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 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#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#70] -Results [2]: [c_customer_id#2 AS customer_id#71, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#50 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#49 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#47 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#48 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#70 AS year_total#72] +Input [10]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106, sum#109, isEmpty#110] +Keys [8]: [c_customer_id#91, c_first_name#92, c_last_name#93, c_preferred_cust_flag#94, c_birth_country#95, c_login#96, c_email_address#97, d_year#106] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 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#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112] +Results [2]: [c_customer_id#91 AS customer_id#113, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#102 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#101 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#99 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#100 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#112 AS year_total#114] (75) BroadcastExchange -Input [2]: [customer_id#71, year_total#72] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#73] +Input [2]: [customer_id#113, year_total#114] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#115] (76) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#71] -Join condition: (CASE WHEN (year_total#62 > 0.000000) THEN CheckOverflow((promote_precision(year_total#72) / promote_precision(year_total#62)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#27 > 0.000000) THEN CheckOverflow((promote_precision(year_total#44) / promote_precision(year_total#27)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#113] +Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), 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) (77) Project [codegen id : 24] -Output [10]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72] -Input [13]: [customer_id#26, year_total#27, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#44, year_total#62, customer_id#71, year_total#72] +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#88, year_total#114] +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#88, customer_id#113, year_total#114] (78) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (79) ColumnarToRow [codegen id : 18] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123] (80) Filter [codegen id : 18] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123] +Condition : (isnotnull(c_customer_sk#116) AND isnotnull(c_customer_id#117)) (81) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Output [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#15)] +PartitionFilters: [isnotnull(ws_sold_date_sk#129), dynamicpruningexpression(ws_sold_date_sk#129 IN dynamicpruning#15)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (82) ColumnarToRow [codegen id : 16] -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Input [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] (83) Filter [codegen id : 16] -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Condition : isnotnull(ws_bill_customer_sk#74) +Input [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] +Condition : isnotnull(ws_bill_customer_sk#124) (84) BroadcastExchange -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#80] +Input [6]: [ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#130] (85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#74] +Left keys [1]: [c_customer_sk#116] +Right keys [1]: [ws_bill_customer_sk#124] Join condition: None (86) Project [codegen id : 18] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Output [12]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] +Input [14]: [c_customer_sk#116, c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_bill_customer_sk#124, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129] (87) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#131, d_year#132] (88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ws_sold_date_sk#79] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ws_sold_date_sk#129] +Right keys [1]: [d_date_sk#131] Join condition: None (89) Project [codegen id : 18] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#17, d_year#18] +Output [12]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, d_year#132] +Input [14]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, ws_sold_date_sk#129, d_date_sk#131, d_year#132] (90) HashAggregate [codegen id : 18] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#81, isEmpty#82] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#83, isEmpty#84] +Input [12]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, ws_ext_discount_amt#125, ws_ext_sales_price#126, ws_ext_wholesale_cost#127, ws_ext_list_price#128, d_year#132] +Keys [8]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#133, isEmpty#134] +Results [10]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, sum#135, isEmpty#136] (91) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#83, isEmpty#84] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#85] +Input [10]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, sum#135, isEmpty#136] +Arguments: hashpartitioning(c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, 5), ENSURE_REQUIREMENTS, [id=#137] (92) HashAggregate [codegen id : 19] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#83, isEmpty#84] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 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#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86] -Results [2]: [c_customer_id#2 AS customer_id#87, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#86 AS year_total#88] +Input [10]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132, sum#135, isEmpty#136] +Keys [8]: [c_customer_id#117, c_first_name#118, c_last_name#119, c_preferred_cust_flag#120, c_birth_country#121, c_login#122, c_email_address#123, d_year#132] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 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#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138] +Results [2]: [c_customer_id#117 AS customer_id#139, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#128 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#127 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#125 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#126 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#138 AS year_total#140] (93) Filter [codegen id : 19] -Input [2]: [customer_id#87, year_total#88] -Condition : (isnotnull(year_total#88) AND (year_total#88 > 0.000000)) +Input [2]: [customer_id#139, year_total#140] +Condition : (isnotnull(year_total#140) AND (year_total#140 > 0.000000)) (94) Project [codegen id : 19] -Output [2]: [customer_id#87 AS customer_id#89, year_total#88 AS year_total#90] -Input [2]: [customer_id#87, year_total#88] +Output [2]: [customer_id#139 AS customer_id#141, year_total#140 AS year_total#142] +Input [2]: [customer_id#139, year_total#140] (95) BroadcastExchange -Input [2]: [customer_id#89, year_total#90] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#91] +Input [2]: [customer_id#141, year_total#142] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#143] (96) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#89] +Right keys [1]: [customer_id#141] Join condition: None (97) Project [codegen id : 24] -Output [11]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72, year_total#90] -Input [12]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72, customer_id#89, year_total#90] +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#88, year_total#114, year_total#142] +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#88, year_total#114, customer_id#141, year_total#142] (98) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (99) ColumnarToRow [codegen id : 22] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] (100) Filter [codegen id : 22] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151] +Condition : (isnotnull(c_customer_sk#144) AND isnotnull(c_customer_id#145)) (101) Scan parquet default.web_sales -Output [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Output [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#79), dynamicpruningexpression(ws_sold_date_sk#79 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(ws_sold_date_sk#157), dynamicpruningexpression(ws_sold_date_sk#157 IN dynamicpruning#42)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 20] -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] (103) Filter [codegen id : 20] -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Condition : isnotnull(ws_bill_customer_sk#74) +Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Condition : isnotnull(ws_bill_customer_sk#152) (104) BroadcastExchange -Input [6]: [ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] +Input [6]: [ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#158] (105) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#74] +Left keys [1]: [c_customer_sk#144] +Right keys [1]: [ws_bill_customer_sk#152] Join condition: None (106) Project [codegen id : 22] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] -Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#74, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79] +Output [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] +Input [14]: [c_customer_sk#144, c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_bill_customer_sk#152, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157] (107) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#159, d_year#160] (108) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ws_sold_date_sk#79] -Right keys [1]: [d_date_sk#17] +Left keys [1]: [ws_sold_date_sk#157] +Right keys [1]: [d_date_sk#159] Join condition: None (109) Project [codegen id : 22] -Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] -Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, ws_sold_date_sk#79, d_date_sk#17, d_year#18] +Output [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, d_year#160] +Input [14]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, ws_sold_date_sk#157, d_date_sk#159, d_year#160] (110) HashAggregate [codegen id : 22] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#75, ws_ext_sales_price#76, ws_ext_wholesale_cost#77, ws_ext_list_price#78, d_year#18] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] -Aggregate Attributes [2]: [sum#93, isEmpty#94] -Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#95, isEmpty#96] +Input [12]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, ws_ext_discount_amt#153, ws_ext_sales_price#154, ws_ext_wholesale_cost#155, ws_ext_list_price#156, d_year#160] +Keys [8]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160] +Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))] +Aggregate Attributes [2]: [sum#161, isEmpty#162] +Results [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] (111) Exchange -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#95, isEmpty#96] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#97] +Input [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] +Arguments: hashpartitioning(c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, 5), ENSURE_REQUIREMENTS, [id=#165] (112) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18, sum#95, isEmpty#96] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#18] -Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 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#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#98] -Results [2]: [c_customer_id#2 AS customer_id#99, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#78 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#77 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#75 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#76 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#98 AS year_total#100] +Input [10]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160, sum#163, isEmpty#164] +Keys [8]: [c_customer_id#145, c_first_name#146, c_last_name#147, c_preferred_cust_flag#148, c_birth_country#149, c_login#150, c_email_address#151, d_year#160] +Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 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#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166] +Results [2]: [c_customer_id#145 AS customer_id#167, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#156 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#155 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#153 as decimal(9,2)))), DecimalType(9,2), true) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#154 as decimal(10,2)))), DecimalType(10,2), true)) / 2.00), DecimalType(14,6), true))#166 AS year_total#168] (113) BroadcastExchange -Input [2]: [customer_id#99, year_total#100] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#101] +Input [2]: [customer_id#167, year_total#168] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#169] (114) BroadcastHashJoin [codegen id : 24] Left keys [1]: [customer_id#26] -Right keys [1]: [customer_id#99] -Join condition: (CASE WHEN (year_total#62 > 0.000000) THEN CheckOverflow((promote_precision(year_total#72) / promote_precision(year_total#62)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#90 > 0.000000) THEN CheckOverflow((promote_precision(year_total#100) / promote_precision(year_total#90)), DecimalType(38,14), true) ELSE null END) +Right keys [1]: [customer_id#167] +Join condition: (CASE WHEN (year_total#88 > 0.000000) THEN CheckOverflow((promote_precision(year_total#114) / promote_precision(year_total#88)), DecimalType(38,14), true) ELSE null END > CASE WHEN (year_total#142 > 0.000000) THEN CheckOverflow((promote_precision(year_total#168) / promote_precision(year_total#142)), DecimalType(38,14), true) ELSE null END) (115) Project [codegen id : 24] -Output [7]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43] -Input [13]: [customer_id#26, customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43, year_total#62, year_total#72, year_total#90, customer_id#99, year_total#100] +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#88, year_total#114, year_total#142, customer_id#167, year_total#168] (116) TakeOrderedAndProject -Input [7]: [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43] -Arguments: 100, [customer_id#37 ASC NULLS FIRST, customer_first_name#38 ASC NULLS FIRST, customer_last_name#39 ASC NULLS FIRST, customer_preferred_cust_flag#40 ASC NULLS FIRST, customer_birth_country#41 ASC NULLS FIRST, customer_login#42 ASC NULLS FIRST, customer_email_address#43 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39, customer_preferred_cust_flag#40, customer_birth_country#41, customer_login#42, customer_email_address#43] +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] ===== Subqueries ===== @@ -673,19 +673,19 @@ ReusedExchange (117) (117) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#17, d_year#18] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#28 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#41 IN dynamicpruning#42 ReusedExchange (118) (118) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#17, d_year#18] +Output [2]: [d_date_sk#44, d_year#45] -Subquery:3 Hosting operator id = 43 Hosting Expression = cs_sold_date_sk#51 IN dynamicpruning#15 +Subquery:3 Hosting operator id = 43 Hosting Expression = cs_sold_date_sk#75 IN dynamicpruning#15 -Subquery:4 Hosting operator id = 63 Hosting Expression = cs_sold_date_sk#51 IN dynamicpruning#28 +Subquery:4 Hosting operator id = 63 Hosting Expression = cs_sold_date_sk#103 IN dynamicpruning#42 -Subquery:5 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#15 +Subquery:5 Hosting operator id = 81 Hosting Expression = ws_sold_date_sk#129 IN dynamicpruning#15 -Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#79 IN dynamicpruning#28 +Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#157 IN dynamicpruning#42 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt index fdb95bd5de9d3..84a8547e3f6ef 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt @@ -102,98 +102,98 @@ Output [2]: [item_sk#11, rnk#17] Input [3]: [item_sk#11, rank_col#12, rnk#17] (15) ReusedExchange [Reuses operator id: 6] -Output [3]: [ss_item_sk#1, sum#18, count#19] +Output [3]: [ss_item_sk#18, sum#19, count#20] (16) HashAggregate [codegen id : 5] -Input [3]: [ss_item_sk#1, sum#18, count#19] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#20] -Results [3]: [ss_item_sk#1 AS item_sk#21, cast((avg(UnscaledValue(ss_net_profit#3))#20 / 100.0) as decimal(11,6)) AS rank_col#22, cast((avg(UnscaledValue(ss_net_profit#3))#20 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#3)#23] +Input [3]: [ss_item_sk#18, sum#19, count#20] +Keys [1]: [ss_item_sk#18] +Functions [1]: [avg(UnscaledValue(ss_net_profit#21))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#21))#22] +Results [3]: [ss_item_sk#18 AS item_sk#23, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS rank_col#24, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#21)#25] (17) Filter [codegen id : 5] -Input [3]: [item_sk#21, rank_col#22, avg(ss_net_profit#3)#23] -Condition : (isnotnull(avg(ss_net_profit#3)#23) AND (cast(avg(ss_net_profit#3)#23 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) +Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] +Condition : (isnotnull(avg(ss_net_profit#21)#25) AND (cast(avg(ss_net_profit#21)#25 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) (18) Project [codegen id : 5] -Output [2]: [item_sk#21, rank_col#22] -Input [3]: [item_sk#21, rank_col#22, avg(ss_net_profit#3)#23] +Output [2]: [item_sk#23, rank_col#24] +Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] (19) Exchange -Input [2]: [item_sk#21, rank_col#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [2]: [item_sk#23, rank_col#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] (20) Sort [codegen id : 6] -Input [2]: [item_sk#21, rank_col#22] -Arguments: [rank_col#22 DESC NULLS LAST], false, 0 +Input [2]: [item_sk#23, rank_col#24] +Arguments: [rank_col#24 DESC NULLS LAST], false, 0 (21) Window -Input [2]: [item_sk#21, rank_col#22] -Arguments: [rank(rank_col#22) windowspecdefinition(rank_col#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#25], [rank_col#22 DESC NULLS LAST] +Input [2]: [item_sk#23, rank_col#24] +Arguments: [rank(rank_col#24) windowspecdefinition(rank_col#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#27], [rank_col#24 DESC NULLS LAST] (22) Filter [codegen id : 7] -Input [3]: [item_sk#21, rank_col#22, rnk#25] -Condition : ((rnk#25 < 11) AND isnotnull(item_sk#21)) +Input [3]: [item_sk#23, rank_col#24, rnk#27] +Condition : ((rnk#27 < 11) AND isnotnull(item_sk#23)) (23) Project [codegen id : 7] -Output [2]: [item_sk#21, rnk#25] -Input [3]: [item_sk#21, rank_col#22, rnk#25] +Output [2]: [item_sk#23, rnk#27] +Input [3]: [item_sk#23, rank_col#24, rnk#27] (24) BroadcastExchange -Input [2]: [item_sk#21, rnk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#26] +Input [2]: [item_sk#23, rnk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [id=#28] (25) BroadcastHashJoin [codegen id : 10] Left keys [1]: [rnk#17] -Right keys [1]: [rnk#25] +Right keys [1]: [rnk#27] Join condition: None (26) Project [codegen id : 10] -Output [3]: [item_sk#11, rnk#17, item_sk#21] -Input [4]: [item_sk#11, rnk#17, item_sk#21, rnk#25] +Output [3]: [item_sk#11, rnk#17, item_sk#23] +Input [4]: [item_sk#11, rnk#17, item_sk#23, rnk#27] (27) Scan parquet default.item -Output [2]: [i_item_sk#27, i_product_name#28] +Output [2]: [i_item_sk#29, i_product_name#30] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 8] -Input [2]: [i_item_sk#27, i_product_name#28] +Input [2]: [i_item_sk#29, i_product_name#30] (29) Filter [codegen id : 8] -Input [2]: [i_item_sk#27, i_product_name#28] -Condition : isnotnull(i_item_sk#27) +Input [2]: [i_item_sk#29, i_product_name#30] +Condition : isnotnull(i_item_sk#29) (30) BroadcastExchange -Input [2]: [i_item_sk#27, i_product_name#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [2]: [i_item_sk#29, i_product_name#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [item_sk#11] -Right keys [1]: [i_item_sk#27] +Right keys [1]: [i_item_sk#29] Join condition: None (32) Project [codegen id : 10] -Output [3]: [rnk#17, item_sk#21, i_product_name#28] -Input [5]: [item_sk#11, rnk#17, item_sk#21, i_item_sk#27, i_product_name#28] +Output [3]: [rnk#17, item_sk#23, i_product_name#30] +Input [5]: [item_sk#11, rnk#17, item_sk#23, i_item_sk#29, i_product_name#30] (33) ReusedExchange [Reuses operator id: 30] -Output [2]: [i_item_sk#30, i_product_name#31] +Output [2]: [i_item_sk#32, i_product_name#33] (34) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [item_sk#21] -Right keys [1]: [i_item_sk#30] +Left keys [1]: [item_sk#23] +Right keys [1]: [i_item_sk#32] Join condition: None (35) Project [codegen id : 10] -Output [3]: [rnk#17, i_product_name#28 AS best_performing#32, i_product_name#31 AS worst_performing#33] -Input [5]: [rnk#17, item_sk#21, i_product_name#28, i_item_sk#30, i_product_name#31] +Output [3]: [rnk#17, i_product_name#30 AS best_performing#34, i_product_name#33 AS worst_performing#35] +Input [5]: [rnk#17, item_sk#23, i_product_name#30, i_item_sk#32, i_product_name#33] (36) TakeOrderedAndProject -Input [3]: [rnk#17, best_performing#32, worst_performing#33] -Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#32, worst_performing#33] +Input [3]: [rnk#17, best_performing#34, worst_performing#35] +Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#34, worst_performing#35] ===== Subqueries ===== @@ -208,40 +208,40 @@ Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery (37) Scan parquet default.store_sales -Output [4]: [ss_addr_sk#34, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Output [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 1] -Input [4]: [ss_addr_sk#34, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] (39) Filter [codegen id : 1] -Input [4]: [ss_addr_sk#34, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) AND isnull(ss_addr_sk#34)) +Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] +Condition : ((isnotnull(ss_store_sk#37) AND (ss_store_sk#37 = 4)) AND isnull(ss_addr_sk#36)) (40) Project [codegen id : 1] -Output [2]: [ss_store_sk#2, ss_net_profit#3] -Input [4]: [ss_addr_sk#34, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Output [2]: [ss_store_sk#37, ss_net_profit#38] +Input [4]: [ss_addr_sk#36, ss_store_sk#37, ss_net_profit#38, ss_sold_date_sk#39] (41) HashAggregate [codegen id : 1] -Input [2]: [ss_store_sk#2, ss_net_profit#3] -Keys [1]: [ss_store_sk#2] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#35, count#36] -Results [3]: [ss_store_sk#2, sum#37, count#38] +Input [2]: [ss_store_sk#37, ss_net_profit#38] +Keys [1]: [ss_store_sk#37] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#38))] +Aggregate Attributes [2]: [sum#40, count#41] +Results [3]: [ss_store_sk#37, sum#42, count#43] (42) Exchange -Input [3]: [ss_store_sk#2, sum#37, count#38] -Arguments: hashpartitioning(ss_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [3]: [ss_store_sk#37, sum#42, count#43] +Arguments: hashpartitioning(ss_store_sk#37, 5), ENSURE_REQUIREMENTS, [id=#44] (43) HashAggregate [codegen id : 2] -Input [3]: [ss_store_sk#2, sum#37, count#38] -Keys [1]: [ss_store_sk#2] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#40] -Results [1]: [cast((avg(UnscaledValue(ss_net_profit#3))#40 / 100.0) as decimal(11,6)) AS rank_col#41] +Input [3]: [ss_store_sk#37, sum#42, count#43] +Keys [1]: [ss_store_sk#37] +Functions [1]: [avg(UnscaledValue(ss_net_profit#38))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#38))#45] +Results [1]: [cast((avg(UnscaledValue(ss_net_profit#38))#45 / 100.0) as decimal(11,6)) AS rank_col#46] Subquery:2 Hosting operator id = 17 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index f52ad2e63762e..b42ddd8db7836 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -107,98 +107,98 @@ Input [2]: [item_sk#11, rnk#17] Arguments: [rnk#17 ASC NULLS FIRST], false, 0 (16) ReusedExchange [Reuses operator id: 6] -Output [3]: [ss_item_sk#1, sum#18, count#19] +Output [3]: [ss_item_sk#18, sum#19, count#20] (17) HashAggregate [codegen id : 6] -Input [3]: [ss_item_sk#1, sum#18, count#19] -Keys [1]: [ss_item_sk#1] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#20] -Results [3]: [ss_item_sk#1 AS item_sk#21, cast((avg(UnscaledValue(ss_net_profit#3))#20 / 100.0) as decimal(11,6)) AS rank_col#22, cast((avg(UnscaledValue(ss_net_profit#3))#20 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#3)#23] +Input [3]: [ss_item_sk#18, sum#19, count#20] +Keys [1]: [ss_item_sk#18] +Functions [1]: [avg(UnscaledValue(ss_net_profit#21))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#21))#22] +Results [3]: [ss_item_sk#18 AS item_sk#23, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS rank_col#24, cast((avg(UnscaledValue(ss_net_profit#21))#22 / 100.0) as decimal(11,6)) AS avg(ss_net_profit#21)#25] (18) Filter [codegen id : 6] -Input [3]: [item_sk#21, rank_col#22, avg(ss_net_profit#3)#23] -Condition : (isnotnull(avg(ss_net_profit#3)#23) AND (cast(avg(ss_net_profit#3)#23 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) +Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] +Condition : (isnotnull(avg(ss_net_profit#21)#25) AND (cast(avg(ss_net_profit#21)#25 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(ReusedSubquery Subquery scalar-subquery#14, [id=#15])), DecimalType(13,7), true))) (19) Project [codegen id : 6] -Output [2]: [item_sk#21, rank_col#22] -Input [3]: [item_sk#21, rank_col#22, avg(ss_net_profit#3)#23] +Output [2]: [item_sk#23, rank_col#24] +Input [3]: [item_sk#23, rank_col#24, avg(ss_net_profit#21)#25] (20) Exchange -Input [2]: [item_sk#21, rank_col#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [2]: [item_sk#23, rank_col#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#26] (21) Sort [codegen id : 7] -Input [2]: [item_sk#21, rank_col#22] -Arguments: [rank_col#22 DESC NULLS LAST], false, 0 +Input [2]: [item_sk#23, rank_col#24] +Arguments: [rank_col#24 DESC NULLS LAST], false, 0 (22) Window -Input [2]: [item_sk#21, rank_col#22] -Arguments: [rank(rank_col#22) windowspecdefinition(rank_col#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#25], [rank_col#22 DESC NULLS LAST] +Input [2]: [item_sk#23, rank_col#24] +Arguments: [rank(rank_col#24) windowspecdefinition(rank_col#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#27], [rank_col#24 DESC NULLS LAST] (23) Filter [codegen id : 8] -Input [3]: [item_sk#21, rank_col#22, rnk#25] -Condition : ((rnk#25 < 11) AND isnotnull(item_sk#21)) +Input [3]: [item_sk#23, rank_col#24, rnk#27] +Condition : ((rnk#27 < 11) AND isnotnull(item_sk#23)) (24) Project [codegen id : 8] -Output [2]: [item_sk#21, rnk#25] -Input [3]: [item_sk#21, rank_col#22, rnk#25] +Output [2]: [item_sk#23, rnk#27] +Input [3]: [item_sk#23, rank_col#24, rnk#27] (25) Sort [codegen id : 8] -Input [2]: [item_sk#21, rnk#25] -Arguments: [rnk#25 ASC NULLS FIRST], false, 0 +Input [2]: [item_sk#23, rnk#27] +Arguments: [rnk#27 ASC NULLS FIRST], false, 0 (26) SortMergeJoin [codegen id : 11] Left keys [1]: [rnk#17] -Right keys [1]: [rnk#25] +Right keys [1]: [rnk#27] Join condition: None (27) Project [codegen id : 11] -Output [3]: [item_sk#11, rnk#17, item_sk#21] -Input [4]: [item_sk#11, rnk#17, item_sk#21, rnk#25] +Output [3]: [item_sk#11, rnk#17, item_sk#23] +Input [4]: [item_sk#11, rnk#17, item_sk#23, rnk#27] (28) Scan parquet default.item -Output [2]: [i_item_sk#26, i_product_name#27] +Output [2]: [i_item_sk#28, i_product_name#29] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 9] -Input [2]: [i_item_sk#26, i_product_name#27] +Input [2]: [i_item_sk#28, i_product_name#29] (30) Filter [codegen id : 9] -Input [2]: [i_item_sk#26, i_product_name#27] -Condition : isnotnull(i_item_sk#26) +Input [2]: [i_item_sk#28, i_product_name#29] +Condition : isnotnull(i_item_sk#28) (31) BroadcastExchange -Input [2]: [i_item_sk#26, i_product_name#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] +Input [2]: [i_item_sk#28, i_product_name#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (32) BroadcastHashJoin [codegen id : 11] Left keys [1]: [item_sk#11] -Right keys [1]: [i_item_sk#26] +Right keys [1]: [i_item_sk#28] Join condition: None (33) Project [codegen id : 11] -Output [3]: [rnk#17, item_sk#21, i_product_name#27] -Input [5]: [item_sk#11, rnk#17, item_sk#21, i_item_sk#26, i_product_name#27] +Output [3]: [rnk#17, item_sk#23, i_product_name#29] +Input [5]: [item_sk#11, rnk#17, item_sk#23, i_item_sk#28, i_product_name#29] (34) ReusedExchange [Reuses operator id: 31] -Output [2]: [i_item_sk#29, i_product_name#30] +Output [2]: [i_item_sk#31, i_product_name#32] (35) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [item_sk#21] -Right keys [1]: [i_item_sk#29] +Left keys [1]: [item_sk#23] +Right keys [1]: [i_item_sk#31] Join condition: None (36) Project [codegen id : 11] -Output [3]: [rnk#17, i_product_name#27 AS best_performing#31, i_product_name#30 AS worst_performing#32] -Input [5]: [rnk#17, item_sk#21, i_product_name#27, i_item_sk#29, i_product_name#30] +Output [3]: [rnk#17, i_product_name#29 AS best_performing#33, i_product_name#32 AS worst_performing#34] +Input [5]: [rnk#17, item_sk#23, i_product_name#29, i_item_sk#31, i_product_name#32] (37) TakeOrderedAndProject -Input [3]: [rnk#17, best_performing#31, worst_performing#32] -Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#31, worst_performing#32] +Input [3]: [rnk#17, best_performing#33, worst_performing#34] +Arguments: 100, [rnk#17 ASC NULLS FIRST], [rnk#17, best_performing#33, worst_performing#34] ===== Subqueries ===== @@ -213,40 +213,40 @@ Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery (38) Scan parquet default.store_sales -Output [4]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Output [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 1] -Input [4]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] (40) Filter [codegen id : 1] -Input [4]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) AND isnull(ss_addr_sk#33)) +Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_store_sk#36) AND (ss_store_sk#36 = 4)) AND isnull(ss_addr_sk#35)) (41) Project [codegen id : 1] -Output [2]: [ss_store_sk#2, ss_net_profit#3] -Input [4]: [ss_addr_sk#33, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Output [2]: [ss_store_sk#36, ss_net_profit#37] +Input [4]: [ss_addr_sk#35, ss_store_sk#36, ss_net_profit#37, ss_sold_date_sk#38] (42) HashAggregate [codegen id : 1] -Input [2]: [ss_store_sk#2, ss_net_profit#3] -Keys [1]: [ss_store_sk#2] -Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [2]: [sum#34, count#35] -Results [3]: [ss_store_sk#2, sum#36, count#37] +Input [2]: [ss_store_sk#36, ss_net_profit#37] +Keys [1]: [ss_store_sk#36] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#37))] +Aggregate Attributes [2]: [sum#39, count#40] +Results [3]: [ss_store_sk#36, sum#41, count#42] (43) Exchange -Input [3]: [ss_store_sk#2, sum#36, count#37] -Arguments: hashpartitioning(ss_store_sk#2, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [3]: [ss_store_sk#36, sum#41, count#42] +Arguments: hashpartitioning(ss_store_sk#36, 5), ENSURE_REQUIREMENTS, [id=#43] (44) HashAggregate [codegen id : 2] -Input [3]: [ss_store_sk#2, sum#36, count#37] -Keys [1]: [ss_store_sk#2] -Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#39] -Results [1]: [cast((avg(UnscaledValue(ss_net_profit#3))#39 / 100.0) as decimal(11,6)) AS rank_col#40] +Input [3]: [ss_store_sk#36, sum#41, count#42] +Keys [1]: [ss_store_sk#36] +Functions [1]: [avg(UnscaledValue(ss_net_profit#37))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#37))#44] +Results [1]: [cast((avg(UnscaledValue(ss_net_profit#37))#44 / 100.0) as decimal(11,6)) AS rank_col#45] Subquery:2 Hosting operator id = 18 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt index 22f746eb1d91a..701414b22eb80 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt @@ -199,30 +199,30 @@ Output [4]: [ws_sales_price#4, ca_city#19, ca_zip#20, i_item_id#12] Input [6]: [ws_bill_customer_sk#3, ws_sales_price#4, i_item_id#12, c_customer_sk#15, ca_city#19, ca_zip#20] (35) Scan parquet default.item -Output [2]: [i_item_sk#11, i_item_id#12] +Output [2]: [i_item_sk#23, i_item_id#24] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_item_sk, [2,3,5,7,11,13,17,19,23,29])] ReadSchema: struct (36) ColumnarToRow [codegen id : 11] -Input [2]: [i_item_sk#11, i_item_id#12] +Input [2]: [i_item_sk#23, i_item_id#24] (37) Filter [codegen id : 11] -Input [2]: [i_item_sk#11, i_item_id#12] -Condition : i_item_sk#11 IN (2,3,5,7,11,13,17,19,23,29) +Input [2]: [i_item_sk#23, i_item_id#24] +Condition : i_item_sk#23 IN (2,3,5,7,11,13,17,19,23,29) (38) Project [codegen id : 11] -Output [1]: [i_item_id#12 AS i_item_id#12#23] -Input [2]: [i_item_sk#11, i_item_id#12] +Output [1]: [i_item_id#24] +Input [2]: [i_item_sk#23, i_item_id#24] (39) BroadcastExchange -Input [1]: [i_item_id#12#23] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#24] +Input [1]: [i_item_id#24] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#25] (40) BroadcastHashJoin [codegen id : 12] Left keys [1]: [i_item_id#12] -Right keys [1]: [i_item_id#12#23] +Right keys [1]: [i_item_id#24] Join condition: None (41) Filter [codegen id : 12] @@ -237,23 +237,23 @@ Input [5]: [ws_sales_price#4, ca_city#19, ca_zip#20, i_item_id#12, exists#1] Input [3]: [ws_sales_price#4, ca_city#19, ca_zip#20] Keys [2]: [ca_zip#20, ca_city#19] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#25] -Results [3]: [ca_zip#20, ca_city#19, sum#26] +Aggregate Attributes [1]: [sum#26] +Results [3]: [ca_zip#20, ca_city#19, sum#27] (44) Exchange -Input [3]: [ca_zip#20, ca_city#19, sum#26] -Arguments: hashpartitioning(ca_zip#20, ca_city#19, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ca_zip#20, ca_city#19, sum#27] +Arguments: hashpartitioning(ca_zip#20, ca_city#19, 5), ENSURE_REQUIREMENTS, [id=#28] (45) HashAggregate [codegen id : 13] -Input [3]: [ca_zip#20, ca_city#19, sum#26] +Input [3]: [ca_zip#20, ca_city#19, sum#27] Keys [2]: [ca_zip#20, ca_city#19] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#28] -Results [3]: [ca_zip#20, ca_city#19, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#28,17,2) AS sum(ws_sales_price)#29] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#29] +Results [3]: [ca_zip#20, ca_city#19, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#29,17,2) AS sum(ws_sales_price)#30] (46) TakeOrderedAndProject -Input [3]: [ca_zip#20, ca_city#19, sum(ws_sales_price)#29] -Arguments: 100, [ca_zip#20 ASC NULLS FIRST, ca_city#19 ASC NULLS FIRST], [ca_zip#20, ca_city#19, sum(ws_sales_price)#29] +Input [3]: [ca_zip#20, ca_city#19, sum(ws_sales_price)#30] +Arguments: 100, [ca_zip#20 ASC NULLS FIRST, ca_city#19 ASC NULLS FIRST], [ca_zip#20, ca_city#19, sum(ws_sales_price)#30] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt index b81498ddcea34..f3a37f9e8767e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt @@ -169,30 +169,30 @@ Output [4]: [ws_sales_price#4, ca_city#11, ca_zip#12, i_item_id#19] Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#11, ca_zip#12, i_item_sk#18, i_item_id#19] (29) Scan parquet default.item -Output [2]: [i_item_sk#18, i_item_id#19] +Output [2]: [i_item_sk#21, i_item_id#22] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_item_sk, [2,3,5,7,11,13,17,19,23,29])] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] -Input [2]: [i_item_sk#18, i_item_id#19] +Input [2]: [i_item_sk#21, i_item_id#22] (31) Filter [codegen id : 5] -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : i_item_sk#18 IN (2,3,5,7,11,13,17,19,23,29) +Input [2]: [i_item_sk#21, i_item_id#22] +Condition : i_item_sk#21 IN (2,3,5,7,11,13,17,19,23,29) (32) Project [codegen id : 5] -Output [1]: [i_item_id#19 AS i_item_id#19#21] -Input [2]: [i_item_sk#18, i_item_id#19] +Output [1]: [i_item_id#22] +Input [2]: [i_item_sk#21, i_item_id#22] (33) BroadcastExchange -Input [1]: [i_item_id#19#21] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#22] +Input [1]: [i_item_id#22] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#23] (34) BroadcastHashJoin [codegen id : 6] Left keys [1]: [i_item_id#19] -Right keys [1]: [i_item_id#19#21] +Right keys [1]: [i_item_id#22] Join condition: None (35) Filter [codegen id : 6] @@ -207,23 +207,23 @@ Input [5]: [ws_sales_price#4, ca_city#11, ca_zip#12, i_item_id#19, exists#1] Input [3]: [ws_sales_price#4, ca_city#11, ca_zip#12] Keys [2]: [ca_zip#12, ca_city#11] Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum#23] -Results [3]: [ca_zip#12, ca_city#11, sum#24] +Aggregate Attributes [1]: [sum#24] +Results [3]: [ca_zip#12, ca_city#11, sum#25] (38) Exchange -Input [3]: [ca_zip#12, ca_city#11, sum#24] -Arguments: hashpartitioning(ca_zip#12, ca_city#11, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [3]: [ca_zip#12, ca_city#11, sum#25] +Arguments: hashpartitioning(ca_zip#12, ca_city#11, 5), ENSURE_REQUIREMENTS, [id=#26] (39) HashAggregate [codegen id : 7] -Input [3]: [ca_zip#12, ca_city#11, sum#24] +Input [3]: [ca_zip#12, ca_city#11, sum#25] Keys [2]: [ca_zip#12, ca_city#11] Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#26] -Results [3]: [ca_zip#12, ca_city#11, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#26,17,2) AS sum(ws_sales_price)#27] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#27] +Results [3]: [ca_zip#12, ca_city#11, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#27,17,2) AS sum(ws_sales_price)#28] (40) TakeOrderedAndProject -Input [3]: [ca_zip#12, ca_city#11, sum(ws_sales_price)#27] -Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#11 ASC NULLS FIRST], [ca_zip#12, ca_city#11, sum(ws_sales_price)#27] +Input [3]: [ca_zip#12, ca_city#11, sum(ws_sales_price)#28] +Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#11 ASC NULLS FIRST], [ca_zip#12, ca_city#11, sum(ws_sales_price)#28] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt index 78ade994364a7..ac35640c7b004 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt @@ -230,55 +230,55 @@ Input [5]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt Arguments: [ss_addr_sk#12 ASC NULLS FIRST], false, 0 (41) ReusedExchange [Reuses operator id: 9] -Output [2]: [ca_address_sk#6, ca_city#7] +Output [2]: [ca_address_sk#31, ca_city#32] (42) Sort [codegen id : 13] -Input [2]: [ca_address_sk#6, ca_city#7] -Arguments: [ca_address_sk#6 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#31, ca_city#32] +Arguments: [ca_address_sk#31 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 14] Left keys [1]: [ss_addr_sk#12] -Right keys [1]: [ca_address_sk#6] +Right keys [1]: [ca_address_sk#31] Join condition: None (44) Project [codegen id : 14] -Output [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#7] -Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_address_sk#6, ca_city#7] +Output [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#32] +Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_address_sk#31, ca_city#32] (45) HashAggregate [codegen id : 14] -Input [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#7] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] +Input [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_coupon_amt#15, ss_net_profit#16, ca_city#32] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32] Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#15)), partial_sum(UnscaledValue(ss_net_profit#16))] -Aggregate Attributes [2]: [sum#31, sum#32] -Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#33, sum#34] +Aggregate Attributes [2]: [sum#33, sum#34] +Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32, sum#35, sum#36] (46) HashAggregate [codegen id : 14] -Input [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#33, sum#34] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] +Input [6]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32, sum#35, sum#36] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#32] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#15)), sum(UnscaledValue(ss_net_profit#16))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#15))#35, sum(UnscaledValue(ss_net_profit#16))#36] -Results [5]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#7 AS bought_city#37, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#15))#35,17,2) AS amt#38, MakeDecimal(sum(UnscaledValue(ss_net_profit#16))#36,17,2) AS profit#39] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#15))#37, sum(UnscaledValue(ss_net_profit#16))#38] +Results [5]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#32 AS bought_city#39, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#15))#37,17,2) AS amt#40, MakeDecimal(sum(UnscaledValue(ss_net_profit#16))#38,17,2) AS profit#41] (47) Exchange -Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#37, amt#38, profit#39] -Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#39, amt#40, profit#41] +Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#42] (48) Sort [codegen id : 15] -Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#37, amt#38, profit#39] +Input [5]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#39, amt#40, profit#41] Arguments: [ss_customer_sk#10 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 16] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#10] -Join condition: NOT (ca_city#7 = bought_city#37) +Join condition: NOT (ca_city#7 = bought_city#39) (50) Project [codegen id : 16] -Output [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#37, ss_ticket_number#14, amt#38, profit#39] -Input [9]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#37, amt#38, profit#39] +Output [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#39, ss_ticket_number#14, amt#40, profit#41] +Input [9]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#39, amt#40, profit#41] (51) TakeOrderedAndProject -Input [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#37, ss_ticket_number#14, amt#38, profit#39] -Arguments: 100, [c_last_name#4 ASC NULLS FIRST, c_first_name#3 ASC NULLS FIRST, ca_city#7 ASC NULLS FIRST, bought_city#37 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#37, ss_ticket_number#14, amt#38, profit#39] +Input [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#39, ss_ticket_number#14, amt#40, profit#41] +Arguments: 100, [c_last_name#4 ASC NULLS FIRST, c_first_name#3 ASC NULLS FIRST, ca_city#7 ASC NULLS FIRST, bought_city#39 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#39, ss_ticket_number#14, amt#40, profit#41] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt index e9bc22389d8a6..2532a14d7907f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt @@ -225,20 +225,20 @@ Output [7]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_ad Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#31, amt#32, profit#33, c_customer_sk#34, c_current_addr_sk#35, c_first_name#36, c_last_name#37] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#21, ca_city#22] +Output [2]: [ca_address_sk#39, ca_city#40] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#21] -Join condition: NOT (ca_city#22 = bought_city#31) +Right keys [1]: [ca_address_sk#39] +Join condition: NOT (ca_city#40 = bought_city#31) (42) Project [codegen id : 8] -Output [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#21, ca_city#22] +Output [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [9]: [ss_ticket_number#5, bought_city#31, amt#32, profit#33, c_current_addr_sk#35, c_first_name#36, c_last_name#37, ca_address_sk#39, ca_city#40] (43) TakeOrderedAndProject -Input [7]: [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] -Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#22 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#22, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Input [7]: [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] +Arguments: 100, [c_last_name#37 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, ca_city#40 ASC NULLS FIRST, bought_city#31 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#37, c_first_name#36, ca_city#40, bought_city#31, ss_ticket_number#5, amt#32, profit#33] ===== Subqueries ===== 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 dc19996f9957b..b7ffba01f92c6 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 @@ -229,78 +229,78 @@ Output [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_ye (39) HashAggregate [codegen id : 21] Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum#36] Keys [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35] -Functions [1]: [sum(UnscaledValue(ss_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#3))#37] -Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#3))#37,17,2) AS sum_sales#38] +Functions [1]: [sum(UnscaledValue(ss_sales_price#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#37))#38] +Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#37))#38,17,2) AS sum_sales#39] (40) Exchange -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#40] (41) Sort [codegen id : 22] -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST], false, 0 (42) Window -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] (43) Project [codegen id : 23] -Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38, rn#40] +Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] (44) Exchange -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1), 5), ENSURE_REQUIREMENTS, [id=#41] +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1), 5), ENSURE_REQUIREMENTS, [id=#42] (45) Sort [codegen id : 24] -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#40 + 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#41 + 1) ASC NULLS FIRST], false, 0 (46) SortMergeJoin [codegen id : 25] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1)] +Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1)] Join condition: None (47) Project [codegen id : 25] -Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38] -Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] +Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39] +Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] (48) ReusedExchange [Reuses operator id: 40] -Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Output [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] (49) Sort [codegen id : 34] -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] +Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST], false, 0 (50) Window -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] +Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] +Arguments: [rank(d_year#47, d_moy#48) windowspecdefinition(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#50], [i_category#43, i_brand#44, s_store_name#45, s_company_name#46], [d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST] (51) Project [codegen id : 35] -Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] +Output [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] (52) Exchange -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1), 5), ENSURE_REQUIREMENTS, [id=#50] +Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Arguments: hashpartitioning(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1), 5), ENSURE_REQUIREMENTS, [id=#51] (53) Sort [codegen id : 36] -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (rn#49 - 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, (rn#50 - 1) ASC NULLS FIRST], false, 0 (54) SortMergeJoin [codegen id : 37] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1)] +Right keys [5]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1)] Join condition: None (55) Project [codegen id : 37] -Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#38 AS psum#51, sum_sales#48 AS nsum#52] -Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] +Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#39 AS psum#52, sum_sales#49 AS nsum#53] +Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39, i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] (56) TakeOrderedAndProject -Input [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#11 ASC NULLS FIRST], [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] +Input [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#11 ASC NULLS FIRST], [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] ===== Subqueries ===== 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 3d58d26e6a3a4..917b21c247449 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 @@ -202,70 +202,70 @@ Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_ye (34) HashAggregate [codegen id : 13] 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#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#35] -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#6))#35,17,2) AS sum_sales#36] +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] (35) 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#36] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#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#37] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] (36) Sort [codegen id : 14] -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36] +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: [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 (37) 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#36] -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#38], [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#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] (38) Project [codegen id : 15] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] -Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] +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] (39) BroadcastExchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] -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=#39] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +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=#40] (40) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#38 + 1)] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] Join condition: None (41) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] -Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] +Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] +Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] (42) ReusedExchange [Reuses operator id: 35] -Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] +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] (43) Sort [codegen id : 21] -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#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, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 +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 (44) Window -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] -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#47], [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] +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] (45) Project [codegen id : 22] -Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] -Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] +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] (46) BroadcastExchange -Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] -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=#48] +Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +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=#49] (47) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -Right keys [5]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#47 - 1)] +Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] Join condition: None (48) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#49, sum_sales#46 AS nsum#50] -Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] +Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] +Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] (49) TakeOrderedAndProject -Input [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] +Input [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, psum#50, nsum#51] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt index e93d0e9503493..3f30183af3a94 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt @@ -250,225 +250,225 @@ Output [5]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_net_profit#47, cs_sold_date_sk#48] (36) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#49] (37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_sold_date_sk#48] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#49] Join condition: None (38) Project [codegen id : 12] Output [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#8] +Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#49] (39) Exchange Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#49] +Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#50] (40) Sort [codegen id : 13] Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] Arguments: [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST], false, 0 (41) Scan parquet default.catalog_returns -Output [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Output [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (42) ColumnarToRow [codegen id : 14] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] (43) Filter [codegen id : 14] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] -Condition : (((isnotnull(cr_return_amount#53) AND (cr_return_amount#53 > 10000.00)) AND isnotnull(cr_order_number#51)) AND isnotnull(cr_item_sk#50)) +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Condition : (((isnotnull(cr_return_amount#54) AND (cr_return_amount#54 > 10000.00)) AND isnotnull(cr_order_number#52)) AND isnotnull(cr_item_sk#51)) (44) Project [codegen id : 14] -Output [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Output [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] (45) Exchange -Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Arguments: hashpartitioning(cr_order_number#52, cr_item_sk#51, 5), ENSURE_REQUIREMENTS, [id=#56] (46) Sort [codegen id : 15] -Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Arguments: [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Arguments: [cr_order_number#52 ASC NULLS FIRST, cr_item_sk#51 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 16] Left keys [2]: [cs_order_number#44, cs_item_sk#43] -Right keys [2]: [cr_order_number#51, cr_item_sk#50] +Right keys [2]: [cr_order_number#52, cr_item_sk#51] Join condition: None (48) Project [codegen id : 16] -Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] -Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] +Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] (49) HashAggregate [codegen id : 16] -Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] +Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] Keys [1]: [cs_item_sk#43] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#52, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#56, sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] -Results [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#53, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] +Results [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] (50) Exchange -Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] -Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#69] (51) HashAggregate [codegen id : 17] -Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] Keys [1]: [cs_item_sk#43] -Functions [4]: [sum(coalesce(cr_return_quantity#52, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#52, 0))#69, sum(coalesce(cs_quantity#45, 0))#70, sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72] -Results [3]: [cs_item_sk#43 AS item#73, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#52, 0))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#70 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#74, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#75] +Functions [4]: [sum(coalesce(cr_return_quantity#53, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#53, 0))#70, sum(coalesce(cs_quantity#45, 0))#71, sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73] +Results [3]: [cs_item_sk#43 AS item#74, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#53, 0))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#71 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#75, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#76] (52) Exchange -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#77] (53) Sort [codegen id : 18] -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: [return_ratio#74 ASC NULLS FIRST], false, 0 +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: [return_ratio#75 ASC NULLS FIRST], false, 0 (54) Window -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: [rank(return_ratio#74) windowspecdefinition(return_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#77], [return_ratio#74 ASC NULLS FIRST] +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: [rank(return_ratio#75) windowspecdefinition(return_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#78], [return_ratio#75 ASC NULLS FIRST] (55) Sort [codegen id : 19] -Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] -Arguments: [currency_ratio#75 ASC NULLS FIRST], false, 0 +Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] +Arguments: [currency_ratio#76 ASC NULLS FIRST], false, 0 (56) Window -Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] -Arguments: [rank(currency_ratio#75) windowspecdefinition(currency_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#78], [currency_ratio#75 ASC NULLS FIRST] +Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] +Arguments: [rank(currency_ratio#76) windowspecdefinition(currency_ratio#76 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#79], [currency_ratio#76 ASC NULLS FIRST] (57) Filter [codegen id : 20] -Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] -Condition : ((return_rank#77 <= 10) OR (currency_rank#78 <= 10)) +Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] +Condition : ((return_rank#78 <= 10) OR (currency_rank#79 <= 10)) (58) Project [codegen id : 20] -Output [5]: [catalog AS channel#79, item#73, return_ratio#74, return_rank#77, currency_rank#78] -Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] +Output [5]: [catalog AS channel#80, item#74, return_ratio#75, return_rank#78, currency_rank#79] +Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] (59) Scan parquet default.store_sales -Output [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Output [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_sold_date_sk#85 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 22] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] (61) Filter [codegen id : 22] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] -Condition : (((((((isnotnull(ss_net_profit#84) AND isnotnull(ss_net_paid#83)) AND isnotnull(ss_quantity#82)) AND (ss_net_profit#84 > 1.00)) AND (ss_net_paid#83 > 0.00)) AND (ss_quantity#82 > 0)) AND isnotnull(ss_ticket_number#81)) AND isnotnull(ss_item_sk#80)) +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Condition : (((((((isnotnull(ss_net_profit#85) AND isnotnull(ss_net_paid#84)) AND isnotnull(ss_quantity#83)) AND (ss_net_profit#85 > 1.00)) AND (ss_net_paid#84 > 0.00)) AND (ss_quantity#83 > 0)) AND isnotnull(ss_ticket_number#82)) AND isnotnull(ss_item_sk#81)) (62) Project [codegen id : 22] -Output [5]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Output [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] (63) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#87] (64) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ss_sold_date_sk#85] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ss_sold_date_sk#86] +Right keys [1]: [d_date_sk#87] Join condition: None (65) Project [codegen id : 22] -Output [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85, d_date_sk#8] +Output [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86, d_date_sk#87] (66) Exchange -Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Arguments: hashpartitioning(cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint), 5), ENSURE_REQUIREMENTS, [id=#86] +Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Arguments: hashpartitioning(cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint), 5), ENSURE_REQUIREMENTS, [id=#88] (67) Sort [codegen id : 23] -Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Arguments: [cast(ss_ticket_number#81 as bigint) ASC NULLS FIRST, cast(ss_item_sk#80 as bigint) ASC NULLS FIRST], false, 0 +Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Arguments: [cast(ss_ticket_number#82 as bigint) ASC NULLS FIRST, cast(ss_item_sk#81 as bigint) ASC NULLS FIRST], false, 0 (68) Scan parquet default.store_returns -Output [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Output [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 24] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] (70) Filter [codegen id : 24] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] -Condition : (((isnotnull(sr_return_amt#90) AND (sr_return_amt#90 > 10000.00)) AND isnotnull(sr_ticket_number#88)) AND isnotnull(sr_item_sk#87)) +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Condition : (((isnotnull(sr_return_amt#92) AND (sr_return_amt#92 > 10000.00)) AND isnotnull(sr_ticket_number#90)) AND isnotnull(sr_item_sk#89)) (71) Project [codegen id : 24] -Output [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Output [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] (72) Exchange -Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Arguments: hashpartitioning(sr_ticket_number#88, sr_item_sk#87, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Arguments: hashpartitioning(sr_ticket_number#90, sr_item_sk#89, 5), ENSURE_REQUIREMENTS, [id=#94] (73) Sort [codegen id : 25] -Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Arguments: [sr_ticket_number#88 ASC NULLS FIRST, sr_item_sk#87 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Arguments: [sr_ticket_number#90 ASC NULLS FIRST, sr_item_sk#89 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 26] -Left keys [2]: [cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint)] -Right keys [2]: [sr_ticket_number#88, sr_item_sk#87] +Left keys [2]: [cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint)] +Right keys [2]: [sr_ticket_number#90, sr_item_sk#89] Join condition: None (75) Project [codegen id : 26] -Output [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] -Input [8]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Output [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] +Input [8]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] (76) HashAggregate [codegen id : 26] -Input [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] -Keys [1]: [ss_item_sk#80] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#89, 0)), partial_sum(coalesce(ss_quantity#82, 0)), partial_sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#93, sum#94, sum#95, isEmpty#96, sum#97, isEmpty#98] -Results [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Input [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] +Keys [1]: [ss_item_sk#81] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#91, 0)), partial_sum(coalesce(ss_quantity#83, 0)), partial_sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#95, sum#96, sum#97, isEmpty#98, sum#99, isEmpty#100] +Results [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] (77) Exchange -Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Arguments: hashpartitioning(ss_item_sk#80, 5), ENSURE_REQUIREMENTS, [id=#105] +Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] +Arguments: hashpartitioning(ss_item_sk#81, 5), ENSURE_REQUIREMENTS, [id=#107] (78) HashAggregate [codegen id : 27] -Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Keys [1]: [ss_item_sk#80] -Functions [4]: [sum(coalesce(sr_return_quantity#89, 0)), sum(coalesce(ss_quantity#82, 0)), sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#89, 0))#106, sum(coalesce(ss_quantity#82, 0))#107, sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108, sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109] -Results [3]: [ss_item_sk#80 AS item#110, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#89, 0))#106 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#82, 0))#107 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#111, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#112] +Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] +Keys [1]: [ss_item_sk#81] +Functions [4]: [sum(coalesce(sr_return_quantity#91, 0)), sum(coalesce(ss_quantity#83, 0)), sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#91, 0))#108, sum(coalesce(ss_quantity#83, 0))#109, sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110, sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111] +Results [3]: [ss_item_sk#81 AS item#112, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#91, 0))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#83, 0))#109 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#113, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#114] (79) Exchange -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] (80) Sort [codegen id : 28] -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: [return_ratio#111 ASC NULLS FIRST], false, 0 +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: [return_ratio#113 ASC NULLS FIRST], false, 0 (81) Window -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: [rank(return_ratio#111) windowspecdefinition(return_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#114], [return_ratio#111 ASC NULLS FIRST] +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: [rank(return_ratio#113) windowspecdefinition(return_ratio#113 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#116], [return_ratio#113 ASC NULLS FIRST] (82) Sort [codegen id : 29] -Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] -Arguments: [currency_ratio#112 ASC NULLS FIRST], false, 0 +Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] +Arguments: [currency_ratio#114 ASC NULLS FIRST], false, 0 (83) Window -Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] -Arguments: [rank(currency_ratio#112) windowspecdefinition(currency_ratio#112 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#115], [currency_ratio#112 ASC NULLS FIRST] +Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] +Arguments: [rank(currency_ratio#114) windowspecdefinition(currency_ratio#114 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#117], [currency_ratio#114 ASC NULLS FIRST] (84) Filter [codegen id : 30] -Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] -Condition : ((return_rank#114 <= 10) OR (currency_rank#115 <= 10)) +Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] +Condition : ((return_rank#116 <= 10) OR (currency_rank#117 <= 10)) (85) Project [codegen id : 30] -Output [5]: [store AS channel#116, item#110, return_ratio#111, return_rank#114, currency_rank#115] -Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] +Output [5]: [store AS channel#118, item#112, return_ratio#113, return_rank#116, currency_rank#117] +Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] (86) Union @@ -481,7 +481,7 @@ Results [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_ran (88) Exchange Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] -Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#117] +Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#119] (89) HashAggregate [codegen id : 32] Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] @@ -505,6 +505,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#85 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index aeaca4deaae0c..a98f59f84be7d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -260,170 +260,170 @@ Output [6]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, Input [9]: [cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_item_sk#49, cr_order_number#50, cr_return_quantity#51, cr_return_amount#52] (40) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#54] (41) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#47] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#54] Join condition: None (42) Project [codegen id : 10] Output [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] -Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#14] +Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#54] (43) HashAggregate [codegen id : 10] Input [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] Keys [1]: [cs_item_sk#42] Functions [4]: [partial_sum(coalesce(cr_return_quantity#51, 0)), partial_sum(coalesce(cs_quantity#44, 0)), partial_sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Results [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] +Aggregate Attributes [6]: [sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Results [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] (44) Exchange -Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] -Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#67] (45) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] +Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] Keys [1]: [cs_item_sk#42] Functions [4]: [sum(coalesce(cr_return_quantity#51, 0)), sum(coalesce(cs_quantity#44, 0)), sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#67, sum(coalesce(cs_quantity#44, 0))#68, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70] -Results [3]: [cs_item_sk#42 AS item#71, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#67 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#68 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#72, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#73] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#68, sum(coalesce(cs_quantity#44, 0))#69, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71] +Results [3]: [cs_item_sk#42 AS item#72, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#68 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#69 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#73, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#74] (46) Exchange -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#74] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#75] (47) Sort [codegen id : 12] -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: [return_ratio#72 ASC NULLS FIRST], false, 0 +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [return_ratio#73 ASC NULLS FIRST], false, 0 (48) Window -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: [rank(return_ratio#72) windowspecdefinition(return_ratio#72 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#72 ASC NULLS FIRST] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#76], [return_ratio#73 ASC NULLS FIRST] (49) Sort [codegen id : 13] -Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] -Arguments: [currency_ratio#73 ASC NULLS FIRST], false, 0 +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 (50) Window -Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] -Arguments: [rank(currency_ratio#73) windowspecdefinition(currency_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#73 ASC NULLS FIRST] +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#77], [currency_ratio#74 ASC NULLS FIRST] (51) Filter [codegen id : 14] -Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] +Condition : ((return_rank#76 <= 10) OR (currency_rank#77 <= 10)) (52) Project [codegen id : 14] -Output [5]: [catalog AS channel#77, item#71, return_ratio#72, return_rank#75, currency_rank#76] -Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] +Output [5]: [catalog AS channel#78, item#72, return_ratio#73, return_rank#76, currency_rank#77] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] (53) Scan parquet default.store_sales -Output [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Output [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#83), dynamicpruningexpression(ss_sold_date_sk#83 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#84), dynamicpruningexpression(ss_sold_date_sk#84 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 15] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] (55) Filter [codegen id : 15] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] -Condition : (((((((isnotnull(ss_net_profit#82) AND isnotnull(ss_net_paid#81)) AND isnotnull(ss_quantity#80)) AND (ss_net_profit#82 > 1.00)) AND (ss_net_paid#81 > 0.00)) AND (ss_quantity#80 > 0)) AND isnotnull(ss_ticket_number#79)) AND isnotnull(ss_item_sk#78)) +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Condition : (((((((isnotnull(ss_net_profit#83) AND isnotnull(ss_net_paid#82)) AND isnotnull(ss_quantity#81)) AND (ss_net_profit#83 > 1.00)) AND (ss_net_paid#82 > 0.00)) AND (ss_quantity#81 > 0)) AND isnotnull(ss_ticket_number#80)) AND isnotnull(ss_item_sk#79)) (56) Project [codegen id : 15] -Output [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Output [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] (57) BroadcastExchange -Input [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#84] +Input [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#85] (58) Scan parquet default.store_returns -Output [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Output [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (59) ColumnarToRow -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] (60) Filter -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] -Condition : (((isnotnull(sr_return_amt#88) AND (sr_return_amt#88 > 10000.00)) AND isnotnull(sr_ticket_number#86)) AND isnotnull(sr_item_sk#85)) +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Condition : (((isnotnull(sr_return_amt#89) AND (sr_return_amt#89 > 10000.00)) AND isnotnull(sr_ticket_number#87)) AND isnotnull(sr_item_sk#86)) (61) Project -Output [4]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Output [4]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] (62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [cast(ss_ticket_number#79 as bigint), cast(ss_item_sk#78 as bigint)] -Right keys [2]: [sr_ticket_number#86, sr_item_sk#85] +Left keys [2]: [cast(ss_ticket_number#80 as bigint), cast(ss_item_sk#79 as bigint)] +Right keys [2]: [sr_ticket_number#87, sr_item_sk#86] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88] -Input [9]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] +Output [6]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89] +Input [9]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] (64) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#91] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#83] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ss_sold_date_sk#84] +Right keys [1]: [d_date_sk#91] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] -Input [7]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88, d_date_sk#14] +Output [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] +Input [7]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89, d_date_sk#91] (67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] -Keys [1]: [ss_item_sk#78] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#87, 0)), partial_sum(coalesce(ss_quantity#80, 0)), partial_sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Results [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Input [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] +Keys [1]: [ss_item_sk#79] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#88, 0)), partial_sum(coalesce(ss_quantity#81, 0)), partial_sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] +Results [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] (68) Exchange -Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Arguments: hashpartitioning(ss_item_sk#78, 5), ENSURE_REQUIREMENTS, [id=#102] +Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Arguments: hashpartitioning(ss_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#104] (69) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Keys [1]: [ss_item_sk#78] -Functions [4]: [sum(coalesce(sr_return_quantity#87, 0)), sum(coalesce(ss_quantity#80, 0)), sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#87, 0))#103, sum(coalesce(ss_quantity#80, 0))#104, sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105, sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106] -Results [3]: [ss_item_sk#78 AS item#107, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#87, 0))#103 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#80, 0))#104 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#108, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#109] +Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Keys [1]: [ss_item_sk#79] +Functions [4]: [sum(coalesce(sr_return_quantity#88, 0)), sum(coalesce(ss_quantity#81, 0)), sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#88, 0))#105, sum(coalesce(ss_quantity#81, 0))#106, sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107, sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108] +Results [3]: [ss_item_sk#79 AS item#109, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#88, 0))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#81, 0))#106 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#110, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#111] (70) Exchange -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#112] (71) Sort [codegen id : 19] -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: [return_ratio#108 ASC NULLS FIRST], false, 0 +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: [return_ratio#110 ASC NULLS FIRST], false, 0 (72) Window -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: [rank(return_ratio#108) windowspecdefinition(return_ratio#108 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#111], [return_ratio#108 ASC NULLS FIRST] +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: [rank(return_ratio#110) windowspecdefinition(return_ratio#110 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#113], [return_ratio#110 ASC NULLS FIRST] (73) Sort [codegen id : 20] -Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] -Arguments: [currency_ratio#109 ASC NULLS FIRST], false, 0 +Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] +Arguments: [currency_ratio#111 ASC NULLS FIRST], false, 0 (74) Window -Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] -Arguments: [rank(currency_ratio#109) windowspecdefinition(currency_ratio#109 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#112], [currency_ratio#109 ASC NULLS FIRST] +Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] +Arguments: [rank(currency_ratio#111) windowspecdefinition(currency_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#114], [currency_ratio#111 ASC NULLS FIRST] (75) Filter [codegen id : 21] -Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] -Condition : ((return_rank#111 <= 10) OR (currency_rank#112 <= 10)) +Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] +Condition : ((return_rank#113 <= 10) OR (currency_rank#114 <= 10)) (76) Project [codegen id : 21] -Output [5]: [store AS channel#113, item#107, return_ratio#108, return_rank#111, currency_rank#112] -Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] +Output [5]: [store AS channel#115, item#109, return_ratio#110, return_rank#113, currency_rank#114] +Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] (77) Union @@ -436,7 +436,7 @@ Results [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_ran (79) Exchange Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] -Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#114] +Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#116] (80) HashAggregate [codegen id : 23] Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] @@ -460,6 +460,6 @@ Output [1]: [d_date_sk#14] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#83 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#84 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index a754cef8f2962..28a457258eff7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -265,193 +265,193 @@ Output [6]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, Input [8]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] (41) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#70] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#25] +Right keys [1]: [d_date_sk#70] Join condition: None (43) Project [codegen id : 11] Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#25] +Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#70] (44) HashAggregate [codegen id : 11] Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] -Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] +Results [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] (45) Exchange -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] +Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#79] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#83, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS RETURNS#84, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#85, catalog channel AS channel#86, concat(catalog_page, cp_catalog_page_id#68) AS id#87] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#84, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS RETURNS#85, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#86, catalog channel AS channel#87, concat(catalog_page, cp_catalog_page_id#68) AS id#88] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] -Condition : isnotnull(ws_web_site_sk#88) +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : isnotnull(ws_web_site_sk#89) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] (53) Exchange -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: hashpartitioning(wr_item_sk#98, wr_order_number#99, 5), ENSURE_REQUIREMENTS, [id=#103] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: hashpartitioning(wr_item_sk#99, wr_order_number#100, 5), ENSURE_REQUIREMENTS, [id=#104] (54) Sort [codegen id : 15] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: [wr_item_sk#98 ASC NULLS FIRST, wr_order_number#99 ASC NULLS FIRST], false, 0 +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: [wr_item_sk#99 ASC NULLS FIRST, wr_order_number#100 ASC NULLS FIRST], false, 0 (55) Scan parquet default.web_sales -Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 16] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (57) Filter [codegen id : 16] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) (58) Project [codegen id : 16] -Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (59) Exchange -Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Arguments: hashpartitioning(cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint), 5), ENSURE_REQUIREMENTS, [id=#106] +Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Arguments: hashpartitioning(cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint), 5), ENSURE_REQUIREMENTS, [id=#109] (60) Sort [codegen id : 17] -Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Arguments: [cast(ws_item_sk#104 as bigint) ASC NULLS FIRST, cast(ws_order_number#105 as bigint) ASC NULLS FIRST], false, 0 +Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Arguments: [cast(ws_item_sk#105 as bigint) ASC NULLS FIRST, cast(ws_order_number#107 as bigint) ASC NULLS FIRST], false, 0 (61) SortMergeJoin [codegen id : 18] -Left keys [2]: [wr_item_sk#98, wr_order_number#99] -Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] +Left keys [2]: [wr_item_sk#99, wr_order_number#100] +Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] Join condition: None (62) Project [codegen id : 18] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#107, wr_returned_date_sk#102 AS date_sk#108, 0.00 AS sales_price#109, 0.00 AS profit#110, wr_return_amt#100 AS return_amt#111, wr_net_loss#101 AS net_loss#112] -Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#110, wr_returned_date_sk#103 AS date_sk#111, 0.00 AS sales_price#112, 0.00 AS profit#113, wr_return_amt#101 AS return_amt#114, wr_net_loss#102 AS net_loss#115] +Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] (63) Union (64) Scan parquet default.web_site -Output [2]: [web_site_sk#113, web_site_id#114] +Output [2]: [web_site_sk#116, web_site_id#117] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 19] -Input [2]: [web_site_sk#113, web_site_id#114] +Input [2]: [web_site_sk#116, web_site_id#117] (66) Filter [codegen id : 19] -Input [2]: [web_site_sk#113, web_site_id#114] -Condition : isnotnull(web_site_sk#113) +Input [2]: [web_site_sk#116, web_site_id#117] +Condition : isnotnull(web_site_sk#116) (67) BroadcastExchange -Input [2]: [web_site_sk#113, web_site_id#114] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] +Input [2]: [web_site_sk#116, web_site_id#117] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] (68) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [wsr_web_site_sk#92] -Right keys [1]: [web_site_sk#113] +Left keys [1]: [wsr_web_site_sk#93] +Right keys [1]: [web_site_sk#116] Join condition: None (69) Project [codegen id : 21] -Output [6]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Input [8]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#113, web_site_id#114] +Output [6]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [8]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] (70) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#119] (71) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [date_sk#93] -Right keys [1]: [cast(d_date_sk#25 as bigint)] +Left keys [1]: [date_sk#94] +Right keys [1]: [cast(d_date_sk#119 as bigint)] Join condition: None (72) Project [codegen id : 21] -Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Input [7]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114, d_date_sk#25] +Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [7]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117, d_date_sk#119] (73) HashAggregate [codegen id : 21] -Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Keys [1]: [web_site_id#114] -Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] -Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] +Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Keys [1]: [web_site_id#117] +Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum#120, sum#121, sum#122, sum#123] +Results [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] (74) Exchange -Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] +Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#128] (75) HashAggregate [codegen id : 22] -Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -Keys [1]: [web_site_id#114] -Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#125, sum(UnscaledValue(return_amt#96))#126, sum(UnscaledValue(profit#95))#127, sum(UnscaledValue(net_loss#97))#128] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#94))#125,17,2) AS sales#129, MakeDecimal(sum(UnscaledValue(return_amt#96))#126,17,2) AS RETURNS#130, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#131, web channel AS channel#132, concat(web_site, web_site_id#114) AS id#133] +Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] +Keys [1]: [web_site_id#117] +Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#129, sum(UnscaledValue(return_amt#97))#130, sum(UnscaledValue(profit#96))#131, sum(UnscaledValue(net_loss#98))#132] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#95))#129,17,2) AS sales#133, MakeDecimal(sum(UnscaledValue(return_amt#97))#130,17,2) AS RETURNS#134, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#131,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#132,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#135, web channel AS channel#136, concat(web_site, web_site_id#117) AS id#137] (76) Union (77) Expand [codegen id : 23] Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45] -Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#134, id#135, spark_grouping_id#136] +Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#138, id#139, spark_grouping_id#140] (78) HashAggregate [codegen id : 23] -Input [6]: [sales#41, returns#42, profit#43, channel#134, id#135, spark_grouping_id#136] -Keys [3]: [channel#134, id#135, spark_grouping_id#136] +Input [6]: [sales#41, returns#42, profit#43, channel#138, id#139, spark_grouping_id#140] +Keys [3]: [channel#138, id#139, spark_grouping_id#140] Functions [3]: [partial_sum(sales#41), partial_sum(returns#42), partial_sum(profit#43)] -Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] -Results [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Aggregate Attributes [6]: [sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Results [9]: [channel#138, id#139, spark_grouping_id#140, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] (79) Exchange -Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -Arguments: hashpartitioning(channel#134, id#135, spark_grouping_id#136, 5), ENSURE_REQUIREMENTS, [id=#149] +Input [9]: [channel#138, id#139, spark_grouping_id#140, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] +Arguments: hashpartitioning(channel#138, id#139, spark_grouping_id#140, 5), ENSURE_REQUIREMENTS, [id=#153] (80) HashAggregate [codegen id : 24] -Input [9]: [channel#134, id#135, spark_grouping_id#136, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -Keys [3]: [channel#134, id#135, spark_grouping_id#136] +Input [9]: [channel#138, id#139, spark_grouping_id#140, sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] +Keys [3]: [channel#138, id#139, spark_grouping_id#140] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] -Aggregate Attributes [3]: [sum(sales#41)#150, sum(returns#42)#151, sum(profit#43)#152] -Results [5]: [channel#134, id#135, sum(sales#41)#150 AS sales#153, sum(returns#42)#151 AS returns#154, sum(profit#43)#152 AS profit#155] +Aggregate Attributes [3]: [sum(sales#41)#154, sum(returns#42)#155, sum(profit#43)#156] +Results [5]: [channel#138, id#139, sum(sales#41)#154 AS sales#157, sum(returns#42)#155 AS returns#158, sum(profit#43)#156 AS profit#159] (81) TakeOrderedAndProject -Input [5]: [channel#134, id#135, sales#153, returns#154, profit#155] -Arguments: 100, [channel#134 ASC NULLS FIRST, id#135 ASC NULLS FIRST], [channel#134, id#135, sales#153, returns#154, profit#155] +Input [5]: [channel#138, id#139, sales#157, returns#158, profit#159] +Arguments: 100, [channel#138 ASC NULLS FIRST, id#139 ASC NULLS FIRST], [channel#138, id#139, sales#157, returns#158, profit#159] ===== Subqueries ===== @@ -469,12 +469,12 @@ ReusedExchange (83) (83) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#70] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index b3eb1f9592ff3..cb130ce17795a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -235,208 +235,208 @@ Input [4]: [cr_catalog_page_sk#57, cr_return_amount#58, cr_net_loss#59, cr_retur (34) Union (35) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#67] (36) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#67] Join condition: None (37) Project [codegen id : 11] Output [5]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56] -Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#22] +Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#67] (38) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Output [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] (40) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] -Condition : isnotnull(cp_catalog_page_sk#67) +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Condition : isnotnull(cp_catalog_page_sk#68) (41) BroadcastExchange -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#69] +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#70] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [page_sk#51] -Right keys [1]: [cp_catalog_page_sk#67] +Right keys [1]: [cp_catalog_page_sk#68] Join condition: None (43) Project [codegen id : 11] -Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] +Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] +Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#68, cp_catalog_page_id#69] (44) HashAggregate [codegen id : 11] -Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Keys [1]: [cp_catalog_page_id#68] +Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] +Keys [1]: [cp_catalog_page_id#69] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] -Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] +Results [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] (45) Exchange -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] +Arguments: hashpartitioning(cp_catalog_page_id#69, 5), ENSURE_REQUIREMENTS, [id=#79] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Keys [1]: [cp_catalog_page_id#68] +Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] +Keys [1]: [cp_catalog_page_id#69] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#83, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS RETURNS#84, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#85, catalog channel AS channel#86, concat(catalog_page, cp_catalog_page_id#68) AS id#87] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#84, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS RETURNS#85, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#86, catalog channel AS channel#87, concat(catalog_page, cp_catalog_page_id#69) AS id#88] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] -Condition : isnotnull(ws_web_site_sk#88) +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : isnotnull(ws_web_site_sk#89) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] (53) BroadcastExchange -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#103] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#104] (54) Scan parquet default.web_sales -Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (55) ColumnarToRow -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (56) Filter -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) (57) Project -Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#98, wr_order_number#99] -Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] +Left keys [2]: [wr_item_sk#99, wr_order_number#100] +Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] Join condition: None (59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#106, wr_returned_date_sk#102 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#100 AS return_amt#110, wr_net_loss#101 AS net_loss#111] -Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#109, wr_returned_date_sk#103 AS date_sk#110, 0.00 AS sales_price#111, 0.00 AS profit#112, wr_return_amt#101 AS return_amt#113, wr_net_loss#102 AS net_loss#114] +Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] (60) Union (61) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#115] (62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#93] -Right keys [1]: [cast(d_date_sk#22 as bigint)] +Left keys [1]: [date_sk#94] +Right keys [1]: [cast(d_date_sk#115 as bigint)] Join condition: None (63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97] -Input [7]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, d_date_sk#22] +Output [5]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98] +Input [7]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, d_date_sk#115] (64) Scan parquet default.web_site -Output [2]: [web_site_sk#112, web_site_id#113] +Output [2]: [web_site_sk#116, web_site_id#117] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#112, web_site_id#113] +Input [2]: [web_site_sk#116, web_site_id#117] (66) Filter [codegen id : 17] -Input [2]: [web_site_sk#112, web_site_id#113] -Condition : isnotnull(web_site_sk#112) +Input [2]: [web_site_sk#116, web_site_id#117] +Condition : isnotnull(web_site_sk#116) (67) BroadcastExchange -Input [2]: [web_site_sk#112, web_site_id#113] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#114] +Input [2]: [web_site_sk#116, web_site_id#117] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] (68) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#92] -Right keys [1]: [web_site_sk#112] +Left keys [1]: [wsr_web_site_sk#93] +Right keys [1]: [web_site_sk#116] Join condition: None (69) Project [codegen id : 18] -Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] -Input [7]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#112, web_site_id#113] +Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [7]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] (70) HashAggregate [codegen id : 18] -Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] -Keys [1]: [web_site_id#113] -Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] -Results [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] +Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Keys [1]: [web_site_id#117] +Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum#119, sum#120, sum#121, sum#122] +Results [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] (71) Exchange -Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -Arguments: hashpartitioning(web_site_id#113, 5), ENSURE_REQUIREMENTS, [id=#123] +Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] +Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#127] (72) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -Keys [1]: [web_site_id#113] -Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#124, sum(UnscaledValue(return_amt#96))#125, sum(UnscaledValue(profit#95))#126, sum(UnscaledValue(net_loss#97))#127] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#94))#124,17,2) AS sales#128, MakeDecimal(sum(UnscaledValue(return_amt#96))#125,17,2) AS RETURNS#129, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#126,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#127,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#130, web channel AS channel#131, concat(web_site, web_site_id#113) AS id#132] +Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] +Keys [1]: [web_site_id#117] +Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#128, sum(UnscaledValue(return_amt#97))#129, sum(UnscaledValue(profit#96))#130, sum(UnscaledValue(net_loss#98))#131] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#95))#128,17,2) AS sales#132, MakeDecimal(sum(UnscaledValue(return_amt#97))#129,17,2) AS RETURNS#133, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#130,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#131,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#134, web channel AS channel#135, concat(web_site, web_site_id#117) AS id#136] (73) Union (74) Expand [codegen id : 20] Input [5]: [sales#41, RETURNS#42, profit#43, channel#44, id#45] -Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#133, id#134, spark_grouping_id#135] +Arguments: [List(sales#41, returns#42, profit#43, channel#44, id#45, 0), List(sales#41, returns#42, profit#43, channel#44, null, 1), List(sales#41, returns#42, profit#43, null, null, 3)], [sales#41, returns#42, profit#43, channel#137, id#138, spark_grouping_id#139] (75) HashAggregate [codegen id : 20] -Input [6]: [sales#41, returns#42, profit#43, channel#133, id#134, spark_grouping_id#135] -Keys [3]: [channel#133, id#134, spark_grouping_id#135] +Input [6]: [sales#41, returns#42, profit#43, channel#137, id#138, spark_grouping_id#139] +Keys [3]: [channel#137, id#138, spark_grouping_id#139] Functions [3]: [partial_sum(sales#41), partial_sum(returns#42), partial_sum(profit#43)] -Aggregate Attributes [6]: [sum#136, isEmpty#137, sum#138, isEmpty#139, sum#140, isEmpty#141] -Results [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] +Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Results [9]: [channel#137, id#138, spark_grouping_id#139, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] (76) Exchange -Input [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -Arguments: hashpartitioning(channel#133, id#134, spark_grouping_id#135, 5), ENSURE_REQUIREMENTS, [id=#148] +Input [9]: [channel#137, id#138, spark_grouping_id#139, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Arguments: hashpartitioning(channel#137, id#138, spark_grouping_id#139, 5), ENSURE_REQUIREMENTS, [id=#152] (77) HashAggregate [codegen id : 21] -Input [9]: [channel#133, id#134, spark_grouping_id#135, sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -Keys [3]: [channel#133, id#134, spark_grouping_id#135] +Input [9]: [channel#137, id#138, spark_grouping_id#139, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Keys [3]: [channel#137, id#138, spark_grouping_id#139] Functions [3]: [sum(sales#41), sum(returns#42), sum(profit#43)] -Aggregate Attributes [3]: [sum(sales#41)#149, sum(returns#42)#150, sum(profit#43)#151] -Results [5]: [channel#133, id#134, sum(sales#41)#149 AS sales#152, sum(returns#42)#150 AS returns#153, sum(profit#43)#151 AS profit#154] +Aggregate Attributes [3]: [sum(sales#41)#153, sum(returns#42)#154, sum(profit#43)#155] +Results [5]: [channel#137, id#138, sum(sales#41)#153 AS sales#156, sum(returns#42)#154 AS returns#157, sum(profit#43)#155 AS profit#158] (78) TakeOrderedAndProject -Input [5]: [channel#133, id#134, sales#152, returns#153, profit#154] -Arguments: 100, [channel#133 ASC NULLS FIRST, id#134 ASC NULLS FIRST], [channel#133, id#134, sales#152, returns#153, profit#154] +Input [5]: [channel#137, id#138, sales#156, returns#157, profit#158] +Arguments: 100, [channel#137 ASC NULLS FIRST, id#138 ASC NULLS FIRST], [channel#137, id#138, sales#156, returns#157, profit#158] ===== Subqueries ===== @@ -454,12 +454,12 @@ ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#67] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt index 85906beeff369..b149bdd3e1e3f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt @@ -297,39 +297,39 @@ Input [3]: [ss_customer_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35] Condition : isnotnull(ss_customer_sk#33) (51) Scan parquet default.date_dim -Output [2]: [d_date_sk#21, d_month_seq#37] +Output [2]: [d_date_sk#37, d_month_seq#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#21, d_month_seq#37] +Input [2]: [d_date_sk#37, d_month_seq#38] (53) Filter [codegen id : 12] -Input [2]: [d_date_sk#21, d_month_seq#37] -Condition : (((isnotnull(d_month_seq#37) AND (d_month_seq#37 >= Subquery scalar-subquery#38, [id=#39])) AND (d_month_seq#37 <= Subquery scalar-subquery#40, [id=#41])) AND isnotnull(d_date_sk#21)) +Input [2]: [d_date_sk#37, d_month_seq#38] +Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= Subquery scalar-subquery#39, [id=#40])) AND (d_month_seq#38 <= Subquery scalar-subquery#41, [id=#42])) AND isnotnull(d_date_sk#37)) (54) Project [codegen id : 12] -Output [1]: [d_date_sk#21] -Input [2]: [d_date_sk#21, d_month_seq#37] +Output [1]: [d_date_sk#37] +Input [2]: [d_date_sk#37, d_month_seq#38] (55) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [d_date_sk#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#43] (56) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ss_sold_date_sk#35] -Right keys [1]: [d_date_sk#21] +Right keys [1]: [d_date_sk#37] Join condition: None (57) Project [codegen id : 13] Output [2]: [ss_customer_sk#33, ss_ext_sales_price#34] -Input [4]: [ss_customer_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35, d_date_sk#21] +Input [4]: [ss_customer_sk#33, ss_ext_sales_price#34, ss_sold_date_sk#35, d_date_sk#37] (58) Exchange Input [2]: [ss_customer_sk#33, ss_ext_sales_price#34] -Arguments: hashpartitioning(ss_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#43] +Arguments: hashpartitioning(ss_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#44] (59) Sort [codegen id : 14] Input [2]: [ss_customer_sk#33, ss_ext_sales_price#34] @@ -348,37 +348,37 @@ Input [3]: [c_customer_sk#30, ss_customer_sk#33, ss_ext_sales_price#34] Input [2]: [c_customer_sk#30, ss_ext_sales_price#34] Keys [1]: [c_customer_sk#30] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#34))] -Aggregate Attributes [1]: [sum#44] -Results [2]: [c_customer_sk#30, sum#45] +Aggregate Attributes [1]: [sum#45] +Results [2]: [c_customer_sk#30, sum#46] (63) HashAggregate [codegen id : 15] -Input [2]: [c_customer_sk#30, sum#45] +Input [2]: [c_customer_sk#30, sum#46] Keys [1]: [c_customer_sk#30] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#34))#46] -Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#46,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#47] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#34))#47] +Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#47,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#48] (64) HashAggregate [codegen id : 15] -Input [1]: [segment#47] -Keys [1]: [segment#47] +Input [1]: [segment#48] +Keys [1]: [segment#48] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#48] -Results [2]: [segment#47, count#49] +Aggregate Attributes [1]: [count#49] +Results [2]: [segment#48, count#50] (65) Exchange -Input [2]: [segment#47, count#49] -Arguments: hashpartitioning(segment#47, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [segment#48, count#50] +Arguments: hashpartitioning(segment#48, 5), ENSURE_REQUIREMENTS, [id=#51] (66) HashAggregate [codegen id : 16] -Input [2]: [segment#47, count#49] -Keys [1]: [segment#47] +Input [2]: [segment#48, count#50] +Keys [1]: [segment#48] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#51] -Results [3]: [segment#47, count(1)#51 AS num_customers#52, (segment#47 * 50) AS segment_base#53] +Aggregate Attributes [1]: [count(1)#52] +Results [3]: [segment#48, count(1)#52 AS num_customers#53, (segment#48 * 50) AS segment_base#54] (67) TakeOrderedAndProject -Input [3]: [segment#47, num_customers#52, segment_base#53] -Arguments: 100, [segment#47 ASC NULLS FIRST, num_customers#52 ASC NULLS FIRST], [segment#47, num_customers#52, segment_base#53] +Input [3]: [segment#48, num_customers#53, segment_base#54] +Arguments: 100, [segment#48 ASC NULLS FIRST, num_customers#53 ASC NULLS FIRST], [segment#48, num_customers#53, segment_base#54] ===== Subqueries ===== @@ -396,9 +396,9 @@ ReusedExchange (69) (69) ReusedExchange [Reuses operator id: 55] -Output [1]: [d_date_sk#21] +Output [1]: [d_date_sk#37] -Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#39, [id=#40] * HashAggregate (76) +- Exchange (75) +- * HashAggregate (74) @@ -409,42 +409,42 @@ Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (70) Scan parquet default.date_dim -Output [3]: [d_month_seq#37, d_year#22, d_moy#23] +Output [3]: [d_month_seq#55, 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,1998), EqualTo(d_moy,12)] ReadSchema: struct (71) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] +Input [3]: [d_month_seq#55, d_year#56, d_moy#57] (72) Filter [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] -Condition : (((isnotnull(d_year#22) AND isnotnull(d_moy#23)) AND (d_year#22 = 1998)) AND (d_moy#23 = 12)) +Input [3]: [d_month_seq#55, d_year#56, d_moy#57] +Condition : (((isnotnull(d_year#56) AND isnotnull(d_moy#57)) AND (d_year#56 = 1998)) AND (d_moy#57 = 12)) (73) Project [codegen id : 1] -Output [1]: [(d_month_seq#37 + 1) AS (d_month_seq + 1)#54] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] +Output [1]: [(d_month_seq#55 + 1) AS (d_month_seq + 1)#58] +Input [3]: [d_month_seq#55, d_year#56, d_moy#57] (74) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 1)#54] -Keys [1]: [(d_month_seq + 1)#54] +Input [1]: [(d_month_seq + 1)#58] +Keys [1]: [(d_month_seq + 1)#58] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#54] +Results [1]: [(d_month_seq + 1)#58] (75) Exchange -Input [1]: [(d_month_seq + 1)#54] -Arguments: hashpartitioning((d_month_seq + 1)#54, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [1]: [(d_month_seq + 1)#58] +Arguments: hashpartitioning((d_month_seq + 1)#58, 5), ENSURE_REQUIREMENTS, [id=#59] (76) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#54] -Keys [1]: [(d_month_seq + 1)#54] +Input [1]: [(d_month_seq + 1)#58] +Keys [1]: [(d_month_seq + 1)#58] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#54] +Results [1]: [(d_month_seq + 1)#58] -Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#40, [id=#41] +Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#41, [id=#42] * HashAggregate (83) +- Exchange (82) +- * HashAggregate (81) @@ -455,39 +455,39 @@ Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (77) Scan parquet default.date_dim -Output [3]: [d_month_seq#37, d_year#22, d_moy#23] +Output [3]: [d_month_seq#60, d_year#61, d_moy#62] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (78) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] +Input [3]: [d_month_seq#60, d_year#61, d_moy#62] (79) Filter [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] -Condition : (((isnotnull(d_year#22) AND isnotnull(d_moy#23)) AND (d_year#22 = 1998)) AND (d_moy#23 = 12)) +Input [3]: [d_month_seq#60, d_year#61, d_moy#62] +Condition : (((isnotnull(d_year#61) AND isnotnull(d_moy#62)) AND (d_year#61 = 1998)) AND (d_moy#62 = 12)) (80) Project [codegen id : 1] -Output [1]: [(d_month_seq#37 + 3) AS (d_month_seq + 3)#56] -Input [3]: [d_month_seq#37, d_year#22, d_moy#23] +Output [1]: [(d_month_seq#60 + 3) AS (d_month_seq + 3)#63] +Input [3]: [d_month_seq#60, d_year#61, d_moy#62] (81) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 3)#56] -Keys [1]: [(d_month_seq + 3)#56] +Input [1]: [(d_month_seq + 3)#63] +Keys [1]: [(d_month_seq + 3)#63] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#56] +Results [1]: [(d_month_seq + 3)#63] (82) Exchange -Input [1]: [(d_month_seq + 3)#56] -Arguments: hashpartitioning((d_month_seq + 3)#56, 5), ENSURE_REQUIREMENTS, [id=#57] +Input [1]: [(d_month_seq + 3)#63] +Arguments: hashpartitioning((d_month_seq + 3)#63, 5), ENSURE_REQUIREMENTS, [id=#64] (83) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#56] -Keys [1]: [(d_month_seq + 3)#56] +Input [1]: [(d_month_seq + 3)#63] +Keys [1]: [(d_month_seq + 3)#63] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#56] +Results [1]: [(d_month_seq + 3)#63] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index 2c1f7c6195ba7..d0638d7606bb5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -295,75 +295,75 @@ Output [3]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28] Input [7]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28, ca_county#32, ca_state#33, s_county#35, s_state#36] (51) Scan parquet default.date_dim -Output [2]: [d_date_sk#18, d_month_seq#38] +Output [2]: [d_date_sk#38, d_month_seq#39] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 10] -Input [2]: [d_date_sk#18, d_month_seq#38] +Input [2]: [d_date_sk#38, d_month_seq#39] (53) Filter [codegen id : 10] -Input [2]: [d_date_sk#18, d_month_seq#38] -Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= Subquery scalar-subquery#39, [id=#40])) AND (d_month_seq#38 <= Subquery scalar-subquery#41, [id=#42])) AND isnotnull(d_date_sk#18)) +Input [2]: [d_date_sk#38, d_month_seq#39] +Condition : (((isnotnull(d_month_seq#39) AND (d_month_seq#39 >= Subquery scalar-subquery#40, [id=#41])) AND (d_month_seq#39 <= Subquery scalar-subquery#42, [id=#43])) AND isnotnull(d_date_sk#38)) (54) Project [codegen id : 10] -Output [1]: [d_date_sk#18] -Input [2]: [d_date_sk#18, d_month_seq#38] +Output [1]: [d_date_sk#38] +Input [2]: [d_date_sk#38, d_month_seq#39] (55) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#43] +Input [1]: [d_date_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (56) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#18] +Right keys [1]: [d_date_sk#38] Join condition: None (57) Project [codegen id : 11] Output [2]: [c_customer_sk#22, ss_ext_sales_price#27] -Input [4]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#18] +Input [4]: [c_customer_sk#22, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#38] (58) HashAggregate [codegen id : 11] Input [2]: [c_customer_sk#22, ss_ext_sales_price#27] Keys [1]: [c_customer_sk#22] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum#44] -Results [2]: [c_customer_sk#22, sum#45] +Aggregate Attributes [1]: [sum#45] +Results [2]: [c_customer_sk#22, sum#46] (59) Exchange -Input [2]: [c_customer_sk#22, sum#45] -Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [2]: [c_customer_sk#22, sum#46] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#47] (60) HashAggregate [codegen id : 12] -Input [2]: [c_customer_sk#22, sum#45] +Input [2]: [c_customer_sk#22, sum#46] Keys [1]: [c_customer_sk#22] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#47] -Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#47,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#48] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#48] +Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#48,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#49] (61) HashAggregate [codegen id : 12] -Input [1]: [segment#48] -Keys [1]: [segment#48] +Input [1]: [segment#49] +Keys [1]: [segment#49] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#49] -Results [2]: [segment#48, count#50] +Aggregate Attributes [1]: [count#50] +Results [2]: [segment#49, count#51] (62) Exchange -Input [2]: [segment#48, count#50] -Arguments: hashpartitioning(segment#48, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [2]: [segment#49, count#51] +Arguments: hashpartitioning(segment#49, 5), ENSURE_REQUIREMENTS, [id=#52] (63) HashAggregate [codegen id : 13] -Input [2]: [segment#48, count#50] -Keys [1]: [segment#48] +Input [2]: [segment#49, count#51] +Keys [1]: [segment#49] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#52] -Results [3]: [segment#48, count(1)#52 AS num_customers#53, (segment#48 * 50) AS segment_base#54] +Aggregate Attributes [1]: [count(1)#53] +Results [3]: [segment#49, count(1)#53 AS num_customers#54, (segment#49 * 50) AS segment_base#55] (64) TakeOrderedAndProject -Input [3]: [segment#48, num_customers#53, segment_base#54] -Arguments: 100, [segment#48 ASC NULLS FIRST, num_customers#53 ASC NULLS FIRST], [segment#48, num_customers#53, segment_base#54] +Input [3]: [segment#49, num_customers#54, segment_base#55] +Arguments: 100, [segment#49 ASC NULLS FIRST, num_customers#54 ASC NULLS FIRST], [segment#49, num_customers#54, segment_base#55] ===== Subqueries ===== @@ -381,9 +381,9 @@ ReusedExchange (66) (66) ReusedExchange [Reuses operator id: 55] -Output [1]: [d_date_sk#18] +Output [1]: [d_date_sk#38] -Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#39, [id=#40] +Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#40, [id=#41] * HashAggregate (73) +- Exchange (72) +- * HashAggregate (71) @@ -394,42 +394,42 @@ Subquery:4 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (67) Scan parquet default.date_dim -Output [3]: [d_month_seq#38, d_year#19, d_moy#20] +Output [3]: [d_month_seq#56, d_year#57, d_moy#58] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (68) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] +Input [3]: [d_month_seq#56, d_year#57, d_moy#58] (69) Filter [codegen id : 1] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] -Condition : (((isnotnull(d_year#19) AND isnotnull(d_moy#20)) AND (d_year#19 = 1998)) AND (d_moy#20 = 12)) +Input [3]: [d_month_seq#56, d_year#57, d_moy#58] +Condition : (((isnotnull(d_year#57) AND isnotnull(d_moy#58)) AND (d_year#57 = 1998)) AND (d_moy#58 = 12)) (70) Project [codegen id : 1] -Output [1]: [(d_month_seq#38 + 1) AS (d_month_seq + 1)#55] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] +Output [1]: [(d_month_seq#56 + 1) AS (d_month_seq + 1)#59] +Input [3]: [d_month_seq#56, d_year#57, d_moy#58] (71) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 1)#55] -Keys [1]: [(d_month_seq + 1)#55] +Input [1]: [(d_month_seq + 1)#59] +Keys [1]: [(d_month_seq + 1)#59] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#55] +Results [1]: [(d_month_seq + 1)#59] (72) Exchange -Input [1]: [(d_month_seq + 1)#55] -Arguments: hashpartitioning((d_month_seq + 1)#55, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [1]: [(d_month_seq + 1)#59] +Arguments: hashpartitioning((d_month_seq + 1)#59, 5), ENSURE_REQUIREMENTS, [id=#60] (73) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#55] -Keys [1]: [(d_month_seq + 1)#55] +Input [1]: [(d_month_seq + 1)#59] +Keys [1]: [(d_month_seq + 1)#59] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#55] +Results [1]: [(d_month_seq + 1)#59] -Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#41, [id=#42] +Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#42, [id=#43] * HashAggregate (80) +- Exchange (79) +- * HashAggregate (78) @@ -440,39 +440,39 @@ Subquery:5 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquer (74) Scan parquet default.date_dim -Output [3]: [d_month_seq#38, d_year#19, d_moy#20] +Output [3]: [d_month_seq#61, d_year#62, d_moy#63] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct (75) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] +Input [3]: [d_month_seq#61, d_year#62, d_moy#63] (76) Filter [codegen id : 1] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] -Condition : (((isnotnull(d_year#19) AND isnotnull(d_moy#20)) AND (d_year#19 = 1998)) AND (d_moy#20 = 12)) +Input [3]: [d_month_seq#61, d_year#62, d_moy#63] +Condition : (((isnotnull(d_year#62) AND isnotnull(d_moy#63)) AND (d_year#62 = 1998)) AND (d_moy#63 = 12)) (77) Project [codegen id : 1] -Output [1]: [(d_month_seq#38 + 3) AS (d_month_seq + 3)#57] -Input [3]: [d_month_seq#38, d_year#19, d_moy#20] +Output [1]: [(d_month_seq#61 + 3) AS (d_month_seq + 3)#64] +Input [3]: [d_month_seq#61, d_year#62, d_moy#63] (78) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 3)#57] -Keys [1]: [(d_month_seq + 3)#57] +Input [1]: [(d_month_seq + 3)#64] +Keys [1]: [(d_month_seq + 3)#64] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#57] +Results [1]: [(d_month_seq + 3)#64] (79) Exchange -Input [1]: [(d_month_seq + 3)#57] -Arguments: hashpartitioning((d_month_seq + 3)#57, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [1]: [(d_month_seq + 3)#64] +Arguments: hashpartitioning((d_month_seq + 3)#64, 5), ENSURE_REQUIREMENTS, [id=#65] (80) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#57] -Keys [1]: [(d_month_seq + 3)#57] +Input [1]: [(d_month_seq + 3)#64] +Keys [1]: [(d_month_seq + 3)#64] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#57] +Results [1]: [(d_month_seq + 3)#64] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt index 8e3bf5bf57bcf..47eda1483a06c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt @@ -160,30 +160,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#14, i_color#15] +Output [2]: [i_item_id#15, i_color#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#14, i_color#15] +Input [2]: [i_item_id#15, i_color#16] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#14, i_color#15] -Condition : i_color#15 IN (slate ,blanched ,burnished ) +Input [2]: [i_item_id#15, i_color#16] +Condition : i_color#16 IN (slate ,blanched ,burnished ) (24) Project [codegen id : 3] -Output [1]: [i_item_id#14 AS i_item_id#14#16] -Input [2]: [i_item_id#14, i_color#15] +Output [1]: [i_item_id#15] +Input [2]: [i_item_id#15, i_color#16] (25) BroadcastExchange -Input [1]: [i_item_id#14#16] +Input [1]: [i_item_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#14#16] +Right keys [1]: [i_item_id#15] Join condition: None (27) BroadcastExchange @@ -233,127 +233,127 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#30, i_item_id#31] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] +Output [2]: [cs_ext_sales_price#26, i_item_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#14] -Keys [1]: [i_item_id#14] +Input [2]: [cs_ext_sales_price#26, i_item_id#31] +Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#14, sum#29] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#31, sum#33] (46) Exchange -Input [2]: [i_item_id#14, sum#29] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [i_item_id#31, sum#33] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#14, sum#29] -Keys [1]: [i_item_id#14] +Input [2]: [i_item_id#31, sum#33] +Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] (48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] (50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#41] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#42] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#43, i_item_id#44] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_item_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] +Output [2]: [ws_ext_sales_price#39, i_item_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_item_id#14, sum#38] +Input [2]: [ws_ext_sales_price#39, i_item_id#44] +Keys [1]: [i_item_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_item_id#44, sum#46] (61) Exchange -Input [2]: [i_item_id#14, sum#38] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [2]: [i_item_id#44, sum#46] +Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] (62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#38] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +Input [2]: [i_item_id#44, sum#46] +Keys [1]: [i_item_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] (63) Union @@ -361,23 +361,23 @@ Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35) Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_item_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_item_id#14, sum#52, isEmpty#53] (65) Exchange -Input [3]: [i_item_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] (66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#44, isEmpty#45] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] (67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#48] -Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] +Input [2]: [i_item_id#14, total_sales#56] +Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] ===== Subqueries ===== @@ -390,6 +390,6 @@ Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 8e3bf5bf57bcf..47eda1483a06c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -160,30 +160,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#14, i_color#15] +Output [2]: [i_item_id#15, i_color#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [In(i_color, [slate ,blanched ,burnished ])] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#14, i_color#15] +Input [2]: [i_item_id#15, i_color#16] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#14, i_color#15] -Condition : i_color#15 IN (slate ,blanched ,burnished ) +Input [2]: [i_item_id#15, i_color#16] +Condition : i_color#16 IN (slate ,blanched ,burnished ) (24) Project [codegen id : 3] -Output [1]: [i_item_id#14 AS i_item_id#14#16] -Input [2]: [i_item_id#14, i_color#15] +Output [1]: [i_item_id#15] +Input [2]: [i_item_id#15, i_color#16] (25) BroadcastExchange -Input [1]: [i_item_id#14#16] +Input [1]: [i_item_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#14#16] +Right keys [1]: [i_item_id#15] Join condition: None (27) BroadcastExchange @@ -233,127 +233,127 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#30, i_item_id#31] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] +Output [2]: [cs_ext_sales_price#26, i_item_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#14] -Keys [1]: [i_item_id#14] +Input [2]: [cs_ext_sales_price#26, i_item_id#31] +Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#14, sum#29] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#31, sum#33] (46) Exchange -Input [2]: [i_item_id#14, sum#29] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [i_item_id#31, sum#33] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#14, sum#29] -Keys [1]: [i_item_id#14] +Input [2]: [i_item_id#31, sum#33] +Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] (48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] (50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#41] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#42] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#43, i_item_id#44] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_item_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] +Output [2]: [ws_ext_sales_price#39, i_item_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_item_id#14, sum#38] +Input [2]: [ws_ext_sales_price#39, i_item_id#44] +Keys [1]: [i_item_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_item_id#44, sum#46] (61) Exchange -Input [2]: [i_item_id#14, sum#38] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [2]: [i_item_id#44, sum#46] +Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] (62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#38] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +Input [2]: [i_item_id#44, sum#46] +Keys [1]: [i_item_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] (63) Union @@ -361,23 +361,23 @@ Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35) Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_item_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_item_id#14, sum#52, isEmpty#53] (65) Exchange -Input [3]: [i_item_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] (66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#44, isEmpty#45] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] (67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#48] -Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] +Input [2]: [i_item_id#14, total_sales#56] +Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] ===== Subqueries ===== @@ -390,6 +390,6 @@ Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 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 f0be099379418..6e39745703215 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 @@ -229,78 +229,78 @@ Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] (39) HashAggregate [codegen id : 21] Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] Keys [5]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33] -Functions [1]: [sum(UnscaledValue(cs_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#3))#35] -Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#3))#35,17,2) AS sum_sales#36] +Functions [1]: [sum(UnscaledValue(cs_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#35))#36] +Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#35))#36,17,2) AS sum_sales#37] (40) Exchange -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] (41) Sort [codegen id : 22] -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 (42) Window -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] -Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] (43) Project [codegen id : 23] -Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] +Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] (44) Exchange -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#38 + 1), 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] (45) Sort [codegen id : 24] -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#38 + 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 (46) SortMergeJoin [codegen id : 25] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#38 + 1)] +Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#39 + 1)] Join condition: None (47) Project [codegen id : 25] -Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] -Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] +Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] +Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] (48) ReusedExchange [Reuses operator id: 40] -Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Output [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] (49) Sort [codegen id : 34] -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 (50) Window -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] +Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] +Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#41, i_brand#42, cc_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#41, i_brand#42, cc_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] (51) Project [codegen id : 35] -Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] +Output [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] (52) Exchange -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (rn#46 - 1), 5), ENSURE_REQUIREMENTS, [id=#47] +Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Arguments: hashpartitioning(i_category#41, i_brand#42, cc_name#43, (rn#47 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] (53) Sort [codegen id : 36] -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, (rn#47 - 1) ASC NULLS FIRST], false, 0 (54) SortMergeJoin [codegen id : 37] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (rn#46 - 1)] +Right keys [4]: [i_category#41, i_brand#42, cc_name#43, (rn#47 - 1)] Join condition: None (55) Project [codegen id : 37] -Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#48, sum_sales#45 AS nsum#49] -Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] +Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#49, sum_sales#46 AS nsum#50] +Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] (56) TakeOrderedAndProject -Input [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] +Input [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] ===== Subqueries ===== 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 1eb7dbe89d6e1..fa24f5ede5453 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 @@ -202,70 +202,70 @@ Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] (34) HashAggregate [codegen id : 13] 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#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#33] -Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#33,17,2) AS sum_sales#34] +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] (35) Exchange -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#35] +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] (36) Sort [codegen id : 14] -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35] 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 (37) Window -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] -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#36], [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#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] (38) Project [codegen id : 15] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] -Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34, rn#36] +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] (39) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#37] +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#38] (40) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#36 + 1)] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] Join condition: None (41) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34] -Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] +Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35] +Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] (42) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] +Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] (43) Sort [codegen id : 21] -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] -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 +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 (44) Window -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] -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#44], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] +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] (45) Project [codegen id : 22] -Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] -Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43, rn#44] +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] (46) BroadcastExchange -Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#45] +Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#46] (47) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -Right keys [4]: [i_category#38, i_brand#39, cc_name#40, (rn#44 - 1)] +Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] Join condition: None (48) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#34 AS psum#46, sum_sales#43 AS nsum#47] -Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34, i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] +Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] +Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] (49) TakeOrderedAndProject -Input [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, 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#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, psum#46, nsum#47] +Input [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, 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#24 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, psum#47, nsum#48] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index f6dd5a59de4d4..b339df0707d2f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -110,30 +110,30 @@ Input [2]: [d_date_sk#4, d_date#5] Condition : isnotnull(d_date_sk#4) (7) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#6, d_week_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (8) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#6, d_week_seq#7] (9) Filter [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = Subquery scalar-subquery#7, [id=#8])) +Input [2]: [d_date#6, d_week_seq#7] +Condition : (isnotnull(d_week_seq#7) AND (d_week_seq#7 = Subquery scalar-subquery#8, [id=#9])) (10) Project [codegen id : 1] -Output [1]: [d_date#5 AS d_date#5#9] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_date#6] +Input [2]: [d_date#6, d_week_seq#7] (11) BroadcastExchange -Input [1]: [d_date#5#9] +Input [1]: [d_date#6] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] (12) BroadcastHashJoin [codegen id : 2] Left keys [1]: [d_date#5] -Right keys [1]: [d_date#5#9] +Right keys [1]: [d_date#6] Join condition: None (13) Project [codegen id : 2] @@ -218,237 +218,237 @@ Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Condition : isnotnull(cs_item_sk#21) (30) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] +Output [2]: [d_date_sk#24, d_date#25] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#4, d_date#5] +Input [2]: [d_date_sk#24, d_date#25] (32) Filter [codegen id : 6] -Input [2]: [d_date_sk#4, d_date#5] -Condition : isnotnull(d_date_sk#4) +Input [2]: [d_date_sk#24, d_date#25] +Condition : isnotnull(d_date_sk#24) (33) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#26, d_week_seq#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (34) ColumnarToRow [codegen id : 5] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#26, d_week_seq#27] (35) Filter [codegen id : 5] -Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) +Input [2]: [d_date#26, d_week_seq#27] +Condition : (isnotnull(d_week_seq#27) AND (d_week_seq#27 = ReusedSubquery Subquery scalar-subquery#8, [id=#9])) (36) Project [codegen id : 5] -Output [1]: [d_date#5 AS d_date#5#24] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_date#26] +Input [2]: [d_date#26, d_week_seq#27] (37) BroadcastExchange -Input [1]: [d_date#5#24] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#25] +Input [1]: [d_date#26] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#28] (38) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [d_date#5] -Right keys [1]: [d_date#5#24] +Left keys [1]: [d_date#25] +Right keys [1]: [d_date#26] Join condition: None (39) Project [codegen id : 6] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_date#25] (40) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#4] +Right keys [1]: [d_date_sk#24] Join condition: None (42) Project [codegen id : 8] Output [2]: [cs_item_sk#21, cs_ext_sales_price#22] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#4] +Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, d_date_sk#24] (43) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#12, i_item_id#13] +Output [2]: [i_item_sk#30, i_item_id#31] (44) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#12] +Right keys [1]: [i_item_sk#30] Join condition: None (45) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#22, i_item_id#13] -Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#12, i_item_id#13] +Output [2]: [cs_ext_sales_price#22, i_item_id#31] +Input [4]: [cs_item_sk#21, cs_ext_sales_price#22, i_item_sk#30, i_item_id#31] (46) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#22, i_item_id#13] -Keys [1]: [i_item_id#13] +Input [2]: [cs_ext_sales_price#22, i_item_id#31] +Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_item_id#13, sum#28] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#31, sum#33] (47) Exchange -Input [2]: [i_item_id#13, sum#28] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [2]: [i_item_id#31, sum#33] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] (48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#13, sum#28] -Keys [1]: [i_item_id#13] +Input [2]: [i_item_id#31, sum#33] +Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#13 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#35] +Results [2]: [i_item_id#31 AS item_id#36, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#35,17,2) AS cs_item_rev#37] (49) Filter [codegen id : 9] -Input [2]: [item_id#31, cs_item_rev#32] -Condition : isnotnull(cs_item_rev#32) +Input [2]: [item_id#36, cs_item_rev#37] +Condition : isnotnull(cs_item_rev#37) (50) BroadcastExchange -Input [2]: [item_id#31, cs_item_rev#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] +Input [2]: [item_id#36, cs_item_rev#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#38] (51) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#31] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) +Right keys [1]: [item_id#36] +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) (52) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] -Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] +Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#37] +Input [4]: [item_id#19, ss_item_rev#20, item_id#36, cs_item_rev#37] (53) Scan parquet default.web_sales -Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] (55) Filter [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#34) +Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] +Condition : isnotnull(ws_item_sk#39) (56) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] +Output [2]: [d_date_sk#42, d_date#43] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 11] -Input [2]: [d_date_sk#4, d_date#5] +Input [2]: [d_date_sk#42, d_date#43] (58) Filter [codegen id : 11] -Input [2]: [d_date_sk#4, d_date#5] -Condition : isnotnull(d_date_sk#4) +Input [2]: [d_date_sk#42, d_date#43] +Condition : isnotnull(d_date_sk#42) (59) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#44, d_week_seq#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (60) ColumnarToRow [codegen id : 10] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#44, d_week_seq#45] (61) Filter [codegen id : 10] -Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_week_seq#6) AND (d_week_seq#6 = ReusedSubquery Subquery scalar-subquery#7, [id=#8])) +Input [2]: [d_date#44, d_week_seq#45] +Condition : (isnotnull(d_week_seq#45) AND (d_week_seq#45 = ReusedSubquery Subquery scalar-subquery#8, [id=#9])) (62) Project [codegen id : 10] -Output [1]: [d_date#5 AS d_date#5#37] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_date#44] +Input [2]: [d_date#44, d_week_seq#45] (63) BroadcastExchange -Input [1]: [d_date#5#37] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#38] +Input [1]: [d_date#44] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#46] (64) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_date#5] -Right keys [1]: [d_date#5#37] +Left keys [1]: [d_date#43] +Right keys [1]: [d_date#44] Join condition: None (65) Project [codegen id : 11] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +Output [1]: [d_date_sk#42] +Input [2]: [d_date_sk#42, d_date#43] (66) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Input [1]: [d_date_sk#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] (67) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#4] +Left keys [1]: [ws_sold_date_sk#41] +Right keys [1]: [d_date_sk#42] Join condition: None (68) Project [codegen id : 13] -Output [2]: [ws_item_sk#34, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#4] +Output [2]: [ws_item_sk#39, ws_ext_sales_price#40] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41, d_date_sk#42] (69) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#12, i_item_id#13] +Output [2]: [i_item_sk#48, i_item_id#49] (70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#12] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#48] Join condition: None (71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#35, i_item_id#13] -Input [4]: [ws_item_sk#34, ws_ext_sales_price#35, i_item_sk#12, i_item_id#13] +Output [2]: [ws_ext_sales_price#40, i_item_id#49] +Input [4]: [ws_item_sk#39, ws_ext_sales_price#40, i_item_sk#48, i_item_id#49] (72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#35, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#13, sum#41] +Input [2]: [ws_ext_sales_price#40, i_item_id#49] +Keys [1]: [i_item_id#49] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#40))] +Aggregate Attributes [1]: [sum#50] +Results [2]: [i_item_id#49, sum#51] (73) Exchange -Input [2]: [i_item_id#13, sum#41] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [2]: [i_item_id#49, sum#51] +Arguments: hashpartitioning(i_item_id#49, 5), ENSURE_REQUIREMENTS, [id=#52] (74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#13, sum#41] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] -Results [2]: [i_item_id#13 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] +Input [2]: [i_item_id#49, sum#51] +Keys [1]: [i_item_id#49] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#40))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#40))#53] +Results [2]: [i_item_id#49 AS item_id#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#40))#53,17,2) AS ws_item_rev#55] (75) Filter [codegen id : 14] -Input [2]: [item_id#44, ws_item_rev#45] -Condition : isnotnull(ws_item_rev#45) +Input [2]: [item_id#54, ws_item_rev#55] +Condition : isnotnull(ws_item_rev#55) (76) BroadcastExchange -Input [2]: [item_id#44, ws_item_rev#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] +Input [2]: [item_id#54, ws_item_rev#55] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] (77) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#44] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) +Right keys [1]: [item_id#54] +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) (78) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#57, cs_item_rev#37, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#37 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#58, ws_item_rev#55, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#55 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#59, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#60] +Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#37, item_id#54, ws_item_rev#55] (79) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] ===== Subqueries ===== -Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#7, [id=#8] +Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery#8, [id=#9] * Project (83) +- * Filter (82) +- * ColumnarToRow (81) @@ -456,25 +456,25 @@ Subquery:1 Hosting operator id = 9 Hosting Expression = Subquery scalar-subquery (80) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#61, d_week_seq#62] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct (81) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#61, d_week_seq#62] (82) Filter [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] -Condition : (isnotnull(d_date#5) AND (d_date#5 = 2000-01-03)) +Input [2]: [d_date#61, d_week_seq#62] +Condition : (isnotnull(d_date#61) AND (d_date#61 = 2000-01-03)) (83) Project [codegen id : 1] -Output [1]: [d_week_seq#6] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_week_seq#62] +Input [2]: [d_date#61, d_week_seq#62] -Subquery:2 Hosting operator id = 35 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] +Subquery:2 Hosting operator id = 35 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] -Subquery:3 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#7, [id=#8] +Subquery:3 Hosting operator id = 61 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index ab4c4ad4ae65b..1f3e6853a3c41 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -137,30 +137,30 @@ Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) (13) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#9, d_week_seq#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (14) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#9, d_week_seq#10] (15) Filter [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = Subquery scalar-subquery#10, [id=#11])) +Input [2]: [d_date#9, d_week_seq#10] +Condition : (isnotnull(d_week_seq#10) AND (d_week_seq#10 = Subquery scalar-subquery#11, [id=#12])) (16) Project [codegen id : 2] -Output [1]: [d_date#8 AS d_date#8#12] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_date#9] +Input [2]: [d_date#9, d_week_seq#10] (17) BroadcastExchange -Input [1]: [d_date#8#12] +Input [1]: [d_date#9] Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] (18) BroadcastHashJoin [codegen id : 3] Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#12] +Right keys [1]: [d_date#9] Join condition: None (19) Project [codegen id : 3] @@ -218,237 +218,237 @@ Input [3]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23] Condition : isnotnull(cs_item_sk#21) (30) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#24, i_item_id#25] (31) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_item_sk#21] -Right keys [1]: [i_item_sk#4] +Right keys [1]: [i_item_sk#24] Join condition: None (32) Project [codegen id : 8] -Output [3]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#5] -Input [5]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_sk#4, i_item_id#5] +Output [3]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25] +Input [5]: [cs_item_sk#21, cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_sk#24, i_item_id#25] (33) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] +Output [2]: [d_date_sk#26, d_date#27] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 7] -Input [2]: [d_date_sk#7, d_date#8] +Input [2]: [d_date_sk#26, d_date#27] (35) Filter [codegen id : 7] -Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) +Input [2]: [d_date_sk#26, d_date#27] +Condition : isnotnull(d_date_sk#26) (36) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#28, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#28, d_week_seq#29] (38) Filter [codegen id : 6] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) +Input [2]: [d_date#28, d_week_seq#29] +Condition : (isnotnull(d_week_seq#29) AND (d_week_seq#29 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) (39) Project [codegen id : 6] -Output [1]: [d_date#8 AS d_date#8#24] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_date#28] +Input [2]: [d_date#28, d_week_seq#29] (40) BroadcastExchange -Input [1]: [d_date#8#24] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#25] +Input [1]: [d_date#28] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#30] (41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#24] +Left keys [1]: [d_date#27] +Right keys [1]: [d_date#28] Join condition: None (42) Project [codegen id : 7] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] +Output [1]: [d_date_sk#26] +Input [2]: [d_date_sk#26, d_date#27] (43) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [d_date_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (44) BroadcastHashJoin [codegen id : 8] Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#26] Join condition: None (45) Project [codegen id : 8] -Output [2]: [cs_ext_sales_price#22, i_item_id#5] -Input [4]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#5, d_date_sk#7] +Output [2]: [cs_ext_sales_price#22, i_item_id#25] +Input [4]: [cs_ext_sales_price#22, cs_sold_date_sk#23, i_item_id#25, d_date_sk#26] (46) HashAggregate [codegen id : 8] -Input [2]: [cs_ext_sales_price#22, i_item_id#5] -Keys [1]: [i_item_id#5] +Input [2]: [cs_ext_sales_price#22, i_item_id#25] +Keys [1]: [i_item_id#25] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_item_id#5, sum#28] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#25, sum#33] (47) Exchange -Input [2]: [i_item_id#5, sum#28] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [2]: [i_item_id#25, sum#33] +Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [id=#34] (48) HashAggregate [codegen id : 9] -Input [2]: [i_item_id#5, sum#28] -Keys [1]: [i_item_id#5] +Input [2]: [i_item_id#25, sum#33] +Keys [1]: [i_item_id#25] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#30] -Results [2]: [i_item_id#5 AS item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#30,17,2) AS cs_item_rev#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#22))#35] +Results [2]: [i_item_id#25 AS item_id#36, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#22))#35,17,2) AS cs_item_rev#37] (49) Filter [codegen id : 9] -Input [2]: [item_id#31, cs_item_rev#32] -Condition : isnotnull(cs_item_rev#32) +Input [2]: [item_id#36, cs_item_rev#37] +Condition : isnotnull(cs_item_rev#37) (50) BroadcastExchange -Input [2]: [item_id#31, cs_item_rev#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#33] +Input [2]: [item_id#36, cs_item_rev#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#38] (51) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#31] -Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) +Right keys [1]: [item_id#36] +Join condition: ((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) (52) Project [codegen id : 15] -Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#32] -Input [4]: [item_id#19, ss_item_rev#20, item_id#31, cs_item_rev#32] +Output [3]: [item_id#19, ss_item_rev#20, cs_item_rev#37] +Input [4]: [item_id#19, ss_item_rev#20, item_id#36, cs_item_rev#37] (53) Scan parquet default.web_sales -Output [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(ws_sold_date_sk#41), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] (55) Filter [codegen id : 13] -Input [3]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : isnotnull(ws_item_sk#34) +Input [3]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41] +Condition : isnotnull(ws_item_sk#39) (56) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#42, i_item_id#43] (57) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_item_sk#34] -Right keys [1]: [i_item_sk#4] +Left keys [1]: [ws_item_sk#39] +Right keys [1]: [i_item_sk#42] Join condition: None (58) Project [codegen id : 13] -Output [3]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#5] -Input [5]: [ws_item_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_sk#4, i_item_id#5] +Output [3]: [ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_id#43] +Input [5]: [ws_item_sk#39, ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_sk#42, i_item_id#43] (59) Scan parquet default.date_dim -Output [2]: [d_date_sk#7, d_date#8] +Output [2]: [d_date_sk#44, d_date#45] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 12] -Input [2]: [d_date_sk#7, d_date#8] +Input [2]: [d_date_sk#44, d_date#45] (61) Filter [codegen id : 12] -Input [2]: [d_date_sk#7, d_date#8] -Condition : isnotnull(d_date_sk#7) +Input [2]: [d_date_sk#44, d_date#45] +Condition : isnotnull(d_date_sk#44) (62) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#46, d_week_seq#47] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq)] ReadSchema: struct (63) ColumnarToRow [codegen id : 11] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#46, d_week_seq#47] (64) Filter [codegen id : 11] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_week_seq#9) AND (d_week_seq#9 = ReusedSubquery Subquery scalar-subquery#10, [id=#11])) +Input [2]: [d_date#46, d_week_seq#47] +Condition : (isnotnull(d_week_seq#47) AND (d_week_seq#47 = ReusedSubquery Subquery scalar-subquery#11, [id=#12])) (65) Project [codegen id : 11] -Output [1]: [d_date#8 AS d_date#8#37] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_date#46] +Input [2]: [d_date#46, d_week_seq#47] (66) BroadcastExchange -Input [1]: [d_date#8#37] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#38] +Input [1]: [d_date#46] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#48] (67) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#37] +Left keys [1]: [d_date#45] +Right keys [1]: [d_date#46] Join condition: None (68) Project [codegen id : 12] -Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_date#8] +Output [1]: [d_date_sk#44] +Input [2]: [d_date_sk#44, d_date#45] (69) BroadcastExchange -Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Input [1]: [d_date_sk#44] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] (70) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [ws_sold_date_sk#41] +Right keys [1]: [d_date_sk#44] Join condition: None (71) Project [codegen id : 13] -Output [2]: [ws_ext_sales_price#35, i_item_id#5] -Input [4]: [ws_ext_sales_price#35, ws_sold_date_sk#36, i_item_id#5, d_date_sk#7] +Output [2]: [ws_ext_sales_price#40, i_item_id#43] +Input [4]: [ws_ext_sales_price#40, ws_sold_date_sk#41, i_item_id#43, d_date_sk#44] (72) HashAggregate [codegen id : 13] -Input [2]: [ws_ext_sales_price#35, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#40] -Results [2]: [i_item_id#5, sum#41] +Input [2]: [ws_ext_sales_price#40, i_item_id#43] +Keys [1]: [i_item_id#43] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#40))] +Aggregate Attributes [1]: [sum#50] +Results [2]: [i_item_id#43, sum#51] (73) Exchange -Input [2]: [i_item_id#5, sum#41] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [2]: [i_item_id#43, sum#51] +Arguments: hashpartitioning(i_item_id#43, 5), ENSURE_REQUIREMENTS, [id=#52] (74) HashAggregate [codegen id : 14] -Input [2]: [i_item_id#5, sum#41] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] -Results [2]: [i_item_id#5 AS item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS ws_item_rev#45] +Input [2]: [i_item_id#43, sum#51] +Keys [1]: [i_item_id#43] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#40))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#40))#53] +Results [2]: [i_item_id#43 AS item_id#54, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#40))#53,17,2) AS ws_item_rev#55] (75) Filter [codegen id : 14] -Input [2]: [item_id#44, ws_item_rev#45] -Condition : isnotnull(ws_item_rev#45) +Input [2]: [item_id#54, ws_item_rev#55] +Condition : isnotnull(ws_item_rev#55) (76) BroadcastExchange -Input [2]: [item_id#44, ws_item_rev#45] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#46] +Input [2]: [item_id#54, ws_item_rev#55] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#56] (77) BroadcastHashJoin [codegen id : 15] Left keys [1]: [item_id#19] -Right keys [1]: [item_id#44] -Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(cs_item_rev#32 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#45)), DecimalType(19,3), true))) AND (cast(cs_item_rev#32 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#45)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#45 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#32)), DecimalType(19,3), true))) AND (cast(ws_item_rev#45 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#32)), DecimalType(20,3), true))) +Right keys [1]: [item_id#54] +Join condition: ((((((((cast(ss_item_rev#20 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true)) AND (cast(ss_item_rev#20 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(cs_item_rev#37 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#55)), DecimalType(19,3), true))) AND (cast(cs_item_rev#37 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#55)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#20)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#20)), DecimalType(20,3), true))) AND (cast(ws_item_rev#55 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#37)), DecimalType(19,3), true))) AND (cast(ws_item_rev#55 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#37)), DecimalType(20,3), true))) (78) Project [codegen id : 15] -Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#47, cs_item_rev#32, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#32 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#48, ws_item_rev#45, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#45 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#32 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#45 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#50] -Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#32, item_id#44, ws_item_rev#45] +Output [8]: [item_id#19, ss_item_rev#20, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ss_dev#57, cs_item_rev#37, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(cs_item_rev#37 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS cs_dev#58, ws_item_rev#55, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ws_item_rev#55 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true))), DecimalType(38,21), true)) / 3.000000000000000000000), DecimalType(38,21), true)) * 100.000000000000000000000), DecimalType(38,17), true) AS ws_dev#59, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#20 as decimal(18,2))) + promote_precision(cast(cs_item_rev#37 as decimal(18,2)))), DecimalType(18,2), true) as decimal(19,2))) + promote_precision(cast(ws_item_rev#55 as decimal(19,2)))), DecimalType(19,2), true)) / 3.00), DecimalType(23,6), true) AS average#60] +Input [5]: [item_id#19, ss_item_rev#20, cs_item_rev#37, item_id#54, ws_item_rev#55] (79) TakeOrderedAndProject -Input [8]: [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] -Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#47, cs_item_rev#32, cs_dev#48, ws_item_rev#45, ws_dev#49, average#50] +Input [8]: [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] +Arguments: 100, [item_id#19 ASC NULLS FIRST, ss_item_rev#20 ASC NULLS FIRST], [item_id#19, ss_item_rev#20, ss_dev#57, cs_item_rev#37, cs_dev#58, ws_item_rev#55, ws_dev#59, average#60] ===== Subqueries ===== -Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#10, [id=#11] +Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquery#11, [id=#12] * Project (83) +- * Filter (82) +- * ColumnarToRow (81) @@ -456,25 +456,25 @@ Subquery:1 Hosting operator id = 15 Hosting Expression = Subquery scalar-subquer (80) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#61, d_week_seq#62] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct (81) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#61, d_week_seq#62] (82) Filter [codegen id : 1] -Input [2]: [d_date#8, d_week_seq#9] -Condition : (isnotnull(d_date#8) AND (d_date#8 = 2000-01-03)) +Input [2]: [d_date#61, d_week_seq#62] +Condition : (isnotnull(d_date#61) AND (d_date#61 = 2000-01-03)) (83) Project [codegen id : 1] -Output [1]: [d_week_seq#9] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_week_seq#62] +Input [2]: [d_date#61, d_week_seq#62] -Subquery:2 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] +Subquery:2 Hosting operator id = 38 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] -Subquery:3 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#10, [id=#11] +Subquery:3 Hosting operator id = 64 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] 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 fb57c1b00ecf3..cf22f178dd33f 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#5, ss_store_sk#1, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] +Output [9]: [d_week_seq#54, ss_store_sk#55, sum#56, sum#57, sum#58, sum#59, sum#60, sum#61, sum#62] (27) HashAggregate [codegen id : 9] -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))#61, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67] -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))#61,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63,17,2) AS tue_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67,17,2) AS sat_sales#36] +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] (28) Scan parquet default.store -Output [2]: [s_store_sk#37, s_store_id#38] +Output [2]: [s_store_sk#72, s_store_id#73] 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#37, s_store_id#38] +Input [2]: [s_store_sk#72, s_store_id#73] (30) Filter [codegen id : 7] -Input [2]: [s_store_sk#37, s_store_id#38] -Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) +Input [2]: [s_store_sk#72, s_store_id#73] +Condition : (isnotnull(s_store_sk#72) AND isnotnull(s_store_id#73)) (31) BroadcastExchange -Input [2]: [s_store_sk#37, s_store_id#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#68] +Input [2]: [s_store_sk#72, s_store_id#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#37] +Left keys [1]: [ss_store_sk#55] +Right keys [1]: [s_store_sk#72] Join condition: None (33) Project [codegen id : 9] -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#38] -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#37, s_store_id#38] +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] (34) Scan parquet default.date_dim -Output [2]: [d_month_seq#69, d_week_seq#70] +Output [2]: [d_month_seq#75, d_week_seq#76] 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#69, d_week_seq#70] +Input [2]: [d_month_seq#75, d_week_seq#76] (36) Filter [codegen id : 8] -Input [2]: [d_month_seq#69, d_week_seq#70] -Condition : (((isnotnull(d_month_seq#69) AND (d_month_seq#69 >= 1224)) AND (d_month_seq#69 <= 1235)) AND isnotnull(d_week_seq#70)) +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)) (37) Project [codegen id : 8] -Output [1]: [d_week_seq#70] -Input [2]: [d_month_seq#69, d_week_seq#70] +Output [1]: [d_week_seq#76] +Input [2]: [d_month_seq#75, d_week_seq#76] (38) BroadcastExchange -Input [1]: [d_week_seq#70] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] +Input [1]: [d_week_seq#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] (39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#70] +Left keys [1]: [d_week_seq#54] +Right keys [1]: [d_week_seq#76] Join condition: None (40) Project [codegen id : 9] -Output [9]: [d_week_seq#5 AS d_week_seq2#72, s_store_id#38 AS s_store_id2#73, sun_sales#30 AS sun_sales2#74, mon_sales#31 AS mon_sales2#75, tue_sales#32 AS tue_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 [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#38, d_week_seq#70] +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] (41) BroadcastExchange -Input [9]: [d_week_seq2#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_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] +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] (42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#73, (d_week_seq2#72 - 52)] +Right keys [2]: [s_store_id2#79, (d_week_seq2#78 - 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#74)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#82, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#75)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#83, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#76)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#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 [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#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] +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] (44) TakeOrderedAndProject -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_sales2)#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_sales2)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] +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] 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 fb57c1b00ecf3..cf22f178dd33f 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#5, ss_store_sk#1, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59, sum#60] +Output [9]: [d_week_seq#54, ss_store_sk#55, sum#56, sum#57, sum#58, sum#59, sum#60, sum#61, sum#62] (27) HashAggregate [codegen id : 9] -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))#61, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67] -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))#61,17,2) AS sun_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 ELSE null END))#62,17,2) AS mon_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 ELSE null END))#63,17,2) AS tue_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 ELSE null END))#64,17,2) AS wed_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 ELSE null END))#65,17,2) AS thu_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 ELSE null END))#66,17,2) AS fri_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 ELSE null END))#67,17,2) AS sat_sales#36] +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] (28) Scan parquet default.store -Output [2]: [s_store_sk#37, s_store_id#38] +Output [2]: [s_store_sk#72, s_store_id#73] 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#37, s_store_id#38] +Input [2]: [s_store_sk#72, s_store_id#73] (30) Filter [codegen id : 7] -Input [2]: [s_store_sk#37, s_store_id#38] -Condition : (isnotnull(s_store_sk#37) AND isnotnull(s_store_id#38)) +Input [2]: [s_store_sk#72, s_store_id#73] +Condition : (isnotnull(s_store_sk#72) AND isnotnull(s_store_id#73)) (31) BroadcastExchange -Input [2]: [s_store_sk#37, s_store_id#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#68] +Input [2]: [s_store_sk#72, s_store_id#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#37] +Left keys [1]: [ss_store_sk#55] +Right keys [1]: [s_store_sk#72] Join condition: None (33) Project [codegen id : 9] -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#38] -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#37, s_store_id#38] +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] (34) Scan parquet default.date_dim -Output [2]: [d_month_seq#69, d_week_seq#70] +Output [2]: [d_month_seq#75, d_week_seq#76] 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#69, d_week_seq#70] +Input [2]: [d_month_seq#75, d_week_seq#76] (36) Filter [codegen id : 8] -Input [2]: [d_month_seq#69, d_week_seq#70] -Condition : (((isnotnull(d_month_seq#69) AND (d_month_seq#69 >= 1224)) AND (d_month_seq#69 <= 1235)) AND isnotnull(d_week_seq#70)) +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)) (37) Project [codegen id : 8] -Output [1]: [d_week_seq#70] -Input [2]: [d_month_seq#69, d_week_seq#70] +Output [1]: [d_week_seq#76] +Input [2]: [d_month_seq#75, d_week_seq#76] (38) BroadcastExchange -Input [1]: [d_week_seq#70] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#71] +Input [1]: [d_week_seq#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] (39) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#70] +Left keys [1]: [d_week_seq#54] +Right keys [1]: [d_week_seq#76] Join condition: None (40) Project [codegen id : 9] -Output [9]: [d_week_seq#5 AS d_week_seq2#72, s_store_id#38 AS s_store_id2#73, sun_sales#30 AS sun_sales2#74, mon_sales#31 AS mon_sales2#75, tue_sales#32 AS tue_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 [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#38, d_week_seq#70] +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] (41) BroadcastExchange -Input [9]: [d_week_seq2#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_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] +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] (42) BroadcastHashJoin [codegen id : 10] Left keys [2]: [s_store_id1#46, d_week_seq1#45] -Right keys [2]: [s_store_id2#73, (d_week_seq2#72 - 52)] +Right keys [2]: [s_store_id2#79, (d_week_seq2#78 - 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#74)), DecimalType(37,20), true) AS (sun_sales1 / sun_sales2)#82, CheckOverflow((promote_precision(mon_sales1#48) / promote_precision(mon_sales2#75)), DecimalType(37,20), true) AS (mon_sales1 / mon_sales2)#83, CheckOverflow((promote_precision(tue_sales1#49) / promote_precision(tue_sales2#76)), DecimalType(37,20), true) AS (tue_sales1 / tue_sales2)#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 [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#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] +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] (44) TakeOrderedAndProject -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_sales2)#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_sales2)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] +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] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt index 10b8d0d9e7f05..49cfd232239aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt @@ -160,30 +160,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#14, i_category#15] +Output [2]: [i_item_id#15, i_category#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#14, i_category#15] +Input [2]: [i_item_id#15, i_category#16] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#14, i_category#15] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) +Input [2]: [i_item_id#15, i_category#16] +Condition : (isnotnull(i_category#16) AND (i_category#16 = Music )) (24) Project [codegen id : 3] -Output [1]: [i_item_id#14 AS i_item_id#14#16] -Input [2]: [i_item_id#14, i_category#15] +Output [1]: [i_item_id#15] +Input [2]: [i_item_id#15, i_category#16] (25) BroadcastExchange -Input [1]: [i_item_id#14#16] +Input [1]: [i_item_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#14#16] +Right keys [1]: [i_item_id#15] Join condition: None (27) BroadcastExchange @@ -233,127 +233,127 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#30, i_item_id#31] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] +Output [2]: [cs_ext_sales_price#26, i_item_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#14] -Keys [1]: [i_item_id#14] +Input [2]: [cs_ext_sales_price#26, i_item_id#31] +Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#14, sum#29] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#31, sum#33] (46) Exchange -Input [2]: [i_item_id#14, sum#29] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [i_item_id#31, sum#33] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#14, sum#29] -Keys [1]: [i_item_id#14] +Input [2]: [i_item_id#31, sum#33] +Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] (48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] (50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#41] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#42] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#43, i_item_id#44] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_item_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] +Output [2]: [ws_ext_sales_price#39, i_item_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_item_id#14, sum#38] +Input [2]: [ws_ext_sales_price#39, i_item_id#44] +Keys [1]: [i_item_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_item_id#44, sum#46] (61) Exchange -Input [2]: [i_item_id#14, sum#38] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [2]: [i_item_id#44, sum#46] +Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] (62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#38] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +Input [2]: [i_item_id#44, sum#46] +Keys [1]: [i_item_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] (63) Union @@ -361,23 +361,23 @@ Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35) Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_item_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_item_id#14, sum#52, isEmpty#53] (65) Exchange -Input [3]: [i_item_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] (66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#44, isEmpty#45] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] (67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#48] -Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] +Input [2]: [i_item_id#14, total_sales#56] +Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] ===== Subqueries ===== @@ -390,6 +390,6 @@ Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 10b8d0d9e7f05..49cfd232239aa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -160,30 +160,30 @@ Input [2]: [i_item_sk#13, i_item_id#14] Condition : isnotnull(i_item_sk#13) (21) Scan parquet default.item -Output [2]: [i_item_id#14, i_category#15] +Output [2]: [i_item_id#15, i_category#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] ReadSchema: struct (22) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#14, i_category#15] +Input [2]: [i_item_id#15, i_category#16] (23) Filter [codegen id : 3] -Input [2]: [i_item_id#14, i_category#15] -Condition : (isnotnull(i_category#15) AND (i_category#15 = Music )) +Input [2]: [i_item_id#15, i_category#16] +Condition : (isnotnull(i_category#16) AND (i_category#16 = Music )) (24) Project [codegen id : 3] -Output [1]: [i_item_id#14 AS i_item_id#14#16] -Input [2]: [i_item_id#14, i_category#15] +Output [1]: [i_item_id#15] +Input [2]: [i_item_id#15, i_category#16] (25) BroadcastExchange -Input [1]: [i_item_id#14#16] +Input [1]: [i_item_id#15] Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#17] (26) BroadcastHashJoin [codegen id : 4] Left keys [1]: [i_item_id#14] -Right keys [1]: [i_item_id#14#16] +Right keys [1]: [i_item_id#15] Join condition: None (27) BroadcastExchange @@ -233,127 +233,127 @@ Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_da Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#6] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#29] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#10] +Right keys [1]: [ca_address_sk#29] Join condition: None (41) Project [codegen id : 11] Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] -Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#10] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#29] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#30, i_item_id#31] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#13] +Right keys [1]: [i_item_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_item_id#14] -Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#13, i_item_id#14] +Output [2]: [cs_ext_sales_price#26, i_item_id#31] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#30, i_item_id#31] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_item_id#14] -Keys [1]: [i_item_id#14] +Input [2]: [cs_ext_sales_price#26, i_item_id#31] +Keys [1]: [i_item_id#31] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [i_item_id#14, sum#29] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#31, sum#33] (46) Exchange -Input [2]: [i_item_id#14, sum#29] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [i_item_id#31, sum#33] +Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 12] -Input [2]: [i_item_id#14, sum#29] -Keys [1]: [i_item_id#14] +Input [2]: [i_item_id#31, sum#33] +Keys [1]: [i_item_id#31] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#31] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#31,17,2) AS total_sales#32] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_item_id#31, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] (48) Scan parquet default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] (50) Filter [codegen id : 17] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#41] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#36] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None (53) Project [codegen id : 17] -Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#6] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#10] +Output [1]: [ca_address_sk#42] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#42] Join condition: None (56) Project [codegen id : 17] -Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#10] +Output [2]: [ws_item_sk#37, ws_ext_sales_price#39] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ca_address_sk#42] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#13, i_item_id#14] +Output [2]: [i_item_sk#43, i_item_id#44] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#13] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#43] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_item_id#14] -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#13, i_item_id#14] +Output [2]: [ws_ext_sales_price#39, i_item_id#44] +Input [4]: [ws_item_sk#37, ws_ext_sales_price#39, i_item_sk#43, i_item_id#44] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_item_id#14] -Keys [1]: [i_item_id#14] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum#37] -Results [2]: [i_item_id#14, sum#38] +Input [2]: [ws_ext_sales_price#39, i_item_id#44] +Keys [1]: [i_item_id#44] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_item_id#44, sum#46] (61) Exchange -Input [2]: [i_item_id#14, sum#38] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [2]: [i_item_id#44, sum#46] +Arguments: hashpartitioning(i_item_id#44, 5), ENSURE_REQUIREMENTS, [id=#47] (62) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#14, sum#38] -Keys [1]: [i_item_id#14] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#40] -Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#40,17,2) AS total_sales#41] +Input [2]: [i_item_id#44, sum#46] +Keys [1]: [i_item_id#44] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_item_id#44, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] (63) Union @@ -361,23 +361,23 @@ Results [2]: [i_item_id#14, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35) Input [2]: [i_item_id#14, total_sales#23] Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(total_sales#23)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_item_id#14, sum#44, isEmpty#45] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_item_id#14, sum#52, isEmpty#53] (65) Exchange -Input [3]: [i_item_id#14, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#54] (66) HashAggregate [codegen id : 20] -Input [3]: [i_item_id#14, sum#44, isEmpty#45] +Input [3]: [i_item_id#14, sum#52, isEmpty#53] Keys [1]: [i_item_id#14] Functions [1]: [sum(total_sales#23)] -Aggregate Attributes [1]: [sum(total_sales#23)#47] -Results [2]: [i_item_id#14, sum(total_sales#23)#47 AS total_sales#48] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_item_id#14, sum(total_sales#23)#55 AS total_sales#56] (67) TakeOrderedAndProject -Input [2]: [i_item_id#14, total_sales#48] -Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#48 ASC NULLS FIRST], [i_item_id#14, total_sales#48] +Input [2]: [i_item_id#14, total_sales#56] +Arguments: 100, [i_item_id#14 ASC NULLS FIRST, total_sales#56 ASC NULLS FIRST], [i_item_id#14, total_sales#56] ===== Subqueries ===== @@ -390,6 +390,6 @@ Output [1]: [d_date_sk#6] Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt index 23807f0561551..410fd9bc3d4e7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt @@ -286,100 +286,100 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#32] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#32,17,2) AS promotions#33] (48) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] +Output [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] (50) Filter [codegen id : 13] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_store_sk#36) AND isnotnull(ss_customer_sk#35)) AND isnotnull(ss_item_sk#34)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#39] (52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ss_sold_date_sk#38] +Right keys [1]: [d_date_sk#39] Join condition: None (53) Project [codegen id : 13] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] +Output [4]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37] +Input [6]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38, d_date_sk#39] (54) ReusedExchange [Reuses operator id: 15] -Output [1]: [i_item_sk#12] +Output [1]: [i_item_sk#40] (55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#12] +Left keys [1]: [ss_item_sk#34] +Right keys [1]: [i_item_sk#40] Join condition: None (56) Project [codegen id : 13] -Output [3]: [ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, i_item_sk#12] +Output [3]: [ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37] +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, i_item_sk#40] (57) ReusedExchange [Reuses operator id: 29] -Output [1]: [s_store_sk#20] +Output [1]: [s_store_sk#41] (58) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#20] +Left keys [1]: [ss_store_sk#36] +Right keys [1]: [s_store_sk#41] Join condition: None (59) Project [codegen id : 13] -Output [2]: [ss_customer_sk#2, ss_ext_sales_price#5] -Input [4]: [ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, s_store_sk#20] +Output [2]: [ss_customer_sk#35, ss_ext_sales_price#37] +Input [4]: [ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, s_store_sk#41] (60) ReusedExchange [Reuses operator id: 42] -Output [1]: [c_customer_sk#23] +Output [1]: [c_customer_sk#42] (61) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#23] +Left keys [1]: [ss_customer_sk#35] +Right keys [1]: [c_customer_sk#42] Join condition: None (62) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#5] -Input [3]: [ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#23] +Output [1]: [ss_ext_sales_price#37] +Input [3]: [ss_customer_sk#35, ss_ext_sales_price#37, c_customer_sk#42] (63) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#5] +Input [1]: [ss_ext_sales_price#37] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum#34] -Results [1]: [sum#35] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#37))] +Aggregate Attributes [1]: [sum#43] +Results [1]: [sum#44] (64) Exchange -Input [1]: [sum#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#36] +Input [1]: [sum#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45] (65) HashAggregate [codegen id : 14] -Input [1]: [sum#35] +Input [1]: [sum#44] Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#37] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#37,17,2) AS total#38] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#37))#46] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#37))#46,17,2) AS total#47] (66) BroadcastExchange -Input [1]: [total#38] -Arguments: IdentityBroadcastMode, [id=#39] +Input [1]: [total#47] +Arguments: IdentityBroadcastMode, [id=#48] (67) BroadcastNestedLoopJoin [codegen id : 15] Join condition: None (68) Project [codegen id : 15] -Output [3]: [promotions#33, total#38, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#38 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] -Input [2]: [promotions#33, total#38] +Output [3]: [promotions#33, total#47, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#47 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#49] +Input [2]: [promotions#33, total#47] (69) Sort [codegen id : 15] -Input [3]: [promotions#33, total#38, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] -Arguments: [promotions#33 ASC NULLS FIRST, total#38 ASC NULLS FIRST], true, 0 +Input [3]: [promotions#33, total#47, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#49] +Arguments: [promotions#33 ASC NULLS FIRST, total#47 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -390,6 +390,6 @@ ReusedExchange (70) (70) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#8] -Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt index c0fad04ce4026..e0bee37e047cb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt @@ -289,112 +289,112 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#32] Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#32,17,2) AS promotions#33] (48) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] +Output [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 13] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] (50) Filter [codegen id : 13] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6] -Condition : ((isnotnull(ss_store_sk#3) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_item_sk#1)) +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38] +Condition : ((isnotnull(ss_store_sk#36) AND isnotnull(ss_customer_sk#35)) AND isnotnull(ss_item_sk#34)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [s_store_sk#8] +Output [1]: [s_store_sk#39] (52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#8] +Left keys [1]: [ss_store_sk#36] +Right keys [1]: [s_store_sk#39] Join condition: None (53) Project [codegen id : 13] -Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ext_sales_price#5, ss_sold_date_sk#6, s_store_sk#8] +Output [4]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37, ss_sold_date_sk#38] +Input [6]: [ss_item_sk#34, ss_customer_sk#35, ss_store_sk#36, ss_ext_sales_price#37, ss_sold_date_sk#38, s_store_sk#39] (54) ReusedExchange [Reuses operator id: 22] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#40] (55) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_date_sk#6] -Right keys [1]: [d_date_sk#16] +Left keys [1]: [ss_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] Join condition: None (56) Project [codegen id : 13] -Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6, d_date_sk#16] +Output [3]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37] +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37, ss_sold_date_sk#38, d_date_sk#40] (57) ReusedExchange [Reuses operator id: 28] -Output [2]: [c_customer_sk#20, c_current_addr_sk#21] +Output [2]: [c_customer_sk#41, c_current_addr_sk#42] (58) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#20] +Left keys [1]: [ss_customer_sk#35] +Right keys [1]: [c_customer_sk#41] Join condition: None (59) Project [codegen id : 13] -Output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#21] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#20, c_current_addr_sk#21] +Output [3]: [ss_item_sk#34, ss_ext_sales_price#37, c_current_addr_sk#42] +Input [5]: [ss_item_sk#34, ss_customer_sk#35, ss_ext_sales_price#37, c_customer_sk#41, c_current_addr_sk#42] (60) ReusedExchange [Reuses operator id: 35] -Output [1]: [ca_address_sk#23] +Output [1]: [ca_address_sk#43] (61) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [c_current_addr_sk#21] -Right keys [1]: [ca_address_sk#23] +Left keys [1]: [c_current_addr_sk#42] +Right keys [1]: [ca_address_sk#43] Join condition: None (62) Project [codegen id : 13] -Output [2]: [ss_item_sk#1, ss_ext_sales_price#5] -Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#21, ca_address_sk#23] +Output [2]: [ss_item_sk#34, ss_ext_sales_price#37] +Input [4]: [ss_item_sk#34, ss_ext_sales_price#37, c_current_addr_sk#42, ca_address_sk#43] (63) ReusedExchange [Reuses operator id: 42] -Output [1]: [i_item_sk#26] +Output [1]: [i_item_sk#44] (64) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#26] +Left keys [1]: [ss_item_sk#34] +Right keys [1]: [i_item_sk#44] Join condition: None (65) Project [codegen id : 13] -Output [1]: [ss_ext_sales_price#5] -Input [3]: [ss_item_sk#1, ss_ext_sales_price#5, i_item_sk#26] +Output [1]: [ss_ext_sales_price#37] +Input [3]: [ss_item_sk#34, ss_ext_sales_price#37, i_item_sk#44] (66) HashAggregate [codegen id : 13] -Input [1]: [ss_ext_sales_price#5] +Input [1]: [ss_ext_sales_price#37] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum#34] -Results [1]: [sum#35] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#37))] +Aggregate Attributes [1]: [sum#45] +Results [1]: [sum#46] (67) Exchange -Input [1]: [sum#35] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#36] +Input [1]: [sum#46] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#47] (68) HashAggregate [codegen id : 14] -Input [1]: [sum#35] +Input [1]: [sum#46] Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#37] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#37,17,2) AS total#38] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#37))#48] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#37))#48,17,2) AS total#49] (69) BroadcastExchange -Input [1]: [total#38] -Arguments: IdentityBroadcastMode, [id=#39] +Input [1]: [total#49] +Arguments: IdentityBroadcastMode, [id=#50] (70) BroadcastNestedLoopJoin [codegen id : 15] Join condition: None (71) Project [codegen id : 15] -Output [3]: [promotions#33, total#38, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#38 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] -Input [2]: [promotions#33, total#38] +Output [3]: [promotions#33, total#49, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#33 as decimal(15,4))) / promote_precision(cast(total#49 as decimal(15,4)))), DecimalType(35,20), true)) * 100.00000000000000000000), DecimalType(38,19), true) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#51] +Input [2]: [promotions#33, total#49] (72) Sort [codegen id : 15] -Input [3]: [promotions#33, total#38, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40] -Arguments: [promotions#33 ASC NULLS FIRST, total#38 ASC NULLS FIRST], true, 0 +Input [3]: [promotions#33, total#49, ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#51] +Arguments: [promotions#33 ASC NULLS FIRST, total#49 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -405,6 +405,6 @@ ReusedExchange (73) (73) ReusedExchange [Reuses operator id: 22] Output [1]: [d_date_sk#16] -Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt index bd8dc07f42967..931e72bba7182 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.sf100/explain.txt @@ -795,353 +795,353 @@ Input [17]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_str Arguments: [item_sk#112 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, store_zip#114 ASC NULLS FIRST], false, 0 (132) Scan parquet default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#129)] +PartitionFilters: [isnotnull(ss_sold_date_sk#140), dynamicpruningexpression(ss_sold_date_sk#140 IN dynamicpruning#141)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (133) ColumnarToRow [codegen id : 44] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] (134) Filter [codegen id : 44] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Condition : (((((((isnotnull(ss_item_sk#129) AND isnotnull(ss_ticket_number#136)) AND isnotnull(ss_store_sk#134)) AND isnotnull(ss_customer_sk#130)) AND isnotnull(ss_cdemo_sk#131)) AND isnotnull(ss_promo_sk#135)) AND isnotnull(ss_hdemo_sk#132)) AND isnotnull(ss_addr_sk#133)) (135) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint), 5), ENSURE_REQUIREMENTS, [id=#130] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Arguments: hashpartitioning(cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint), 5), ENSURE_REQUIREMENTS, [id=#142] (136) Sort [codegen id : 45] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [cast(ss_item_sk#1 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#8 as bigint) ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Arguments: [cast(ss_item_sk#129 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#136 as bigint) ASC NULLS FIRST], false, 0 (137) ReusedExchange [Reuses operator id: 10] -Output [2]: [sr_item_sk#15, sr_ticket_number#16] +Output [2]: [sr_item_sk#143, sr_ticket_number#144] (138) Sort [codegen id : 47] -Input [2]: [sr_item_sk#15, sr_ticket_number#16] -Arguments: [sr_item_sk#15 ASC NULLS FIRST, sr_ticket_number#16 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#143, sr_ticket_number#144] +Arguments: [sr_item_sk#143 ASC NULLS FIRST, sr_ticket_number#144 ASC NULLS FIRST], false, 0 (139) SortMergeJoin [codegen id : 56] -Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] -Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] +Left keys [2]: [cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint)] +Right keys [2]: [sr_item_sk#143, sr_ticket_number#144] Join condition: None (140) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, sr_item_sk#143, sr_ticket_number#144] (141) ReusedExchange [Reuses operator id: 33] -Output [1]: [cs_item_sk#19] +Output [1]: [cs_item_sk#145] (142) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#19] +Left keys [1]: [ss_item_sk#129] +Right keys [1]: [cs_item_sk#145] Join condition: None (143) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, cs_item_sk#145] (144) Scan parquet default.date_dim -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#146, d_year#147] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (145) ColumnarToRow [codegen id : 54] -Input [2]: [d_date_sk#43, d_year#44] +Input [2]: [d_date_sk#146, d_year#147] (146) Filter [codegen id : 54] -Input [2]: [d_date_sk#43, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#43)) +Input [2]: [d_date_sk#146, d_year#147] +Condition : ((isnotnull(d_year#147) AND (d_year#147 = 2000)) AND isnotnull(d_date_sk#146)) (147) BroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#131] +Input [2]: [d_date_sk#146, d_year#147] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#148] (148) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#43] +Left keys [1]: [ss_sold_date_sk#140] +Right keys [1]: [d_date_sk#146] Join condition: None (149) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#43, d_year#44] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147] +Input [13]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, d_date_sk#146, d_year#147] (150) ReusedExchange [Reuses operator id: 45] -Output [3]: [s_store_sk#46, s_store_name#47, s_zip#48] +Output [3]: [s_store_sk#149, s_store_name#150, s_zip#151] (151) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#46] +Left keys [1]: [ss_store_sk#134] +Right keys [1]: [s_store_sk#149] Join condition: None (152) Project [codegen id : 56] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_sk#46, s_store_name#47, s_zip#48] +Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_sk#149, s_store_name#150, s_zip#151] (153) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#132] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Arguments: hashpartitioning(ss_customer_sk#130, 5), ENSURE_REQUIREMENTS, [id=#152] (154) Sort [codegen id : 57] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Arguments: [ss_customer_sk#130 ASC NULLS FIRST], false, 0 (155) ReusedExchange [Reuses operator id: 53] -Output [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Output [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] (156) Sort [codegen id : 59] -Input [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] -Arguments: [c_customer_sk#51 ASC NULLS FIRST], false, 0 +Input [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Arguments: [c_customer_sk#153 ASC NULLS FIRST], false, 0 (157) SortMergeJoin [codegen id : 62] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#51] +Left keys [1]: [ss_customer_sk#130] +Right keys [1]: [c_customer_sk#153] Join condition: None (158) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Input [18]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] (159) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#133, d_year#134] +Output [2]: [d_date_sk#159, d_year#160] (160) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_sales_date_sk#56] -Right keys [1]: [d_date_sk#133] +Left keys [1]: [c_first_sales_date_sk#158] +Right keys [1]: [d_date_sk#159] Join condition: None (161) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56, d_date_sk#133, d_year#134] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158, d_date_sk#159, d_year#160] (162) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#135, d_year#136] +Output [2]: [d_date_sk#161, d_year#162] (163) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_shipto_date_sk#55] -Right keys [1]: [d_date_sk#135] +Left keys [1]: [c_first_shipto_date_sk#157] +Right keys [1]: [d_date_sk#161] Join condition: None (164) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134, d_date_sk#135, d_year#136] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160, d_date_sk#161, d_year#162] (165) Exchange -Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Arguments: hashpartitioning(ss_cdemo_sk#3, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Arguments: hashpartitioning(ss_cdemo_sk#131, 5), ENSURE_REQUIREMENTS, [id=#163] (166) Sort [codegen id : 63] -Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Arguments: [ss_cdemo_sk#3 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Arguments: [ss_cdemo_sk#131 ASC NULLS FIRST], false, 0 (167) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#64, cd_marital_status#65] +Output [2]: [cd_demo_sk#164, cd_marital_status#165] (168) Sort [codegen id : 65] -Input [2]: [cd_demo_sk#64, cd_marital_status#65] -Arguments: [cd_demo_sk#64 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#164, cd_marital_status#165] +Arguments: [cd_demo_sk#164 ASC NULLS FIRST], false, 0 (169) SortMergeJoin [codegen id : 66] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#64] +Left keys [1]: [ss_cdemo_sk#131] +Right keys [1]: [cd_demo_sk#164] Join condition: None (170) Project [codegen id : 66] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_demo_sk#64, cd_marital_status#65] +Output [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_demo_sk#164, cd_marital_status#165] (171) Exchange -Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Arguments: hashpartitioning(c_current_cdemo_sk#52, 5), ENSURE_REQUIREMENTS, [id=#138] +Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Arguments: hashpartitioning(c_current_cdemo_sk#154, 5), ENSURE_REQUIREMENTS, [id=#166] (172) Sort [codegen id : 67] -Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Arguments: [c_current_cdemo_sk#52 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Arguments: [c_current_cdemo_sk#154 ASC NULLS FIRST], false, 0 (173) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#139, cd_marital_status#140] +Output [2]: [cd_demo_sk#167, cd_marital_status#168] (174) Sort [codegen id : 69] -Input [2]: [cd_demo_sk#139, cd_marital_status#140] -Arguments: [cd_demo_sk#139 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#167, cd_marital_status#168] +Arguments: [cd_demo_sk#167 ASC NULLS FIRST], false, 0 (175) SortMergeJoin [codegen id : 73] -Left keys [1]: [c_current_cdemo_sk#52] -Right keys [1]: [cd_demo_sk#139] -Join condition: NOT (cd_marital_status#65 = cd_marital_status#140) +Left keys [1]: [c_current_cdemo_sk#154] +Right keys [1]: [cd_demo_sk#167] +Join condition: NOT (cd_marital_status#165 = cd_marital_status#168) (176) Project [codegen id : 73] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65, cd_demo_sk#139, cd_marital_status#140] +Output [14]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [18]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165, cd_demo_sk#167, cd_marital_status#168] (177) ReusedExchange [Reuses operator id: 84] -Output [1]: [p_promo_sk#70] +Output [1]: [p_promo_sk#169] (178) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#70] +Left keys [1]: [ss_promo_sk#135] +Right keys [1]: [p_promo_sk#169] Join condition: None (179) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, p_promo_sk#70] +Output [13]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, p_promo_sk#169] (180) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#72, hd_income_band_sk#73] +Output [2]: [hd_demo_sk#170, hd_income_band_sk#171] (181) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#72] +Left keys [1]: [ss_hdemo_sk#132] +Right keys [1]: [hd_demo_sk#170] Join condition: None (182) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_demo_sk#72, hd_income_band_sk#73] +Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171] +Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_demo_sk#170, hd_income_band_sk#171] (183) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] +Output [2]: [hd_demo_sk#172, hd_income_band_sk#173] (184) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [c_current_hdemo_sk#53] -Right keys [1]: [hd_demo_sk#141] +Left keys [1]: [c_current_hdemo_sk#155] +Right keys [1]: [hd_demo_sk#172] Join condition: None (185) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_demo_sk#141, hd_income_band_sk#142] +Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Input [15]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_demo_sk#172, hd_income_band_sk#173] (186) Exchange -Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Arguments: hashpartitioning(ss_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#143] +Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Arguments: hashpartitioning(ss_addr_sk#133, 5), ENSURE_REQUIREMENTS, [id=#174] (187) Sort [codegen id : 74] -Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Arguments: [ss_addr_sk#5 ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Arguments: [ss_addr_sk#133 ASC NULLS FIRST], false, 0 (188) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Output [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] (189) Sort [codegen id : 76] -Input [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: [ca_address_sk#78 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: [ca_address_sk#175 ASC NULLS FIRST], false, 0 (190) SortMergeJoin [codegen id : 77] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#78] +Left keys [1]: [ss_addr_sk#133] +Right keys [1]: [ca_address_sk#175] Join condition: None (191) Project [codegen id : 77] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Output [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Input [18]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] (192) Exchange -Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: hashpartitioning(c_current_addr_sk#54, 5), ENSURE_REQUIREMENTS, [id=#144] +Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: hashpartitioning(c_current_addr_sk#156, 5), ENSURE_REQUIREMENTS, [id=#180] (193) Sort [codegen id : 78] -Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: [c_current_addr_sk#54 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: [c_current_addr_sk#156 ASC NULLS FIRST], false, 0 (194) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Output [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] (195) Sort [codegen id : 80] -Input [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Arguments: [ca_address_sk#145 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Arguments: [ca_address_sk#181 ASC NULLS FIRST], false, 0 (196) SortMergeJoin [codegen id : 84] -Left keys [1]: [c_current_addr_sk#54] -Right keys [1]: [ca_address_sk#145] +Left keys [1]: [c_current_addr_sk#156] +Right keys [1]: [ca_address_sk#181] Join condition: None (197) Project [codegen id : 84] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Output [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [21]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] (198) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#90] +Output [1]: [ib_income_band_sk#186] (199) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#73] -Right keys [1]: [ib_income_band_sk#90] +Left keys [1]: [hd_income_band_sk#171] +Right keys [1]: [ib_income_band_sk#186] Join condition: None (200) Project [codegen id : 84] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#90] +Output [18]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [20]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#186] (201) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#150] +Output [1]: [ib_income_band_sk#187] (202) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#142] -Right keys [1]: [ib_income_band_sk#150] +Left keys [1]: [hd_income_band_sk#173] +Right keys [1]: [ib_income_band_sk#187] Join condition: None (203) Project [codegen id : 84] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#150] +Output [17]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#187] (204) ReusedExchange [Reuses operator id: 124] -Output [2]: [i_item_sk#93, i_product_name#96] +Output [2]: [i_item_sk#188, i_product_name#189] (205) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#93] +Left keys [1]: [ss_item_sk#129] +Right keys [1]: [i_item_sk#188] Join condition: None (206) Project [codegen id : 84] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] +Output [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] +Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] (207) HashAggregate [codegen id : 84] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] -Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#151, sum#152, sum#153, sum#154] -Results [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] +Input [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] +Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#137)), partial_sum(UnscaledValue(ss_list_price#138)), partial_sum(UnscaledValue(ss_coupon_amt#139))] +Aggregate Attributes [4]: [count#190, sum#191, sum#192, sum#193] +Results [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] (208) Exchange -Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] -Arguments: hashpartitioning(i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, 5), ENSURE_REQUIREMENTS, [id=#159] +Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] +Arguments: hashpartitioning(i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, 5), ENSURE_REQUIREMENTS, [id=#198] (209) HashAggregate [codegen id : 85] -Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] -Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#160, sum(UnscaledValue(ss_wholesale_cost#9))#161, sum(UnscaledValue(ss_list_price#10))#162, sum(UnscaledValue(ss_coupon_amt#11))#163] -Results [8]: [i_item_sk#93 AS item_sk#164, s_store_name#47 AS store_name#165, s_zip#48 AS store_zip#166, d_year#44 AS syear#167, count(1)#160 AS cnt#168, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#161,17,2) AS s1#169, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#162,17,2) AS s2#170, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#163,17,2) AS s3#171] +Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] +Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#137)), sum(UnscaledValue(ss_list_price#138)), sum(UnscaledValue(ss_coupon_amt#139))] +Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#137))#200, sum(UnscaledValue(ss_list_price#138))#201, sum(UnscaledValue(ss_coupon_amt#139))#202] +Results [8]: [i_item_sk#188 AS item_sk#203, s_store_name#150 AS store_name#204, s_zip#151 AS store_zip#205, d_year#147 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#137))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#138))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#139))#202,17,2) AS s3#210] (210) Exchange -Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] -Arguments: hashpartitioning(item_sk#164, store_name#165, store_zip#166, 5), ENSURE_REQUIREMENTS, [id=#172] +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] (211) Sort [codegen id : 86] -Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] -Arguments: [item_sk#164 ASC NULLS FIRST, store_name#165 ASC NULLS FIRST, store_zip#166 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 (212) SortMergeJoin [codegen id : 87] Left keys [3]: [item_sk#112, store_name#113, store_zip#114] -Right keys [3]: [item_sk#164, store_name#165, store_zip#166] -Join condition: (cnt#168 <= cnt#124) +Right keys [3]: [item_sk#203, store_name#204, store_zip#205] +Join condition: (cnt#207 <= cnt#124) (213) Project [codegen id : 87] -Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] +Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] (214) Exchange -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#173] +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] (215) Sort [codegen id : 88] -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1152,11 +1152,11 @@ ReusedExchange (216) (216) ReusedExchange [Reuses operator id: 39] Output [2]: [d_date_sk#43, d_year#44] -Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#129 +Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#140 IN dynamicpruning#141 ReusedExchange (217) (217) ReusedExchange [Reuses operator id: 147] -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#146, d_year#147] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index 216e7f5530128..a866c8aa9b486 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt @@ -701,323 +701,323 @@ Input [17]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_str Arguments: [item_sk#105 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, store_zip#107 ASC NULLS FIRST], false, 0 (115) Scan parquet default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#122)] +PartitionFilters: [isnotnull(ss_sold_date_sk#133), dynamicpruningexpression(ss_sold_date_sk#133 IN dynamicpruning#134)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (116) ColumnarToRow [codegen id : 27] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] (117) Filter [codegen id : 27] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Condition : (((((((isnotnull(ss_item_sk#122) AND isnotnull(ss_ticket_number#129)) AND isnotnull(ss_store_sk#127)) AND isnotnull(ss_customer_sk#123)) AND isnotnull(ss_cdemo_sk#124)) AND isnotnull(ss_promo_sk#128)) AND isnotnull(ss_hdemo_sk#125)) AND isnotnull(ss_addr_sk#126)) (118) BroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#123] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#135] (119) Scan parquet default.store_returns -Output [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Output [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct (120) ColumnarToRow -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] (121) Filter -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] -Condition : (isnotnull(sr_item_sk#15) AND isnotnull(sr_ticket_number#16)) +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Condition : (isnotnull(sr_item_sk#136) AND isnotnull(sr_ticket_number#137)) (122) Project -Output [2]: [sr_item_sk#15, sr_ticket_number#16] -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Output [2]: [sr_item_sk#136, sr_ticket_number#137] +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] (123) BroadcastHashJoin [codegen id : 28] -Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] -Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] +Left keys [2]: [cast(ss_item_sk#122 as bigint), cast(ss_ticket_number#129 as bigint)] +Right keys [2]: [sr_item_sk#136, sr_ticket_number#137] Join condition: None (124) Project [codegen id : 28] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, sr_item_sk#136, sr_ticket_number#137] (125) Exchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: hashpartitioning(ss_item_sk#122, 5), ENSURE_REQUIREMENTS, [id=#139] (126) Sort [codegen id : 29] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: [ss_item_sk#122 ASC NULLS FIRST], false, 0 (127) ReusedExchange [Reuses operator id: 28] -Output [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] +Output [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] (128) HashAggregate [codegen id : 35] -Input [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] -Keys [1]: [cs_item_sk#19] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#21)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#21))#128, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129] -Results [3]: [cs_item_sk#19, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#21))#128,17,2) AS sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] +Input [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] +Keys [1]: [cs_item_sk#140] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#144)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#144))#148, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149] +Results [3]: [cs_item_sk#140, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#144))#148,17,2) AS sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] (129) Filter [codegen id : 35] -Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] -Condition : (isnotnull(sum(cs_ext_list_price#21)#130) AND (cast(sum(cs_ext_list_price#21)#130 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131)), DecimalType(21,2), true))) +Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] +Condition : (isnotnull(sum(cs_ext_list_price#144)#150) AND (cast(sum(cs_ext_list_price#144)#150 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151)), DecimalType(21,2), true))) (130) Project [codegen id : 35] -Output [1]: [cs_item_sk#19] -Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] +Output [1]: [cs_item_sk#140] +Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] (131) Sort [codegen id : 35] -Input [1]: [cs_item_sk#19] -Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 +Input [1]: [cs_item_sk#140] +Arguments: [cs_item_sk#140 ASC NULLS FIRST], false, 0 (132) SortMergeJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#19] +Left keys [1]: [ss_item_sk#122] +Right keys [1]: [cs_item_sk#140] Join condition: None (133) Project [codegen id : 51] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, cs_item_sk#140] (134) Scan parquet default.date_dim -Output [2]: [d_date_sk#42, d_year#43] +Output [2]: [d_date_sk#152, d_year#153] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (135) ColumnarToRow [codegen id : 36] -Input [2]: [d_date_sk#42, d_year#43] +Input [2]: [d_date_sk#152, d_year#153] (136) Filter [codegen id : 36] -Input [2]: [d_date_sk#42, d_year#43] -Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2000)) AND isnotnull(d_date_sk#42)) +Input [2]: [d_date_sk#152, d_year#153] +Condition : ((isnotnull(d_year#153) AND (d_year#153 = 2000)) AND isnotnull(d_date_sk#152)) (137) BroadcastExchange -Input [2]: [d_date_sk#42, d_year#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#132] +Input [2]: [d_date_sk#152, d_year#153] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#154] (138) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#42] +Left keys [1]: [ss_sold_date_sk#133] +Right keys [1]: [d_date_sk#152] Join condition: None (139) Project [codegen id : 51] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#42, d_year#43] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153] +Input [13]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, d_date_sk#152, d_year#153] (140) ReusedExchange [Reuses operator id: 44] -Output [3]: [s_store_sk#45, s_store_name#46, s_zip#47] +Output [3]: [s_store_sk#155, s_store_name#156, s_zip#157] (141) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#45] +Left keys [1]: [ss_store_sk#127] +Right keys [1]: [s_store_sk#155] Join condition: None (142) Project [codegen id : 51] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_sk#45, s_store_name#46, s_zip#47] +Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157] +Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_sk#155, s_store_name#156, s_zip#157] (143) ReusedExchange [Reuses operator id: 50] -Output [6]: [c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] +Output [6]: [c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] (144) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#49] +Left keys [1]: [ss_customer_sk#123] +Right keys [1]: [c_customer_sk#158] Join condition: None (145) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] +Input [18]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] (146) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#133, d_year#134] +Output [2]: [d_date_sk#164, d_year#165] (147) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_sales_date_sk#54] -Right keys [1]: [d_date_sk#133] +Left keys [1]: [c_first_sales_date_sk#163] +Right keys [1]: [d_date_sk#164] Join condition: None (148) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54, d_date_sk#133, d_year#134] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163, d_date_sk#164, d_year#165] (149) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#135, d_year#136] +Output [2]: [d_date_sk#166, d_year#167] (150) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_shipto_date_sk#53] -Right keys [1]: [d_date_sk#135] +Left keys [1]: [c_first_shipto_date_sk#162] +Right keys [1]: [d_date_sk#166] Join condition: None (151) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134, d_date_sk#135, d_year#136] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165, d_date_sk#166, d_year#167] (152) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#61, cd_marital_status#62] +Output [2]: [cd_demo_sk#168, cd_marital_status#169] (153) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#61] +Left keys [1]: [ss_cdemo_sk#124] +Right keys [1]: [cd_demo_sk#168] Join condition: None (154) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_demo_sk#61, cd_marital_status#62] +Output [16]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_demo_sk#168, cd_marital_status#169] (155) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#137, cd_marital_status#138] +Output [2]: [cd_demo_sk#170, cd_marital_status#171] (156) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_cdemo_sk#50] -Right keys [1]: [cd_demo_sk#137] -Join condition: NOT (cd_marital_status#62 = cd_marital_status#138) +Left keys [1]: [c_current_cdemo_sk#159] +Right keys [1]: [cd_demo_sk#170] +Join condition: NOT (cd_marital_status#169 = cd_marital_status#171) (157) Project [codegen id : 51] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62, cd_demo_sk#137, cd_marital_status#138] +Output [14]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [18]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169, cd_demo_sk#170, cd_marital_status#171] (158) ReusedExchange [Reuses operator id: 74] -Output [1]: [p_promo_sk#66] +Output [1]: [p_promo_sk#172] (159) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#66] +Left keys [1]: [ss_promo_sk#128] +Right keys [1]: [p_promo_sk#172] Join condition: None (160) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, p_promo_sk#66] +Output [13]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, p_promo_sk#172] (161) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#68, hd_income_band_sk#69] +Output [2]: [hd_demo_sk#173, hd_income_band_sk#174] (162) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#68] +Left keys [1]: [ss_hdemo_sk#125] +Right keys [1]: [hd_demo_sk#173] Join condition: None (163) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_demo_sk#68, hd_income_band_sk#69] +Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174] +Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_demo_sk#173, hd_income_band_sk#174] (164) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] +Output [2]: [hd_demo_sk#175, hd_income_band_sk#176] (165) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_hdemo_sk#51] -Right keys [1]: [hd_demo_sk#139] +Left keys [1]: [c_current_hdemo_sk#160] +Right keys [1]: [hd_demo_sk#175] Join condition: None (166) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_demo_sk#139, hd_income_band_sk#140] +Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176] +Input [15]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_demo_sk#175, hd_income_band_sk#176] (167) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] +Output [5]: [ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] (168) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#73] +Left keys [1]: [ss_addr_sk#126] +Right keys [1]: [ca_address_sk#177] Join condition: None (169) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] +Output [16]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] +Input [18]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] (170) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Output [5]: [ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] (171) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_addr_sk#52] -Right keys [1]: [ca_address_sk#141] +Left keys [1]: [c_current_addr_sk#161] +Right keys [1]: [ca_address_sk#182] Join condition: None (172) Project [codegen id : 51] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Output [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [21]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] (173) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#84] +Output [1]: [ib_income_band_sk#187] (174) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#69] -Right keys [1]: [ib_income_band_sk#84] +Left keys [1]: [hd_income_band_sk#174] +Right keys [1]: [ib_income_band_sk#187] Join condition: None (175) Project [codegen id : 51] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#84] +Output [18]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [20]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#187] (176) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#146] +Output [1]: [ib_income_band_sk#188] (177) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#140] -Right keys [1]: [ib_income_band_sk#146] +Left keys [1]: [hd_income_band_sk#176] +Right keys [1]: [ib_income_band_sk#188] Join condition: None (178) Project [codegen id : 51] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#146] +Output [17]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#188] (179) ReusedExchange [Reuses operator id: 108] -Output [2]: [i_item_sk#87, i_product_name#90] +Output [2]: [i_item_sk#189, i_product_name#190] (180) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#87] +Left keys [1]: [ss_item_sk#122] +Right keys [1]: [i_item_sk#189] Join condition: None (181) Project [codegen id : 51] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] +Output [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] (182) HashAggregate [codegen id : 51] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] -Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#147, sum#148, sum#149, sum#150] -Results [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] +Input [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#130)), partial_sum(UnscaledValue(ss_list_price#131)), partial_sum(UnscaledValue(ss_coupon_amt#132))] +Aggregate Attributes [4]: [count#191, sum#192, sum#193, sum#194] +Results [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] (183) HashAggregate [codegen id : 51] -Input [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#155, sum(UnscaledValue(ss_wholesale_cost#9))#156, sum(UnscaledValue(ss_list_price#10))#157, sum(UnscaledValue(ss_coupon_amt#11))#158] -Results [8]: [i_item_sk#87 AS item_sk#159, s_store_name#46 AS store_name#160, s_zip#47 AS store_zip#161, d_year#43 AS syear#162, count(1)#155 AS cnt#163, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#156,17,2) AS s1#164, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#157,17,2) AS s2#165, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#158,17,2) AS s3#166] +Input [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] +Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#130)), sum(UnscaledValue(ss_list_price#131)), sum(UnscaledValue(ss_coupon_amt#132))] +Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#130))#200, sum(UnscaledValue(ss_list_price#131))#201, sum(UnscaledValue(ss_coupon_amt#132))#202] +Results [8]: [i_item_sk#189 AS item_sk#203, s_store_name#156 AS store_name#204, s_zip#157 AS store_zip#205, d_year#153 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#130))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#131))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#132))#202,17,2) AS s3#210] (184) Exchange -Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] -Arguments: hashpartitioning(item_sk#159, store_name#160, store_zip#161, 5), ENSURE_REQUIREMENTS, [id=#167] +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] (185) Sort [codegen id : 52] -Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] -Arguments: [item_sk#159 ASC NULLS FIRST, store_name#160 ASC NULLS FIRST, store_zip#161 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 (186) SortMergeJoin [codegen id : 53] Left keys [3]: [item_sk#105, store_name#106, store_zip#107] -Right keys [3]: [item_sk#159, store_name#160, store_zip#161] -Join condition: (cnt#163 <= cnt#117) +Right keys [3]: [item_sk#203, store_name#204, store_zip#205] +Join condition: (cnt#207 <= cnt#117) (187) Project [codegen id : 53] -Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] +Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] (188) Exchange -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#168] +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] (189) Sort [codegen id : 54] -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1028,11 +1028,11 @@ ReusedExchange (190) (190) ReusedExchange [Reuses operator id: 38] Output [2]: [d_date_sk#42, d_year#43] -Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#122 +Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#133 IN dynamicpruning#134 ReusedExchange (191) (191) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#42, d_year#43] +Output [2]: [d_date_sk#152, d_year#153] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt index 41c2ffcc75e7a..84e49ab9373e4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt @@ -130,135 +130,135 @@ Input [4]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17 Condition : isnotnull(ss_store_sk#15) (18) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#18] (19) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ss_sold_date_sk#17] -Right keys [1]: [d_date_sk#6] +Right keys [1]: [d_date_sk#18] Join condition: None (20) Project [codegen id : 4] Output [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] -Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#6] +Input [5]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16, ss_sold_date_sk#17, d_date_sk#18] (21) HashAggregate [codegen id : 4] Input [3]: [ss_item_sk#14, ss_store_sk#15, ss_sales_price#16] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum#18] -Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] +Aggregate Attributes [1]: [sum#19] +Results [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] (22) Exchange -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] -Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] +Arguments: hashpartitioning(ss_store_sk#15, ss_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#21] (23) HashAggregate [codegen id : 5] -Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#19] +Input [3]: [ss_store_sk#15, ss_item_sk#14, sum#20] Keys [2]: [ss_store_sk#15, ss_item_sk#14] Functions [1]: [sum(UnscaledValue(ss_sales_price#16))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#21] -Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#21,17,2) AS revenue#22] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#16))#22] +Results [2]: [ss_store_sk#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#16))#22,17,2) AS revenue#23] (24) HashAggregate [codegen id : 5] -Input [2]: [ss_store_sk#15, revenue#22] +Input [2]: [ss_store_sk#15, revenue#23] Keys [1]: [ss_store_sk#15] -Functions [1]: [partial_avg(revenue#22)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ss_store_sk#15, sum#25, count#26] +Functions [1]: [partial_avg(revenue#23)] +Aggregate Attributes [2]: [sum#24, count#25] +Results [3]: [ss_store_sk#15, sum#26, count#27] (25) Exchange -Input [3]: [ss_store_sk#15, sum#25, count#26] -Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ss_store_sk#15, sum#26, count#27] +Arguments: hashpartitioning(ss_store_sk#15, 5), ENSURE_REQUIREMENTS, [id=#28] (26) HashAggregate [codegen id : 6] -Input [3]: [ss_store_sk#15, sum#25, count#26] +Input [3]: [ss_store_sk#15, sum#26, count#27] Keys [1]: [ss_store_sk#15] -Functions [1]: [avg(revenue#22)] -Aggregate Attributes [1]: [avg(revenue#22)#28] -Results [2]: [ss_store_sk#15, avg(revenue#22)#28 AS ave#29] +Functions [1]: [avg(revenue#23)] +Aggregate Attributes [1]: [avg(revenue#23)#29] +Results [2]: [ss_store_sk#15, avg(revenue#23)#29 AS ave#30] (27) BroadcastExchange -Input [2]: [ss_store_sk#15, ave#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#30] +Input [2]: [ss_store_sk#15, ave#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#31] (28) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] Right keys [1]: [ss_store_sk#15] -Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#29)), DecimalType(23,7), true)) +Join condition: (cast(revenue#13 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#30)), DecimalType(23,7), true)) (29) Project [codegen id : 8] Output [3]: [ss_store_sk#2, ss_item_sk#1, revenue#13] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#29] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, ss_store_sk#15, ave#30] (30) Scan parquet default.store -Output [2]: [s_store_sk#31, s_store_name#32] +Output [2]: [s_store_sk#32, s_store_name#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [2]: [s_store_sk#31, s_store_name#32] +Input [2]: [s_store_sk#32, s_store_name#33] (32) Filter [codegen id : 7] -Input [2]: [s_store_sk#31, s_store_name#32] -Condition : isnotnull(s_store_sk#31) +Input [2]: [s_store_sk#32, s_store_name#33] +Condition : isnotnull(s_store_sk#32) (33) BroadcastExchange -Input [2]: [s_store_sk#31, s_store_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] +Input [2]: [s_store_sk#32, s_store_name#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] (34) BroadcastHashJoin [codegen id : 8] Left keys [1]: [ss_store_sk#2] -Right keys [1]: [s_store_sk#31] +Right keys [1]: [s_store_sk#32] Join condition: None (35) Project [codegen id : 8] -Output [3]: [ss_item_sk#1, revenue#13, s_store_name#32] -Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#31, s_store_name#32] +Output [3]: [ss_item_sk#1, revenue#13, s_store_name#33] +Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#13, s_store_sk#32, s_store_name#33] (36) Exchange -Input [3]: [ss_item_sk#1, revenue#13, s_store_name#32] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [ss_item_sk#1, revenue#13, s_store_name#33] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#35] (37) Sort [codegen id : 9] -Input [3]: [ss_item_sk#1, revenue#13, s_store_name#32] +Input [3]: [ss_item_sk#1, revenue#13, s_store_name#33] Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 (38) Scan parquet default.item -Output [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Output [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 10] -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] (40) Filter [codegen id : 10] -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Condition : isnotnull(i_item_sk#35) +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Condition : isnotnull(i_item_sk#36) (41) Exchange -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Arguments: hashpartitioning(i_item_sk#35, 5), ENSURE_REQUIREMENTS, [id=#40] +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Arguments: hashpartitioning(i_item_sk#36, 5), ENSURE_REQUIREMENTS, [id=#41] (42) Sort [codegen id : 11] -Input [5]: [i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Arguments: [i_item_sk#35 ASC NULLS FIRST], false, 0 +Input [5]: [i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Arguments: [i_item_sk#36 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 12] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#35] +Right keys [1]: [i_item_sk#36] Join condition: None (44) Project [codegen id : 12] -Output [6]: [s_store_name#32, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Input [8]: [ss_item_sk#1, revenue#13, s_store_name#32, i_item_sk#35, i_item_desc#36, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Output [6]: [s_store_name#33, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Input [8]: [ss_item_sk#1, revenue#13, s_store_name#33, i_item_sk#36, i_item_desc#37, i_current_price#38, i_wholesale_cost#39, i_brand#40] (45) TakeOrderedAndProject -Input [6]: [s_store_name#32, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] -Arguments: 100, [s_store_name#32 ASC NULLS FIRST, i_item_desc#36 ASC NULLS FIRST], [s_store_name#32, i_item_desc#36, revenue#13, i_current_price#37, i_wholesale_cost#38, i_brand#39] +Input [6]: [s_store_name#33, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] +Arguments: 100, [s_store_name#33 ASC NULLS FIRST, i_item_desc#37 ASC NULLS FIRST], [s_store_name#33, i_item_desc#37, revenue#13, i_current_price#38, i_wholesale_cost#39, i_brand#40] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt index 52de9873db590..45c7c051601c5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt @@ -181,65 +181,65 @@ Input [4]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26 Condition : isnotnull(ss_store_sk#24) (30) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#27] (31) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_sold_date_sk#26] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#27] Join condition: None (32) Project [codegen id : 6] Output [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] -Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#8] +Input [5]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#27] (33) HashAggregate [codegen id : 6] Input [3]: [ss_item_sk#23, ss_store_sk#24, ss_sales_price#25] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum#27] -Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] +Aggregate Attributes [1]: [sum#28] +Results [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] (34) Exchange -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] -Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] +Arguments: hashpartitioning(ss_store_sk#24, ss_item_sk#23, 5), ENSURE_REQUIREMENTS, [id=#30] (35) HashAggregate [codegen id : 7] -Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#28] +Input [3]: [ss_store_sk#24, ss_item_sk#23, sum#29] Keys [2]: [ss_store_sk#24, ss_item_sk#23] Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#30] -Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#30,17,2) AS revenue#31] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] +Results [2]: [ss_store_sk#24, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS revenue#32] (36) HashAggregate [codegen id : 7] -Input [2]: [ss_store_sk#24, revenue#31] +Input [2]: [ss_store_sk#24, revenue#32] Keys [1]: [ss_store_sk#24] -Functions [1]: [partial_avg(revenue#31)] -Aggregate Attributes [2]: [sum#32, count#33] -Results [3]: [ss_store_sk#24, sum#34, count#35] +Functions [1]: [partial_avg(revenue#32)] +Aggregate Attributes [2]: [sum#33, count#34] +Results [3]: [ss_store_sk#24, sum#35, count#36] (37) Exchange -Input [3]: [ss_store_sk#24, sum#34, count#35] -Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ss_store_sk#24, sum#35, count#36] +Arguments: hashpartitioning(ss_store_sk#24, 5), ENSURE_REQUIREMENTS, [id=#37] (38) HashAggregate [codegen id : 8] -Input [3]: [ss_store_sk#24, sum#34, count#35] +Input [3]: [ss_store_sk#24, sum#35, count#36] Keys [1]: [ss_store_sk#24] -Functions [1]: [avg(revenue#31)] -Aggregate Attributes [1]: [avg(revenue#31)#37] -Results [2]: [ss_store_sk#24, avg(revenue#31)#37 AS ave#38] +Functions [1]: [avg(revenue#32)] +Aggregate Attributes [1]: [avg(revenue#32)#38] +Results [2]: [ss_store_sk#24, avg(revenue#32)#38 AS ave#39] (39) BroadcastExchange -Input [2]: [ss_store_sk#24, ave#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#39] +Input [2]: [ss_store_sk#24, ave#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#4] Right keys [1]: [ss_store_sk#24] -Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#38)), DecimalType(23,7), true)) +Join condition: (cast(revenue#15 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#39)), DecimalType(23,7), true)) (41) Project [codegen id : 9] Output [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#38] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#15, i_item_desc#18, i_current_price#19, i_wholesale_cost#20, i_brand#21, ss_store_sk#24, ave#39] (42) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#18, revenue#15, i_current_price#19, i_wholesale_cost#20, i_brand#21] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt index 51298b80bbbbe..8c342961cf970 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt @@ -221,70 +221,70 @@ Input [7]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_qu Condition : ((isnotnull(cs_warehouse_sk#176) AND isnotnull(cs_sold_time_sk#174)) AND isnotnull(cs_ship_mode_sk#175)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [sm_ship_mode_sk#9] +Output [1]: [sm_ship_mode_sk#181] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_mode_sk#175] -Right keys [1]: [sm_ship_mode_sk#9] +Right keys [1]: [sm_ship_mode_sk#181] Join condition: None (38) Project [codegen id : 11] Output [6]: [cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180] -Input [8]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, sm_ship_mode_sk#9] +Input [8]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, sm_ship_mode_sk#181] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [t_time_sk#12] +Output [1]: [t_time_sk#182] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_time_sk#174] -Right keys [1]: [t_time_sk#12] +Right keys [1]: [t_time_sk#182] Join condition: None (41) Project [codegen id : 11] Output [5]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180] -Input [7]: [cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, t_time_sk#12] +Input [7]: [cs_sold_time_sk#174, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, t_time_sk#182] (42) ReusedExchange [Reuses operator id: 21] -Output [3]: [d_date_sk#15, d_year#16, d_moy#17] +Output [3]: [d_date_sk#183, d_year#184, d_moy#185] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#180] -Right keys [1]: [d_date_sk#15] +Right keys [1]: [d_date_sk#183] Join condition: None (44) Project [codegen id : 11] -Output [6]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#16, d_moy#17] -Input [8]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, d_date_sk#15, d_year#16, d_moy#17] +Output [6]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#184, d_moy#185] +Input [8]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, d_date_sk#183, d_year#184, d_moy#185] (45) ReusedExchange [Reuses operator id: 27] -Output [7]: [w_warehouse_sk#19, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25] +Output [7]: [w_warehouse_sk#186, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192] (46) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_warehouse_sk#176] -Right keys [1]: [w_warehouse_sk#19] +Right keys [1]: [w_warehouse_sk#186] Join condition: None (47) Project [codegen id : 11] -Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, d_moy#17] -Input [13]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#16, d_moy#17, w_warehouse_sk#19, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25] +Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, d_moy#185] +Input [13]: [cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, d_year#184, d_moy#185, w_warehouse_sk#186, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192] (48) HashAggregate [codegen id : 11] -Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, d_moy#17] -Keys [7]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16] -Functions [24]: [partial_sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228] -Results [55]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] +Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, d_moy#185] +Keys [7]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184] +Functions [24]: [partial_sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [48]: [sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240] +Results [55]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] (49) Exchange -Input [55]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] -Arguments: hashpartitioning(w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#277] +Input [55]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] +Arguments: hashpartitioning(w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, 5), ENSURE_REQUIREMENTS, [id=#289] (50) HashAggregate [codegen id : 12] -Input [55]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] -Keys [7]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, d_year#16] -Functions [24]: [sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301] -Results [32]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, DHL,BARIAN AS ship_carriers#302, d_year#16 AS year#303, sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278 AS jan_sales#304, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279 AS feb_sales#305, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280 AS mar_sales#306, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281 AS apr_sales#307, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282 AS may_sales#308, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283 AS jun_sales#309, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284 AS jul_sales#310, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285 AS aug_sales#311, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286 AS sep_sales#312, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287 AS oct_sales#313, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288 AS nov_sales#314, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289 AS dec_sales#315, sum(CASE WHEN (d_moy#17 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_net#316, sum(CASE WHEN (d_moy#17 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_net#317, sum(CASE WHEN (d_moy#17 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_net#318, sum(CASE WHEN (d_moy#17 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_net#319, sum(CASE WHEN (d_moy#17 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_net#320, sum(CASE WHEN (d_moy#17 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_net#321, sum(CASE WHEN (d_moy#17 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_net#322, sum(CASE WHEN (d_moy#17 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_net#323, sum(CASE WHEN (d_moy#17 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_net#324, sum(CASE WHEN (d_moy#17 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_net#325, sum(CASE WHEN (d_moy#17 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_net#326, sum(CASE WHEN (d_moy#17 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_net#327] +Input [55]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] +Keys [7]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, d_year#184] +Functions [24]: [sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313] +Results [32]: [w_warehouse_name#187, w_warehouse_sq_ft#188, w_city#189, w_county#190, w_state#191, w_country#192, DHL,BARIAN AS ship_carriers#314, d_year#184 AS year#315, sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_sales#316, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_sales#317, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_sales#318, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_sales#319, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_sales#320, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_sales#321, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_sales#322, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_sales#323, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_sales#324, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_sales#325, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_sales#326, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_sales#327, sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302 AS jan_net#328, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303 AS feb_net#329, sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304 AS mar_net#330, sum(CASE WHEN (d_moy#185 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305 AS apr_net#331, sum(CASE WHEN (d_moy#185 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306 AS may_net#332, sum(CASE WHEN (d_moy#185 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307 AS jun_net#333, sum(CASE WHEN (d_moy#185 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308 AS jul_net#334, sum(CASE WHEN (d_moy#185 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309 AS aug_net#335, sum(CASE WHEN (d_moy#185 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310 AS sep_net#336, sum(CASE WHEN (d_moy#185 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311 AS oct_net#337, sum(CASE WHEN (d_moy#185 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312 AS nov_net#338, sum(CASE WHEN (d_moy#185 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313 AS dec_net#339] (51) Union @@ -292,23 +292,23 @@ Results [32]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23 Input [32]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#150, feb_sales#151, mar_sales#152, apr_sales#153, may_sales#154, jun_sales#155, jul_sales#156, aug_sales#157, sep_sales#158, oct_sales#159, nov_sales#160, dec_sales#161, jan_net#162, feb_net#163, mar_net#164, apr_net#165, may_net#166, jun_net#167, jul_net#168, aug_net#169, sep_net#170, oct_net#171, nov_net#172, dec_net#173] Keys [8]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149] Functions [36]: [partial_sum(jan_sales#150), partial_sum(feb_sales#151), partial_sum(mar_sales#152), partial_sum(apr_sales#153), partial_sum(may_sales#154), partial_sum(jun_sales#155), partial_sum(jul_sales#156), partial_sum(aug_sales#157), partial_sum(sep_sales#158), partial_sum(oct_sales#159), partial_sum(nov_sales#160), partial_sum(dec_sales#161), partial_sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(jan_net#162), partial_sum(feb_net#163), partial_sum(mar_net#164), partial_sum(apr_net#165), partial_sum(may_net#166), partial_sum(jun_net#167), partial_sum(jul_net#168), partial_sum(aug_net#169), partial_sum(sep_net#170), partial_sum(oct_net#171), partial_sum(nov_net#172), partial_sum(dec_net#173)] -Aggregate Attributes [72]: [sum#328, isEmpty#329, sum#330, isEmpty#331, sum#332, isEmpty#333, sum#334, isEmpty#335, sum#336, isEmpty#337, sum#338, isEmpty#339, sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399] -Results [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] +Aggregate Attributes [72]: [sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411] +Results [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] (53) Exchange -Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] -Arguments: hashpartitioning(w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#472] +Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] +Arguments: hashpartitioning(w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#484] (54) HashAggregate [codegen id : 14] -Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] +Input [80]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] Keys [8]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149] Functions [36]: [sum(jan_sales#150), sum(feb_sales#151), sum(mar_sales#152), sum(apr_sales#153), sum(may_sales#154), sum(jun_sales#155), sum(jul_sales#156), sum(aug_sales#157), sum(sep_sales#158), sum(oct_sales#159), sum(nov_sales#160), sum(dec_sales#161), sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(jan_net#162), sum(feb_net#163), sum(mar_net#164), sum(apr_net#165), sum(may_net#166), sum(jun_net#167), sum(jul_net#168), sum(aug_net#169), sum(sep_net#170), sum(oct_net#171), sum(nov_net#172), sum(dec_net#173)] -Aggregate Attributes [36]: [sum(jan_sales#150)#473, sum(feb_sales#151)#474, sum(mar_sales#152)#475, sum(apr_sales#153)#476, sum(may_sales#154)#477, sum(jun_sales#155)#478, sum(jul_sales#156)#479, sum(aug_sales#157)#480, sum(sep_sales#158)#481, sum(oct_sales#159)#482, sum(nov_sales#160)#483, sum(dec_sales#161)#484, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496, sum(jan_net#162)#497, sum(feb_net#163)#498, sum(mar_net#164)#499, sum(apr_net#165)#500, sum(may_net#166)#501, sum(jun_net#167)#502, sum(jul_net#168)#503, sum(aug_net#169)#504, sum(sep_net#170)#505, sum(oct_net#171)#506, sum(nov_net#172)#507, sum(dec_net#173)#508] -Results [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum(jan_sales#150)#473 AS jan_sales#509, sum(feb_sales#151)#474 AS feb_sales#510, sum(mar_sales#152)#475 AS mar_sales#511, sum(apr_sales#153)#476 AS apr_sales#512, sum(may_sales#154)#477 AS may_sales#513, sum(jun_sales#155)#478 AS jun_sales#514, sum(jul_sales#156)#479 AS jul_sales#515, sum(aug_sales#157)#480 AS aug_sales#516, sum(sep_sales#158)#481 AS sep_sales#517, sum(oct_sales#159)#482 AS oct_sales#518, sum(nov_sales#160)#483 AS nov_sales#519, sum(dec_sales#161)#484 AS dec_sales#520, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485 AS jan_sales_per_sq_foot#521, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486 AS feb_sales_per_sq_foot#522, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487 AS mar_sales_per_sq_foot#523, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488 AS apr_sales_per_sq_foot#524, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489 AS may_sales_per_sq_foot#525, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490 AS jun_sales_per_sq_foot#526, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491 AS jul_sales_per_sq_foot#527, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492 AS aug_sales_per_sq_foot#528, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493 AS sep_sales_per_sq_foot#529, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494 AS oct_sales_per_sq_foot#530, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495 AS nov_sales_per_sq_foot#531, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496 AS dec_sales_per_sq_foot#532, sum(jan_net#162)#497 AS jan_net#533, sum(feb_net#163)#498 AS feb_net#534, sum(mar_net#164)#499 AS mar_net#535, sum(apr_net#165)#500 AS apr_net#536, sum(may_net#166)#501 AS may_net#537, sum(jun_net#167)#502 AS jun_net#538, sum(jul_net#168)#503 AS jul_net#539, sum(aug_net#169)#504 AS aug_net#540, sum(sep_net#170)#505 AS sep_net#541, sum(oct_net#171)#506 AS oct_net#542, sum(nov_net#172)#507 AS nov_net#543, sum(dec_net#173)#508 AS dec_net#544] +Aggregate Attributes [36]: [sum(jan_sales#150)#485, sum(feb_sales#151)#486, sum(mar_sales#152)#487, sum(apr_sales#153)#488, sum(may_sales#154)#489, sum(jun_sales#155)#490, sum(jul_sales#156)#491, sum(aug_sales#157)#492, sum(sep_sales#158)#493, sum(oct_sales#159)#494, sum(nov_sales#160)#495, sum(dec_sales#161)#496, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508, sum(jan_net#162)#509, sum(feb_net#163)#510, sum(mar_net#164)#511, sum(apr_net#165)#512, sum(may_net#166)#513, sum(jun_net#167)#514, sum(jul_net#168)#515, sum(aug_net#169)#516, sum(sep_net#170)#517, sum(oct_net#171)#518, sum(nov_net#172)#519, sum(dec_net#173)#520] +Results [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, sum(jan_sales#150)#485 AS jan_sales#521, sum(feb_sales#151)#486 AS feb_sales#522, sum(mar_sales#152)#487 AS mar_sales#523, sum(apr_sales#153)#488 AS apr_sales#524, sum(may_sales#154)#489 AS may_sales#525, sum(jun_sales#155)#490 AS jun_sales#526, sum(jul_sales#156)#491 AS jul_sales#527, sum(aug_sales#157)#492 AS aug_sales#528, sum(sep_sales#158)#493 AS sep_sales#529, sum(oct_sales#159)#494 AS oct_sales#530, sum(nov_sales#160)#495 AS nov_sales#531, sum(dec_sales#161)#496 AS dec_sales#532, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497 AS jan_sales_per_sq_foot#533, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498 AS feb_sales_per_sq_foot#534, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499 AS mar_sales_per_sq_foot#535, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500 AS apr_sales_per_sq_foot#536, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501 AS may_sales_per_sq_foot#537, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502 AS jun_sales_per_sq_foot#538, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503 AS jul_sales_per_sq_foot#539, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504 AS aug_sales_per_sq_foot#540, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505 AS sep_sales_per_sq_foot#541, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506 AS oct_sales_per_sq_foot#542, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507 AS nov_sales_per_sq_foot#543, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#21 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508 AS dec_sales_per_sq_foot#544, sum(jan_net#162)#509 AS jan_net#545, sum(feb_net#163)#510 AS feb_net#546, sum(mar_net#164)#511 AS mar_net#547, sum(apr_net#165)#512 AS apr_net#548, sum(may_net#166)#513 AS may_net#549, sum(jun_net#167)#514 AS jun_net#550, sum(jul_net#168)#515 AS jul_net#551, sum(aug_net#169)#516 AS aug_net#552, sum(sep_net#170)#517 AS sep_net#553, sum(oct_net#171)#518 AS oct_net#554, sum(nov_net#172)#519 AS nov_net#555, sum(dec_net#173)#520 AS dec_net#556] (55) TakeOrderedAndProject -Input [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, may_sales_per_sq_foot#525, jun_sales_per_sq_foot#526, jul_sales_per_sq_foot#527, aug_sales_per_sq_foot#528, sep_sales_per_sq_foot#529, oct_sales_per_sq_foot#530, nov_sales_per_sq_foot#531, dec_sales_per_sq_foot#532, jan_net#533, feb_net#534, mar_net#535, apr_net#536, may_net#537, jun_net#538, jul_net#539, aug_net#540, sep_net#541, oct_net#542, nov_net#543, dec_net#544] -Arguments: 100, [w_warehouse_name#20 ASC NULLS FIRST], [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, ... 20 more fields] +Input [44]: [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, may_sales_per_sq_foot#537, jun_sales_per_sq_foot#538, jul_sales_per_sq_foot#539, aug_sales_per_sq_foot#540, sep_sales_per_sq_foot#541, oct_sales_per_sq_foot#542, nov_sales_per_sq_foot#543, dec_sales_per_sq_foot#544, jan_net#545, feb_net#546, mar_net#547, apr_net#548, may_net#549, jun_net#550, jul_net#551, aug_net#552, sep_net#553, oct_net#554, nov_net#555, dec_net#556] +Arguments: 100, [w_warehouse_name#20 ASC NULLS FIRST], [w_warehouse_name#20, w_warehouse_sq_ft#21, w_city#22, w_county#23, w_state#24, w_country#25, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, ... 20 more fields] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt index 3d44b22396486..832965c1aaa31 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt @@ -221,70 +221,70 @@ Input [7]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_qu Condition : ((isnotnull(cs_warehouse_sk#176) AND isnotnull(cs_sold_time_sk#174)) AND isnotnull(cs_ship_mode_sk#175)) (36) ReusedExchange [Reuses operator id: 7] -Output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Output [7]: [w_warehouse_sk#181, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187] (37) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_warehouse_sk#176] -Right keys [1]: [w_warehouse_sk#9] +Right keys [1]: [w_warehouse_sk#181] Join condition: None (38) Project [codegen id : 11] -Output [12]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] -Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Output [12]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187] +Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_warehouse_sk#176, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_sk#181, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187] (39) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] +Output [3]: [d_date_sk#188, d_year#189, d_moy#190] (40) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_date_sk#180] -Right keys [1]: [d_date_sk#17] +Right keys [1]: [d_date_sk#188] Join condition: None (41) Project [codegen id : 11] -Output [13]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] -Input [15]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_date_sk#17, d_year#18, d_moy#19] +Output [13]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] +Input [15]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, cs_sold_date_sk#180, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_date_sk#188, d_year#189, d_moy#190] (42) ReusedExchange [Reuses operator id: 20] -Output [1]: [t_time_sk#21] +Output [1]: [t_time_sk#191] (43) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_sold_time_sk#174] -Right keys [1]: [t_time_sk#21] +Right keys [1]: [t_time_sk#191] Join condition: None (44) Project [codegen id : 11] -Output [12]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] -Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19, t_time_sk#21] +Output [12]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] +Input [14]: [cs_sold_time_sk#174, cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190, t_time_sk#191] (45) ReusedExchange [Reuses operator id: 27] -Output [1]: [sm_ship_mode_sk#24] +Output [1]: [sm_ship_mode_sk#192] (46) BroadcastHashJoin [codegen id : 11] Left keys [1]: [cs_ship_mode_sk#175] -Right keys [1]: [sm_ship_mode_sk#24] +Right keys [1]: [sm_ship_mode_sk#192] Join condition: None (47) Project [codegen id : 11] -Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] -Input [13]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19, sm_ship_mode_sk#24] +Output [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] +Input [13]: [cs_ship_mode_sk#175, cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190, sm_ship_mode_sk#192] (48) HashAggregate [codegen id : 11] -Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, d_moy#19] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18] -Functions [24]: [partial_sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [48]: [sum#181, isEmpty#182, sum#183, isEmpty#184, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228] -Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] +Input [11]: [cs_quantity#177, cs_sales_price#178, cs_net_paid_inc_tax#179, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, d_moy#190] +Keys [7]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189] +Functions [24]: [partial_sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [48]: [sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240] +Results [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] (49) Exchange -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, 5), ENSURE_REQUIREMENTS, [id=#277] +Input [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] +Arguments: hashpartitioning(w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, 5), ENSURE_REQUIREMENTS, [id=#289] (50) HashAggregate [codegen id : 12] -Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276] -Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#18] -Functions [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] -Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301] -Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, DHL,BARIAN AS ship_carriers#302, d_year#18 AS year#303, sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#278 AS jan_sales#304, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#279 AS feb_sales#305, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#280 AS mar_sales#306, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#281 AS apr_sales#307, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#282 AS may_sales#308, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#283 AS jun_sales#309, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#284 AS jul_sales#310, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#285 AS aug_sales#311, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#286 AS sep_sales#312, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#287 AS oct_sales#313, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#288 AS nov_sales#314, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#289 AS dec_sales#315, sum(CASE WHEN (d_moy#19 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_net#316, sum(CASE WHEN (d_moy#19 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_net#317, sum(CASE WHEN (d_moy#19 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_net#318, sum(CASE WHEN (d_moy#19 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_net#319, sum(CASE WHEN (d_moy#19 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_net#320, sum(CASE WHEN (d_moy#19 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_net#321, sum(CASE WHEN (d_moy#19 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_net#322, sum(CASE WHEN (d_moy#19 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_net#323, sum(CASE WHEN (d_moy#19 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_net#324, sum(CASE WHEN (d_moy#19 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_net#325, sum(CASE WHEN (d_moy#19 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_net#326, sum(CASE WHEN (d_moy#19 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_net#327] +Input [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284, sum#285, isEmpty#286, sum#287, isEmpty#288] +Keys [7]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#189] +Functions [24]: [sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END), sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313] +Results [32]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, DHL,BARIAN AS ship_carriers#314, d_year#189 AS year#315, sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#290 AS jan_sales#316, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#291 AS feb_sales#317, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#292 AS mar_sales#318, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#293 AS apr_sales#319, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#294 AS may_sales#320, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#295 AS jun_sales#321, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#296 AS jul_sales#322, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#297 AS aug_sales#323, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#298 AS sep_sales#324, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#299 AS oct_sales#325, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#300 AS nov_sales#326, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_sales_price#178 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#301 AS dec_sales#327, sum(CASE WHEN (d_moy#190 = 1) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#302 AS jan_net#328, sum(CASE WHEN (d_moy#190 = 2) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#303 AS feb_net#329, sum(CASE WHEN (d_moy#190 = 3) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#304 AS mar_net#330, sum(CASE WHEN (d_moy#190 = 4) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#305 AS apr_net#331, sum(CASE WHEN (d_moy#190 = 5) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#306 AS may_net#332, sum(CASE WHEN (d_moy#190 = 6) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#307 AS jun_net#333, sum(CASE WHEN (d_moy#190 = 7) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#308 AS jul_net#334, sum(CASE WHEN (d_moy#190 = 8) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#309 AS aug_net#335, sum(CASE WHEN (d_moy#190 = 9) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#310 AS sep_net#336, sum(CASE WHEN (d_moy#190 = 10) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#311 AS oct_net#337, sum(CASE WHEN (d_moy#190 = 11) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#312 AS nov_net#338, sum(CASE WHEN (d_moy#190 = 12) THEN CheckOverflow((promote_precision(cast(cs_net_paid_inc_tax#179 as decimal(12,2))) * promote_precision(cast(cast(cs_quantity#177 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true) ELSE 0.00 END)#313 AS dec_net#339] (51) Union @@ -292,23 +292,23 @@ Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13 Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#150, feb_sales#151, mar_sales#152, apr_sales#153, may_sales#154, jun_sales#155, jul_sales#156, aug_sales#157, sep_sales#158, oct_sales#159, nov_sales#160, dec_sales#161, jan_net#162, feb_net#163, mar_net#164, apr_net#165, may_net#166, jun_net#167, jul_net#168, aug_net#169, sep_net#170, oct_net#171, nov_net#172, dec_net#173] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149] Functions [36]: [partial_sum(jan_sales#150), partial_sum(feb_sales#151), partial_sum(mar_sales#152), partial_sum(apr_sales#153), partial_sum(may_sales#154), partial_sum(jun_sales#155), partial_sum(jul_sales#156), partial_sum(aug_sales#157), partial_sum(sep_sales#158), partial_sum(oct_sales#159), partial_sum(nov_sales#160), partial_sum(dec_sales#161), partial_sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), partial_sum(jan_net#162), partial_sum(feb_net#163), partial_sum(mar_net#164), partial_sum(apr_net#165), partial_sum(may_net#166), partial_sum(jun_net#167), partial_sum(jul_net#168), partial_sum(aug_net#169), partial_sum(sep_net#170), partial_sum(oct_net#171), partial_sum(nov_net#172), partial_sum(dec_net#173)] -Aggregate Attributes [72]: [sum#328, isEmpty#329, sum#330, isEmpty#331, sum#332, isEmpty#333, sum#334, isEmpty#335, sum#336, isEmpty#337, sum#338, isEmpty#339, sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399] -Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] +Aggregate Attributes [72]: [sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, sum#376, isEmpty#377, sum#378, isEmpty#379, sum#380, isEmpty#381, sum#382, isEmpty#383, sum#384, isEmpty#385, sum#386, isEmpty#387, sum#388, isEmpty#389, sum#390, isEmpty#391, sum#392, isEmpty#393, sum#394, isEmpty#395, sum#396, isEmpty#397, sum#398, isEmpty#399, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411] +Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] (53) Exchange -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] -Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#472] +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, 5), ENSURE_REQUIREMENTS, [id=#484] (54) HashAggregate [codegen id : 14] -Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#400, isEmpty#401, sum#402, isEmpty#403, sum#404, isEmpty#405, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471] +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#438, isEmpty#439, sum#440, isEmpty#441, sum#442, isEmpty#443, sum#444, isEmpty#445, sum#446, isEmpty#447, sum#448, isEmpty#449, sum#450, isEmpty#451, sum#452, isEmpty#453, sum#454, isEmpty#455, sum#456, isEmpty#457, sum#458, isEmpty#459, sum#460, isEmpty#461, sum#462, isEmpty#463, sum#464, isEmpty#465, sum#466, isEmpty#467, sum#468, isEmpty#469, sum#470, isEmpty#471, sum#472, isEmpty#473, sum#474, isEmpty#475, sum#476, isEmpty#477, sum#478, isEmpty#479, sum#480, isEmpty#481, sum#482, isEmpty#483] Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149] Functions [36]: [sum(jan_sales#150), sum(feb_sales#151), sum(mar_sales#152), sum(apr_sales#153), sum(may_sales#154), sum(jun_sales#155), sum(jul_sales#156), sum(aug_sales#157), sum(sep_sales#158), sum(oct_sales#159), sum(nov_sales#160), sum(dec_sales#161), sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true)), sum(jan_net#162), sum(feb_net#163), sum(mar_net#164), sum(apr_net#165), sum(may_net#166), sum(jun_net#167), sum(jul_net#168), sum(aug_net#169), sum(sep_net#170), sum(oct_net#171), sum(nov_net#172), sum(dec_net#173)] -Aggregate Attributes [36]: [sum(jan_sales#150)#473, sum(feb_sales#151)#474, sum(mar_sales#152)#475, sum(apr_sales#153)#476, sum(may_sales#154)#477, sum(jun_sales#155)#478, sum(jul_sales#156)#479, sum(aug_sales#157)#480, sum(sep_sales#158)#481, sum(oct_sales#159)#482, sum(nov_sales#160)#483, sum(dec_sales#161)#484, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496, sum(jan_net#162)#497, sum(feb_net#163)#498, sum(mar_net#164)#499, sum(apr_net#165)#500, sum(may_net#166)#501, sum(jun_net#167)#502, sum(jul_net#168)#503, sum(aug_net#169)#504, sum(sep_net#170)#505, sum(oct_net#171)#506, sum(nov_net#172)#507, sum(dec_net#173)#508] -Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum(jan_sales#150)#473 AS jan_sales#509, sum(feb_sales#151)#474 AS feb_sales#510, sum(mar_sales#152)#475 AS mar_sales#511, sum(apr_sales#153)#476 AS apr_sales#512, sum(may_sales#154)#477 AS may_sales#513, sum(jun_sales#155)#478 AS jun_sales#514, sum(jul_sales#156)#479 AS jul_sales#515, sum(aug_sales#157)#480 AS aug_sales#516, sum(sep_sales#158)#481 AS sep_sales#517, sum(oct_sales#159)#482 AS oct_sales#518, sum(nov_sales#160)#483 AS nov_sales#519, sum(dec_sales#161)#484 AS dec_sales#520, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#485 AS jan_sales_per_sq_foot#521, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#486 AS feb_sales_per_sq_foot#522, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#487 AS mar_sales_per_sq_foot#523, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#488 AS apr_sales_per_sq_foot#524, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#489 AS may_sales_per_sq_foot#525, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#490 AS jun_sales_per_sq_foot#526, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#491 AS jul_sales_per_sq_foot#527, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#492 AS aug_sales_per_sq_foot#528, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#493 AS sep_sales_per_sq_foot#529, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#494 AS oct_sales_per_sq_foot#530, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#495 AS nov_sales_per_sq_foot#531, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#496 AS dec_sales_per_sq_foot#532, sum(jan_net#162)#497 AS jan_net#533, sum(feb_net#163)#498 AS feb_net#534, sum(mar_net#164)#499 AS mar_net#535, sum(apr_net#165)#500 AS apr_net#536, sum(may_net#166)#501 AS may_net#537, sum(jun_net#167)#502 AS jun_net#538, sum(jul_net#168)#503 AS jul_net#539, sum(aug_net#169)#504 AS aug_net#540, sum(sep_net#170)#505 AS sep_net#541, sum(oct_net#171)#506 AS oct_net#542, sum(nov_net#172)#507 AS nov_net#543, sum(dec_net#173)#508 AS dec_net#544] +Aggregate Attributes [36]: [sum(jan_sales#150)#485, sum(feb_sales#151)#486, sum(mar_sales#152)#487, sum(apr_sales#153)#488, sum(may_sales#154)#489, sum(jun_sales#155)#490, sum(jul_sales#156)#491, sum(aug_sales#157)#492, sum(sep_sales#158)#493, sum(oct_sales#159)#494, sum(nov_sales#160)#495, sum(dec_sales#161)#496, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508, sum(jan_net#162)#509, sum(feb_net#163)#510, sum(mar_net#164)#511, sum(apr_net#165)#512, sum(may_net#166)#513, sum(jun_net#167)#514, sum(jul_net#168)#515, sum(aug_net#169)#516, sum(sep_net#170)#517, sum(oct_net#171)#518, sum(nov_net#172)#519, sum(dec_net#173)#520] +Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, sum(jan_sales#150)#485 AS jan_sales#521, sum(feb_sales#151)#486 AS feb_sales#522, sum(mar_sales#152)#487 AS mar_sales#523, sum(apr_sales#153)#488 AS apr_sales#524, sum(may_sales#154)#489 AS may_sales#525, sum(jun_sales#155)#490 AS jun_sales#526, sum(jul_sales#156)#491 AS jul_sales#527, sum(aug_sales#157)#492 AS aug_sales#528, sum(sep_sales#158)#493 AS sep_sales#529, sum(oct_sales#159)#494 AS oct_sales#530, sum(nov_sales#160)#495 AS nov_sales#531, sum(dec_sales#161)#496 AS dec_sales#532, sum(CheckOverflow((promote_precision(jan_sales#150) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#497 AS jan_sales_per_sq_foot#533, sum(CheckOverflow((promote_precision(feb_sales#151) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#498 AS feb_sales_per_sq_foot#534, sum(CheckOverflow((promote_precision(mar_sales#152) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#499 AS mar_sales_per_sq_foot#535, sum(CheckOverflow((promote_precision(apr_sales#153) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#500 AS apr_sales_per_sq_foot#536, sum(CheckOverflow((promote_precision(may_sales#154) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#501 AS may_sales_per_sq_foot#537, sum(CheckOverflow((promote_precision(jun_sales#155) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#502 AS jun_sales_per_sq_foot#538, sum(CheckOverflow((promote_precision(jul_sales#156) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#503 AS jul_sales_per_sq_foot#539, sum(CheckOverflow((promote_precision(aug_sales#157) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#504 AS aug_sales_per_sq_foot#540, sum(CheckOverflow((promote_precision(sep_sales#158) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#505 AS sep_sales_per_sq_foot#541, sum(CheckOverflow((promote_precision(oct_sales#159) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#506 AS oct_sales_per_sq_foot#542, sum(CheckOverflow((promote_precision(nov_sales#160) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#507 AS nov_sales_per_sq_foot#543, sum(CheckOverflow((promote_precision(dec_sales#161) / promote_precision(cast(cast(w_warehouse_sq_ft#11 as decimal(10,0)) as decimal(28,2)))), DecimalType(38,12), true))#508 AS dec_sales_per_sq_foot#544, sum(jan_net#162)#509 AS jan_net#545, sum(feb_net#163)#510 AS feb_net#546, sum(mar_net#164)#511 AS mar_net#547, sum(apr_net#165)#512 AS apr_net#548, sum(may_net#166)#513 AS may_net#549, sum(jun_net#167)#514 AS jun_net#550, sum(jul_net#168)#515 AS jul_net#551, sum(aug_net#169)#516 AS aug_net#552, sum(sep_net#170)#517 AS sep_net#553, sum(oct_net#171)#518 AS oct_net#554, sum(nov_net#172)#519 AS nov_net#555, sum(dec_net#173)#520 AS dec_net#556] (55) TakeOrderedAndProject -Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, may_sales_per_sq_foot#525, jun_sales_per_sq_foot#526, jul_sales_per_sq_foot#527, aug_sales_per_sq_foot#528, sep_sales_per_sq_foot#529, oct_sales_per_sq_foot#530, nov_sales_per_sq_foot#531, dec_sales_per_sq_foot#532, jan_net#533, feb_net#534, mar_net#535, apr_net#536, may_net#537, jun_net#538, jul_net#539, aug_net#540, sep_net#541, oct_net#542, nov_net#543, dec_net#544] -Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#509, feb_sales#510, mar_sales#511, apr_sales#512, may_sales#513, jun_sales#514, jul_sales#515, aug_sales#516, sep_sales#517, oct_sales#518, nov_sales#519, dec_sales#520, jan_sales_per_sq_foot#521, feb_sales_per_sq_foot#522, mar_sales_per_sq_foot#523, apr_sales_per_sq_foot#524, ... 20 more fields] +Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, may_sales_per_sq_foot#537, jun_sales_per_sq_foot#538, jul_sales_per_sq_foot#539, aug_sales_per_sq_foot#540, sep_sales_per_sq_foot#541, oct_sales_per_sq_foot#542, nov_sales_per_sq_foot#543, dec_sales_per_sq_foot#544, jan_net#545, feb_net#546, mar_net#547, apr_net#548, may_net#549, jun_net#550, jul_net#551, aug_net#552, sep_net#553, oct_net#554, nov_net#555, dec_net#556] +Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#148, year#149, jan_sales#521, feb_sales#522, mar_sales#523, apr_sales#524, may_sales#525, jun_sales#526, jul_sales#527, aug_sales#528, sep_sales#529, oct_sales#530, nov_sales#531, dec_sales#532, jan_sales_per_sq_foot#533, feb_sales_per_sq_foot#534, mar_sales_per_sq_foot#535, apr_sales_per_sq_foot#536, ... 20 more fields] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt index 744fe4b5a594d..b6a609ec193b4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt @@ -230,55 +230,55 @@ Input [6]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_ Arguments: [ss_addr_sk#12 ASC NULLS FIRST], false, 0 (41) ReusedExchange [Reuses operator id: 9] -Output [2]: [ca_address_sk#6, ca_city#7] +Output [2]: [ca_address_sk#32, ca_city#33] (42) Sort [codegen id : 13] -Input [2]: [ca_address_sk#6, ca_city#7] -Arguments: [ca_address_sk#6 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#32, ca_city#33] +Arguments: [ca_address_sk#32 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 14] Left keys [1]: [ss_addr_sk#12] -Right keys [1]: [ca_address_sk#6] +Right keys [1]: [ca_address_sk#32] Join condition: None (44) Project [codegen id : 14] -Output [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#7] -Input [8]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_address_sk#6, ca_city#7] +Output [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#33] +Input [8]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_address_sk#32, ca_city#33] (45) HashAggregate [codegen id : 14] -Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#7] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] +Input [7]: [ss_customer_sk#10, ss_addr_sk#12, ss_ticket_number#14, ss_ext_sales_price#15, ss_ext_list_price#16, ss_ext_tax#17, ca_city#33] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33] Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#15)), partial_sum(UnscaledValue(ss_ext_list_price#16)), partial_sum(UnscaledValue(ss_ext_tax#17))] -Aggregate Attributes [3]: [sum#32, sum#33, sum#34] -Results [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#35, sum#36, sum#37] +Aggregate Attributes [3]: [sum#34, sum#35, sum#36] +Results [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33, sum#37, sum#38, sum#39] (46) HashAggregate [codegen id : 14] -Input [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7, sum#35, sum#36, sum#37] -Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#7] +Input [7]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33, sum#37, sum#38, sum#39] +Keys [4]: [ss_ticket_number#14, ss_customer_sk#10, ss_addr_sk#12, ca_city#33] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#15)), sum(UnscaledValue(ss_ext_list_price#16)), sum(UnscaledValue(ss_ext_tax#17))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#15))#38, sum(UnscaledValue(ss_ext_list_price#16))#39, sum(UnscaledValue(ss_ext_tax#17))#40] -Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#7 AS bought_city#41, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#38,17,2) AS extended_price#42, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#16))#39,17,2) AS list_price#43, MakeDecimal(sum(UnscaledValue(ss_ext_tax#17))#40,17,2) AS extended_tax#44] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#15))#40, sum(UnscaledValue(ss_ext_list_price#16))#41, sum(UnscaledValue(ss_ext_tax#17))#42] +Results [6]: [ss_ticket_number#14, ss_customer_sk#10, ca_city#33 AS bought_city#43, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#40,17,2) AS extended_price#44, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#16))#41,17,2) AS list_price#45, MakeDecimal(sum(UnscaledValue(ss_ext_tax#17))#42,17,2) AS extended_tax#46] (47) Exchange -Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#41, extended_price#42, list_price#43, extended_tax#44] -Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#43, extended_price#44, list_price#45, extended_tax#46] +Arguments: hashpartitioning(ss_customer_sk#10, 5), ENSURE_REQUIREMENTS, [id=#47] (48) Sort [codegen id : 15] -Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#41, extended_price#42, list_price#43, extended_tax#44] +Input [6]: [ss_ticket_number#14, ss_customer_sk#10, bought_city#43, extended_price#44, list_price#45, extended_tax#46] Arguments: [ss_customer_sk#10 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 16] Left keys [1]: [c_customer_sk#1] Right keys [1]: [ss_customer_sk#10] -Join condition: NOT (ca_city#7 = bought_city#41) +Join condition: NOT (ca_city#7 = bought_city#43) (50) Project [codegen id : 16] -Output [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#41, ss_ticket_number#14, extended_price#42, extended_tax#44, list_price#43] -Input [10]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#41, extended_price#42, list_price#43, extended_tax#44] +Output [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#43, ss_ticket_number#14, extended_price#44, extended_tax#46, list_price#45] +Input [10]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket_number#14, ss_customer_sk#10, bought_city#43, extended_price#44, list_price#45, extended_tax#46] (51) TakeOrderedAndProject -Input [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#41, ss_ticket_number#14, extended_price#42, extended_tax#44, list_price#43] -Arguments: 100, [c_last_name#4 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#41, ss_ticket_number#14, extended_price#42, extended_tax#44, list_price#43] +Input [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#43, ss_ticket_number#14, extended_price#44, extended_tax#46, list_price#45] +Arguments: 100, [c_last_name#4 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#43, ss_ticket_number#14, extended_price#44, extended_tax#46, list_price#45] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt index 1a9fafda6250b..101e4f272f98a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt @@ -225,20 +225,20 @@ Output [8]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#3 Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_customer_sk#39, c_current_addr_sk#40, c_first_name#41, c_last_name#42] (40) ReusedExchange [Reuses operator id: 28] -Output [2]: [ca_address_sk#22, ca_city#23] +Output [2]: [ca_address_sk#44, ca_city#45] (41) BroadcastHashJoin [codegen id : 8] Left keys [1]: [c_current_addr_sk#40] -Right keys [1]: [ca_address_sk#22] -Join condition: NOT (ca_city#23 = bought_city#35) +Right keys [1]: [ca_address_sk#44] +Join condition: NOT (ca_city#45 = bought_city#35) (42) Project [codegen id : 8] -Output [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#22, ca_city#23] +Output [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [10]: [ss_ticket_number#5, bought_city#35, extended_price#36, list_price#37, extended_tax#38, c_current_addr_sk#40, c_first_name#41, c_last_name#42, ca_address_sk#44, ca_city#45] (43) TakeOrderedAndProject -Input [8]: [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] -Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#23, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Input [8]: [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] +Arguments: 100, [c_last_name#42 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, ca_city#45, bought_city#35, ss_ticket_number#5, extended_price#36, extended_tax#38, list_price#37] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt index 97d7a81baf2a7..a0532462eff64 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt @@ -139,20 +139,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#15] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#13] -Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] +Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] (23) Exchange Input [1]: [ws_bill_customer_sk#13] -Arguments: hashpartitioning(ws_bill_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#15] +Arguments: hashpartitioning(ws_bill_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id=#16] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#13] @@ -164,38 +164,38 @@ Right keys [1]: [ws_bill_customer_sk#13] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#19] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#16] -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] +Output [1]: [cs_ship_customer_sk#17] +Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] (31) Exchange -Input [1]: [cs_ship_customer_sk#16] -Arguments: hashpartitioning(cs_ship_customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [1]: [cs_ship_customer_sk#17] +Arguments: hashpartitioning(cs_ship_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#20] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#16] -Arguments: [cs_ship_customer_sk#16 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#17] +Arguments: [cs_ship_customer_sk#17 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#16] +Right keys [1]: [cs_ship_customer_sk#17] Join condition: None (34) Project [codegen id : 13] @@ -203,84 +203,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (35) Scan parquet default.customer_address -Output [2]: [ca_address_sk#19, ca_state#20] +Output [2]: [ca_address_sk#21, ca_state#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 12] -Input [2]: [ca_address_sk#19, ca_state#20] +Input [2]: [ca_address_sk#21, ca_state#22] (37) Filter [codegen id : 12] -Input [2]: [ca_address_sk#19, ca_state#20] -Condition : (ca_state#20 IN (KY,GA,NM) AND isnotnull(ca_address_sk#19)) +Input [2]: [ca_address_sk#21, ca_state#22] +Condition : (ca_state#22 IN (KY,GA,NM) AND isnotnull(ca_address_sk#21)) (38) Project [codegen id : 12] -Output [1]: [ca_address_sk#19] -Input [2]: [ca_address_sk#19, ca_state#20] +Output [1]: [ca_address_sk#21] +Input [2]: [ca_address_sk#21, ca_state#22] (39) BroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [ca_address_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#19] +Right keys [1]: [ca_address_sk#21] Join condition: None (41) Project [codegen id : 13] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21] (42) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (43) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (44) ColumnarToRow -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] (45) Filter -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Condition : isnotnull(cd_demo_sk#23) +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Condition : isnotnull(cd_demo_sk#25) (46) BroadcastHashJoin [codegen id : 14] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#25] Join condition: None (47) Project [codegen id : 14] -Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Output [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] (48) HashAggregate [codegen id : 14] -Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] -Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] +Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] +Aggregate Attributes [1]: [count#31] +Results [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] (49) Exchange -Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] -Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] +Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, 5), ENSURE_REQUIREMENTS, [id=#33] (50) HashAggregate [codegen id : 15] -Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#30] -Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32] +Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#32 AS cnt1#33, cd_purchase_estimate#27, count(1)#32 AS cnt2#34, cd_credit_rating#28, count(1)#32 AS cnt3#35] +Aggregate Attributes [1]: [count(1)#34] +Results [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, count(1)#34 AS cnt1#35, cd_purchase_estimate#29, count(1)#34 AS cnt2#36, cd_credit_rating#30, count(1)#34 AS cnt3#37] (51) TakeOrderedAndProject -Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#33, cd_purchase_estimate#27, cnt2#34, cd_credit_rating#28, cnt3#35] -Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#33, cd_purchase_estimate#27, cnt2#34, cd_credit_rating#28, cnt3#35] +Input [8]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#35, cd_purchase_estimate#29, cnt2#36, cd_credit_rating#30, cnt3#37] +Arguments: 100, [cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_education_status#28 ASC NULLS FIRST, cd_purchase_estimate#29 ASC NULLS FIRST, cd_credit_rating#30 ASC NULLS FIRST], [cd_gender#26, cd_marital_status#27, cd_education_status#28, cnt1#35, cd_purchase_estimate#29, cnt2#36, cd_credit_rating#30, cnt3#37] ===== Subqueries ===== @@ -293,6 +293,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index b5307abaf688c..d0b3c2231d997 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -122,20 +122,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#12] -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] @@ -143,34 +143,34 @@ Right keys [1]: [ws_bill_customer_sk#12] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#6)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#18] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#18] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#15] -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] +Output [1]: [cs_ship_customer_sk#16] +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#18] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Input [1]: [cs_ship_customer_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cs_ship_customer_sk#15] +Right keys [1]: [cs_ship_customer_sk#16] Join condition: None (29) Project [codegen id : 9] @@ -178,84 +178,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (30) Scan parquet default.customer_address -Output [2]: [ca_address_sk#18, ca_state#19] +Output [2]: [ca_address_sk#20, ca_state#21] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_state#19] +Input [2]: [ca_address_sk#20, ca_state#21] (32) Filter [codegen id : 7] -Input [2]: [ca_address_sk#18, ca_state#19] -Condition : (ca_state#19 IN (KY,GA,NM) AND isnotnull(ca_address_sk#18)) +Input [2]: [ca_address_sk#20, ca_state#21] +Condition : (ca_state#21 IN (KY,GA,NM) AND isnotnull(ca_address_sk#20)) (33) Project [codegen id : 7] -Output [1]: [ca_address_sk#18] -Input [2]: [ca_address_sk#18, ca_state#19] +Output [1]: [ca_address_sk#20] +Input [2]: [ca_address_sk#20, ca_state#21] (34) BroadcastExchange -Input [1]: [ca_address_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +Input [1]: [ca_address_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#18] +Right keys [1]: [ca_address_sk#20] Join condition: None (36) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] (37) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] (39) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -Condition : isnotnull(cd_demo_sk#21) +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Condition : isnotnull(cd_demo_sk#23) (40) BroadcastExchange -Input [6]: [cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] (41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#21] +Right keys [1]: [cd_demo_sk#23] Join condition: None (42) Project [codegen id : 9] -Output [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#21, cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Output [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] (43) HashAggregate [codegen id : 9] -Input [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] -Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Input [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] +Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#28] -Results [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] +Aggregate Attributes [1]: [count#30] +Results [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#31] (44) Exchange -Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] -Arguments: hashpartitioning(cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#31] +Arguments: hashpartitioning(cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, 5), ENSURE_REQUIREMENTS, [id=#32] (45) HashAggregate [codegen id : 10] -Input [6]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26, count#29] -Keys [5]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cd_purchase_estimate#25, cd_credit_rating#26] +Input [6]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28, count#31] +Keys [5]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cd_purchase_estimate#27, cd_credit_rating#28] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#31] -Results [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, count(1)#31 AS cnt1#32, cd_purchase_estimate#25, count(1)#31 AS cnt2#33, cd_credit_rating#26, count(1)#31 AS cnt3#34] +Aggregate Attributes [1]: [count(1)#33] +Results [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count(1)#33 AS cnt1#34, cd_purchase_estimate#27, count(1)#33 AS cnt2#35, cd_credit_rating#28, count(1)#33 AS cnt3#36] (46) TakeOrderedAndProject -Input [8]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#32, cd_purchase_estimate#25, cnt2#33, cd_credit_rating#26, cnt3#34] -Arguments: 100, [cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_education_status#24 ASC NULLS FIRST, cd_purchase_estimate#25 ASC NULLS FIRST, cd_credit_rating#26 ASC NULLS FIRST], [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#32, cd_purchase_estimate#25, cnt2#33, cd_credit_rating#26, cnt3#34] +Input [8]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#34, cd_purchase_estimate#27, cnt2#35, cd_credit_rating#28, cnt3#36] +Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#34, cd_purchase_estimate#27, cnt2#35, cd_credit_rating#28, cnt3#36] ===== Subqueries ===== @@ -268,6 +268,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#6 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index 9e2d33f2bca19..7b8445a46ac54 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -109,109 +109,109 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] (16) Filter [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_store_sk#11) (17) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#13] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] -Output [2]: [ss_store_sk#1, ss_net_profit#2] -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] +Output [2]: [ss_store_sk#11, ss_net_profit#12] +Input [4]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13, d_date_sk#14] (20) Scan parquet default.store -Output [2]: [s_store_sk#8, s_state#10] +Output [2]: [s_store_sk#15, s_state#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#8, s_state#10] +Input [2]: [s_store_sk#15, s_state#16] (22) Filter [codegen id : 3] -Input [2]: [s_store_sk#8, s_state#10] -Condition : isnotnull(s_store_sk#8) +Input [2]: [s_store_sk#15, s_state#16] +Condition : isnotnull(s_store_sk#15) (23) BroadcastExchange -Input [2]: [s_store_sk#8, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +Input [2]: [s_store_sk#15, s_state#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#8] +Left keys [1]: [ss_store_sk#11] +Right keys [1]: [s_store_sk#15] Join condition: None (25) Project [codegen id : 4] -Output [2]: [ss_net_profit#2, s_state#10] -Input [4]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_state#10] +Output [2]: [ss_net_profit#12, s_state#16] +Input [4]: [ss_store_sk#11, ss_net_profit#12, s_store_sk#15, s_state#16] (26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#2, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#12] -Results [2]: [s_state#10, sum#13] +Input [2]: [ss_net_profit#12, s_state#16] +Keys [1]: [s_state#16] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum#18] +Results [2]: [s_state#16, sum#19] (27) Exchange -Input [2]: [s_state#10, sum#13] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [2]: [s_state#16, sum#19] +Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, [id=#20] (28) HashAggregate [codegen id : 5] -Input [2]: [s_state#10, sum#13] -Keys [1]: [s_state#10] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#15] -Results [3]: [s_state#10 AS s_state#16, s_state#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#15,17,2) AS _w2#17] +Input [2]: [s_state#16, sum#19] +Keys [1]: [s_state#16] +Functions [1]: [sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#12))#21] +Results [3]: [s_state#16 AS s_state#22, s_state#16, MakeDecimal(sum(UnscaledValue(ss_net_profit#12))#21,17,2) AS _w2#23] (29) Exchange -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, [id=#24] (30) Sort [codegen id : 6] -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [s_state#10 ASC NULLS FIRST, _w2#17 DESC NULLS LAST], false, 0 +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: [s_state#16 ASC NULLS FIRST, _w2#23 DESC NULLS LAST], false, 0 (31) Window -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#10], [_w2#17 DESC NULLS LAST] +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: [rank(_w2#23) windowspecdefinition(s_state#16, _w2#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#25], [s_state#16], [_w2#23 DESC NULLS LAST] (32) Filter [codegen id : 7] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (ranking#19 <= 5) +Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] +Condition : (ranking#25 <= 5) (33) Project [codegen id : 7] -Output [1]: [s_state#16] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] +Output [1]: [s_state#22] +Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] (34) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#20] +Input [1]: [s_state#22] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#26] (35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#10] -Right keys [1]: [s_state#16] +Right keys [1]: [s_state#22] Join condition: None (36) BroadcastExchange Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] @@ -224,45 +224,45 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_county#9, s_state#10 (39) Expand [codegen id : 9] Input [3]: [ss_net_profit#2, s_state#10, s_county#9] -Arguments: [List(ss_net_profit#2, s_state#10, s_county#9, 0), List(ss_net_profit#2, s_state#10, null, 1), List(ss_net_profit#2, null, null, 3)], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] +Arguments: [List(ss_net_profit#2, s_state#10, s_county#9, 0), List(ss_net_profit#2, s_state#10, null, 1), List(ss_net_profit#2, null, null, 3)], [ss_net_profit#2, s_state#28, s_county#29, spark_grouping_id#30] (40) HashAggregate [codegen id : 9] -Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] +Input [4]: [ss_net_profit#2, s_state#28, s_county#29, spark_grouping_id#30] +Keys [3]: [s_state#28, s_county#29, spark_grouping_id#30] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#25] -Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Aggregate Attributes [1]: [sum#31] +Results [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] (41) Exchange -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] +Arguments: hashpartitioning(s_state#28, s_county#29, spark_grouping_id#30, 5), ENSURE_REQUIREMENTS, [id=#33] (42) HashAggregate [codegen id : 10] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] +Input [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] +Keys [3]: [s_state#28, s_county#29, spark_grouping_id#30] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#28] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS total_sum#29, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS _w3#33] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#34] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS total_sum#35, s_state#28, s_county#29, (cast((shiftright(spark_grouping_id#30, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint)) AS lochierarchy#36, (cast((shiftright(spark_grouping_id#30, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint)) AS _w1#37, CASE WHEN (cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint) = 0) THEN s_state#28 END AS _w2#38, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS _w3#39] (43) Exchange -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: hashpartitioning(_w1#37, _w2#38, 5), ENSURE_REQUIREMENTS, [id=#40] (44) Sort [codegen id : 11] -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w3#33 DESC NULLS LAST], false, 0 +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: [_w1#37 ASC NULLS FIRST, _w2#38 ASC NULLS FIRST, _w3#39 DESC NULLS LAST], false, 0 (45) Window -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: [rank(_w3#33) windowspecdefinition(_w1#31, _w2#32, _w3#33 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#35], [_w1#31, _w2#32], [_w3#33 DESC NULLS LAST] +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: [rank(_w3#39) windowspecdefinition(_w1#37, _w2#38, _w3#39 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [_w1#37, _w2#38], [_w3#39 DESC NULLS LAST] (46) Project [codegen id : 12] -Output [5]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] -Input [8]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33, rank_within_parent#35] +Output [5]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] +Input [8]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39, rank_within_parent#41] (47) TakeOrderedAndProject -Input [5]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#35 ASC NULLS FIRST], [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] +Input [5]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] +Arguments: 100, [lochierarchy#36 DESC NULLS LAST, CASE WHEN (lochierarchy#36 = 0) THEN s_state#28 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] ===== Subqueries ===== @@ -273,6 +273,6 @@ ReusedExchange (48) (48) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#5] -Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index cd12e362c44f0..91cfba59bb5bc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -109,109 +109,109 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] (16) Filter [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_store_sk#11) (17) Scan parquet default.store -Output [2]: [s_store_sk#8, s_state#10] +Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#8, s_state#10] +Input [2]: [s_store_sk#14, s_state#15] (19) Filter [codegen id : 2] -Input [2]: [s_store_sk#8, s_state#10] -Condition : isnotnull(s_store_sk#8) +Input [2]: [s_store_sk#14, s_state#15] +Condition : isnotnull(s_store_sk#14) (20) BroadcastExchange -Input [2]: [s_store_sk#8, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +Input [2]: [s_store_sk#14, s_state#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#16] (21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#8] +Left keys [1]: [ss_store_sk#11] +Right keys [1]: [s_store_sk#14] Join condition: None (22) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, ss_sold_date_sk#3, s_state#10] -Input [5]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, s_store_sk#8, s_state#10] +Output [3]: [ss_net_profit#12, ss_sold_date_sk#13, s_state#15] +Input [5]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13, s_store_sk#14, s_state#15] (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#17] (24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#13] +Right keys [1]: [d_date_sk#17] Join condition: None (25) Project [codegen id : 4] -Output [2]: [ss_net_profit#2, s_state#10] -Input [4]: [ss_net_profit#2, ss_sold_date_sk#3, s_state#10, d_date_sk#5] +Output [2]: [ss_net_profit#12, s_state#15] +Input [4]: [ss_net_profit#12, ss_sold_date_sk#13, s_state#15, d_date_sk#17] (26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#2, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#12] -Results [2]: [s_state#10, sum#13] +Input [2]: [ss_net_profit#12, s_state#15] +Keys [1]: [s_state#15] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum#18] +Results [2]: [s_state#15, sum#19] (27) Exchange -Input [2]: [s_state#10, sum#13] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [2]: [s_state#15, sum#19] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [id=#20] (28) HashAggregate [codegen id : 5] -Input [2]: [s_state#10, sum#13] -Keys [1]: [s_state#10] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#15] -Results [3]: [s_state#10 AS s_state#16, s_state#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#15,17,2) AS _w2#17] +Input [2]: [s_state#15, sum#19] +Keys [1]: [s_state#15] +Functions [1]: [sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#12))#21] +Results [3]: [s_state#15 AS s_state#22, s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#12))#21,17,2) AS _w2#23] (29) Exchange -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [id=#24] (30) Sort [codegen id : 6] -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [s_state#10 ASC NULLS FIRST, _w2#17 DESC NULLS LAST], false, 0 +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: [s_state#15 ASC NULLS FIRST, _w2#23 DESC NULLS LAST], false, 0 (31) Window -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#10], [_w2#17 DESC NULLS LAST] +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: [rank(_w2#23) windowspecdefinition(s_state#15, _w2#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#25], [s_state#15], [_w2#23 DESC NULLS LAST] (32) Filter [codegen id : 7] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (ranking#19 <= 5) +Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] +Condition : (ranking#25 <= 5) (33) Project [codegen id : 7] -Output [1]: [s_state#16] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] +Output [1]: [s_state#22] +Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] (34) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#20] +Input [1]: [s_state#22] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#26] (35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#10] -Right keys [1]: [s_state#16] +Right keys [1]: [s_state#22] Join condition: None (36) BroadcastExchange Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] @@ -224,45 +224,45 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_county#9, s_state#10 (39) Expand [codegen id : 9] Input [3]: [ss_net_profit#2, s_state#10, s_county#9] -Arguments: [List(ss_net_profit#2, s_state#10, s_county#9, 0), List(ss_net_profit#2, s_state#10, null, 1), List(ss_net_profit#2, null, null, 3)], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] +Arguments: [List(ss_net_profit#2, s_state#10, s_county#9, 0), List(ss_net_profit#2, s_state#10, null, 1), List(ss_net_profit#2, null, null, 3)], [ss_net_profit#2, s_state#28, s_county#29, spark_grouping_id#30] (40) HashAggregate [codegen id : 9] -Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] +Input [4]: [ss_net_profit#2, s_state#28, s_county#29, spark_grouping_id#30] +Keys [3]: [s_state#28, s_county#29, spark_grouping_id#30] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#25] -Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Aggregate Attributes [1]: [sum#31] +Results [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] (41) Exchange -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] +Arguments: hashpartitioning(s_state#28, s_county#29, spark_grouping_id#30, 5), ENSURE_REQUIREMENTS, [id=#33] (42) HashAggregate [codegen id : 10] -Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] -Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] +Input [4]: [s_state#28, s_county#29, spark_grouping_id#30, sum#32] +Keys [3]: [s_state#28, s_county#29, spark_grouping_id#30] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#28] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS total_sum#29, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS _w3#33] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#34] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS total_sum#35, s_state#28, s_county#29, (cast((shiftright(spark_grouping_id#30, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint)) AS lochierarchy#36, (cast((shiftright(spark_grouping_id#30, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint)) AS _w1#37, CASE WHEN (cast((shiftright(spark_grouping_id#30, 0) & 1) as tinyint) = 0) THEN s_state#28 END AS _w2#38, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS _w3#39] (43) Exchange -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: hashpartitioning(_w1#37, _w2#38, 5), ENSURE_REQUIREMENTS, [id=#40] (44) Sort [codegen id : 11] -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w3#33 DESC NULLS LAST], false, 0 +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: [_w1#37 ASC NULLS FIRST, _w2#38 ASC NULLS FIRST, _w3#39 DESC NULLS LAST], false, 0 (45) Window -Input [7]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33] -Arguments: [rank(_w3#33) windowspecdefinition(_w1#31, _w2#32, _w3#33 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#35], [_w1#31, _w2#32], [_w3#33 DESC NULLS LAST] +Input [7]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39] +Arguments: [rank(_w3#39) windowspecdefinition(_w1#37, _w2#38, _w3#39 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#41], [_w1#37, _w2#38], [_w3#39 DESC NULLS LAST] (46) Project [codegen id : 12] -Output [5]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] -Input [8]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, _w1#31, _w2#32, _w3#33, rank_within_parent#35] +Output [5]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] +Input [8]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, _w1#37, _w2#38, _w3#39, rank_within_parent#41] (47) TakeOrderedAndProject -Input [5]: [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#35 ASC NULLS FIRST], [total_sum#29, s_state#22, s_county#23, lochierarchy#30, rank_within_parent#35] +Input [5]: [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] +Arguments: 100, [lochierarchy#36 DESC NULLS LAST, CASE WHEN (lochierarchy#36 = 0) THEN s_state#28 END ASC NULLS FIRST, rank_within_parent#41 ASC NULLS FIRST], [total_sum#35, s_state#28, s_county#29, lochierarchy#36, rank_within_parent#41] ===== Subqueries ===== @@ -273,6 +273,6 @@ ReusedExchange (48) (48) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#5] -Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt index cd88a84ede266..274d0c02b5583 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.sf100/explain.txt @@ -127,43 +127,43 @@ Input [4]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_da Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_sold_time_sk#18)) (19) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#22] (20) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#11] +Right keys [1]: [d_date_sk#22] Join condition: None (21) Project [codegen id : 5] -Output [3]: [cs_ext_sales_price#20 AS ext_price#22, cs_item_sk#19 AS sold_item_sk#23, cs_sold_time_sk#18 AS time_sk#24] -Input [5]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#11] +Output [3]: [cs_ext_sales_price#20 AS ext_price#23, cs_item_sk#19 AS sold_item_sk#24, cs_sold_time_sk#18 AS time_sk#25] +Input [5]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#22] (22) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 7] -Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] (24) Filter [codegen id : 7] -Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -Condition : (isnotnull(ss_item_sk#26) AND isnotnull(ss_sold_time_sk#25)) +Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Condition : (isnotnull(ss_item_sk#27) AND isnotnull(ss_sold_time_sk#26)) (25) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#30] (26) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ss_sold_date_sk#29] +Right keys [1]: [d_date_sk#30] Join condition: None (27) Project [codegen id : 7] -Output [3]: [ss_ext_sales_price#27 AS ext_price#29, ss_item_sk#26 AS sold_item_sk#30, ss_sold_time_sk#25 AS time_sk#31] -Input [5]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#11] +Output [3]: [ss_ext_sales_price#28 AS ext_price#31, ss_item_sk#27 AS sold_item_sk#32, ss_sold_time_sk#26 AS time_sk#33] +Input [5]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#30] (28) Union @@ -177,61 +177,61 @@ Output [4]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17] Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#15, sold_item_sk#16, time_sk#17] (31) Scan parquet default.time_dim -Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Output [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] (33) Filter [codegen id : 8] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Condition : (((t_meal_time#35 = breakfast ) OR (t_meal_time#35 = dinner )) AND isnotnull(t_time_sk#32)) +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Condition : (((t_meal_time#37 = breakfast ) OR (t_meal_time#37 = dinner )) AND isnotnull(t_time_sk#34)) (34) Project [codegen id : 8] -Output [3]: [t_time_sk#32, t_hour#33, t_minute#34] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Output [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] (35) BroadcastExchange -Input [3]: [t_time_sk#32, t_hour#33, t_minute#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [time_sk#17] -Right keys [1]: [t_time_sk#32] +Right keys [1]: [t_time_sk#34] Join condition: None (37) Project [codegen id : 9] -Output [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#33, t_minute#34] -Input [7]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17, t_time_sk#32, t_hour#33, t_minute#34] +Output [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#35, t_minute#36] +Input [7]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17, t_time_sk#34, t_hour#35, t_minute#36] (38) HashAggregate [codegen id : 9] -Input [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#33, t_minute#34] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] +Input [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#35, t_minute#36] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] Functions [1]: [partial_sum(UnscaledValue(ext_price#15))] -Aggregate Attributes [1]: [sum#37] -Results [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] +Aggregate Attributes [1]: [sum#39] +Results [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] (39) Exchange -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] -Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] +Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, 5), ENSURE_REQUIREMENTS, [id=#41] (40) HashAggregate [codegen id : 10] -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] Functions [1]: [sum(UnscaledValue(ext_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#15))#40] -Results [5]: [i_brand_id#2 AS brand_id#41, i_brand#3 AS brand#42, t_hour#33, t_minute#34, MakeDecimal(sum(UnscaledValue(ext_price#15))#40,17,2) AS ext_price#43] +Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#15))#42] +Results [5]: [i_brand_id#2 AS brand_id#43, i_brand#3 AS brand#44, t_hour#35, t_minute#36, MakeDecimal(sum(UnscaledValue(ext_price#15))#42,17,2) AS ext_price#45] (41) Exchange -Input [5]: [brand_id#41, brand#42, t_hour#33, t_minute#34, ext_price#43] -Arguments: rangepartitioning(ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [5]: [brand_id#43, brand#44, t_hour#35, t_minute#36, ext_price#45] +Arguments: rangepartitioning(ext_price#45 DESC NULLS LAST, brand_id#43 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#46] (42) Sort [codegen id : 11] -Input [5]: [brand_id#41, brand#42, t_hour#33, t_minute#34, ext_price#43] -Arguments: [ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST], true, 0 +Input [5]: [brand_id#43, brand#44, t_hour#35, t_minute#36, ext_price#45] +Arguments: [ext_price#45 DESC NULLS LAST, brand_id#43 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -244,6 +244,6 @@ Output [1]: [d_date_sk#11] Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index cd88a84ede266..274d0c02b5583 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt @@ -127,43 +127,43 @@ Input [4]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_da Condition : (isnotnull(cs_item_sk#19) AND isnotnull(cs_sold_time_sk#18)) (19) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#22] (20) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#21] -Right keys [1]: [d_date_sk#11] +Right keys [1]: [d_date_sk#22] Join condition: None (21) Project [codegen id : 5] -Output [3]: [cs_ext_sales_price#20 AS ext_price#22, cs_item_sk#19 AS sold_item_sk#23, cs_sold_time_sk#18 AS time_sk#24] -Input [5]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#11] +Output [3]: [cs_ext_sales_price#20 AS ext_price#23, cs_item_sk#19 AS sold_item_sk#24, cs_sold_time_sk#18 AS time_sk#25] +Input [5]: [cs_sold_time_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#22] (22) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Output [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#10)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 7] -Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] (24) Filter [codegen id : 7] -Input [4]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] -Condition : (isnotnull(ss_item_sk#26) AND isnotnull(ss_sold_time_sk#25)) +Input [4]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Condition : (isnotnull(ss_item_sk#27) AND isnotnull(ss_sold_time_sk#26)) (25) ReusedExchange [Reuses operator id: 13] -Output [1]: [d_date_sk#11] +Output [1]: [d_date_sk#30] (26) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [ss_sold_date_sk#28] -Right keys [1]: [d_date_sk#11] +Left keys [1]: [ss_sold_date_sk#29] +Right keys [1]: [d_date_sk#30] Join condition: None (27) Project [codegen id : 7] -Output [3]: [ss_ext_sales_price#27 AS ext_price#29, ss_item_sk#26 AS sold_item_sk#30, ss_sold_time_sk#25 AS time_sk#31] -Input [5]: [ss_sold_time_sk#25, ss_item_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#11] +Output [3]: [ss_ext_sales_price#28 AS ext_price#31, ss_item_sk#27 AS sold_item_sk#32, ss_sold_time_sk#26 AS time_sk#33] +Input [5]: [ss_sold_time_sk#26, ss_item_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#30] (28) Union @@ -177,61 +177,61 @@ Output [4]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17] Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#15, sold_item_sk#16, time_sk#17] (31) Scan parquet default.time_dim -Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Output [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 8] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] (33) Filter [codegen id : 8] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] -Condition : (((t_meal_time#35 = breakfast ) OR (t_meal_time#35 = dinner )) AND isnotnull(t_time_sk#32)) +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] +Condition : (((t_meal_time#37 = breakfast ) OR (t_meal_time#37 = dinner )) AND isnotnull(t_time_sk#34)) (34) Project [codegen id : 8] -Output [3]: [t_time_sk#32, t_hour#33, t_minute#34] -Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Output [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Input [4]: [t_time_sk#34, t_hour#35, t_minute#36, t_meal_time#37] (35) BroadcastExchange -Input [3]: [t_time_sk#32, t_hour#33, t_minute#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] +Input [3]: [t_time_sk#34, t_hour#35, t_minute#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#38] (36) BroadcastHashJoin [codegen id : 9] Left keys [1]: [time_sk#17] -Right keys [1]: [t_time_sk#32] +Right keys [1]: [t_time_sk#34] Join condition: None (37) Project [codegen id : 9] -Output [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#33, t_minute#34] -Input [7]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17, t_time_sk#32, t_hour#33, t_minute#34] +Output [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#35, t_minute#36] +Input [7]: [i_brand_id#2, i_brand#3, ext_price#15, time_sk#17, t_time_sk#34, t_hour#35, t_minute#36] (38) HashAggregate [codegen id : 9] -Input [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#33, t_minute#34] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] +Input [5]: [i_brand_id#2, i_brand#3, ext_price#15, t_hour#35, t_minute#36] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] Functions [1]: [partial_sum(UnscaledValue(ext_price#15))] -Aggregate Attributes [1]: [sum#37] -Results [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] +Aggregate Attributes [1]: [sum#39] +Results [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] (39) Exchange -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] -Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] +Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, 5), ENSURE_REQUIREMENTS, [id=#41] (40) HashAggregate [codegen id : 10] -Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#38] -Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36, sum#40] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#35, t_minute#36] Functions [1]: [sum(UnscaledValue(ext_price#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#15))#40] -Results [5]: [i_brand_id#2 AS brand_id#41, i_brand#3 AS brand#42, t_hour#33, t_minute#34, MakeDecimal(sum(UnscaledValue(ext_price#15))#40,17,2) AS ext_price#43] +Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#15))#42] +Results [5]: [i_brand_id#2 AS brand_id#43, i_brand#3 AS brand#44, t_hour#35, t_minute#36, MakeDecimal(sum(UnscaledValue(ext_price#15))#42,17,2) AS ext_price#45] (41) Exchange -Input [5]: [brand_id#41, brand#42, t_hour#33, t_minute#34, ext_price#43] -Arguments: rangepartitioning(ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [5]: [brand_id#43, brand#44, t_hour#35, t_minute#36, ext_price#45] +Arguments: rangepartitioning(ext_price#45 DESC NULLS LAST, brand_id#43 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#46] (42) Sort [codegen id : 11] -Input [5]: [brand_id#41, brand#42, t_hour#33, t_minute#34, ext_price#43] -Arguments: [ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST], true, 0 +Input [5]: [brand_id#43, brand#44, t_hour#35, t_minute#36, ext_price#45] +Arguments: [ext_price#45 DESC NULLS LAST, brand_id#43 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -244,6 +244,6 @@ Output [1]: [d_date_sk#11] Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#10 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt index 74cbccc79fdc5..9b2ead7ea96f7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.sf100/explain.txt @@ -199,285 +199,285 @@ Input [2]: [customer_id#18, year_total#19] Arguments: [customer_id#18 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] +Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] +Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] (27) Filter [codegen id : 10] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_customer_sk#1) +Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Condition : isnotnull(ss_customer_sk#21) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#25, d_year#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#5, d_year#6] +Input [2]: [d_date_sk#25, d_year#26] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#5, d_year#6] -Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2002)) AND d_year#6 IN (2001,2002)) AND isnotnull(d_date_sk#5)) +Input [2]: [d_date_sk#25, d_year#26] +Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) (31) BroadcastExchange -Input [2]: [d_date_sk#5, d_year#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +Input [2]: [d_date_sk#25, d_year#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#23] +Right keys [1]: [d_date_sk#25] Join condition: None (33) Project [codegen id : 10] -Output [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Input [5]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6] +Output [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Input [5]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] (34) Exchange -Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Arguments: hashpartitioning(ss_customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#28] (35) Sort [codegen id : 11] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Arguments: [ss_customer_sk#21 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] (37) Sort [codegen id : 13] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ss_customer_sk#21] +Right keys [1]: [c_customer_sk#29] Join condition: None (39) Project [codegen id : 14] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] -Input [7]: [ss_customer_sk#1, ss_net_paid#2, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] +Input [7]: [ss_customer_sk#21, ss_net_paid#22, d_year#26, c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] (40) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#2))] -Aggregate Attributes [1]: [sum#24] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] +Aggregate Attributes [1]: [sum#33] +Results [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] (41) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] +Arguments: hashpartitioning(c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, 5), ENSURE_REQUIREMENTS, [id=#35] (42) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ss_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#2))#27] -Results [4]: [c_customer_id#10 AS customer_id#28, c_first_name#11 AS customer_first_name#29, c_last_name#12 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#2))#27,17,2) AS year_total#31] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] +Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#22))#36] +Results [4]: [c_customer_id#30 AS customer_id#37, c_first_name#31 AS customer_first_name#38, c_last_name#32 AS customer_last_name#39, MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#36,17,2) AS year_total#40] (43) Exchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: hashpartitioning(customer_id#28, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] +Arguments: hashpartitioning(customer_id#37, 5), ENSURE_REQUIREMENTS, [id=#41] (44) Sort [codegen id : 16] -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#28 ASC NULLS FIRST], false, 0 +Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] +Arguments: [customer_id#37 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#28] +Right keys [1]: [customer_id#37] Join condition: None (46) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] (48) Filter [codegen id : 19] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_bill_customer_sk#33) +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_bill_customer_sk#42) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#45, d_year#46] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (51) Project [codegen id : 19] -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] +Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Input [5]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] (52) Exchange -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#47] (53) Sort [codegen id : 20] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Arguments: [ws_bill_customer_sk#42 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] (55) Sort [codegen id : 22] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Arguments: [c_customer_sk#48 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#33] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ws_bill_customer_sk#42] +Right keys [1]: [c_customer_sk#48] Join condition: None (57) Project [codegen id : 23] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] +Input [7]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46, c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] (58) HashAggregate [codegen id : 23] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum#37] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] +Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] +Aggregate Attributes [1]: [sum#52] +Results [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] (59) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] +Arguments: hashpartitioning(c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, 5), ENSURE_REQUIREMENTS, [id=#54] (60) HashAggregate [codegen id : 24] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#40] -Results [2]: [c_customer_id#10 AS customer_id#41, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#40,17,2) AS year_total#42] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] +Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] +Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#55] +Results [2]: [c_customer_id#49 AS customer_id#56, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#55,17,2) AS year_total#57] (61) Filter [codegen id : 24] -Input [2]: [customer_id#41, year_total#42] -Condition : (isnotnull(year_total#42) AND (year_total#42 > 0.00)) +Input [2]: [customer_id#56, year_total#57] +Condition : (isnotnull(year_total#57) AND (year_total#57 > 0.00)) (62) Project [codegen id : 24] -Output [2]: [customer_id#41 AS customer_id#43, year_total#42 AS year_total#44] -Input [2]: [customer_id#41, year_total#42] +Output [2]: [customer_id#56 AS customer_id#58, year_total#57 AS year_total#59] +Input [2]: [customer_id#56, year_total#57] (63) Exchange -Input [2]: [customer_id#43, year_total#44] -Arguments: hashpartitioning(customer_id#43, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [2]: [customer_id#58, year_total#59] +Arguments: hashpartitioning(customer_id#58, 5), ENSURE_REQUIREMENTS, [id=#60] (64) Sort [codegen id : 25] -Input [2]: [customer_id#43, year_total#44] -Arguments: [customer_id#43 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#58, year_total#59] +Arguments: [customer_id#58 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#43] +Right keys [1]: [customer_id#58] Join condition: None (66) Project [codegen id : 26] -Output [7]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44] -Input [8]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#43, year_total#44] +Output [7]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59] +Input [8]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, customer_id#58, year_total#59] (67) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] (69) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_bill_customer_sk#33) +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_bill_customer_sk#61) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#64, d_year#65] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#63] +Right keys [1]: [d_date_sk#64] Join condition: None (72) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] +Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Input [5]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65] (73) Exchange -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Arguments: hashpartitioning(ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#66] (74) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Arguments: [ws_bill_customer_sk#61 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] (76) Sort [codegen id : 31] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] +Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#33] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ws_bill_customer_sk#61] +Right keys [1]: [c_customer_sk#67] Join condition: None (78) Project [codegen id : 32] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] +Input [7]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65, c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] (79) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum#47] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] +Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#62))] +Aggregate Attributes [1]: [sum#71] +Results [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] (80) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] +Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#73] (81) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#50] -Results [2]: [c_customer_id#10 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#50,17,2) AS year_total#52] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] +Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] +Functions [1]: [sum(UnscaledValue(ws_net_paid#62))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#62))#74] +Results [2]: [c_customer_id#68 AS customer_id#75, MakeDecimal(sum(UnscaledValue(ws_net_paid#62))#74,17,2) AS year_total#76] (82) Exchange -Input [2]: [customer_id#51, year_total#52] -Arguments: hashpartitioning(customer_id#51, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [2]: [customer_id#75, year_total#76] +Arguments: hashpartitioning(customer_id#75, 5), ENSURE_REQUIREMENTS, [id=#77] (83) Sort [codegen id : 34] -Input [2]: [customer_id#51, year_total#52] -Arguments: [customer_id#51 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#75, year_total#76] +Arguments: [customer_id#75 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#51] -Join condition: (CASE WHEN (year_total#44 > 0.00) THEN CheckOverflow((promote_precision(year_total#52) / promote_precision(year_total#44)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#31) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#75] +Join condition: (CASE WHEN (year_total#59 > 0.00) THEN CheckOverflow((promote_precision(year_total#76) / promote_precision(year_total#59)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#40) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) (85) Project [codegen id : 35] -Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Input [9]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44, customer_id#51, year_total#52] +Output [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] +Input [9]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59, customer_id#75, year_total#76] (86) TakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: 100, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] +Input [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] +Arguments: 100, [customer_id#37 ASC NULLS FIRST, customer_id#37 ASC NULLS FIRST, customer_id#37 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39] ===== Subqueries ===== @@ -488,15 +488,15 @@ ReusedExchange (87) (87) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#5, d_year#6] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#21 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#25, d_year#26] -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#21 +Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt index db3a0a43be591..9fccc4c4ba66d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/explain.txt @@ -171,282 +171,282 @@ Input [2]: [customer_id#17, year_total#18] Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) (20) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] (22) Filter [codegen id : 6] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Condition : (isnotnull(c_customer_sk#19) AND isnotnull(c_customer_id#20)) (23) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] (25) Filter [codegen id : 4] -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_customer_sk#5) +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_customer_sk#23) (26) BroadcastExchange -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#5] +Left keys [1]: [c_customer_sk#19] +Right keys [1]: [ss_customer_sk#23] Join condition: None (28) Project [codegen id : 6] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25] +Input [7]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#28, d_year#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#10, d_year#11] +Input [2]: [d_date_sk#28, d_year#29] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#10, d_year#11] -Condition : (((isnotnull(d_year#11) AND (d_year#11 = 2002)) AND d_year#11 IN (2001,2002)) AND isnotnull(d_date_sk#10)) +Input [2]: [d_date_sk#28, d_year#29] +Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) (32) BroadcastExchange -Input [2]: [d_date_sk#10, d_year#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Input [2]: [d_date_sk#28, d_year#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#25] +Right keys [1]: [d_date_sk#28] Join condition: None (34) Project [codegen id : 6] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] +Input [7]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#28, d_year#29] (35) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum#22] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] +Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] +Aggregate Attributes [1]: [sum#31] +Results [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] (36) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] +Arguments: hashpartitioning(c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, 5), ENSURE_REQUIREMENTS, [id=#33] (37) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#25] -Results [4]: [c_customer_id#2 AS customer_id#26, c_first_name#3 AS customer_first_name#27, c_last_name#4 AS customer_last_name#28, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#25,17,2) AS year_total#29] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] +Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] +Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#34] +Results [4]: [c_customer_id#20 AS customer_id#35, c_first_name#21 AS customer_first_name#36, c_last_name#22 AS customer_last_name#37, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#34,17,2) AS year_total#38] (38) BroadcastExchange -Input [4]: [customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] +Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#39] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#26] +Right keys [1]: [customer_id#35] Join condition: None (40) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] (42) Filter [codegen id : 10] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] +Condition : (isnotnull(c_customer_sk#40) AND isnotnull(c_customer_id#41)) (43) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] (45) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_bill_customer_sk#31) +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Condition : isnotnull(ws_bill_customer_sk#44) (46) BroadcastExchange -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#31] +Left keys [1]: [c_customer_sk#40] +Right keys [1]: [ws_bill_customer_sk#44] Join condition: None (48) Project [codegen id : 10] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46] +Input [7]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#48, d_year#49] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#46] +Right keys [1]: [d_date_sk#48] Join condition: None (51) Project [codegen id : 10] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] +Input [7]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46, d_date_sk#48, d_year#49] (52) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum#35] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] +Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#45))] +Aggregate Attributes [1]: [sum#50] +Results [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] (53) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] +Arguments: hashpartitioning(c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, 5), ENSURE_REQUIREMENTS, [id=#52] (54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#38] -Results [2]: [c_customer_id#2 AS customer_id#39, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#38,17,2) AS year_total#40] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] +Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] +Functions [1]: [sum(UnscaledValue(ws_net_paid#45))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#45))#53] +Results [2]: [c_customer_id#41 AS customer_id#54, MakeDecimal(sum(UnscaledValue(ws_net_paid#45))#53,17,2) AS year_total#55] (55) Filter [codegen id : 11] -Input [2]: [customer_id#39, year_total#40] -Condition : (isnotnull(year_total#40) AND (year_total#40 > 0.00)) +Input [2]: [customer_id#54, year_total#55] +Condition : (isnotnull(year_total#55) AND (year_total#55 > 0.00)) (56) Project [codegen id : 11] -Output [2]: [customer_id#39 AS customer_id#41, year_total#40 AS year_total#42] -Input [2]: [customer_id#39, year_total#40] +Output [2]: [customer_id#54 AS customer_id#56, year_total#55 AS year_total#57] +Input [2]: [customer_id#54, year_total#55] (57) BroadcastExchange -Input [2]: [customer_id#41, year_total#42] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#43] +Input [2]: [customer_id#56, year_total#57] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#58] (58) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#41] +Right keys [1]: [customer_id#56] Join condition: None (59) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42] -Input [8]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, customer_id#41, year_total#42] +Output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57] +Input [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, customer_id#56, year_total#57] (60) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] (62) Filter [codegen id : 14] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] +Condition : (isnotnull(c_customer_sk#59) AND isnotnull(c_customer_id#60)) (63) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] (65) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_bill_customer_sk#31) +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Condition : isnotnull(ws_bill_customer_sk#63) (66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#31] +Left keys [1]: [c_customer_sk#59] +Right keys [1]: [ws_bill_customer_sk#63] Join condition: None (68) Project [codegen id : 14] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65] +Input [7]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#67, d_year#68] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#65] +Right keys [1]: [d_date_sk#67] Join condition: None (71) Project [codegen id : 14] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] +Input [7]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65, d_date_sk#67, d_year#68] (72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum#45] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] +Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#64))] +Aggregate Attributes [1]: [sum#69] +Results [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] (73) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] +Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, 5), ENSURE_REQUIREMENTS, [id=#71] (74) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#48] -Results [2]: [c_customer_id#2 AS customer_id#49, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#48,17,2) AS year_total#50] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] +Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] +Functions [1]: [sum(UnscaledValue(ws_net_paid#64))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#64))#72] +Results [2]: [c_customer_id#60 AS customer_id#73, MakeDecimal(sum(UnscaledValue(ws_net_paid#64))#72,17,2) AS year_total#74] (75) BroadcastExchange -Input [2]: [customer_id#49, year_total#50] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#51] +Input [2]: [customer_id#73, year_total#74] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#75] (76) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#49] -Join condition: (CASE WHEN (year_total#42 > 0.00) THEN CheckOverflow((promote_precision(year_total#50) / promote_precision(year_total#42)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#29) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#73] +Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#57)), 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) (77) Project [codegen id : 16] -Output [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] -Input [9]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42, customer_id#49, year_total#50] +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#57, customer_id#73, year_total#74] (78) TakeOrderedAndProject -Input [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] -Arguments: 100, [customer_id#26 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST], [customer_id#26, customer_first_name#27, customer_last_name#28] +Input [3]: [customer_id#35, customer_first_name#36, customer_last_name#37] +Arguments: 100, [customer_id#35 ASC NULLS FIRST, customer_id#35 ASC NULLS FIRST, customer_id#35 ASC NULLS FIRST], [customer_id#35, customer_first_name#36, customer_last_name#37] ===== Subqueries ===== @@ -457,15 +457,15 @@ ReusedExchange (79) (79) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#10, d_year#11] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#19 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#28, d_year#29] -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#19 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#26 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt index f87f822166e23..a06e41562f8ff 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.sf100/explain.txt @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#31] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#36, d_year#37] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#36] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] -Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (42) Exchange -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_item_sk#47) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ws_item_sk#47] +Right keys [1]: [i_item_sk#52] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#57, d_year#58] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#57] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] (55) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] -Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (61) Exchange -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] +Right keys [2]: [wr_order_number#61, wr_item_sk#60] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] +Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#55, sum#56] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Aggregate Attributes [2]: [sum#69, sum#70] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] +Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Condition : isnotnull(cs_item_sk#79) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [cs_item_sk#79] +Right keys [1]: [i_item_sk#85] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] +Input [2]: [d_date_sk#90, d_year#91] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] -Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) +Input [2]: [d_date_sk#90, d_year#91] +Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) (83) BroadcastExchange -Input [2]: [d_date_sk#71, d_year#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [d_date_sk#90, d_year#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [cs_sold_date_sk#83] +Right keys [1]: [d_date_sk#90] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] (86) Exchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] -Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#2, cs_item_sk#1] -Right keys [2]: [cr_order_number#19, cr_item_sk#18] +Left keys [2]: [cs_order_number#80, cs_item_sk#79] +Right keys [2]: [cr_order_number#95, cr_item_sk#94] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Condition : isnotnull(ss_item_sk#26) +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Condition : isnotnull(ss_item_sk#98) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#98] +Right keys [1]: [i_item_sk#103] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#108, d_year#109] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ss_sold_date_sk#102] +Right keys [1]: [d_date_sk#108] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] (101) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] +Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] +Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Condition : isnotnull(ws_item_sk#116) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ws_item_sk#116] +Right keys [1]: [i_item_sk#121] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#126, d_year#127] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ws_sold_date_sk#120] +Right keys [1]: [d_date_sk#126] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] (116) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] +Right keys [2]: [wr_order_number#130, wr_item_sk#129] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] +Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Aggregate Attributes [2]: [sum#135, sum#136] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] (127) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] +Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] (129) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] (130) Sort [codegen id : 50] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 (131) 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#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] (133) TakeOrderedAndProject -Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] -Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt index f87f822166e23..a06e41562f8ff 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/explain.txt @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#31] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#36, d_year#37] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#36] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] -Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (42) Exchange -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_item_sk#47) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ws_item_sk#47] +Right keys [1]: [i_item_sk#52] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#57, d_year#58] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#57] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] (55) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] -Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (61) Exchange -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] +Right keys [2]: [wr_order_number#61, wr_item_sk#60] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] +Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#55, sum#56] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Aggregate Attributes [2]: [sum#69, sum#70] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] +Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Condition : isnotnull(cs_item_sk#79) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [cs_item_sk#79] +Right keys [1]: [i_item_sk#85] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] +Input [2]: [d_date_sk#90, d_year#91] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] -Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) +Input [2]: [d_date_sk#90, d_year#91] +Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) (83) BroadcastExchange -Input [2]: [d_date_sk#71, d_year#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [d_date_sk#90, d_year#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [cs_sold_date_sk#83] +Right keys [1]: [d_date_sk#90] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] (86) Exchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] -Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#2, cs_item_sk#1] -Right keys [2]: [cr_order_number#19, cr_item_sk#18] +Left keys [2]: [cs_order_number#80, cs_item_sk#79] +Right keys [2]: [cr_order_number#95, cr_item_sk#94] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Condition : isnotnull(ss_item_sk#26) +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Condition : isnotnull(ss_item_sk#98) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#98] +Right keys [1]: [i_item_sk#103] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#108, d_year#109] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ss_sold_date_sk#102] +Right keys [1]: [d_date_sk#108] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] (101) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] +Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] +Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Condition : isnotnull(ws_item_sk#116) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ws_item_sk#116] +Right keys [1]: [i_item_sk#121] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#126, d_year#127] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ws_sold_date_sk#120] +Right keys [1]: [d_date_sk#126] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] (116) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] +Right keys [2]: [wr_order_number#130, wr_item_sk#129] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] +Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Aggregate Attributes [2]: [sum#135, sum#136] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] (127) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] +Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] (129) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] (130) Sort [codegen id : 50] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 (131) 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#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] (133) TakeOrderedAndProject -Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] -Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt index 0092354e54be0..9284172139688 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt @@ -134,93 +134,93 @@ Input [4]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sol Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [id=#19] (20) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Output [3]: [d_date_sk#20, d_year#21, d_qoy#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct (21) ColumnarToRow -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] (22) Filter -Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Condition : isnotnull(d_date_sk#5) +Input [3]: [d_date_sk#20, d_year#21, d_qoy#22] +Condition : isnotnull(d_date_sk#20) (23) BroadcastHashJoin [codegen id : 5] Left keys [1]: [ws_sold_date_sk#18] -Right keys [1]: [d_date_sk#5] +Right keys [1]: [d_date_sk#20] Join condition: None (24) Project [codegen id : 5] -Output [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#6, d_qoy#7] -Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, d_date_sk#5, d_year#6, d_qoy#7] +Output [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#21, d_qoy#22] +Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, d_date_sk#20, d_year#21, d_qoy#22] (25) BroadcastExchange -Input [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +Input [5]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#21, d_qoy#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (26) Scan parquet default.item -Output [2]: [i_item_sk#9, i_category#10] +Output [2]: [i_item_sk#24, i_category#25] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (27) ColumnarToRow -Input [2]: [i_item_sk#9, i_category#10] +Input [2]: [i_item_sk#24, i_category#25] (28) Filter -Input [2]: [i_item_sk#9, i_category#10] -Condition : isnotnull(i_item_sk#9) +Input [2]: [i_item_sk#24, i_category#25] +Condition : isnotnull(i_item_sk#24) (29) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#15] -Right keys [1]: [i_item_sk#9] +Right keys [1]: [i_item_sk#24] Join condition: None (30) Project [codegen id : 6] -Output [6]: [web AS channel#21, ws_ship_customer_sk#16 AS col_name#22, d_year#6, d_qoy#7, i_category#10, ws_ext_sales_price#17 AS ext_sales_price#23] -Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#6, d_qoy#7, i_item_sk#9, i_category#10] +Output [6]: [web AS channel#26, ws_ship_customer_sk#16 AS col_name#27, d_year#21, d_qoy#22, i_category#25, ws_ext_sales_price#17 AS ext_sales_price#28] +Input [7]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, d_year#21, d_qoy#22, i_item_sk#24, i_category#25] (31) Scan parquet default.catalog_sales -Output [4]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +Output [4]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#27)] +PartitionFilters: [isnotnull(cs_sold_date_sk#32)] PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 9] -Input [4]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +Input [4]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32] (33) Filter [codegen id : 9] -Input [4]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] -Condition : (isnull(cs_ship_addr_sk#24) AND isnotnull(cs_item_sk#25)) +Input [4]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32] +Condition : (isnull(cs_ship_addr_sk#29) AND isnotnull(cs_item_sk#30)) (34) ReusedExchange [Reuses operator id: 7] -Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] (35) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#27] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [cs_sold_date_sk#32] +Right keys [1]: [d_date_sk#33] Join condition: None (36) Project [codegen id : 9] -Output [5]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_year#6, d_qoy#7] -Input [7]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#5, d_year#6, d_qoy#7] +Output [5]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, d_year#34, d_qoy#35] +Input [7]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, cs_sold_date_sk#32, d_date_sk#33, d_year#34, d_qoy#35] (37) ReusedExchange [Reuses operator id: 13] -Output [2]: [i_item_sk#9, i_category#10] +Output [2]: [i_item_sk#36, i_category#37] (38) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#9] +Left keys [1]: [cs_item_sk#30] +Right keys [1]: [i_item_sk#36] Join condition: None (39) Project [codegen id : 9] -Output [6]: [catalog AS channel#28, cs_ship_addr_sk#24 AS col_name#29, d_year#6, d_qoy#7, i_category#10, cs_ext_sales_price#26 AS ext_sales_price#30] -Input [7]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_year#6, d_qoy#7, i_item_sk#9, i_category#10] +Output [6]: [catalog AS channel#38, cs_ship_addr_sk#29 AS col_name#39, d_year#34, d_qoy#35, i_category#37, cs_ext_sales_price#31 AS ext_sales_price#40] +Input [7]: [cs_ship_addr_sk#29, cs_item_sk#30, cs_ext_sales_price#31, d_year#34, d_qoy#35, i_item_sk#36, i_category#37] (40) Union @@ -228,21 +228,21 @@ Input [7]: [cs_ship_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_year#6, Input [6]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, ext_sales_price#14] Keys [5]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10] Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count#31, sum#32] -Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#33, sum#34] +Aggregate Attributes [2]: [count#41, sum#42] +Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#43, sum#44] (42) Exchange -Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#33, sum#34] -Arguments: hashpartitioning(channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#43, sum#44] +Arguments: hashpartitioning(channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, 5), ENSURE_REQUIREMENTS, [id=#45] (43) HashAggregate [codegen id : 11] -Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#33, sum#34] +Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count#43, sum#44] Keys [5]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count(1)#36, sum(UnscaledValue(ext_sales_price#14))#37] -Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count(1)#36 AS sales_cnt#38, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#37,17,2) AS sales_amt#39] +Aggregate Attributes [2]: [count(1)#46, sum(UnscaledValue(ext_sales_price#14))#47] +Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count(1)#46 AS sales_cnt#48, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#47,17,2) AS sales_amt#49] (44) TakeOrderedAndProject -Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#38, sales_amt#39] -Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#6 ASC NULLS FIRST, d_qoy#7 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#38, sales_amt#39] +Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#48, sales_amt#49] +Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#6 ASC NULLS FIRST, d_qoy#7 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#48, sales_amt#49] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt index 1250725d1782e..026c9396cd025 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt @@ -124,67 +124,67 @@ Input [4]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sol Condition : (isnull(ws_ship_customer_sk#16) AND isnotnull(ws_item_sk#15)) (19) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#5, i_category#6] +Output [2]: [i_item_sk#19, i_category#20] (20) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_item_sk#15] -Right keys [1]: [i_item_sk#5] +Right keys [1]: [i_item_sk#19] Join condition: None (21) Project [codegen id : 6] -Output [4]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#6] -Input [6]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_item_sk#5, i_category#6] +Output [4]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#20] +Input [6]: [ws_item_sk#15, ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_item_sk#19, i_category#20] (22) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] +Output [3]: [d_date_sk#21, d_year#22, d_qoy#23] (23) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ws_sold_date_sk#18] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#21] Join condition: None (24) Project [codegen id : 6] -Output [6]: [web AS channel#19, ws_ship_customer_sk#16 AS col_name#20, d_year#9, d_qoy#10, i_category#6, ws_ext_sales_price#17 AS ext_sales_price#21] -Input [7]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#6, d_date_sk#8, d_year#9, d_qoy#10] +Output [6]: [web AS channel#24, ws_ship_customer_sk#16 AS col_name#25, d_year#22, d_qoy#23, i_category#20, ws_ext_sales_price#17 AS ext_sales_price#26] +Input [7]: [ws_ship_customer_sk#16, ws_ext_sales_price#17, ws_sold_date_sk#18, i_category#20, d_date_sk#21, d_year#22, d_qoy#23] (25) Scan parquet default.catalog_sales -Output [4]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25] +Output [4]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#25)] +PartitionFilters: [isnotnull(cs_sold_date_sk#30)] PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 9] -Input [4]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25] +Input [4]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30] (27) Filter [codegen id : 9] -Input [4]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25] -Condition : (isnull(cs_ship_addr_sk#22) AND isnotnull(cs_item_sk#23)) +Input [4]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30] +Condition : (isnull(cs_ship_addr_sk#27) AND isnotnull(cs_item_sk#28)) (28) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#5, i_category#6] +Output [2]: [i_item_sk#31, i_category#32] (29) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_item_sk#23] -Right keys [1]: [i_item_sk#5] +Left keys [1]: [cs_item_sk#28] +Right keys [1]: [i_item_sk#31] Join condition: None (30) Project [codegen id : 9] -Output [4]: [cs_ship_addr_sk#22, cs_ext_sales_price#24, cs_sold_date_sk#25, i_category#6] -Input [6]: [cs_ship_addr_sk#22, cs_item_sk#23, cs_ext_sales_price#24, cs_sold_date_sk#25, i_item_sk#5, i_category#6] +Output [4]: [cs_ship_addr_sk#27, cs_ext_sales_price#29, cs_sold_date_sk#30, i_category#32] +Input [6]: [cs_ship_addr_sk#27, cs_item_sk#28, cs_ext_sales_price#29, cs_sold_date_sk#30, i_item_sk#31, i_category#32] (31) ReusedExchange [Reuses operator id: 13] -Output [3]: [d_date_sk#8, d_year#9, d_qoy#10] +Output [3]: [d_date_sk#33, d_year#34, d_qoy#35] (32) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#25] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#30] +Right keys [1]: [d_date_sk#33] Join condition: None (33) Project [codegen id : 9] -Output [6]: [catalog AS channel#26, cs_ship_addr_sk#22 AS col_name#27, d_year#9, d_qoy#10, i_category#6, cs_ext_sales_price#24 AS ext_sales_price#28] -Input [7]: [cs_ship_addr_sk#22, cs_ext_sales_price#24, cs_sold_date_sk#25, i_category#6, d_date_sk#8, d_year#9, d_qoy#10] +Output [6]: [catalog AS channel#36, cs_ship_addr_sk#27 AS col_name#37, d_year#34, d_qoy#35, i_category#32, cs_ext_sales_price#29 AS ext_sales_price#38] +Input [7]: [cs_ship_addr_sk#27, cs_ext_sales_price#29, cs_sold_date_sk#30, i_category#32, d_date_sk#33, d_year#34, d_qoy#35] (34) Union @@ -192,21 +192,21 @@ Input [7]: [cs_ship_addr_sk#22, cs_ext_sales_price#24, cs_sold_date_sk#25, i_cat Input [6]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, ext_sales_price#14] Keys [5]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6] Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count#29, sum#30] -Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] +Aggregate Attributes [2]: [count#39, sum#40] +Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#41, sum#42] (36) Exchange -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] -Arguments: hashpartitioning(channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, 5), ENSURE_REQUIREMENTS, [id=#33] +Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#41, sum#42] +Arguments: hashpartitioning(channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, 5), ENSURE_REQUIREMENTS, [id=#43] (37) HashAggregate [codegen id : 11] -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#31, sum#32] +Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count#41, sum#42] Keys [5]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#14))] -Aggregate Attributes [2]: [count(1)#34, sum(UnscaledValue(ext_sales_price#14))#35] -Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count(1)#34 AS sales_cnt#36, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#35,17,2) AS sales_amt#37] +Aggregate Attributes [2]: [count(1)#44, sum(UnscaledValue(ext_sales_price#14))#45] +Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count(1)#44 AS sales_cnt#46, MakeDecimal(sum(UnscaledValue(ext_sales_price#14))#45,17,2) AS sales_amt#47] (38) TakeOrderedAndProject -Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#36, sales_amt#37] -Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#36, sales_amt#37] +Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#46, sales_amt#47] +Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#46, sales_amt#47] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt index a3626dfe3293d..4b2299ca2e749 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt @@ -209,38 +209,38 @@ Output [4]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_s Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25] (26) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#26] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Right keys [1]: [cast(d_date_sk#26 as bigint)] Join condition: None (28) Project [codegen id : 6] Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#6] +Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#26] (29) HashAggregate [codegen id : 6] Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] Keys [1]: [s_store_sk#25] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#26, sum#27] -Results [3]: [s_store_sk#25, sum#28, sum#29] +Aggregate Attributes [2]: [sum#27, sum#28] +Results [3]: [s_store_sk#25, sum#29, sum#30] (30) Exchange -Input [3]: [s_store_sk#25, sum#28, sum#29] -Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [s_store_sk#25, sum#29, sum#30] +Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#25, sum#28, sum#29] +Input [3]: [s_store_sk#25, sum#29, sum#30] Keys [1]: [s_store_sk#25] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] -Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] +Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] (32) BroadcastExchange -Input [3]: [s_store_sk#25, returns#33, profit_loss#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [3]: [s_store_sk#25, returns#34, profit_loss#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] @@ -248,269 +248,269 @@ Right keys [1]: [s_store_sk#25] Join condition: None (34) Project [codegen id : 8] -Output [5]: [sales#18, coalesce(returns#33, 0.00) AS returns#36, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#37, store channel AS channel#38, s_store_sk#9 AS id#39] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] +Output [5]: [sales#18, coalesce(returns#34, 0.00) AS returns#37, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#38, store channel AS channel#39, s_store_sk#9 AS id#40] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#34, profit_loss#35] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#45] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#43] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cs_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] +Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum#44, sum#45] -Results [3]: [cs_call_center_sk#40, sum#46, sum#47] +Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum#46, sum#47] +Results [3]: [cs_call_center_sk#41, sum#48, sum#49] (41) Exchange -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] (42) HashAggregate [codegen id : 14] -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] -Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] +Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] (43) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +Output [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#55), dynamicpruningexpression(cr_returned_date_sk#55 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#5)] ReadSchema: struct (44) ColumnarToRow [codegen id : 12] -Input [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +Input [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] (45) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#58] (46) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cr_returned_date_sk#55] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#57] +Right keys [1]: [d_date_sk#58] Join condition: None (47) Project [codegen id : 12] -Output [2]: [cr_return_amount#53, cr_net_loss#54] -Input [4]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55, d_date_sk#6] +Output [2]: [cr_return_amount#55, cr_net_loss#56] +Input [4]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57, d_date_sk#58] (48) HashAggregate [codegen id : 12] -Input [2]: [cr_return_amount#53, cr_net_loss#54] +Input [2]: [cr_return_amount#55, cr_net_loss#56] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#53)), partial_sum(UnscaledValue(cr_net_loss#54))] -Aggregate Attributes [2]: [sum#56, sum#57] -Results [2]: [sum#58, sum#59] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#55)), partial_sum(UnscaledValue(cr_net_loss#56))] +Aggregate Attributes [2]: [sum#59, sum#60] +Results [2]: [sum#61, sum#62] (49) Exchange -Input [2]: [sum#58, sum#59] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#60] +Input [2]: [sum#61, sum#62] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] (50) HashAggregate [codegen id : 13] -Input [2]: [sum#58, sum#59] +Input [2]: [sum#61, sum#62] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#53)), sum(UnscaledValue(cr_net_loss#54))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#53))#61, sum(UnscaledValue(cr_net_loss#54))#62] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#53))#61,17,2) AS returns#63, MakeDecimal(sum(UnscaledValue(cr_net_loss#54))#62,17,2) AS profit_loss#64] +Functions [2]: [sum(UnscaledValue(cr_return_amount#55)), sum(UnscaledValue(cr_net_loss#56))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#55))#64, sum(UnscaledValue(cr_net_loss#56))#65] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#55))#64,17,2) AS returns#66, MakeDecimal(sum(UnscaledValue(cr_net_loss#56))#65,17,2) AS profit_loss#67] (51) BroadcastExchange -Input [2]: [returns#63, profit_loss#64] -Arguments: IdentityBroadcastMode, [id=#65] +Input [2]: [returns#66, profit_loss#67] +Arguments: IdentityBroadcastMode, [id=#68] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [sales#51, returns#63, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#64 as decimal(18,2)))), DecimalType(18,2), true) AS profit#66, catalog channel AS channel#67, cs_call_center_sk#40 AS id#68] -Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#63, profit_loss#64] +Output [5]: [sales#53, returns#66, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#67 as decimal(18,2)))), DecimalType(18,2), true) AS profit#69, catalog channel AS channel#70, cs_call_center_sk#41 AS id#71] +Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#66, profit_loss#67] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] -Condition : isnotnull(ws_web_page_sk#69) +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Condition : isnotnull(ws_web_page_sk#72) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#76] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#72] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#75] +Right keys [1]: [d_date_sk#76] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] -Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] +Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] +Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#73] +Output [1]: [wp_web_page_sk#77] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#73] +Input [1]: [wp_web_page_sk#77] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#73] -Condition : isnotnull(wp_web_page_sk#73) +Input [1]: [wp_web_page_sk#77] +Condition : isnotnull(wp_web_page_sk#77) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [1]: [wp_web_page_sk#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#69] -Right keys [1]: [wp_web_page_sk#73] +Left keys [1]: [ws_web_page_sk#72] +Right keys [1]: [wp_web_page_sk#77] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum#75, sum#76] -Results [3]: [wp_web_page_sk#73, sum#77, sum#78] +Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum#79, sum#80] +Results [3]: [wp_web_page_sk#77, sum#81, sum#82] (67) Exchange -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] -Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] +Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : isnotnull(wr_web_page_sk#84) +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Condition : isnotnull(wr_web_page_sk#88) (72) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#88] +Output [1]: [wp_web_page_sk#92] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#84] -Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] +Left keys [1]: [wr_web_page_sk#88] +Right keys [1]: [cast(wp_web_page_sk#92 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [4]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] -Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] +Output [4]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] +Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] (75) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#93] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#87] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#91] +Right keys [1]: [cast(d_date_sk#93 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Input [5]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88, d_date_sk#6] +Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] +Input [5]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92, d_date_sk#93] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum#89, sum#90] -Results [3]: [wp_web_page_sk#88, sum#91, sum#92] +Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] +Keys [1]: [wp_web_page_sk#92] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum#94, sum#95] +Results [3]: [wp_web_page_sk#92, sum#96, sum#97] (79) Exchange -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [3]: [wp_web_page_sk#92, sum#96, sum#97] +Arguments: hashpartitioning(wp_web_page_sk#92, 5), ENSURE_REQUIREMENTS, [id=#98] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] -Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] +Input [3]: [wp_web_page_sk#92, sum#96, sum#97] +Keys [1]: [wp_web_page_sk#92] +Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] +Results [3]: [wp_web_page_sk#92, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] +Input [3]: [wp_web_page_sk#92, returns#101, profit_loss#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#73] -Right keys [1]: [wp_web_page_sk#88] +Left keys [1]: [wp_web_page_sk#77] +Right keys [1]: [wp_web_page_sk#92] Join condition: None (83) Project [codegen id : 22] -Output [5]: [sales#82, coalesce(returns#96, 0.00) AS returns#99, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#100, web channel AS channel#101, wp_web_page_sk#73 AS id#102] -Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] +Output [5]: [sales#86, coalesce(returns#101, 0.00) AS returns#104, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#105, web channel AS channel#106, wp_web_page_sk#77 AS id#107] +Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#92, returns#101, profit_loss#102] (84) Union (85) Expand [codegen id : 23] -Input [5]: [sales#18, returns#36, profit#37, channel#38, id#39] -Arguments: [List(sales#18, returns#36, profit#37, channel#38, id#39, 0), List(sales#18, returns#36, profit#37, channel#38, null, 1), List(sales#18, returns#36, profit#37, null, null, 3)], [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] +Input [5]: [sales#18, returns#37, profit#38, channel#39, id#40] +Arguments: [List(sales#18, returns#37, profit#38, channel#39, id#40, 0), List(sales#18, returns#37, profit#38, channel#39, null, 1), List(sales#18, returns#37, profit#38, null, null, 3)], [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] (86) HashAggregate [codegen id : 23] -Input [6]: [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] -Keys [3]: [channel#103, id#104, spark_grouping_id#105] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#36), partial_sum(profit#37)] -Aggregate Attributes [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Results [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Input [6]: [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] +Keys [3]: [channel#108, id#109, spark_grouping_id#110] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#37), partial_sum(profit#38)] +Aggregate Attributes [6]: [sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] +Results [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] (87) Exchange -Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Arguments: hashpartitioning(channel#103, id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, [id=#118] +Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Arguments: hashpartitioning(channel#108, id#109, spark_grouping_id#110, 5), ENSURE_REQUIREMENTS, [id=#123] (88) HashAggregate [codegen id : 24] -Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Keys [3]: [channel#103, id#104, spark_grouping_id#105] -Functions [3]: [sum(sales#18), sum(returns#36), sum(profit#37)] -Aggregate Attributes [3]: [sum(sales#18)#119, sum(returns#36)#120, sum(profit#37)#121] -Results [5]: [channel#103, id#104, sum(sales#18)#119 AS sales#122, sum(returns#36)#120 AS returns#123, sum(profit#37)#121 AS profit#124] +Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Keys [3]: [channel#108, id#109, spark_grouping_id#110] +Functions [3]: [sum(sales#18), sum(returns#37), sum(profit#38)] +Aggregate Attributes [3]: [sum(sales#18)#124, sum(returns#37)#125, sum(profit#38)#126] +Results [5]: [channel#108, id#109, sum(sales#18)#124 AS sales#127, sum(returns#37)#125 AS returns#128, sum(profit#38)#126 AS profit#129] (89) TakeOrderedAndProject -Input [5]: [channel#103, id#104, sales#122, returns#123, profit#124] -Arguments: 100, [channel#103 ASC NULLS FIRST, id#104 ASC NULLS FIRST], [channel#103, id#104, sales#122, returns#123, profit#124] +Input [5]: [channel#108, id#109, sales#127, returns#128, profit#129] +Arguments: 100, [channel#108 ASC NULLS FIRST, id#109 ASC NULLS FIRST], [channel#108, id#109, sales#127, returns#128, profit#129] ===== Subqueries ===== @@ -526,14 +526,14 @@ ReusedExchange (91) (91) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#26] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#55 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt index 7a277d5362e32..618da39637e23 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt @@ -197,320 +197,320 @@ Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_s Condition : isnotnull(sr_store_sk#20) (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#25] (24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None (25) Project [codegen id : 6] Output [3]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22] -Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#6] +Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#25] (26) ReusedExchange [Reuses operator id: 14] -Output [1]: [s_store_sk#25] +Output [1]: [s_store_sk#26] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_store_sk#20] -Right keys [1]: [cast(s_store_sk#25 as bigint)] +Right keys [1]: [cast(s_store_sk#26 as bigint)] Join condition: None (28) Project [codegen id : 6] -Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#25] +Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] +Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#26] (29) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Keys [1]: [s_store_sk#25] +Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] +Keys [1]: [s_store_sk#26] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#26, sum#27] -Results [3]: [s_store_sk#25, sum#28, sum#29] +Aggregate Attributes [2]: [sum#27, sum#28] +Results [3]: [s_store_sk#26, sum#29, sum#30] (30) Exchange -Input [3]: [s_store_sk#25, sum#28, sum#29] -Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [s_store_sk#26, sum#29, sum#30] +Arguments: hashpartitioning(s_store_sk#26, 5), ENSURE_REQUIREMENTS, [id=#31] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#25, sum#28, sum#29] -Keys [1]: [s_store_sk#25] +Input [3]: [s_store_sk#26, sum#29, sum#30] +Keys [1]: [s_store_sk#26] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] -Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] +Results [3]: [s_store_sk#26, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] (32) BroadcastExchange -Input [3]: [s_store_sk#25, returns#33, profit_loss#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [3]: [s_store_sk#26, returns#34, profit_loss#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] -Right keys [1]: [s_store_sk#25] +Right keys [1]: [s_store_sk#26] Join condition: None (34) Project [codegen id : 8] -Output [5]: [sales#18, coalesce(returns#33, 0.00) AS returns#36, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#37, store channel AS channel#38, s_store_sk#9 AS id#39] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] +Output [5]: [sales#18, coalesce(returns#34, 0.00) AS returns#37, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#38, store channel AS channel#39, s_store_sk#9 AS id#40] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#26, returns#34, profit_loss#35] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#45] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#43] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cs_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] +Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum#44, sum#45] -Results [3]: [cs_call_center_sk#40, sum#46, sum#47] +Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum#46, sum#47] +Results [3]: [cs_call_center_sk#41, sum#48, sum#49] (41) Exchange -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] -Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] +Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] (43) BroadcastExchange -Input [3]: [cs_call_center_sk#40, sales#51, profit#52] -Arguments: IdentityBroadcastMode, [id=#53] +Input [3]: [cs_call_center_sk#41, sales#53, profit#54] +Arguments: IdentityBroadcastMode, [id=#55] (44) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Output [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#58), dynamicpruningexpression(cr_returned_date_sk#58 IN dynamicpruning#5)] ReadSchema: struct (45) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Input [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] (46) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#59] (47) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#56] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#58] +Right keys [1]: [d_date_sk#59] Join condition: None (48) Project [codegen id : 13] -Output [2]: [cr_return_amount#54, cr_net_loss#55] -Input [4]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56, d_date_sk#6] +Output [2]: [cr_return_amount#56, cr_net_loss#57] +Input [4]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58, d_date_sk#59] (49) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#54, cr_net_loss#55] +Input [2]: [cr_return_amount#56, cr_net_loss#57] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#54)), partial_sum(UnscaledValue(cr_net_loss#55))] -Aggregate Attributes [2]: [sum#57, sum#58] -Results [2]: [sum#59, sum#60] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#56)), partial_sum(UnscaledValue(cr_net_loss#57))] +Aggregate Attributes [2]: [sum#60, sum#61] +Results [2]: [sum#62, sum#63] (50) Exchange -Input [2]: [sum#59, sum#60] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] +Input [2]: [sum#62, sum#63] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#64] (51) HashAggregate -Input [2]: [sum#59, sum#60] +Input [2]: [sum#62, sum#63] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#54)), sum(UnscaledValue(cr_net_loss#55))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#54))#62, sum(UnscaledValue(cr_net_loss#55))#63] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#54))#62,17,2) AS returns#64, MakeDecimal(sum(UnscaledValue(cr_net_loss#55))#63,17,2) AS profit_loss#65] +Functions [2]: [sum(UnscaledValue(cr_return_amount#56)), sum(UnscaledValue(cr_net_loss#57))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#56))#65, sum(UnscaledValue(cr_net_loss#57))#66] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#56))#65,17,2) AS returns#67, MakeDecimal(sum(UnscaledValue(cr_net_loss#57))#66,17,2) AS profit_loss#68] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [sales#51, returns#64, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#65 as decimal(18,2)))), DecimalType(18,2), true) AS profit#66, catalog channel AS channel#67, cs_call_center_sk#40 AS id#68] -Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#64, profit_loss#65] +Output [5]: [sales#53, returns#67, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#68 as decimal(18,2)))), DecimalType(18,2), true) AS profit#69, catalog channel AS channel#70, cs_call_center_sk#41 AS id#71] +Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#67, profit_loss#68] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] -Condition : isnotnull(ws_web_page_sk#69) +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Condition : isnotnull(ws_web_page_sk#72) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#76] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#72] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#75] +Right keys [1]: [d_date_sk#76] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] -Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] +Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] +Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#73] +Output [1]: [wp_web_page_sk#77] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#73] +Input [1]: [wp_web_page_sk#77] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#73] -Condition : isnotnull(wp_web_page_sk#73) +Input [1]: [wp_web_page_sk#77] +Condition : isnotnull(wp_web_page_sk#77) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [1]: [wp_web_page_sk#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#69] -Right keys [1]: [wp_web_page_sk#73] +Left keys [1]: [ws_web_page_sk#72] +Right keys [1]: [wp_web_page_sk#77] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum#75, sum#76] -Results [3]: [wp_web_page_sk#73, sum#77, sum#78] +Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum#79, sum#80] +Results [3]: [wp_web_page_sk#77, sum#81, sum#82] (67) Exchange -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] -Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] +Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : isnotnull(wr_web_page_sk#84) +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Condition : isnotnull(wr_web_page_sk#88) (72) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#92] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#87] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#91] +Right keys [1]: [cast(d_date_sk#92 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86] -Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, d_date_sk#6] +Output [3]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90] +Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, d_date_sk#92] (75) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#88] +Output [1]: [wp_web_page_sk#93] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#84] -Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] +Left keys [1]: [wr_web_page_sk#88] +Right keys [1]: [cast(wp_web_page_sk#93 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum#89, sum#90] -Results [3]: [wp_web_page_sk#88, sum#91, sum#92] +Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] +Keys [1]: [wp_web_page_sk#93] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum#94, sum#95] +Results [3]: [wp_web_page_sk#93, sum#96, sum#97] (79) Exchange -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [3]: [wp_web_page_sk#93, sum#96, sum#97] +Arguments: hashpartitioning(wp_web_page_sk#93, 5), ENSURE_REQUIREMENTS, [id=#98] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] -Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] +Input [3]: [wp_web_page_sk#93, sum#96, sum#97] +Keys [1]: [wp_web_page_sk#93] +Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] +Results [3]: [wp_web_page_sk#93, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] +Input [3]: [wp_web_page_sk#93, returns#101, profit_loss#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#73] -Right keys [1]: [wp_web_page_sk#88] +Left keys [1]: [wp_web_page_sk#77] +Right keys [1]: [wp_web_page_sk#93] Join condition: None (83) Project [codegen id : 22] -Output [5]: [sales#82, coalesce(returns#96, 0.00) AS returns#99, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#100, web channel AS channel#101, wp_web_page_sk#73 AS id#102] -Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] +Output [5]: [sales#86, coalesce(returns#101, 0.00) AS returns#104, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#105, web channel AS channel#106, wp_web_page_sk#77 AS id#107] +Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#93, returns#101, profit_loss#102] (84) Union (85) Expand [codegen id : 23] -Input [5]: [sales#18, returns#36, profit#37, channel#38, id#39] -Arguments: [List(sales#18, returns#36, profit#37, channel#38, id#39, 0), List(sales#18, returns#36, profit#37, channel#38, null, 1), List(sales#18, returns#36, profit#37, null, null, 3)], [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] +Input [5]: [sales#18, returns#37, profit#38, channel#39, id#40] +Arguments: [List(sales#18, returns#37, profit#38, channel#39, id#40, 0), List(sales#18, returns#37, profit#38, channel#39, null, 1), List(sales#18, returns#37, profit#38, null, null, 3)], [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] (86) HashAggregate [codegen id : 23] -Input [6]: [sales#18, returns#36, profit#37, channel#103, id#104, spark_grouping_id#105] -Keys [3]: [channel#103, id#104, spark_grouping_id#105] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#36), partial_sum(profit#37)] -Aggregate Attributes [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Results [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Input [6]: [sales#18, returns#37, profit#38, channel#108, id#109, spark_grouping_id#110] +Keys [3]: [channel#108, id#109, spark_grouping_id#110] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#37), partial_sum(profit#38)] +Aggregate Attributes [6]: [sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] +Results [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] (87) Exchange -Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Arguments: hashpartitioning(channel#103, id#104, spark_grouping_id#105, 5), ENSURE_REQUIREMENTS, [id=#118] +Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Arguments: hashpartitioning(channel#108, id#109, spark_grouping_id#110, 5), ENSURE_REQUIREMENTS, [id=#123] (88) HashAggregate [codegen id : 24] -Input [9]: [channel#103, id#104, spark_grouping_id#105, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] -Keys [3]: [channel#103, id#104, spark_grouping_id#105] -Functions [3]: [sum(sales#18), sum(returns#36), sum(profit#37)] -Aggregate Attributes [3]: [sum(sales#18)#119, sum(returns#36)#120, sum(profit#37)#121] -Results [5]: [channel#103, id#104, sum(sales#18)#119 AS sales#122, sum(returns#36)#120 AS returns#123, sum(profit#37)#121 AS profit#124] +Input [9]: [channel#108, id#109, spark_grouping_id#110, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Keys [3]: [channel#108, id#109, spark_grouping_id#110] +Functions [3]: [sum(sales#18), sum(returns#37), sum(profit#38)] +Aggregate Attributes [3]: [sum(sales#18)#124, sum(returns#37)#125, sum(profit#38)#126] +Results [5]: [channel#108, id#109, sum(sales#18)#124 AS sales#127, sum(returns#37)#125 AS returns#128, sum(profit#38)#126 AS profit#129] (89) TakeOrderedAndProject -Input [5]: [channel#103, id#104, sales#122, returns#123, profit#124] -Arguments: 100, [channel#103 ASC NULLS FIRST, id#104 ASC NULLS FIRST], [channel#103, id#104, sales#122, returns#123, profit#124] +Input [5]: [channel#108, id#109, sales#127, returns#128, profit#129] +Arguments: 100, [channel#108 ASC NULLS FIRST, id#109 ASC NULLS FIRST], [channel#108, id#109, sales#127, returns#128, profit#129] ===== Subqueries ===== @@ -526,14 +526,14 @@ ReusedExchange (91) (91) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#25] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#58 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt index ec22a1048c116..b74247d7aef3f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.sf100/explain.txt @@ -248,164 +248,164 @@ Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale Input [9]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#43, d_year#44] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#43] Join condition: None (41) Project [codegen id : 13] -Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] -Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#14, d_year#15] +Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] +Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#43, d_year#44] (42) HashAggregate [codegen id : 13] -Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] -Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] +Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [partial_sum(cs_quantity#34), partial_sum(UnscaledValue(cs_wholesale_cost#35)), partial_sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum#43, sum#44, sum#45] -Results [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] +Aggregate Attributes [3]: [sum#45, sum#46, sum#47] +Results [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] (43) Exchange -Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] -Arguments: hashpartitioning(d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] +Arguments: hashpartitioning(d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#51] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] -Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] +Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [sum(cs_quantity#34), sum(UnscaledValue(cs_wholesale_cost#35)), sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum(cs_quantity#34)#50, sum(UnscaledValue(cs_wholesale_cost#35))#51, sum(UnscaledValue(cs_sales_price#36))#52] -Results [6]: [d_year#15 AS cs_sold_year#53, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#54, sum(cs_quantity#34)#50 AS cs_qty#55, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#51,17,2) AS cs_wc#56, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#52,17,2) AS cs_sp#57] +Aggregate Attributes [3]: [sum(cs_quantity#34)#52, sum(UnscaledValue(cs_wholesale_cost#35))#53, sum(UnscaledValue(cs_sales_price#36))#54] +Results [6]: [d_year#44 AS cs_sold_year#55, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#56, sum(cs_quantity#34)#52 AS cs_qty#57, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#53,17,2) AS cs_wc#58, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#54,17,2) AS cs_sp#59] (45) Filter [codegen id : 14] -Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] -Condition : (coalesce(cs_qty#55, 0) > 0) +Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] +Condition : (coalesce(cs_qty#57, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] -Arguments: [cs_sold_year#53 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] +Arguments: [cs_sold_year#55 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#56 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54] +Right keys [3]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] (49) Scan parquet default.web_sales -Output [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Output [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] (51) Filter [codegen id : 16] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Condition : (isnotnull(ws_item_sk#58) AND isnotnull(ws_bill_customer_sk#59)) +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Condition : (isnotnull(ws_item_sk#60) AND isnotnull(ws_bill_customer_sk#61)) (52) Exchange -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Arguments: hashpartitioning(cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint), 5), ENSURE_REQUIREMENTS, [id=#65] +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Arguments: hashpartitioning(cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint), 5), ENSURE_REQUIREMENTS, [id=#67] (53) Sort [codegen id : 17] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Arguments: [cast(ws_order_number#60 as bigint) ASC NULLS FIRST, cast(ws_item_sk#58 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Arguments: [cast(ws_order_number#62 as bigint) ASC NULLS FIRST, cast(ws_item_sk#60 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.web_returns -Output [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Output [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] (56) Filter [codegen id : 18] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] -Condition : (isnotnull(wr_order_number#67) AND isnotnull(wr_item_sk#66)) +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Condition : (isnotnull(wr_order_number#69) AND isnotnull(wr_item_sk#68)) (57) Project [codegen id : 18] -Output [2]: [wr_item_sk#66, wr_order_number#67] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Output [2]: [wr_item_sk#68, wr_order_number#69] +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] (58) Exchange -Input [2]: [wr_item_sk#66, wr_order_number#67] -Arguments: hashpartitioning(wr_order_number#67, wr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [wr_item_sk#68, wr_order_number#69] +Arguments: hashpartitioning(wr_order_number#69, wr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] (59) Sort [codegen id : 19] -Input [2]: [wr_item_sk#66, wr_order_number#67] -Arguments: [wr_order_number#67 ASC NULLS FIRST, wr_item_sk#66 ASC NULLS FIRST], false, 0 +Input [2]: [wr_item_sk#68, wr_order_number#69] +Arguments: [wr_order_number#69 ASC NULLS FIRST, wr_item_sk#68 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint)] -Right keys [2]: [wr_order_number#67, wr_item_sk#66] +Left keys [2]: [cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint)] +Right keys [2]: [wr_order_number#69, wr_item_sk#68] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] -Condition : isnull(wr_order_number#67) +Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] +Condition : isnull(wr_order_number#69) (62) Project [codegen id : 21] -Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] +Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#72, d_year#73] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [ws_sold_date_sk#64] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#66] +Right keys [1]: [d_date_sk#72] Join condition: None (65) Project [codegen id : 21] -Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] -Input [8]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, d_date_sk#14, d_year#15] +Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] +Input [8]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, d_date_sk#72, d_year#73] (66) HashAggregate [codegen id : 21] -Input [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] -Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] -Functions [3]: [partial_sum(ws_quantity#61), partial_sum(UnscaledValue(ws_wholesale_cost#62)), partial_sum(UnscaledValue(ws_sales_price#63))] -Aggregate Attributes [3]: [sum#70, sum#71, sum#72] -Results [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] +Input [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] +Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] +Functions [3]: [partial_sum(ws_quantity#63), partial_sum(UnscaledValue(ws_wholesale_cost#64)), partial_sum(UnscaledValue(ws_sales_price#65))] +Aggregate Attributes [3]: [sum#74, sum#75, sum#76] +Results [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] (67) Exchange -Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] +Arguments: hashpartitioning(d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#80] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] -Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] -Functions [3]: [sum(ws_quantity#61), sum(UnscaledValue(ws_wholesale_cost#62)), sum(UnscaledValue(ws_sales_price#63))] -Aggregate Attributes [3]: [sum(ws_quantity#61)#77, sum(UnscaledValue(ws_wholesale_cost#62))#78, sum(UnscaledValue(ws_sales_price#63))#79] -Results [6]: [d_year#15 AS ws_sold_year#80, ws_item_sk#58, ws_bill_customer_sk#59 AS ws_customer_sk#81, sum(ws_quantity#61)#77 AS ws_qty#82, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#62))#78,17,2) AS ws_wc#83, MakeDecimal(sum(UnscaledValue(ws_sales_price#63))#79,17,2) AS ws_sp#84] +Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] +Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] +Functions [3]: [sum(ws_quantity#63), sum(UnscaledValue(ws_wholesale_cost#64)), sum(UnscaledValue(ws_sales_price#65))] +Aggregate Attributes [3]: [sum(ws_quantity#63)#81, sum(UnscaledValue(ws_wholesale_cost#64))#82, sum(UnscaledValue(ws_sales_price#65))#83] +Results [6]: [d_year#73 AS ws_sold_year#84, ws_item_sk#60, ws_bill_customer_sk#61 AS ws_customer_sk#85, sum(ws_quantity#63)#81 AS ws_qty#86, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#64))#82,17,2) AS ws_wc#87, MakeDecimal(sum(UnscaledValue(ws_sales_price#65))#83,17,2) AS ws_sp#88] (69) Filter [codegen id : 22] -Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] -Condition : (coalesce(ws_qty#82, 0) > 0) +Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] +Condition : (coalesce(ws_qty#86, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] -Arguments: [ws_sold_year#80 ASC NULLS FIRST, ws_item_sk#58 ASC NULLS FIRST, ws_customer_sk#81 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] +Arguments: [ws_sold_year#84 ASC NULLS FIRST, ws_item_sk#60 ASC NULLS FIRST, ws_customer_sk#85 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81] +Right keys [3]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85] Join condition: None (72) Project [codegen id : 23] -Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#82 + cs_qty#55), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#82, 0) + coalesce(cs_qty#55, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#83, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#56, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#84, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#57, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#82, cs_qty#55] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57, ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] +Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#86 + cs_qty#57), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#86, 0) + coalesce(cs_qty#57, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#87, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#58, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#88, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#59, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#86, cs_qty#57] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59, ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] (73) TakeOrderedAndProject -Input [12]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#82, cs_qty#55] -Arguments: 100, [ratio#85 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#82 + cs_qty#55), 1) as double)), 2) ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] +Input [12]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#86, cs_qty#57] +Arguments: 100, [ratio#89 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#86 + cs_qty#57), 1) as double)), 2) ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt index f5eff516f9b36..c7da3a1e23c62 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/explain.txt @@ -248,164 +248,164 @@ Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale Input [9]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_order_number#33, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, wr_item_sk#39, wr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#43, d_year#44] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#37] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#43] Join condition: None (41) Project [codegen id : 13] -Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] -Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#14, d_year#15] +Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] +Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#43, d_year#44] (42) HashAggregate [codegen id : 13] -Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] -Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] +Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [partial_sum(ws_quantity#34), partial_sum(UnscaledValue(ws_wholesale_cost#35)), partial_sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum#43, sum#44, sum#45] -Results [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] +Aggregate Attributes [3]: [sum#45, sum#46, sum#47] +Results [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] (43) Exchange -Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] -Arguments: hashpartitioning(d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] +Arguments: hashpartitioning(d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#51] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] -Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] +Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [sum(ws_quantity#34), sum(UnscaledValue(ws_wholesale_cost#35)), sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum(ws_quantity#34)#50, sum(UnscaledValue(ws_wholesale_cost#35))#51, sum(UnscaledValue(ws_sales_price#36))#52] -Results [6]: [d_year#15 AS ws_sold_year#53, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#54, sum(ws_quantity#34)#50 AS ws_qty#55, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#51,17,2) AS ws_wc#56, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#52,17,2) AS ws_sp#57] +Aggregate Attributes [3]: [sum(ws_quantity#34)#52, sum(UnscaledValue(ws_wholesale_cost#35))#53, sum(UnscaledValue(ws_sales_price#36))#54] +Results [6]: [d_year#44 AS ws_sold_year#55, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#56, sum(ws_quantity#34)#52 AS ws_qty#57, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#53,17,2) AS ws_wc#58, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#54,17,2) AS ws_sp#59] (45) Filter [codegen id : 14] -Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] -Condition : (coalesce(ws_qty#55, 0) > 0) +Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] +Condition : (coalesce(ws_qty#57, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] -Arguments: [ws_sold_year#53 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] +Arguments: [ws_sold_year#55 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#56 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54] +Right keys [3]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] (49) Scan parquet default.catalog_sales -Output [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Output [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#66), dynamicpruningexpression(cs_sold_date_sk#66 IN dynamicpruning#8)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] (51) Filter [codegen id : 16] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Condition : (isnotnull(cs_item_sk#59) AND isnotnull(cs_bill_customer_sk#58)) +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Condition : (isnotnull(cs_item_sk#61) AND isnotnull(cs_bill_customer_sk#60)) (52) Exchange -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Arguments: hashpartitioning(cs_order_number#60, cs_item_sk#59, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Arguments: hashpartitioning(cs_order_number#62, cs_item_sk#61, 5), ENSURE_REQUIREMENTS, [id=#67] (53) Sort [codegen id : 17] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Arguments: [cs_order_number#60 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST], false, 0 +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Arguments: [cs_order_number#62 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST], false, 0 (54) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Output [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] (56) Filter [codegen id : 18] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] -Condition : (isnotnull(cr_order_number#67) AND isnotnull(cr_item_sk#66)) +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Condition : (isnotnull(cr_order_number#69) AND isnotnull(cr_item_sk#68)) (57) Project [codegen id : 18] -Output [2]: [cr_item_sk#66, cr_order_number#67] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Output [2]: [cr_item_sk#68, cr_order_number#69] +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] (58) Exchange -Input [2]: [cr_item_sk#66, cr_order_number#67] -Arguments: hashpartitioning(cr_order_number#67, cr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [cr_item_sk#68, cr_order_number#69] +Arguments: hashpartitioning(cr_order_number#69, cr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] (59) Sort [codegen id : 19] -Input [2]: [cr_item_sk#66, cr_order_number#67] -Arguments: [cr_order_number#67 ASC NULLS FIRST, cr_item_sk#66 ASC NULLS FIRST], false, 0 +Input [2]: [cr_item_sk#68, cr_order_number#69] +Arguments: [cr_order_number#69 ASC NULLS FIRST, cr_item_sk#68 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cs_order_number#60, cs_item_sk#59] -Right keys [2]: [cr_order_number#67, cr_item_sk#66] +Left keys [2]: [cs_order_number#62, cs_item_sk#61] +Right keys [2]: [cr_order_number#69, cr_item_sk#68] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] -Condition : isnull(cr_order_number#67) +Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] +Condition : isnull(cr_order_number#69) (62) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] +Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#72, d_year#73] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [cs_sold_date_sk#64] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [cs_sold_date_sk#66] +Right keys [1]: [d_date_sk#72] Join condition: None (65) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] -Input [8]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, d_date_sk#14, d_year#15] +Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] +Input [8]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, d_date_sk#72, d_year#73] (66) HashAggregate [codegen id : 21] -Input [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] -Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] -Functions [3]: [partial_sum(cs_quantity#61), partial_sum(UnscaledValue(cs_wholesale_cost#62)), partial_sum(UnscaledValue(cs_sales_price#63))] -Aggregate Attributes [3]: [sum#70, sum#71, sum#72] -Results [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] +Input [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] +Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] +Functions [3]: [partial_sum(cs_quantity#63), partial_sum(UnscaledValue(cs_wholesale_cost#64)), partial_sum(UnscaledValue(cs_sales_price#65))] +Aggregate Attributes [3]: [sum#74, sum#75, sum#76] +Results [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] (67) Exchange -Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] +Arguments: hashpartitioning(d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#80] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] -Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] -Functions [3]: [sum(cs_quantity#61), sum(UnscaledValue(cs_wholesale_cost#62)), sum(UnscaledValue(cs_sales_price#63))] -Aggregate Attributes [3]: [sum(cs_quantity#61)#77, sum(UnscaledValue(cs_wholesale_cost#62))#78, sum(UnscaledValue(cs_sales_price#63))#79] -Results [6]: [d_year#15 AS cs_sold_year#80, cs_item_sk#59, cs_bill_customer_sk#58 AS cs_customer_sk#81, sum(cs_quantity#61)#77 AS cs_qty#82, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#62))#78,17,2) AS cs_wc#83, MakeDecimal(sum(UnscaledValue(cs_sales_price#63))#79,17,2) AS cs_sp#84] +Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] +Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] +Functions [3]: [sum(cs_quantity#63), sum(UnscaledValue(cs_wholesale_cost#64)), sum(UnscaledValue(cs_sales_price#65))] +Aggregate Attributes [3]: [sum(cs_quantity#63)#81, sum(UnscaledValue(cs_wholesale_cost#64))#82, sum(UnscaledValue(cs_sales_price#65))#83] +Results [6]: [d_year#73 AS cs_sold_year#84, cs_item_sk#61, cs_bill_customer_sk#60 AS cs_customer_sk#85, sum(cs_quantity#63)#81 AS cs_qty#86, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#64))#82,17,2) AS cs_wc#87, MakeDecimal(sum(UnscaledValue(cs_sales_price#65))#83,17,2) AS cs_sp#88] (69) Filter [codegen id : 22] -Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] -Condition : (coalesce(cs_qty#82, 0) > 0) +Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] +Condition : (coalesce(cs_qty#86, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] -Arguments: [cs_sold_year#80 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST, cs_customer_sk#81 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] +Arguments: [cs_sold_year#84 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST, cs_customer_sk#85 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81] +Right keys [3]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85] Join condition: None (72) Project [codegen id : 23] -Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#55 + cs_qty#82), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#55, 0) + coalesce(cs_qty#82, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#56, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#83, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#57, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#84, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, cs_qty#82] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57, cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] +Output [12]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#57 + cs_qty#86), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#57, 0) + coalesce(cs_qty#86, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#58, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#87, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#59, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#88, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, cs_qty#86] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59, cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] (73) TakeOrderedAndProject -Input [12]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, cs_qty#82] -Arguments: 100, [ratio#85 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#55 + cs_qty#82), 1) as double)), 2) ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] +Input [12]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, cs_qty#86] +Arguments: 100, [ratio#89 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#57 + cs_qty#86), 1) as double)), 2) ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt index 48521c72281f4..d13b9623d1b34 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt @@ -149,157 +149,157 @@ Input [1]: [ca_zip#14] Condition : (substr(ca_zip#14, 1, 5) INSET (56910,69952,63792,39371,74351,11101,25003,97189,57834,73134,62377,51200,32754,22752,86379,14171,91110,40162,98569,28709,13394,66162,25733,25782,26065,18383,51949,87343,50298,83849,33786,64528,23470,67030,46136,25280,46820,77721,99076,18426,31880,17871,98235,45748,49156,18652,72013,51622,43848,78567,41248,13695,44165,67853,54917,53179,64034,10567,71791,68908,55565,59402,64147,85816,57855,61547,27700,68100,28810,58263,15723,83933,51103,58058,90578,82276,81096,81426,96451,77556,38607,76638,18906,62971,57047,48425,35576,11928,30625,83444,73520,51650,57647,60099,30122,94983,24128,10445,41368,26233,26859,21756,24676,19849,36420,38193,58470,39127,13595,87501,24317,15455,69399,98025,81019,48033,11376,39516,67875,92712,14867,38122,29741,42961,30469,51211,56458,15559,16021,33123,33282,33515,72823,54601,76698,56240,72175,60279,20004,68806,72325,28488,43933,50412,45200,22246,78668,79777,96765,67301,73273,49448,82636,23932,47305,29839,39192,18799,61265,37125,58943,64457,88424,24610,84935,89360,68893,30431,28898,10336,90257,59166,46081,26105,96888,36634,86284,35258,39972,22927,73241,53268,24206,27385,99543,31671,14663,30903,39861,24996,63089,88086,83921,21076,67897,66708,45721,60576,25103,52867,30450,36233,30010,96576,73171,56571,56575,64544,13955,78451,43285,18119,16725,83041,76107,79994,54364,35942,56691,19769,63435,34102,18845,22744,13354,75691,45549,23968,31387,83144,13375,15765,28577,88190,19736,73650,37930,25989,83926,94898,51798,39736,22437,55253,38415,71256,18376,42029,25858,44438,19515,38935,51649,71954,15882,18767,63193,25486,49130,37126,40604,34425,17043,12305,11634,26653,94167,36446,10516,67473,66864,72425,63981,18842,22461,42666,47770,69035,70372,28587,45266,15371,15798,45375,90225,16807,31016,68014,21337,19505,50016,10144,84093,21286,19430,34322,91068,94945,72305,24671,58048,65084,28545,21195,20548,22245,77191,96976,48583,76231,15734,61810,11356,68621,68786,98359,41367,26689,69913,76614,68101,88885,50308,79077,18270,28915,29178,53672,62878,10390,14922,68341,56529,41766,68309,56616,15126,61860,97789,11489,45692,41918,72151,72550,27156,36495,70738,17879,53535,17920,68880,78890,35850,14089,58078,65164,27068,26231,13376,57665,32213,77610,87816,21309,15146,86198,91137,55307,67467,40558,94627,82136,22351,89091,20260,23006,91393,47537,62496,98294,18840,71286,81312,31029,70466,35458,14060,22685,28286,25631,19512,40081,63837,14328,35474,22152,76232,51061,86057,17183) AND isnotnull(substr(ca_zip#14, 1, 5))) (22) Scan parquet default.customer_address -Output [2]: [ca_address_sk#15, ca_zip#14] +Output [2]: [ca_address_sk#15, ca_zip#16] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (23) ColumnarToRow [codegen id : 5] -Input [2]: [ca_address_sk#15, ca_zip#14] +Input [2]: [ca_address_sk#15, ca_zip#16] (24) Filter [codegen id : 5] -Input [2]: [ca_address_sk#15, ca_zip#14] +Input [2]: [ca_address_sk#15, ca_zip#16] Condition : isnotnull(ca_address_sk#15) (25) Exchange -Input [2]: [ca_address_sk#15, ca_zip#14] -Arguments: hashpartitioning(ca_address_sk#15, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [2]: [ca_address_sk#15, ca_zip#16] +Arguments: hashpartitioning(ca_address_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] (26) Sort [codegen id : 6] -Input [2]: [ca_address_sk#15, ca_zip#14] +Input [2]: [ca_address_sk#15, ca_zip#16] Arguments: [ca_address_sk#15 ASC NULLS FIRST], false, 0 (27) Scan parquet default.customer -Output [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] +Output [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct (28) ColumnarToRow [codegen id : 7] -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] +Input [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] (29) Filter [codegen id : 7] -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] -Condition : ((isnotnull(c_preferred_cust_flag#18) AND (c_preferred_cust_flag#18 = Y)) AND isnotnull(c_current_addr_sk#17)) +Input [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] +Condition : ((isnotnull(c_preferred_cust_flag#19) AND (c_preferred_cust_flag#19 = Y)) AND isnotnull(c_current_addr_sk#18)) (30) Project [codegen id : 7] -Output [1]: [c_current_addr_sk#17] -Input [2]: [c_current_addr_sk#17, c_preferred_cust_flag#18] +Output [1]: [c_current_addr_sk#18] +Input [2]: [c_current_addr_sk#18, c_preferred_cust_flag#19] (31) Exchange -Input [1]: [c_current_addr_sk#17] -Arguments: hashpartitioning(c_current_addr_sk#17, 5), ENSURE_REQUIREMENTS, [id=#19] +Input [1]: [c_current_addr_sk#18] +Arguments: hashpartitioning(c_current_addr_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] (32) Sort [codegen id : 8] -Input [1]: [c_current_addr_sk#17] -Arguments: [c_current_addr_sk#17 ASC NULLS FIRST], false, 0 +Input [1]: [c_current_addr_sk#18] +Arguments: [c_current_addr_sk#18 ASC NULLS FIRST], false, 0 (33) SortMergeJoin [codegen id : 9] Left keys [1]: [ca_address_sk#15] -Right keys [1]: [c_current_addr_sk#17] +Right keys [1]: [c_current_addr_sk#18] Join condition: None (34) Project [codegen id : 9] -Output [1]: [ca_zip#14] -Input [3]: [ca_address_sk#15, ca_zip#14, c_current_addr_sk#17] +Output [1]: [ca_zip#16] +Input [3]: [ca_address_sk#15, ca_zip#16, c_current_addr_sk#18] (35) HashAggregate [codegen id : 9] -Input [1]: [ca_zip#14] -Keys [1]: [ca_zip#14] +Input [1]: [ca_zip#16] +Keys [1]: [ca_zip#16] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#20] -Results [2]: [ca_zip#14, count#21] +Aggregate Attributes [1]: [count#21] +Results [2]: [ca_zip#16, count#22] (36) Exchange -Input [2]: [ca_zip#14, count#21] -Arguments: hashpartitioning(ca_zip#14, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [2]: [ca_zip#16, count#22] +Arguments: hashpartitioning(ca_zip#16, 5), ENSURE_REQUIREMENTS, [id=#23] (37) HashAggregate [codegen id : 10] -Input [2]: [ca_zip#14, count#21] -Keys [1]: [ca_zip#14] +Input [2]: [ca_zip#16, count#22] +Keys [1]: [ca_zip#16] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#23] -Results [2]: [substr(ca_zip#14, 1, 5) AS ca_zip#24, count(1)#23 AS count(1)#25] +Aggregate Attributes [1]: [count(1)#24] +Results [2]: [substr(ca_zip#16, 1, 5) AS ca_zip#25, count(1)#24 AS count(1)#26] (38) Filter [codegen id : 10] -Input [2]: [ca_zip#24, count(1)#25] -Condition : (count(1)#25 > 10) +Input [2]: [ca_zip#25, count(1)#26] +Condition : (count(1)#26 > 10) (39) Project [codegen id : 10] -Output [1]: [ca_zip#24] -Input [2]: [ca_zip#24, count(1)#25] +Output [1]: [ca_zip#25] +Input [2]: [ca_zip#25, count(1)#26] (40) BroadcastExchange -Input [1]: [ca_zip#24] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#26] +Input [1]: [ca_zip#25] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#27] (41) BroadcastHashJoin [codegen id : 11] Left keys [2]: [coalesce(substr(ca_zip#14, 1, 5), ), isnull(substr(ca_zip#14, 1, 5))] -Right keys [2]: [coalesce(ca_zip#24, ), isnull(ca_zip#24)] +Right keys [2]: [coalesce(ca_zip#25, ), isnull(ca_zip#25)] Join condition: None (42) Project [codegen id : 11] -Output [1]: [substr(ca_zip#14, 1, 5) AS ca_zip#27] +Output [1]: [substr(ca_zip#14, 1, 5) AS ca_zip#28] Input [1]: [ca_zip#14] (43) HashAggregate [codegen id : 11] -Input [1]: [ca_zip#27] -Keys [1]: [ca_zip#27] +Input [1]: [ca_zip#28] +Keys [1]: [ca_zip#28] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#27] +Results [1]: [ca_zip#28] (44) Exchange -Input [1]: [ca_zip#27] -Arguments: hashpartitioning(ca_zip#27, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [1]: [ca_zip#28] +Arguments: hashpartitioning(ca_zip#28, 5), ENSURE_REQUIREMENTS, [id=#29] (45) HashAggregate [codegen id : 12] -Input [1]: [ca_zip#27] -Keys [1]: [ca_zip#27] +Input [1]: [ca_zip#28] +Keys [1]: [ca_zip#28] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#27] +Results [1]: [ca_zip#28] (46) Exchange -Input [1]: [ca_zip#27] -Arguments: hashpartitioning(substr(ca_zip#27, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#29] +Input [1]: [ca_zip#28] +Arguments: hashpartitioning(substr(ca_zip#28, 1, 2), 5), ENSURE_REQUIREMENTS, [id=#30] (47) Sort [codegen id : 13] -Input [1]: [ca_zip#27] -Arguments: [substr(ca_zip#27, 1, 2) ASC NULLS FIRST], false, 0 +Input [1]: [ca_zip#28] +Arguments: [substr(ca_zip#28, 1, 2) ASC NULLS FIRST], false, 0 (48) SortMergeJoin [codegen id : 14] Left keys [1]: [substr(s_zip#11, 1, 2)] -Right keys [1]: [substr(ca_zip#27, 1, 2)] +Right keys [1]: [substr(ca_zip#28, 1, 2)] Join condition: None (49) Project [codegen id : 14] Output [2]: [ss_net_profit#2, s_store_name#10] -Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#27] +Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#28] (50) HashAggregate [codegen id : 14] Input [2]: [ss_net_profit#2, s_store_name#10] Keys [1]: [s_store_name#10] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#30] -Results [2]: [s_store_name#10, sum#31] +Aggregate Attributes [1]: [sum#31] +Results [2]: [s_store_name#10, sum#32] (51) Exchange -Input [2]: [s_store_name#10, sum#31] -Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [2]: [s_store_name#10, sum#32] +Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#33] (52) HashAggregate [codegen id : 15] -Input [2]: [s_store_name#10, sum#31] +Input [2]: [s_store_name#10, sum#32] Keys [1]: [s_store_name#10] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#33] -Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#33,17,2) AS sum(ss_net_profit)#34] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#34] +Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#34,17,2) AS sum(ss_net_profit)#35] (53) TakeOrderedAndProject -Input [2]: [s_store_name#10, sum(ss_net_profit)#34] -Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#34] +Input [2]: [s_store_name#10, sum(ss_net_profit)#35] +Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#35] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt index 09e1189003ed6..dbb5e1f606a8d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt @@ -135,141 +135,141 @@ Input [1]: [ca_zip#13] Condition : (substr(ca_zip#13, 1, 5) INSET (56910,69952,63792,39371,74351,11101,25003,97189,57834,73134,62377,51200,32754,22752,86379,14171,91110,40162,98569,28709,13394,66162,25733,25782,26065,18383,51949,87343,50298,83849,33786,64528,23470,67030,46136,25280,46820,77721,99076,18426,31880,17871,98235,45748,49156,18652,72013,51622,43848,78567,41248,13695,44165,67853,54917,53179,64034,10567,71791,68908,55565,59402,64147,85816,57855,61547,27700,68100,28810,58263,15723,83933,51103,58058,90578,82276,81096,81426,96451,77556,38607,76638,18906,62971,57047,48425,35576,11928,30625,83444,73520,51650,57647,60099,30122,94983,24128,10445,41368,26233,26859,21756,24676,19849,36420,38193,58470,39127,13595,87501,24317,15455,69399,98025,81019,48033,11376,39516,67875,92712,14867,38122,29741,42961,30469,51211,56458,15559,16021,33123,33282,33515,72823,54601,76698,56240,72175,60279,20004,68806,72325,28488,43933,50412,45200,22246,78668,79777,96765,67301,73273,49448,82636,23932,47305,29839,39192,18799,61265,37125,58943,64457,88424,24610,84935,89360,68893,30431,28898,10336,90257,59166,46081,26105,96888,36634,86284,35258,39972,22927,73241,53268,24206,27385,99543,31671,14663,30903,39861,24996,63089,88086,83921,21076,67897,66708,45721,60576,25103,52867,30450,36233,30010,96576,73171,56571,56575,64544,13955,78451,43285,18119,16725,83041,76107,79994,54364,35942,56691,19769,63435,34102,18845,22744,13354,75691,45549,23968,31387,83144,13375,15765,28577,88190,19736,73650,37930,25989,83926,94898,51798,39736,22437,55253,38415,71256,18376,42029,25858,44438,19515,38935,51649,71954,15882,18767,63193,25486,49130,37126,40604,34425,17043,12305,11634,26653,94167,36446,10516,67473,66864,72425,63981,18842,22461,42666,47770,69035,70372,28587,45266,15371,15798,45375,90225,16807,31016,68014,21337,19505,50016,10144,84093,21286,19430,34322,91068,94945,72305,24671,58048,65084,28545,21195,20548,22245,77191,96976,48583,76231,15734,61810,11356,68621,68786,98359,41367,26689,69913,76614,68101,88885,50308,79077,18270,28915,29178,53672,62878,10390,14922,68341,56529,41766,68309,56616,15126,61860,97789,11489,45692,41918,72151,72550,27156,36495,70738,17879,53535,17920,68880,78890,35850,14089,58078,65164,27068,26231,13376,57665,32213,77610,87816,21309,15146,86198,91137,55307,67467,40558,94627,82136,22351,89091,20260,23006,91393,47537,62496,98294,18840,71286,81312,31029,70466,35458,14060,22685,28286,25631,19512,40081,63837,14328,35474,22152,76232,51061,86057,17183) AND isnotnull(substr(ca_zip#13, 1, 5))) (20) Scan parquet default.customer_address -Output [2]: [ca_address_sk#14, ca_zip#13] +Output [2]: [ca_address_sk#14, ca_zip#15] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#14, ca_zip#13] +Input [2]: [ca_address_sk#14, ca_zip#15] (22) Filter [codegen id : 4] -Input [2]: [ca_address_sk#14, ca_zip#13] +Input [2]: [ca_address_sk#14, ca_zip#15] Condition : isnotnull(ca_address_sk#14) (23) Scan parquet default.customer -Output [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Output [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 3] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] (25) Filter [codegen id : 3] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] -Condition : ((isnotnull(c_preferred_cust_flag#16) AND (c_preferred_cust_flag#16 = Y)) AND isnotnull(c_current_addr_sk#15)) +Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] +Condition : ((isnotnull(c_preferred_cust_flag#17) AND (c_preferred_cust_flag#17 = Y)) AND isnotnull(c_current_addr_sk#16)) (26) Project [codegen id : 3] -Output [1]: [c_current_addr_sk#15] -Input [2]: [c_current_addr_sk#15, c_preferred_cust_flag#16] +Output [1]: [c_current_addr_sk#16] +Input [2]: [c_current_addr_sk#16, c_preferred_cust_flag#17] (27) BroadcastExchange -Input [1]: [c_current_addr_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Input [1]: [c_current_addr_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (28) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ca_address_sk#14] -Right keys [1]: [c_current_addr_sk#15] +Right keys [1]: [c_current_addr_sk#16] Join condition: None (29) Project [codegen id : 4] -Output [1]: [ca_zip#13] -Input [3]: [ca_address_sk#14, ca_zip#13, c_current_addr_sk#15] +Output [1]: [ca_zip#15] +Input [3]: [ca_address_sk#14, ca_zip#15, c_current_addr_sk#16] (30) HashAggregate [codegen id : 4] -Input [1]: [ca_zip#13] -Keys [1]: [ca_zip#13] +Input [1]: [ca_zip#15] +Keys [1]: [ca_zip#15] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#18] -Results [2]: [ca_zip#13, count#19] +Aggregate Attributes [1]: [count#19] +Results [2]: [ca_zip#15, count#20] (31) Exchange -Input [2]: [ca_zip#13, count#19] -Arguments: hashpartitioning(ca_zip#13, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [2]: [ca_zip#15, count#20] +Arguments: hashpartitioning(ca_zip#15, 5), ENSURE_REQUIREMENTS, [id=#21] (32) HashAggregate [codegen id : 5] -Input [2]: [ca_zip#13, count#19] -Keys [1]: [ca_zip#13] +Input [2]: [ca_zip#15, count#20] +Keys [1]: [ca_zip#15] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#21] -Results [2]: [substr(ca_zip#13, 1, 5) AS ca_zip#22, count(1)#21 AS count(1)#23] +Aggregate Attributes [1]: [count(1)#22] +Results [2]: [substr(ca_zip#15, 1, 5) AS ca_zip#23, count(1)#22 AS count(1)#24] (33) Filter [codegen id : 5] -Input [2]: [ca_zip#22, count(1)#23] -Condition : (count(1)#23 > 10) +Input [2]: [ca_zip#23, count(1)#24] +Condition : (count(1)#24 > 10) (34) Project [codegen id : 5] -Output [1]: [ca_zip#22] -Input [2]: [ca_zip#22, count(1)#23] +Output [1]: [ca_zip#23] +Input [2]: [ca_zip#23, count(1)#24] (35) BroadcastExchange -Input [1]: [ca_zip#22] -Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#24] +Input [1]: [ca_zip#23] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [id=#25] (36) BroadcastHashJoin [codegen id : 6] Left keys [2]: [coalesce(substr(ca_zip#13, 1, 5), ), isnull(substr(ca_zip#13, 1, 5))] -Right keys [2]: [coalesce(ca_zip#22, ), isnull(ca_zip#22)] +Right keys [2]: [coalesce(ca_zip#23, ), isnull(ca_zip#23)] Join condition: None (37) Project [codegen id : 6] -Output [1]: [substr(ca_zip#13, 1, 5) AS ca_zip#25] +Output [1]: [substr(ca_zip#13, 1, 5) AS ca_zip#26] Input [1]: [ca_zip#13] (38) HashAggregate [codegen id : 6] -Input [1]: [ca_zip#25] -Keys [1]: [ca_zip#25] +Input [1]: [ca_zip#26] +Keys [1]: [ca_zip#26] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#25] +Results [1]: [ca_zip#26] (39) Exchange -Input [1]: [ca_zip#25] -Arguments: hashpartitioning(ca_zip#25, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [1]: [ca_zip#26] +Arguments: hashpartitioning(ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#27] (40) HashAggregate [codegen id : 7] -Input [1]: [ca_zip#25] -Keys [1]: [ca_zip#25] +Input [1]: [ca_zip#26] +Keys [1]: [ca_zip#26] Functions: [] Aggregate Attributes: [] -Results [1]: [ca_zip#25] +Results [1]: [ca_zip#26] (41) BroadcastExchange -Input [1]: [ca_zip#25] -Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [id=#27] +Input [1]: [ca_zip#26] +Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [id=#28] (42) BroadcastHashJoin [codegen id : 8] Left keys [1]: [substr(s_zip#11, 1, 2)] -Right keys [1]: [substr(ca_zip#25, 1, 2)] +Right keys [1]: [substr(ca_zip#26, 1, 2)] Join condition: None (43) Project [codegen id : 8] Output [2]: [ss_net_profit#2, s_store_name#10] -Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#25] +Input [4]: [ss_net_profit#2, s_store_name#10, s_zip#11, ca_zip#26] (44) HashAggregate [codegen id : 8] Input [2]: [ss_net_profit#2, s_store_name#10] Keys [1]: [s_store_name#10] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#28] -Results [2]: [s_store_name#10, sum#29] +Aggregate Attributes [1]: [sum#29] +Results [2]: [s_store_name#10, sum#30] (45) Exchange -Input [2]: [s_store_name#10, sum#29] -Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [2]: [s_store_name#10, sum#30] +Arguments: hashpartitioning(s_store_name#10, 5), ENSURE_REQUIREMENTS, [id=#31] (46) HashAggregate [codegen id : 9] -Input [2]: [s_store_name#10, sum#29] +Input [2]: [s_store_name#10, sum#30] Keys [1]: [s_store_name#10] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] -Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS sum(ss_net_profit)#32] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#32] +Results [2]: [s_store_name#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#32,17,2) AS sum(ss_net_profit)#33] (47) TakeOrderedAndProject -Input [2]: [s_store_name#10, sum(ss_net_profit)#32] -Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#32] +Input [2]: [s_store_name#10, sum(ss_net_profit)#33] +Arguments: 100, [s_store_name#10 ASC NULLS FIRST], [s_store_name#10, sum(ss_net_profit)#33] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt index e5c59a0ecfaad..bdb1a52a18f2d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt @@ -367,252 +367,252 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#16] +Output [1]: [i_item_sk#61] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#16] +Right keys [1]: [i_item_sk#61] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#16] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#61] (60) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#19] +Output [1]: [p_promo_sk#62] (61) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#19] +Right keys [1]: [p_promo_sk#62] Join condition: None (62) Project [codegen id : 19] Output [6]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#19] +Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#62] (63) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#63] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#63] Join condition: None (65) Project [codegen id : 19] Output [5]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#22] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#63] (66) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 18] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] (68) Filter [codegen id : 18] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Condition : isnotnull(cp_catalog_page_sk#61) +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Condition : isnotnull(cp_catalog_page_sk#64) (69) BroadcastExchange -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#61] +Right keys [1]: [cp_catalog_page_sk#64] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#64, cp_catalog_page_id#65] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Keys [1]: [cp_catalog_page_id#62] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] +Keys [1]: [cp_catalog_page_id#65] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Results [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] (73) Exchange -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [id=#77] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [cp_catalog_page_id#62] +Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Keys [1]: [cp_catalog_page_id#65] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#80, catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#62) AS id#82] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] +Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#81, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#82, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#83, catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#65) AS id#85] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) (78) Exchange -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] -Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (84) Exchange -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] -Right keys [2]: [wr_item_sk#91, wr_order_number#92] +Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] +Right keys [2]: [wr_item_sk#94, wr_order_number#95] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#16] +Output [1]: [i_item_sk#100] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#83] -Right keys [1]: [i_item_sk#16] +Left keys [1]: [ws_item_sk#86] +Right keys [1]: [i_item_sk#100] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, i_item_sk#16] +Output [7]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, i_item_sk#100] (91) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#19] +Output [1]: [p_promo_sk#101] (92) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#85] -Right keys [1]: [p_promo_sk#19] +Left keys [1]: [ws_promo_sk#88] +Right keys [1]: [p_promo_sk#101] Join condition: None (93) Project [codegen id : 29] -Output [6]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [8]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, p_promo_sk#19] +Output [6]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [8]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, p_promo_sk#101] (94) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#102] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#89] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [ws_sold_date_sk#92] +Right keys [1]: [d_date_sk#102] Join condition: None (96) Project [codegen id : 29] -Output [5]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] -Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#22] +Output [5]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] +Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#102] (97) Scan parquet default.web_site -Output [2]: [web_site_sk#97, web_site_id#98] +Output [2]: [web_site_sk#103, web_site_id#104] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (98) ColumnarToRow [codegen id : 28] -Input [2]: [web_site_sk#97, web_site_id#98] +Input [2]: [web_site_sk#103, web_site_id#104] (99) Filter [codegen id : 28] -Input [2]: [web_site_sk#97, web_site_id#98] -Condition : isnotnull(web_site_sk#97) +Input [2]: [web_site_sk#103, web_site_id#104] +Condition : isnotnull(web_site_sk#103) (100) BroadcastExchange -Input [2]: [web_site_sk#97, web_site_id#98] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Input [2]: [web_site_sk#103, web_site_id#104] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#105] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#84] -Right keys [1]: [web_site_sk#97] +Left keys [1]: [ws_web_site_sk#87] +Right keys [1]: [web_site_sk#103] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] +Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] +Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#103, web_site_id#104] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Keys [1]: [web_site_id#98] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] +Keys [1]: [web_site_id#104] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Results [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] (104) Exchange -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] +Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, [id=#116] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Keys [1]: [web_site_id#98] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#114, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#115, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#116, web channel AS channel#117, concat(web_site, web_site_id#98) AS id#118] +Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Keys [1]: [web_site_id#104] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] +Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#120, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#121, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#122, web channel AS channel#123, concat(web_site, web_site_id#104) AS id#124] (106) Union (107) Expand [codegen id : 31] Input [5]: [sales#42, returns#43, profit#44, channel#45, id#46] -Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] +Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] (108) HashAggregate [codegen id : 31] -Input [6]: [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] -Keys [3]: [channel#119, id#120, spark_grouping_id#121] +Input [6]: [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] +Keys [3]: [channel#125, id#126, spark_grouping_id#127] Functions [3]: [partial_sum(sales#42), partial_sum(returns#43), partial_sum(profit#44)] -Aggregate Attributes [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Results [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Aggregate Attributes [6]: [sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Results [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] (109) Exchange -Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] -Arguments: hashpartitioning(channel#119, id#120, spark_grouping_id#121, 5), ENSURE_REQUIREMENTS, [id=#134] +Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Arguments: hashpartitioning(channel#125, id#126, spark_grouping_id#127, 5), ENSURE_REQUIREMENTS, [id=#140] (110) HashAggregate [codegen id : 32] -Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] -Keys [3]: [channel#119, id#120, spark_grouping_id#121] +Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Keys [3]: [channel#125, id#126, spark_grouping_id#127] Functions [3]: [sum(sales#42), sum(returns#43), sum(profit#44)] -Aggregate Attributes [3]: [sum(sales#42)#135, sum(returns#43)#136, sum(profit#44)#137] -Results [5]: [channel#119, id#120, sum(sales#42)#135 AS sales#138, sum(returns#43)#136 AS returns#139, sum(profit#44)#137 AS profit#140] +Aggregate Attributes [3]: [sum(sales#42)#141, sum(returns#43)#142, sum(profit#44)#143] +Results [5]: [channel#125, id#126, sum(sales#42)#141 AS sales#144, sum(returns#43)#142 AS returns#145, sum(profit#44)#143 AS profit#146] (111) TakeOrderedAndProject -Input [5]: [channel#119, id#120, sales#138, returns#139, profit#140] -Arguments: 100, [channel#119 ASC NULLS FIRST, id#120 ASC NULLS FIRST], [channel#119, id#120, sales#138, returns#139, profit#140] +Input [5]: [channel#125, id#126, sales#144, returns#145, profit#146] +Arguments: 100, [channel#125 ASC NULLS FIRST, id#126 ASC NULLS FIRST], [channel#125, id#126, sales#144, returns#145, profit#146] ===== Subqueries ===== @@ -625,6 +625,6 @@ Output [1]: [d_date_sk#22] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt index 3d087efe472bb..aa15d27d4e562 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt @@ -367,252 +367,252 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#61] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#61] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#16] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#61] (60) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] (62) Filter [codegen id : 16] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Condition : isnotnull(cp_catalog_page_sk#61) +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Condition : isnotnull(cp_catalog_page_sk#62) (63) BroadcastExchange -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#61] +Right keys [1]: [cp_catalog_page_sk#62] Join condition: None (65) Project [codegen id : 19] -Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#62, cp_catalog_page_id#63] (66) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#22] +Output [1]: [i_item_sk#65] (67) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#22] +Right keys [1]: [i_item_sk#65] Join condition: None (68) Project [codegen id : 19] -Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, i_item_sk#22] +Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, i_item_sk#65] (69) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#25] +Output [1]: [p_promo_sk#66] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#25] +Right keys [1]: [p_promo_sk#66] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, p_promo_sk#25] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, p_promo_sk#66] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Keys [1]: [cp_catalog_page_id#62] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Keys [1]: [cp_catalog_page_id#63] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Results [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] (73) Exchange -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Arguments: hashpartitioning(cp_catalog_page_id#63, 5), ENSURE_REQUIREMENTS, [id=#77] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [cp_catalog_page_id#62] +Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Keys [1]: [cp_catalog_page_id#63] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#80, catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#62) AS id#82] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] +Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#81, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#82, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#83, catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#63) AS id#85] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) (78) Exchange -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] -Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (84) Exchange -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] -Right keys [2]: [wr_item_sk#91, wr_order_number#92] +Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] +Right keys [2]: [wr_item_sk#94, wr_order_number#95] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#100] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#89] -Right keys [1]: [d_date_sk#16] +Left keys [1]: [ws_sold_date_sk#92] +Right keys [1]: [d_date_sk#100] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#16] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#100] (91) Scan parquet default.web_site -Output [2]: [web_site_sk#97, web_site_id#98] +Output [2]: [web_site_sk#101, web_site_id#102] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 26] -Input [2]: [web_site_sk#97, web_site_id#98] +Input [2]: [web_site_sk#101, web_site_id#102] (93) Filter [codegen id : 26] -Input [2]: [web_site_sk#97, web_site_id#98] -Condition : isnotnull(web_site_sk#97) +Input [2]: [web_site_sk#101, web_site_id#102] +Condition : isnotnull(web_site_sk#101) (94) BroadcastExchange -Input [2]: [web_site_sk#97, web_site_id#98] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Input [2]: [web_site_sk#101, web_site_id#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#84] -Right keys [1]: [web_site_sk#97] +Left keys [1]: [ws_web_site_sk#87] +Right keys [1]: [web_site_sk#101] Join condition: None (96) Project [codegen id : 29] -Output [7]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] +Output [7]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#101, web_site_id#102] (97) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#22] +Output [1]: [i_item_sk#104] (98) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#83] -Right keys [1]: [i_item_sk#22] +Left keys [1]: [ws_item_sk#86] +Right keys [1]: [i_item_sk#104] Join condition: None (99) Project [codegen id : 29] -Output [6]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [8]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, i_item_sk#22] +Output [6]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [8]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, i_item_sk#104] (100) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#25] +Output [1]: [p_promo_sk#105] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#85] -Right keys [1]: [p_promo_sk#25] +Left keys [1]: [ws_promo_sk#88] +Right keys [1]: [p_promo_sk#105] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [7]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, p_promo_sk#25] +Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [7]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, p_promo_sk#105] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Keys [1]: [web_site_id#98] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Keys [1]: [web_site_id#102] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Results [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] (104) Exchange -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] +Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Arguments: hashpartitioning(web_site_id#102, 5), ENSURE_REQUIREMENTS, [id=#116] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Keys [1]: [web_site_id#98] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#114, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#115, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#116, web channel AS channel#117, concat(web_site, web_site_id#98) AS id#118] +Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Keys [1]: [web_site_id#102] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] +Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#120, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#121, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#122, web channel AS channel#123, concat(web_site, web_site_id#102) AS id#124] (106) Union (107) Expand [codegen id : 31] Input [5]: [sales#42, returns#43, profit#44, channel#45, id#46] -Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] +Arguments: [List(sales#42, returns#43, profit#44, channel#45, id#46, 0), List(sales#42, returns#43, profit#44, channel#45, null, 1), List(sales#42, returns#43, profit#44, null, null, 3)], [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] (108) HashAggregate [codegen id : 31] -Input [6]: [sales#42, returns#43, profit#44, channel#119, id#120, spark_grouping_id#121] -Keys [3]: [channel#119, id#120, spark_grouping_id#121] +Input [6]: [sales#42, returns#43, profit#44, channel#125, id#126, spark_grouping_id#127] +Keys [3]: [channel#125, id#126, spark_grouping_id#127] Functions [3]: [partial_sum(sales#42), partial_sum(returns#43), partial_sum(profit#44)] -Aggregate Attributes [6]: [sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Results [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Aggregate Attributes [6]: [sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] +Results [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] (109) Exchange -Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] -Arguments: hashpartitioning(channel#119, id#120, spark_grouping_id#121, 5), ENSURE_REQUIREMENTS, [id=#134] +Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Arguments: hashpartitioning(channel#125, id#126, spark_grouping_id#127, 5), ENSURE_REQUIREMENTS, [id=#140] (110) HashAggregate [codegen id : 32] -Input [9]: [channel#119, id#120, spark_grouping_id#121, sum#128, isEmpty#129, sum#130, isEmpty#131, sum#132, isEmpty#133] -Keys [3]: [channel#119, id#120, spark_grouping_id#121] +Input [9]: [channel#125, id#126, spark_grouping_id#127, sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] +Keys [3]: [channel#125, id#126, spark_grouping_id#127] Functions [3]: [sum(sales#42), sum(returns#43), sum(profit#44)] -Aggregate Attributes [3]: [sum(sales#42)#135, sum(returns#43)#136, sum(profit#44)#137] -Results [5]: [channel#119, id#120, sum(sales#42)#135 AS sales#138, sum(returns#43)#136 AS returns#139, sum(profit#44)#137 AS profit#140] +Aggregate Attributes [3]: [sum(sales#42)#141, sum(returns#43)#142, sum(profit#44)#143] +Results [5]: [channel#125, id#126, sum(sales#42)#141 AS sales#144, sum(returns#43)#142 AS returns#145, sum(profit#44)#143 AS profit#146] (111) TakeOrderedAndProject -Input [5]: [channel#119, id#120, sales#138, returns#139, profit#140] -Arguments: 100, [channel#119 ASC NULLS FIRST, id#120 ASC NULLS FIRST], [channel#119, id#120, sales#138, returns#139, profit#140] +Input [5]: [channel#125, id#126, sales#144, returns#145, profit#146] +Arguments: 100, [channel#125 ASC NULLS FIRST, id#126 ASC NULLS FIRST], [channel#125, id#126, sales#144, returns#145, profit#146] ===== Subqueries ===== @@ -625,6 +625,6 @@ Output [1]: [d_date_sk#16] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 IN dynamicpruning#8 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 167e142598ae8..d2d70bef8c3ee 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 @@ -170,178 +170,178 @@ Input [3]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19] Condition : isnotnull(ctr_total_return#19) (24) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Output [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(cr_returned_date_sk#23 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct (25) ColumnarToRow [codegen id : 8] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Input [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] (26) Filter [codegen id : 8] -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) +Input [4]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23] +Condition : isnotnull(cr_returning_addr_sk#21) (27) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#24] (28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None (29) Project [codegen id : 8] -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] +Output [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] +Input [5]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, cr_returned_date_sk#23, d_date_sk#24] (30) Exchange -Input [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Arguments: hashpartitioning(cr_returning_addr_sk#2, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] +Arguments: hashpartitioning(cr_returning_addr_sk#21, 5), ENSURE_REQUIREMENTS, [id=#25] (31) Sort [codegen id : 9] -Input [3]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3] -Arguments: [cr_returning_addr_sk#2 ASC NULLS FIRST], false, 0 +Input [3]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22] +Arguments: [cr_returning_addr_sk#21 ASC NULLS FIRST], false, 0 (32) ReusedExchange [Reuses operator id: 16] -Output [2]: [ca_address_sk#10, ca_state#11] +Output [2]: [ca_address_sk#26, ca_state#27] (33) Sort [codegen id : 11] -Input [2]: [ca_address_sk#10, ca_state#11] -Arguments: [ca_address_sk#10 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#26, ca_state#27] +Arguments: [ca_address_sk#26 ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 12] -Left keys [1]: [cr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [cr_returning_addr_sk#21] +Right keys [1]: [ca_address_sk#26] Join condition: None (35) Project [codegen id : 12] -Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#11] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#10, ca_state#11] +Output [3]: [cr_returning_customer_sk#20, cr_return_amt_inc_tax#22, ca_state#27] +Input [5]: [cr_returning_customer_sk#20, cr_returning_addr_sk#21, cr_return_amt_inc_tax#22, ca_address_sk#26, ca_state#27] (36) HashAggregate [codegen id : 12] -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#11] -Keys [2]: [cr_returning_customer_sk#1, ca_state#11] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum#21] -Results [3]: [cr_returning_customer_sk#1, ca_state#11, sum#22] +Input [3]: [cr_returning_customer_sk#20, cr_return_amt_inc_tax#22, ca_state#27] +Keys [2]: [cr_returning_customer_sk#20, ca_state#27] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#22))] +Aggregate Attributes [1]: [sum#28] +Results [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] (37) Exchange -Input [3]: [cr_returning_customer_sk#1, ca_state#11, sum#22] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#11, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] +Arguments: hashpartitioning(cr_returning_customer_sk#20, ca_state#27, 5), ENSURE_REQUIREMENTS, [id=#30] (38) HashAggregate [codegen id : 13] -Input [3]: [cr_returning_customer_sk#1, ca_state#11, sum#22] -Keys [2]: [cr_returning_customer_sk#1, ca_state#11] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#24] -Results [2]: [ca_state#11 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#24,17,2) AS ctr_total_return#19] +Input [3]: [cr_returning_customer_sk#20, ca_state#27, sum#29] +Keys [2]: [cr_returning_customer_sk#20, ca_state#27] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#22))#31] +Results [2]: [ca_state#27 AS ctr_state#18, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#22))#31,17,2) AS ctr_total_return#19] (39) HashAggregate [codegen id : 13] Input [2]: [ctr_state#18, ctr_total_return#19] Keys [1]: [ctr_state#18] Functions [1]: [partial_avg(ctr_total_return#19)] -Aggregate Attributes [2]: [sum#25, count#26] -Results [3]: [ctr_state#18, sum#27, count#28] +Aggregate Attributes [2]: [sum#32, count#33] +Results [3]: [ctr_state#18, sum#34, count#35] (40) Exchange -Input [3]: [ctr_state#18, sum#27, count#28] -Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [ctr_state#18, sum#34, count#35] +Arguments: hashpartitioning(ctr_state#18, 5), ENSURE_REQUIREMENTS, [id=#36] (41) HashAggregate [codegen id : 14] -Input [3]: [ctr_state#18, sum#27, count#28] +Input [3]: [ctr_state#18, sum#34, count#35] Keys [1]: [ctr_state#18] Functions [1]: [avg(ctr_total_return#19)] -Aggregate Attributes [1]: [avg(ctr_total_return#19)#30] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#30) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#31, ctr_state#18 AS ctr_state#18#32] +Aggregate Attributes [1]: [avg(ctr_total_return#19)#37] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#19)#37) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#38, ctr_state#18 AS ctr_state#18#39] (42) Filter [codegen id : 14] -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#31) +Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#38) (43) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#33] +Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#40] (44) BroadcastHashJoin [codegen id : 15] Left keys [1]: [ctr_state#18] -Right keys [1]: [ctr_state#18#32] -Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#31) +Right keys [1]: [ctr_state#18#39] +Join condition: (cast(ctr_total_return#19 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#38) (45) Project [codegen id : 15] Output [2]: [ctr_customer_sk#17, ctr_total_return#19] -Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#31, ctr_state#18#32] +Input [5]: [ctr_customer_sk#17, ctr_state#18, ctr_total_return#19, (avg(ctr_total_return) * 1.2)#38, ctr_state#18#39] (46) Exchange Input [2]: [ctr_customer_sk#17, ctr_total_return#19] -Arguments: hashpartitioning(ctr_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#34] +Arguments: hashpartitioning(ctr_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#41] (47) Sort [codegen id : 16] Input [2]: [ctr_customer_sk#17, ctr_total_return#19] Arguments: [ctr_customer_sk#17 ASC NULLS FIRST], false, 0 (48) Scan parquet default.customer -Output [6]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40] +Output [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (49) ColumnarToRow [codegen id : 18] -Input [6]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40] +Input [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] (50) Filter [codegen id : 18] -Input [6]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40] -Condition : (isnotnull(c_customer_sk#35) AND isnotnull(c_current_addr_sk#37)) +Input [6]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47] +Condition : (isnotnull(c_customer_sk#42) AND isnotnull(c_current_addr_sk#44)) (51) Scan parquet default.customer_address -Output [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Output [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] 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 (52) ColumnarToRow [codegen id : 17] -Input [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] (53) Filter [codegen id : 17] -Input [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Condition : ((isnotnull(ca_state#11) AND (ca_state#11 = GA)) AND isnotnull(ca_address_sk#10)) +Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Condition : ((isnotnull(ca_state#55) AND (ca_state#55 = GA)) AND isnotnull(ca_address_sk#48)) (54) BroadcastExchange -Input [12]: [ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] +Input [12]: [ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#60] (55) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#37] -Right keys [1]: [ca_address_sk#10] +Left keys [1]: [c_current_addr_sk#44] +Right keys [1]: [ca_address_sk#48] Join condition: None (56) Project [codegen id : 18] -Output [16]: [c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Input [18]: [c_customer_sk#35, c_customer_id#36, c_current_addr_sk#37, c_salutation#38, c_first_name#39, c_last_name#40, ca_address_sk#10, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Output [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Input [18]: [c_customer_sk#42, c_customer_id#43, c_current_addr_sk#44, c_salutation#45, c_first_name#46, c_last_name#47, ca_address_sk#48, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] (57) Exchange -Input [16]: [c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Arguments: hashpartitioning(c_customer_sk#35, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Arguments: hashpartitioning(c_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#61] (58) Sort [codegen id : 19] -Input [16]: [c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] -Arguments: [c_customer_sk#35 ASC NULLS FIRST], false, 0 +Input [16]: [c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] +Arguments: [c_customer_sk#42 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 20] Left keys [1]: [ctr_customer_sk#17] -Right keys [1]: [c_customer_sk#35] +Right keys [1]: [c_customer_sk#42] Join condition: None (60) Project [codegen id : 20] -Output [16]: [c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#19] -Input [18]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#35, c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Output [16]: [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] +Input [18]: [ctr_customer_sk#17, ctr_total_return#19, c_customer_sk#42, c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59] (61) TakeOrderedAndProject -Input [16]: [c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#19] -Arguments: 100, [c_customer_id#36 ASC NULLS FIRST, c_salutation#38 ASC NULLS FIRST, c_first_name#39 ASC NULLS FIRST, c_last_name#40 ASC NULLS FIRST, ca_street_number#41 ASC NULLS FIRST, ca_street_name#42 ASC NULLS FIRST, ca_street_type#43 ASC NULLS FIRST, ca_suite_number#44 ASC NULLS FIRST, ca_city#45 ASC NULLS FIRST, ca_county#46 ASC NULLS FIRST, ca_state#11 ASC NULLS FIRST, ca_zip#47 ASC NULLS FIRST, ca_country#48 ASC NULLS FIRST, ca_gmt_offset#49 ASC NULLS FIRST, ca_location_type#50 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#36, c_salutation#38, c_first_name#39, c_last_name#40, ca_street_number#41, ca_street_name#42, ca_street_type#43, ca_suite_number#44, ca_city#45, ca_county#46, ca_state#11, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#19] +Input [16]: [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] +Arguments: 100, [c_customer_id#43 ASC NULLS FIRST, c_salutation#45 ASC NULLS FIRST, c_first_name#46 ASC NULLS FIRST, c_last_name#47 ASC NULLS FIRST, ca_street_number#49 ASC NULLS FIRST, ca_street_name#50 ASC NULLS FIRST, ca_street_type#51 ASC NULLS FIRST, ca_suite_number#52 ASC NULLS FIRST, ca_city#53 ASC NULLS FIRST, ca_county#54 ASC NULLS FIRST, ca_state#55 ASC NULLS FIRST, ca_zip#56 ASC NULLS FIRST, ca_country#57 ASC NULLS FIRST, ca_gmt_offset#58 ASC NULLS FIRST, ca_location_type#59 ASC NULLS FIRST, ctr_total_return#19 ASC NULLS FIRST], [c_customer_id#43, c_salutation#45, c_first_name#46, c_last_name#47, ca_street_number#49, ca_street_name#50, ca_street_type#51, ca_suite_number#52, ca_city#53, ca_county#54, ca_state#55, ca_zip#56, ca_country#57, ca_gmt_offset#58, ca_location_type#59, ctr_total_return#19] ===== Subqueries ===== @@ -352,6 +352,6 @@ ReusedExchange (62) (62) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#23 IN dynamicpruning#5 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 6a8f9e5264d49..04371a7f43d2a 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 @@ -149,154 +149,154 @@ Input [3]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18] Condition : isnotnull(ctr_total_return#18) (21) Scan parquet default.catalog_returns -Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Output [4]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#22), dynamicpruningexpression(cr_returned_date_sk#22 IN dynamicpruning#5)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct (22) ColumnarToRow [codegen id : 6] -Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Input [4]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22] (23) Filter [codegen id : 6] -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) +Input [4]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22] +Condition : isnotnull(cr_returning_addr_sk#20) (24) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#23] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returned_date_sk#4] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#22] +Right keys [1]: [d_date_sk#23] Join condition: None (26) Project [codegen id : 6] -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] +Output [3]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21] +Input [5]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, cr_returned_date_sk#22, d_date_sk#23] (27) ReusedExchange [Reuses operator id: 14] -Output [2]: [ca_address_sk#9, ca_state#10] +Output [2]: [ca_address_sk#24, ca_state#25] (28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cr_returning_addr_sk#2] -Right keys [1]: [ca_address_sk#9] +Left keys [1]: [cr_returning_addr_sk#20] +Right keys [1]: [ca_address_sk#24] Join condition: None (29) Project [codegen id : 6] -Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] -Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#9, ca_state#10] +Output [3]: [cr_returning_customer_sk#19, cr_return_amt_inc_tax#21, ca_state#25] +Input [5]: [cr_returning_customer_sk#19, cr_returning_addr_sk#20, cr_return_amt_inc_tax#21, ca_address_sk#24, ca_state#25] (30) HashAggregate [codegen id : 6] -Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#10] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum#19] -Results [3]: [cr_returning_customer_sk#1, ca_state#10, sum#20] +Input [3]: [cr_returning_customer_sk#19, cr_return_amt_inc_tax#21, ca_state#25] +Keys [2]: [cr_returning_customer_sk#19, ca_state#25] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#21))] +Aggregate Attributes [1]: [sum#26] +Results [3]: [cr_returning_customer_sk#19, ca_state#25, sum#27] (31) Exchange -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#20] -Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#21] +Input [3]: [cr_returning_customer_sk#19, ca_state#25, sum#27] +Arguments: hashpartitioning(cr_returning_customer_sk#19, ca_state#25, 5), ENSURE_REQUIREMENTS, [id=#28] (32) HashAggregate [codegen id : 7] -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#20] -Keys [2]: [cr_returning_customer_sk#1, ca_state#10] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#22] -Results [2]: [ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#22,17,2) AS ctr_total_return#18] +Input [3]: [cr_returning_customer_sk#19, ca_state#25, sum#27] +Keys [2]: [cr_returning_customer_sk#19, ca_state#25] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#21))#29] +Results [2]: [ca_state#25 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#21))#29,17,2) AS ctr_total_return#18] (33) HashAggregate [codegen id : 7] Input [2]: [ctr_state#17, ctr_total_return#18] Keys [1]: [ctr_state#17] Functions [1]: [partial_avg(ctr_total_return#18)] -Aggregate Attributes [2]: [sum#23, count#24] -Results [3]: [ctr_state#17, sum#25, count#26] +Aggregate Attributes [2]: [sum#30, count#31] +Results [3]: [ctr_state#17, sum#32, count#33] (34) Exchange -Input [3]: [ctr_state#17, sum#25, count#26] -Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [ctr_state#17, sum#32, count#33] +Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#34] (35) HashAggregate [codegen id : 8] -Input [3]: [ctr_state#17, sum#25, count#26] +Input [3]: [ctr_state#17, sum#32, count#33] Keys [1]: [ctr_state#17] Functions [1]: [avg(ctr_total_return#18)] -Aggregate Attributes [1]: [avg(ctr_total_return#18)#28] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#28) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#29, ctr_state#17 AS ctr_state#17#30] +Aggregate Attributes [1]: [avg(ctr_total_return#18)#35] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#35) * 1.200000), DecimalType(24,7), true) AS (avg(ctr_total_return) * 1.2)#36, ctr_state#17 AS ctr_state#17#37] (36) Filter [codegen id : 8] -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#29) +Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#36) (37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#31] +Input [2]: [(avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#38] (38) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_state#17] -Right keys [1]: [ctr_state#17#30] -Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#29) +Right keys [1]: [ctr_state#17#37] +Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#36) (39) Project [codegen id : 11] Output [2]: [ctr_customer_sk#16, ctr_total_return#18] -Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#29, ctr_state#17#30] +Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (avg(ctr_total_return) * 1.2)#36, ctr_state#17#37] (40) Scan parquet default.customer -Output [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +Output [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (41) ColumnarToRow [codegen id : 9] -Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +Input [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] (42) Filter [codegen id : 9] -Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] -Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#34)) +Input [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] +Condition : (isnotnull(c_customer_sk#39) AND isnotnull(c_current_addr_sk#41)) (43) BroadcastExchange -Input [6]: [c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#38] +Input [6]: [c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#45] (44) BroadcastHashJoin [codegen id : 11] Left keys [1]: [ctr_customer_sk#16] -Right keys [1]: [c_customer_sk#32] +Right keys [1]: [c_customer_sk#39] Join condition: None (45) Project [codegen id : 11] -Output [6]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] -Input [8]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#32, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37] +Output [6]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] +Input [8]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#39, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44] (46) Scan parquet default.customer_address -Output [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] +Output [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] 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 (47) ColumnarToRow [codegen id : 10] -Input [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] +Input [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] (48) Filter [codegen id : 10] -Input [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] -Condition : ((isnotnull(ca_state#10) AND (ca_state#10 = GA)) AND isnotnull(ca_address_sk#9)) +Input [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] +Condition : ((isnotnull(ca_state#53) AND (ca_state#53 = GA)) AND isnotnull(ca_address_sk#46)) (49) BroadcastExchange -Input [12]: [ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#49] +Input [12]: [ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#58] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#34] -Right keys [1]: [ca_address_sk#9] +Left keys [1]: [c_current_addr_sk#41] +Right keys [1]: [ca_address_sk#46] Join condition: None (51) Project [codegen id : 11] -Output [16]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48, ctr_total_return#18] -Input [18]: [ctr_total_return#18, c_customer_id#33, c_current_addr_sk#34, c_salutation#35, c_first_name#36, c_last_name#37, ca_address_sk#9, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48] +Output [16]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57, ctr_total_return#18] +Input [18]: [ctr_total_return#18, c_customer_id#40, c_current_addr_sk#41, c_salutation#42, c_first_name#43, c_last_name#44, ca_address_sk#46, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57] (52) TakeOrderedAndProject -Input [16]: [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48, ctr_total_return#18] -Arguments: 100, [c_customer_id#33 ASC NULLS FIRST, c_salutation#35 ASC NULLS FIRST, c_first_name#36 ASC NULLS FIRST, c_last_name#37 ASC NULLS FIRST, ca_street_number#39 ASC NULLS FIRST, ca_street_name#40 ASC NULLS FIRST, ca_street_type#41 ASC NULLS FIRST, ca_suite_number#42 ASC NULLS FIRST, ca_city#43 ASC NULLS FIRST, ca_county#44 ASC NULLS FIRST, ca_state#10 ASC NULLS FIRST, ca_zip#45 ASC NULLS FIRST, ca_country#46 ASC NULLS FIRST, ca_gmt_offset#47 ASC NULLS FIRST, ca_location_type#48 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#33, c_salutation#35, c_first_name#36, c_last_name#37, ca_street_number#39, ca_street_name#40, ca_street_type#41, ca_suite_number#42, ca_city#43, ca_county#44, ca_state#10, ca_zip#45, ca_country#46, ca_gmt_offset#47, ca_location_type#48, ctr_total_return#18] +Input [16]: [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57, ctr_total_return#18] +Arguments: 100, [c_customer_id#40 ASC NULLS FIRST, c_salutation#42 ASC NULLS FIRST, c_first_name#43 ASC NULLS FIRST, c_last_name#44 ASC NULLS FIRST, ca_street_number#47 ASC NULLS FIRST, ca_street_name#48 ASC NULLS FIRST, ca_street_type#49 ASC NULLS FIRST, ca_suite_number#50 ASC NULLS FIRST, ca_city#51 ASC NULLS FIRST, ca_county#52 ASC NULLS FIRST, ca_state#53 ASC NULLS FIRST, ca_zip#54 ASC NULLS FIRST, ca_country#55 ASC NULLS FIRST, ca_gmt_offset#56 ASC NULLS FIRST, ca_location_type#57 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#40, c_salutation#42, c_first_name#43, c_last_name#44, ca_street_number#47, ca_street_name#48, ca_street_type#49, ca_suite_number#50, ca_city#51, ca_county#52, ca_state#53, ca_zip#54, ca_country#55, ca_gmt_offset#56, ca_location_type#57, ctr_total_return#18] ===== Subqueries ===== @@ -307,6 +307,6 @@ ReusedExchange (53) (53) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#6] -Subquery:2 Hosting operator id = 21 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 21 Hosting Expression = cr_returned_date_sk#22 IN dynamicpruning#5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt index d006b61d20c33..a7c71a0668676 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt @@ -92,51 +92,51 @@ Input [2]: [d_date_sk#4, d_date#5] Condition : isnotnull(d_date_sk#4) (7) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#6, d_week_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (8) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#6, d_week_seq#7] (9) Scan parquet default.date_dim -Output [2]: [d_date#5, d_week_seq#6] +Output [2]: [d_date#8, d_week_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (10) ColumnarToRow [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] +Input [2]: [d_date#8, d_week_seq#9] (11) Filter [codegen id : 1] -Input [2]: [d_date#5, d_week_seq#6] -Condition : cast(d_date#5 as string) IN (2000-06-30,2000-09-27,2000-11-17) +Input [2]: [d_date#8, d_week_seq#9] +Condition : cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) (12) Project [codegen id : 1] -Output [1]: [d_week_seq#6 AS d_week_seq#6#7] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_week_seq#9] +Input [2]: [d_date#8, d_week_seq#9] (13) BroadcastExchange -Input [1]: [d_week_seq#6#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] +Input [1]: [d_week_seq#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#10] (14) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [d_week_seq#6] -Right keys [1]: [d_week_seq#6#7] +Left keys [1]: [d_week_seq#7] +Right keys [1]: [d_week_seq#9] Join condition: None (15) Project [codegen id : 2] -Output [1]: [d_date#5 AS d_date#5#9] -Input [2]: [d_date#5, d_week_seq#6] +Output [1]: [d_date#6] +Input [2]: [d_date#6, d_week_seq#7] (16) BroadcastExchange -Input [1]: [d_date#5#9] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#10] +Input [1]: [d_date#6] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#11] (17) BroadcastHashJoin [codegen id : 3] Left keys [1]: [d_date#5] -Right keys [1]: [d_date#5#9] +Right keys [1]: [d_date#6] Join condition: None (18) Project [codegen id : 3] @@ -145,7 +145,7 @@ Input [2]: [d_date_sk#4, d_date#5] (19) BroadcastExchange Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (20) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] @@ -157,191 +157,191 @@ Output [2]: [sr_item_sk#1, sr_return_quantity#2] Input [4]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, d_date_sk#4] (22) Scan parquet default.item -Output [2]: [i_item_sk#12, i_item_id#13] +Output [2]: [i_item_sk#13, i_item_id#14] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] ReadSchema: struct (23) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#12, i_item_id#13] +Input [2]: [i_item_sk#13, i_item_id#14] (24) Filter [codegen id : 4] -Input [2]: [i_item_sk#12, i_item_id#13] -Condition : (isnotnull(i_item_sk#12) AND isnotnull(i_item_id#13)) +Input [2]: [i_item_sk#13, i_item_id#14] +Condition : (isnotnull(i_item_sk#13) AND isnotnull(i_item_id#14)) (25) BroadcastExchange -Input [2]: [i_item_sk#12, i_item_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] +Input [2]: [i_item_sk#13, i_item_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] (26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_item_sk#1] -Right keys [1]: [cast(i_item_sk#12 as bigint)] +Right keys [1]: [cast(i_item_sk#13 as bigint)] Join condition: None (27) Project [codegen id : 5] -Output [2]: [sr_return_quantity#2, i_item_id#13] -Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#12, i_item_id#13] +Output [2]: [sr_return_quantity#2, i_item_id#14] +Input [4]: [sr_item_sk#1, sr_return_quantity#2, i_item_sk#13, i_item_id#14] (28) HashAggregate [codegen id : 5] -Input [2]: [sr_return_quantity#2, i_item_id#13] -Keys [1]: [i_item_id#13] +Input [2]: [sr_return_quantity#2, i_item_id#14] +Keys [1]: [i_item_id#14] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#13, sum#16] +Aggregate Attributes [1]: [sum#16] +Results [2]: [i_item_id#14, sum#17] (29) Exchange -Input [2]: [i_item_id#13, sum#16] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [2]: [i_item_id#14, sum#17] +Arguments: hashpartitioning(i_item_id#14, 5), ENSURE_REQUIREMENTS, [id=#18] (30) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#13, sum#16] -Keys [1]: [i_item_id#13] +Input [2]: [i_item_id#14, sum#17] +Keys [1]: [i_item_id#14] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#18] -Results [2]: [i_item_id#13 AS item_id#19, sum(sr_return_quantity#2)#18 AS sr_item_qty#20] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] +Results [2]: [i_item_id#14 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] (31) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] (33) Filter [codegen id : 10] -Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] -Condition : isnotnull(cr_item_sk#21) +Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +Condition : isnotnull(cr_item_sk#22) (34) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#4] +Output [1]: [d_date_sk#25] (35) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#23] -Right keys [1]: [d_date_sk#4] +Left keys [1]: [cr_returned_date_sk#24] +Right keys [1]: [d_date_sk#25] Join condition: None (36) Project [codegen id : 10] -Output [2]: [cr_item_sk#21, cr_return_quantity#22] -Input [4]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, d_date_sk#4] +Output [2]: [cr_item_sk#22, cr_return_quantity#23] +Input [4]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, d_date_sk#25] (37) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#12, i_item_id#13] +Output [2]: [i_item_sk#26, i_item_id#27] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#21] -Right keys [1]: [i_item_sk#12] +Left keys [1]: [cr_item_sk#22] +Right keys [1]: [i_item_sk#26] Join condition: None (39) Project [codegen id : 10] -Output [2]: [cr_return_quantity#22, i_item_id#13] -Input [4]: [cr_item_sk#21, cr_return_quantity#22, i_item_sk#12, i_item_id#13] +Output [2]: [cr_return_quantity#23, i_item_id#27] +Input [4]: [cr_item_sk#22, cr_return_quantity#23, i_item_sk#26, i_item_id#27] (40) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#22, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum#24] -Results [2]: [i_item_id#13, sum#25] +Input [2]: [cr_return_quantity#23, i_item_id#27] +Keys [1]: [i_item_id#27] +Functions [1]: [partial_sum(cr_return_quantity#23)] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_item_id#27, sum#29] (41) Exchange -Input [2]: [i_item_id#13, sum#25] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [i_item_id#27, sum#29] +Arguments: hashpartitioning(i_item_id#27, 5), ENSURE_REQUIREMENTS, [id=#30] (42) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#13, sum#25] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum(cr_return_quantity#22)#27] -Results [2]: [i_item_id#13 AS item_id#28, sum(cr_return_quantity#22)#27 AS cr_item_qty#29] +Input [2]: [i_item_id#27, sum#29] +Keys [1]: [i_item_id#27] +Functions [1]: [sum(cr_return_quantity#23)] +Aggregate Attributes [1]: [sum(cr_return_quantity#23)#31] +Results [2]: [i_item_id#27 AS item_id#32, sum(cr_return_quantity#23)#31 AS cr_item_qty#33] (43) BroadcastExchange -Input [2]: [item_id#28, cr_item_qty#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] +Input [2]: [item_id#32, cr_item_qty#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] (44) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#28] +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#32] Join condition: None (45) Project [codegen id : 18] -Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] -Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] +Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#33] +Input [4]: [item_id#20, sr_item_qty#21, item_id#32, cr_item_qty#33] (46) Scan parquet default.web_returns -Output [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +Output [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#33), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(wr_returned_date_sk#37), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] (48) Filter [codegen id : 16] -Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] -Condition : isnotnull(wr_item_sk#31) +Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +Condition : isnotnull(wr_item_sk#35) (49) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#4] +Output [1]: [d_date_sk#38] (50) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#33] -Right keys [1]: [cast(d_date_sk#4 as bigint)] +Left keys [1]: [wr_returned_date_sk#37] +Right keys [1]: [cast(d_date_sk#38 as bigint)] Join condition: None (51) Project [codegen id : 16] -Output [2]: [wr_item_sk#31, wr_return_quantity#32] -Input [4]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33, d_date_sk#4] +Output [2]: [wr_item_sk#35, wr_return_quantity#36] +Input [4]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37, d_date_sk#38] (52) ReusedExchange [Reuses operator id: 25] -Output [2]: [i_item_sk#12, i_item_id#13] +Output [2]: [i_item_sk#39, i_item_id#40] (53) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#31] -Right keys [1]: [cast(i_item_sk#12 as bigint)] +Left keys [1]: [wr_item_sk#35] +Right keys [1]: [cast(i_item_sk#39 as bigint)] Join condition: None (54) Project [codegen id : 16] -Output [2]: [wr_return_quantity#32, i_item_id#13] -Input [4]: [wr_item_sk#31, wr_return_quantity#32, i_item_sk#12, i_item_id#13] +Output [2]: [wr_return_quantity#36, i_item_id#40] +Input [4]: [wr_item_sk#35, wr_return_quantity#36, i_item_sk#39, i_item_id#40] (55) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#32, i_item_id#13] -Keys [1]: [i_item_id#13] -Functions [1]: [partial_sum(wr_return_quantity#32)] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#13, sum#35] +Input [2]: [wr_return_quantity#36, i_item_id#40] +Keys [1]: [i_item_id#40] +Functions [1]: [partial_sum(wr_return_quantity#36)] +Aggregate Attributes [1]: [sum#41] +Results [2]: [i_item_id#40, sum#42] (56) Exchange -Input [2]: [i_item_id#13, sum#35] -Arguments: hashpartitioning(i_item_id#13, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [2]: [i_item_id#40, sum#42] +Arguments: hashpartitioning(i_item_id#40, 5), ENSURE_REQUIREMENTS, [id=#43] (57) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#13, sum#35] -Keys [1]: [i_item_id#13] -Functions [1]: [sum(wr_return_quantity#32)] -Aggregate Attributes [1]: [sum(wr_return_quantity#32)#37] -Results [2]: [i_item_id#13 AS item_id#38, sum(wr_return_quantity#32)#37 AS wr_item_qty#39] +Input [2]: [i_item_id#40, sum#42] +Keys [1]: [i_item_id#40] +Functions [1]: [sum(wr_return_quantity#36)] +Aggregate Attributes [1]: [sum(wr_return_quantity#36)#44] +Results [2]: [i_item_id#40 AS item_id#45, sum(wr_return_quantity#36)#44 AS wr_item_qty#46] (58) BroadcastExchange -Input [2]: [item_id#38, wr_item_qty#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] +Input [2]: [item_id#45, wr_item_qty#46] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#47] (59) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#38] +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#45] Join condition: None (60) Project [codegen id : 18] -Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] -Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] +Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS sr_dev#48, cr_item_qty#33, (((cast(cr_item_qty#33 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS cr_dev#49, wr_item_qty#46, (((cast(wr_item_qty#46 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS wr_dev#50, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#51] +Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#33, item_id#45, wr_item_qty#46] (61) TakeOrderedAndProject -Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] -Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +Input [8]: [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] +Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index c926ab7e3ccb6..fd73cee998f71 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -119,51 +119,51 @@ Input [2]: [d_date_sk#7, d_date#8] Condition : isnotnull(d_date_sk#7) (13) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#9, d_week_seq#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (14) ColumnarToRow [codegen id : 3] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#9, d_week_seq#10] (15) Scan parquet default.date_dim -Output [2]: [d_date#8, d_week_seq#9] +Output [2]: [d_date#11, d_week_seq#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] ReadSchema: struct (16) ColumnarToRow [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] +Input [2]: [d_date#11, d_week_seq#12] (17) Filter [codegen id : 2] -Input [2]: [d_date#8, d_week_seq#9] -Condition : cast(d_date#8 as string) IN (2000-06-30,2000-09-27,2000-11-17) +Input [2]: [d_date#11, d_week_seq#12] +Condition : cast(d_date#11 as string) IN (2000-06-30,2000-09-27,2000-11-17) (18) Project [codegen id : 2] -Output [1]: [d_week_seq#9 AS d_week_seq#9#10] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_week_seq#12] +Input [2]: [d_date#11, d_week_seq#12] (19) BroadcastExchange -Input [1]: [d_week_seq#9#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] +Input [1]: [d_week_seq#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] (20) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_week_seq#9] -Right keys [1]: [d_week_seq#9#10] +Left keys [1]: [d_week_seq#10] +Right keys [1]: [d_week_seq#12] Join condition: None (21) Project [codegen id : 3] -Output [1]: [d_date#8 AS d_date#8#12] -Input [2]: [d_date#8, d_week_seq#9] +Output [1]: [d_date#9] +Input [2]: [d_date#9, d_week_seq#10] (22) BroadcastExchange -Input [1]: [d_date#8#12] -Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#13] +Input [1]: [d_date#9] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [id=#14] (23) BroadcastHashJoin [codegen id : 4] Left keys [1]: [d_date#8] -Right keys [1]: [d_date#8#12] +Right keys [1]: [d_date#9] Join condition: None (24) Project [codegen id : 4] @@ -172,7 +172,7 @@ Input [2]: [d_date_sk#7, d_date#8] (25) BroadcastExchange Input [1]: [d_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (26) BroadcastHashJoin [codegen id : 5] Left keys [1]: [sr_returned_date_sk#3] @@ -187,161 +187,161 @@ Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#5, d_date_sk# Input [2]: [sr_return_quantity#2, i_item_id#5] Keys [1]: [i_item_id#5] Functions [1]: [partial_sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum#15] -Results [2]: [i_item_id#5, sum#16] +Aggregate Attributes [1]: [sum#16] +Results [2]: [i_item_id#5, sum#17] (29) Exchange -Input [2]: [i_item_id#5, sum#16] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [2]: [i_item_id#5, sum#17] +Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#18] (30) HashAggregate [codegen id : 18] -Input [2]: [i_item_id#5, sum#16] +Input [2]: [i_item_id#5, sum#17] Keys [1]: [i_item_id#5] Functions [1]: [sum(sr_return_quantity#2)] -Aggregate Attributes [1]: [sum(sr_return_quantity#2)#18] -Results [2]: [i_item_id#5 AS item_id#19, sum(sr_return_quantity#2)#18 AS sr_item_qty#20] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#19] +Results [2]: [i_item_id#5 AS item_id#20, sum(sr_return_quantity#2)#19 AS sr_item_qty#21] (31) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Output [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#23), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(cr_returned_date_sk#24), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(cr_item_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 10] -Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] +Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] (33) Filter [codegen id : 10] -Input [3]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23] -Condition : isnotnull(cr_item_sk#21) +Input [3]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24] +Condition : isnotnull(cr_item_sk#22) (34) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#25, i_item_id#26] (35) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_item_sk#21] -Right keys [1]: [i_item_sk#4] +Left keys [1]: [cr_item_sk#22] +Right keys [1]: [i_item_sk#25] Join condition: None (36) Project [codegen id : 10] -Output [3]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#5] -Input [5]: [cr_item_sk#21, cr_return_quantity#22, cr_returned_date_sk#23, i_item_sk#4, i_item_id#5] +Output [3]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#26] +Input [5]: [cr_item_sk#22, cr_return_quantity#23, cr_returned_date_sk#24, i_item_sk#25, i_item_id#26] (37) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#27] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cr_returned_date_sk#23] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [cr_returned_date_sk#24] +Right keys [1]: [d_date_sk#27] Join condition: None (39) Project [codegen id : 10] -Output [2]: [cr_return_quantity#22, i_item_id#5] -Input [4]: [cr_return_quantity#22, cr_returned_date_sk#23, i_item_id#5, d_date_sk#7] +Output [2]: [cr_return_quantity#23, i_item_id#26] +Input [4]: [cr_return_quantity#23, cr_returned_date_sk#24, i_item_id#26, d_date_sk#27] (40) HashAggregate [codegen id : 10] -Input [2]: [cr_return_quantity#22, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum#24] -Results [2]: [i_item_id#5, sum#25] +Input [2]: [cr_return_quantity#23, i_item_id#26] +Keys [1]: [i_item_id#26] +Functions [1]: [partial_sum(cr_return_quantity#23)] +Aggregate Attributes [1]: [sum#28] +Results [2]: [i_item_id#26, sum#29] (41) Exchange -Input [2]: [i_item_id#5, sum#25] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [2]: [i_item_id#26, sum#29] +Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, [id=#30] (42) HashAggregate [codegen id : 11] -Input [2]: [i_item_id#5, sum#25] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(cr_return_quantity#22)] -Aggregate Attributes [1]: [sum(cr_return_quantity#22)#27] -Results [2]: [i_item_id#5 AS item_id#28, sum(cr_return_quantity#22)#27 AS cr_item_qty#29] +Input [2]: [i_item_id#26, sum#29] +Keys [1]: [i_item_id#26] +Functions [1]: [sum(cr_return_quantity#23)] +Aggregate Attributes [1]: [sum(cr_return_quantity#23)#31] +Results [2]: [i_item_id#26 AS item_id#32, sum(cr_return_quantity#23)#31 AS cr_item_qty#33] (43) BroadcastExchange -Input [2]: [item_id#28, cr_item_qty#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] +Input [2]: [item_id#32, cr_item_qty#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#34] (44) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#28] +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#32] Join condition: None (45) Project [codegen id : 18] -Output [3]: [item_id#19, sr_item_qty#20, cr_item_qty#29] -Input [4]: [item_id#19, sr_item_qty#20, item_id#28, cr_item_qty#29] +Output [3]: [item_id#20, sr_item_qty#21, cr_item_qty#33] +Input [4]: [item_id#20, sr_item_qty#21, item_id#32, cr_item_qty#33] (46) Scan parquet default.web_returns -Output [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +Output [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#33), dynamicpruningexpression(true)] +PartitionFilters: [isnotnull(wr_returned_date_sk#37), dynamicpruningexpression(true)] PushedFilters: [IsNotNull(wr_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 16] -Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] +Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] (48) Filter [codegen id : 16] -Input [3]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33] -Condition : isnotnull(wr_item_sk#31) +Input [3]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37] +Condition : isnotnull(wr_item_sk#35) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [i_item_sk#4, i_item_id#5] +Output [2]: [i_item_sk#38, i_item_id#39] (50) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_item_sk#31] -Right keys [1]: [cast(i_item_sk#4 as bigint)] +Left keys [1]: [wr_item_sk#35] +Right keys [1]: [cast(i_item_sk#38 as bigint)] Join condition: None (51) Project [codegen id : 16] -Output [3]: [wr_return_quantity#32, wr_returned_date_sk#33, i_item_id#5] -Input [5]: [wr_item_sk#31, wr_return_quantity#32, wr_returned_date_sk#33, i_item_sk#4, i_item_id#5] +Output [3]: [wr_return_quantity#36, wr_returned_date_sk#37, i_item_id#39] +Input [5]: [wr_item_sk#35, wr_return_quantity#36, wr_returned_date_sk#37, i_item_sk#38, i_item_id#39] (52) ReusedExchange [Reuses operator id: 25] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#40] (53) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [wr_returned_date_sk#33] -Right keys [1]: [cast(d_date_sk#7 as bigint)] +Left keys [1]: [wr_returned_date_sk#37] +Right keys [1]: [cast(d_date_sk#40 as bigint)] Join condition: None (54) Project [codegen id : 16] -Output [2]: [wr_return_quantity#32, i_item_id#5] -Input [4]: [wr_return_quantity#32, wr_returned_date_sk#33, i_item_id#5, d_date_sk#7] +Output [2]: [wr_return_quantity#36, i_item_id#39] +Input [4]: [wr_return_quantity#36, wr_returned_date_sk#37, i_item_id#39, d_date_sk#40] (55) HashAggregate [codegen id : 16] -Input [2]: [wr_return_quantity#32, i_item_id#5] -Keys [1]: [i_item_id#5] -Functions [1]: [partial_sum(wr_return_quantity#32)] -Aggregate Attributes [1]: [sum#34] -Results [2]: [i_item_id#5, sum#35] +Input [2]: [wr_return_quantity#36, i_item_id#39] +Keys [1]: [i_item_id#39] +Functions [1]: [partial_sum(wr_return_quantity#36)] +Aggregate Attributes [1]: [sum#41] +Results [2]: [i_item_id#39, sum#42] (56) Exchange -Input [2]: [i_item_id#5, sum#35] -Arguments: hashpartitioning(i_item_id#5, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [2]: [i_item_id#39, sum#42] +Arguments: hashpartitioning(i_item_id#39, 5), ENSURE_REQUIREMENTS, [id=#43] (57) HashAggregate [codegen id : 17] -Input [2]: [i_item_id#5, sum#35] -Keys [1]: [i_item_id#5] -Functions [1]: [sum(wr_return_quantity#32)] -Aggregate Attributes [1]: [sum(wr_return_quantity#32)#37] -Results [2]: [i_item_id#5 AS item_id#38, sum(wr_return_quantity#32)#37 AS wr_item_qty#39] +Input [2]: [i_item_id#39, sum#42] +Keys [1]: [i_item_id#39] +Functions [1]: [sum(wr_return_quantity#36)] +Aggregate Attributes [1]: [sum(wr_return_quantity#36)#44] +Results [2]: [i_item_id#39 AS item_id#45, sum(wr_return_quantity#36)#44 AS wr_item_qty#46] (58) BroadcastExchange -Input [2]: [item_id#38, wr_item_qty#39] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#40] +Input [2]: [item_id#45, wr_item_qty#46] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#47] (59) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [item_id#19] -Right keys [1]: [item_id#38] +Left keys [1]: [item_id#20] +Right keys [1]: [item_id#45] Join condition: None (60) Project [codegen id : 18] -Output [8]: [item_id#19, sr_item_qty#20, (((cast(sr_item_qty#20 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS sr_dev#41, cr_item_qty#29, (((cast(cr_item_qty#29 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS cr_dev#42, wr_item_qty#39, (((cast(wr_item_qty#39 as double) / cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as double)) / 3.0) * 100.0) AS wr_dev#43, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#20 + cr_item_qty#29) + wr_item_qty#39) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#44] -Input [5]: [item_id#19, sr_item_qty#20, cr_item_qty#29, item_id#38, wr_item_qty#39] +Output [8]: [item_id#20, sr_item_qty#21, (((cast(sr_item_qty#21 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS sr_dev#48, cr_item_qty#33, (((cast(cr_item_qty#33 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS cr_dev#49, wr_item_qty#46, (((cast(wr_item_qty#46 as double) / cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as double)) / 3.0) * 100.0) AS wr_dev#50, CheckOverflow((promote_precision(cast(cast(((sr_item_qty#21 + cr_item_qty#33) + wr_item_qty#46) as decimal(20,0)) as decimal(21,1))) / 3.0), DecimalType(27,6), true) AS average#51] +Input [5]: [item_id#20, sr_item_qty#21, cr_item_qty#33, item_id#45, wr_item_qty#46] (61) TakeOrderedAndProject -Input [8]: [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] -Arguments: 100, [item_id#19 ASC NULLS FIRST, sr_item_qty#20 ASC NULLS FIRST], [item_id#19, sr_item_qty#20, sr_dev#41, cr_item_qty#29, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +Input [8]: [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] +Arguments: 100, [item_id#20 ASC NULLS FIRST, sr_item_qty#21 ASC NULLS FIRST], [item_id#20, sr_item_qty#21, sr_dev#48, cr_item_qty#33, cr_dev#49, wr_item_qty#46, wr_dev#50, average#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt index f9d73c29db3d8..e72928545d080 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.sf100/explain.txt @@ -313,687 +313,687 @@ Aggregate Attributes [1]: [count(1)#19] Results [1]: [count(1)#19 AS h8_30_to_9#20] (29) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] (31) Filter [codegen id : 8] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Condition : ((isnotnull(ss_hdemo_sk#22) AND isnotnull(ss_sold_time_sk#21)) AND isnotnull(ss_store_sk#23)) (32) Project [codegen id : 8] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] (33) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#25, t_hour#26, t_minute#27] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#25, t_hour#26, t_minute#27] (35) Filter [codegen id : 5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 9)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#25, t_hour#26, t_minute#27] +Condition : ((((isnotnull(t_hour#26) AND isnotnull(t_minute#27)) AND (t_hour#26 = 9)) AND (t_minute#27 < 30)) AND isnotnull(t_time_sk#25)) (36) Project [codegen id : 5] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#25] +Input [3]: [t_time_sk#25, t_hour#26, t_minute#27] (37) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [t_time_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] (38) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#21] +Right keys [1]: [t_time_sk#25] Join condition: None (39) Project [codegen id : 8] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#22, ss_store_sk#23] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, t_time_sk#25] (40) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#29] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#23] +Right keys [1]: [s_store_sk#29] Join condition: None (42) Project [codegen id : 8] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#22] +Input [3]: [ss_hdemo_sk#22, ss_store_sk#23, s_store_sk#29] (43) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#30] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#22] +Right keys [1]: [hd_demo_sk#30] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#22, hd_demo_sk#30] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#22] -Results [1]: [count#23] +Aggregate Attributes [1]: [count#31] +Results [1]: [count#32] (47) Exchange -Input [1]: [count#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [1]: [count#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] (48) HashAggregate [codegen id : 9] -Input [1]: [count#23] +Input [1]: [count#32] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#25] -Results [1]: [count(1)#25 AS h9_to_9_30#26] +Aggregate Attributes [1]: [count(1)#34] +Results [1]: [count(1)#34 AS h9_to_9_30#35] (49) BroadcastExchange -Input [1]: [h9_to_9_30#26] -Arguments: IdentityBroadcastMode, [id=#27] +Input [1]: [h9_to_9_30#35] +Arguments: IdentityBroadcastMode, [id=#36] (50) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (51) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 13] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] (53) Filter [codegen id : 13] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) (54) Project [codegen id : 13] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] (55) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#41, t_hour#42, t_minute#43] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 10] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#41, t_hour#42, t_minute#43] (57) Filter [codegen id : 10] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 9)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#41, t_hour#42, t_minute#43] +Condition : ((((isnotnull(t_hour#42) AND isnotnull(t_minute#43)) AND (t_hour#42 = 9)) AND (t_minute#43 >= 30)) AND isnotnull(t_time_sk#41)) (58) Project [codegen id : 10] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#41] +Input [3]: [t_time_sk#41, t_hour#42, t_minute#43] (59) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [t_time_sk#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] (60) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#37] +Right keys [1]: [t_time_sk#41] Join condition: None (61) Project [codegen id : 13] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#38, ss_store_sk#39] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, t_time_sk#41] (62) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#45] (63) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#39] +Right keys [1]: [s_store_sk#45] Join condition: None (64) Project [codegen id : 13] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#38] +Input [3]: [ss_hdemo_sk#38, ss_store_sk#39, s_store_sk#45] (65) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#46] (66) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#38] +Right keys [1]: [hd_demo_sk#46] Join condition: None (67) Project [codegen id : 13] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#38, hd_demo_sk#46] (68) HashAggregate [codegen id : 13] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [1]: [count#30] +Aggregate Attributes [1]: [count#47] +Results [1]: [count#48] (69) Exchange -Input [1]: [count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] +Input [1]: [count#48] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] (70) HashAggregate [codegen id : 14] -Input [1]: [count#30] +Input [1]: [count#48] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [1]: [count(1)#32 AS h9_30_to_10#33] +Aggregate Attributes [1]: [count(1)#50] +Results [1]: [count(1)#50 AS h9_30_to_10#51] (71) BroadcastExchange -Input [1]: [h9_30_to_10#33] -Arguments: IdentityBroadcastMode, [id=#34] +Input [1]: [h9_30_to_10#51] +Arguments: IdentityBroadcastMode, [id=#52] (72) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (73) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 18] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] (75) Filter [codegen id : 18] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Condition : ((isnotnull(ss_hdemo_sk#54) AND isnotnull(ss_sold_time_sk#53)) AND isnotnull(ss_store_sk#55)) (76) Project [codegen id : 18] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] (77) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#57, t_hour#58, t_minute#59] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (78) ColumnarToRow [codegen id : 15] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] (79) Filter [codegen id : 15] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 10)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] +Condition : ((((isnotnull(t_hour#58) AND isnotnull(t_minute#59)) AND (t_hour#58 = 10)) AND (t_minute#59 < 30)) AND isnotnull(t_time_sk#57)) (80) Project [codegen id : 15] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#57] +Input [3]: [t_time_sk#57, t_hour#58, t_minute#59] (81) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [1]: [t_time_sk#57] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#60] (82) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#53] +Right keys [1]: [t_time_sk#57] Join condition: None (83) Project [codegen id : 18] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#54, ss_store_sk#55] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, t_time_sk#57] (84) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#61] (85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#55] +Right keys [1]: [s_store_sk#61] Join condition: None (86) Project [codegen id : 18] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#54] +Input [3]: [ss_hdemo_sk#54, ss_store_sk#55, s_store_sk#61] (87) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#62] (88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#54] +Right keys [1]: [hd_demo_sk#62] Join condition: None (89) Project [codegen id : 18] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#54, hd_demo_sk#62] (90) HashAggregate [codegen id : 18] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#36] -Results [1]: [count#37] +Aggregate Attributes [1]: [count#63] +Results [1]: [count#64] (91) Exchange -Input [1]: [count#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#38] +Input [1]: [count#64] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] (92) HashAggregate [codegen id : 19] -Input [1]: [count#37] +Input [1]: [count#64] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#39] -Results [1]: [count(1)#39 AS h10_to_10_30#40] +Aggregate Attributes [1]: [count(1)#66] +Results [1]: [count(1)#66 AS h10_to_10_30#67] (93) BroadcastExchange -Input [1]: [h10_to_10_30#40] -Arguments: IdentityBroadcastMode, [id=#41] +Input [1]: [h10_to_10_30#67] +Arguments: IdentityBroadcastMode, [id=#68] (94) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (95) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (96) ColumnarToRow [codegen id : 23] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] (97) Filter [codegen id : 23] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Condition : ((isnotnull(ss_hdemo_sk#70) AND isnotnull(ss_sold_time_sk#69)) AND isnotnull(ss_store_sk#71)) (98) Project [codegen id : 23] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] (99) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#73, t_hour#74, t_minute#75] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (100) ColumnarToRow [codegen id : 20] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#73, t_hour#74, t_minute#75] (101) Filter [codegen id : 20] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 10)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#73, t_hour#74, t_minute#75] +Condition : ((((isnotnull(t_hour#74) AND isnotnull(t_minute#75)) AND (t_hour#74 = 10)) AND (t_minute#75 >= 30)) AND isnotnull(t_time_sk#73)) (102) Project [codegen id : 20] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#73] +Input [3]: [t_time_sk#73, t_hour#74, t_minute#75] (103) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [t_time_sk#73] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#76] (104) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#69] +Right keys [1]: [t_time_sk#73] Join condition: None (105) Project [codegen id : 23] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#70, ss_store_sk#71] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, t_time_sk#73] (106) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#77] (107) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#71] +Right keys [1]: [s_store_sk#77] Join condition: None (108) Project [codegen id : 23] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#70] +Input [3]: [ss_hdemo_sk#70, ss_store_sk#71, s_store_sk#77] (109) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#78] (110) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#70] +Right keys [1]: [hd_demo_sk#78] Join condition: None (111) Project [codegen id : 23] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#70, hd_demo_sk#78] (112) HashAggregate [codegen id : 23] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#43] -Results [1]: [count#44] +Aggregate Attributes [1]: [count#79] +Results [1]: [count#80] (113) Exchange -Input [1]: [count#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45] +Input [1]: [count#80] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] (114) HashAggregate [codegen id : 24] -Input [1]: [count#44] +Input [1]: [count#80] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#46] -Results [1]: [count(1)#46 AS h10_30_to_11#47] +Aggregate Attributes [1]: [count(1)#82] +Results [1]: [count(1)#82 AS h10_30_to_11#83] (115) BroadcastExchange -Input [1]: [h10_30_to_11#47] -Arguments: IdentityBroadcastMode, [id=#48] +Input [1]: [h10_30_to_11#83] +Arguments: IdentityBroadcastMode, [id=#84] (116) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (117) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (118) ColumnarToRow [codegen id : 28] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] (119) Filter [codegen id : 28] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Condition : ((isnotnull(ss_hdemo_sk#86) AND isnotnull(ss_sold_time_sk#85)) AND isnotnull(ss_store_sk#87)) (120) Project [codegen id : 28] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] (121) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#89, t_hour#90, t_minute#91] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (122) ColumnarToRow [codegen id : 25] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#89, t_hour#90, t_minute#91] (123) Filter [codegen id : 25] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 11)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#89, t_hour#90, t_minute#91] +Condition : ((((isnotnull(t_hour#90) AND isnotnull(t_minute#91)) AND (t_hour#90 = 11)) AND (t_minute#91 < 30)) AND isnotnull(t_time_sk#89)) (124) Project [codegen id : 25] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#89] +Input [3]: [t_time_sk#89, t_hour#90, t_minute#91] (125) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] +Input [1]: [t_time_sk#89] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#92] (126) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#85] +Right keys [1]: [t_time_sk#89] Join condition: None (127) Project [codegen id : 28] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#86, ss_store_sk#87] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, t_time_sk#89] (128) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#93] (129) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#87] +Right keys [1]: [s_store_sk#93] Join condition: None (130) Project [codegen id : 28] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#86] +Input [3]: [ss_hdemo_sk#86, ss_store_sk#87, s_store_sk#93] (131) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#94] (132) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#86] +Right keys [1]: [hd_demo_sk#94] Join condition: None (133) Project [codegen id : 28] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#86, hd_demo_sk#94] (134) HashAggregate [codegen id : 28] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#50] -Results [1]: [count#51] +Aggregate Attributes [1]: [count#95] +Results [1]: [count#96] (135) Exchange -Input [1]: [count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] +Input [1]: [count#96] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#97] (136) HashAggregate [codegen id : 29] -Input [1]: [count#51] +Input [1]: [count#96] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#53] -Results [1]: [count(1)#53 AS h11_to_11_30#54] +Aggregate Attributes [1]: [count(1)#98] +Results [1]: [count(1)#98 AS h11_to_11_30#99] (137) BroadcastExchange -Input [1]: [h11_to_11_30#54] -Arguments: IdentityBroadcastMode, [id=#55] +Input [1]: [h11_to_11_30#99] +Arguments: IdentityBroadcastMode, [id=#100] (138) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (139) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (140) ColumnarToRow [codegen id : 33] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] (141) Filter [codegen id : 33] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Condition : ((isnotnull(ss_hdemo_sk#102) AND isnotnull(ss_sold_time_sk#101)) AND isnotnull(ss_store_sk#103)) (142) Project [codegen id : 33] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] (143) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#105, t_hour#106, t_minute#107] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (144) ColumnarToRow [codegen id : 30] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#105, t_hour#106, t_minute#107] (145) Filter [codegen id : 30] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 11)) AND (t_minute#7 >= 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#105, t_hour#106, t_minute#107] +Condition : ((((isnotnull(t_hour#106) AND isnotnull(t_minute#107)) AND (t_hour#106 = 11)) AND (t_minute#107 >= 30)) AND isnotnull(t_time_sk#105)) (146) Project [codegen id : 30] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#105] +Input [3]: [t_time_sk#105, t_hour#106, t_minute#107] (147) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +Input [1]: [t_time_sk#105] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#108] (148) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#101] +Right keys [1]: [t_time_sk#105] Join condition: None (149) Project [codegen id : 33] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#102, ss_store_sk#103] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, t_time_sk#105] (150) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#109] (151) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#103] +Right keys [1]: [s_store_sk#109] Join condition: None (152) Project [codegen id : 33] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#102] +Input [3]: [ss_hdemo_sk#102, ss_store_sk#103, s_store_sk#109] (153) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#110] (154) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#102] +Right keys [1]: [hd_demo_sk#110] Join condition: None (155) Project [codegen id : 33] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#102, hd_demo_sk#110] (156) HashAggregate [codegen id : 33] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#57] -Results [1]: [count#58] +Aggregate Attributes [1]: [count#111] +Results [1]: [count#112] (157) Exchange -Input [1]: [count#58] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#59] +Input [1]: [count#112] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] (158) HashAggregate [codegen id : 34] -Input [1]: [count#58] +Input [1]: [count#112] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#60] -Results [1]: [count(1)#60 AS h11_30_to_12#61] +Aggregate Attributes [1]: [count(1)#114] +Results [1]: [count(1)#114 AS h11_30_to_12#115] (159) BroadcastExchange -Input [1]: [h11_30_to_12#61] -Arguments: IdentityBroadcastMode, [id=#62] +Input [1]: [h11_30_to_12#115] +Arguments: IdentityBroadcastMode, [id=#116] (160) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (161) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (162) ColumnarToRow [codegen id : 38] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] (163) Filter [codegen id : 38] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Condition : ((isnotnull(ss_hdemo_sk#118) AND isnotnull(ss_sold_time_sk#117)) AND isnotnull(ss_store_sk#119)) (164) Project [codegen id : 38] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] (165) Scan parquet default.time_dim -Output [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [3]: [t_time_sk#121, t_hour#122, t_minute#123] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (166) ColumnarToRow [codegen id : 35] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Input [3]: [t_time_sk#121, t_hour#122, t_minute#123] (167) Filter [codegen id : 35] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] -Condition : ((((isnotnull(t_hour#6) AND isnotnull(t_minute#7)) AND (t_hour#6 = 12)) AND (t_minute#7 < 30)) AND isnotnull(t_time_sk#5)) +Input [3]: [t_time_sk#121, t_hour#122, t_minute#123] +Condition : ((((isnotnull(t_hour#122) AND isnotnull(t_minute#123)) AND (t_hour#122 = 12)) AND (t_minute#123 < 30)) AND isnotnull(t_time_sk#121)) (168) Project [codegen id : 35] -Output [1]: [t_time_sk#5] -Input [3]: [t_time_sk#5, t_hour#6, t_minute#7] +Output [1]: [t_time_sk#121] +Input [3]: [t_time_sk#121, t_hour#122, t_minute#123] (169) BroadcastExchange -Input [1]: [t_time_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] +Input [1]: [t_time_sk#121] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#124] (170) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#5] +Left keys [1]: [ss_sold_time_sk#117] +Right keys [1]: [t_time_sk#121] Join condition: None (171) Project [codegen id : 38] -Output [2]: [ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, t_time_sk#5] +Output [2]: [ss_hdemo_sk#118, ss_store_sk#119] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, t_time_sk#121] (172) ReusedExchange [Reuses operator id: 16] -Output [1]: [s_store_sk#9] +Output [1]: [s_store_sk#125] (173) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#9] +Left keys [1]: [ss_store_sk#119] +Right keys [1]: [s_store_sk#125] Join condition: None (174) Project [codegen id : 38] -Output [1]: [ss_hdemo_sk#2] -Input [3]: [ss_hdemo_sk#2, ss_store_sk#3, s_store_sk#9] +Output [1]: [ss_hdemo_sk#118] +Input [3]: [ss_hdemo_sk#118, ss_store_sk#119, s_store_sk#125] (175) ReusedExchange [Reuses operator id: 23] -Output [1]: [hd_demo_sk#12] +Output [1]: [hd_demo_sk#126] (176) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#12] +Left keys [1]: [ss_hdemo_sk#118] +Right keys [1]: [hd_demo_sk#126] Join condition: None (177) Project [codegen id : 38] Output: [] -Input [2]: [ss_hdemo_sk#2, hd_demo_sk#12] +Input [2]: [ss_hdemo_sk#118, hd_demo_sk#126] (178) HashAggregate [codegen id : 38] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#64] -Results [1]: [count#65] +Aggregate Attributes [1]: [count#127] +Results [1]: [count#128] (179) Exchange -Input [1]: [count#65] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] +Input [1]: [count#128] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] (180) HashAggregate [codegen id : 39] -Input [1]: [count#65] +Input [1]: [count#128] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#67] -Results [1]: [count(1)#67 AS h12_to_12_30#68] +Aggregate Attributes [1]: [count(1)#130] +Results [1]: [count(1)#130 AS h12_to_12_30#131] (181) BroadcastExchange -Input [1]: [h12_to_12_30#68] -Arguments: IdentityBroadcastMode, [id=#69] +Input [1]: [h12_to_12_30#131] +Arguments: IdentityBroadcastMode, [id=#132] (182) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt index 453906c2b5f34..9f56c71154a66 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/explain.txt @@ -313,687 +313,687 @@ Aggregate Attributes [1]: [count(1)#19] Results [1]: [count(1)#19 AS h8_30_to_9#20] (29) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] (31) Filter [codegen id : 8] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] +Condition : ((isnotnull(ss_hdemo_sk#22) AND isnotnull(ss_sold_time_sk#21)) AND isnotnull(ss_store_sk#23)) (32) Project [codegen id : 8] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, ss_sold_date_sk#24] (33) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#25] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#22] +Right keys [1]: [hd_demo_sk#25] Join condition: None (35) Project [codegen id : 8] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#21, ss_store_sk#23] +Input [4]: [ss_sold_time_sk#21, ss_hdemo_sk#22, ss_store_sk#23, hd_demo_sk#25] (36) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#26, t_hour#27, t_minute#28] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#26, t_hour#27, t_minute#28] (38) Filter [codegen id : 6] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 9)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#26, t_hour#27, t_minute#28] +Condition : ((((isnotnull(t_hour#27) AND isnotnull(t_minute#28)) AND (t_hour#27 = 9)) AND (t_minute#28 < 30)) AND isnotnull(t_time_sk#26)) (39) Project [codegen id : 6] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#26] +Input [3]: [t_time_sk#26, t_hour#27, t_minute#28] (40) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [t_time_sk#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#21] +Right keys [1]: [t_time_sk#26] Join condition: None (42) Project [codegen id : 8] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#23] +Input [3]: [ss_sold_time_sk#21, ss_store_sk#23, t_time_sk#26] (43) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#30] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#23] +Right keys [1]: [s_store_sk#30] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#23, s_store_sk#30] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#22] -Results [1]: [count#23] +Aggregate Attributes [1]: [count#31] +Results [1]: [count#32] (47) Exchange -Input [1]: [count#23] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] +Input [1]: [count#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] (48) HashAggregate [codegen id : 9] -Input [1]: [count#23] +Input [1]: [count#32] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#25] -Results [1]: [count(1)#25 AS h9_to_9_30#26] +Aggregate Attributes [1]: [count(1)#34] +Results [1]: [count(1)#34 AS h9_to_9_30#35] (49) BroadcastExchange -Input [1]: [h9_to_9_30#26] -Arguments: IdentityBroadcastMode, [id=#27] +Input [1]: [h9_to_9_30#35] +Arguments: IdentityBroadcastMode, [id=#36] (50) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (51) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (52) ColumnarToRow [codegen id : 13] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] (53) Filter [codegen id : 13] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] +Condition : ((isnotnull(ss_hdemo_sk#38) AND isnotnull(ss_sold_time_sk#37)) AND isnotnull(ss_store_sk#39)) (54) Project [codegen id : 13] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, ss_sold_date_sk#40] (55) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#41] (56) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#38] +Right keys [1]: [hd_demo_sk#41] Join condition: None (57) Project [codegen id : 13] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#37, ss_store_sk#39] +Input [4]: [ss_sold_time_sk#37, ss_hdemo_sk#38, ss_store_sk#39, hd_demo_sk#41] (58) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#42, t_hour#43, t_minute#44] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 11] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] (60) Filter [codegen id : 11] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 9)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] +Condition : ((((isnotnull(t_hour#43) AND isnotnull(t_minute#44)) AND (t_hour#43 = 9)) AND (t_minute#44 >= 30)) AND isnotnull(t_time_sk#42)) (61) Project [codegen id : 11] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#42] +Input [3]: [t_time_sk#42, t_hour#43, t_minute#44] (62) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#28] +Input [1]: [t_time_sk#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#45] (63) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#37] +Right keys [1]: [t_time_sk#42] Join condition: None (64) Project [codegen id : 13] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#39] +Input [3]: [ss_sold_time_sk#37, ss_store_sk#39, t_time_sk#42] (65) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#46] (66) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#39] +Right keys [1]: [s_store_sk#46] Join condition: None (67) Project [codegen id : 13] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#39, s_store_sk#46] (68) HashAggregate [codegen id : 13] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#29] -Results [1]: [count#30] +Aggregate Attributes [1]: [count#47] +Results [1]: [count#48] (69) Exchange -Input [1]: [count#30] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#31] +Input [1]: [count#48] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] (70) HashAggregate [codegen id : 14] -Input [1]: [count#30] +Input [1]: [count#48] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#32] -Results [1]: [count(1)#32 AS h9_30_to_10#33] +Aggregate Attributes [1]: [count(1)#50] +Results [1]: [count(1)#50 AS h9_30_to_10#51] (71) BroadcastExchange -Input [1]: [h9_30_to_10#33] -Arguments: IdentityBroadcastMode, [id=#34] +Input [1]: [h9_30_to_10#51] +Arguments: IdentityBroadcastMode, [id=#52] (72) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (73) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 18] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] (75) Filter [codegen id : 18] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] +Condition : ((isnotnull(ss_hdemo_sk#54) AND isnotnull(ss_sold_time_sk#53)) AND isnotnull(ss_store_sk#55)) (76) Project [codegen id : 18] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, ss_sold_date_sk#56] (77) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#57] (78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#54] +Right keys [1]: [hd_demo_sk#57] Join condition: None (79) Project [codegen id : 18] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#53, ss_store_sk#55] +Input [4]: [ss_sold_time_sk#53, ss_hdemo_sk#54, ss_store_sk#55, hd_demo_sk#57] (80) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#58, t_hour#59, t_minute#60] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (81) ColumnarToRow [codegen id : 16] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#58, t_hour#59, t_minute#60] (82) Filter [codegen id : 16] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 10)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#58, t_hour#59, t_minute#60] +Condition : ((((isnotnull(t_hour#59) AND isnotnull(t_minute#60)) AND (t_hour#59 = 10)) AND (t_minute#60 < 30)) AND isnotnull(t_time_sk#58)) (83) Project [codegen id : 16] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#58] +Input [3]: [t_time_sk#58, t_hour#59, t_minute#60] (84) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [1]: [t_time_sk#58] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] (85) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#53] +Right keys [1]: [t_time_sk#58] Join condition: None (86) Project [codegen id : 18] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#55] +Input [3]: [ss_sold_time_sk#53, ss_store_sk#55, t_time_sk#58] (87) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#62] (88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#55] +Right keys [1]: [s_store_sk#62] Join condition: None (89) Project [codegen id : 18] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#55, s_store_sk#62] (90) HashAggregate [codegen id : 18] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#36] -Results [1]: [count#37] +Aggregate Attributes [1]: [count#63] +Results [1]: [count#64] (91) Exchange -Input [1]: [count#37] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#38] +Input [1]: [count#64] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] (92) HashAggregate [codegen id : 19] -Input [1]: [count#37] +Input [1]: [count#64] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#39] -Results [1]: [count(1)#39 AS h10_to_10_30#40] +Aggregate Attributes [1]: [count(1)#66] +Results [1]: [count(1)#66 AS h10_to_10_30#67] (93) BroadcastExchange -Input [1]: [h10_to_10_30#40] -Arguments: IdentityBroadcastMode, [id=#41] +Input [1]: [h10_to_10_30#67] +Arguments: IdentityBroadcastMode, [id=#68] (94) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (95) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (96) ColumnarToRow [codegen id : 23] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] (97) Filter [codegen id : 23] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Condition : ((isnotnull(ss_hdemo_sk#70) AND isnotnull(ss_sold_time_sk#69)) AND isnotnull(ss_store_sk#71)) (98) Project [codegen id : 23] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] (99) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#73] (100) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#70] +Right keys [1]: [hd_demo_sk#73] Join condition: None (101) Project [codegen id : 23] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#69, ss_store_sk#71] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, hd_demo_sk#73] (102) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#74, t_hour#75, t_minute#76] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (103) ColumnarToRow [codegen id : 21] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] (104) Filter [codegen id : 21] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 10)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] +Condition : ((((isnotnull(t_hour#75) AND isnotnull(t_minute#76)) AND (t_hour#75 = 10)) AND (t_minute#76 >= 30)) AND isnotnull(t_time_sk#74)) (105) Project [codegen id : 21] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#74] +Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] (106) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +Input [1]: [t_time_sk#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#77] (107) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#69] +Right keys [1]: [t_time_sk#74] Join condition: None (108) Project [codegen id : 23] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#71] +Input [3]: [ss_sold_time_sk#69, ss_store_sk#71, t_time_sk#74] (109) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#78] (110) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#71] +Right keys [1]: [s_store_sk#78] Join condition: None (111) Project [codegen id : 23] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#71, s_store_sk#78] (112) HashAggregate [codegen id : 23] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#43] -Results [1]: [count#44] +Aggregate Attributes [1]: [count#79] +Results [1]: [count#80] (113) Exchange -Input [1]: [count#44] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#45] +Input [1]: [count#80] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] (114) HashAggregate [codegen id : 24] -Input [1]: [count#44] +Input [1]: [count#80] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#46] -Results [1]: [count(1)#46 AS h10_30_to_11#47] +Aggregate Attributes [1]: [count(1)#82] +Results [1]: [count(1)#82 AS h10_30_to_11#83] (115) BroadcastExchange -Input [1]: [h10_30_to_11#47] -Arguments: IdentityBroadcastMode, [id=#48] +Input [1]: [h10_30_to_11#83] +Arguments: IdentityBroadcastMode, [id=#84] (116) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (117) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (118) ColumnarToRow [codegen id : 28] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] (119) Filter [codegen id : 28] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] +Condition : ((isnotnull(ss_hdemo_sk#86) AND isnotnull(ss_sold_time_sk#85)) AND isnotnull(ss_store_sk#87)) (120) Project [codegen id : 28] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, ss_sold_date_sk#88] (121) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#89] (122) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#86] +Right keys [1]: [hd_demo_sk#89] Join condition: None (123) Project [codegen id : 28] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#85, ss_store_sk#87] +Input [4]: [ss_sold_time_sk#85, ss_hdemo_sk#86, ss_store_sk#87, hd_demo_sk#89] (124) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#90, t_hour#91, t_minute#92] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (125) ColumnarToRow [codegen id : 26] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#90, t_hour#91, t_minute#92] (126) Filter [codegen id : 26] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 11)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#90, t_hour#91, t_minute#92] +Condition : ((((isnotnull(t_hour#91) AND isnotnull(t_minute#92)) AND (t_hour#91 = 11)) AND (t_minute#92 < 30)) AND isnotnull(t_time_sk#90)) (127) Project [codegen id : 26] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#90] +Input [3]: [t_time_sk#90, t_hour#91, t_minute#92] (128) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] +Input [1]: [t_time_sk#90] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#93] (129) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#85] +Right keys [1]: [t_time_sk#90] Join condition: None (130) Project [codegen id : 28] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#87] +Input [3]: [ss_sold_time_sk#85, ss_store_sk#87, t_time_sk#90] (131) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#94] (132) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#87] +Right keys [1]: [s_store_sk#94] Join condition: None (133) Project [codegen id : 28] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#87, s_store_sk#94] (134) HashAggregate [codegen id : 28] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#50] -Results [1]: [count#51] +Aggregate Attributes [1]: [count#95] +Results [1]: [count#96] (135) Exchange -Input [1]: [count#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#52] +Input [1]: [count#96] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#97] (136) HashAggregate [codegen id : 29] -Input [1]: [count#51] +Input [1]: [count#96] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#53] -Results [1]: [count(1)#53 AS h11_to_11_30#54] +Aggregate Attributes [1]: [count(1)#98] +Results [1]: [count(1)#98 AS h11_to_11_30#99] (137) BroadcastExchange -Input [1]: [h11_to_11_30#54] -Arguments: IdentityBroadcastMode, [id=#55] +Input [1]: [h11_to_11_30#99] +Arguments: IdentityBroadcastMode, [id=#100] (138) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (139) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (140) ColumnarToRow [codegen id : 33] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] (141) Filter [codegen id : 33] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] +Condition : ((isnotnull(ss_hdemo_sk#102) AND isnotnull(ss_sold_time_sk#101)) AND isnotnull(ss_store_sk#103)) (142) Project [codegen id : 33] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, ss_sold_date_sk#104] (143) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#105] (144) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#102] +Right keys [1]: [hd_demo_sk#105] Join condition: None (145) Project [codegen id : 33] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#101, ss_store_sk#103] +Input [4]: [ss_sold_time_sk#101, ss_hdemo_sk#102, ss_store_sk#103, hd_demo_sk#105] (146) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#106, t_hour#107, t_minute#108] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (147) ColumnarToRow [codegen id : 31] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#106, t_hour#107, t_minute#108] (148) Filter [codegen id : 31] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 11)) AND (t_minute#11 >= 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#106, t_hour#107, t_minute#108] +Condition : ((((isnotnull(t_hour#107) AND isnotnull(t_minute#108)) AND (t_hour#107 = 11)) AND (t_minute#108 >= 30)) AND isnotnull(t_time_sk#106)) (149) Project [codegen id : 31] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#106] +Input [3]: [t_time_sk#106, t_hour#107, t_minute#108] (150) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#56] +Input [1]: [t_time_sk#106] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#109] (151) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#101] +Right keys [1]: [t_time_sk#106] Join condition: None (152) Project [codegen id : 33] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#103] +Input [3]: [ss_sold_time_sk#101, ss_store_sk#103, t_time_sk#106] (153) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#110] (154) BroadcastHashJoin [codegen id : 33] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#103] +Right keys [1]: [s_store_sk#110] Join condition: None (155) Project [codegen id : 33] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#103, s_store_sk#110] (156) HashAggregate [codegen id : 33] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#57] -Results [1]: [count#58] +Aggregate Attributes [1]: [count#111] +Results [1]: [count#112] (157) Exchange -Input [1]: [count#58] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#59] +Input [1]: [count#112] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] (158) HashAggregate [codegen id : 34] -Input [1]: [count#58] +Input [1]: [count#112] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#60] -Results [1]: [count(1)#60 AS h11_30_to_12#61] +Aggregate Attributes [1]: [count(1)#114] +Results [1]: [count(1)#114 AS h11_30_to_12#115] (159) BroadcastExchange -Input [1]: [h11_30_to_12#61] -Arguments: IdentityBroadcastMode, [id=#62] +Input [1]: [h11_30_to_12#115] +Arguments: IdentityBroadcastMode, [id=#116] (160) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None (161) Scan parquet default.store_sales -Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] ReadSchema: struct (162) ColumnarToRow [codegen id : 38] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] (163) Filter [codegen id : 38] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] -Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] +Condition : ((isnotnull(ss_hdemo_sk#118) AND isnotnull(ss_sold_time_sk#117)) AND isnotnull(ss_store_sk#119)) (164) Project [codegen id : 38] -Output [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Output [3]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, ss_sold_date_sk#120] (165) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#121] (166) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ss_hdemo_sk#118] +Right keys [1]: [hd_demo_sk#121] Join condition: None (167) Project [codegen id : 38] -Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] -Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] +Output [2]: [ss_sold_time_sk#117, ss_store_sk#119] +Input [4]: [ss_sold_time_sk#117, ss_hdemo_sk#118, ss_store_sk#119, hd_demo_sk#121] (168) Scan parquet default.time_dim -Output [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [3]: [t_time_sk#122, t_hour#123, t_minute#124] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] ReadSchema: struct (169) ColumnarToRow [codegen id : 36] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Input [3]: [t_time_sk#122, t_hour#123, t_minute#124] (170) Filter [codegen id : 36] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] -Condition : ((((isnotnull(t_hour#10) AND isnotnull(t_minute#11)) AND (t_hour#10 = 12)) AND (t_minute#11 < 30)) AND isnotnull(t_time_sk#9)) +Input [3]: [t_time_sk#122, t_hour#123, t_minute#124] +Condition : ((((isnotnull(t_hour#123) AND isnotnull(t_minute#124)) AND (t_hour#123 = 12)) AND (t_minute#124 < 30)) AND isnotnull(t_time_sk#122)) (171) Project [codegen id : 36] -Output [1]: [t_time_sk#9] -Input [3]: [t_time_sk#9, t_hour#10, t_minute#11] +Output [1]: [t_time_sk#122] +Input [3]: [t_time_sk#122, t_hour#123, t_minute#124] (172) BroadcastExchange -Input [1]: [t_time_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#63] +Input [1]: [t_time_sk#122] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#125] (173) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_sold_time_sk#1] -Right keys [1]: [t_time_sk#9] +Left keys [1]: [ss_sold_time_sk#117] +Right keys [1]: [t_time_sk#122] Join condition: None (174) Project [codegen id : 38] -Output [1]: [ss_store_sk#3] -Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#9] +Output [1]: [ss_store_sk#119] +Input [3]: [ss_sold_time_sk#117, ss_store_sk#119, t_time_sk#122] (175) ReusedExchange [Reuses operator id: 23] -Output [1]: [s_store_sk#13] +Output [1]: [s_store_sk#126] (176) BroadcastHashJoin [codegen id : 38] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#13] +Left keys [1]: [ss_store_sk#119] +Right keys [1]: [s_store_sk#126] Join condition: None (177) Project [codegen id : 38] Output: [] -Input [2]: [ss_store_sk#3, s_store_sk#13] +Input [2]: [ss_store_sk#119, s_store_sk#126] (178) HashAggregate [codegen id : 38] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#64] -Results [1]: [count#65] +Aggregate Attributes [1]: [count#127] +Results [1]: [count#128] (179) Exchange -Input [1]: [count#65] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] +Input [1]: [count#128] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#129] (180) HashAggregate [codegen id : 39] -Input [1]: [count#65] +Input [1]: [count#128] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#67] -Results [1]: [count(1)#67 AS h12_to_12_30#68] +Aggregate Attributes [1]: [count(1)#130] +Results [1]: [count(1)#130 AS h12_to_12_30#131] (181) BroadcastExchange -Input [1]: [h12_to_12_30#68] -Arguments: IdentityBroadcastMode, [id=#69] +Input [1]: [h12_to_12_30#131] +Arguments: IdentityBroadcastMode, [id=#132] (182) BroadcastNestedLoopJoin [codegen id : 40] Join condition: None diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt index 65c66eb083f55..8736c9861a5ce 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.sf100/explain.txt @@ -82,40 +82,40 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (12) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (13) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] (14) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Condition : ((isnotnull(ss_quantity#44) AND (ss_quantity#44 >= 1)) AND (ss_quantity#44 <= 20)) (15) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#45] +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] (16) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#45] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#45, count#46] -Results [2]: [sum#47, count#48] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#45))] +Aggregate Attributes [2]: [sum#47, count#48] +Results [2]: [sum#49, count#50] (17) Exchange -Input [2]: [sum#47, count#48] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] +Input [2]: [sum#49, count#50] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#51] (18) HashAggregate [codegen id : 2] -Input [2]: [sum#47, count#48] +Input [2]: [sum#49, count#50] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#50] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#50 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#51] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))#52] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#45))#52 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#53] Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#6, [id=#7] * HashAggregate (25) @@ -128,40 +128,40 @@ Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (19) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (20) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] (21) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Condition : ((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 1)) AND (ss_quantity#54 <= 20)) (22) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#55] +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] (23) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#55] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#53, count#54] -Results [2]: [sum#55, count#56] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [2]: [sum#57, count#58] +Results [2]: [sum#59, count#60] (24) Exchange -Input [2]: [sum#55, count#56] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#57] +Input [2]: [sum#59, count#60] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] (25) HashAggregate [codegen id : 2] -Input [2]: [sum#55, count#56] +Input [2]: [sum#59, count#60] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#58] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#58 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#59] +Functions [1]: [avg(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#55))#62] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#55))#62 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#63] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#9, [id=#10] * HashAggregate (32) @@ -174,40 +174,40 @@ Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (26) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#64, ss_sold_date_sk#65] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (27) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] (28) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +Condition : ((isnotnull(ss_quantity#64) AND (ss_quantity#64 >= 21)) AND (ss_quantity#64 <= 40)) (29) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] (30) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#60] -Results [1]: [count#61] +Aggregate Attributes [1]: [count#66] +Results [1]: [count#67] (31) Exchange -Input [1]: [count#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] +Input [1]: [count#67] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#68] (32) HashAggregate [codegen id : 2] -Input [1]: [count#61] +Input [1]: [count#67] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#63] -Results [1]: [count(1)#63 AS count(1)#64] +Aggregate Attributes [1]: [count(1)#69] +Results [1]: [count(1)#69 AS count(1)#70] Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] * HashAggregate (39) @@ -220,40 +220,40 @@ Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (33) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (34) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] (35) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Condition : ((isnotnull(ss_quantity#71) AND (ss_quantity#71 >= 21)) AND (ss_quantity#71 <= 40)) (36) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#72] +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] (37) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#72] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#65, count#66] -Results [2]: [sum#67, count#68] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#72))] +Aggregate Attributes [2]: [sum#74, count#75] +Results [2]: [sum#76, count#77] (38) Exchange -Input [2]: [sum#67, count#68] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [sum#76, count#77] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] (39) HashAggregate [codegen id : 2] -Input [2]: [sum#67, count#68] +Input [2]: [sum#76, count#77] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#70] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#70 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#71] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))#79] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#72))#79 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#80] Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#13, [id=#14] * HashAggregate (46) @@ -266,40 +266,40 @@ Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (40) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (41) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] (42) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Condition : ((isnotnull(ss_quantity#81) AND (ss_quantity#81 >= 21)) AND (ss_quantity#81 <= 40)) (43) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#82] +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] (44) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#82] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#72, count#73] -Results [2]: [sum#74, count#75] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#82))] +Aggregate Attributes [2]: [sum#84, count#85] +Results [2]: [sum#86, count#87] (45) Exchange -Input [2]: [sum#74, count#75] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] +Input [2]: [sum#86, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] (46) HashAggregate [codegen id : 2] -Input [2]: [sum#74, count#75] +Input [2]: [sum#86, count#87] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#77] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#77 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#78] +Functions [1]: [avg(UnscaledValue(ss_net_paid#82))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#82))#89] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#82))#89 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#90] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#16, [id=#17] * HashAggregate (53) @@ -312,40 +312,40 @@ Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (47) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#91, ss_sold_date_sk#92] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (48) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] (49) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +Condition : ((isnotnull(ss_quantity#91) AND (ss_quantity#91 >= 41)) AND (ss_quantity#91 <= 60)) (50) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] (51) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#79] -Results [1]: [count#80] +Aggregate Attributes [1]: [count#93] +Results [1]: [count#94] (52) Exchange -Input [1]: [count#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +Input [1]: [count#94] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] (53) HashAggregate [codegen id : 2] -Input [1]: [count#80] +Input [1]: [count#94] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#82] -Results [1]: [count(1)#82 AS count(1)#83] +Aggregate Attributes [1]: [count(1)#96] +Results [1]: [count(1)#96 AS count(1)#97] Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#18, [id=#19] * HashAggregate (60) @@ -358,40 +358,40 @@ Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (54) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (55) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] (56) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Condition : ((isnotnull(ss_quantity#98) AND (ss_quantity#98 >= 41)) AND (ss_quantity#98 <= 60)) (57) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#99] +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] (58) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#99] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#84, count#85] -Results [2]: [sum#86, count#87] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#99))] +Aggregate Attributes [2]: [sum#101, count#102] +Results [2]: [sum#103, count#104] (59) Exchange -Input [2]: [sum#86, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] +Input [2]: [sum#103, count#104] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] (60) HashAggregate [codegen id : 2] -Input [2]: [sum#86, count#87] +Input [2]: [sum#103, count#104] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#89] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#89 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#90] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))#106] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#99))#106 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#107] Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#20, [id=#21] * HashAggregate (67) @@ -404,40 +404,40 @@ Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (61) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (62) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] (63) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Condition : ((isnotnull(ss_quantity#108) AND (ss_quantity#108 >= 41)) AND (ss_quantity#108 <= 60)) (64) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#109] +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] (65) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#109] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#91, count#92] -Results [2]: [sum#93, count#94] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#109))] +Aggregate Attributes [2]: [sum#111, count#112] +Results [2]: [sum#113, count#114] (66) Exchange -Input [2]: [sum#93, count#94] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] +Input [2]: [sum#113, count#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] (67) HashAggregate [codegen id : 2] -Input [2]: [sum#93, count#94] +Input [2]: [sum#113, count#114] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#96] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#96 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#97] +Functions [1]: [avg(UnscaledValue(ss_net_paid#109))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#109))#116] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#109))#116 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#117] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#23, [id=#24] * HashAggregate (74) @@ -450,40 +450,40 @@ Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (68) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#118, ss_sold_date_sk#119] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (69) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] (70) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +Condition : ((isnotnull(ss_quantity#118) AND (ss_quantity#118 >= 61)) AND (ss_quantity#118 <= 80)) (71) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] (72) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#98] -Results [1]: [count#99] +Aggregate Attributes [1]: [count#120] +Results [1]: [count#121] (73) Exchange -Input [1]: [count#99] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#100] +Input [1]: [count#121] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#122] (74) HashAggregate [codegen id : 2] -Input [1]: [count#99] +Input [1]: [count#121] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#101] -Results [1]: [count(1)#101 AS count(1)#102] +Aggregate Attributes [1]: [count(1)#123] +Results [1]: [count(1)#123 AS count(1)#124] Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#25, [id=#26] * HashAggregate (81) @@ -496,40 +496,40 @@ Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (75) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (76) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] (77) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Condition : ((isnotnull(ss_quantity#125) AND (ss_quantity#125 >= 61)) AND (ss_quantity#125 <= 80)) (78) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#126] +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] (79) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#126] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#103, count#104] -Results [2]: [sum#105, count#106] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#126))] +Aggregate Attributes [2]: [sum#128, count#129] +Results [2]: [sum#130, count#131] (80) Exchange -Input [2]: [sum#105, count#106] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#107] +Input [2]: [sum#130, count#131] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#132] (81) HashAggregate [codegen id : 2] -Input [2]: [sum#105, count#106] +Input [2]: [sum#130, count#131] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#108] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#108 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#109] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))#133] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#126))#133 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#134] Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#27, [id=#28] * HashAggregate (88) @@ -542,40 +542,40 @@ Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (82) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (83) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] (84) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Condition : ((isnotnull(ss_quantity#135) AND (ss_quantity#135 >= 61)) AND (ss_quantity#135 <= 80)) (85) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#136] +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] (86) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#136] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#110, count#111] -Results [2]: [sum#112, count#113] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#136))] +Aggregate Attributes [2]: [sum#138, count#139] +Results [2]: [sum#140, count#141] (87) Exchange -Input [2]: [sum#112, count#113] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] +Input [2]: [sum#140, count#141] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#142] (88) HashAggregate [codegen id : 2] -Input [2]: [sum#112, count#113] +Input [2]: [sum#140, count#141] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#115] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#115 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#116] +Functions [1]: [avg(UnscaledValue(ss_net_paid#136))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#136))#143] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#136))#143 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#144] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#30, [id=#31] * HashAggregate (95) @@ -588,40 +588,40 @@ Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (89) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#145, ss_sold_date_sk#146] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (90) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] (91) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] +Condition : ((isnotnull(ss_quantity#145) AND (ss_quantity#145 >= 81)) AND (ss_quantity#145 <= 100)) (92) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] (93) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#117] -Results [1]: [count#118] +Aggregate Attributes [1]: [count#147] +Results [1]: [count#148] (94) Exchange -Input [1]: [count#118] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#119] +Input [1]: [count#148] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#149] (95) HashAggregate [codegen id : 2] -Input [1]: [count#118] +Input [1]: [count#148] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#120] -Results [1]: [count(1)#120 AS count(1)#121] +Aggregate Attributes [1]: [count(1)#150] +Results [1]: [count(1)#150 AS count(1)#151] Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#32, [id=#33] * HashAggregate (102) @@ -634,40 +634,40 @@ Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (96) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (97) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] (98) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Condition : ((isnotnull(ss_quantity#152) AND (ss_quantity#152 >= 81)) AND (ss_quantity#152 <= 100)) (99) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#153] +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] (100) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#153] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#122, count#123] -Results [2]: [sum#124, count#125] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#153))] +Aggregate Attributes [2]: [sum#155, count#156] +Results [2]: [sum#157, count#158] (101) Exchange -Input [2]: [sum#124, count#125] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#126] +Input [2]: [sum#157, count#158] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#159] (102) HashAggregate [codegen id : 2] -Input [2]: [sum#124, count#125] +Input [2]: [sum#157, count#158] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#127] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#127 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#128] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))#160] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#153))#160 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#161] Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#34, [id=#35] * HashAggregate (109) @@ -680,39 +680,39 @@ Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (103) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] (105) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Condition : ((isnotnull(ss_quantity#162) AND (ss_quantity#162 >= 81)) AND (ss_quantity#162 <= 100)) (106) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#163] +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] (107) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#163] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#129, count#130] -Results [2]: [sum#131, count#132] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#163))] +Aggregate Attributes [2]: [sum#165, count#166] +Results [2]: [sum#167, count#168] (108) Exchange -Input [2]: [sum#131, count#132] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#133] +Input [2]: [sum#167, count#168] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#169] (109) HashAggregate [codegen id : 2] -Input [2]: [sum#131, count#132] +Input [2]: [sum#167, count#168] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#134] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#134 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#135] +Functions [1]: [avg(UnscaledValue(ss_net_paid#163))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#163))#170] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#163))#170 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#171] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt index 65c66eb083f55..8736c9861a5ce 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/explain.txt @@ -82,40 +82,40 @@ Subquery:2 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (12) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (13) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] (14) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] +Condition : ((isnotnull(ss_quantity#44) AND (ss_quantity#44 >= 1)) AND (ss_quantity#44 <= 20)) (15) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#45] +Input [3]: [ss_quantity#44, ss_ext_discount_amt#45, ss_sold_date_sk#46] (16) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#45] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#45, count#46] -Results [2]: [sum#47, count#48] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#45))] +Aggregate Attributes [2]: [sum#47, count#48] +Results [2]: [sum#49, count#50] (17) Exchange -Input [2]: [sum#47, count#48] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#49] +Input [2]: [sum#49, count#50] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#51] (18) HashAggregate [codegen id : 2] -Input [2]: [sum#47, count#48] +Input [2]: [sum#49, count#50] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#50] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#50 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#51] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#45))#52] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#45))#52 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#53] Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#6, [id=#7] * HashAggregate (25) @@ -128,40 +128,40 @@ Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (19) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] ReadSchema: struct (20) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] (21) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 1)) AND (ss_quantity#37 <= 20)) +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] +Condition : ((isnotnull(ss_quantity#54) AND (ss_quantity#54 >= 1)) AND (ss_quantity#54 <= 20)) (22) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#55] +Input [3]: [ss_quantity#54, ss_net_paid#55, ss_sold_date_sk#56] (23) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#55] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#53, count#54] -Results [2]: [sum#55, count#56] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [2]: [sum#57, count#58] +Results [2]: [sum#59, count#60] (24) Exchange -Input [2]: [sum#55, count#56] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#57] +Input [2]: [sum#59, count#60] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] (25) HashAggregate [codegen id : 2] -Input [2]: [sum#55, count#56] +Input [2]: [sum#59, count#60] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#58] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#58 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#59] +Functions [1]: [avg(UnscaledValue(ss_net_paid#55))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#55))#62] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#55))#62 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#63] Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#9, [id=#10] * HashAggregate (32) @@ -174,40 +174,40 @@ Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (26) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#64, ss_sold_date_sk#65] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (27) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] (28) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] +Condition : ((isnotnull(ss_quantity#64) AND (ss_quantity#64 >= 21)) AND (ss_quantity#64 <= 40)) (29) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#64, ss_sold_date_sk#65] (30) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#60] -Results [1]: [count#61] +Aggregate Attributes [1]: [count#66] +Results [1]: [count#67] (31) Exchange -Input [1]: [count#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] +Input [1]: [count#67] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#68] (32) HashAggregate [codegen id : 2] -Input [1]: [count#61] +Input [1]: [count#67] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#63] -Results [1]: [count(1)#63 AS count(1)#64] +Aggregate Attributes [1]: [count(1)#69] +Results [1]: [count(1)#69 AS count(1)#70] Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] * HashAggregate (39) @@ -220,40 +220,40 @@ Subquery:5 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (33) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (34) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] (35) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] +Condition : ((isnotnull(ss_quantity#71) AND (ss_quantity#71 >= 21)) AND (ss_quantity#71 <= 40)) (36) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#72] +Input [3]: [ss_quantity#71, ss_ext_discount_amt#72, ss_sold_date_sk#73] (37) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#72] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#65, count#66] -Results [2]: [sum#67, count#68] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#72))] +Aggregate Attributes [2]: [sum#74, count#75] +Results [2]: [sum#76, count#77] (38) Exchange -Input [2]: [sum#67, count#68] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [sum#76, count#77] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#78] (39) HashAggregate [codegen id : 2] -Input [2]: [sum#67, count#68] +Input [2]: [sum#76, count#77] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#70] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#70 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#71] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#72))#79] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#72))#79 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#80] Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#13, [id=#14] * HashAggregate (46) @@ -266,40 +266,40 @@ Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (40) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] ReadSchema: struct (41) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] (42) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 21)) AND (ss_quantity#37 <= 40)) +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] +Condition : ((isnotnull(ss_quantity#81) AND (ss_quantity#81 >= 21)) AND (ss_quantity#81 <= 40)) (43) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#82] +Input [3]: [ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#83] (44) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#82] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#72, count#73] -Results [2]: [sum#74, count#75] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#82))] +Aggregate Attributes [2]: [sum#84, count#85] +Results [2]: [sum#86, count#87] (45) Exchange -Input [2]: [sum#74, count#75] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] +Input [2]: [sum#86, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] (46) HashAggregate [codegen id : 2] -Input [2]: [sum#74, count#75] +Input [2]: [sum#86, count#87] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#77] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#77 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#78] +Functions [1]: [avg(UnscaledValue(ss_net_paid#82))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#82))#89] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#82))#89 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#90] Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#16, [id=#17] * HashAggregate (53) @@ -312,40 +312,40 @@ Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (47) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#91, ss_sold_date_sk#92] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (48) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] (49) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] +Condition : ((isnotnull(ss_quantity#91) AND (ss_quantity#91 >= 41)) AND (ss_quantity#91 <= 60)) (50) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#91, ss_sold_date_sk#92] (51) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#79] -Results [1]: [count#80] +Aggregate Attributes [1]: [count#93] +Results [1]: [count#94] (52) Exchange -Input [1]: [count#80] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#81] +Input [1]: [count#94] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] (53) HashAggregate [codegen id : 2] -Input [1]: [count#80] +Input [1]: [count#94] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#82] -Results [1]: [count(1)#82 AS count(1)#83] +Aggregate Attributes [1]: [count(1)#96] +Results [1]: [count(1)#96 AS count(1)#97] Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#18, [id=#19] * HashAggregate (60) @@ -358,40 +358,40 @@ Subquery:8 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (54) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (55) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] (56) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] +Condition : ((isnotnull(ss_quantity#98) AND (ss_quantity#98 >= 41)) AND (ss_quantity#98 <= 60)) (57) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#99] +Input [3]: [ss_quantity#98, ss_ext_discount_amt#99, ss_sold_date_sk#100] (58) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#99] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#84, count#85] -Results [2]: [sum#86, count#87] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#99))] +Aggregate Attributes [2]: [sum#101, count#102] +Results [2]: [sum#103, count#104] (59) Exchange -Input [2]: [sum#86, count#87] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#88] +Input [2]: [sum#103, count#104] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#105] (60) HashAggregate [codegen id : 2] -Input [2]: [sum#86, count#87] +Input [2]: [sum#103, count#104] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#89] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#89 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#90] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#99))#106] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#99))#106 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#107] Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#20, [id=#21] * HashAggregate (67) @@ -404,40 +404,40 @@ Subquery:9 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery (61) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] ReadSchema: struct (62) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] (63) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 41)) AND (ss_quantity#37 <= 60)) +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] +Condition : ((isnotnull(ss_quantity#108) AND (ss_quantity#108 >= 41)) AND (ss_quantity#108 <= 60)) (64) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#109] +Input [3]: [ss_quantity#108, ss_net_paid#109, ss_sold_date_sk#110] (65) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#109] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#91, count#92] -Results [2]: [sum#93, count#94] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#109))] +Aggregate Attributes [2]: [sum#111, count#112] +Results [2]: [sum#113, count#114] (66) Exchange -Input [2]: [sum#93, count#94] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] +Input [2]: [sum#113, count#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] (67) HashAggregate [codegen id : 2] -Input [2]: [sum#93, count#94] +Input [2]: [sum#113, count#114] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#96] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#96 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#97] +Functions [1]: [avg(UnscaledValue(ss_net_paid#109))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#109))#116] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#109))#116 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#117] Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#23, [id=#24] * HashAggregate (74) @@ -450,40 +450,40 @@ Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (68) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#118, ss_sold_date_sk#119] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (69) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] (70) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] +Condition : ((isnotnull(ss_quantity#118) AND (ss_quantity#118 >= 61)) AND (ss_quantity#118 <= 80)) (71) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#118, ss_sold_date_sk#119] (72) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#98] -Results [1]: [count#99] +Aggregate Attributes [1]: [count#120] +Results [1]: [count#121] (73) Exchange -Input [1]: [count#99] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#100] +Input [1]: [count#121] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#122] (74) HashAggregate [codegen id : 2] -Input [1]: [count#99] +Input [1]: [count#121] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#101] -Results [1]: [count(1)#101 AS count(1)#102] +Aggregate Attributes [1]: [count(1)#123] +Results [1]: [count(1)#123 AS count(1)#124] Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#25, [id=#26] * HashAggregate (81) @@ -496,40 +496,40 @@ Subquery:11 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (75) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (76) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] (77) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] +Condition : ((isnotnull(ss_quantity#125) AND (ss_quantity#125 >= 61)) AND (ss_quantity#125 <= 80)) (78) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#126] +Input [3]: [ss_quantity#125, ss_ext_discount_amt#126, ss_sold_date_sk#127] (79) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#126] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#103, count#104] -Results [2]: [sum#105, count#106] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#126))] +Aggregate Attributes [2]: [sum#128, count#129] +Results [2]: [sum#130, count#131] (80) Exchange -Input [2]: [sum#105, count#106] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#107] +Input [2]: [sum#130, count#131] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#132] (81) HashAggregate [codegen id : 2] -Input [2]: [sum#105, count#106] +Input [2]: [sum#130, count#131] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#108] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#108 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#109] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#126))#133] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#126))#133 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#134] Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#27, [id=#28] * HashAggregate (88) @@ -542,40 +542,40 @@ Subquery:12 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (82) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] ReadSchema: struct (83) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] (84) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 61)) AND (ss_quantity#37 <= 80)) +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] +Condition : ((isnotnull(ss_quantity#135) AND (ss_quantity#135 >= 61)) AND (ss_quantity#135 <= 80)) (85) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#136] +Input [3]: [ss_quantity#135, ss_net_paid#136, ss_sold_date_sk#137] (86) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#136] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#110, count#111] -Results [2]: [sum#112, count#113] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#136))] +Aggregate Attributes [2]: [sum#138, count#139] +Results [2]: [sum#140, count#141] (87) Exchange -Input [2]: [sum#112, count#113] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#114] +Input [2]: [sum#140, count#141] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#142] (88) HashAggregate [codegen id : 2] -Input [2]: [sum#112, count#113] +Input [2]: [sum#140, count#141] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#115] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#115 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#116] +Functions [1]: [avg(UnscaledValue(ss_net_paid#136))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#136))#143] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#136))#143 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#144] Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#30, [id=#31] * HashAggregate (95) @@ -588,40 +588,40 @@ Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (89) Scan parquet default.store_sales -Output [2]: [ss_quantity#37, ss_sold_date_sk#38] +Output [2]: [ss_quantity#145, ss_sold_date_sk#146] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (90) ColumnarToRow [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] (91) Filter [codegen id : 1] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] +Condition : ((isnotnull(ss_quantity#145) AND (ss_quantity#145 >= 81)) AND (ss_quantity#145 <= 100)) (92) Project [codegen id : 1] Output: [] -Input [2]: [ss_quantity#37, ss_sold_date_sk#38] +Input [2]: [ss_quantity#145, ss_sold_date_sk#146] (93) HashAggregate [codegen id : 1] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#117] -Results [1]: [count#118] +Aggregate Attributes [1]: [count#147] +Results [1]: [count#148] (94) Exchange -Input [1]: [count#118] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#119] +Input [1]: [count#148] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#149] (95) HashAggregate [codegen id : 2] -Input [1]: [count#118] +Input [1]: [count#148] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#120] -Results [1]: [count(1)#120 AS count(1)#121] +Aggregate Attributes [1]: [count(1)#150] +Results [1]: [count(1)#150 AS count(1)#151] Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#32, [id=#33] * HashAggregate (102) @@ -634,40 +634,40 @@ Subquery:14 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (96) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (97) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] (98) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] +Condition : ((isnotnull(ss_quantity#152) AND (ss_quantity#152 >= 81)) AND (ss_quantity#152 <= 100)) (99) Project [codegen id : 1] -Output [1]: [ss_ext_discount_amt#44] -Input [3]: [ss_quantity#37, ss_ext_discount_amt#44, ss_sold_date_sk#38] +Output [1]: [ss_ext_discount_amt#153] +Input [3]: [ss_quantity#152, ss_ext_discount_amt#153, ss_sold_date_sk#154] (100) HashAggregate [codegen id : 1] -Input [1]: [ss_ext_discount_amt#44] +Input [1]: [ss_ext_discount_amt#153] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [2]: [sum#122, count#123] -Results [2]: [sum#124, count#125] +Functions [1]: [partial_avg(UnscaledValue(ss_ext_discount_amt#153))] +Aggregate Attributes [2]: [sum#155, count#156] +Results [2]: [sum#157, count#158] (101) Exchange -Input [2]: [sum#124, count#125] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#126] +Input [2]: [sum#157, count#158] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#159] (102) HashAggregate [codegen id : 2] -Input [2]: [sum#124, count#125] +Input [2]: [sum#157, count#158] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#44))#127] -Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#44))#127 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#128] +Functions [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_ext_discount_amt#153))#160] +Results [1]: [cast((avg(UnscaledValue(ss_ext_discount_amt#153))#160 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#161] Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#34, [id=#35] * HashAggregate (109) @@ -680,39 +680,39 @@ Subquery:15 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquer (103) Scan parquet default.store_sales -Output [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] ReadSchema: struct (104) ColumnarToRow [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] (105) Filter [codegen id : 1] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] -Condition : ((isnotnull(ss_quantity#37) AND (ss_quantity#37 >= 81)) AND (ss_quantity#37 <= 100)) +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] +Condition : ((isnotnull(ss_quantity#162) AND (ss_quantity#162 >= 81)) AND (ss_quantity#162 <= 100)) (106) Project [codegen id : 1] -Output [1]: [ss_net_paid#52] -Input [3]: [ss_quantity#37, ss_net_paid#52, ss_sold_date_sk#38] +Output [1]: [ss_net_paid#163] +Input [3]: [ss_quantity#162, ss_net_paid#163, ss_sold_date_sk#164] (107) HashAggregate [codegen id : 1] -Input [1]: [ss_net_paid#52] +Input [1]: [ss_net_paid#163] Keys: [] -Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [2]: [sum#129, count#130] -Results [2]: [sum#131, count#132] +Functions [1]: [partial_avg(UnscaledValue(ss_net_paid#163))] +Aggregate Attributes [2]: [sum#165, count#166] +Results [2]: [sum#167, count#168] (108) Exchange -Input [2]: [sum#131, count#132] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#133] +Input [2]: [sum#167, count#168] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#169] (109) HashAggregate [codegen id : 2] -Input [2]: [sum#131, count#132] +Input [2]: [sum#167, count#168] Keys: [] -Functions [1]: [avg(UnscaledValue(ss_net_paid#52))] -Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#52))#134] -Results [1]: [cast((avg(UnscaledValue(ss_net_paid#52))#134 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#135] +Functions [1]: [avg(UnscaledValue(ss_net_paid#163))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_paid#163))#170] +Results [1]: [cast((avg(UnscaledValue(ss_net_paid#163))#170 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#171] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt index a74b232e8c667..5226fce4ef512 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt @@ -183,108 +183,108 @@ Aggregate Attributes [1]: [count(1)#17] Results [1]: [count(1)#17 AS amc#18] (29) Scan parquet default.web_sales -Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Output [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] (31) Filter [codegen id : 8] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Condition : ((isnotnull(ws_ship_hdemo_sk#20) AND isnotnull(ws_sold_time_sk#19)) AND isnotnull(ws_web_page_sk#21)) (32) Project [codegen id : 8] -Output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Output [3]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] (33) ReusedExchange [Reuses operator id: 9] -Output [1]: [wp_web_page_sk#5] +Output [1]: [wp_web_page_sk#23] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#3] -Right keys [1]: [wp_web_page_sk#5] +Left keys [1]: [ws_web_page_sk#21] +Right keys [1]: [wp_web_page_sk#23] Join condition: None (35) Project [codegen id : 8] -Output [2]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, wp_web_page_sk#5] +Output [2]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, wp_web_page_sk#23] (36) ReusedExchange [Reuses operator id: 16] -Output [1]: [hd_demo_sk#8] +Output [1]: [hd_demo_sk#24] (37) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_ship_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#8] +Left keys [1]: [ws_ship_hdemo_sk#20] +Right keys [1]: [hd_demo_sk#24] Join condition: None (38) Project [codegen id : 8] -Output [1]: [ws_sold_time_sk#1] -Input [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, hd_demo_sk#8] +Output [1]: [ws_sold_time_sk#19] +Input [3]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, hd_demo_sk#24] (39) Scan parquet default.time_dim -Output [2]: [t_time_sk#11, t_hour#12] +Output [2]: [t_time_sk#25, t_hour#26] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 7] -Input [2]: [t_time_sk#11, t_hour#12] +Input [2]: [t_time_sk#25, t_hour#26] (41) Filter [codegen id : 7] -Input [2]: [t_time_sk#11, t_hour#12] -Condition : (((isnotnull(t_hour#12) AND (t_hour#12 >= 19)) AND (t_hour#12 <= 20)) AND isnotnull(t_time_sk#11)) +Input [2]: [t_time_sk#25, t_hour#26] +Condition : (((isnotnull(t_hour#26) AND (t_hour#26 >= 19)) AND (t_hour#26 <= 20)) AND isnotnull(t_time_sk#25)) (42) Project [codegen id : 7] -Output [1]: [t_time_sk#11] -Input [2]: [t_time_sk#11, t_hour#12] +Output [1]: [t_time_sk#25] +Input [2]: [t_time_sk#25, t_hour#26] (43) BroadcastExchange -Input [1]: [t_time_sk#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [t_time_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_time_sk#1] -Right keys [1]: [t_time_sk#11] +Left keys [1]: [ws_sold_time_sk#19] +Right keys [1]: [t_time_sk#25] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ws_sold_time_sk#1, t_time_sk#11] +Input [2]: [ws_sold_time_sk#19, t_time_sk#25] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#20] -Results [1]: [count#21] +Aggregate Attributes [1]: [count#28] +Results [1]: [count#29] (47) Exchange -Input [1]: [count#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [count#29] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#30] (48) HashAggregate [codegen id : 9] -Input [1]: [count#21] +Input [1]: [count#29] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#23] -Results [1]: [count(1)#23 AS pmc#24] +Aggregate Attributes [1]: [count(1)#31] +Results [1]: [count(1)#31 AS pmc#32] (49) BroadcastExchange -Input [1]: [pmc#24] -Arguments: IdentityBroadcastMode, [id=#25] +Input [1]: [pmc#32] +Arguments: IdentityBroadcastMode, [id=#33] (50) BroadcastNestedLoopJoin [codegen id : 10] Join condition: None (51) Project [codegen id : 10] -Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#24 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#26] -Input [2]: [amc#18, pmc#24] +Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#32 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#34] +Input [2]: [amc#18, pmc#32] (52) Sort [codegen id : 10] -Input [1]: [am_pm_ratio#26] -Arguments: [am_pm_ratio#26 ASC NULLS FIRST], true, 0 +Input [1]: [am_pm_ratio#34] +Arguments: [am_pm_ratio#34 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt index 070b497ac4d18..e369a027040d3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt @@ -183,108 +183,108 @@ Aggregate Attributes [1]: [count(1)#17] Results [1]: [count(1)#17 AS amc#18] (29) Scan parquet default.web_sales -Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Output [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 8] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] (31) Filter [codegen id : 8] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] -Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] +Condition : ((isnotnull(ws_ship_hdemo_sk#20) AND isnotnull(ws_sold_time_sk#19)) AND isnotnull(ws_web_page_sk#21)) (32) Project [codegen id : 8] -Output [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Output [3]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, ws_sold_date_sk#22] (33) ReusedExchange [Reuses operator id: 9] -Output [1]: [hd_demo_sk#5] +Output [1]: [hd_demo_sk#23] (34) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_ship_hdemo_sk#2] -Right keys [1]: [hd_demo_sk#5] +Left keys [1]: [ws_ship_hdemo_sk#20] +Right keys [1]: [hd_demo_sk#23] Join condition: None (35) Project [codegen id : 8] -Output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] -Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] +Output [2]: [ws_sold_time_sk#19, ws_web_page_sk#21] +Input [4]: [ws_sold_time_sk#19, ws_ship_hdemo_sk#20, ws_web_page_sk#21, hd_demo_sk#23] (36) Scan parquet default.time_dim -Output [2]: [t_time_sk#8, t_hour#9] +Output [2]: [t_time_sk#24, t_hour#25] Batched: true Location [not included in comparison]/{warehouse_dir}/time_dim] PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 6] -Input [2]: [t_time_sk#8, t_hour#9] +Input [2]: [t_time_sk#24, t_hour#25] (38) Filter [codegen id : 6] -Input [2]: [t_time_sk#8, t_hour#9] -Condition : (((isnotnull(t_hour#9) AND (t_hour#9 >= 19)) AND (t_hour#9 <= 20)) AND isnotnull(t_time_sk#8)) +Input [2]: [t_time_sk#24, t_hour#25] +Condition : (((isnotnull(t_hour#25) AND (t_hour#25 >= 19)) AND (t_hour#25 <= 20)) AND isnotnull(t_time_sk#24)) (39) Project [codegen id : 6] -Output [1]: [t_time_sk#8] -Input [2]: [t_time_sk#8, t_hour#9] +Output [1]: [t_time_sk#24] +Input [2]: [t_time_sk#24, t_hour#25] (40) BroadcastExchange -Input [1]: [t_time_sk#8] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [t_time_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (41) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_sold_time_sk#1] -Right keys [1]: [t_time_sk#8] +Left keys [1]: [ws_sold_time_sk#19] +Right keys [1]: [t_time_sk#24] Join condition: None (42) Project [codegen id : 8] -Output [1]: [ws_web_page_sk#3] -Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#8] +Output [1]: [ws_web_page_sk#21] +Input [3]: [ws_sold_time_sk#19, ws_web_page_sk#21, t_time_sk#24] (43) ReusedExchange [Reuses operator id: 23] -Output [1]: [wp_web_page_sk#11] +Output [1]: [wp_web_page_sk#27] (44) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [ws_web_page_sk#3] -Right keys [1]: [wp_web_page_sk#11] +Left keys [1]: [ws_web_page_sk#21] +Right keys [1]: [wp_web_page_sk#27] Join condition: None (45) Project [codegen id : 8] Output: [] -Input [2]: [ws_web_page_sk#3, wp_web_page_sk#11] +Input [2]: [ws_web_page_sk#21, wp_web_page_sk#27] (46) HashAggregate [codegen id : 8] Input: [] Keys: [] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#20] -Results [1]: [count#21] +Aggregate Attributes [1]: [count#28] +Results [1]: [count#29] (47) Exchange -Input [1]: [count#21] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#22] +Input [1]: [count#29] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#30] (48) HashAggregate [codegen id : 9] -Input [1]: [count#21] +Input [1]: [count#29] Keys: [] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#23] -Results [1]: [count(1)#23 AS pmc#24] +Aggregate Attributes [1]: [count(1)#31] +Results [1]: [count(1)#31 AS pmc#32] (49) BroadcastExchange -Input [1]: [pmc#24] -Arguments: IdentityBroadcastMode, [id=#25] +Input [1]: [pmc#32] +Arguments: IdentityBroadcastMode, [id=#33] (50) BroadcastNestedLoopJoin [codegen id : 10] Join condition: None (51) Project [codegen id : 10] -Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#24 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#26] -Input [2]: [amc#18, pmc#24] +Output [1]: [CheckOverflow((promote_precision(cast(amc#18 as decimal(15,4))) / promote_precision(cast(pmc#32 as decimal(15,4)))), DecimalType(35,20), true) AS am_pm_ratio#34] +Input [2]: [amc#18, pmc#32] (52) Sort [codegen id : 10] -Input [1]: [am_pm_ratio#26] -Arguments: [am_pm_ratio#26 ASC NULLS FIRST], true, 0 +Input [1]: [am_pm_ratio#34] +Arguments: [am_pm_ratio#34 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt index c77590bf71044..8a5b567285939 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.sf100/explain.txt @@ -77,30 +77,30 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet default.web_sales -Output [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Output [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Input [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] (9) Project [codegen id : 3] -Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#10, ws_order_number#5 AS ws_order_number#5#11] -Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Output [2]: [ws_warehouse_sk#10, ws_order_number#11] +Input [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] (10) Exchange -Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] -Arguments: hashpartitioning(ws_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [2]: [ws_warehouse_sk#10, ws_order_number#11] +Arguments: hashpartitioning(ws_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 4] -Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] -Arguments: [ws_order_number#5#11 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#10, ws_order_number#11] +Arguments: [ws_order_number#11 ASC NULLS FIRST], false, 0 (12) SortMergeJoin Left keys [1]: [ws_order_number#5] -Right keys [1]: [ws_order_number#5#11] -Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#10) +Right keys [1]: [ws_order_number#11] +Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#10) (13) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -108,168 +108,168 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (14) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#13] +Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#14] (15) Sort [codegen id : 6] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: [cast(ws_order_number#5 as bigint) ASC NULLS FIRST], false, 0 (16) Scan parquet default.web_returns -Output [2]: [wr_order_number#14, wr_returned_date_sk#15] +Output [2]: [wr_order_number#15, wr_returned_date_sk#16] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] ReadSchema: struct (17) ColumnarToRow [codegen id : 7] -Input [2]: [wr_order_number#14, wr_returned_date_sk#15] +Input [2]: [wr_order_number#15, wr_returned_date_sk#16] (18) Project [codegen id : 7] -Output [1]: [wr_order_number#14] -Input [2]: [wr_order_number#14, wr_returned_date_sk#15] +Output [1]: [wr_order_number#15] +Input [2]: [wr_order_number#15, wr_returned_date_sk#16] (19) Exchange -Input [1]: [wr_order_number#14] -Arguments: hashpartitioning(wr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [1]: [wr_order_number#15] +Arguments: hashpartitioning(wr_order_number#15, 5), ENSURE_REQUIREMENTS, [id=#17] (20) Sort [codegen id : 8] -Input [1]: [wr_order_number#14] -Arguments: [wr_order_number#14 ASC NULLS FIRST], false, 0 +Input [1]: [wr_order_number#15] +Arguments: [wr_order_number#15 ASC NULLS FIRST], false, 0 (21) SortMergeJoin Left keys [1]: [cast(ws_order_number#5 as bigint)] -Right keys [1]: [wr_order_number#14] +Right keys [1]: [wr_order_number#15] Join condition: None (22) Scan parquet default.customer_address -Output [2]: [ca_address_sk#17, ca_state#18] +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 (23) ColumnarToRow [codegen id : 9] -Input [2]: [ca_address_sk#17, ca_state#18] +Input [2]: [ca_address_sk#18, ca_state#19] (24) Filter [codegen id : 9] -Input [2]: [ca_address_sk#17, ca_state#18] -Condition : ((isnotnull(ca_state#18) AND (ca_state#18 = IL)) AND isnotnull(ca_address_sk#17)) +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)) (25) Project [codegen id : 9] -Output [1]: [ca_address_sk#17] -Input [2]: [ca_address_sk#17, ca_state#18] +Output [1]: [ca_address_sk#18] +Input [2]: [ca_address_sk#18, ca_state#19] (26) BroadcastExchange -Input [1]: [ca_address_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [ca_address_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (27) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#17] +Right keys [1]: [ca_address_sk#18] Join condition: None (28) Project [codegen id : 12] Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#17] +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#18] (29) Scan parquet default.web_site -Output [2]: [web_site_sk#20, web_company_name#21] +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 (30) ColumnarToRow [codegen id : 10] -Input [2]: [web_site_sk#20, web_company_name#21] +Input [2]: [web_site_sk#21, web_company_name#22] (31) Filter [codegen id : 10] -Input [2]: [web_site_sk#20, web_company_name#21] -Condition : ((isnotnull(web_company_name#21) AND (web_company_name#21 = pri )) AND isnotnull(web_site_sk#20)) +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)) (32) Project [codegen id : 10] -Output [1]: [web_site_sk#20] -Input [2]: [web_site_sk#20, web_company_name#21] +Output [1]: [web_site_sk#21] +Input [2]: [web_site_sk#21, web_company_name#22] (33) BroadcastExchange -Input [1]: [web_site_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [web_site_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#20] +Right keys [1]: [web_site_sk#21] Join condition: None (35) Project [codegen id : 12] Output [4]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#20] +Input [6]: [ws_ship_date_sk#1, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#21] (36) Scan parquet default.date_dim -Output [2]: [d_date_sk#23, d_date#24] +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 (37) ColumnarToRow [codegen id : 11] -Input [2]: [d_date_sk#23, d_date#24] +Input [2]: [d_date_sk#24, d_date#25] (38) Filter [codegen id : 11] -Input [2]: [d_date_sk#23, d_date#24] -Condition : (((isnotnull(d_date#24) AND (d_date#24 >= 1999-02-01)) AND (d_date#24 <= 1999-04-02)) AND isnotnull(d_date_sk#23)) +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)) (39) Project [codegen id : 11] -Output [1]: [d_date_sk#23] -Input [2]: [d_date_sk#23, d_date#24] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_date#25] (40) BroadcastExchange -Input [1]: [d_date_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#23] +Right keys [1]: [d_date_sk#24] Join condition: None (42) Project [codegen id : 12] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [5]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#23] +Input [5]: [ws_ship_date_sk#1, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#24] (43) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] -Results [3]: [ws_order_number#5, sum#28, sum#29] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28] +Results [3]: [ws_order_number#5, sum#29, sum#30] (44) Exchange -Input [3]: [ws_order_number#5, sum#28, sum#29] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [ws_order_number#5, sum#29, sum#30] +Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#31] (45) HashAggregate [codegen id : 13] -Input [3]: [ws_order_number#5, sum#28, sum#29] +Input [3]: [ws_order_number#5, sum#29, sum#30] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] -Results [3]: [ws_order_number#5, sum#28, sum#29] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28] +Results [3]: [ws_order_number#5, sum#29, sum#30] (46) HashAggregate [codegen id : 13] -Input [3]: [ws_order_number#5, sum#28, sum#29] +Input [3]: [ws_order_number#5, sum#29, sum#30] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] -Results [3]: [sum#28, sum#29, count#32] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28, count(ws_order_number#5)#32] +Results [3]: [sum#29, sum#30, count#33] (47) Exchange -Input [3]: [sum#28, sum#29, count#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [sum#29, sum#30, count#33] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#34] (48) HashAggregate [codegen id : 14] -Input [3]: [sum#28, sum#29, count#32] +Input [3]: [sum#29, sum#30, count#33] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] -Results [3]: [count(ws_order_number#5)#31 AS order count #34, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#26,17,2) AS total shipping cost #35, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#27,17,2) AS total net profit #36] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28, count(ws_order_number#5)#32] +Results [3]: [count(ws_order_number#5)#32 AS order count #35, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#27,17,2) AS total shipping cost #36, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#28,17,2) AS total net profit #37] (49) Sort [codegen id : 14] -Input [3]: [order count #34, total shipping cost #35, total net profit #36] -Arguments: [order count #34 ASC NULLS FIRST], true, 0 +Input [3]: [order count #35, total shipping cost #36, total net profit #37] +Arguments: [order count #35 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt index 9558a01423452..042d4a3e347a2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/explain.txt @@ -77,30 +77,30 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 (7) Scan parquet default.web_sales -Output [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Output [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Input [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] (9) Project [codegen id : 3] -Output [2]: [ws_warehouse_sk#4 AS ws_warehouse_sk#4#10, ws_order_number#5 AS ws_order_number#5#11] -Input [3]: [ws_warehouse_sk#4, ws_order_number#5, ws_sold_date_sk#8] +Output [2]: [ws_warehouse_sk#10, ws_order_number#11] +Input [3]: [ws_warehouse_sk#10, ws_order_number#11, ws_sold_date_sk#12] (10) Exchange -Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] -Arguments: hashpartitioning(ws_order_number#5#11, 5), ENSURE_REQUIREMENTS, [id=#12] +Input [2]: [ws_warehouse_sk#10, ws_order_number#11] +Arguments: hashpartitioning(ws_order_number#11, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 4] -Input [2]: [ws_warehouse_sk#4#10, ws_order_number#5#11] -Arguments: [ws_order_number#5#11 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#10, ws_order_number#11] +Arguments: [ws_order_number#11 ASC NULLS FIRST], false, 0 (12) SortMergeJoin Left keys [1]: [ws_order_number#5] -Right keys [1]: [ws_order_number#5#11] -Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#4#10) +Right keys [1]: [ws_order_number#11] +Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#10) (13) Project [codegen id : 5] Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] @@ -108,168 +108,168 @@ Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse (14) Exchange Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#13] +Arguments: hashpartitioning(cast(ws_order_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#14] (15) Sort [codegen id : 6] Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Arguments: [cast(ws_order_number#5 as bigint) ASC NULLS FIRST], false, 0 (16) Scan parquet default.web_returns -Output [2]: [wr_order_number#14, wr_returned_date_sk#15] +Output [2]: [wr_order_number#15, wr_returned_date_sk#16] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] ReadSchema: struct (17) ColumnarToRow [codegen id : 7] -Input [2]: [wr_order_number#14, wr_returned_date_sk#15] +Input [2]: [wr_order_number#15, wr_returned_date_sk#16] (18) Project [codegen id : 7] -Output [1]: [wr_order_number#14] -Input [2]: [wr_order_number#14, wr_returned_date_sk#15] +Output [1]: [wr_order_number#15] +Input [2]: [wr_order_number#15, wr_returned_date_sk#16] (19) Exchange -Input [1]: [wr_order_number#14] -Arguments: hashpartitioning(wr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16] +Input [1]: [wr_order_number#15] +Arguments: hashpartitioning(wr_order_number#15, 5), ENSURE_REQUIREMENTS, [id=#17] (20) Sort [codegen id : 8] -Input [1]: [wr_order_number#14] -Arguments: [wr_order_number#14 ASC NULLS FIRST], false, 0 +Input [1]: [wr_order_number#15] +Arguments: [wr_order_number#15 ASC NULLS FIRST], false, 0 (21) SortMergeJoin Left keys [1]: [cast(ws_order_number#5 as bigint)] -Right keys [1]: [wr_order_number#14] +Right keys [1]: [wr_order_number#15] Join condition: None (22) Scan parquet default.date_dim -Output [2]: [d_date_sk#17, d_date#18] +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 (23) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#17, d_date#18] +Input [2]: [d_date_sk#18, d_date#19] (24) Filter [codegen id : 9] -Input [2]: [d_date_sk#17, d_date#18] -Condition : (((isnotnull(d_date#18) AND (d_date#18 >= 1999-02-01)) AND (d_date#18 <= 1999-04-02)) AND isnotnull(d_date_sk#17)) +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)) (25) Project [codegen id : 9] -Output [1]: [d_date_sk#17] -Input [2]: [d_date_sk#17, d_date#18] +Output [1]: [d_date_sk#18] +Input [2]: [d_date_sk#18, d_date#19] (26) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [d_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (27) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#17] +Right keys [1]: [d_date_sk#18] Join condition: None (28) Project [codegen id : 12] Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#17] +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#18] (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +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 (30) ColumnarToRow [codegen id : 10] -Input [2]: [ca_address_sk#20, ca_state#21] +Input [2]: [ca_address_sk#21, ca_state#22] (31) Filter [codegen id : 10] -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : ((isnotnull(ca_state#21) AND (ca_state#21 = IL)) AND isnotnull(ca_address_sk#20)) +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)) (32) Project [codegen id : 10] -Output [1]: [ca_address_sk#20] -Input [2]: [ca_address_sk#20, ca_state#21] +Output [1]: [ca_address_sk#21] +Input [2]: [ca_address_sk#21, ca_state#22] (33) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [ca_address_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (34) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#21] Join condition: None (35) Project [codegen id : 12] Output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#20] +Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#21] (36) Scan parquet default.web_site -Output [2]: [web_site_sk#23, web_company_name#24] +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 (37) ColumnarToRow [codegen id : 11] -Input [2]: [web_site_sk#23, web_company_name#24] +Input [2]: [web_site_sk#24, web_company_name#25] (38) Filter [codegen id : 11] -Input [2]: [web_site_sk#23, web_company_name#24] -Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri )) AND isnotnull(web_site_sk#23)) +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)) (39) Project [codegen id : 11] -Output [1]: [web_site_sk#23] -Input [2]: [web_site_sk#23, web_company_name#24] +Output [1]: [web_site_sk#24] +Input [2]: [web_site_sk#24, web_company_name#25] (40) BroadcastExchange -Input [1]: [web_site_sk#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] +Input [1]: [web_site_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (41) BroadcastHashJoin [codegen id : 12] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#23] +Right keys [1]: [web_site_sk#24] Join condition: None (42) Project [codegen id : 12] Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] -Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#23] +Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#24] (43) HashAggregate [codegen id : 12] Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] Keys [1]: [ws_order_number#5] Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] -Results [3]: [ws_order_number#5, sum#28, sum#29] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28] +Results [3]: [ws_order_number#5, sum#29, sum#30] (44) Exchange -Input [3]: [ws_order_number#5, sum#28, sum#29] -Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [ws_order_number#5, sum#29, sum#30] +Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [id=#31] (45) HashAggregate [codegen id : 13] -Input [3]: [ws_order_number#5, sum#28, sum#29] +Input [3]: [ws_order_number#5, sum#29, sum#30] Keys [1]: [ws_order_number#5] Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27] -Results [3]: [ws_order_number#5, sum#28, sum#29] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28] +Results [3]: [ws_order_number#5, sum#29, sum#30] (46) HashAggregate [codegen id : 13] -Input [3]: [ws_order_number#5, sum#28, sum#29] +Input [3]: [ws_order_number#5, sum#29, sum#30] Keys: [] Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] -Results [3]: [sum#28, sum#29, count#32] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28, count(ws_order_number#5)#32] +Results [3]: [sum#29, sum#30, count#33] (47) Exchange -Input [3]: [sum#28, sum#29, count#32] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#33] +Input [3]: [sum#29, sum#30, count#33] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#34] (48) HashAggregate [codegen id : 14] -Input [3]: [sum#28, sum#29, count#32] +Input [3]: [sum#29, sum#30, count#33] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#26, sum(UnscaledValue(ws_net_profit#7))#27, count(ws_order_number#5)#31] -Results [3]: [count(ws_order_number#5)#31 AS order count #34, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#26,17,2) AS total shipping cost #35, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#27,17,2) AS total net profit #36] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#27, sum(UnscaledValue(ws_net_profit#7))#28, count(ws_order_number#5)#32] +Results [3]: [count(ws_order_number#5)#32 AS order count #35, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#27,17,2) AS total shipping cost #36, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#28,17,2) AS total net profit #37] (49) Sort [codegen id : 14] -Input [3]: [order count #34, total shipping cost #35, total net profit #36] -Arguments: [order count #34 ASC NULLS FIRST], true, 0 +Input [3]: [order count #35, total shipping cost #36, total net profit #37] +Arguments: [order count #35 ASC NULLS FIRST], true, 0 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 320a93e19bb27..ee82855f8c010 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 @@ -92,261 +92,261 @@ Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (7) Scan parquet default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] (9) Filter [codegen id : 3] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] -Condition : (isnotnull(ws_order_number#4) AND isnotnull(ws_warehouse_sk#9)) +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] +Condition : (isnotnull(ws_order_number#10) AND isnotnull(ws_warehouse_sk#9)) (10) Project [codegen id : 3] -Output [2]: [ws_warehouse_sk#9, ws_order_number#4] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Output [2]: [ws_warehouse_sk#9, ws_order_number#10] +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] (11) Exchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#10] +Input [2]: [ws_warehouse_sk#9, ws_order_number#10] +Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, [id=#12] (12) Sort [codegen id : 4] -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: [ws_order_number#4 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 (13) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] +Output [2]: [ws_warehouse_sk#13, ws_order_number#14] (14) Sort [codegen id : 6] -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#12 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 (15) SortMergeJoin [codegen id : 7] -Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#12] -Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#11) +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) (16) Project [codegen id : 7] -Output [1]: [ws_order_number#4 AS ws_order_number#4#13] -Input [4]: [ws_warehouse_sk#9, ws_order_number#4, ws_warehouse_sk#11, ws_order_number#12] +Output [1]: [ws_order_number#10] +Input [4]: [ws_warehouse_sk#9, ws_order_number#10, ws_warehouse_sk#13, ws_order_number#14] (17) SortMergeJoin Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#4#13] +Right keys [1]: [ws_order_number#10] Join condition: None (18) Exchange Input [6]: [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] -Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#15] (19) Sort [codegen id : 8] Input [6]: [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] Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 (20) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#9, ws_order_number#4] +Output [2]: [ws_warehouse_sk#16, ws_order_number#17] (21) Sort [codegen id : 10] -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#16, ws_order_number#17] +Arguments: [ws_order_number#17 ASC NULLS FIRST], false, 0 (22) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#15, ws_order_number#16] +Output [2]: [ws_warehouse_sk#18, ws_order_number#19] (23) Sort [codegen id : 12] -Input [2]: [ws_warehouse_sk#15, ws_order_number#16] -Arguments: [ws_order_number#16 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#18, ws_order_number#19] +Arguments: [ws_order_number#19 ASC NULLS FIRST], false, 0 (24) SortMergeJoin [codegen id : 13] -Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#16] -Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#15) +Left keys [1]: [ws_order_number#17] +Right keys [1]: [ws_order_number#19] +Join condition: NOT (ws_warehouse_sk#16 = ws_warehouse_sk#18) (25) Project [codegen id : 13] -Output [1]: [ws_order_number#4] -Input [4]: [ws_warehouse_sk#9, ws_order_number#4, ws_warehouse_sk#15, ws_order_number#16] +Output [1]: [ws_order_number#17] +Input [4]: [ws_warehouse_sk#16, ws_order_number#17, ws_warehouse_sk#18, ws_order_number#19] (26) Exchange -Input [1]: [ws_order_number#4] -Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#17] +Input [1]: [ws_order_number#17] +Arguments: hashpartitioning(cast(ws_order_number#17 as bigint), 5), ENSURE_REQUIREMENTS, [id=#20] (27) Sort [codegen id : 14] -Input [1]: [ws_order_number#4] -Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 +Input [1]: [ws_order_number#17] +Arguments: [cast(ws_order_number#17 as bigint) ASC NULLS FIRST], false, 0 (28) Scan parquet default.web_returns -Output [2]: [wr_order_number#18, wr_returned_date_sk#19] +Output [2]: [wr_order_number#21, wr_returned_date_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct (29) ColumnarToRow [codegen id : 15] -Input [2]: [wr_order_number#18, wr_returned_date_sk#19] +Input [2]: [wr_order_number#21, wr_returned_date_sk#22] (30) Filter [codegen id : 15] -Input [2]: [wr_order_number#18, wr_returned_date_sk#19] -Condition : isnotnull(wr_order_number#18) +Input [2]: [wr_order_number#21, wr_returned_date_sk#22] +Condition : isnotnull(wr_order_number#21) (31) Project [codegen id : 15] -Output [1]: [wr_order_number#18] -Input [2]: [wr_order_number#18, wr_returned_date_sk#19] +Output [1]: [wr_order_number#21] +Input [2]: [wr_order_number#21, wr_returned_date_sk#22] (32) Exchange -Input [1]: [wr_order_number#18] -Arguments: hashpartitioning(wr_order_number#18, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [wr_order_number#21] +Arguments: hashpartitioning(wr_order_number#21, 5), ENSURE_REQUIREMENTS, [id=#23] (33) Sort [codegen id : 16] -Input [1]: [wr_order_number#18] -Arguments: [wr_order_number#18 ASC NULLS FIRST], false, 0 +Input [1]: [wr_order_number#21] +Arguments: [wr_order_number#21 ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 17] -Left keys [1]: [cast(ws_order_number#4 as bigint)] -Right keys [1]: [wr_order_number#18] +Left keys [1]: [cast(ws_order_number#17 as bigint)] +Right keys [1]: [wr_order_number#21] Join condition: None (35) Project [codegen id : 17] -Output [1]: [wr_order_number#18] -Input [2]: [ws_order_number#4, wr_order_number#18] +Output [1]: [wr_order_number#21] +Input [2]: [ws_order_number#17, wr_order_number#21] (36) SortMergeJoin Left keys [1]: [cast(ws_order_number#4 as bigint)] -Right keys [1]: [wr_order_number#18] +Right keys [1]: [wr_order_number#21] Join condition: None (37) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] +Output [2]: [ca_address_sk#24, ca_state#25] 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 (38) ColumnarToRow [codegen id : 18] -Input [2]: [ca_address_sk#21, ca_state#22] +Input [2]: [ca_address_sk#24, ca_state#25] (39) Filter [codegen id : 18] -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)) +Input [2]: [ca_address_sk#24, ca_state#25] +Condition : ((isnotnull(ca_state#25) AND (ca_state#25 = IL)) AND isnotnull(ca_address_sk#24)) (40) Project [codegen id : 18] -Output [1]: [ca_address_sk#21] -Input [2]: [ca_address_sk#21, ca_state#22] +Output [1]: [ca_address_sk#24] +Input [2]: [ca_address_sk#24, ca_state#25] (41) BroadcastExchange -Input [1]: [ca_address_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [ca_address_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (42) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#24] Join condition: None (43) Project [codegen id : 21] 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#21] +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#24] (44) Scan parquet default.web_site -Output [2]: [web_site_sk#24, web_company_name#25] +Output [2]: [web_site_sk#27, web_company_name#28] 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 (45) ColumnarToRow [codegen id : 19] -Input [2]: [web_site_sk#24, web_company_name#25] +Input [2]: [web_site_sk#27, web_company_name#28] (46) Filter [codegen id : 19] -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)) +Input [2]: [web_site_sk#27, web_company_name#28] +Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri )) AND isnotnull(web_site_sk#27)) (47) Project [codegen id : 19] -Output [1]: [web_site_sk#24] -Input [2]: [web_site_sk#24, web_company_name#25] +Output [1]: [web_site_sk#27] +Input [2]: [web_site_sk#27, web_company_name#28] (48) BroadcastExchange -Input [1]: [web_site_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [web_site_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (49) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#24] +Right keys [1]: [web_site_sk#27] Join condition: None (50) Project [codegen id : 21] 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#24] +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#27] (51) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_date#28] +Output [2]: [d_date_sk#30, d_date#31] 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 (52) ColumnarToRow [codegen id : 20] -Input [2]: [d_date_sk#27, d_date#28] +Input [2]: [d_date_sk#30, d_date#31] (53) Filter [codegen id : 20] -Input [2]: [d_date_sk#27, d_date#28] -Condition : (((isnotnull(d_date#28) AND (d_date#28 >= 1999-02-01)) AND (d_date#28 <= 1999-04-02)) AND isnotnull(d_date_sk#27)) +Input [2]: [d_date_sk#30, d_date#31] +Condition : (((isnotnull(d_date#31) AND (d_date#31 >= 1999-02-01)) AND (d_date#31 <= 1999-04-02)) AND isnotnull(d_date_sk#30)) (54) Project [codegen id : 20] -Output [1]: [d_date_sk#27] -Input [2]: [d_date_sk#27, d_date#28] +Output [1]: [d_date_sk#30] +Input [2]: [d_date_sk#30, d_date#31] (55) BroadcastExchange -Input [1]: [d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] +Input [1]: [d_date_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] (56) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#27] +Right keys [1]: [d_date_sk#30] Join condition: None (57) Project [codegen id : 21] 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#27] +Input [5]: [ws_ship_date_sk#1, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#30] (58) HashAggregate [codegen id : 21] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31] -Results [3]: [ws_order_number#4, sum#32, sum#33] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34] +Results [3]: [ws_order_number#4, sum#35, sum#36] (59) Exchange -Input [3]: [ws_order_number#4, sum#32, sum#33] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [ws_order_number#4, sum#35, sum#36] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#37] (60) HashAggregate [codegen id : 22] -Input [3]: [ws_order_number#4, sum#32, sum#33] +Input [3]: [ws_order_number#4, sum#35, sum#36] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31] -Results [3]: [ws_order_number#4, sum#32, sum#33] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34] +Results [3]: [ws_order_number#4, sum#35, sum#36] (61) HashAggregate [codegen id : 22] -Input [3]: [ws_order_number#4, sum#32, sum#33] +Input [3]: [ws_order_number#4, sum#35, sum#36] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31, count(ws_order_number#4)#35] -Results [3]: [sum#32, sum#33, count#36] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34, count(ws_order_number#4)#38] +Results [3]: [sum#35, sum#36, count#39] (62) Exchange -Input [3]: [sum#32, sum#33, count#36] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [sum#35, sum#36, count#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] (63) HashAggregate [codegen id : 23] -Input [3]: [sum#32, sum#33, count#36] +Input [3]: [sum#35, sum#36, count#39] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31, 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))#30,17,2) AS total shipping cost #39, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#31,17,2) AS total net profit #40] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34, count(ws_order_number#4)#38] +Results [3]: [count(ws_order_number#4)#38 AS order count #41, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#33,17,2) AS total shipping cost #42, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#34,17,2) AS total net profit #43] (64) Sort [codegen id : 23] -Input [3]: [order count #38, total shipping cost #39, total net profit #40] -Arguments: [order count #38 ASC NULLS FIRST], true, 0 +Input [3]: [order count #41, total shipping cost #42, total net profit #43] +Arguments: [order count #41 ASC NULLS FIRST], true, 0 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 2f719a4be1630..73f7f88aeb68d 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 @@ -92,261 +92,261 @@ Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_num Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 (7) Scan parquet default.web_sales -Output [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] ReadSchema: struct (8) ColumnarToRow [codegen id : 3] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] (9) Filter [codegen id : 3] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] -Condition : (isnotnull(ws_order_number#4) AND isnotnull(ws_warehouse_sk#9)) +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] +Condition : (isnotnull(ws_order_number#10) AND isnotnull(ws_warehouse_sk#9)) (10) Project [codegen id : 3] -Output [2]: [ws_warehouse_sk#9, ws_order_number#4] -Input [3]: [ws_warehouse_sk#9, ws_order_number#4, ws_sold_date_sk#7] +Output [2]: [ws_warehouse_sk#9, ws_order_number#10] +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] (11) Exchange -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#10] +Input [2]: [ws_warehouse_sk#9, ws_order_number#10] +Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, [id=#12] (12) Sort [codegen id : 4] -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: [ws_order_number#4 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 (13) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#11, ws_order_number#12] +Output [2]: [ws_warehouse_sk#13, ws_order_number#14] (14) Sort [codegen id : 6] -Input [2]: [ws_warehouse_sk#11, ws_order_number#12] -Arguments: [ws_order_number#12 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 (15) SortMergeJoin [codegen id : 7] -Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#12] -Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#11) +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) (16) Project [codegen id : 7] -Output [1]: [ws_order_number#4 AS ws_order_number#4#13] -Input [4]: [ws_warehouse_sk#9, ws_order_number#4, ws_warehouse_sk#11, ws_order_number#12] +Output [1]: [ws_order_number#10] +Input [4]: [ws_warehouse_sk#9, ws_order_number#10, ws_warehouse_sk#13, ws_order_number#14] (17) SortMergeJoin Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#4#13] +Right keys [1]: [ws_order_number#10] Join condition: None (18) Exchange Input [6]: [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] -Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#15] (19) Sort [codegen id : 8] Input [6]: [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] Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 (20) Scan parquet default.web_returns -Output [2]: [wr_order_number#15, wr_returned_date_sk#16] +Output [2]: [wr_order_number#16, wr_returned_date_sk#17] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number)] ReadSchema: struct (21) ColumnarToRow [codegen id : 9] -Input [2]: [wr_order_number#15, wr_returned_date_sk#16] +Input [2]: [wr_order_number#16, wr_returned_date_sk#17] (22) Filter [codegen id : 9] -Input [2]: [wr_order_number#15, wr_returned_date_sk#16] -Condition : isnotnull(wr_order_number#15) +Input [2]: [wr_order_number#16, wr_returned_date_sk#17] +Condition : isnotnull(wr_order_number#16) (23) Project [codegen id : 9] -Output [1]: [wr_order_number#15] -Input [2]: [wr_order_number#15, wr_returned_date_sk#16] +Output [1]: [wr_order_number#16] +Input [2]: [wr_order_number#16, wr_returned_date_sk#17] (24) Exchange -Input [1]: [wr_order_number#15] -Arguments: hashpartitioning(wr_order_number#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Input [1]: [wr_order_number#16] +Arguments: hashpartitioning(wr_order_number#16, 5), ENSURE_REQUIREMENTS, [id=#18] (25) Sort [codegen id : 10] -Input [1]: [wr_order_number#15] -Arguments: [wr_order_number#15 ASC NULLS FIRST], false, 0 +Input [1]: [wr_order_number#16] +Arguments: [wr_order_number#16 ASC NULLS FIRST], false, 0 (26) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#9, ws_order_number#4] +Output [2]: [ws_warehouse_sk#19, ws_order_number#20] (27) Sort [codegen id : 12] -Input [2]: [ws_warehouse_sk#9, ws_order_number#4] -Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 +Input [2]: [ws_warehouse_sk#19, ws_order_number#20] +Arguments: [ws_order_number#20 ASC NULLS FIRST], false, 0 (28) ReusedExchange [Reuses operator id: 11] -Output [2]: [ws_warehouse_sk#18, ws_order_number#19] +Output [2]: [ws_warehouse_sk#21, ws_order_number#22] (29) Sort [codegen id : 14] -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#21, ws_order_number#22] +Arguments: [ws_order_number#22 ASC NULLS FIRST], false, 0 (30) SortMergeJoin [codegen id : 15] -Left keys [1]: [ws_order_number#4] -Right keys [1]: [ws_order_number#19] -Join condition: NOT (ws_warehouse_sk#9 = ws_warehouse_sk#18) +Left keys [1]: [ws_order_number#20] +Right keys [1]: [ws_order_number#22] +Join condition: NOT (ws_warehouse_sk#19 = ws_warehouse_sk#21) (31) Project [codegen id : 15] -Output [1]: [ws_order_number#4] -Input [4]: [ws_warehouse_sk#9, ws_order_number#4, ws_warehouse_sk#18, ws_order_number#19] +Output [1]: [ws_order_number#20] +Input [4]: [ws_warehouse_sk#19, ws_order_number#20, ws_warehouse_sk#21, ws_order_number#22] (32) Exchange -Input [1]: [ws_order_number#4] -Arguments: hashpartitioning(cast(ws_order_number#4 as bigint), 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [ws_order_number#20] +Arguments: hashpartitioning(cast(ws_order_number#20 as bigint), 5), ENSURE_REQUIREMENTS, [id=#23] (33) Sort [codegen id : 16] -Input [1]: [ws_order_number#4] -Arguments: [cast(ws_order_number#4 as bigint) ASC NULLS FIRST], false, 0 +Input [1]: [ws_order_number#20] +Arguments: [cast(ws_order_number#20 as bigint) ASC NULLS FIRST], false, 0 (34) SortMergeJoin [codegen id : 17] -Left keys [1]: [wr_order_number#15] -Right keys [1]: [cast(ws_order_number#4 as bigint)] +Left keys [1]: [wr_order_number#16] +Right keys [1]: [cast(ws_order_number#20 as bigint)] Join condition: None (35) Project [codegen id : 17] -Output [1]: [wr_order_number#15] -Input [2]: [wr_order_number#15, ws_order_number#4] +Output [1]: [wr_order_number#16] +Input [2]: [wr_order_number#16, ws_order_number#20] (36) SortMergeJoin Left keys [1]: [cast(ws_order_number#4 as bigint)] -Right keys [1]: [wr_order_number#15] +Right keys [1]: [wr_order_number#16] Join condition: None (37) Scan parquet default.date_dim -Output [2]: [d_date_sk#21, d_date#22] +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 (38) ColumnarToRow [codegen id : 18] -Input [2]: [d_date_sk#21, d_date#22] +Input [2]: [d_date_sk#24, d_date#25] (39) Filter [codegen id : 18] -Input [2]: [d_date_sk#21, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 1999-02-01)) AND (d_date#22 <= 1999-04-02)) AND isnotnull(d_date_sk#21)) +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)) (40) Project [codegen id : 18] -Output [1]: [d_date_sk#21] -Input [2]: [d_date_sk#21, d_date#22] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_date#25] (41) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (42) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_date_sk#1] -Right keys [1]: [d_date_sk#21] +Right keys [1]: [d_date_sk#24] Join condition: None (43) Project [codegen id : 21] 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#21] +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#24] (44) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#25] +Output [2]: [ca_address_sk#27, ca_state#28] 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 (45) ColumnarToRow [codegen id : 19] -Input [2]: [ca_address_sk#24, ca_state#25] +Input [2]: [ca_address_sk#27, ca_state#28] (46) Filter [codegen id : 19] -Input [2]: [ca_address_sk#24, ca_state#25] -Condition : ((isnotnull(ca_state#25) AND (ca_state#25 = IL)) AND isnotnull(ca_address_sk#24)) +Input [2]: [ca_address_sk#27, ca_state#28] +Condition : ((isnotnull(ca_state#28) AND (ca_state#28 = IL)) AND isnotnull(ca_address_sk#27)) (47) Project [codegen id : 19] -Output [1]: [ca_address_sk#24] -Input [2]: [ca_address_sk#24, ca_state#25] +Output [1]: [ca_address_sk#27] +Input [2]: [ca_address_sk#27, ca_state#28] (48) BroadcastExchange -Input [1]: [ca_address_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [ca_address_sk#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] (49) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_ship_addr_sk#2] -Right keys [1]: [ca_address_sk#24] +Right keys [1]: [ca_address_sk#27] Join condition: None (50) Project [codegen id : 21] 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#24] +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#27] (51) Scan parquet default.web_site -Output [2]: [web_site_sk#27, web_company_name#28] +Output [2]: [web_site_sk#30, web_company_name#31] 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 (52) ColumnarToRow [codegen id : 20] -Input [2]: [web_site_sk#27, web_company_name#28] +Input [2]: [web_site_sk#30, web_company_name#31] (53) Filter [codegen id : 20] -Input [2]: [web_site_sk#27, web_company_name#28] -Condition : ((isnotnull(web_company_name#28) AND (web_company_name#28 = pri )) AND isnotnull(web_site_sk#27)) +Input [2]: [web_site_sk#30, web_company_name#31] +Condition : ((isnotnull(web_company_name#31) AND (web_company_name#31 = pri )) AND isnotnull(web_site_sk#30)) (54) Project [codegen id : 20] -Output [1]: [web_site_sk#27] -Input [2]: [web_site_sk#27, web_company_name#28] +Output [1]: [web_site_sk#30] +Input [2]: [web_site_sk#30, web_company_name#31] (55) BroadcastExchange -Input [1]: [web_site_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] +Input [1]: [web_site_sk#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#32] (56) BroadcastHashJoin [codegen id : 21] Left keys [1]: [ws_web_site_sk#3] -Right keys [1]: [web_site_sk#27] +Right keys [1]: [web_site_sk#30] Join condition: None (57) Project [codegen id : 21] 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#27] +Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#30] (58) HashAggregate [codegen id : 21] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31] -Results [3]: [ws_order_number#4, sum#32, sum#33] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34] +Results [3]: [ws_order_number#4, sum#35, sum#36] (59) Exchange -Input [3]: [ws_order_number#4, sum#32, sum#33] -Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [3]: [ws_order_number#4, sum#35, sum#36] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [id=#37] (60) HashAggregate [codegen id : 22] -Input [3]: [ws_order_number#4, sum#32, sum#33] +Input [3]: [ws_order_number#4, sum#35, sum#36] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31] -Results [3]: [ws_order_number#4, sum#32, sum#33] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34] +Results [3]: [ws_order_number#4, sum#35, sum#36] (61) HashAggregate [codegen id : 22] -Input [3]: [ws_order_number#4, sum#32, sum#33] +Input [3]: [ws_order_number#4, sum#35, sum#36] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31, count(ws_order_number#4)#35] -Results [3]: [sum#32, sum#33, count#36] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34, count(ws_order_number#4)#38] +Results [3]: [sum#35, sum#36, count#39] (62) Exchange -Input [3]: [sum#32, sum#33, count#36] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [sum#35, sum#36, count#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] (63) HashAggregate [codegen id : 23] -Input [3]: [sum#32, sum#33, count#36] +Input [3]: [sum#35, sum#36, count#39] 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))#30, sum(UnscaledValue(ws_net_profit#6))#31, 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))#30,17,2) AS total shipping cost #39, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#31,17,2) AS total net profit #40] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#33, sum(UnscaledValue(ws_net_profit#6))#34, count(ws_order_number#4)#38] +Results [3]: [count(ws_order_number#4)#38 AS order count #41, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#33,17,2) AS total shipping cost #42, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#34,17,2) AS total net profit #43] (64) Sort [codegen id : 23] -Input [3]: [order count #38, total shipping cost #39, total net profit #40] -Arguments: [order count #38 ASC NULLS FIRST], true, 0 +Input [3]: [order count #41, total shipping cost #42, total net profit #43] +Arguments: [order count #41 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt index 245357ef5560c..c01f3465ed693 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.sf100/explain.txt @@ -102,16 +102,16 @@ ReadSchema: struct Input [3]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13] (16) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#14] (17) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#5] +Right keys [1]: [d_date_sk#14] Join condition: None (18) Project [codegen id : 5] Output [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#5] +Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#14] (19) HashAggregate [codegen id : 5] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] @@ -122,45 +122,45 @@ Results [2]: [cs_bill_customer_sk#11, cs_item_sk#12] (20) Exchange Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#15] (21) HashAggregate [codegen id : 6] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Keys [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Functions: [] Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#11 AS customer_sk#15, cs_item_sk#12 AS item_sk#16] +Results [2]: [cs_bill_customer_sk#11 AS customer_sk#16, cs_item_sk#12 AS item_sk#17] (22) Sort [codegen id : 6] -Input [2]: [customer_sk#15, item_sk#16] -Arguments: [customer_sk#15 ASC NULLS FIRST, item_sk#16 ASC NULLS FIRST], false, 0 +Input [2]: [customer_sk#16, item_sk#17] +Arguments: [customer_sk#16 ASC NULLS FIRST, item_sk#17 ASC NULLS FIRST], false, 0 (23) SortMergeJoin Left keys [2]: [customer_sk#9, item_sk#10] -Right keys [2]: [customer_sk#15, item_sk#16] +Right keys [2]: [customer_sk#16, item_sk#17] Join condition: None (24) Project [codegen id : 7] -Output [2]: [customer_sk#9, customer_sk#15] -Input [4]: [customer_sk#9, item_sk#10, customer_sk#15, item_sk#16] +Output [2]: [customer_sk#9, customer_sk#16] +Input [4]: [customer_sk#9, item_sk#10, customer_sk#16, item_sk#17] (25) HashAggregate [codegen id : 7] -Input [2]: [customer_sk#9, customer_sk#15] +Input [2]: [customer_sk#9, customer_sk#16] Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum#17, sum#18, sum#19] -Results [3]: [sum#20, sum#21, sum#22] +Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum#18, sum#19, sum#20] +Results [3]: [sum#21, sum#22, sum#23] (26) Exchange -Input [3]: [sum#20, sum#21, sum#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [sum#21, sum#22, sum#23] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] (27) HashAggregate [codegen id : 8] -Input [3]: [sum#20, sum#21, sum#22] +Input [3]: [sum#21, sum#22, sum#23] Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26] -Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24 AS store_only#27, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25 AS catalog_only#28, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26 AS store_and_catalog#29] +Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27] +Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25 AS store_only#28, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26 AS catalog_only#29, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27 AS store_and_catalog#30] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt index 245357ef5560c..c01f3465ed693 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/explain.txt @@ -102,16 +102,16 @@ ReadSchema: struct Input [3]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13] (16) ReusedExchange [Reuses operator id: 7] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#14] (17) BroadcastHashJoin [codegen id : 5] Left keys [1]: [cs_sold_date_sk#13] -Right keys [1]: [d_date_sk#5] +Right keys [1]: [d_date_sk#14] Join condition: None (18) Project [codegen id : 5] Output [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#5] +Input [4]: [cs_bill_customer_sk#11, cs_item_sk#12, cs_sold_date_sk#13, d_date_sk#14] (19) HashAggregate [codegen id : 5] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] @@ -122,45 +122,45 @@ Results [2]: [cs_bill_customer_sk#11, cs_item_sk#12] (20) Exchange Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] -Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#14] +Arguments: hashpartitioning(cs_bill_customer_sk#11, cs_item_sk#12, 5), ENSURE_REQUIREMENTS, [id=#15] (21) HashAggregate [codegen id : 6] Input [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Keys [2]: [cs_bill_customer_sk#11, cs_item_sk#12] Functions: [] Aggregate Attributes: [] -Results [2]: [cs_bill_customer_sk#11 AS customer_sk#15, cs_item_sk#12 AS item_sk#16] +Results [2]: [cs_bill_customer_sk#11 AS customer_sk#16, cs_item_sk#12 AS item_sk#17] (22) Sort [codegen id : 6] -Input [2]: [customer_sk#15, item_sk#16] -Arguments: [customer_sk#15 ASC NULLS FIRST, item_sk#16 ASC NULLS FIRST], false, 0 +Input [2]: [customer_sk#16, item_sk#17] +Arguments: [customer_sk#16 ASC NULLS FIRST, item_sk#17 ASC NULLS FIRST], false, 0 (23) SortMergeJoin Left keys [2]: [customer_sk#9, item_sk#10] -Right keys [2]: [customer_sk#15, item_sk#16] +Right keys [2]: [customer_sk#16, item_sk#17] Join condition: None (24) Project [codegen id : 7] -Output [2]: [customer_sk#9, customer_sk#15] -Input [4]: [customer_sk#9, item_sk#10, customer_sk#15, item_sk#16] +Output [2]: [customer_sk#9, customer_sk#16] +Input [4]: [customer_sk#9, item_sk#10, customer_sk#16, item_sk#17] (25) HashAggregate [codegen id : 7] -Input [2]: [customer_sk#9, customer_sk#15] +Input [2]: [customer_sk#9, customer_sk#16] Keys: [] -Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum#17, sum#18, sum#19] -Results [3]: [sum#20, sum#21, sum#22] +Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum#18, sum#19, sum#20] +Results [3]: [sum#21, sum#22, sum#23] (26) Exchange -Input [3]: [sum#20, sum#21, sum#22] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [sum#21, sum#22, sum#23] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#24] (27) HashAggregate [codegen id : 8] -Input [3]: [sum#20, sum#21, sum#22] +Input [3]: [sum#21, sum#22, sum#23] Keys: [] -Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)] -Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26] -Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#15)) THEN 1 ELSE 0 END)#24 AS store_only#27, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#25 AS catalog_only#28, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#15)) THEN 1 ELSE 0 END)#26 AS store_and_catalog#29] +Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27] +Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#9) AND isnull(customer_sk#16)) THEN 1 ELSE 0 END)#25 AS store_only#28, sum(CASE WHEN (isnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#26 AS catalog_only#29, sum(CASE WHEN (isnotnull(customer_sk#9) AND isnotnull(customer_sk#16)) THEN 1 ELSE 0 END)#27 AS store_and_catalog#30] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt index 58d28b28dae10..2e710b7eda4b3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt @@ -137,52 +137,52 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#15] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (22) Project [codegen id : 7] -Output [1]: [ws_bill_customer_sk#13 AS customer_sk#15] -Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] +Output [1]: [ws_bill_customer_sk#13 AS customer_sk#16] +Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] (23) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] ReadSchema: struct (24) ColumnarToRow [codegen id : 9] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (25) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#19] (26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (27) Project [codegen id : 9] -Output [1]: [cs_ship_customer_sk#16 AS customer_sk#18] -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] +Output [1]: [cs_ship_customer_sk#17 AS customer_sk#20] +Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] (28) Union (29) Exchange -Input [1]: [customer_sk#15] -Arguments: hashpartitioning(customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#19] +Input [1]: [customer_sk#16] +Arguments: hashpartitioning(customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#21] (30) Sort [codegen id : 10] -Input [1]: [customer_sk#15] -Arguments: [customer_sk#15 ASC NULLS FIRST], false, 0 +Input [1]: [customer_sk#16] +Arguments: [customer_sk#16 ASC NULLS FIRST], false, 0 (31) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#15] +Right keys [1]: [customer_sk#16] Join condition: None (32) Project [codegen id : 12] @@ -190,84 +190,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (33) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_county#21] +Output [2]: [ca_address_sk#22, ca_county#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -Input [2]: [ca_address_sk#20, ca_county#21] +Input [2]: [ca_address_sk#22, ca_county#23] (35) Filter [codegen id : 11] -Input [2]: [ca_address_sk#20, ca_county#21] -Condition : (ca_county#21 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#20)) +Input [2]: [ca_address_sk#22, ca_county#23] +Condition : (ca_county#23 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#22)) (36) Project [codegen id : 11] -Output [1]: [ca_address_sk#20] -Input [2]: [ca_address_sk#20, ca_county#21] +Output [1]: [ca_address_sk#22] +Input [2]: [ca_address_sk#22, ca_county#23] (37) BroadcastExchange -Input [1]: [ca_address_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +Input [1]: [ca_address_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] (38) BroadcastHashJoin [codegen id : 12] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#22] Join condition: None (39) Project [codegen id : 12] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#20] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#22] (40) BroadcastExchange Input [1]: [c_current_cdemo_sk#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#25] (41) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Output [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (42) ColumnarToRow -Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] (43) Filter -Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Condition : isnotnull(cd_demo_sk#24) +Input [9]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Condition : isnotnull(cd_demo_sk#26) (44) BroadcastHashJoin [codegen id : 13] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#24] +Right keys [1]: [cd_demo_sk#26] Join condition: None (45) Project [codegen id : 13] -Output [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Output [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] (46) HashAggregate [codegen id : 13] -Input [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] -Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] +Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#33] -Results [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] +Aggregate Attributes [1]: [count#35] +Results [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] (47) Exchange -Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] -Arguments: hashpartitioning(cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] +Arguments: hashpartitioning(cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, 5), ENSURE_REQUIREMENTS, [id=#37] (48) HashAggregate [codegen id : 14] -Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#34] -Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [9]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34, count#36] +Keys [8]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cd_purchase_estimate#30, cd_credit_rating#31, cd_dep_count#32, cd_dep_employed_count#33, cd_dep_college_count#34] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#36] -Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count(1)#36 AS cnt1#37, cd_purchase_estimate#28, count(1)#36 AS cnt2#38, cd_credit_rating#29, count(1)#36 AS cnt3#39, cd_dep_count#30, count(1)#36 AS cnt4#40, cd_dep_employed_count#31, count(1)#36 AS cnt5#41, cd_dep_college_count#32, count(1)#36 AS cnt6#42] +Aggregate Attributes [1]: [count(1)#38] +Results [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, count(1)#38 AS cnt1#39, cd_purchase_estimate#30, count(1)#38 AS cnt2#40, cd_credit_rating#31, count(1)#38 AS cnt3#41, cd_dep_count#32, count(1)#38 AS cnt4#42, cd_dep_employed_count#33, count(1)#38 AS cnt5#43, cd_dep_college_count#34, count(1)#38 AS cnt6#44] (49) TakeOrderedAndProject -Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] -Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] +Input [14]: [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#39, cd_purchase_estimate#30, cnt2#40, cd_credit_rating#31, cnt3#41, cd_dep_count#32, cnt4#42, cd_dep_employed_count#33, cnt5#43, cd_dep_college_count#34, cnt6#44] +Arguments: 100, [cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_education_status#29 ASC NULLS FIRST, cd_purchase_estimate#30 ASC NULLS FIRST, cd_credit_rating#31 ASC NULLS FIRST, cd_dep_count#32 ASC NULLS FIRST, cd_dep_employed_count#33 ASC NULLS FIRST, cd_dep_college_count#34 ASC NULLS FIRST], [cd_gender#27, cd_marital_status#28, cd_education_status#29, cnt1#39, cd_purchase_estimate#30, cnt2#40, cd_credit_rating#31, cnt3#41, cd_dep_count#32, cnt4#42, cd_dep_employed_count#33, cnt5#43, cd_dep_college_count#34, cnt6#44] ===== Subqueries ===== @@ -280,6 +280,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index eac6acc3543ca..62054fe96dd5c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -121,48 +121,48 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#12 AS customer_sk#14] -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] +Output [1]: [ws_bill_customer_sk#12 AS customer_sk#15] +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] (20) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#6)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] (22) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#18] (23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#18] Join condition: None (24) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#15 AS customer_sk#17] -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] +Output [1]: [cs_ship_customer_sk#16 AS customer_sk#19] +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#18] (25) Union (26) BroadcastExchange -Input [1]: [customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Input [1]: [customer_sk#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (27) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customer_sk#14] +Right keys [1]: [customer_sk#15] Join condition: None (28) Project [codegen id : 9] @@ -170,84 +170,84 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#19, ca_county#20] +Output [2]: [ca_address_sk#21, ca_county#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_county, [Walker County,Richland County,Gaines County,Douglas County,Dona Ana County]), IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#19, ca_county#20] +Input [2]: [ca_address_sk#21, ca_county#22] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#19, ca_county#20] -Condition : (ca_county#20 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#19)) +Input [2]: [ca_address_sk#21, ca_county#22] +Condition : (ca_county#22 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#21)) (32) Project [codegen id : 7] -Output [1]: [ca_address_sk#19] -Input [2]: [ca_address_sk#19, ca_county#20] +Output [1]: [ca_address_sk#21] +Input [2]: [ca_address_sk#21, ca_county#22] (33) BroadcastExchange -Input [1]: [ca_address_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [ca_address_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] (34) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#19] +Right keys [1]: [ca_address_sk#21] Join condition: None (35) Project [codegen id : 9] Output [1]: [c_current_cdemo_sk#2] -Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21] (36) Scan parquet default.customer_demographics -Output [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (37) ColumnarToRow [codegen id : 8] -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] (38) Filter [codegen id : 8] -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#22) +Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Condition : isnotnull(cd_demo_sk#24) (39) BroadcastExchange -Input [9]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] +Input [9]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] (40) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#22] +Right keys [1]: [cd_demo_sk#24] Join condition: None (41) Project [codegen id : 9] -Output [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] (42) HashAggregate [codegen id : 9] -Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#32] -Results [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] +Aggregate Attributes [1]: [count#34] +Results [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#35] (43) Exchange -Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] -Arguments: hashpartitioning(cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#34] +Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#35] +Arguments: hashpartitioning(cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#36] (44) HashAggregate [codegen id : 10] -Input [9]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#33] -Keys [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cd_purchase_estimate#26, cd_credit_rating#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [9]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#35] +Keys [8]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cd_purchase_estimate#28, cd_credit_rating#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#35] -Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(1)#35 AS cnt1#36, cd_purchase_estimate#26, count(1)#35 AS cnt2#37, cd_credit_rating#27, count(1)#35 AS cnt3#38, cd_dep_count#28, count(1)#35 AS cnt4#39, cd_dep_employed_count#29, count(1)#35 AS cnt5#40, cd_dep_college_count#30, count(1)#35 AS cnt6#41] +Aggregate Attributes [1]: [count(1)#37] +Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count(1)#37 AS cnt1#38, cd_purchase_estimate#28, count(1)#37 AS cnt2#39, cd_credit_rating#29, count(1)#37 AS cnt3#40, cd_dep_count#30, count(1)#37 AS cnt4#41, cd_dep_employed_count#31, count(1)#37 AS cnt5#42, cd_dep_college_count#32, count(1)#37 AS cnt6#43] (45) TakeOrderedAndProject -Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] -Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] +Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#38, cd_purchase_estimate#28, cnt2#39, cd_credit_rating#29, cnt3#40, cd_dep_count#30, cnt4#41, cd_dep_employed_count#31, cnt5#42, cd_dep_college_count#32, cnt6#43] +Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#38, cd_purchase_estimate#28, cnt2#39, cd_credit_rating#29, cnt3#40, cd_dep_count#30, cnt4#41, cd_dep_employed_count#31, cnt5#42, cd_dep_college_count#32, cnt6#43] ===== Subqueries ===== @@ -260,6 +260,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#6 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 6eb3be35a05bc..453a35d8d5175 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 @@ -199,285 +199,285 @@ Input [2]: [customer_id#23, year_total#24] Arguments: [customer_id#23 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] +Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] (27) Filter [codegen id : 10] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4] -Condition : isnotnull(ss_customer_sk#1) +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29] +Condition : isnotnull(ss_customer_sk#26) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#31, d_year#32] 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 (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#6, d_year#7] +Input [2]: [d_date_sk#31, d_year#32] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#6, d_year#7] -Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#31, d_year#32] +Condition : ((isnotnull(d_year#32) AND (d_year#32 = 2002)) AND isnotnull(d_date_sk#31)) (31) BroadcastExchange -Input [2]: [d_date_sk#6, d_year#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] +Input [2]: [d_date_sk#31, d_year#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#33] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#4] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ss_sold_date_sk#29] +Right keys [1]: [d_date_sk#31] Join condition: None (33) Project [codegen id : 10] -Output [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Input [6]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, ss_sold_date_sk#4, d_date_sk#6, d_year#7] +Output [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Input [6]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, ss_sold_date_sk#29, d_date_sk#31, d_year#32] (34) Exchange -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#28] +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Arguments: hashpartitioning(ss_customer_sk#26, 5), ENSURE_REQUIREMENTS, [id=#34] (35) Sort [codegen id : 11] -Input [4]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [4]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] (37) Sort [codegen id : 13] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Arguments: [c_customer_sk#35 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#35] Join condition: None (39) Project [codegen id : 14] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Input [12]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Input [12]: [ss_customer_sk#26, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32, c_customer_sk#35, c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] (40) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#29] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] +Input [10]: [c_customer_id#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, ss_ext_discount_amt#27, ss_ext_list_price#28, d_year#32] +Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#43] +Results [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] (41) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] +Arguments: hashpartitioning(c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, 5), ENSURE_REQUIREMENTS, [id=#45] (42) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, sum#30] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, d_year#7, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32] -Results [5]: [c_customer_id#11 AS customer_id#33, c_first_name#12 AS customer_first_name#34, c_last_name#13 AS customer_last_name#35, c_email_address#17 AS customer_email_address#36, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2), true)))#32,18,2) AS year_total#37] +Input [9]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42, sum#44] +Keys [8]: [c_customer_id#36, c_first_name#37, c_last_name#38, d_year#32, c_preferred_cust_flag#39, c_birth_country#40, c_login#41, c_email_address#42] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46] +Results [5]: [c_customer_id#36 AS customer_id#47, c_first_name#37 AS customer_first_name#48, c_last_name#38 AS customer_last_name#49, c_email_address#42 AS customer_email_address#50, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#28 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#27 as decimal(8,2)))), DecimalType(8,2), true)))#46,18,2) AS year_total#51] (43) Exchange -Input [5]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37] -Arguments: hashpartitioning(customer_id#33, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] +Arguments: hashpartitioning(customer_id#47, 5), ENSURE_REQUIREMENTS, [id=#52] (44) Sort [codegen id : 16] -Input [5]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37] -Arguments: [customer_id#33 ASC NULLS FIRST], false, 0 +Input [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] +Arguments: [customer_id#47 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#33] +Right keys [1]: [customer_id#47] Join condition: None (46) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] +Output [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#56), dynamicpruningexpression(ws_sold_date_sk#56 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] +Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56] (48) Filter [codegen id : 19] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] -Condition : isnotnull(ws_bill_customer_sk#39) +Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56] +Condition : isnotnull(ws_bill_customer_sk#53) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#57, d_year#58] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#56] +Right keys [1]: [d_date_sk#57] Join condition: None (51) Project [codegen id : 19] -Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Input [6]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42, d_date_sk#6, d_year#7] +Output [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] +Input [6]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, ws_sold_date_sk#56, d_date_sk#57, d_year#58] (52) Exchange -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Arguments: hashpartitioning(ws_bill_customer_sk#39, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] +Arguments: hashpartitioning(ws_bill_customer_sk#53, 5), ENSURE_REQUIREMENTS, [id=#59] (53) Sort [codegen id : 20] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Arguments: [ws_bill_customer_sk#39 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] +Arguments: [ws_bill_customer_sk#53 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +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] (55) Sort [codegen id : 22] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +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] +Arguments: [c_customer_sk#60 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#39] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ws_bill_customer_sk#53] +Right keys [1]: [c_customer_sk#60] Join condition: None (57) Project [codegen id : 23] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Input [12]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [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, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] +Input [12]: [ws_bill_customer_sk#53, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58, 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] (58) HashAggregate [codegen id : 23] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#44] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#45] +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, ws_ext_discount_amt#54, ws_ext_list_price#55, d_year#58] +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#58] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#68] +Results [9]: [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#58, sum#69] (59) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#45] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [9]: [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#58, sum#69] +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#58, 5), ENSURE_REQUIREMENTS, [id=#70] (60) HashAggregate [codegen id : 24] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#45] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#47] -Results [2]: [c_customer_id#11 AS customer_id#48, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#47,18,2) AS year_total#49] +Input [9]: [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#58, sum#69] +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#58] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))#71] +Results [2]: [c_customer_id#61 AS customer_id#72, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#55 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#54 as decimal(8,2)))), DecimalType(8,2), true)))#71,18,2) AS year_total#73] (61) Filter [codegen id : 24] -Input [2]: [customer_id#48, year_total#49] -Condition : (isnotnull(year_total#49) AND (year_total#49 > 0.00)) +Input [2]: [customer_id#72, year_total#73] +Condition : (isnotnull(year_total#73) AND (year_total#73 > 0.00)) (62) Project [codegen id : 24] -Output [2]: [customer_id#48 AS customer_id#50, year_total#49 AS year_total#51] -Input [2]: [customer_id#48, year_total#49] +Output [2]: [customer_id#72 AS customer_id#74, year_total#73 AS year_total#75] +Input [2]: [customer_id#72, year_total#73] (63) Exchange -Input [2]: [customer_id#50, year_total#51] -Arguments: hashpartitioning(customer_id#50, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [2]: [customer_id#74, year_total#75] +Arguments: hashpartitioning(customer_id#74, 5), ENSURE_REQUIREMENTS, [id=#76] (64) Sort [codegen id : 25] -Input [2]: [customer_id#50, year_total#51] -Arguments: [customer_id#50 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#74, year_total#75] +Arguments: [customer_id#74 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#50] +Right keys [1]: [customer_id#74] Join condition: None (66) Project [codegen id : 26] -Output [8]: [customer_id#23, year_total#24, customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37, year_total#51] -Input [9]: [customer_id#23, year_total#24, customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37, customer_id#50, year_total#51] +Output [8]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75] +Input [9]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, customer_id#74, year_total#75] (67) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] +Output [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#42), dynamicpruningexpression(ws_sold_date_sk#42 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ws_sold_date_sk#80), dynamicpruningexpression(ws_sold_date_sk#80 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] +Input [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80] (69) Filter [codegen id : 28] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42] -Condition : isnotnull(ws_bill_customer_sk#39) +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) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#81, d_year#82] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#42] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#80] +Right keys [1]: [d_date_sk#81] Join condition: None (72) Project [codegen id : 28] -Output [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Input [6]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, ws_sold_date_sk#42, d_date_sk#6, d_year#7] +Output [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] +Input [6]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, ws_sold_date_sk#80, d_date_sk#81, d_year#82] (73) Exchange -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Arguments: hashpartitioning(ws_bill_customer_sk#39, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] +Arguments: hashpartitioning(ws_bill_customer_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] (74) Sort [codegen id : 29] -Input [4]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Arguments: [ws_bill_customer_sk#39 ASC NULLS FIRST], false, 0 +Input [4]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] +Arguments: [ws_bill_customer_sk#77 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [8]: [c_customer_sk#84, c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91] (76) Sort [codegen id : 31] -Input [8]: [c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] -Arguments: [c_customer_sk#10 ASC NULLS FIRST], false, 0 +Input [8]: [c_customer_sk#84, c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91] +Arguments: [c_customer_sk#84 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#39] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [ws_bill_customer_sk#77] +Right keys [1]: [c_customer_sk#84] Join condition: None (78) Project [codegen id : 32] -Output [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Input [12]: [ws_bill_customer_sk#39, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7, c_customer_sk#10, c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17] +Output [10]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] +Input [12]: [ws_bill_customer_sk#77, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82, c_customer_sk#84, c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91] (79) HashAggregate [codegen id : 32] -Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ws_ext_discount_amt#40, ws_ext_list_price#41, d_year#7] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#54] -Results [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#55] +Input [10]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, ws_ext_discount_amt#78, ws_ext_list_price#79, d_year#82] +Keys [8]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82] +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#92] +Results [9]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, sum#93] (80) Exchange -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#55] -Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [9]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, sum#93] +Arguments: hashpartitioning(c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, 5), ENSURE_REQUIREMENTS, [id=#94] (81) HashAggregate [codegen id : 33] -Input [9]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7, sum#55] -Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#7] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#57] -Results [2]: [c_customer_id#11 AS customer_id#58, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#41 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#40 as decimal(8,2)))), DecimalType(8,2), true)))#57,18,2) AS year_total#59] +Input [9]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82, sum#93] +Keys [8]: [c_customer_id#85, c_first_name#86, c_last_name#87, c_preferred_cust_flag#88, c_birth_country#89, c_login#90, c_email_address#91, d_year#82] +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)))#95] +Results [2]: [c_customer_id#85 AS customer_id#96, 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)))#95,18,2) AS year_total#97] (82) Exchange -Input [2]: [customer_id#58, year_total#59] -Arguments: hashpartitioning(customer_id#58, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [2]: [customer_id#96, year_total#97] +Arguments: hashpartitioning(customer_id#96, 5), ENSURE_REQUIREMENTS, [id=#98] (83) Sort [codegen id : 34] -Input [2]: [customer_id#58, year_total#59] -Arguments: [customer_id#58 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#96, year_total#97] +Arguments: [customer_id#96 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#23] -Right keys [1]: [customer_id#58] -Join condition: (CASE WHEN (year_total#51 > 0.00) THEN CheckOverflow((promote_precision(year_total#59) / promote_precision(year_total#51)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#37) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE 0E-20 END) +Right keys [1]: [customer_id#96] +Join condition: (CASE WHEN (year_total#75 > 0.00) THEN CheckOverflow((promote_precision(year_total#97) / promote_precision(year_total#75)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN CheckOverflow((promote_precision(year_total#51) / promote_precision(year_total#24)), DecimalType(38,20), true) ELSE 0E-20 END) (85) Project [codegen id : 35] -Output [4]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36] -Input [10]: [customer_id#23, year_total#24, customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36, year_total#37, year_total#51, customer_id#58, year_total#59] +Output [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] +Input [10]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75, customer_id#96, year_total#97] (86) TakeOrderedAndProject -Input [4]: [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36] -Arguments: 100, [customer_id#33 ASC NULLS FIRST, customer_first_name#34 ASC NULLS FIRST, customer_last_name#35 ASC NULLS FIRST, customer_email_address#36 ASC NULLS FIRST], [customer_id#33, customer_first_name#34, customer_last_name#35, customer_email_address#36] +Input [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] +Arguments: 100, [customer_id#47 ASC NULLS FIRST, customer_first_name#48 ASC NULLS FIRST, customer_last_name#49 ASC NULLS FIRST, customer_email_address#50 ASC NULLS FIRST], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] ===== Subqueries ===== @@ -488,15 +488,15 @@ ReusedExchange (87) (87) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#6, d_year#7] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#26 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#30 ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#6, d_year#7] +Output [2]: [d_date_sk#31, d_year#32] -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#56 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#42 IN dynamicpruning#26 +Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#80 IN dynamicpruning#30 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 8f7b2faaa1220..17d16033bb5be 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 @@ -171,282 +171,282 @@ Input [2]: [customer_id#22, year_total#23] Condition : (isnotnull(year_total#23) AND (year_total#23 > 0.00)) (20) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] (22) Filter [codegen id : 6] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_customer_id#25)) (23) Scan parquet default.store_sales -Output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Output [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ss_sold_date_sk#35), dynamicpruningexpression(ss_sold_date_sk#35 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] (25) Filter [codegen id : 4] -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Condition : isnotnull(ss_customer_sk#9) +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Condition : isnotnull(ss_customer_sk#32) (26) BroadcastExchange -Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#25] +Input [4]: [ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#37] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#9] +Left keys [1]: [c_customer_sk#24] +Right keys [1]: [ss_customer_sk#32] Join condition: None (28) Project [codegen id : 6] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] +Input [12]: [c_customer_sk#24, c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_customer_sk#32, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#38, d_year#39] 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 (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#15, d_year#16] +Input [2]: [d_date_sk#38, d_year#39] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#15, d_year#16] -Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2002)) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#38, d_year#39] +Condition : ((isnotnull(d_year#39) AND (d_year#39 = 2002)) AND isnotnull(d_date_sk#38)) (32) BroadcastExchange -Input [2]: [d_date_sk#15, d_year#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +Input [2]: [d_date_sk#38, d_year#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#40] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#35] +Right keys [1]: [d_date_sk#38] Join condition: None (34) Project [codegen id : 6] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#15, d_year#16] +Output [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] +Input [12]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, ss_sold_date_sk#35, d_date_sk#38, d_year#39] (35) HashAggregate [codegen id : 6] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#27] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] +Input [10]: [c_customer_id#25, c_first_name#26, c_last_name#27, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, ss_ext_discount_amt#33, ss_ext_list_price#34, d_year#39] +Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#41] +Results [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] (36) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] +Arguments: hashpartitioning(c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, 5), ENSURE_REQUIREMENTS, [id=#43] (37) HashAggregate [codegen id : 7] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#28] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#16, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30] -Results [5]: [c_customer_id#2 AS customer_id#31, c_first_name#3 AS customer_first_name#32, c_last_name#4 AS customer_last_name#33, c_email_address#8 AS customer_email_address#34, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2), true)))#30,18,2) AS year_total#35] +Input [9]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31, sum#42] +Keys [8]: [c_customer_id#25, c_first_name#26, c_last_name#27, d_year#39, c_preferred_cust_flag#28, c_birth_country#29, c_login#30, c_email_address#31] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44] +Results [5]: [c_customer_id#25 AS customer_id#45, c_first_name#26 AS customer_first_name#46, c_last_name#27 AS customer_last_name#47, c_email_address#31 AS customer_email_address#48, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#34 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#33 as decimal(8,2)))), DecimalType(8,2), true)))#44,18,2) AS year_total#49] (38) BroadcastExchange -Input [5]: [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#36] +Input [5]: [customer_id#45, customer_first_name#46, customer_last_name#47, customer_email_address#48, year_total#49] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#31] +Right keys [1]: [customer_id#45] Join condition: None (40) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] (42) Filter [codegen id : 10] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] +Condition : (isnotnull(c_customer_sk#51) AND isnotnull(c_customer_id#52)) (43) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#13)] +PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#13)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Input [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] (45) Filter [codegen id : 8] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_customer_sk#37) +Input [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] +Condition : isnotnull(ws_bill_customer_sk#59) (46) BroadcastExchange -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#41] +Input [4]: [ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#37] +Left keys [1]: [c_customer_sk#51] +Right keys [1]: [ws_bill_customer_sk#59] Join condition: None (48) Project [codegen id : 10] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] +Input [12]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_bill_customer_sk#59, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#64, d_year#65] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ws_sold_date_sk#62] +Right keys [1]: [d_date_sk#64] Join condition: None (51) Project [codegen id : 10] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#15, d_year#16] +Output [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, d_year#65] +Input [12]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, ws_sold_date_sk#62, d_date_sk#64, d_year#65] (52) HashAggregate [codegen id : 10] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#42] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#43] +Input [10]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, ws_ext_discount_amt#60, ws_ext_list_price#61, d_year#65] +Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#66] +Results [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, sum#67] (53) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#43] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#44] +Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, sum#67] +Arguments: hashpartitioning(c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#68] (54) HashAggregate [codegen id : 11] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#43] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45] -Results [2]: [c_customer_id#2 AS customer_id#46, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#45,18,2) AS year_total#47] +Input [9]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65, sum#67] +Keys [8]: [c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58, d_year#65] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))#69] +Results [2]: [c_customer_id#52 AS customer_id#70, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#61 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#60 as decimal(8,2)))), DecimalType(8,2), true)))#69,18,2) AS year_total#71] (55) Filter [codegen id : 11] -Input [2]: [customer_id#46, year_total#47] -Condition : (isnotnull(year_total#47) AND (year_total#47 > 0.00)) +Input [2]: [customer_id#70, year_total#71] +Condition : (isnotnull(year_total#71) AND (year_total#71 > 0.00)) (56) Project [codegen id : 11] -Output [2]: [customer_id#46 AS customer_id#48, year_total#47 AS year_total#49] -Input [2]: [customer_id#46, year_total#47] +Output [2]: [customer_id#70 AS customer_id#72, year_total#71 AS year_total#73] +Input [2]: [customer_id#70, year_total#71] (57) BroadcastExchange -Input [2]: [customer_id#48, year_total#49] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#50] +Input [2]: [customer_id#72, year_total#73] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#74] (58) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#48] +Right keys [1]: [customer_id#72] Join condition: None (59) Project [codegen id : 16] -Output [8]: [customer_id#22, year_total#23, customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35, year_total#49] -Input [9]: [customer_id#22, year_total#23, customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35, customer_id#48, year_total#49] +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#73] +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#72, year_total#73] (60) Scan parquet default.customer -Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Output [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] (62) Filter [codegen id : 14] -Input [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [8]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82] +Condition : (isnotnull(c_customer_sk#75) AND isnotnull(c_customer_id#76)) (63) Scan parquet default.web_sales -Output [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#36)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] (65) Filter [codegen id : 12] -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Condition : isnotnull(ws_bill_customer_sk#37) +Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Condition : isnotnull(ws_bill_customer_sk#83) (66) BroadcastExchange -Input [4]: [ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#51] +Input [4]: [ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#87] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#37] +Left keys [1]: [c_customer_sk#75] +Right keys [1]: [ws_bill_customer_sk#83] Join condition: None (68) Project [codegen id : 14] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] -Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_bill_customer_sk#37, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40] +Output [10]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] +Input [12]: [c_customer_sk#75, c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_bill_customer_sk#83, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#88, d_year#89] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#40] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ws_sold_date_sk#86] +Right keys [1]: [d_date_sk#88] Join condition: None (71) Project [codegen id : 14] -Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] -Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, ws_sold_date_sk#40, d_date_sk#15, d_year#16] +Output [10]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] +Input [12]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, ws_sold_date_sk#86, d_date_sk#88, d_year#89] (72) HashAggregate [codegen id : 14] -Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ws_ext_discount_amt#38, ws_ext_list_price#39, d_year#16] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum#52] -Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#53] +Input [10]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, ws_ext_discount_amt#84, ws_ext_list_price#85, d_year#89] +Keys [8]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89] +Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum#90] +Results [9]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, sum#91] (73) Exchange -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#53] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, 5), ENSURE_REQUIREMENTS, [id=#54] +Input [9]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, sum#91] +Arguments: hashpartitioning(c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, 5), ENSURE_REQUIREMENTS, [id=#92] (74) HashAggregate [codegen id : 15] -Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16, sum#53] -Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#16] -Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))] -Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55] -Results [2]: [c_customer_id#2 AS customer_id#56, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#39 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#38 as decimal(8,2)))), DecimalType(8,2), true)))#55,18,2) AS year_total#57] +Input [9]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89, sum#91] +Keys [8]: [c_customer_id#76, c_first_name#77, c_last_name#78, c_preferred_cust_flag#79, c_birth_country#80, c_login#81, c_email_address#82, d_year#89] +Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))] +Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))#93] +Results [2]: [c_customer_id#76 AS customer_id#94, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#85 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#84 as decimal(8,2)))), DecimalType(8,2), true)))#93,18,2) AS year_total#95] (75) BroadcastExchange -Input [2]: [customer_id#56, year_total#57] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#58] +Input [2]: [customer_id#94, year_total#95] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#96] (76) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#22] -Right keys [1]: [customer_id#56] -Join condition: (CASE WHEN (year_total#49 > 0.00) THEN CheckOverflow((promote_precision(year_total#57) / promote_precision(year_total#49)), DecimalType(38,20), true) ELSE 0E-20 END > CASE WHEN (year_total#23 > 0.00) THEN CheckOverflow((promote_precision(year_total#35) / promote_precision(year_total#23)), DecimalType(38,20), true) ELSE 0E-20 END) +Right keys [1]: [customer_id#94] +Join condition: (CASE WHEN (year_total#73 > 0.00) THEN CheckOverflow((promote_precision(year_total#95) / promote_precision(year_total#73)), 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) (77) Project [codegen id : 16] -Output [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34] -Input [10]: [customer_id#22, year_total#23, customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34, year_total#35, year_total#49, customer_id#56, year_total#57] +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#73, customer_id#94, year_total#95] (78) TakeOrderedAndProject -Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34] -Arguments: 100, [customer_id#31 ASC NULLS FIRST, customer_first_name#32 ASC NULLS FIRST, customer_last_name#33 ASC NULLS FIRST, customer_email_address#34 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33, customer_email_address#34] +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] ===== Subqueries ===== @@ -457,15 +457,15 @@ ReusedExchange (79) (79) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#15, d_year#16] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#24 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#35 IN dynamicpruning#36 ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#38, d_year#39] -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#13 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#13 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#24 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#36 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 15f74a7802b27..56fa48006d894 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 @@ -393,484 +393,484 @@ Aggregate Attributes [7]: [avg(agg1#37)#73, avg(agg2#38)#74, avg(agg3#39)#75, av Results [11]: [i_item_id#20, ca_country#34, ca_state#33, ca_county#32, avg(agg1#37)#73 AS agg1#80, avg(agg2#38)#74 AS agg2#81, avg(agg3#39)#75 AS agg3#82, avg(agg4#40)#76 AS agg4#83, avg(agg5#41)#77 AS agg5#84, avg(agg6#42)#78 AS agg6#85, avg(agg7#43)#79 AS agg7#86] (52) ReusedExchange [Reuses operator id: unknown] -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#20] +Output [8]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94] (53) Sort [codegen id : 19] -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#20] -Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94] +Arguments: [cs_bill_customer_sk#87 ASC NULLS FIRST], false, 0 (54) Scan parquet default.customer -Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 21] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] (56) Filter [codegen id : 21] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] +Condition : (((c_birth_month#98 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#95)) AND isnotnull(c_current_cdemo_sk#96)) AND isnotnull(c_current_addr_sk#97)) (57) Project [codegen id : 21] -Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [4]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#99] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] (58) Scan parquet default.customer_address -Output [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Output [3]: [ca_address_sk#100, ca_state#101, ca_country#102] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] (60) Filter [codegen id : 20] -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] -Condition : (ca_state#33 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#31)) +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Condition : (ca_state#101 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#100)) (61) BroadcastExchange -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#87] +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] (62) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [c_current_addr_sk#25] -Right keys [1]: [ca_address_sk#31] +Left keys [1]: [c_current_addr_sk#97] +Right keys [1]: [ca_address_sk#100] Join condition: None (63) Project [codegen id : 21] -Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34] -Input [7]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31, ca_state#33, ca_country#34] +Output [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Input [7]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#99, ca_address_sk#100, ca_state#101, ca_country#102] (64) Exchange -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34] -Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: hashpartitioning(c_current_cdemo_sk#96, 5), ENSURE_REQUIREMENTS, [id=#104] (65) Sort [codegen id : 22] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34] -Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: [c_current_cdemo_sk#96 ASC NULLS FIRST], false, 0 (66) ReusedExchange [Reuses operator id: 35] -Output [1]: [cd_demo_sk#89] +Output [1]: [cd_demo_sk#105] (67) Sort [codegen id : 24] -Input [1]: [cd_demo_sk#89] -Arguments: [cd_demo_sk#89 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#105] +Arguments: [cd_demo_sk#105 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 25] -Left keys [1]: [c_current_cdemo_sk#24] -Right keys [1]: [cd_demo_sk#89] +Left keys [1]: [c_current_cdemo_sk#96] +Right keys [1]: [cd_demo_sk#105] Join condition: None (69) Project [codegen id : 25] -Output [4]: [c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] -Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_state#33, ca_country#34, cd_demo_sk#89] +Output [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Input [6]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102, cd_demo_sk#105] (70) Exchange -Input [4]: [c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#90] +Input [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: hashpartitioning(c_customer_sk#95, 5), ENSURE_REQUIREMENTS, [id=#106] (71) Sort [codegen id : 26] -Input [4]: [c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] -Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: [c_customer_sk#95 ASC NULLS FIRST], false, 0 (72) SortMergeJoin [codegen id : 27] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#23] +Left keys [1]: [cs_bill_customer_sk#87] +Right keys [1]: [c_customer_sk#95] Join condition: None (73) Project [codegen id : 27] -Output [10]: [i_item_id#20, ca_country#34, ca_state#33, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] -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#20, c_customer_sk#23, c_birth_year#27, ca_state#33, ca_country#34] +Output [10]: [i_item_id#94, ca_country#102, ca_state#101, cast(cs_quantity#88 as decimal(12,2)) AS agg1#37, cast(cs_list_price#89 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#91 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#90 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#92 as decimal(12,2)) AS agg5#41, cast(c_birth_year#99 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#93 as decimal(12,2)) AS agg7#43] +Input [12]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94, c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] (74) HashAggregate [codegen id : 27] -Input [10]: [i_item_id#20, ca_country#34, ca_state#33, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] -Keys [3]: [i_item_id#20, ca_country#34, ca_state#33] +Input [10]: [i_item_id#94, ca_country#102, ca_state#101, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [3]: [i_item_id#94, ca_country#102, ca_state#101] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100, sum#101, count#102, sum#103, count#104] -Results [17]: [i_item_id#20, ca_country#34, ca_state#33, 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] +Aggregate Attributes [14]: [sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] +Results [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] (75) Exchange -Input [17]: [i_item_id#20, ca_country#34, ca_state#33, 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] -Arguments: hashpartitioning(i_item_id#20, ca_country#34, ca_state#33, 5), ENSURE_REQUIREMENTS, [id=#119] +Input [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] +Arguments: hashpartitioning(i_item_id#94, ca_country#102, ca_state#101, 5), ENSURE_REQUIREMENTS, [id=#135] (76) HashAggregate [codegen id : 28] -Input [17]: [i_item_id#20, ca_country#34, ca_state#33, 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] -Keys [3]: [i_item_id#20, ca_country#34, ca_state#33] +Input [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] +Keys [3]: [i_item_id#94, ca_country#102, ca_state#101] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#120, avg(agg2#38)#121, avg(agg3#39)#122, avg(agg4#40)#123, avg(agg5#41)#124, avg(agg6#42)#125, avg(agg7#43)#126] -Results [11]: [i_item_id#20, ca_country#34, ca_state#33, null AS county#127, avg(agg1#37)#120 AS agg1#128, avg(agg2#38)#121 AS agg2#129, avg(agg3#39)#122 AS agg3#130, avg(agg4#40)#123 AS agg4#131, avg(agg5#41)#124 AS agg5#132, avg(agg6#42)#125 AS agg6#133, avg(agg7#43)#126 AS agg7#134] +Aggregate Attributes [7]: [avg(agg1#37)#136, avg(agg2#38)#137, avg(agg3#39)#138, avg(agg4#40)#139, avg(agg5#41)#140, avg(agg6#42)#141, avg(agg7#43)#142] +Results [11]: [i_item_id#94, ca_country#102, ca_state#101, null AS county#143, avg(agg1#37)#136 AS agg1#144, avg(agg2#38)#137 AS agg2#145, avg(agg3#39)#138 AS agg3#146, avg(agg4#40)#139 AS agg4#147, avg(agg5#41)#140 AS agg5#148, avg(agg6#42)#141 AS agg6#149, avg(agg7#43)#142 AS agg7#150] (77) ReusedExchange [Reuses operator id: unknown] -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#20] +Output [8]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158] (78) Sort [codegen id : 33] -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#20] -Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158] +Arguments: [cs_bill_customer_sk#151 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer -Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (80) ColumnarToRow [codegen id : 35] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] (81) Filter [codegen id : 35] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] +Condition : (((c_birth_month#162 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#159)) AND isnotnull(c_current_cdemo_sk#160)) AND isnotnull(c_current_addr_sk#161)) (82) Project [codegen id : 35] -Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_year#163] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] (83) Scan parquet default.customer_address -Output [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Output [3]: [ca_address_sk#164, ca_state#165, ca_country#166] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (84) ColumnarToRow [codegen id : 34] -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] (85) Filter [codegen id : 34] -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] -Condition : (ca_state#33 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#31)) +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] +Condition : (ca_state#165 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#164)) (86) Project [codegen id : 34] -Output [2]: [ca_address_sk#31, ca_country#34] -Input [3]: [ca_address_sk#31, ca_state#33, ca_country#34] +Output [2]: [ca_address_sk#164, ca_country#166] +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] (87) BroadcastExchange -Input [2]: [ca_address_sk#31, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#135] +Input [2]: [ca_address_sk#164, ca_country#166] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#167] (88) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [c_current_addr_sk#25] -Right keys [1]: [ca_address_sk#31] +Left keys [1]: [c_current_addr_sk#161] +Right keys [1]: [ca_address_sk#164] Join condition: None (89) Project [codegen id : 35] -Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34] -Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31, ca_country#34] +Output [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Input [6]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_year#163, ca_address_sk#164, ca_country#166] (90) Exchange -Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34] -Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#136] +Input [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Arguments: hashpartitioning(c_current_cdemo_sk#160, 5), ENSURE_REQUIREMENTS, [id=#168] (91) Sort [codegen id : 36] -Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34] -Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Arguments: [c_current_cdemo_sk#160 ASC NULLS FIRST], false, 0 (92) ReusedExchange [Reuses operator id: 35] -Output [1]: [cd_demo_sk#137] +Output [1]: [cd_demo_sk#169] (93) Sort [codegen id : 38] -Input [1]: [cd_demo_sk#137] -Arguments: [cd_demo_sk#137 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#169] +Arguments: [cd_demo_sk#169 ASC NULLS FIRST], false, 0 (94) SortMergeJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#24] -Right keys [1]: [cd_demo_sk#137] +Left keys [1]: [c_current_cdemo_sk#160] +Right keys [1]: [cd_demo_sk#169] Join condition: None (95) Project [codegen id : 39] -Output [3]: [c_customer_sk#23, c_birth_year#27, ca_country#34] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_country#34, cd_demo_sk#137] +Output [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166, cd_demo_sk#169] (96) Exchange -Input [3]: [c_customer_sk#23, c_birth_year#27, ca_country#34] -Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#138] +Input [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Arguments: hashpartitioning(c_customer_sk#159, 5), ENSURE_REQUIREMENTS, [id=#170] (97) Sort [codegen id : 40] -Input [3]: [c_customer_sk#23, c_birth_year#27, ca_country#34] -Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Arguments: [c_customer_sk#159 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 41] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#23] +Left keys [1]: [cs_bill_customer_sk#151] +Right keys [1]: [c_customer_sk#159] Join condition: None (99) Project [codegen id : 41] -Output [9]: [i_item_id#20, ca_country#34, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] -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#20, c_customer_sk#23, c_birth_year#27, ca_country#34] +Output [9]: [i_item_id#158, ca_country#166, cast(cs_quantity#152 as decimal(12,2)) AS agg1#37, cast(cs_list_price#153 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#155 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#154 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#156 as decimal(12,2)) AS agg5#41, cast(c_birth_year#163 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#157 as decimal(12,2)) AS agg7#43] +Input [11]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158, c_customer_sk#159, c_birth_year#163, ca_country#166] (100) HashAggregate [codegen id : 41] -Input [9]: [i_item_id#20, ca_country#34, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] -Keys [2]: [i_item_id#20, ca_country#34] +Input [9]: [i_item_id#158, ca_country#166, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [2]: [i_item_id#158, ca_country#166] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#139, count#140, sum#141, count#142, sum#143, count#144, sum#145, count#146, sum#147, count#148, sum#149, count#150, sum#151, count#152] -Results [16]: [i_item_id#20, ca_country#34, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] +Aggregate Attributes [14]: [sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178, sum#179, count#180, sum#181, count#182, sum#183, count#184] +Results [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] (101) Exchange -Input [16]: [i_item_id#20, ca_country#34, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] -Arguments: hashpartitioning(i_item_id#20, ca_country#34, 5), ENSURE_REQUIREMENTS, [id=#167] +Input [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] +Arguments: hashpartitioning(i_item_id#158, ca_country#166, 5), ENSURE_REQUIREMENTS, [id=#199] (102) HashAggregate [codegen id : 42] -Input [16]: [i_item_id#20, ca_country#34, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160, sum#161, count#162, sum#163, count#164, sum#165, count#166] -Keys [2]: [i_item_id#20, ca_country#34] +Input [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] +Keys [2]: [i_item_id#158, ca_country#166] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#168, avg(agg2#38)#169, avg(agg3#39)#170, avg(agg4#40)#171, avg(agg5#41)#172, avg(agg6#42)#173, avg(agg7#43)#174] -Results [11]: [i_item_id#20, ca_country#34, null AS ca_state#175, null AS county#176, avg(agg1#37)#168 AS agg1#177, avg(agg2#38)#169 AS agg2#178, avg(agg3#39)#170 AS agg3#179, avg(agg4#40)#171 AS agg4#180, avg(agg5#41)#172 AS agg5#181, avg(agg6#42)#173 AS agg6#182, avg(agg7#43)#174 AS agg7#183] +Aggregate Attributes [7]: [avg(agg1#37)#200, avg(agg2#38)#201, avg(agg3#39)#202, avg(agg4#40)#203, avg(agg5#41)#204, avg(agg6#42)#205, avg(agg7#43)#206] +Results [11]: [i_item_id#158, ca_country#166, null AS ca_state#207, null AS county#208, avg(agg1#37)#200 AS agg1#209, avg(agg2#38)#201 AS agg2#210, avg(agg3#39)#202 AS agg3#211, avg(agg4#40)#203 AS agg4#212, avg(agg5#41)#204 AS agg5#213, avg(agg6#42)#205 AS agg6#214, avg(agg7#43)#206 AS agg7#215] (103) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#224), dynamicpruningexpression(cs_sold_date_sk#224 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (104) ColumnarToRow [codegen id : 49] -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] +Input [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] (105) Filter [codegen id : 49] -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)) +Input [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] +Condition : ((isnotnull(cs_bill_cdemo_sk#217) AND isnotnull(cs_bill_customer_sk#216)) AND isnotnull(cs_item_sk#218)) (106) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#225, cd_dep_count#226] (107) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#217] +Right keys [1]: [cd_demo_sk#225] Join condition: None (108) Project [codegen id : 49] -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] +Output [9]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_dep_count#226] +Input [11]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_demo_sk#225, cd_dep_count#226] (109) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#227] (110) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#16] +Left keys [1]: [cs_sold_date_sk#224] +Right keys [1]: [d_date_sk#227] Join condition: None (111) Project [codegen id : 49] -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] +Output [8]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226] +Input [10]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_dep_count#226, d_date_sk#227] (112) Scan parquet default.customer -Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct (113) ColumnarToRow [codegen id : 46] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] (114) Filter [codegen id : 46] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] +Condition : (((c_birth_month#231 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#228)) AND isnotnull(c_current_cdemo_sk#229)) AND isnotnull(c_current_addr_sk#230)) (115) Project [codegen id : 46] -Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Output [4]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_year#232] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] (116) Scan parquet default.customer_address -Output [2]: [ca_address_sk#31, ca_state#33] +Output [2]: [ca_address_sk#233, ca_state#234] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (117) ColumnarToRow [codegen id : 45] -Input [2]: [ca_address_sk#31, ca_state#33] +Input [2]: [ca_address_sk#233, ca_state#234] (118) Filter [codegen id : 45] -Input [2]: [ca_address_sk#31, ca_state#33] -Condition : (ca_state#33 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#31)) +Input [2]: [ca_address_sk#233, ca_state#234] +Condition : (ca_state#234 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#233)) (119) Project [codegen id : 45] -Output [1]: [ca_address_sk#31] -Input [2]: [ca_address_sk#31, ca_state#33] +Output [1]: [ca_address_sk#233] +Input [2]: [ca_address_sk#233, ca_state#234] (120) BroadcastExchange -Input [1]: [ca_address_sk#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#184] +Input [1]: [ca_address_sk#233] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#235] (121) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [c_current_addr_sk#25] -Right keys [1]: [ca_address_sk#31] +Left keys [1]: [c_current_addr_sk#230] +Right keys [1]: [ca_address_sk#233] Join condition: None (122) Project [codegen id : 46] -Output [3]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27] -Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#31] +Output [3]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_year#232, ca_address_sk#233] (123) BroadcastExchange -Input [3]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#185] +Input [3]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#236] (124) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#186] +Output [1]: [cd_demo_sk#237] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (125) ColumnarToRow -Input [1]: [cd_demo_sk#186] +Input [1]: [cd_demo_sk#237] (126) Filter -Input [1]: [cd_demo_sk#186] -Condition : isnotnull(cd_demo_sk#186) +Input [1]: [cd_demo_sk#237] +Condition : isnotnull(cd_demo_sk#237) (127) BroadcastHashJoin [codegen id : 47] -Left keys [1]: [c_current_cdemo_sk#24] -Right keys [1]: [cd_demo_sk#186] +Left keys [1]: [c_current_cdemo_sk#229] +Right keys [1]: [cd_demo_sk#237] Join condition: None (128) Project [codegen id : 47] -Output [2]: [c_customer_sk#23, c_birth_year#27] -Input [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, cd_demo_sk#186] +Output [2]: [c_customer_sk#228, c_birth_year#232] +Input [4]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232, cd_demo_sk#237] (129) BroadcastExchange -Input [2]: [c_customer_sk#23, c_birth_year#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#187] +Input [2]: [c_customer_sk#228, c_birth_year#232] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#238] (130) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#23] +Left keys [1]: [cs_bill_customer_sk#216] +Right keys [1]: [c_customer_sk#228] Join condition: None (131) Project [codegen id : 49] -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#27] -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#23, c_birth_year#27] +Output [8]: [cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_birth_year#232] +Input [10]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_customer_sk#228, c_birth_year#232] (132) ReusedExchange [Reuses operator id: 21] -Output [2]: [i_item_sk#19, i_item_id#20] +Output [2]: [i_item_sk#239, i_item_id#240] (133) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#19] +Left keys [1]: [cs_item_sk#218] +Right keys [1]: [i_item_sk#239] Join condition: None (134) Project [codegen id : 49] -Output [8]: [i_item_id#20, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] -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#27, i_item_sk#19, i_item_id#20] +Output [8]: [i_item_id#240, cast(cs_quantity#219 as decimal(12,2)) AS agg1#37, cast(cs_list_price#220 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#222 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#221 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#223 as decimal(12,2)) AS agg5#41, cast(c_birth_year#232 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#226 as decimal(12,2)) AS agg7#43] +Input [10]: [cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_birth_year#232, i_item_sk#239, i_item_id#240] (135) HashAggregate [codegen id : 49] -Input [8]: [i_item_id#20, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] -Keys [1]: [i_item_id#20] +Input [8]: [i_item_id#240, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [1]: [i_item_id#240] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#188, count#189, sum#190, count#191, sum#192, count#193, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201] -Results [15]: [i_item_id#20, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215] +Aggregate Attributes [14]: [sum#241, count#242, sum#243, count#244, sum#245, count#246, sum#247, count#248, sum#249, count#250, sum#251, count#252, sum#253, count#254] +Results [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] (136) Exchange -Input [15]: [i_item_id#20, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215] -Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, [id=#216] +Input [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] +Arguments: hashpartitioning(i_item_id#240, 5), ENSURE_REQUIREMENTS, [id=#269] (137) HashAggregate [codegen id : 50] -Input [15]: [i_item_id#20, sum#202, count#203, sum#204, count#205, sum#206, count#207, sum#208, count#209, sum#210, count#211, sum#212, count#213, sum#214, count#215] -Keys [1]: [i_item_id#20] +Input [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] +Keys [1]: [i_item_id#240] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#217, avg(agg2#38)#218, avg(agg3#39)#219, avg(agg4#40)#220, avg(agg5#41)#221, avg(agg6#42)#222, avg(agg7#43)#223] -Results [11]: [i_item_id#20, null AS ca_country#224, null AS ca_state#225, null AS county#226, avg(agg1#37)#217 AS agg1#227, avg(agg2#38)#218 AS agg2#228, avg(agg3#39)#219 AS agg3#229, avg(agg4#40)#220 AS agg4#230, avg(agg5#41)#221 AS agg5#231, avg(agg6#42)#222 AS agg6#232, avg(agg7#43)#223 AS agg7#233] +Aggregate Attributes [7]: [avg(agg1#37)#270, avg(agg2#38)#271, avg(agg3#39)#272, avg(agg4#40)#273, avg(agg5#41)#274, avg(agg6#42)#275, avg(agg7#43)#276] +Results [11]: [i_item_id#240, null AS ca_country#277, null AS ca_state#278, null AS county#279, avg(agg1#37)#270 AS agg1#280, avg(agg2#38)#271 AS agg2#281, avg(agg3#39)#272 AS agg3#282, avg(agg4#40)#273 AS agg4#283, avg(agg5#41)#274 AS agg5#284, avg(agg6#42)#275 AS agg6#285, avg(agg7#43)#276 AS agg7#286] (138) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#295), dynamicpruningexpression(cs_sold_date_sk#295 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (139) ColumnarToRow [codegen id : 57] -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] +Input [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] (140) Filter [codegen id : 57] -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)) +Input [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] +Condition : ((isnotnull(cs_bill_cdemo_sk#288) AND isnotnull(cs_bill_customer_sk#287)) AND isnotnull(cs_item_sk#289)) (141) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#296, cd_dep_count#297] (142) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#288] +Right keys [1]: [cd_demo_sk#296] Join condition: None (143) Project [codegen id : 57] -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] +Output [9]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_dep_count#297] +Input [11]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_demo_sk#296, cd_dep_count#297] (144) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#298] (145) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#16] +Left keys [1]: [cs_sold_date_sk#295] +Right keys [1]: [d_date_sk#298] Join condition: None (146) Project [codegen id : 57] -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] +Output [8]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297] +Input [10]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_dep_count#297, d_date_sk#298] (147) Scan parquet default.item -Output [1]: [i_item_sk#19] +Output [1]: [i_item_sk#299] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (148) ColumnarToRow [codegen id : 53] -Input [1]: [i_item_sk#19] +Input [1]: [i_item_sk#299] (149) Filter [codegen id : 53] -Input [1]: [i_item_sk#19] -Condition : isnotnull(i_item_sk#19) +Input [1]: [i_item_sk#299] +Condition : isnotnull(i_item_sk#299) (150) BroadcastExchange -Input [1]: [i_item_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#234] +Input [1]: [i_item_sk#299] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#300] (151) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#19] +Left keys [1]: [cs_item_sk#289] +Right keys [1]: [i_item_sk#299] Join condition: None (152) Project [codegen id : 57] -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#19] +Output [7]: [cs_bill_customer_sk#287, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297] +Input [9]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297, i_item_sk#299] (153) ReusedExchange [Reuses operator id: 129] -Output [2]: [c_customer_sk#23, c_birth_year#27] +Output [2]: [c_customer_sk#301, c_birth_year#302] (154) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#23] +Left keys [1]: [cs_bill_customer_sk#287] +Right keys [1]: [c_customer_sk#301] Join condition: None (155) Project [codegen id : 57] -Output [7]: [cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] -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#23, c_birth_year#27] +Output [7]: [cast(cs_quantity#290 as decimal(12,2)) AS agg1#37, cast(cs_list_price#291 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#293 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#292 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#294 as decimal(12,2)) AS agg5#41, cast(c_birth_year#302 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#297 as decimal(12,2)) AS agg7#43] +Input [9]: [cs_bill_customer_sk#287, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297, c_customer_sk#301, c_birth_year#302] (156) HashAggregate [codegen id : 57] Input [7]: [agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] Keys: [] Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] -Aggregate Attributes [14]: [sum#235, count#236, sum#237, count#238, sum#239, count#240, sum#241, count#242, sum#243, count#244, sum#245, count#246, sum#247, count#248] -Results [14]: [sum#249, count#250, sum#251, count#252, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262] +Aggregate Attributes [14]: [sum#303, count#304, sum#305, count#306, sum#307, count#308, sum#309, count#310, sum#311, count#312, sum#313, count#314, sum#315, count#316] +Results [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] (157) Exchange -Input [14]: [sum#249, count#250, sum#251, count#252, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#263] +Input [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#331] (158) HashAggregate [codegen id : 58] -Input [14]: [sum#249, count#250, sum#251, count#252, sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262] +Input [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] Keys: [] Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] -Aggregate Attributes [7]: [avg(agg1#37)#264, avg(agg2#38)#265, avg(agg3#39)#266, avg(agg4#40)#267, avg(agg5#41)#268, avg(agg6#42)#269, avg(agg7#43)#270] -Results [11]: [null AS i_item_id#271, null AS ca_country#272, null AS ca_state#273, null AS county#274, avg(agg1#37)#264 AS agg1#275, avg(agg2#38)#265 AS agg2#276, avg(agg3#39)#266 AS agg3#277, avg(agg4#40)#267 AS agg4#278, avg(agg5#41)#268 AS agg5#279, avg(agg6#42)#269 AS agg6#280, avg(agg7#43)#270 AS agg7#281] +Aggregate Attributes [7]: [avg(agg1#37)#332, avg(agg2#38)#333, avg(agg3#39)#334, avg(agg4#40)#335, avg(agg5#41)#336, avg(agg6#42)#337, avg(agg7#43)#338] +Results [11]: [null AS i_item_id#339, null AS ca_country#340, null AS ca_state#341, null AS county#342, avg(agg1#37)#332 AS agg1#343, avg(agg2#38)#333 AS agg2#344, avg(agg3#39)#334 AS agg3#345, avg(agg4#40)#335 AS agg4#346, avg(agg5#41)#336 AS agg5#347, avg(agg6#42)#337 AS agg6#348, avg(agg7#43)#338 AS agg7#349] (159) Union @@ -887,8 +887,8 @@ ReusedExchange (161) (161) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#16] -Subquery:2 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#224 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 138 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 138 Hosting Expression = cs_sold_date_sk#295 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 0cb9a331b406a..8e3c5958d2831 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 @@ -366,492 +366,492 @@ Aggregate Attributes [7]: [avg(agg1#35)#71, avg(agg2#36)#72, avg(agg3#37)#73, av Results [11]: [i_item_id#33, ca_country#27, ca_state#26, ca_county#25, 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] (46) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#93), dynamicpruningexpression(cs_sold_date_sk#93 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 15] -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] +Input [9]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93] (48) Filter [codegen id : 15] -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)) +Input [9]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93] +Condition : ((isnotnull(cs_bill_cdemo_sk#86) AND isnotnull(cs_bill_customer_sk#85)) AND isnotnull(cs_item_sk#87)) (49) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#94, cd_dep_count#95] (50) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#86] +Right keys [1]: [cd_demo_sk#94] Join condition: None (51) Project [codegen id : 15] -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] +Output [9]: [cs_bill_customer_sk#85, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95] +Input [11]: [cs_bill_customer_sk#85, cs_bill_cdemo_sk#86, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_demo_sk#94, cd_dep_count#95] (52) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] +Output [4]: [c_customer_sk#96, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99] (53) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#16] +Left keys [1]: [cs_bill_customer_sk#85] +Right keys [1]: [c_customer_sk#96] Join condition: None (54) Project [codegen id : 15] -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] +Output [11]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99] +Input [13]: [cs_bill_customer_sk#85, cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_customer_sk#96, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99] (55) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#85] +Output [1]: [cd_demo_sk#100] (56) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_cdemo_sk#17] -Right keys [1]: [cd_demo_sk#85] +Left keys [1]: [c_current_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#100] Join condition: None (57) Project [codegen id : 15] -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#85] +Output [10]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_addr_sk#98, c_birth_year#99] +Input [12]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_cdemo_sk#97, c_current_addr_sk#98, c_birth_year#99, cd_demo_sk#100] (58) Scan parquet default.customer_address -Output [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Output [3]: [ca_address_sk#101, ca_state#102, ca_country#103] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 12] -Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Input [3]: [ca_address_sk#101, ca_state#102, ca_country#103] (60) Filter [codegen id : 12] -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)) +Input [3]: [ca_address_sk#101, ca_state#102, ca_country#103] +Condition : (ca_state#102 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#101)) (61) BroadcastExchange -Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#86] +Input [3]: [ca_address_sk#101, ca_state#102, ca_country#103] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#104] (62) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [c_current_addr_sk#18] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [c_current_addr_sk#98] +Right keys [1]: [ca_address_sk#101] Join condition: None (63) Project [codegen id : 15] -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] +Output [11]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103] +Input [13]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_current_addr_sk#98, c_birth_year#99, ca_address_sk#101, ca_state#102, ca_country#103] (64) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#105] (65) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [cs_sold_date_sk#93] +Right keys [1]: [d_date_sk#105] Join condition: None (66) Project [codegen id : 15] -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] +Output [10]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103] +Input [12]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cs_sold_date_sk#93, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103, d_date_sk#105] (67) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#32, i_item_id#33] +Output [2]: [i_item_sk#106, i_item_id#107] (68) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [cs_item_sk#87] +Right keys [1]: [i_item_sk#106] Join condition: None (69) Project [codegen id : 15] -Output [10]: [i_item_id#33, ca_country#27, ca_state#26, 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#20 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] -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#32, i_item_id#33] +Output [10]: [i_item_id#107, ca_country#103, ca_state#102, cast(cs_quantity#88 as decimal(12,2)) AS agg1#35, cast(cs_list_price#89 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#91 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#90 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#92 as decimal(12,2)) AS agg5#39, cast(c_birth_year#99 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#95 as decimal(12,2)) AS agg7#41] +Input [12]: [cs_item_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#95, c_birth_year#99, ca_state#102, ca_country#103, i_item_sk#106, i_item_id#107] (70) HashAggregate [codegen id : 15] -Input [10]: [i_item_id#33, ca_country#27, ca_state#26, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [3]: [i_item_id#33, ca_country#27, ca_state#26] +Input [10]: [i_item_id#107, ca_country#103, ca_state#102, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [3]: [i_item_id#107, ca_country#103, ca_state#102] 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#87, count#88, sum#89, count#90, sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98, sum#99, count#100] -Results [17]: [i_item_id#33, ca_country#27, ca_state#26, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] +Aggregate Attributes [14]: [sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117, sum#118, count#119, sum#120, count#121] +Results [17]: [i_item_id#107, ca_country#103, ca_state#102, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] (71) Exchange -Input [17]: [i_item_id#33, ca_country#27, ca_state#26, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] -Arguments: hashpartitioning(i_item_id#33, ca_country#27, ca_state#26, 5), ENSURE_REQUIREMENTS, [id=#115] +Input [17]: [i_item_id#107, ca_country#103, ca_state#102, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] +Arguments: hashpartitioning(i_item_id#107, ca_country#103, ca_state#102, 5), ENSURE_REQUIREMENTS, [id=#136] (72) HashAggregate [codegen id : 16] -Input [17]: [i_item_id#33, ca_country#27, ca_state#26, sum#101, count#102, sum#103, count#104, sum#105, count#106, sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114] -Keys [3]: [i_item_id#33, ca_country#27, ca_state#26] +Input [17]: [i_item_id#107, ca_country#103, ca_state#102, sum#122, count#123, sum#124, count#125, sum#126, count#127, sum#128, count#129, sum#130, count#131, sum#132, count#133, sum#134, count#135] +Keys [3]: [i_item_id#107, ca_country#103, ca_state#102] 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)#116, avg(agg2#36)#117, avg(agg3#37)#118, avg(agg4#38)#119, avg(agg5#39)#120, avg(agg6#40)#121, avg(agg7#41)#122] -Results [11]: [i_item_id#33, ca_country#27, ca_state#26, null AS county#123, avg(agg1#35)#116 AS agg1#124, avg(agg2#36)#117 AS agg2#125, avg(agg3#37)#118 AS agg3#126, avg(agg4#38)#119 AS agg4#127, avg(agg5#39)#120 AS agg5#128, avg(agg6#40)#121 AS agg6#129, avg(agg7#41)#122 AS agg7#130] +Aggregate Attributes [7]: [avg(agg1#35)#137, avg(agg2#36)#138, avg(agg3#37)#139, avg(agg4#38)#140, avg(agg5#39)#141, avg(agg6#40)#142, avg(agg7#41)#143] +Results [11]: [i_item_id#107, ca_country#103, ca_state#102, null AS county#144, avg(agg1#35)#137 AS agg1#145, avg(agg2#36)#138 AS agg2#146, avg(agg3#37)#139 AS agg3#147, avg(agg4#38)#140 AS agg4#148, avg(agg5#39)#141 AS agg5#149, avg(agg6#40)#142 AS agg6#150, avg(agg7#41)#143 AS agg7#151] (73) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#160), dynamicpruningexpression(cs_sold_date_sk#160 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (74) ColumnarToRow [codegen id : 23] -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] +Input [9]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160] (75) Filter [codegen id : 23] -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)) +Input [9]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160] +Condition : ((isnotnull(cs_bill_cdemo_sk#153) AND isnotnull(cs_bill_customer_sk#152)) AND isnotnull(cs_item_sk#154)) (76) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#161, cd_dep_count#162] (77) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#153] +Right keys [1]: [cd_demo_sk#161] Join condition: None (78) Project [codegen id : 23] -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] +Output [9]: [cs_bill_customer_sk#152, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162] +Input [11]: [cs_bill_customer_sk#152, cs_bill_cdemo_sk#153, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_demo_sk#161, cd_dep_count#162] (79) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] +Output [4]: [c_customer_sk#163, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] (80) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#16] +Left keys [1]: [cs_bill_customer_sk#152] +Right keys [1]: [c_customer_sk#163] Join condition: None (81) Project [codegen id : 23] -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] +Output [11]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] +Input [13]: [cs_bill_customer_sk#152, cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_customer_sk#163, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166] (82) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#131] +Output [1]: [cd_demo_sk#167] (83) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_cdemo_sk#17] -Right keys [1]: [cd_demo_sk#131] +Left keys [1]: [c_current_cdemo_sk#164] +Right keys [1]: [cd_demo_sk#167] Join condition: None (84) Project [codegen id : 23] -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#131] +Output [10]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_addr_sk#165, c_birth_year#166] +Input [12]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_cdemo_sk#164, c_current_addr_sk#165, c_birth_year#166, cd_demo_sk#167] (85) Scan parquet default.customer_address -Output [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Output [3]: [ca_address_sk#168, ca_state#169, ca_country#170] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (86) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] (87) Filter [codegen id : 20] -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)) +Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] +Condition : (ca_state#169 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#168)) (88) Project [codegen id : 20] -Output [2]: [ca_address_sk#24, ca_country#27] -Input [3]: [ca_address_sk#24, ca_state#26, ca_country#27] +Output [2]: [ca_address_sk#168, ca_country#170] +Input [3]: [ca_address_sk#168, ca_state#169, ca_country#170] (89) BroadcastExchange -Input [2]: [ca_address_sk#24, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#132] +Input [2]: [ca_address_sk#168, ca_country#170] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#171] (90) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [c_current_addr_sk#18] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [c_current_addr_sk#165] +Right keys [1]: [ca_address_sk#168] Join condition: None (91) Project [codegen id : 23] -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] +Output [10]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_birth_year#166, ca_country#170] +Input [12]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_current_addr_sk#165, c_birth_year#166, ca_address_sk#168, ca_country#170] (92) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#172] (93) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [cs_sold_date_sk#160] +Right keys [1]: [d_date_sk#172] Join condition: None (94) Project [codegen id : 23] -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] +Output [9]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cd_dep_count#162, c_birth_year#166, ca_country#170] +Input [11]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cs_sold_date_sk#160, cd_dep_count#162, c_birth_year#166, ca_country#170, d_date_sk#172] (95) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#32, i_item_id#33] +Output [2]: [i_item_sk#173, i_item_id#174] (96) BroadcastHashJoin [codegen id : 23] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [cs_item_sk#154] +Right keys [1]: [i_item_sk#173] Join condition: None (97) Project [codegen id : 23] -Output [9]: [i_item_id#33, ca_country#27, 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#20 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] -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#32, i_item_id#33] +Output [9]: [i_item_id#174, ca_country#170, cast(cs_quantity#155 as decimal(12,2)) AS agg1#35, cast(cs_list_price#156 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#158 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#157 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#159 as decimal(12,2)) AS agg5#39, cast(c_birth_year#166 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#162 as decimal(12,2)) AS agg7#41] +Input [11]: [cs_item_sk#154, cs_quantity#155, cs_list_price#156, cs_sales_price#157, cs_coupon_amt#158, cs_net_profit#159, cd_dep_count#162, c_birth_year#166, ca_country#170, i_item_sk#173, i_item_id#174] (98) HashAggregate [codegen id : 23] -Input [9]: [i_item_id#33, ca_country#27, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [2]: [i_item_id#33, ca_country#27] +Input [9]: [i_item_id#174, ca_country#170, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [2]: [i_item_id#174, ca_country#170] 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#133, count#134, sum#135, count#136, sum#137, count#138, sum#139, count#140, sum#141, count#142, sum#143, count#144, sum#145, count#146] -Results [16]: [i_item_id#33, ca_country#27, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160] +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 [16]: [i_item_id#174, ca_country#170, 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] (99) Exchange -Input [16]: [i_item_id#33, ca_country#27, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160] -Arguments: hashpartitioning(i_item_id#33, ca_country#27, 5), ENSURE_REQUIREMENTS, [id=#161] +Input [16]: [i_item_id#174, ca_country#170, 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#174, ca_country#170, 5), ENSURE_REQUIREMENTS, [id=#203] (100) HashAggregate [codegen id : 24] -Input [16]: [i_item_id#33, ca_country#27, sum#147, count#148, sum#149, count#150, sum#151, count#152, sum#153, count#154, sum#155, count#156, sum#157, count#158, sum#159, count#160] -Keys [2]: [i_item_id#33, ca_country#27] +Input [16]: [i_item_id#174, ca_country#170, 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 [2]: [i_item_id#174, ca_country#170] 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)#162, avg(agg2#36)#163, avg(agg3#37)#164, avg(agg4#38)#165, avg(agg5#39)#166, avg(agg6#40)#167, avg(agg7#41)#168] -Results [11]: [i_item_id#33, ca_country#27, null AS ca_state#169, null AS county#170, avg(agg1#35)#162 AS agg1#171, avg(agg2#36)#163 AS agg2#172, avg(agg3#37)#164 AS agg3#173, avg(agg4#38)#165 AS agg4#174, avg(agg5#39)#166 AS agg5#175, avg(agg6#40)#167 AS agg6#176, avg(agg7#41)#168 AS agg7#177] +Aggregate Attributes [7]: [avg(agg1#35)#204, avg(agg2#36)#205, avg(agg3#37)#206, avg(agg4#38)#207, avg(agg5#39)#208, avg(agg6#40)#209, avg(agg7#41)#210] +Results [11]: [i_item_id#174, ca_country#170, null AS ca_state#211, null AS county#212, avg(agg1#35)#204 AS agg1#213, avg(agg2#36)#205 AS agg2#214, avg(agg3#37)#206 AS agg3#215, avg(agg4#38)#207 AS agg4#216, avg(agg5#39)#208 AS agg5#217, avg(agg6#40)#209 AS agg6#218, avg(agg7#41)#210 AS agg7#219] (101) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#228), dynamicpruningexpression(cs_sold_date_sk#228 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (102) ColumnarToRow [codegen id : 31] -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] +Input [9]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228] (103) Filter [codegen id : 31] -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)) +Input [9]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228] +Condition : ((isnotnull(cs_bill_cdemo_sk#221) AND isnotnull(cs_bill_customer_sk#220)) AND isnotnull(cs_item_sk#222)) (104) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#229, cd_dep_count#230] (105) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#221] +Right keys [1]: [cd_demo_sk#229] Join condition: None (106) Project [codegen id : 31] -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] +Output [9]: [cs_bill_customer_sk#220, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230] +Input [11]: [cs_bill_customer_sk#220, cs_bill_cdemo_sk#221, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_demo_sk#229, cd_dep_count#230] (107) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] +Output [4]: [c_customer_sk#231, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234] (108) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#16] +Left keys [1]: [cs_bill_customer_sk#220] +Right keys [1]: [c_customer_sk#231] Join condition: None (109) Project [codegen id : 31] -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] +Output [11]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234] +Input [13]: [cs_bill_customer_sk#220, cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_customer_sk#231, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234] (110) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#178] +Output [1]: [cd_demo_sk#235] (111) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_cdemo_sk#17] -Right keys [1]: [cd_demo_sk#178] +Left keys [1]: [c_current_cdemo_sk#232] +Right keys [1]: [cd_demo_sk#235] Join condition: None (112) Project [codegen id : 31] -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#178] +Output [10]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_addr_sk#233, c_birth_year#234] +Input [12]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_cdemo_sk#232, c_current_addr_sk#233, c_birth_year#234, cd_demo_sk#235] (113) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_state#26] +Output [2]: [ca_address_sk#236, ca_state#237] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (114) ColumnarToRow [codegen id : 28] -Input [2]: [ca_address_sk#24, ca_state#26] +Input [2]: [ca_address_sk#236, ca_state#237] (115) Filter [codegen id : 28] -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)) +Input [2]: [ca_address_sk#236, ca_state#237] +Condition : (ca_state#237 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#236)) (116) Project [codegen id : 28] -Output [1]: [ca_address_sk#24] -Input [2]: [ca_address_sk#24, ca_state#26] +Output [1]: [ca_address_sk#236] +Input [2]: [ca_address_sk#236, ca_state#237] (117) BroadcastExchange -Input [1]: [ca_address_sk#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#179] +Input [1]: [ca_address_sk#236] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#238] (118) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [c_current_addr_sk#18] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [c_current_addr_sk#233] +Right keys [1]: [ca_address_sk#236] Join condition: None (119) Project [codegen id : 31] -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] +Output [9]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_birth_year#234] +Input [11]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_current_addr_sk#233, c_birth_year#234, ca_address_sk#236] (120) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#239] (121) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [cs_sold_date_sk#228] +Right keys [1]: [d_date_sk#239] Join condition: None (122) Project [codegen id : 31] -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] +Output [8]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cd_dep_count#230, c_birth_year#234] +Input [10]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cs_sold_date_sk#228, cd_dep_count#230, c_birth_year#234, d_date_sk#239] (123) ReusedExchange [Reuses operator id: 40] -Output [2]: [i_item_sk#32, i_item_id#33] +Output [2]: [i_item_sk#240, i_item_id#241] (124) BroadcastHashJoin [codegen id : 31] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [cs_item_sk#222] +Right keys [1]: [i_item_sk#240] Join condition: None (125) Project [codegen id : 31] -Output [8]: [i_item_id#33, 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#20 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#20, i_item_sk#32, i_item_id#33] +Output [8]: [i_item_id#241, cast(cs_quantity#223 as decimal(12,2)) AS agg1#35, cast(cs_list_price#224 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#226 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#225 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#227 as decimal(12,2)) AS agg5#39, cast(c_birth_year#234 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#230 as decimal(12,2)) AS agg7#41] +Input [10]: [cs_item_sk#222, cs_quantity#223, cs_list_price#224, cs_sales_price#225, cs_coupon_amt#226, cs_net_profit#227, cd_dep_count#230, c_birth_year#234, i_item_sk#240, i_item_id#241] (126) HashAggregate [codegen id : 31] -Input [8]: [i_item_id#33, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] -Keys [1]: [i_item_id#33] +Input [8]: [i_item_id#241, agg1#35, agg2#36, agg3#37, agg4#38, agg5#39, agg6#40, agg7#41] +Keys [1]: [i_item_id#241] 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#180, count#181, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193] -Results [15]: [i_item_id#33, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] +Aggregate Attributes [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] +Results [15]: [i_item_id#241, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] (127) Exchange -Input [15]: [i_item_id#33, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] -Arguments: hashpartitioning(i_item_id#33, 5), ENSURE_REQUIREMENTS, [id=#208] +Input [15]: [i_item_id#241, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] +Arguments: hashpartitioning(i_item_id#241, 5), ENSURE_REQUIREMENTS, [id=#270] (128) HashAggregate [codegen id : 32] -Input [15]: [i_item_id#33, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] -Keys [1]: [i_item_id#33] +Input [15]: [i_item_id#241, sum#256, count#257, sum#258, count#259, sum#260, count#261, sum#262, count#263, sum#264, count#265, sum#266, count#267, sum#268, count#269] +Keys [1]: [i_item_id#241] 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)#209, avg(agg2#36)#210, avg(agg3#37)#211, avg(agg4#38)#212, avg(agg5#39)#213, avg(agg6#40)#214, avg(agg7#41)#215] -Results [11]: [i_item_id#33, null AS ca_country#216, null AS ca_state#217, null AS county#218, avg(agg1#35)#209 AS agg1#219, avg(agg2#36)#210 AS agg2#220, avg(agg3#37)#211 AS agg3#221, avg(agg4#38)#212 AS agg4#222, avg(agg5#39)#213 AS agg5#223, avg(agg6#40)#214 AS agg6#224, avg(agg7#41)#215 AS agg7#225] +Aggregate Attributes [7]: [avg(agg1#35)#271, avg(agg2#36)#272, avg(agg3#37)#273, avg(agg4#38)#274, avg(agg5#39)#275, avg(agg6#40)#276, avg(agg7#41)#277] +Results [11]: [i_item_id#241, null AS ca_country#278, null AS ca_state#279, null AS county#280, avg(agg1#35)#271 AS agg1#281, avg(agg2#36)#272 AS agg2#282, avg(agg3#37)#273 AS agg3#283, avg(agg4#38)#274 AS agg4#284, avg(agg5#39)#275 AS agg5#285, avg(agg6#40)#276 AS agg6#286, avg(agg7#41)#277 AS agg7#287] (129) Scan parquet default.catalog_sales -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] +Output [9]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PartitionFilters: [isnotnull(cs_sold_date_sk#296), dynamicpruningexpression(cs_sold_date_sk#296 IN dynamicpruning#10)] PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] ReadSchema: struct (130) ColumnarToRow [codegen id : 39] -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] +Input [9]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296] (131) Filter [codegen id : 39] -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)) +Input [9]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296] +Condition : ((isnotnull(cs_bill_cdemo_sk#289) AND isnotnull(cs_bill_customer_sk#288)) AND isnotnull(cs_item_sk#290)) (132) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Output [2]: [cd_demo_sk#297, cd_dep_count#298] (133) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#11] +Left keys [1]: [cs_bill_cdemo_sk#289] +Right keys [1]: [cd_demo_sk#297] Join condition: None (134) Project [codegen id : 39] -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] +Output [9]: [cs_bill_customer_sk#288, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298] +Input [11]: [cs_bill_customer_sk#288, cs_bill_cdemo_sk#289, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_demo_sk#297, cd_dep_count#298] (135) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#16, c_current_cdemo_sk#17, c_current_addr_sk#18, c_birth_year#20] +Output [4]: [c_customer_sk#299, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302] (136) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_bill_customer_sk#1] -Right keys [1]: [c_customer_sk#16] +Left keys [1]: [cs_bill_customer_sk#288] +Right keys [1]: [c_customer_sk#299] Join condition: None (137) Project [codegen id : 39] -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] +Output [11]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302] +Input [13]: [cs_bill_customer_sk#288, cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_customer_sk#299, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302] (138) ReusedExchange [Reuses operator id: 21] -Output [1]: [cd_demo_sk#226] +Output [1]: [cd_demo_sk#303] (139) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#17] -Right keys [1]: [cd_demo_sk#226] +Left keys [1]: [c_current_cdemo_sk#300] +Right keys [1]: [cd_demo_sk#303] Join condition: None (140) Project [codegen id : 39] -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#226] +Output [10]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_addr_sk#301, c_birth_year#302] +Input [12]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_cdemo_sk#300, c_current_addr_sk#301, c_birth_year#302, cd_demo_sk#303] (141) ReusedExchange [Reuses operator id: 117] -Output [1]: [ca_address_sk#24] +Output [1]: [ca_address_sk#304] (142) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [c_current_addr_sk#18] -Right keys [1]: [ca_address_sk#24] +Left keys [1]: [c_current_addr_sk#301] +Right keys [1]: [ca_address_sk#304] Join condition: None (143) Project [codegen id : 39] -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] +Output [9]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_birth_year#302] +Input [11]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_current_addr_sk#301, c_birth_year#302, ca_address_sk#304] (144) ReusedExchange [Reuses operator id: 34] -Output [1]: [d_date_sk#29] +Output [1]: [d_date_sk#305] (145) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_sold_date_sk#9] -Right keys [1]: [d_date_sk#29] +Left keys [1]: [cs_sold_date_sk#296] +Right keys [1]: [d_date_sk#305] Join condition: None (146) Project [codegen id : 39] -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] +Output [8]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cd_dep_count#298, c_birth_year#302] +Input [10]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cs_sold_date_sk#296, cd_dep_count#298, c_birth_year#302, d_date_sk#305] (147) Scan parquet default.item -Output [1]: [i_item_sk#32] +Output [1]: [i_item_sk#306] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (148) ColumnarToRow [codegen id : 38] -Input [1]: [i_item_sk#32] +Input [1]: [i_item_sk#306] (149) Filter [codegen id : 38] -Input [1]: [i_item_sk#32] -Condition : isnotnull(i_item_sk#32) +Input [1]: [i_item_sk#306] +Condition : isnotnull(i_item_sk#306) (150) BroadcastExchange -Input [1]: [i_item_sk#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#227] +Input [1]: [i_item_sk#306] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#307] (151) BroadcastHashJoin [codegen id : 39] -Left keys [1]: [cs_item_sk#3] -Right keys [1]: [i_item_sk#32] +Left keys [1]: [cs_item_sk#290] +Right keys [1]: [i_item_sk#306] Join condition: None (152) Project [codegen id : 39] -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#20 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#41] -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#32] +Output [7]: [cast(cs_quantity#291 as decimal(12,2)) AS agg1#35, cast(cs_list_price#292 as decimal(12,2)) AS agg2#36, cast(cs_coupon_amt#294 as decimal(12,2)) AS agg3#37, cast(cs_sales_price#293 as decimal(12,2)) AS agg4#38, cast(cs_net_profit#295 as decimal(12,2)) AS agg5#39, cast(c_birth_year#302 as decimal(12,2)) AS agg6#40, cast(cd_dep_count#298 as decimal(12,2)) AS agg7#41] +Input [9]: [cs_item_sk#290, cs_quantity#291, cs_list_price#292, cs_sales_price#293, cs_coupon_amt#294, cs_net_profit#295, cd_dep_count#298, c_birth_year#302, i_item_sk#306] (153) HashAggregate [codegen id : 39] 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#228, count#229, sum#230, count#231, sum#232, count#233, sum#234, count#235, sum#236, count#237, sum#238, count#239, sum#240, count#241] -Results [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] +Aggregate Attributes [14]: [sum#308, count#309, sum#310, count#311, sum#312, count#313, sum#314, count#315, sum#316, count#317, sum#318, count#319, sum#320, count#321] +Results [14]: [sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335] (154) Exchange -Input [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#256] +Input [14]: [sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#336] (155) HashAggregate [codegen id : 40] -Input [14]: [sum#242, count#243, sum#244, count#245, sum#246, count#247, sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255] +Input [14]: [sum#322, count#323, sum#324, count#325, sum#326, count#327, sum#328, count#329, sum#330, count#331, sum#332, count#333, sum#334, count#335] 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)#257, avg(agg2#36)#258, avg(agg3#37)#259, avg(agg4#38)#260, avg(agg5#39)#261, avg(agg6#40)#262, avg(agg7#41)#263] -Results [11]: [null AS i_item_id#264, null AS ca_country#265, null AS ca_state#266, null AS county#267, avg(agg1#35)#257 AS agg1#268, avg(agg2#36)#258 AS agg2#269, avg(agg3#37)#259 AS agg3#270, avg(agg4#38)#260 AS agg4#271, avg(agg5#39)#261 AS agg5#272, avg(agg6#40)#262 AS agg6#273, avg(agg7#41)#263 AS agg7#274] +Aggregate Attributes [7]: [avg(agg1#35)#337, avg(agg2#36)#338, avg(agg3#37)#339, avg(agg4#38)#340, avg(agg5#39)#341, avg(agg6#40)#342, avg(agg7#41)#343] +Results [11]: [null AS i_item_id#344, null AS ca_country#345, null AS ca_state#346, null AS county#347, avg(agg1#35)#337 AS agg1#348, avg(agg2#36)#338 AS agg2#349, avg(agg3#37)#339 AS agg3#350, avg(agg4#38)#340 AS agg4#351, avg(agg5#39)#341 AS agg5#352, avg(agg6#40)#342 AS agg6#353, avg(agg7#41)#343 AS agg7#354] (156) Union @@ -868,12 +868,12 @@ ReusedExchange (158) (158) ReusedExchange [Reuses operator id: 34] Output [1]: [d_date_sk#29] -Subquery:2 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:2 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#93 IN dynamicpruning#10 -Subquery:3 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:3 Hosting operator id = 73 Hosting Expression = cs_sold_date_sk#160 IN dynamicpruning#10 -Subquery:4 Hosting operator id = 101 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:4 Hosting operator id = 101 Hosting Expression = cs_sold_date_sk#228 IN dynamicpruning#10 -Subquery:5 Hosting operator id = 129 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +Subquery:5 Hosting operator id = 129 Hosting Expression = cs_sold_date_sk#296 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 014b89cecd475..f5a7a9135cf29 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 @@ -198,116 +198,116 @@ Aggregate Attributes [1]: [avg(qoh#24)#29] Results [5]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, avg(qoh#24)#29 AS qoh#30] (31) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#31, count#32] +Output [6]: [i_product_name#31, i_brand#32, i_class#33, i_category#34, sum#35, count#36] (32) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#31, count#32] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#33] -Results [4]: [i_product_name#16, i_brand#13, i_class#14, avg(inv_quantity_on_hand#3)#33 AS qoh#24] +Input [6]: [i_product_name#31, i_brand#32, i_class#33, i_category#34, sum#35, count#36] +Keys [4]: [i_product_name#31, i_brand#32, i_class#33, i_category#34] +Functions [1]: [avg(inv_quantity_on_hand#37)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#37)#38] +Results [4]: [i_product_name#31, i_brand#32, i_class#33, avg(inv_quantity_on_hand#37)#38 AS qoh#24] (33) HashAggregate [codegen id : 16] -Input [4]: [i_product_name#16, i_brand#13, i_class#14, qoh#24] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] +Input [4]: [i_product_name#31, i_brand#32, i_class#33, qoh#24] +Keys [3]: [i_product_name#31, i_brand#32, i_class#33] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#34, count#35] -Results [5]: [i_product_name#16, i_brand#13, i_class#14, sum#36, count#37] +Aggregate Attributes [2]: [sum#39, count#40] +Results [5]: [i_product_name#31, i_brand#32, i_class#33, sum#41, count#42] (34) Exchange -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#36, count#37] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, i_class#14, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [5]: [i_product_name#31, i_brand#32, i_class#33, sum#41, count#42] +Arguments: hashpartitioning(i_product_name#31, i_brand#32, i_class#33, 5), ENSURE_REQUIREMENTS, [id=#43] (35) HashAggregate [codegen id : 17] -Input [5]: [i_product_name#16, i_brand#13, i_class#14, sum#36, count#37] -Keys [3]: [i_product_name#16, i_brand#13, i_class#14] +Input [5]: [i_product_name#31, i_brand#32, i_class#33, sum#41, count#42] +Keys [3]: [i_product_name#31, i_brand#32, i_class#33] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#39] -Results [5]: [i_product_name#16, i_brand#13, i_class#14, null AS i_category#40, avg(qoh#24)#39 AS qoh#41] +Aggregate Attributes [1]: [avg(qoh#24)#44] +Results [5]: [i_product_name#31, i_brand#32, i_class#33, null AS i_category#45, avg(qoh#24)#44 AS qoh#46] (36) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#42, count#43] +Output [6]: [i_product_name#47, i_brand#48, i_class#49, i_category#50, sum#51, count#52] (37) HashAggregate [codegen id : 25] -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#42, count#43] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#44] -Results [3]: [i_product_name#16, i_brand#13, avg(inv_quantity_on_hand#3)#44 AS qoh#24] +Input [6]: [i_product_name#47, i_brand#48, i_class#49, i_category#50, sum#51, count#52] +Keys [4]: [i_product_name#47, i_brand#48, i_class#49, i_category#50] +Functions [1]: [avg(inv_quantity_on_hand#53)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#53)#54] +Results [3]: [i_product_name#47, i_brand#48, avg(inv_quantity_on_hand#53)#54 AS qoh#24] (38) HashAggregate [codegen id : 25] -Input [3]: [i_product_name#16, i_brand#13, qoh#24] -Keys [2]: [i_product_name#16, i_brand#13] +Input [3]: [i_product_name#47, i_brand#48, qoh#24] +Keys [2]: [i_product_name#47, i_brand#48] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#45, count#46] -Results [4]: [i_product_name#16, i_brand#13, sum#47, count#48] +Aggregate Attributes [2]: [sum#55, count#56] +Results [4]: [i_product_name#47, i_brand#48, sum#57, count#58] (39) Exchange -Input [4]: [i_product_name#16, i_brand#13, sum#47, count#48] -Arguments: hashpartitioning(i_product_name#16, i_brand#13, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [4]: [i_product_name#47, i_brand#48, sum#57, count#58] +Arguments: hashpartitioning(i_product_name#47, i_brand#48, 5), ENSURE_REQUIREMENTS, [id=#59] (40) HashAggregate [codegen id : 26] -Input [4]: [i_product_name#16, i_brand#13, sum#47, count#48] -Keys [2]: [i_product_name#16, i_brand#13] +Input [4]: [i_product_name#47, i_brand#48, sum#57, count#58] +Keys [2]: [i_product_name#47, i_brand#48] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#50] -Results [5]: [i_product_name#16, i_brand#13, null AS i_class#51, null AS i_category#52, avg(qoh#24)#50 AS qoh#53] +Aggregate Attributes [1]: [avg(qoh#24)#60] +Results [5]: [i_product_name#47, i_brand#48, null AS i_class#61, null AS i_category#62, avg(qoh#24)#60 AS qoh#63] (41) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#54, count#55] +Output [6]: [i_product_name#64, i_brand#65, i_class#66, i_category#67, sum#68, count#69] (42) HashAggregate [codegen id : 34] -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#54, count#55] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#56] -Results [2]: [i_product_name#16, avg(inv_quantity_on_hand#3)#56 AS qoh#24] +Input [6]: [i_product_name#64, i_brand#65, i_class#66, i_category#67, sum#68, count#69] +Keys [4]: [i_product_name#64, i_brand#65, i_class#66, i_category#67] +Functions [1]: [avg(inv_quantity_on_hand#70)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#70)#71] +Results [2]: [i_product_name#64, avg(inv_quantity_on_hand#70)#71 AS qoh#24] (43) HashAggregate [codegen id : 34] -Input [2]: [i_product_name#16, qoh#24] -Keys [1]: [i_product_name#16] +Input [2]: [i_product_name#64, qoh#24] +Keys [1]: [i_product_name#64] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#57, count#58] -Results [3]: [i_product_name#16, sum#59, count#60] +Aggregate Attributes [2]: [sum#72, count#73] +Results [3]: [i_product_name#64, sum#74, count#75] (44) Exchange -Input [3]: [i_product_name#16, sum#59, count#60] -Arguments: hashpartitioning(i_product_name#16, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [3]: [i_product_name#64, sum#74, count#75] +Arguments: hashpartitioning(i_product_name#64, 5), ENSURE_REQUIREMENTS, [id=#76] (45) HashAggregate [codegen id : 35] -Input [3]: [i_product_name#16, sum#59, count#60] -Keys [1]: [i_product_name#16] +Input [3]: [i_product_name#64, sum#74, count#75] +Keys [1]: [i_product_name#64] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#62] -Results [5]: [i_product_name#16, null AS i_brand#63, null AS i_class#64, null AS i_category#65, avg(qoh#24)#62 AS qoh#66] +Aggregate Attributes [1]: [avg(qoh#24)#77] +Results [5]: [i_product_name#64, null AS i_brand#78, null AS i_class#79, null AS i_category#80, avg(qoh#24)#77 AS qoh#81] (46) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#67, count#68] +Output [6]: [i_product_name#82, i_brand#83, i_class#84, i_category#85, sum#86, count#87] (47) HashAggregate [codegen id : 43] -Input [6]: [i_product_name#16, i_brand#13, i_class#14, i_category#15, sum#67, count#68] -Keys [4]: [i_product_name#16, i_brand#13, i_class#14, i_category#15] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#69] -Results [1]: [avg(inv_quantity_on_hand#3)#69 AS qoh#24] +Input [6]: [i_product_name#82, i_brand#83, i_class#84, i_category#85, sum#86, count#87] +Keys [4]: [i_product_name#82, i_brand#83, i_class#84, i_category#85] +Functions [1]: [avg(inv_quantity_on_hand#88)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#88)#89] +Results [1]: [avg(inv_quantity_on_hand#88)#89 AS qoh#24] (48) HashAggregate [codegen id : 43] Input [1]: [qoh#24] Keys: [] Functions [1]: [partial_avg(qoh#24)] -Aggregate Attributes [2]: [sum#70, count#71] -Results [2]: [sum#72, count#73] +Aggregate Attributes [2]: [sum#90, count#91] +Results [2]: [sum#92, count#93] (49) Exchange -Input [2]: [sum#72, count#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#74] +Input [2]: [sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] (50) HashAggregate [codegen id : 44] -Input [2]: [sum#72, count#73] +Input [2]: [sum#92, count#93] Keys: [] Functions [1]: [avg(qoh#24)] -Aggregate Attributes [1]: [avg(qoh#24)#75] -Results [5]: [null AS i_product_name#76, null AS i_brand#77, null AS i_class#78, null AS i_category#79, avg(qoh#24)#75 AS qoh#80] +Aggregate Attributes [1]: [avg(qoh#24)#95] +Results [5]: [null AS i_product_name#96, null AS i_brand#97, null AS i_class#98, null AS i_category#99, avg(qoh#24)#95 AS qoh#100] (51) Union 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 74dd076851c1d..527d2eb5e86a4 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 @@ -183,116 +183,116 @@ Aggregate Attributes [1]: [avg(qoh#23)#28] Results [5]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, avg(qoh#23)#28 AS qoh#29] (28) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#30, count#31] +Output [6]: [i_product_name#30, i_brand#31, i_class#32, i_category#33, sum#34, count#35] (29) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#30, count#31] -Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#32] -Results [4]: [i_product_name#13, i_brand#10, i_class#11, avg(inv_quantity_on_hand#3)#32 AS qoh#23] +Input [6]: [i_product_name#30, i_brand#31, i_class#32, i_category#33, sum#34, count#35] +Keys [4]: [i_product_name#30, i_brand#31, i_class#32, i_category#33] +Functions [1]: [avg(inv_quantity_on_hand#36)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#36)#37] +Results [4]: [i_product_name#30, i_brand#31, i_class#32, avg(inv_quantity_on_hand#36)#37 AS qoh#23] (30) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#13, i_brand#10, i_class#11, qoh#23] -Keys [3]: [i_product_name#13, i_brand#10, i_class#11] +Input [4]: [i_product_name#30, i_brand#31, i_class#32, qoh#23] +Keys [3]: [i_product_name#30, i_brand#31, i_class#32] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#33, count#34] -Results [5]: [i_product_name#13, i_brand#10, i_class#11, sum#35, count#36] +Aggregate Attributes [2]: [sum#38, count#39] +Results [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] (31) Exchange -Input [5]: [i_product_name#13, i_brand#10, i_class#11, sum#35, count#36] -Arguments: hashpartitioning(i_product_name#13, i_brand#10, i_class#11, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] +Arguments: hashpartitioning(i_product_name#30, i_brand#31, i_class#32, 5), ENSURE_REQUIREMENTS, [id=#42] (32) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#13, i_brand#10, i_class#11, sum#35, count#36] -Keys [3]: [i_product_name#13, i_brand#10, i_class#11] +Input [5]: [i_product_name#30, i_brand#31, i_class#32, sum#40, count#41] +Keys [3]: [i_product_name#30, i_brand#31, i_class#32] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#38] -Results [5]: [i_product_name#13, i_brand#10, i_class#11, null AS i_category#39, avg(qoh#23)#38 AS qoh#40] +Aggregate Attributes [1]: [avg(qoh#23)#43] +Results [5]: [i_product_name#30, i_brand#31, i_class#32, null AS i_category#44, avg(qoh#23)#43 AS qoh#45] (33) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#41, count#42] +Output [6]: [i_product_name#46, i_brand#47, i_class#48, i_category#49, sum#50, count#51] (34) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#41, count#42] -Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#43] -Results [3]: [i_product_name#13, i_brand#10, avg(inv_quantity_on_hand#3)#43 AS qoh#23] +Input [6]: [i_product_name#46, i_brand#47, i_class#48, i_category#49, sum#50, count#51] +Keys [4]: [i_product_name#46, i_brand#47, i_class#48, i_category#49] +Functions [1]: [avg(inv_quantity_on_hand#52)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#52)#53] +Results [3]: [i_product_name#46, i_brand#47, avg(inv_quantity_on_hand#52)#53 AS qoh#23] (35) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#13, i_brand#10, qoh#23] -Keys [2]: [i_product_name#13, i_brand#10] +Input [3]: [i_product_name#46, i_brand#47, qoh#23] +Keys [2]: [i_product_name#46, i_brand#47] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#44, count#45] -Results [4]: [i_product_name#13, i_brand#10, sum#46, count#47] +Aggregate Attributes [2]: [sum#54, count#55] +Results [4]: [i_product_name#46, i_brand#47, sum#56, count#57] (36) Exchange -Input [4]: [i_product_name#13, i_brand#10, sum#46, count#47] -Arguments: hashpartitioning(i_product_name#13, i_brand#10, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [4]: [i_product_name#46, i_brand#47, sum#56, count#57] +Arguments: hashpartitioning(i_product_name#46, i_brand#47, 5), ENSURE_REQUIREMENTS, [id=#58] (37) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#13, i_brand#10, sum#46, count#47] -Keys [2]: [i_product_name#13, i_brand#10] +Input [4]: [i_product_name#46, i_brand#47, sum#56, count#57] +Keys [2]: [i_product_name#46, i_brand#47] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#49] -Results [5]: [i_product_name#13, i_brand#10, null AS i_class#50, null AS i_category#51, avg(qoh#23)#49 AS qoh#52] +Aggregate Attributes [1]: [avg(qoh#23)#59] +Results [5]: [i_product_name#46, i_brand#47, null AS i_class#60, null AS i_category#61, avg(qoh#23)#59 AS qoh#62] (38) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#53, count#54] +Output [6]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, sum#67, count#68] (39) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#53, count#54] -Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#55] -Results [2]: [i_product_name#13, avg(inv_quantity_on_hand#3)#55 AS qoh#23] +Input [6]: [i_product_name#63, i_brand#64, i_class#65, i_category#66, sum#67, count#68] +Keys [4]: [i_product_name#63, i_brand#64, i_class#65, i_category#66] +Functions [1]: [avg(inv_quantity_on_hand#69)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#69)#70] +Results [2]: [i_product_name#63, avg(inv_quantity_on_hand#69)#70 AS qoh#23] (40) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#13, qoh#23] -Keys [1]: [i_product_name#13] +Input [2]: [i_product_name#63, qoh#23] +Keys [1]: [i_product_name#63] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#56, count#57] -Results [3]: [i_product_name#13, sum#58, count#59] +Aggregate Attributes [2]: [sum#71, count#72] +Results [3]: [i_product_name#63, sum#73, count#74] (41) Exchange -Input [3]: [i_product_name#13, sum#58, count#59] -Arguments: hashpartitioning(i_product_name#13, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [3]: [i_product_name#63, sum#73, count#74] +Arguments: hashpartitioning(i_product_name#63, 5), ENSURE_REQUIREMENTS, [id=#75] (42) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#13, sum#58, count#59] -Keys [1]: [i_product_name#13] +Input [3]: [i_product_name#63, sum#73, count#74] +Keys [1]: [i_product_name#63] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#61] -Results [5]: [i_product_name#13, null AS i_brand#62, null AS i_class#63, null AS i_category#64, avg(qoh#23)#61 AS qoh#65] +Aggregate Attributes [1]: [avg(qoh#23)#76] +Results [5]: [i_product_name#63, null AS i_brand#77, null AS i_class#78, null AS i_category#79, avg(qoh#23)#76 AS qoh#80] (43) ReusedExchange [Reuses operator id: unknown] -Output [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#66, count#67] +Output [6]: [i_product_name#81, i_brand#82, i_class#83, i_category#84, sum#85, count#86] (44) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#13, i_brand#10, i_class#11, i_category#12, sum#66, count#67] -Keys [4]: [i_product_name#13, i_brand#10, i_class#11, i_category#12] -Functions [1]: [avg(inv_quantity_on_hand#3)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#68] -Results [1]: [avg(inv_quantity_on_hand#3)#68 AS qoh#23] +Input [6]: [i_product_name#81, i_brand#82, i_class#83, i_category#84, sum#85, count#86] +Keys [4]: [i_product_name#81, i_brand#82, i_class#83, i_category#84] +Functions [1]: [avg(inv_quantity_on_hand#87)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#87)#88] +Results [1]: [avg(inv_quantity_on_hand#87)#88 AS qoh#23] (45) HashAggregate [codegen id : 28] Input [1]: [qoh#23] Keys: [] Functions [1]: [partial_avg(qoh#23)] -Aggregate Attributes [2]: [sum#69, count#70] -Results [2]: [sum#71, count#72] +Aggregate Attributes [2]: [sum#89, count#90] +Results [2]: [sum#91, count#92] (46) Exchange -Input [2]: [sum#71, count#72] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#73] +Input [2]: [sum#91, count#92] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#93] (47) HashAggregate [codegen id : 29] -Input [2]: [sum#71, count#72] +Input [2]: [sum#91, count#92] Keys: [] Functions [1]: [avg(qoh#23)] -Aggregate Attributes [1]: [avg(qoh#23)#74] -Results [5]: [null AS i_product_name#75, null AS i_brand#76, null AS i_class#77, null AS i_category#78, avg(qoh#23)#74 AS qoh#79] +Aggregate Attributes [1]: [avg(qoh#23)#94] +Results [5]: [null AS i_product_name#95, null AS i_brand#96, null AS i_class#97, null AS i_category#98, avg(qoh#23)#94 AS qoh#99] (48) Union 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 8031170978906..1c0ed5c8c188b 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 @@ -329,224 +329,224 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (50) Scan parquet default.store -Output [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] +Output [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (51) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] +Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] (52) Filter [codegen id : 1] -Input [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] -Condition : (((isnotnull(s_market_id#3) AND (s_market_id#3 = 8)) AND isnotnull(s_store_sk#1)) AND isnotnull(s_zip#5)) +Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] +Condition : (((isnotnull(s_market_id#54) AND (s_market_id#54 = 8)) AND isnotnull(s_store_sk#52)) AND isnotnull(s_zip#56)) (53) Project [codegen id : 1] -Output [4]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5] -Input [5]: [s_store_sk#1, s_store_name#2, s_market_id#3, s_state#4, s_zip#5] +Output [4]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56] +Input [5]: [s_store_sk#52, s_store_name#53, s_market_id#54, s_state#55, s_zip#56] (54) BroadcastExchange -Input [4]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5] -Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#52] +Input [4]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56] +Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#57] (55) Scan parquet default.customer_address -Output [4]: [ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] +Output [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (56) ColumnarToRow -Input [4]: [ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] +Input [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] (57) Filter -Input [4]: [ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] -Condition : ((isnotnull(ca_address_sk#7) AND isnotnull(ca_country#10)) AND isnotnull(ca_zip#9)) +Input [4]: [ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] +Condition : ((isnotnull(ca_address_sk#58) AND isnotnull(ca_country#61)) AND isnotnull(ca_zip#60)) (58) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [s_zip#5] -Right keys [1]: [ca_zip#9] +Left keys [1]: [s_zip#56] +Right keys [1]: [ca_zip#60] Join condition: None (59) Project [codegen id : 2] -Output [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10] -Input [8]: [s_store_sk#1, s_store_name#2, s_state#4, s_zip#5, ca_address_sk#7, ca_state#8, ca_zip#9, ca_country#10] +Output [6]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61] +Input [8]: [s_store_sk#52, s_store_name#53, s_state#55, s_zip#56, ca_address_sk#58, ca_state#59, ca_zip#60, ca_country#61] (60) BroadcastExchange -Input [6]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10] -Arguments: HashedRelationBroadcastMode(List(input[3, int, true], upper(input[5, string, true])),false), [id=#53] +Input [6]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61] +Arguments: HashedRelationBroadcastMode(List(input[3, int, true], upper(input[5, string, true])),false), [id=#62] (61) Scan parquet default.customer -Output [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] +Output [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct (62) ColumnarToRow -Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] +Input [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] (63) Filter -Input [5]: [c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] -Condition : ((isnotnull(c_customer_sk#12) AND isnotnull(c_current_addr_sk#13)) AND isnotnull(c_birth_country#16)) +Input [5]: [c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] +Condition : ((isnotnull(c_customer_sk#63) AND isnotnull(c_current_addr_sk#64)) AND isnotnull(c_birth_country#67)) (64) BroadcastHashJoin [codegen id : 3] -Left keys [2]: [ca_address_sk#7, upper(ca_country#10)] -Right keys [2]: [c_current_addr_sk#13, c_birth_country#16] +Left keys [2]: [ca_address_sk#58, upper(ca_country#61)] +Right keys [2]: [c_current_addr_sk#64, c_birth_country#67] Join condition: None (65) Project [codegen id : 3] -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] -Input [11]: [s_store_sk#1, s_store_name#2, s_state#4, ca_address_sk#7, ca_state#8, ca_country#10, c_customer_sk#12, c_current_addr_sk#13, c_first_name#14, c_last_name#15, c_birth_country#16] +Output [7]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66] +Input [11]: [s_store_sk#52, s_store_name#53, s_state#55, ca_address_sk#58, ca_state#59, ca_country#61, c_customer_sk#63, c_current_addr_sk#64, c_first_name#65, c_last_name#66, c_birth_country#67] (66) BroadcastExchange -Input [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] -Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#54] +Input [7]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[4, int, true] as bigint) & 4294967295))),false), [id=#68] (67) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (68) ColumnarToRow -Input [6]: [ss_item_sk#18, ss_customer_sk#19, ss_store_sk#20, ss_ticket_number#21, ss_net_paid#22, ss_sold_date_sk#23] +Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] (69) Filter -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)) +Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] +Condition : (((isnotnull(ss_ticket_number#72) AND isnotnull(ss_item_sk#69)) AND isnotnull(ss_store_sk#71)) AND isnotnull(ss_customer_sk#70)) (70) Project -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] +Output [5]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73] +Input [6]: [ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73, ss_sold_date_sk#74] (71) BroadcastHashJoin [codegen id : 4] -Left keys [2]: [s_store_sk#1, c_customer_sk#12] -Right keys [2]: [ss_store_sk#20, ss_customer_sk#19] +Left keys [2]: [s_store_sk#52, c_customer_sk#63] +Right keys [2]: [ss_store_sk#71, ss_customer_sk#70] Join condition: None (72) Project [codegen id : 4] -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] +Output [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] +Input [12]: [s_store_sk#52, s_store_name#53, s_state#55, ca_state#59, c_customer_sk#63, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_customer_sk#70, ss_store_sk#71, ss_ticket_number#72, ss_net_paid#73] (73) Exchange -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=#55] +Input [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] +Arguments: hashpartitioning(ss_item_sk#69, 5), ENSURE_REQUIREMENTS, [id=#75] (74) Sort [codegen id : 5] -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 +Input [8]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73] +Arguments: [ss_item_sk#69 ASC NULLS FIRST], false, 0 (75) Scan parquet default.item -Output [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Output [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#24, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] +Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] (77) Filter [codegen id : 6] -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) +Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Condition : isnotnull(i_item_sk#76) (78) Exchange -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=#56] +Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Arguments: hashpartitioning(i_item_sk#76, 5), ENSURE_REQUIREMENTS, [id=#82] (79) Sort [codegen id : 7] -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 +Input [6]: [i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Arguments: [i_item_sk#76 ASC NULLS FIRST], false, 0 (80) SortMergeJoin [codegen id : 8] -Left keys [1]: [ss_item_sk#18] -Right keys [1]: [i_item_sk#24] +Left keys [1]: [ss_item_sk#69] +Right keys [1]: [i_item_sk#76] Join condition: None (81) Project [codegen id : 8] -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] +Output [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Input [14]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_item_sk#76, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] (82) Exchange -Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] -Arguments: hashpartitioning(cast(ss_ticket_number#21 as bigint), cast(ss_item_sk#18 as bigint), 5), ENSURE_REQUIREMENTS, [id=#57] +Input [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Arguments: hashpartitioning(cast(ss_ticket_number#72 as bigint), cast(ss_item_sk#69 as bigint), 5), ENSURE_REQUIREMENTS, [id=#83] (83) Sort [codegen id : 9] -Input [13]: [s_store_name#2, s_state#4, ca_state#8, c_first_name#14, c_last_name#15, ss_item_sk#18, ss_ticket_number#21, ss_net_paid#22, i_current_price#25, i_size#26, i_color#27, i_units#28, i_manager_id#29] -Arguments: [cast(ss_ticket_number#21 as bigint) ASC NULLS FIRST, cast(ss_item_sk#18 as bigint) ASC NULLS FIRST], false, 0 +Input [13]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81] +Arguments: [cast(ss_ticket_number#72 as bigint) ASC NULLS FIRST, cast(ss_item_sk#69 as bigint) ASC NULLS FIRST], false, 0 (84) Scan parquet default.store_returns -Output [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Output [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (85) ColumnarToRow [codegen id : 10] -Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] (86) Filter [codegen id : 10] -Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] -Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) +Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] +Condition : (isnotnull(sr_ticket_number#85) AND isnotnull(sr_item_sk#84)) (87) Project [codegen id : 10] -Output [2]: [sr_item_sk#32, sr_ticket_number#33] -Input [3]: [sr_item_sk#32, sr_ticket_number#33, sr_returned_date_sk#34] +Output [2]: [sr_item_sk#84, sr_ticket_number#85] +Input [3]: [sr_item_sk#84, sr_ticket_number#85, sr_returned_date_sk#86] (88) Exchange -Input [2]: [sr_item_sk#32, sr_ticket_number#33] -Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [2]: [sr_item_sk#84, sr_ticket_number#85] +Arguments: hashpartitioning(sr_ticket_number#85, sr_item_sk#84, 5), ENSURE_REQUIREMENTS, [id=#87] (89) Sort [codegen id : 11] -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 +Input [2]: [sr_item_sk#84, sr_ticket_number#85] +Arguments: [sr_ticket_number#85 ASC NULLS FIRST, sr_item_sk#84 ASC NULLS FIRST], false, 0 (90) SortMergeJoin [codegen id : 12] -Left keys [2]: [cast(ss_ticket_number#21 as bigint), cast(ss_item_sk#18 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Left keys [2]: [cast(ss_ticket_number#72 as bigint), cast(ss_item_sk#69 as bigint)] +Right keys [2]: [sr_ticket_number#85, sr_item_sk#84] Join condition: None (91) Project [codegen id : 12] -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] +Output [11]: [ss_net_paid#73, s_store_name#53, s_state#55, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, c_first_name#65, c_last_name#66, ca_state#59] +Input [15]: [s_store_name#53, s_state#55, ca_state#59, c_first_name#65, c_last_name#66, ss_item_sk#69, ss_ticket_number#72, ss_net_paid#73, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, sr_item_sk#84, sr_ticket_number#85] (92) HashAggregate [codegen id : 12] -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#59] -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#60] +Input [11]: [ss_net_paid#73, s_store_name#53, s_state#55, i_current_price#77, i_size#78, i_color#79, i_units#80, i_manager_id#81, c_first_name#65, c_last_name#66, ca_state#59] +Keys [10]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#73))] +Aggregate Attributes [1]: [sum#88] +Results [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] (93) Exchange -Input [11]: [c_last_name#15, c_first_name#14, s_store_name#2, ca_state#8, s_state#4, i_color#27, i_current_price#25, i_manager_id#29, i_units#28, i_size#26, sum#60] -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=#61] +Input [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] +Arguments: hashpartitioning(c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, 5), ENSURE_REQUIREMENTS, [id=#90] (94) HashAggregate [codegen id : 13] -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#60] -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))#62] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#62,17,2) AS netpaid#40] +Input [11]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78, sum#89] +Keys [10]: [c_last_name#66, c_first_name#65, s_store_name#53, ca_state#59, s_state#55, i_color#79, i_current_price#77, i_manager_id#81, i_units#80, i_size#78] +Functions [1]: [sum(UnscaledValue(ss_net_paid#73))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#73))#91] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#73))#91,17,2) AS netpaid#40] (95) HashAggregate [codegen id : 13] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#63, count#64] -Results [2]: [sum#65, count#66] +Aggregate Attributes [2]: [sum#92, count#93] +Results [2]: [sum#94, count#95] (96) Exchange -Input [2]: [sum#65, count#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#67] +Input [2]: [sum#94, count#95] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#96] (97) HashAggregate [codegen id : 14] -Input [2]: [sum#65, count#66] +Input [2]: [sum#94, count#95] Keys: [] Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#68] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#68)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#69] +Aggregate Attributes [1]: [avg(netpaid#40)#97] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#97)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#98] 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 540feb0d88a3f..7259a3d829613 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 @@ -326,212 +326,212 @@ Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquer (50) Scan parquet default.store_sales -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] +Output [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (51) ColumnarToRow [codegen id : 1] -Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] (52) Filter [codegen id : 1] -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)) +Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] +Condition : (((isnotnull(ss_ticket_number#55) AND isnotnull(ss_item_sk#52)) AND isnotnull(ss_store_sk#54)) AND isnotnull(ss_customer_sk#53)) (53) Project [codegen id : 1] -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] +Output [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] +Input [6]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, ss_sold_date_sk#57] (54) Exchange -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#52] +Input [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] +Arguments: hashpartitioning(cast(ss_ticket_number#55 as bigint), cast(ss_item_sk#52 as bigint), 5), ENSURE_REQUIREMENTS, [id=#58] (55) Sort [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 +Input [5]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56] +Arguments: [cast(ss_ticket_number#55 as bigint) ASC NULLS FIRST, cast(ss_item_sk#52 as bigint) ASC NULLS FIRST], false, 0 (56) Scan parquet default.store_returns -Output [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (57) ColumnarToRow [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] (58) Filter [codegen id : 3] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] -Condition : (isnotnull(sr_ticket_number#9) AND isnotnull(sr_item_sk#8)) +Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] +Condition : (isnotnull(sr_ticket_number#60) AND isnotnull(sr_item_sk#59)) (59) Project [codegen id : 3] -Output [2]: [sr_item_sk#8, sr_ticket_number#9] -Input [3]: [sr_item_sk#8, sr_ticket_number#9, sr_returned_date_sk#10] +Output [2]: [sr_item_sk#59, sr_ticket_number#60] +Input [3]: [sr_item_sk#59, sr_ticket_number#60, sr_returned_date_sk#61] (60) Exchange -Input [2]: [sr_item_sk#8, sr_ticket_number#9] -Arguments: hashpartitioning(sr_ticket_number#9, sr_item_sk#8, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [2]: [sr_item_sk#59, sr_ticket_number#60] +Arguments: hashpartitioning(sr_ticket_number#60, sr_item_sk#59, 5), ENSURE_REQUIREMENTS, [id=#62] (61) Sort [codegen id : 4] -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 +Input [2]: [sr_item_sk#59, sr_ticket_number#60] +Arguments: [sr_ticket_number#60 ASC NULLS FIRST, sr_item_sk#59 ASC NULLS FIRST], false, 0 (62) SortMergeJoin [codegen id : 9] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#9, sr_item_sk#8] +Left keys [2]: [cast(ss_ticket_number#55 as bigint), cast(ss_item_sk#52 as bigint)] +Right keys [2]: [sr_ticket_number#60, sr_item_sk#59] Join condition: None (63) Project [codegen id : 9] -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] +Output [4]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_net_paid#56] +Input [7]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_ticket_number#55, ss_net_paid#56, sr_item_sk#59, sr_ticket_number#60] (64) Scan parquet default.store -Output [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct (65) ColumnarToRow [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] (66) Filter [codegen id : 5] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] -Condition : (((isnotnull(s_market_id#14) AND (s_market_id#14 = 8)) AND isnotnull(s_store_sk#12)) AND isnotnull(s_zip#16)) +Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] +Condition : (((isnotnull(s_market_id#65) AND (s_market_id#65 = 8)) AND isnotnull(s_store_sk#63)) AND isnotnull(s_zip#67)) (67) Project [codegen id : 5] -Output [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Input [5]: [s_store_sk#12, s_store_name#13, s_market_id#14, s_state#15, s_zip#16] +Output [4]: [s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] +Input [5]: [s_store_sk#63, s_store_name#64, s_market_id#65, s_state#66, s_zip#67] (68) BroadcastExchange -Input [4]: [s_store_sk#12, s_store_name#13, s_state#15, s_zip#16] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [4]: [s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#68] (69) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#12] +Left keys [1]: [ss_store_sk#54] +Right keys [1]: [s_store_sk#63] Join condition: None (70) Project [codegen id : 9] -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] +Output [6]: [ss_item_sk#52, ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67] +Input [8]: [ss_item_sk#52, ss_customer_sk#53, ss_store_sk#54, ss_net_paid#56, s_store_sk#63, s_store_name#64, s_state#66, s_zip#67] (71) Scan parquet default.item -Output [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Output [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 6] -Input [6]: [i_item_sk#18, i_current_price#19, i_size#20, i_color#21, i_units#22, i_manager_id#23] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (73) Filter [codegen id : 6] -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) +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Condition : isnotnull(i_item_sk#69) (74) BroadcastExchange -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=#55] +Input [6]: [i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#75] (75) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [ss_item_sk#52] +Right keys [1]: [i_item_sk#69] Join condition: None (76) Project [codegen id : 9] -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] +Output [10]: [ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] +Input [12]: [ss_item_sk#52, ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_item_sk#69, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74] (77) Scan parquet default.customer -Output [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] +Output [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] ReadSchema: struct (78) ColumnarToRow [codegen id : 7] -Input [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] +Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] (79) Filter [codegen id : 7] -Input [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Condition : ((isnotnull(c_customer_sk#25) AND isnotnull(c_current_addr_sk#26)) AND isnotnull(c_birth_country#29)) +Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +Condition : ((isnotnull(c_customer_sk#76) AND isnotnull(c_current_addr_sk#77)) AND isnotnull(c_birth_country#80)) (80) BroadcastExchange -Input [5]: [c_customer_sk#25, c_current_addr_sk#26, c_first_name#27, c_last_name#28, c_birth_country#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#56] +Input [5]: [c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#81] (81) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#25] +Left keys [1]: [ss_customer_sk#53] +Right keys [1]: [c_customer_sk#76] Join condition: None (82) Project [codegen id : 9] -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] +Output [13]: [ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] +Input [15]: [ss_customer_sk#53, ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_customer_sk#76, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80] (83) Scan parquet default.customer_address -Output [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] +Output [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct (84) ColumnarToRow [codegen id : 8] -Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] +Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] (85) Filter [codegen id : 8] -Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] -Condition : ((isnotnull(ca_address_sk#31) AND isnotnull(ca_country#34)) AND isnotnull(ca_zip#33)) +Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] +Condition : ((isnotnull(ca_address_sk#82) AND isnotnull(ca_country#85)) AND isnotnull(ca_zip#84)) (86) BroadcastExchange -Input [4]: [ca_address_sk#31, ca_state#32, ca_zip#33, ca_country#34] -Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [id=#57] +Input [4]: [ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] +Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [id=#86] (87) BroadcastHashJoin [codegen id : 9] -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] +Left keys [3]: [c_current_addr_sk#77, c_birth_country#80, s_zip#67] +Right keys [3]: [ca_address_sk#82, upper(ca_country#85), ca_zip#84] Join condition: None (88) Project [codegen id : 9] -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] +Output [11]: [ss_net_paid#56, s_store_name#64, s_state#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#78, c_last_name#79, ca_state#83] +Input [17]: [ss_net_paid#56, s_store_name#64, s_state#66, s_zip#67, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_current_addr_sk#77, c_first_name#78, c_last_name#79, c_birth_country#80, ca_address_sk#82, ca_state#83, ca_zip#84, ca_country#85] (89) HashAggregate [codegen id : 9] -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#58] -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#59] +Input [11]: [ss_net_paid#56, s_store_name#64, s_state#66, i_current_price#70, i_size#71, i_color#72, i_units#73, i_manager_id#74, c_first_name#78, c_last_name#79, ca_state#83] +Keys [10]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#56))] +Aggregate Attributes [1]: [sum#87] +Results [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] (90) Exchange -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#59] -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=#60] +Input [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] +Arguments: hashpartitioning(c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, 5), ENSURE_REQUIREMENTS, [id=#89] (91) HashAggregate [codegen id : 10] -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#59] -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))#61] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#61,17,2) AS netpaid#40] +Input [11]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71, sum#88] +Keys [10]: [c_last_name#79, c_first_name#78, s_store_name#64, ca_state#83, s_state#66, i_color#72, i_current_price#70, i_manager_id#74, i_units#73, i_size#71] +Functions [1]: [sum(UnscaledValue(ss_net_paid#56))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#56))#90] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#56))#90,17,2) AS netpaid#40] (92) HashAggregate [codegen id : 10] Input [1]: [netpaid#40] Keys: [] Functions [1]: [partial_avg(netpaid#40)] -Aggregate Attributes [2]: [sum#62, count#63] -Results [2]: [sum#64, count#65] +Aggregate Attributes [2]: [sum#91, count#92] +Results [2]: [sum#93, count#94] (93) Exchange -Input [2]: [sum#64, count#65] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#66] +Input [2]: [sum#93, count#94] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#95] (94) HashAggregate [codegen id : 11] -Input [2]: [sum#64, count#65] +Input [2]: [sum#93, count#94] Keys: [] Functions [1]: [avg(netpaid#40)] -Aggregate Attributes [1]: [avg(netpaid#40)#67] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#67)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#68] +Aggregate Attributes [1]: [avg(netpaid#40)#96] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#40)#96)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#97] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt index 6c48953be5aeb..1fb9533024b2f 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#61), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -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] +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] (35) Filter [codegen id : 11] -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)) +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#62] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#55] +Right keys [1]: [cd_demo_sk#62] Join condition: None (38) Project [codegen id : 11] -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] +Output [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, cd_demo_sk#62] (39) Scan parquet default.store -Output [2]: [s_store_sk#18, s_state#19] +Output [2]: [s_store_sk#63, s_state#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (40) ColumnarToRow [codegen id : 8] -Input [2]: [s_store_sk#18, s_state#19] +Input [2]: [s_store_sk#63, s_state#64] (41) Filter [codegen id : 8] -Input [2]: [s_store_sk#18, s_state#19] -Condition : ((isnotnull(s_state#19) AND (s_state#19 = TN)) AND isnotnull(s_store_sk#18)) +Input [2]: [s_store_sk#63, s_state#64] +Condition : ((isnotnull(s_state#64) AND (s_state#64 = TN)) AND isnotnull(s_store_sk#63)) (42) Project [codegen id : 8] -Output [1]: [s_store_sk#18] -Input [2]: [s_store_sk#18, s_state#19] +Output [1]: [s_store_sk#63] +Input [2]: [s_store_sk#63, s_state#64] (43) BroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [s_store_sk#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#65] (44) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#56] +Right keys [1]: [s_store_sk#63] Join condition: None (45) Project [codegen id : 11] -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#18] +Output [6]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Input [8]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, s_store_sk#63] (46) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#66] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#61] +Right keys [1]: [d_date_sk#66] Join condition: None (48) Project [codegen id : 11] -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] +Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#66] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#21, i_item_id#22] +Output [2]: [i_item_sk#67, i_item_id#68] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#67] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] +Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] +Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#22] +Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#68] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] +Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] (53) Exchange -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [1]: [i_item_id#22] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Keys [1]: [i_item_id#68] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] -Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] +Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] +Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] (55) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 17] -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] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] (57) Filter [codegen id : 17] -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)) +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#104] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#104] Join condition: None (60) Project [codegen id : 17] -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] +Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] (61) ReusedExchange [Reuses operator id: 43] -Output [1]: [s_store_sk#18] +Output [1]: [s_store_sk#105] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#98] +Right keys [1]: [s_store_sk#105] Join condition: None (63) Project [codegen id : 17] -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#18] +Output [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, s_store_sk#105] (64) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#106] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#103] +Right keys [1]: [d_date_sk#106] Join condition: None (66) Project [codegen id : 17] -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] +Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [7]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#106] (67) Scan parquet default.item -Output [1]: [i_item_sk#21] +Output [1]: [i_item_sk#107] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#21] +Input [1]: [i_item_sk#107] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#21] -Condition : isnotnull(i_item_sk#21) +Input [1]: [i_item_sk#107] +Condition : isnotnull(i_item_sk#107) (70) BroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] +Input [1]: [i_item_sk#107] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#96] +Right keys [1]: [i_item_sk#107] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] +Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] +Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] -Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] +Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] (74) Exchange -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] -Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] +Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] +Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#15] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 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 04c79a5413ad9..f05f4e9f2e43e 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 @@ -228,200 +228,200 @@ Aggregate Attributes [4]: [avg(agg1#24)#45, avg(UnscaledValue(agg2#25))#46, avg( Results [7]: [i_item_id#22, s_state#19, 0 AS g_state#49, avg(agg1#24)#45 AS agg1#50, cast((avg(UnscaledValue(agg2#25))#46 / 100.0) as decimal(11,6)) AS agg2#51, cast((avg(UnscaledValue(agg3#26))#47 / 100.0) as decimal(11,6)) AS agg3#52, cast((avg(UnscaledValue(agg4#27))#48 / 100.0) as decimal(11,6)) AS agg4#53] (33) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#61), dynamicpruningexpression(ss_sold_date_sk#61 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -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] +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] (35) Filter [codegen id : 11] -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)) +Input [8]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Condition : ((isnotnull(ss_cdemo_sk#55) AND isnotnull(ss_store_sk#56)) AND isnotnull(ss_item_sk#54)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#62] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#55] +Right keys [1]: [cd_demo_sk#62] Join condition: None (38) Project [codegen id : 11] -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] +Output [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61] +Input [9]: [ss_item_sk#54, ss_cdemo_sk#55, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, cd_demo_sk#62] (39) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#63] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#61] +Right keys [1]: [d_date_sk#63] Join condition: None (41) Project [codegen id : 11] -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] +Output [6]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [8]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, ss_sold_date_sk#61, d_date_sk#63] (42) Scan parquet default.store -Output [2]: [s_store_sk#18, s_state#19] +Output [2]: [s_store_sk#64, s_state#65] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] ReadSchema: struct (43) ColumnarToRow [codegen id : 9] -Input [2]: [s_store_sk#18, s_state#19] +Input [2]: [s_store_sk#64, s_state#65] (44) Filter [codegen id : 9] -Input [2]: [s_store_sk#18, s_state#19] -Condition : ((isnotnull(s_state#19) AND (s_state#19 = TN)) AND isnotnull(s_store_sk#18)) +Input [2]: [s_store_sk#64, s_state#65] +Condition : ((isnotnull(s_state#65) AND (s_state#65 = TN)) AND isnotnull(s_store_sk#64)) (45) Project [codegen id : 9] -Output [1]: [s_store_sk#18] -Input [2]: [s_store_sk#18, s_state#19] +Output [1]: [s_store_sk#64] +Input [2]: [s_store_sk#64, s_state#65] (46) BroadcastExchange -Input [1]: [s_store_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#54] +Input [1]: [s_store_sk#64] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#66] (47) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#56] +Right keys [1]: [s_store_sk#64] Join condition: None (48) Project [codegen id : 11] -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#18] +Output [5]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60] +Input [7]: [ss_item_sk#54, ss_store_sk#56, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, s_store_sk#64] (49) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#21, i_item_id#22] +Output [2]: [i_item_sk#67, i_item_id#68] (50) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#54] +Right keys [1]: [i_item_sk#67] Join condition: None (51) Project [codegen id : 11] -Output [5]: [i_item_id#22, ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [7]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21, i_item_id#22] +Output [5]: [i_item_id#68, ss_quantity#57 AS agg1#24, ss_list_price#58 AS agg2#25, ss_coupon_amt#60 AS agg3#26, ss_sales_price#59 AS agg4#27] +Input [7]: [ss_item_sk#54, ss_quantity#57, ss_list_price#58, ss_sales_price#59, ss_coupon_amt#60, i_item_sk#67, i_item_id#68] (52) HashAggregate [codegen id : 11] -Input [5]: [i_item_id#22, agg1#24, agg2#25, agg3#26, agg4#27] -Keys [1]: [i_item_id#22] +Input [5]: [i_item_id#68, agg1#24, agg2#25, agg3#26, agg4#27] +Keys [1]: [i_item_id#68] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] -Results [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Aggregate Attributes [8]: [sum#69, count#70, sum#71, count#72, sum#73, count#74, sum#75, count#76] +Results [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] (53) Exchange -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#22, 5), ENSURE_REQUIREMENTS, [id=#71] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Arguments: hashpartitioning(i_item_id#68, 5), ENSURE_REQUIREMENTS, [id=#85] (54) HashAggregate [codegen id : 12] -Input [9]: [i_item_id#22, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [1]: [i_item_id#22] +Input [9]: [i_item_id#68, sum#77, count#78, sum#79, count#80, sum#81, count#82, sum#83, count#84] +Keys [1]: [i_item_id#68] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#72, avg(UnscaledValue(agg2#25))#73, avg(UnscaledValue(agg3#26))#74, avg(UnscaledValue(agg4#27))#75] -Results [7]: [i_item_id#22, null AS s_state#76, 1 AS g_state#77, avg(agg1#24)#72 AS agg1#78, cast((avg(UnscaledValue(agg2#25))#73 / 100.0) as decimal(11,6)) AS agg2#79, cast((avg(UnscaledValue(agg3#26))#74 / 100.0) as decimal(11,6)) AS agg3#80, cast((avg(UnscaledValue(agg4#27))#75 / 100.0) as decimal(11,6)) AS agg4#81] +Aggregate Attributes [4]: [avg(agg1#24)#86, avg(UnscaledValue(agg2#25))#87, avg(UnscaledValue(agg3#26))#88, avg(UnscaledValue(agg4#27))#89] +Results [7]: [i_item_id#68, null AS s_state#90, 1 AS g_state#91, avg(agg1#24)#86 AS agg1#92, cast((avg(UnscaledValue(agg2#25))#87 / 100.0) as decimal(11,6)) AS agg2#93, cast((avg(UnscaledValue(agg3#26))#88 / 100.0) as decimal(11,6)) AS agg3#94, cast((avg(UnscaledValue(agg4#27))#89 / 100.0) as decimal(11,6)) AS agg4#95] (55) Scan parquet default.store_sales -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] +Output [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(ss_sold_date_sk#103), dynamicpruningexpression(ss_sold_date_sk#103 IN dynamicpruning#9)] PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 17] -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] +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] (57) Filter [codegen id : 17] -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)) +Input [8]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Condition : ((isnotnull(ss_cdemo_sk#97) AND isnotnull(ss_store_sk#98)) AND isnotnull(ss_item_sk#96)) (58) ReusedExchange [Reuses operator id: 8] -Output [1]: [cd_demo_sk#10] +Output [1]: [cd_demo_sk#104] (59) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [ss_cdemo_sk#97] +Right keys [1]: [cd_demo_sk#104] Join condition: None (60) Project [codegen id : 17] -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] +Output [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103] +Input [9]: [ss_item_sk#96, ss_cdemo_sk#97, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, cd_demo_sk#104] (61) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#105] (62) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#8] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [ss_sold_date_sk#103] +Right keys [1]: [d_date_sk#105] Join condition: None (63) Project [codegen id : 17] -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] +Output [6]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [8]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, ss_sold_date_sk#103, d_date_sk#105] (64) ReusedExchange [Reuses operator id: 46] -Output [1]: [s_store_sk#18] +Output [1]: [s_store_sk#106] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#18] +Left keys [1]: [ss_store_sk#98] +Right keys [1]: [s_store_sk#106] Join condition: None (66) Project [codegen id : 17] -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#18] +Output [5]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102] +Input [7]: [ss_item_sk#96, ss_store_sk#98, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, s_store_sk#106] (67) Scan parquet default.item -Output [1]: [i_item_sk#21] +Output [1]: [i_item_sk#107] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 16] -Input [1]: [i_item_sk#21] +Input [1]: [i_item_sk#107] (69) Filter [codegen id : 16] -Input [1]: [i_item_sk#21] -Condition : isnotnull(i_item_sk#21) +Input [1]: [i_item_sk#107] +Condition : isnotnull(i_item_sk#107) (70) BroadcastExchange -Input [1]: [i_item_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#82] +Input [1]: [i_item_sk#107] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#108] (71) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#21] +Left keys [1]: [ss_item_sk#96] +Right keys [1]: [i_item_sk#107] Join condition: None (72) Project [codegen id : 17] -Output [4]: [ss_quantity#4 AS agg1#24, ss_list_price#5 AS agg2#25, ss_coupon_amt#7 AS agg3#26, ss_sales_price#6 AS agg4#27] -Input [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#21] +Output [4]: [ss_quantity#99 AS agg1#24, ss_list_price#100 AS agg2#25, ss_coupon_amt#102 AS agg3#26, ss_sales_price#101 AS agg4#27] +Input [6]: [ss_item_sk#96, ss_quantity#99, ss_list_price#100, ss_sales_price#101, ss_coupon_amt#102, i_item_sk#107] (73) HashAggregate [codegen id : 17] Input [4]: [agg1#24, agg2#25, agg3#26, agg4#27] Keys: [] Functions [4]: [partial_avg(agg1#24), partial_avg(UnscaledValue(agg2#25)), partial_avg(UnscaledValue(agg3#26)), partial_avg(UnscaledValue(agg4#27))] -Aggregate Attributes [8]: [sum#83, count#84, sum#85, count#86, sum#87, count#88, sum#89, count#90] -Results [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Aggregate Attributes [8]: [sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116] +Results [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] (74) Exchange -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#99] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#125] (75) HashAggregate [codegen id : 18] -Input [8]: [sum#91, count#92, sum#93, count#94, sum#95, count#96, sum#97, count#98] +Input [8]: [sum#117, count#118, sum#119, count#120, sum#121, count#122, sum#123, count#124] Keys: [] Functions [4]: [avg(agg1#24), avg(UnscaledValue(agg2#25)), avg(UnscaledValue(agg3#26)), avg(UnscaledValue(agg4#27))] -Aggregate Attributes [4]: [avg(agg1#24)#100, avg(UnscaledValue(agg2#25))#101, avg(UnscaledValue(agg3#26))#102, avg(UnscaledValue(agg4#27))#103] -Results [7]: [null AS i_item_id#104, null AS s_state#105, 1 AS g_state#106, avg(agg1#24)#100 AS agg1#107, cast((avg(UnscaledValue(agg2#25))#101 / 100.0) as decimal(11,6)) AS agg2#108, cast((avg(UnscaledValue(agg3#26))#102 / 100.0) as decimal(11,6)) AS agg3#109, cast((avg(UnscaledValue(agg4#27))#103 / 100.0) as decimal(11,6)) AS agg4#110] +Aggregate Attributes [4]: [avg(agg1#24)#126, avg(UnscaledValue(agg2#25))#127, avg(UnscaledValue(agg3#26))#128, avg(UnscaledValue(agg4#27))#129] +Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#24)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#25))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#26))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#27))#129 / 100.0) as decimal(11,6)) AS agg4#136] (76) Union @@ -438,8 +438,8 @@ ReusedExchange (78) (78) ReusedExchange [Reuses operator id: 15] Output [1]: [d_date_sk#15] -Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:2 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#61 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 55 Hosting Expression = ss_sold_date_sk#103 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt index 0a861f4ce8e6d..f7fff421817c0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt @@ -145,20 +145,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#15, ws_sold_date_sk#16] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#17] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#16] -Right keys [1]: [d_date_sk#10] +Right keys [1]: [d_date_sk#17] Join condition: None (22) Project [codegen id : 7] Output [1]: [ws_bill_customer_sk#15] -Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#10] +Input [3]: [ws_bill_customer_sk#15, ws_sold_date_sk#16, d_date_sk#17] (23) Exchange Input [1]: [ws_bill_customer_sk#15] -Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#17] +Arguments: hashpartitioning(ws_bill_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#18] (24) Sort [codegen id : 8] Input [1]: [ws_bill_customer_sk#15] @@ -170,38 +170,38 @@ Right keys [1]: [ws_bill_customer_sk#15] Join condition: None (26) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Output [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#9)] +PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#9)] ReadSchema: struct (27) ColumnarToRow [codegen id : 10] -Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] +Input [2]: [cs_ship_customer_sk#19, cs_sold_date_sk#20] (28) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#10] +Output [1]: [d_date_sk#21] (29) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#19] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [cs_sold_date_sk#20] +Right keys [1]: [d_date_sk#21] Join condition: None (30) Project [codegen id : 10] -Output [1]: [cs_ship_customer_sk#18] -Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#10] +Output [1]: [cs_ship_customer_sk#19] +Input [3]: [cs_ship_customer_sk#19, cs_sold_date_sk#20, d_date_sk#21] (31) Exchange -Input [1]: [cs_ship_customer_sk#18] -Arguments: hashpartitioning(cs_ship_customer_sk#18, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [1]: [cs_ship_customer_sk#19] +Arguments: hashpartitioning(cs_ship_customer_sk#19, 5), ENSURE_REQUIREMENTS, [id=#22] (32) Sort [codegen id : 11] -Input [1]: [cs_ship_customer_sk#18] -Arguments: [cs_ship_customer_sk#18 ASC NULLS FIRST], false, 0 +Input [1]: [cs_ship_customer_sk#19] +Arguments: [cs_ship_customer_sk#19 ASC NULLS FIRST], false, 0 (33) SortMergeJoin Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#18] +Right keys [1]: [cs_ship_customer_sk#19] Join condition: None (34) Filter [codegen id : 12] @@ -214,103 +214,103 @@ Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2 (36) Exchange Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] -Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#21] +Arguments: hashpartitioning(c_current_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#23] (37) Sort [codegen id : 13] Input [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Arguments: [c_current_addr_sk#5 ASC NULLS FIRST], false, 0 (38) Scan parquet default.customer_address -Output [2]: [ca_address_sk#22, ca_state#23] +Output [2]: [ca_address_sk#24, ca_state#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 14] -Input [2]: [ca_address_sk#22, ca_state#23] +Input [2]: [ca_address_sk#24, ca_state#25] (40) Filter [codegen id : 14] -Input [2]: [ca_address_sk#22, ca_state#23] -Condition : isnotnull(ca_address_sk#22) +Input [2]: [ca_address_sk#24, ca_state#25] +Condition : isnotnull(ca_address_sk#24) (41) Exchange -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: hashpartitioning(ca_address_sk#22, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] (42) Sort [codegen id : 15] -Input [2]: [ca_address_sk#22, ca_state#23] -Arguments: [ca_address_sk#22 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#24, ca_state#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 (43) SortMergeJoin [codegen id : 16] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#22] +Right keys [1]: [ca_address_sk#24] Join condition: None (44) Project [codegen id : 16] -Output [2]: [c_current_cdemo_sk#4, ca_state#23] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] +Output [2]: [c_current_cdemo_sk#4, ca_state#25] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#24, ca_state#25] (45) Exchange -Input [2]: [c_current_cdemo_sk#4, ca_state#23] -Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#25] +Input [2]: [c_current_cdemo_sk#4, ca_state#25] +Arguments: hashpartitioning(c_current_cdemo_sk#4, 5), ENSURE_REQUIREMENTS, [id=#27] (46) Sort [codegen id : 17] -Input [2]: [c_current_cdemo_sk#4, ca_state#23] +Input [2]: [c_current_cdemo_sk#4, ca_state#25] Arguments: [c_current_cdemo_sk#4 ASC NULLS FIRST], false, 0 (47) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 18] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (49) Filter [codegen id : 18] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Condition : isnotnull(cd_demo_sk#26) +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Condition : isnotnull(cd_demo_sk#28) (50) Exchange -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: hashpartitioning(cd_demo_sk#26, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#34] (51) Sort [codegen id : 19] -Input [6]: [cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Arguments: [cd_demo_sk#26 ASC NULLS FIRST], false, 0 +Input [6]: [cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 20] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#26] +Right keys [1]: [cd_demo_sk#28] Join condition: None (53) Project [codegen id : 20] -Output [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#26, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] +Output [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Input [8]: [c_current_cdemo_sk#4, ca_state#25, cd_demo_sk#28, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] (54) HashAggregate [codegen id : 20] -Input [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#29), partial_max(cd_dep_count#29), partial_sum(cd_dep_count#29), partial_avg(cd_dep_employed_count#30), partial_max(cd_dep_employed_count#30), partial_sum(cd_dep_employed_count#30), partial_avg(cd_dep_college_count#31), partial_max(cd_dep_college_count#31), partial_sum(cd_dep_college_count#31)] -Aggregate Attributes [13]: [count#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41, sum#42, count#43, max#44, sum#45] -Results [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] +Input [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#31), partial_max(cd_dep_count#31), partial_sum(cd_dep_count#31), partial_avg(cd_dep_employed_count#32), partial_max(cd_dep_employed_count#32), partial_sum(cd_dep_employed_count#32), partial_avg(cd_dep_college_count#33), partial_max(cd_dep_college_count#33), partial_sum(cd_dep_college_count#33)] +Aggregate Attributes [13]: [count#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47] +Results [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] (55) Exchange -Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] -Arguments: hashpartitioning(ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] +Arguments: hashpartitioning(ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, 5), ENSURE_REQUIREMENTS, [id=#61] (56) HashAggregate [codegen id : 21] -Input [19]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31, count#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54, sum#55, count#56, max#57, sum#58] -Keys [6]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cd_dep_employed_count#30, cd_dep_college_count#31] -Functions [10]: [count(1), avg(cd_dep_count#29), max(cd_dep_count#29), sum(cd_dep_count#29), avg(cd_dep_employed_count#30), max(cd_dep_employed_count#30), sum(cd_dep_employed_count#30), avg(cd_dep_college_count#31), max(cd_dep_college_count#31), sum(cd_dep_college_count#31)] -Aggregate Attributes [10]: [count(1)#60, avg(cd_dep_count#29)#61, max(cd_dep_count#29)#62, sum(cd_dep_count#29)#63, avg(cd_dep_employed_count#30)#64, max(cd_dep_employed_count#30)#65, sum(cd_dep_employed_count#30)#66, avg(cd_dep_college_count#31)#67, max(cd_dep_college_count#31)#68, sum(cd_dep_college_count#31)#69] -Results [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, count(1)#60 AS cnt1#70, avg(cd_dep_count#29)#61 AS avg(cd_dep_count)#71, max(cd_dep_count#29)#62 AS max(cd_dep_count)#72, sum(cd_dep_count#29)#63 AS sum(cd_dep_count)#73, cd_dep_employed_count#30, count(1)#60 AS cnt2#74, avg(cd_dep_employed_count#30)#64 AS avg(cd_dep_employed_count)#75, max(cd_dep_employed_count#30)#65 AS max(cd_dep_employed_count)#76, sum(cd_dep_employed_count#30)#66 AS sum(cd_dep_employed_count)#77, cd_dep_college_count#31, count(1)#60 AS cnt3#78, avg(cd_dep_college_count#31)#67 AS avg(cd_dep_college_count)#79, max(cd_dep_college_count#31)#68 AS max(cd_dep_college_count)#80, sum(cd_dep_college_count#31)#69 AS sum(cd_dep_college_count)#81] +Input [19]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33, count#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56, sum#57, count#58, max#59, sum#60] +Keys [6]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cd_dep_employed_count#32, cd_dep_college_count#33] +Functions [10]: [count(1), avg(cd_dep_count#31), max(cd_dep_count#31), sum(cd_dep_count#31), avg(cd_dep_employed_count#32), max(cd_dep_employed_count#32), sum(cd_dep_employed_count#32), avg(cd_dep_college_count#33), max(cd_dep_college_count#33), sum(cd_dep_college_count#33)] +Aggregate Attributes [10]: [count(1)#62, avg(cd_dep_count#31)#63, max(cd_dep_count#31)#64, sum(cd_dep_count#31)#65, avg(cd_dep_employed_count#32)#66, max(cd_dep_employed_count#32)#67, sum(cd_dep_employed_count#32)#68, avg(cd_dep_college_count#33)#69, max(cd_dep_college_count#33)#70, sum(cd_dep_college_count#33)#71] +Results [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, count(1)#62 AS cnt1#72, avg(cd_dep_count#31)#63 AS avg(cd_dep_count)#73, max(cd_dep_count#31)#64 AS max(cd_dep_count)#74, sum(cd_dep_count#31)#65 AS sum(cd_dep_count)#75, cd_dep_employed_count#32, count(1)#62 AS cnt2#76, avg(cd_dep_employed_count#32)#66 AS avg(cd_dep_employed_count)#77, max(cd_dep_employed_count#32)#67 AS max(cd_dep_employed_count)#78, sum(cd_dep_employed_count#32)#68 AS sum(cd_dep_employed_count)#79, cd_dep_college_count#33, count(1)#62 AS cnt3#80, avg(cd_dep_college_count#33)#69 AS avg(cd_dep_college_count)#81, max(cd_dep_college_count#33)#70 AS max(cd_dep_college_count)#82, sum(cd_dep_college_count#33)#71 AS sum(cd_dep_college_count)#83] (57) TakeOrderedAndProject -Input [18]: [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#70, avg(cd_dep_count)#71, max(cd_dep_count)#72, sum(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, avg(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, sum(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, avg(cd_dep_college_count)#79, max(cd_dep_college_count)#80, sum(cd_dep_college_count)#81] -Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#27 ASC NULLS FIRST, cd_marital_status#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [ca_state#23, cd_gender#27, cd_marital_status#28, cd_dep_count#29, cnt1#70, avg(cd_dep_count)#71, max(cd_dep_count)#72, sum(cd_dep_count)#73, cd_dep_employed_count#30, cnt2#74, avg(cd_dep_employed_count)#75, max(cd_dep_employed_count)#76, sum(cd_dep_employed_count)#77, cd_dep_college_count#31, cnt3#78, avg(cd_dep_college_count)#79, max(cd_dep_college_count)#80, sum(cd_dep_college_count)#81] +Input [18]: [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cnt1#72, avg(cd_dep_count)#73, max(cd_dep_count)#74, sum(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, avg(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, sum(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, avg(cd_dep_college_count)#81, max(cd_dep_college_count)#82, sum(cd_dep_college_count)#83] +Arguments: 100, [ca_state#25 ASC NULLS FIRST, cd_gender#29 ASC NULLS FIRST, cd_marital_status#30 ASC NULLS FIRST, cd_dep_count#31 ASC NULLS FIRST, cd_dep_employed_count#32 ASC NULLS FIRST, cd_dep_college_count#33 ASC NULLS FIRST], [ca_state#25, cd_gender#29, cd_marital_status#30, cd_dep_count#31, cnt1#72, avg(cd_dep_count)#73, max(cd_dep_count)#74, sum(cd_dep_count)#75, cd_dep_employed_count#32, cnt2#76, avg(cd_dep_employed_count)#77, max(cd_dep_employed_count)#78, sum(cd_dep_employed_count)#79, cd_dep_college_count#33, cnt3#80, avg(cd_dep_college_count)#81, max(cd_dep_college_count)#82, sum(cd_dep_college_count)#83] ===== Subqueries ===== @@ -323,6 +323,6 @@ Output [1]: [d_date_sk#10] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#16 IN dynamicpruning#9 -Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#9 +Subquery:3 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index 12c8230c7e3bc..642bf989159d3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -122,20 +122,20 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#14, ws_sold_date_sk#15] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#16] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#15] -Right keys [1]: [d_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (19) Project [codegen id : 4] Output [1]: [ws_bill_customer_sk#14] -Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#9] +Input [3]: [ws_bill_customer_sk#14, ws_sold_date_sk#15, d_date_sk#16] (20) BroadcastExchange Input [1]: [ws_bill_customer_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] (21) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] @@ -143,34 +143,34 @@ Right keys [1]: [ws_bill_customer_sk#14] Join condition: None (22) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Output [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#8)] ReadSchema: struct (23) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] +Input [2]: [cs_ship_customer_sk#18, cs_sold_date_sk#19] (24) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#9] +Output [1]: [d_date_sk#20] (25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#18] -Right keys [1]: [d_date_sk#9] +Left keys [1]: [cs_sold_date_sk#19] +Right keys [1]: [d_date_sk#20] Join condition: None (26) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#17] -Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#9] +Output [1]: [cs_ship_customer_sk#18] +Input [3]: [cs_ship_customer_sk#18, cs_sold_date_sk#19, d_date_sk#20] (27) BroadcastExchange -Input [1]: [cs_ship_customer_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +Input [1]: [cs_ship_customer_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (28) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#3] -Right keys [1]: [cs_ship_customer_sk#17] +Right keys [1]: [cs_ship_customer_sk#18] Join condition: None (29) Filter [codegen id : 9] @@ -182,80 +182,80 @@ Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] (31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#20, ca_state#21] +Output [2]: [ca_address_sk#22, ca_state#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (32) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_state#21] +Input [2]: [ca_address_sk#22, ca_state#23] (33) Filter [codegen id : 7] -Input [2]: [ca_address_sk#20, ca_state#21] -Condition : isnotnull(ca_address_sk#20) +Input [2]: [ca_address_sk#22, ca_state#23] +Condition : isnotnull(ca_address_sk#22) (34) BroadcastExchange -Input [2]: [ca_address_sk#20, ca_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +Input [2]: [ca_address_sk#22, ca_state#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#24] (35) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#5] -Right keys [1]: [ca_address_sk#20] +Right keys [1]: [ca_address_sk#22] Join condition: None (36) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#4, ca_state#21] -Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#20, ca_state#21] +Output [2]: [c_current_cdemo_sk#4, ca_state#23] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#22, ca_state#23] (37) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (38) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (39) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Condition : isnotnull(cd_demo_sk#23) +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Condition : isnotnull(cd_demo_sk#25) (40) BroadcastExchange -Input [6]: [cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#29] +Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] (41) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#4] -Right keys [1]: [cd_demo_sk#23] +Right keys [1]: [cd_demo_sk#25] Join condition: None (42) Project [codegen id : 9] -Output [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Input [8]: [c_current_cdemo_sk#4, ca_state#21, cd_demo_sk#23, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Output [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [8]: [c_current_cdemo_sk#4, ca_state#23, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] (43) HashAggregate [codegen id : 9] -Input [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#26), partial_max(cd_dep_count#26), partial_sum(cd_dep_count#26), partial_avg(cd_dep_employed_count#27), partial_max(cd_dep_employed_count#27), partial_sum(cd_dep_employed_count#27), partial_avg(cd_dep_college_count#28), partial_max(cd_dep_college_count#28), partial_sum(cd_dep_college_count#28)] -Aggregate Attributes [13]: [count#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42] -Results [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] +Input [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#28), partial_max(cd_dep_count#28), partial_sum(cd_dep_count#28), partial_avg(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_sum(cd_dep_employed_count#29), partial_avg(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_sum(cd_dep_college_count#30)] +Aggregate Attributes [13]: [count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] +Results [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] (44) Exchange -Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] -Arguments: hashpartitioning(ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [id=#56] +Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] +Arguments: hashpartitioning(ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] (45) HashAggregate [codegen id : 10] -Input [19]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55] -Keys [6]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] -Functions [10]: [count(1), avg(cd_dep_count#26), max(cd_dep_count#26), sum(cd_dep_count#26), avg(cd_dep_employed_count#27), max(cd_dep_employed_count#27), sum(cd_dep_employed_count#27), avg(cd_dep_college_count#28), max(cd_dep_college_count#28), sum(cd_dep_college_count#28)] -Aggregate Attributes [10]: [count(1)#57, avg(cd_dep_count#26)#58, max(cd_dep_count#26)#59, sum(cd_dep_count#26)#60, avg(cd_dep_employed_count#27)#61, max(cd_dep_employed_count#27)#62, sum(cd_dep_employed_count#27)#63, avg(cd_dep_college_count#28)#64, max(cd_dep_college_count#28)#65, sum(cd_dep_college_count#28)#66] -Results [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, count(1)#57 AS cnt1#67, avg(cd_dep_count#26)#58 AS avg(cd_dep_count)#68, max(cd_dep_count#26)#59 AS max(cd_dep_count)#69, sum(cd_dep_count#26)#60 AS sum(cd_dep_count)#70, cd_dep_employed_count#27, count(1)#57 AS cnt2#71, avg(cd_dep_employed_count#27)#61 AS avg(cd_dep_employed_count)#72, max(cd_dep_employed_count#27)#62 AS max(cd_dep_employed_count)#73, sum(cd_dep_employed_count#27)#63 AS sum(cd_dep_employed_count)#74, cd_dep_college_count#28, count(1)#57 AS cnt3#75, avg(cd_dep_college_count#28)#64 AS avg(cd_dep_college_count)#76, max(cd_dep_college_count#28)#65 AS max(cd_dep_college_count)#77, sum(cd_dep_college_count#28)#66 AS sum(cd_dep_college_count)#78] +Input [19]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] +Keys [6]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Functions [10]: [count(1), avg(cd_dep_count#28), max(cd_dep_count#28), sum(cd_dep_count#28), avg(cd_dep_employed_count#29), max(cd_dep_employed_count#29), sum(cd_dep_employed_count#29), avg(cd_dep_college_count#30), max(cd_dep_college_count#30), sum(cd_dep_college_count#30)] +Aggregate Attributes [10]: [count(1)#59, avg(cd_dep_count#28)#60, max(cd_dep_count#28)#61, sum(cd_dep_count#28)#62, avg(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, sum(cd_dep_employed_count#29)#65, avg(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, sum(cd_dep_college_count#30)#68] +Results [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, count(1)#59 AS cnt1#69, avg(cd_dep_count#28)#60 AS avg(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, sum(cd_dep_count#28)#62 AS sum(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, avg(cd_dep_employed_count#29)#63 AS avg(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, sum(cd_dep_employed_count#29)#65 AS sum(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, avg(cd_dep_college_count#30)#66 AS avg(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, sum(cd_dep_college_count#30)#68 AS sum(cd_dep_college_count)#80] (46) TakeOrderedAndProject -Input [18]: [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] -Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [ca_state#21, cd_gender#24, cd_marital_status#25, cd_dep_count#26, cnt1#67, avg(cd_dep_count)#68, max(cd_dep_count)#69, sum(cd_dep_count)#70, cd_dep_employed_count#27, cnt2#71, avg(cd_dep_employed_count)#72, max(cd_dep_employed_count)#73, sum(cd_dep_employed_count)#74, cd_dep_college_count#28, cnt3#75, avg(cd_dep_college_count)#76, max(cd_dep_college_count)#77, sum(cd_dep_college_count)#78] +Input [18]: [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] +Arguments: 100, [ca_state#23 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#23, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] ===== Subqueries ===== @@ -268,6 +268,6 @@ Output [1]: [d_date_sk#9] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#15 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt index 1c1b8ad01abfa..df75bec72326d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt @@ -142,52 +142,52 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#13, ws_sold_date_sk#14] (20) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#15] (21) BroadcastHashJoin [codegen id : 7] Left keys [1]: [ws_sold_date_sk#14] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#15] Join condition: None (22) Project [codegen id : 7] -Output [1]: [ws_bill_customer_sk#13 AS customsk#15] -Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#8] +Output [1]: [ws_bill_customer_sk#13 AS customsk#16] +Input [3]: [ws_bill_customer_sk#13, ws_sold_date_sk#14, d_date_sk#15] (23) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Output [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)] ReadSchema: struct (24) ColumnarToRow [codegen id : 9] -Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] +Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18] (25) ReusedExchange [Reuses operator id: 12] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#19] (26) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [cs_sold_date_sk#17] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#19] Join condition: None (27) Project [codegen id : 9] -Output [1]: [cs_ship_customer_sk#16 AS customsk#18] -Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#8] +Output [1]: [cs_ship_customer_sk#17 AS customsk#20] +Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19] (28) Union (29) Exchange -Input [1]: [customsk#15] -Arguments: hashpartitioning(customsk#15, 5), ENSURE_REQUIREMENTS, [id=#19] +Input [1]: [customsk#16] +Arguments: hashpartitioning(customsk#16, 5), ENSURE_REQUIREMENTS, [id=#21] (30) Sort [codegen id : 10] -Input [1]: [customsk#15] -Arguments: [customsk#15 ASC NULLS FIRST], false, 0 +Input [1]: [customsk#16] +Arguments: [customsk#16 ASC NULLS FIRST], false, 0 (31) SortMergeJoin Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#15] +Right keys [1]: [customsk#16] Join condition: None (32) Project [codegen id : 11] @@ -196,103 +196,103 @@ Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (33) Exchange Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] -Arguments: hashpartitioning(c_current_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#20] +Arguments: hashpartitioning(c_current_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#22] (34) Sort [codegen id : 12] Input [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Arguments: [c_current_addr_sk#3 ASC NULLS FIRST], false, 0 (35) Scan parquet default.customer_address -Output [2]: [ca_address_sk#21, ca_state#22] +Output [2]: [ca_address_sk#23, ca_state#24] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 13] -Input [2]: [ca_address_sk#21, ca_state#22] +Input [2]: [ca_address_sk#23, ca_state#24] (37) Filter [codegen id : 13] -Input [2]: [ca_address_sk#21, ca_state#22] -Condition : isnotnull(ca_address_sk#21) +Input [2]: [ca_address_sk#23, ca_state#24] +Condition : isnotnull(ca_address_sk#23) (38) Exchange -Input [2]: [ca_address_sk#21, ca_state#22] -Arguments: hashpartitioning(ca_address_sk#21, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [2]: [ca_address_sk#23, ca_state#24] +Arguments: hashpartitioning(ca_address_sk#23, 5), ENSURE_REQUIREMENTS, [id=#25] (39) Sort [codegen id : 14] -Input [2]: [ca_address_sk#21, ca_state#22] -Arguments: [ca_address_sk#21 ASC NULLS FIRST], false, 0 +Input [2]: [ca_address_sk#23, ca_state#24] +Arguments: [ca_address_sk#23 ASC NULLS FIRST], false, 0 (40) SortMergeJoin [codegen id : 15] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#21] +Right keys [1]: [ca_address_sk#23] Join condition: None (41) Project [codegen id : 15] -Output [2]: [c_current_cdemo_sk#2, ca_state#22] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21, ca_state#22] +Output [2]: [c_current_cdemo_sk#2, ca_state#24] +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#23, ca_state#24] (42) Exchange -Input [2]: [c_current_cdemo_sk#2, ca_state#22] -Arguments: hashpartitioning(c_current_cdemo_sk#2, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [2]: [c_current_cdemo_sk#2, ca_state#24] +Arguments: hashpartitioning(c_current_cdemo_sk#2, 5), ENSURE_REQUIREMENTS, [id=#26] (43) Sort [codegen id : 16] -Input [2]: [c_current_cdemo_sk#2, ca_state#22] +Input [2]: [c_current_cdemo_sk#2, ca_state#24] Arguments: [c_current_cdemo_sk#2 ASC NULLS FIRST], false, 0 (44) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (45) ColumnarToRow [codegen id : 17] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] (46) Filter [codegen id : 17] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Condition : isnotnull(cd_demo_sk#25) +Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Condition : isnotnull(cd_demo_sk#27) (47) Exchange -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: hashpartitioning(cd_demo_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] +Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Arguments: hashpartitioning(cd_demo_sk#27, 5), ENSURE_REQUIREMENTS, [id=#33] (48) Sort [codegen id : 18] -Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Arguments: [cd_demo_sk#25 ASC NULLS FIRST], false, 0 +Input [6]: [cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Arguments: [cd_demo_sk#27 ASC NULLS FIRST], false, 0 (49) SortMergeJoin [codegen id : 19] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#25] +Right keys [1]: [cd_demo_sk#27] Join condition: None (50) Project [codegen id : 19] -Output [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] +Output [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Input [8]: [c_current_cdemo_sk#2, ca_state#24, cd_demo_sk#27, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] (51) HashAggregate [codegen id : 19] -Input [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Keys [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#28), partial_max(cd_dep_count#28), partial_sum(cd_dep_count#28), partial_avg(cd_dep_employed_count#29), partial_max(cd_dep_employed_count#29), partial_sum(cd_dep_employed_count#29), partial_avg(cd_dep_college_count#30), partial_max(cd_dep_college_count#30), partial_sum(cd_dep_college_count#30)] -Aggregate Attributes [13]: [count#32, sum#33, count#34, max#35, sum#36, sum#37, count#38, max#39, sum#40, sum#41, count#42, max#43, sum#44] -Results [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] +Input [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Keys [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#30), partial_max(cd_dep_count#30), partial_sum(cd_dep_count#30), partial_avg(cd_dep_employed_count#31), partial_max(cd_dep_employed_count#31), partial_sum(cd_dep_employed_count#31), partial_avg(cd_dep_college_count#32), partial_max(cd_dep_college_count#32), partial_sum(cd_dep_college_count#32)] +Aggregate Attributes [13]: [count#34, sum#35, count#36, max#37, sum#38, sum#39, count#40, max#41, sum#42, sum#43, count#44, max#45, sum#46] +Results [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] (52) Exchange -Input [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] -Arguments: hashpartitioning(ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] +Arguments: hashpartitioning(ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, 5), ENSURE_REQUIREMENTS, [id=#60] (53) HashAggregate [codegen id : 20] -Input [19]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30, count#45, sum#46, count#47, max#48, sum#49, sum#50, count#51, max#52, sum#53, sum#54, count#55, max#56, sum#57] -Keys [6]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cd_dep_employed_count#29, cd_dep_college_count#30] -Functions [10]: [count(1), avg(cd_dep_count#28), max(cd_dep_count#28), sum(cd_dep_count#28), avg(cd_dep_employed_count#29), max(cd_dep_employed_count#29), sum(cd_dep_employed_count#29), avg(cd_dep_college_count#30), max(cd_dep_college_count#30), sum(cd_dep_college_count#30)] -Aggregate Attributes [10]: [count(1)#59, avg(cd_dep_count#28)#60, max(cd_dep_count#28)#61, sum(cd_dep_count#28)#62, avg(cd_dep_employed_count#29)#63, max(cd_dep_employed_count#29)#64, sum(cd_dep_employed_count#29)#65, avg(cd_dep_college_count#30)#66, max(cd_dep_college_count#30)#67, sum(cd_dep_college_count#30)#68] -Results [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, count(1)#59 AS cnt1#69, avg(cd_dep_count#28)#60 AS avg(cd_dep_count)#70, max(cd_dep_count#28)#61 AS max(cd_dep_count)#71, sum(cd_dep_count#28)#62 AS sum(cd_dep_count)#72, cd_dep_employed_count#29, count(1)#59 AS cnt2#73, avg(cd_dep_employed_count#29)#63 AS avg(cd_dep_employed_count)#74, max(cd_dep_employed_count#29)#64 AS max(cd_dep_employed_count)#75, sum(cd_dep_employed_count#29)#65 AS sum(cd_dep_employed_count)#76, cd_dep_college_count#30, count(1)#59 AS cnt3#77, avg(cd_dep_college_count#30)#66 AS avg(cd_dep_college_count)#78, max(cd_dep_college_count#30)#67 AS max(cd_dep_college_count)#79, sum(cd_dep_college_count#30)#68 AS sum(cd_dep_college_count)#80] +Input [19]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32, count#47, sum#48, count#49, max#50, sum#51, sum#52, count#53, max#54, sum#55, sum#56, count#57, max#58, sum#59] +Keys [6]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cd_dep_employed_count#31, cd_dep_college_count#32] +Functions [10]: [count(1), avg(cd_dep_count#30), max(cd_dep_count#30), sum(cd_dep_count#30), avg(cd_dep_employed_count#31), max(cd_dep_employed_count#31), sum(cd_dep_employed_count#31), avg(cd_dep_college_count#32), max(cd_dep_college_count#32), sum(cd_dep_college_count#32)] +Aggregate Attributes [10]: [count(1)#61, avg(cd_dep_count#30)#62, max(cd_dep_count#30)#63, sum(cd_dep_count#30)#64, avg(cd_dep_employed_count#31)#65, max(cd_dep_employed_count#31)#66, sum(cd_dep_employed_count#31)#67, avg(cd_dep_college_count#32)#68, max(cd_dep_college_count#32)#69, sum(cd_dep_college_count#32)#70] +Results [18]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, count(1)#61 AS cnt1#71, avg(cd_dep_count#30)#62 AS avg(cd_dep_count)#72, max(cd_dep_count#30)#63 AS max(cd_dep_count)#73, sum(cd_dep_count#30)#64 AS sum(cd_dep_count)#74, cd_dep_employed_count#31, count(1)#61 AS cnt2#75, avg(cd_dep_employed_count#31)#65 AS avg(cd_dep_employed_count)#76, max(cd_dep_employed_count#31)#66 AS max(cd_dep_employed_count)#77, sum(cd_dep_employed_count#31)#67 AS sum(cd_dep_employed_count)#78, cd_dep_college_count#32, count(1)#61 AS cnt3#79, avg(cd_dep_college_count#32)#68 AS avg(cd_dep_college_count)#80, max(cd_dep_college_count#32)#69 AS max(cd_dep_college_count)#81, sum(cd_dep_college_count#32)#70 AS sum(cd_dep_college_count)#82] (54) TakeOrderedAndProject -Input [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] -Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] +Input [18]: [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cnt1#71, avg(cd_dep_count)#72, max(cd_dep_count)#73, sum(cd_dep_count)#74, cd_dep_employed_count#31, cnt2#75, avg(cd_dep_employed_count)#76, max(cd_dep_employed_count)#77, sum(cd_dep_employed_count)#78, cd_dep_college_count#32, cnt3#79, avg(cd_dep_college_count)#80, max(cd_dep_college_count)#81, sum(cd_dep_college_count)#82] +Arguments: 100, [ca_state#24 ASC NULLS FIRST, cd_gender#28 ASC NULLS FIRST, cd_marital_status#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [ca_state#24, cd_gender#28, cd_marital_status#29, cd_dep_count#30, cnt1#71, avg(cd_dep_count)#72, max(cd_dep_count)#73, sum(cd_dep_count)#74, cd_dep_employed_count#31, cnt2#75, avg(cd_dep_employed_count)#76, max(cd_dep_employed_count)#77, sum(cd_dep_employed_count)#78, cd_dep_college_count#32, cnt3#79, avg(cd_dep_college_count)#80, max(cd_dep_college_count)#81, sum(cd_dep_college_count)#82] ===== Subqueries ===== @@ -305,6 +305,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 18 Hosting Expression = ws_sold_date_sk#14 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index ffb07d398de71..57710f427f0f8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -120,48 +120,48 @@ ReadSchema: struct Input [2]: [ws_bill_customer_sk#12, ws_sold_date_sk#13] (17) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] Left keys [1]: [ws_sold_date_sk#13] -Right keys [1]: [d_date_sk#7] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] -Output [1]: [ws_bill_customer_sk#12 AS customsk#14] -Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#7] +Output [1]: [ws_bill_customer_sk#12 AS customsk#15] +Input [3]: [ws_bill_customer_sk#12, ws_sold_date_sk#13, d_date_sk#14] (20) Scan parquet default.catalog_sales -Output [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Output [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#6)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [2]: [cs_ship_customer_sk#15, cs_sold_date_sk#16] +Input [2]: [cs_ship_customer_sk#16, cs_sold_date_sk#17] (22) ReusedExchange [Reuses operator id: 10] -Output [1]: [d_date_sk#7] +Output [1]: [d_date_sk#18] (23) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [cs_sold_date_sk#16] -Right keys [1]: [d_date_sk#7] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#18] Join condition: None (24) Project [codegen id : 6] -Output [1]: [cs_ship_customer_sk#15 AS customsk#17] -Input [3]: [cs_ship_customer_sk#15, cs_sold_date_sk#16, d_date_sk#7] +Output [1]: [cs_ship_customer_sk#16 AS customsk#19] +Input [3]: [cs_ship_customer_sk#16, cs_sold_date_sk#17, d_date_sk#18] (25) Union (26) BroadcastExchange -Input [1]: [customsk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] +Input [1]: [customsk#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (27) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_customer_sk#1] -Right keys [1]: [customsk#14] +Right keys [1]: [customsk#15] Join condition: None (28) Project [codegen id : 9] @@ -169,80 +169,80 @@ Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] (29) Scan parquet default.customer_address -Output [2]: [ca_address_sk#19, ca_state#20] +Output [2]: [ca_address_sk#21, ca_state#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#19, ca_state#20] +Input [2]: [ca_address_sk#21, ca_state#22] (31) Filter [codegen id : 7] -Input [2]: [ca_address_sk#19, ca_state#20] -Condition : isnotnull(ca_address_sk#19) +Input [2]: [ca_address_sk#21, ca_state#22] +Condition : isnotnull(ca_address_sk#21) (32) BroadcastExchange -Input [2]: [ca_address_sk#19, ca_state#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Input [2]: [ca_address_sk#21, ca_state#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] (33) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#19] +Right keys [1]: [ca_address_sk#21] Join condition: None (34) Project [codegen id : 9] -Output [2]: [c_current_cdemo_sk#2, ca_state#20] -Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#19, ca_state#20] +Output [2]: [c_current_cdemo_sk#2, ca_state#22] +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21, ca_state#22] (35) Scan parquet default.customer_demographics -Output [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Output [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (36) ColumnarToRow [codegen id : 8] -Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] (37) Filter [codegen id : 8] -Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Condition : isnotnull(cd_demo_sk#22) +Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Condition : isnotnull(cd_demo_sk#24) (38) BroadcastExchange -Input [6]: [cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] +Input [6]: [cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (39) BroadcastHashJoin [codegen id : 9] Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#22] +Right keys [1]: [cd_demo_sk#24] Join condition: None (40) Project [codegen id : 9] -Output [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Input [8]: [c_current_cdemo_sk#2, ca_state#20, cd_demo_sk#22, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] +Output [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Input [8]: [c_current_cdemo_sk#2, ca_state#22, cd_demo_sk#24, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] (41) HashAggregate [codegen id : 9] -Input [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Functions [10]: [partial_count(1), partial_avg(cd_dep_count#25), partial_max(cd_dep_count#25), partial_sum(cd_dep_count#25), partial_avg(cd_dep_employed_count#26), partial_max(cd_dep_employed_count#26), partial_sum(cd_dep_employed_count#26), partial_avg(cd_dep_college_count#27), partial_max(cd_dep_college_count#27), partial_sum(cd_dep_college_count#27)] -Aggregate Attributes [13]: [count#29, sum#30, count#31, max#32, sum#33, sum#34, count#35, max#36, sum#37, sum#38, count#39, max#40, sum#41] -Results [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] +Input [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Keys [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#27), partial_max(cd_dep_count#27), partial_sum(cd_dep_count#27), partial_avg(cd_dep_employed_count#28), partial_max(cd_dep_employed_count#28), partial_sum(cd_dep_employed_count#28), partial_avg(cd_dep_college_count#29), partial_max(cd_dep_college_count#29), partial_sum(cd_dep_college_count#29)] +Aggregate Attributes [13]: [count#31, sum#32, count#33, max#34, sum#35, sum#36, count#37, max#38, sum#39, sum#40, count#41, max#42, sum#43] +Results [19]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] (42) Exchange -Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Arguments: hashpartitioning(ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [19]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] +Arguments: hashpartitioning(ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, 5), ENSURE_REQUIREMENTS, [id=#57] (43) HashAggregate [codegen id : 10] -Input [19]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27, count#42, sum#43, count#44, max#45, sum#46, sum#47, count#48, max#49, sum#50, sum#51, count#52, max#53, sum#54] -Keys [6]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cd_dep_employed_count#26, cd_dep_college_count#27] -Functions [10]: [count(1), avg(cd_dep_count#25), max(cd_dep_count#25), sum(cd_dep_count#25), avg(cd_dep_employed_count#26), max(cd_dep_employed_count#26), sum(cd_dep_employed_count#26), avg(cd_dep_college_count#27), max(cd_dep_college_count#27), sum(cd_dep_college_count#27)] -Aggregate Attributes [10]: [count(1)#56, avg(cd_dep_count#25)#57, max(cd_dep_count#25)#58, sum(cd_dep_count#25)#59, avg(cd_dep_employed_count#26)#60, max(cd_dep_employed_count#26)#61, sum(cd_dep_employed_count#26)#62, avg(cd_dep_college_count#27)#63, max(cd_dep_college_count#27)#64, sum(cd_dep_college_count#27)#65] -Results [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, count(1)#56 AS cnt1#66, avg(cd_dep_count#25)#57 AS avg(cd_dep_count)#67, max(cd_dep_count#25)#58 AS max(cd_dep_count)#68, sum(cd_dep_count#25)#59 AS sum(cd_dep_count)#69, cd_dep_employed_count#26, count(1)#56 AS cnt2#70, avg(cd_dep_employed_count#26)#60 AS avg(cd_dep_employed_count)#71, max(cd_dep_employed_count#26)#61 AS max(cd_dep_employed_count)#72, sum(cd_dep_employed_count#26)#62 AS sum(cd_dep_employed_count)#73, cd_dep_college_count#27, count(1)#56 AS cnt3#74, avg(cd_dep_college_count#27)#63 AS avg(cd_dep_college_count)#75, max(cd_dep_college_count#27)#64 AS max(cd_dep_college_count)#76, sum(cd_dep_college_count#27)#65 AS sum(cd_dep_college_count)#77] +Input [19]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29, count#44, sum#45, count#46, max#47, sum#48, sum#49, count#50, max#51, sum#52, sum#53, count#54, max#55, sum#56] +Keys [6]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cd_dep_employed_count#28, cd_dep_college_count#29] +Functions [10]: [count(1), avg(cd_dep_count#27), max(cd_dep_count#27), sum(cd_dep_count#27), avg(cd_dep_employed_count#28), max(cd_dep_employed_count#28), sum(cd_dep_employed_count#28), avg(cd_dep_college_count#29), max(cd_dep_college_count#29), sum(cd_dep_college_count#29)] +Aggregate Attributes [10]: [count(1)#58, avg(cd_dep_count#27)#59, max(cd_dep_count#27)#60, sum(cd_dep_count#27)#61, avg(cd_dep_employed_count#28)#62, max(cd_dep_employed_count#28)#63, sum(cd_dep_employed_count#28)#64, avg(cd_dep_college_count#29)#65, max(cd_dep_college_count#29)#66, sum(cd_dep_college_count#29)#67] +Results [18]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, count(1)#58 AS cnt1#68, avg(cd_dep_count#27)#59 AS avg(cd_dep_count)#69, max(cd_dep_count#27)#60 AS max(cd_dep_count)#70, sum(cd_dep_count#27)#61 AS sum(cd_dep_count)#71, cd_dep_employed_count#28, count(1)#58 AS cnt2#72, avg(cd_dep_employed_count#28)#62 AS avg(cd_dep_employed_count)#73, max(cd_dep_employed_count#28)#63 AS max(cd_dep_employed_count)#74, sum(cd_dep_employed_count#28)#64 AS sum(cd_dep_employed_count)#75, cd_dep_college_count#29, count(1)#58 AS cnt3#76, avg(cd_dep_college_count#29)#65 AS avg(cd_dep_college_count)#77, max(cd_dep_college_count#29)#66 AS max(cd_dep_college_count)#78, sum(cd_dep_college_count#29)#67 AS sum(cd_dep_college_count)#79] (44) TakeOrderedAndProject -Input [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] -Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] +Input [18]: [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#28, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#29, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79] +Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_dep_count#27 ASC NULLS FIRST, cd_dep_employed_count#28 ASC NULLS FIRST, cd_dep_college_count#29 ASC NULLS FIRST], [ca_state#22, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#28, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#29, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79] ===== Subqueries ===== @@ -255,6 +255,6 @@ Output [1]: [d_date_sk#7] Subquery:2 Hosting operator id = 15 Hosting Expression = ws_sold_date_sk#13 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 20 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#6 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 9f594df6754ba..79037befbd6b5 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 @@ -169,60 +169,60 @@ Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#22, sum(UnscaledV Results [6]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#22,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#23,17,2))), DecimalType(37,20), true) as decimal(38,20)) AS gross_margin#24, i_category#15, i_class#14, 0 AS t_category#25, 0 AS t_class#26, 0 AS lochierarchy#27] (27) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#15, i_class#14, sum#28, sum#29] +Output [4]: [i_category#28, i_class#29, sum#30, sum#31] (28) HashAggregate [codegen id : 10] -Input [4]: [i_category#15, i_class#14, sum#28, sum#29] -Keys [2]: [i_category#15, i_class#14] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#30, sum(UnscaledValue(ss_ext_sales_price#3))#31] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#30,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#31,17,2) AS ss_ext_sales_price#33, i_category#15] +Input [4]: [i_category#28, i_class#29, sum#30, sum#31] +Keys [2]: [i_category#28, i_class#29] +Functions [2]: [sum(UnscaledValue(ss_net_profit#32)), sum(UnscaledValue(ss_ext_sales_price#33))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#32))#34, sum(UnscaledValue(ss_ext_sales_price#33))#35] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#32))#34,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#33))#35,17,2) AS ss_ext_sales_price#37, i_category#28] (29) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#32, ss_ext_sales_price#33, i_category#15] -Keys [1]: [i_category#15] -Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] -Aggregate Attributes [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] -Results [5]: [i_category#15, sum#38, isEmpty#39, sum#40, isEmpty#41] +Input [3]: [ss_net_profit#36, ss_ext_sales_price#37, i_category#28] +Keys [1]: [i_category#28] +Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] +Aggregate Attributes [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] +Results [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] (30) Exchange -Input [5]: [i_category#15, sum#38, isEmpty#39, sum#40, isEmpty#41] -Arguments: hashpartitioning(i_category#15, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, [id=#46] (31) HashAggregate [codegen id : 11] -Input [5]: [i_category#15, sum#38, isEmpty#39, sum#40, isEmpty#41] -Keys [1]: [i_category#15] -Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] -Aggregate Attributes [2]: [sum(ss_net_profit#32)#43, sum(ss_ext_sales_price#33)#44] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#43) / promote_precision(sum(ss_ext_sales_price#33)#44)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#45, i_category#15, null AS i_class#46, 0 AS t_category#47, 1 AS t_class#48, 1 AS lochierarchy#49] +Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] +Keys [1]: [i_category#28] +Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] +Aggregate Attributes [2]: [sum(ss_net_profit#36)#47, sum(ss_ext_sales_price#37)#48] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#47) / promote_precision(sum(ss_ext_sales_price#37)#48)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#49, i_category#28, null AS i_class#50, 0 AS t_category#51, 1 AS t_class#52, 1 AS lochierarchy#53] (32) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#15, i_class#14, sum#50, sum#51] +Output [4]: [i_category#54, i_class#55, sum#56, sum#57] (33) HashAggregate [codegen id : 16] -Input [4]: [i_category#15, i_class#14, sum#50, sum#51] -Keys [2]: [i_category#15, i_class#14] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#52, sum(UnscaledValue(ss_ext_sales_price#3))#53] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#52,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#53,17,2) AS ss_ext_sales_price#33] +Input [4]: [i_category#54, i_class#55, sum#56, sum#57] +Keys [2]: [i_category#54, i_class#55] +Functions [2]: [sum(UnscaledValue(ss_net_profit#58)), sum(UnscaledValue(ss_ext_sales_price#59))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#58))#60, sum(UnscaledValue(ss_ext_sales_price#59))#61] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#58))#60,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#59))#61,17,2) AS ss_ext_sales_price#37] (34) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#32, ss_ext_sales_price#33] +Input [2]: [ss_net_profit#36, ss_ext_sales_price#37] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] -Aggregate Attributes [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] -Results [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] +Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] +Aggregate Attributes [4]: [sum#62, isEmpty#63, sum#64, isEmpty#65] +Results [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] (35) Exchange -Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] +Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#70] (36) HashAggregate [codegen id : 17] -Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] +Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] Keys: [] -Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] -Aggregate Attributes [2]: [sum(ss_net_profit#32)#63, sum(ss_ext_sales_price#33)#64] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#63) / promote_precision(sum(ss_ext_sales_price#33)#64)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#65, null AS i_category#66, null AS i_class#67, 1 AS t_category#68, 1 AS t_class#69, 2 AS lochierarchy#70] +Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] +Aggregate Attributes [2]: [sum(ss_net_profit#36)#71, sum(ss_ext_sales_price#37)#72] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#71) / promote_precision(sum(ss_ext_sales_price#37)#72)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#73, null AS i_category#74, null AS i_class#75, 1 AS t_category#76, 1 AS t_class#77, 2 AS lochierarchy#78] (37) Union @@ -235,34 +235,34 @@ Results [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class (39) Exchange Input [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27] -Arguments: hashpartitioning(gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#71] +Arguments: hashpartitioning(gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#79] (40) HashAggregate [codegen id : 19] Input [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27] Keys [6]: [gross_margin#24, i_category#15, i_class#14, t_category#25, t_class#26, lochierarchy#27] Functions: [] Aggregate Attributes: [] -Results [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#15 END AS _w0#72] +Results [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#15 END AS _w0#80] (41) Exchange -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72] -Arguments: hashpartitioning(lochierarchy#27, _w0#72, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80] +Arguments: hashpartitioning(lochierarchy#27, _w0#80, 5), ENSURE_REQUIREMENTS, [id=#81] (42) Sort [codegen id : 20] -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72] -Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80] +Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#80 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 (43) Window -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72] -Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#72, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#74], [lochierarchy#27, _w0#72], [gross_margin#24 ASC NULLS FIRST] +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80] +Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#80, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#82], [lochierarchy#27, _w0#80], [gross_margin#24 ASC NULLS FIRST] (44) Project [codegen id : 21] -Output [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#74] -Input [6]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#72, rank_within_parent#74] +Output [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#82] +Input [6]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, _w0#80, rank_within_parent#82] (45) TakeOrderedAndProject -Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#74] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#74 ASC NULLS FIRST], [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#74] +Input [5]: [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#82] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#82 ASC NULLS FIRST], [gross_margin#24, i_category#15, i_class#14, lochierarchy#27, rank_within_parent#82] ===== Subqueries ===== 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 5e58913557f22..bc2931129868b 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 @@ -169,60 +169,60 @@ Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#22, sum(UnscaledV Results [6]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#22,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#23,17,2))), DecimalType(37,20), true) as decimal(38,20)) AS gross_margin#24, i_category#12, i_class#11, 0 AS t_category#25, 0 AS t_class#26, 0 AS lochierarchy#27] (27) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#12, i_class#11, sum#28, sum#29] +Output [4]: [i_category#28, i_class#29, sum#30, sum#31] (28) HashAggregate [codegen id : 10] -Input [4]: [i_category#12, i_class#11, sum#28, sum#29] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#30, sum(UnscaledValue(ss_ext_sales_price#3))#31] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#30,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#31,17,2) AS ss_ext_sales_price#33, i_category#12] +Input [4]: [i_category#28, i_class#29, sum#30, sum#31] +Keys [2]: [i_category#28, i_class#29] +Functions [2]: [sum(UnscaledValue(ss_net_profit#32)), sum(UnscaledValue(ss_ext_sales_price#33))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#32))#34, sum(UnscaledValue(ss_ext_sales_price#33))#35] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#32))#34,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#33))#35,17,2) AS ss_ext_sales_price#37, i_category#28] (29) HashAggregate [codegen id : 10] -Input [3]: [ss_net_profit#32, ss_ext_sales_price#33, i_category#12] -Keys [1]: [i_category#12] -Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] -Aggregate Attributes [4]: [sum#34, isEmpty#35, sum#36, isEmpty#37] -Results [5]: [i_category#12, sum#38, isEmpty#39, sum#40, isEmpty#41] +Input [3]: [ss_net_profit#36, ss_ext_sales_price#37, i_category#28] +Keys [1]: [i_category#28] +Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] +Aggregate Attributes [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] +Results [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] (30) Exchange -Input [5]: [i_category#12, sum#38, isEmpty#39, sum#40, isEmpty#41] -Arguments: hashpartitioning(i_category#12, 5), ENSURE_REQUIREMENTS, [id=#42] +Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] +Arguments: hashpartitioning(i_category#28, 5), ENSURE_REQUIREMENTS, [id=#46] (31) HashAggregate [codegen id : 11] -Input [5]: [i_category#12, sum#38, isEmpty#39, sum#40, isEmpty#41] -Keys [1]: [i_category#12] -Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] -Aggregate Attributes [2]: [sum(ss_net_profit#32)#43, sum(ss_ext_sales_price#33)#44] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#43) / promote_precision(sum(ss_ext_sales_price#33)#44)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#45, i_category#12, null AS i_class#46, 0 AS t_category#47, 1 AS t_class#48, 1 AS lochierarchy#49] +Input [5]: [i_category#28, sum#42, isEmpty#43, sum#44, isEmpty#45] +Keys [1]: [i_category#28] +Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] +Aggregate Attributes [2]: [sum(ss_net_profit#36)#47, sum(ss_ext_sales_price#37)#48] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#47) / promote_precision(sum(ss_ext_sales_price#37)#48)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#49, i_category#28, null AS i_class#50, 0 AS t_category#51, 1 AS t_class#52, 1 AS lochierarchy#53] (32) ReusedExchange [Reuses operator id: unknown] -Output [4]: [i_category#12, i_class#11, sum#50, sum#51] +Output [4]: [i_category#54, i_class#55, sum#56, sum#57] (33) HashAggregate [codegen id : 16] -Input [4]: [i_category#12, i_class#11, sum#50, sum#51] -Keys [2]: [i_category#12, i_class#11] -Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#52, sum(UnscaledValue(ss_ext_sales_price#3))#53] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#52,17,2) AS ss_net_profit#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#53,17,2) AS ss_ext_sales_price#33] +Input [4]: [i_category#54, i_class#55, sum#56, sum#57] +Keys [2]: [i_category#54, i_class#55] +Functions [2]: [sum(UnscaledValue(ss_net_profit#58)), sum(UnscaledValue(ss_ext_sales_price#59))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#58))#60, sum(UnscaledValue(ss_ext_sales_price#59))#61] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#58))#60,17,2) AS ss_net_profit#36, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#59))#61,17,2) AS ss_ext_sales_price#37] (34) HashAggregate [codegen id : 16] -Input [2]: [ss_net_profit#32, ss_ext_sales_price#33] +Input [2]: [ss_net_profit#36, ss_ext_sales_price#37] Keys: [] -Functions [2]: [partial_sum(ss_net_profit#32), partial_sum(ss_ext_sales_price#33)] -Aggregate Attributes [4]: [sum#54, isEmpty#55, sum#56, isEmpty#57] -Results [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] +Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] +Aggregate Attributes [4]: [sum#62, isEmpty#63, sum#64, isEmpty#65] +Results [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] (35) Exchange -Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] +Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#70] (36) HashAggregate [codegen id : 17] -Input [4]: [sum#58, isEmpty#59, sum#60, isEmpty#61] +Input [4]: [sum#66, isEmpty#67, sum#68, isEmpty#69] Keys: [] -Functions [2]: [sum(ss_net_profit#32), sum(ss_ext_sales_price#33)] -Aggregate Attributes [2]: [sum(ss_net_profit#32)#63, sum(ss_ext_sales_price#33)#64] -Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#32)#63) / promote_precision(sum(ss_ext_sales_price#33)#64)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#65, null AS i_category#66, null AS i_class#67, 1 AS t_category#68, 1 AS t_class#69, 2 AS lochierarchy#70] +Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] +Aggregate Attributes [2]: [sum(ss_net_profit#36)#71, sum(ss_ext_sales_price#37)#72] +Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#36)#71) / promote_precision(sum(ss_ext_sales_price#37)#72)), DecimalType(38,11), true) as decimal(38,20)) AS gross_margin#73, null AS i_category#74, null AS i_class#75, 1 AS t_category#76, 1 AS t_class#77, 2 AS lochierarchy#78] (37) Union @@ -235,34 +235,34 @@ Results [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class (39) Exchange Input [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27] -Arguments: hashpartitioning(gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#71] +Arguments: hashpartitioning(gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [id=#79] (40) HashAggregate [codegen id : 19] Input [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27] Keys [6]: [gross_margin#24, i_category#12, i_class#11, t_category#25, t_class#26, lochierarchy#27] Functions: [] Aggregate Attributes: [] -Results [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#12 END AS _w0#72] +Results [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, CASE WHEN (t_class#26 = 0) THEN i_category#12 END AS _w0#80] (41) Exchange -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72] -Arguments: hashpartitioning(lochierarchy#27, _w0#72, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80] +Arguments: hashpartitioning(lochierarchy#27, _w0#80, 5), ENSURE_REQUIREMENTS, [id=#81] (42) Sort [codegen id : 20] -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72] -Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80] +Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#80 ASC NULLS FIRST, gross_margin#24 ASC NULLS FIRST], false, 0 (43) Window -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72] -Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#72, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#74], [lochierarchy#27, _w0#72], [gross_margin#24 ASC NULLS FIRST] +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80] +Arguments: [rank(gross_margin#24) windowspecdefinition(lochierarchy#27, _w0#80, gross_margin#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#82], [lochierarchy#27, _w0#80], [gross_margin#24 ASC NULLS FIRST] (44) Project [codegen id : 21] -Output [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#74] -Input [6]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#72, rank_within_parent#74] +Output [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#82] +Input [6]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, _w0#80, rank_within_parent#82] (45) TakeOrderedAndProject -Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#74] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#74 ASC NULLS FIRST], [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#74] +Input [5]: [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#82] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#82 ASC NULLS FIRST], [gross_margin#24, i_category#12, i_class#11, lochierarchy#27, rank_within_parent#82] ===== Subqueries ===== 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 a0737e88bd760..636fa32f94212 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 @@ -229,78 +229,78 @@ Output [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_ye (39) HashAggregate [codegen id : 21] Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum#36] Keys [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35] -Functions [1]: [sum(UnscaledValue(ss_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#3))#37] -Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#3))#37,17,2) AS sum_sales#38] +Functions [1]: [sum(UnscaledValue(ss_sales_price#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#37))#38] +Results [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#37))#38,17,2) AS sum_sales#39] (40) Exchange -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, 5), ENSURE_REQUIREMENTS, [id=#40] (41) Sort [codegen id : 22] -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST], false, 0 (42) Window -Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +Input [7]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#41], [i_category#30, i_brand#31, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] (43) Project [codegen id : 23] -Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#38, rn#40] +Output [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Input [8]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#39, rn#41] (44) Exchange -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1), 5), ENSURE_REQUIREMENTS, [id=#41] +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Arguments: hashpartitioning(i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1), 5), ENSURE_REQUIREMENTS, [id=#42] (45) Sort [codegen id : 24] -Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] -Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#40 + 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] +Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, (rn#41 + 1) ASC NULLS FIRST], false, 0 (46) SortMergeJoin [codegen id : 25] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#40 + 1)] +Right keys [5]: [i_category#30, i_brand#31, s_store_name#32, s_company_name#33, (rn#41 + 1)] Join condition: None (47) Project [codegen id : 25] -Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38] -Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#38, rn#40] +Output [10]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39] +Input [15]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, i_category#30, i_brand#31, s_store_name#32, s_company_name#33, sum_sales#39, rn#41] (48) ReusedExchange [Reuses operator id: 40] -Output [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] +Output [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] (49) Sort [codegen id : 34] -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST], false, 0 +Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] +Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST], false, 0 (50) Window -Input [7]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48] -Arguments: [rank(d_year#46, d_moy#47) windowspecdefinition(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#49], [i_category#42, i_brand#43, s_store_name#44, s_company_name#45], [d_year#46 ASC NULLS FIRST, d_moy#47 ASC NULLS FIRST] +Input [7]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49] +Arguments: [rank(d_year#47, d_moy#48) windowspecdefinition(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#50], [i_category#43, i_brand#44, s_store_name#45, s_company_name#46], [d_year#47 ASC NULLS FIRST, d_moy#48 ASC NULLS FIRST] (51) Project [codegen id : 35] -Output [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Input [8]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, d_year#46, d_moy#47, sum_sales#48, rn#49] +Output [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Input [8]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, d_year#47, d_moy#48, sum_sales#49, rn#50] (52) Exchange -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Arguments: hashpartitioning(i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1), 5), ENSURE_REQUIREMENTS, [id=#50] +Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Arguments: hashpartitioning(i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1), 5), ENSURE_REQUIREMENTS, [id=#51] (53) Sort [codegen id : 36] -Input [6]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] -Arguments: [i_category#42 ASC NULLS FIRST, i_brand#43 ASC NULLS FIRST, s_store_name#44 ASC NULLS FIRST, s_company_name#45 ASC NULLS FIRST, (rn#49 - 1) ASC NULLS FIRST], false, 0 +Input [6]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] +Arguments: [i_category#43 ASC NULLS FIRST, i_brand#44 ASC NULLS FIRST, s_store_name#45 ASC NULLS FIRST, s_company_name#46 ASC NULLS FIRST, (rn#50 - 1) ASC NULLS FIRST], false, 0 (54) SortMergeJoin [codegen id : 37] Left keys [5]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, rn#28] -Right keys [5]: [i_category#42, i_brand#43, s_store_name#44, s_company_name#45, (rn#49 - 1)] +Right keys [5]: [i_category#43, i_brand#44, s_store_name#45, s_company_name#46, (rn#50 - 1)] Join condition: None (55) Project [codegen id : 37] -Output [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#38 AS psum#51, sum_sales#48 AS nsum#52] -Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#38, i_category#42, i_brand#43, s_store_name#44, s_company_name#45, sum_sales#48, rn#49] +Output [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, sum_sales#39 AS psum#52, sum_sales#49 AS nsum#53] +Input [16]: [i_category#17, i_brand#16, s_store_name#11, s_company_name#12, d_year#7, d_moy#8, sum_sales#23, avg_monthly_sales#26, rn#28, sum_sales#39, i_category#43, i_brand#44, s_store_name#45, s_company_name#46, sum_sales#49, rn#50] (56) TakeOrderedAndProject -Input [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#51, nsum#52] +Input [7]: [i_category#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] +Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#23 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#17, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#23, psum#52, nsum#53] ===== Subqueries ===== 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 b0a231fc01379..734eec1be3565 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 @@ -202,70 +202,70 @@ Output [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_ye (34) HashAggregate [codegen id : 13] 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#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#35] -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#6))#35,17,2) AS sum_sales#36] +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] (35) 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#36] -Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#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#37] +Arguments: hashpartitioning(i_category#28, i_brand#29, s_store_name#30, s_company_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] (36) Sort [codegen id : 14] -Input [7]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36] +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: [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 (37) 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#36] -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#38], [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#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] (38) Project [codegen id : 15] -Output [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] -Input [8]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] +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] (39) BroadcastExchange -Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] -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=#39] +Input [6]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] +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=#40] (40) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#38 + 1)] +Right keys [5]: [i_category#28, i_brand#29, s_store_name#30, s_company_name#31, (rn#39 + 1)] Join condition: None (41) Project [codegen id : 23] -Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] -Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#36, rn#38] +Output [10]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] +Input [15]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#28, i_brand#29, s_store_name#30, s_company_name#31, sum_sales#37, rn#39] (42) ReusedExchange [Reuses operator id: 35] -Output [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] +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] (43) Sort [codegen id : 21] -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#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, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 +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 (44) Window -Input [7]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46] -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#47], [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] +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] (45) Project [codegen id : 22] -Output [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] -Input [8]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] +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] (46) BroadcastExchange -Input [6]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] -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=#48] +Input [6]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] +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=#49] (47) BroadcastHashJoin [codegen id : 23] Left keys [5]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, rn#27] -Right keys [5]: [i_category#40, i_brand#41, s_store_name#42, s_company_name#43, (rn#47 - 1)] +Right keys [5]: [i_category#41, i_brand#42, s_store_name#43, s_company_name#44, (rn#48 - 1)] Join condition: None (48) Project [codegen id : 23] -Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#49, sum_sales#46 AS nsum#50] -Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, s_store_name#42, s_company_name#43, sum_sales#46, rn#47] +Output [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#50, sum_sales#47 AS nsum#51] +Input [16]: [i_category#3, i_brand#2, s_store_name#15, s_company_name#16, d_year#11, d_moy#12, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, s_store_name#43, s_company_name#44, sum_sales#47, rn#48] (49) TakeOrderedAndProject -Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, 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#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#22, psum#49, nsum#50] +Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, 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#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#22, psum#50, nsum#51] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt index 57e57951851aa..a1c7e4bfb35dc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt @@ -250,225 +250,225 @@ Output [5]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_net_profit#47, cs_sold_date_sk#48] (36) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#49] (37) BroadcastHashJoin [codegen id : 12] Left keys [1]: [cs_sold_date_sk#48] -Right keys [1]: [d_date_sk#8] +Right keys [1]: [d_date_sk#49] Join condition: None (38) Project [codegen id : 12] Output [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#8] +Input [6]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cs_sold_date_sk#48, d_date_sk#49] (39) Exchange Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] -Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#49] +Arguments: hashpartitioning(cs_order_number#44, cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#50] (40) Sort [codegen id : 13] Input [4]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46] Arguments: [cs_order_number#44 ASC NULLS FIRST, cs_item_sk#43 ASC NULLS FIRST], false, 0 (41) Scan parquet default.catalog_returns -Output [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Output [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (42) ColumnarToRow [codegen id : 14] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] (43) Filter [codegen id : 14] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] -Condition : (((isnotnull(cr_return_amount#53) AND (cr_return_amount#53 > 10000.00)) AND isnotnull(cr_order_number#51)) AND isnotnull(cr_item_sk#50)) +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] +Condition : (((isnotnull(cr_return_amount#54) AND (cr_return_amount#54 > 10000.00)) AND isnotnull(cr_order_number#52)) AND isnotnull(cr_item_sk#51)) (44) Project [codegen id : 14] -Output [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Input [5]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53, cr_returned_date_sk#54] +Output [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Input [5]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54, cr_returned_date_sk#55] (45) Exchange -Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Arguments: hashpartitioning(cr_order_number#51, cr_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Arguments: hashpartitioning(cr_order_number#52, cr_item_sk#51, 5), ENSURE_REQUIREMENTS, [id=#56] (46) Sort [codegen id : 15] -Input [4]: [cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] -Arguments: [cr_order_number#51 ASC NULLS FIRST, cr_item_sk#50 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] +Arguments: [cr_order_number#52 ASC NULLS FIRST, cr_item_sk#51 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 16] Left keys [2]: [cs_order_number#44, cs_item_sk#43] -Right keys [2]: [cr_order_number#51, cr_item_sk#50] +Right keys [2]: [cr_order_number#52, cr_item_sk#51] Join condition: None (48) Project [codegen id : 16] -Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] -Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#50, cr_order_number#51, cr_return_quantity#52, cr_return_amount#53] +Output [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] +Input [8]: [cs_item_sk#43, cs_order_number#44, cs_quantity#45, cs_net_paid#46, cr_item_sk#51, cr_order_number#52, cr_return_quantity#53, cr_return_amount#54] (49) HashAggregate [codegen id : 16] -Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#52, cr_return_amount#53] +Input [5]: [cs_item_sk#43, cs_quantity#45, cs_net_paid#46, cr_return_quantity#53, cr_return_amount#54] Keys [1]: [cs_item_sk#43] -Functions [4]: [partial_sum(coalesce(cr_return_quantity#52, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#56, sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] -Results [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#53, 0)), partial_sum(coalesce(cs_quantity#45, 0)), partial_sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#57, sum#58, sum#59, isEmpty#60, sum#61, isEmpty#62] +Results [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] (50) Exchange -Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] -Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#68] +Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] +Arguments: hashpartitioning(cs_item_sk#43, 5), ENSURE_REQUIREMENTS, [id=#69] (51) HashAggregate [codegen id : 17] -Input [7]: [cs_item_sk#43, sum#62, sum#63, sum#64, isEmpty#65, sum#66, isEmpty#67] +Input [7]: [cs_item_sk#43, sum#63, sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] Keys [1]: [cs_item_sk#43] -Functions [4]: [sum(coalesce(cr_return_quantity#52, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#52, 0))#69, sum(coalesce(cs_quantity#45, 0))#70, sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72] -Results [3]: [cs_item_sk#43 AS item#73, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#52, 0))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#70 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#74, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#53 as decimal(12,2)), 0.00))#71 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#72 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#75] +Functions [4]: [sum(coalesce(cr_return_quantity#53, 0)), sum(coalesce(cs_quantity#45, 0)), sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#53, 0))#70, sum(coalesce(cs_quantity#45, 0))#71, sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72, sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73] +Results [3]: [cs_item_sk#43 AS item#74, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#53, 0))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#45, 0))#71 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#75, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#54 as decimal(12,2)), 0.00))#72 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#46 as decimal(12,2)), 0.00))#73 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#76] (52) Exchange -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#76] +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#77] (53) Sort [codegen id : 18] -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: [return_ratio#74 ASC NULLS FIRST], false, 0 +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: [return_ratio#75 ASC NULLS FIRST], false, 0 (54) Window -Input [3]: [item#73, return_ratio#74, currency_ratio#75] -Arguments: [rank(return_ratio#74) windowspecdefinition(return_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#77], [return_ratio#74 ASC NULLS FIRST] +Input [3]: [item#74, return_ratio#75, currency_ratio#76] +Arguments: [rank(return_ratio#75) windowspecdefinition(return_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#78], [return_ratio#75 ASC NULLS FIRST] (55) Sort [codegen id : 19] -Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] -Arguments: [currency_ratio#75 ASC NULLS FIRST], false, 0 +Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] +Arguments: [currency_ratio#76 ASC NULLS FIRST], false, 0 (56) Window -Input [4]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77] -Arguments: [rank(currency_ratio#75) windowspecdefinition(currency_ratio#75 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#78], [currency_ratio#75 ASC NULLS FIRST] +Input [4]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78] +Arguments: [rank(currency_ratio#76) windowspecdefinition(currency_ratio#76 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#79], [currency_ratio#76 ASC NULLS FIRST] (57) Filter [codegen id : 20] -Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] -Condition : ((return_rank#77 <= 10) OR (currency_rank#78 <= 10)) +Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] +Condition : ((return_rank#78 <= 10) OR (currency_rank#79 <= 10)) (58) Project [codegen id : 20] -Output [5]: [catalog AS channel#79, item#73, return_ratio#74, return_rank#77, currency_rank#78] -Input [5]: [item#73, return_ratio#74, currency_ratio#75, return_rank#77, currency_rank#78] +Output [5]: [catalog AS channel#80, item#74, return_ratio#75, return_rank#78, currency_rank#79] +Input [5]: [item#74, return_ratio#75, currency_ratio#76, return_rank#78, currency_rank#79] (59) Scan parquet default.store_sales -Output [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Output [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#85), dynamicpruningexpression(ss_sold_date_sk#85 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#86), dynamicpruningexpression(ss_sold_date_sk#86 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (60) ColumnarToRow [codegen id : 22] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] (61) Filter [codegen id : 22] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] -Condition : (((((((isnotnull(ss_net_profit#84) AND isnotnull(ss_net_paid#83)) AND isnotnull(ss_quantity#82)) AND (ss_net_profit#84 > 1.00)) AND (ss_net_paid#83 > 0.00)) AND (ss_quantity#82 > 0)) AND isnotnull(ss_ticket_number#81)) AND isnotnull(ss_item_sk#80)) +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] +Condition : (((((((isnotnull(ss_net_profit#85) AND isnotnull(ss_net_paid#84)) AND isnotnull(ss_quantity#83)) AND (ss_net_profit#85 > 1.00)) AND (ss_net_paid#84 > 0.00)) AND (ss_quantity#83 > 0)) AND isnotnull(ss_ticket_number#82)) AND isnotnull(ss_item_sk#81)) (62) Project [codegen id : 22] -Output [5]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_net_profit#84, ss_sold_date_sk#85] +Output [5]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_net_profit#85, ss_sold_date_sk#86] (63) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#8] +Output [1]: [d_date_sk#87] (64) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [ss_sold_date_sk#85] -Right keys [1]: [d_date_sk#8] +Left keys [1]: [ss_sold_date_sk#86] +Right keys [1]: [d_date_sk#87] Join condition: None (65) Project [codegen id : 22] -Output [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Input [6]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, ss_sold_date_sk#85, d_date_sk#8] +Output [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Input [6]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, ss_sold_date_sk#86, d_date_sk#87] (66) Exchange -Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Arguments: hashpartitioning(cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint), 5), ENSURE_REQUIREMENTS, [id=#86] +Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Arguments: hashpartitioning(cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint), 5), ENSURE_REQUIREMENTS, [id=#88] (67) Sort [codegen id : 23] -Input [4]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83] -Arguments: [cast(ss_ticket_number#81 as bigint) ASC NULLS FIRST, cast(ss_item_sk#80 as bigint) ASC NULLS FIRST], false, 0 +Input [4]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84] +Arguments: [cast(ss_ticket_number#82 as bigint) ASC NULLS FIRST, cast(ss_item_sk#81 as bigint) ASC NULLS FIRST], false, 0 (68) Scan parquet default.store_returns -Output [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Output [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (69) ColumnarToRow [codegen id : 24] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] (70) Filter [codegen id : 24] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] -Condition : (((isnotnull(sr_return_amt#90) AND (sr_return_amt#90 > 10000.00)) AND isnotnull(sr_ticket_number#88)) AND isnotnull(sr_item_sk#87)) +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] +Condition : (((isnotnull(sr_return_amt#92) AND (sr_return_amt#92 > 10000.00)) AND isnotnull(sr_ticket_number#90)) AND isnotnull(sr_item_sk#89)) (71) Project [codegen id : 24] -Output [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Input [5]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90, sr_returned_date_sk#91] +Output [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Input [5]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92, sr_returned_date_sk#93] (72) Exchange -Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Arguments: hashpartitioning(sr_ticket_number#88, sr_item_sk#87, 5), ENSURE_REQUIREMENTS, [id=#92] +Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Arguments: hashpartitioning(sr_ticket_number#90, sr_item_sk#89, 5), ENSURE_REQUIREMENTS, [id=#94] (73) Sort [codegen id : 25] -Input [4]: [sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] -Arguments: [sr_ticket_number#88 ASC NULLS FIRST, sr_item_sk#87 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] +Arguments: [sr_ticket_number#90 ASC NULLS FIRST, sr_item_sk#89 ASC NULLS FIRST], false, 0 (74) SortMergeJoin [codegen id : 26] -Left keys [2]: [cast(ss_ticket_number#81 as bigint), cast(ss_item_sk#80 as bigint)] -Right keys [2]: [sr_ticket_number#88, sr_item_sk#87] +Left keys [2]: [cast(ss_ticket_number#82 as bigint), cast(ss_item_sk#81 as bigint)] +Right keys [2]: [sr_ticket_number#90, sr_item_sk#89] Join condition: None (75) Project [codegen id : 26] -Output [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] -Input [8]: [ss_item_sk#80, ss_ticket_number#81, ss_quantity#82, ss_net_paid#83, sr_item_sk#87, sr_ticket_number#88, sr_return_quantity#89, sr_return_amt#90] +Output [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] +Input [8]: [ss_item_sk#81, ss_ticket_number#82, ss_quantity#83, ss_net_paid#84, sr_item_sk#89, sr_ticket_number#90, sr_return_quantity#91, sr_return_amt#92] (76) HashAggregate [codegen id : 26] -Input [5]: [ss_item_sk#80, ss_quantity#82, ss_net_paid#83, sr_return_quantity#89, sr_return_amt#90] -Keys [1]: [ss_item_sk#80] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#89, 0)), partial_sum(coalesce(ss_quantity#82, 0)), partial_sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#93, sum#94, sum#95, isEmpty#96, sum#97, isEmpty#98] -Results [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Input [5]: [ss_item_sk#81, ss_quantity#83, ss_net_paid#84, sr_return_quantity#91, sr_return_amt#92] +Keys [1]: [ss_item_sk#81] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#91, 0)), partial_sum(coalesce(ss_quantity#83, 0)), partial_sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#95, sum#96, sum#97, isEmpty#98, sum#99, isEmpty#100] +Results [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] (77) Exchange -Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Arguments: hashpartitioning(ss_item_sk#80, 5), ENSURE_REQUIREMENTS, [id=#105] +Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] +Arguments: hashpartitioning(ss_item_sk#81, 5), ENSURE_REQUIREMENTS, [id=#107] (78) HashAggregate [codegen id : 27] -Input [7]: [ss_item_sk#80, sum#99, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Keys [1]: [ss_item_sk#80] -Functions [4]: [sum(coalesce(sr_return_quantity#89, 0)), sum(coalesce(ss_quantity#82, 0)), sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#89, 0))#106, sum(coalesce(ss_quantity#82, 0))#107, sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108, sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109] -Results [3]: [ss_item_sk#80 AS item#110, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#89, 0))#106 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#82, 0))#107 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#111, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#90 as decimal(12,2)), 0.00))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#83 as decimal(12,2)), 0.00))#109 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#112] +Input [7]: [ss_item_sk#81, sum#101, sum#102, sum#103, isEmpty#104, sum#105, isEmpty#106] +Keys [1]: [ss_item_sk#81] +Functions [4]: [sum(coalesce(sr_return_quantity#91, 0)), sum(coalesce(ss_quantity#83, 0)), sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#91, 0))#108, sum(coalesce(ss_quantity#83, 0))#109, sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110, sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111] +Results [3]: [ss_item_sk#81 AS item#112, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#91, 0))#108 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#83, 0))#109 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#113, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#92 as decimal(12,2)), 0.00))#110 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#84 as decimal(12,2)), 0.00))#111 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#114] (79) Exchange -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#113] +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#115] (80) Sort [codegen id : 28] -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: [return_ratio#111 ASC NULLS FIRST], false, 0 +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: [return_ratio#113 ASC NULLS FIRST], false, 0 (81) Window -Input [3]: [item#110, return_ratio#111, currency_ratio#112] -Arguments: [rank(return_ratio#111) windowspecdefinition(return_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#114], [return_ratio#111 ASC NULLS FIRST] +Input [3]: [item#112, return_ratio#113, currency_ratio#114] +Arguments: [rank(return_ratio#113) windowspecdefinition(return_ratio#113 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#116], [return_ratio#113 ASC NULLS FIRST] (82) Sort [codegen id : 29] -Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] -Arguments: [currency_ratio#112 ASC NULLS FIRST], false, 0 +Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] +Arguments: [currency_ratio#114 ASC NULLS FIRST], false, 0 (83) Window -Input [4]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114] -Arguments: [rank(currency_ratio#112) windowspecdefinition(currency_ratio#112 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#115], [currency_ratio#112 ASC NULLS FIRST] +Input [4]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116] +Arguments: [rank(currency_ratio#114) windowspecdefinition(currency_ratio#114 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#117], [currency_ratio#114 ASC NULLS FIRST] (84) Filter [codegen id : 30] -Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] -Condition : ((return_rank#114 <= 10) OR (currency_rank#115 <= 10)) +Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] +Condition : ((return_rank#116 <= 10) OR (currency_rank#117 <= 10)) (85) Project [codegen id : 30] -Output [5]: [store AS channel#116, item#110, return_ratio#111, return_rank#114, currency_rank#115] -Input [5]: [item#110, return_ratio#111, currency_ratio#112, return_rank#114, currency_rank#115] +Output [5]: [store AS channel#118, item#112, return_ratio#113, return_rank#116, currency_rank#117] +Input [5]: [item#112, return_ratio#113, currency_ratio#114, return_rank#116, currency_rank#117] (86) Union @@ -481,7 +481,7 @@ Results [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_ran (88) Exchange Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] -Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#117] +Arguments: hashpartitioning(channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41, 5), ENSURE_REQUIREMENTS, [id=#119] (89) HashAggregate [codegen id : 32] Input [5]: [channel#42, item#36, return_ratio#37, return_rank#40, currency_rank#41] @@ -505,6 +505,6 @@ Output [1]: [d_date_sk#8] Subquery:2 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#48 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#85 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 59 Hosting Expression = ss_sold_date_sk#86 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index ada8c0415c672..74b789e4a7a65 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -260,170 +260,170 @@ Output [6]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, Input [9]: [cs_item_sk#42, cs_order_number#43, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_item_sk#49, cr_order_number#50, cr_return_quantity#51, cr_return_amount#52] (40) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#54] (41) BroadcastHashJoin [codegen id : 10] Left keys [1]: [cs_sold_date_sk#47] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#54] Join condition: None (42) Project [codegen id : 10] Output [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] -Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#14] +Input [7]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cs_sold_date_sk#47, cr_return_quantity#51, cr_return_amount#52, d_date_sk#54] (43) HashAggregate [codegen id : 10] Input [5]: [cs_item_sk#42, cs_quantity#44, cs_net_paid#45, cr_return_quantity#51, cr_return_amount#52] Keys [1]: [cs_item_sk#42] Functions [4]: [partial_sum(coalesce(cr_return_quantity#51, 0)), partial_sum(coalesce(cs_quantity#44, 0)), partial_sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Results [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] +Aggregate Attributes [6]: [sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Results [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] (44) Exchange -Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] -Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#66] +Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Arguments: hashpartitioning(cs_item_sk#42, 5), ENSURE_REQUIREMENTS, [id=#67] (45) HashAggregate [codegen id : 11] -Input [7]: [cs_item_sk#42, sum#60, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65] +Input [7]: [cs_item_sk#42, sum#61, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] Keys [1]: [cs_item_sk#42] Functions [4]: [sum(coalesce(cr_return_quantity#51, 0)), sum(coalesce(cs_quantity#44, 0)), sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#67, sum(coalesce(cs_quantity#44, 0))#68, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70] -Results [3]: [cs_item_sk#42 AS item#71, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#67 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#68 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#72, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#69 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#70 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#73] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#51, 0))#68, sum(coalesce(cs_quantity#44, 0))#69, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70, sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71] +Results [3]: [cs_item_sk#42 AS item#72, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#51, 0))#68 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#44, 0))#69 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#73, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#70 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#45 as decimal(12,2)), 0.00))#71 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#74] (46) Exchange -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#74] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#75] (47) Sort [codegen id : 12] -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: [return_ratio#72 ASC NULLS FIRST], false, 0 +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [return_ratio#73 ASC NULLS FIRST], false, 0 (48) Window -Input [3]: [item#71, return_ratio#72, currency_ratio#73] -Arguments: [rank(return_ratio#72) windowspecdefinition(return_ratio#72 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#72 ASC NULLS FIRST] +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#76], [return_ratio#73 ASC NULLS FIRST] (49) Sort [codegen id : 13] -Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] -Arguments: [currency_ratio#73 ASC NULLS FIRST], false, 0 +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 (50) Window -Input [4]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75] -Arguments: [rank(currency_ratio#73) windowspecdefinition(currency_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#73 ASC NULLS FIRST] +Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#77], [currency_ratio#74 ASC NULLS FIRST] (51) Filter [codegen id : 14] -Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] -Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] +Condition : ((return_rank#76 <= 10) OR (currency_rank#77 <= 10)) (52) Project [codegen id : 14] -Output [5]: [catalog AS channel#77, item#71, return_ratio#72, return_rank#75, currency_rank#76] -Input [5]: [item#71, return_ratio#72, currency_ratio#73, return_rank#75, currency_rank#76] +Output [5]: [catalog AS channel#78, item#72, return_ratio#73, return_rank#76, currency_rank#77] +Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#76, currency_rank#77] (53) Scan parquet default.store_sales -Output [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Output [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#83), dynamicpruningexpression(ss_sold_date_sk#83 IN dynamicpruning#7)] +PartitionFilters: [isnotnull(ss_sold_date_sk#84), dynamicpruningexpression(ss_sold_date_sk#84 IN dynamicpruning#7)] PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct (54) ColumnarToRow [codegen id : 15] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] (55) Filter [codegen id : 15] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] -Condition : (((((((isnotnull(ss_net_profit#82) AND isnotnull(ss_net_paid#81)) AND isnotnull(ss_quantity#80)) AND (ss_net_profit#82 > 1.00)) AND (ss_net_paid#81 > 0.00)) AND (ss_quantity#80 > 0)) AND isnotnull(ss_ticket_number#79)) AND isnotnull(ss_item_sk#78)) +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] +Condition : (((((((isnotnull(ss_net_profit#83) AND isnotnull(ss_net_paid#82)) AND isnotnull(ss_quantity#81)) AND (ss_net_profit#83 > 1.00)) AND (ss_net_paid#82 > 0.00)) AND (ss_quantity#81 > 0)) AND isnotnull(ss_ticket_number#80)) AND isnotnull(ss_item_sk#79)) (56) Project [codegen id : 15] -Output [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] -Input [6]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_net_profit#82, ss_sold_date_sk#83] +Output [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] +Input [6]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_net_profit#83, ss_sold_date_sk#84] (57) BroadcastExchange -Input [5]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#84] +Input [5]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint), cast(input[0, int, true] as bigint)),false), [id=#85] (58) Scan parquet default.store_returns -Output [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Output [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (59) ColumnarToRow -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] (60) Filter -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] -Condition : (((isnotnull(sr_return_amt#88) AND (sr_return_amt#88 > 10000.00)) AND isnotnull(sr_ticket_number#86)) AND isnotnull(sr_item_sk#85)) +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] +Condition : (((isnotnull(sr_return_amt#89) AND (sr_return_amt#89 > 10000.00)) AND isnotnull(sr_ticket_number#87)) AND isnotnull(sr_item_sk#86)) (61) Project -Output [4]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] -Input [5]: [sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88, sr_returned_date_sk#89] +Output [4]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] +Input [5]: [sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89, sr_returned_date_sk#90] (62) BroadcastHashJoin [codegen id : 17] -Left keys [2]: [cast(ss_ticket_number#79 as bigint), cast(ss_item_sk#78 as bigint)] -Right keys [2]: [sr_ticket_number#86, sr_item_sk#85] +Left keys [2]: [cast(ss_ticket_number#80 as bigint), cast(ss_item_sk#79 as bigint)] +Right keys [2]: [sr_ticket_number#87, sr_item_sk#86] Join condition: None (63) Project [codegen id : 17] -Output [6]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88] -Input [9]: [ss_item_sk#78, ss_ticket_number#79, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_item_sk#85, sr_ticket_number#86, sr_return_quantity#87, sr_return_amt#88] +Output [6]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89] +Input [9]: [ss_item_sk#79, ss_ticket_number#80, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_item_sk#86, sr_ticket_number#87, sr_return_quantity#88, sr_return_amt#89] (64) ReusedExchange [Reuses operator id: 16] -Output [1]: [d_date_sk#14] +Output [1]: [d_date_sk#91] (65) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ss_sold_date_sk#83] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ss_sold_date_sk#84] +Right keys [1]: [d_date_sk#91] Join condition: None (66) Project [codegen id : 17] -Output [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] -Input [7]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, ss_sold_date_sk#83, sr_return_quantity#87, sr_return_amt#88, d_date_sk#14] +Output [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] +Input [7]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, ss_sold_date_sk#84, sr_return_quantity#88, sr_return_amt#89, d_date_sk#91] (67) HashAggregate [codegen id : 17] -Input [5]: [ss_item_sk#78, ss_quantity#80, ss_net_paid#81, sr_return_quantity#87, sr_return_amt#88] -Keys [1]: [ss_item_sk#78] -Functions [4]: [partial_sum(coalesce(sr_return_quantity#87, 0)), partial_sum(coalesce(ss_quantity#80, 0)), partial_sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] -Aggregate Attributes [6]: [sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] -Results [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Input [5]: [ss_item_sk#79, ss_quantity#81, ss_net_paid#82, sr_return_quantity#88, sr_return_amt#89] +Keys [1]: [ss_item_sk#79] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#88, 0)), partial_sum(coalesce(ss_quantity#81, 0)), partial_sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#92, sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] +Results [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] (68) Exchange -Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Arguments: hashpartitioning(ss_item_sk#78, 5), ENSURE_REQUIREMENTS, [id=#102] +Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Arguments: hashpartitioning(ss_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#104] (69) HashAggregate [codegen id : 18] -Input [7]: [ss_item_sk#78, sum#96, sum#97, sum#98, isEmpty#99, sum#100, isEmpty#101] -Keys [1]: [ss_item_sk#78] -Functions [4]: [sum(coalesce(sr_return_quantity#87, 0)), sum(coalesce(ss_quantity#80, 0)), sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))] -Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#87, 0))#103, sum(coalesce(ss_quantity#80, 0))#104, sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105, sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106] -Results [3]: [ss_item_sk#78 AS item#107, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#87, 0))#103 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#80, 0))#104 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#108, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#88 as decimal(12,2)), 0.00))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#81 as decimal(12,2)), 0.00))#106 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#109] +Input [7]: [ss_item_sk#79, sum#98, sum#99, sum#100, isEmpty#101, sum#102, isEmpty#103] +Keys [1]: [ss_item_sk#79] +Functions [4]: [sum(coalesce(sr_return_quantity#88, 0)), sum(coalesce(ss_quantity#81, 0)), sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#88, 0))#105, sum(coalesce(ss_quantity#81, 0))#106, sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107, sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108] +Results [3]: [ss_item_sk#79 AS item#109, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#88, 0))#105 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#81, 0))#106 as decimal(15,4)))), DecimalType(35,20), true) AS return_ratio#110, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#89 as decimal(12,2)), 0.00))#107 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#82 as decimal(12,2)), 0.00))#108 as decimal(15,4)))), DecimalType(35,20), true) AS currency_ratio#111] (70) Exchange -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#110] +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#112] (71) Sort [codegen id : 19] -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: [return_ratio#108 ASC NULLS FIRST], false, 0 +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: [return_ratio#110 ASC NULLS FIRST], false, 0 (72) Window -Input [3]: [item#107, return_ratio#108, currency_ratio#109] -Arguments: [rank(return_ratio#108) windowspecdefinition(return_ratio#108 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#111], [return_ratio#108 ASC NULLS FIRST] +Input [3]: [item#109, return_ratio#110, currency_ratio#111] +Arguments: [rank(return_ratio#110) windowspecdefinition(return_ratio#110 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#113], [return_ratio#110 ASC NULLS FIRST] (73) Sort [codegen id : 20] -Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] -Arguments: [currency_ratio#109 ASC NULLS FIRST], false, 0 +Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] +Arguments: [currency_ratio#111 ASC NULLS FIRST], false, 0 (74) Window -Input [4]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111] -Arguments: [rank(currency_ratio#109) windowspecdefinition(currency_ratio#109 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#112], [currency_ratio#109 ASC NULLS FIRST] +Input [4]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113] +Arguments: [rank(currency_ratio#111) windowspecdefinition(currency_ratio#111 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#114], [currency_ratio#111 ASC NULLS FIRST] (75) Filter [codegen id : 21] -Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] -Condition : ((return_rank#111 <= 10) OR (currency_rank#112 <= 10)) +Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] +Condition : ((return_rank#113 <= 10) OR (currency_rank#114 <= 10)) (76) Project [codegen id : 21] -Output [5]: [store AS channel#113, item#107, return_ratio#108, return_rank#111, currency_rank#112] -Input [5]: [item#107, return_ratio#108, currency_ratio#109, return_rank#111, currency_rank#112] +Output [5]: [store AS channel#115, item#109, return_ratio#110, return_rank#113, currency_rank#114] +Input [5]: [item#109, return_ratio#110, currency_ratio#111, return_rank#113, currency_rank#114] (77) Union @@ -436,7 +436,7 @@ Results [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_ran (79) Exchange Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] -Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#114] +Arguments: hashpartitioning(channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40, 5), ENSURE_REQUIREMENTS, [id=#116] (80) HashAggregate [codegen id : 23] Input [5]: [channel#41, item#35, return_ratio#36, return_rank#39, currency_rank#40] @@ -460,6 +460,6 @@ Output [1]: [d_date_sk#14] Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#47 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#83 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 53 Hosting Expression = ss_sold_date_sk#84 IN dynamicpruning#7 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt index bcf47f68f2e59..0a8fdb66edb77 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 @@ -164,252 +164,252 @@ Input [4]: [item_sk#13, d_date#6, sumws#14, rk#16] Arguments: [item_sk#13 ASC NULLS FIRST], false, 0 (20) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] +Output [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] (21) Sort [codegen id : 10] -Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] -Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] +Arguments: [ws_item_sk#21 ASC NULLS FIRST, d_date#19 ASC NULLS FIRST], false, 0 (22) Window -Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#21], [ws_item_sk#1], [d_date#19 ASC NULLS FIRST] +Input [4]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21] +Arguments: [row_number() windowspecdefinition(ws_item_sk#21, d_date#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#22], [ws_item_sk#21], [d_date#19 ASC NULLS FIRST] (23) Project [codegen id : 11] -Output [3]: [item_sk#18, sumws#20, rk#21] -Input [5]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#1, rk#21] +Output [3]: [item_sk#18, sumws#20, rk#22] +Input [5]: [item_sk#18, d_date#19, sumws#20, ws_item_sk#21, rk#22] (24) Exchange -Input [3]: [item_sk#18, sumws#20, rk#21] -Arguments: hashpartitioning(item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#22] +Input [3]: [item_sk#18, sumws#20, rk#22] +Arguments: hashpartitioning(item_sk#18, 5), ENSURE_REQUIREMENTS, [id=#23] (25) Sort [codegen id : 12] -Input [3]: [item_sk#18, sumws#20, rk#21] +Input [3]: [item_sk#18, sumws#20, rk#22] Arguments: [item_sk#18 ASC NULLS FIRST], false, 0 (26) SortMergeJoin [codegen id : 13] Left keys [1]: [item_sk#13] Right keys [1]: [item_sk#18] -Join condition: (rk#16 >= rk#21) +Join condition: (rk#16 >= rk#22) (27) Project [codegen id : 13] Output [4]: [item_sk#13, d_date#6, sumws#14, sumws#20] -Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#18, sumws#20, rk#21] +Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#18, sumws#20, rk#22] (28) HashAggregate [codegen id : 13] Input [4]: [item_sk#13, d_date#6, sumws#14, sumws#20] Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [partial_sum(sumws#20)] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +Aggregate Attributes [2]: [sum#24, isEmpty#25] +Results [5]: [item_sk#13, d_date#6, sumws#14, sum#26, isEmpty#27] (29) HashAggregate [codegen id : 13] -Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +Input [5]: [item_sk#13, d_date#6, sumws#14, sum#26, isEmpty#27] Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [sum(sumws#20)] -Aggregate Attributes [1]: [sum(sumws#20)#27] -Results [3]: [item_sk#13, d_date#6, sum(sumws#20)#27 AS cume_sales#28] +Aggregate Attributes [1]: [sum(sumws#20)#28] +Results [3]: [item_sk#13, d_date#6, sum(sumws#20)#28 AS cume_sales#29] (30) Exchange -Input [3]: [item_sk#13, d_date#6, cume_sales#28] -Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [item_sk#13, d_date#6, cume_sales#29] +Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#30] (31) Sort [codegen id : 14] -Input [3]: [item_sk#13, d_date#6, cume_sales#28] +Input [3]: [item_sk#13, d_date#6, cume_sales#29] Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 (32) Scan parquet default.store_sales -Output [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +Output [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#33), dynamicpruningexpression(ss_sold_date_sk#33 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (33) ColumnarToRow [codegen id : 16] -Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] (34) Filter [codegen id : 16] -Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] -Condition : isnotnull(ss_item_sk#30) +Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] +Condition : isnotnull(ss_item_sk#31) (35) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#33, d_date#34] +Output [2]: [d_date_sk#34, d_date#35] (36) BroadcastHashJoin [codegen id : 16] -Left keys [1]: [ss_sold_date_sk#32] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [ss_sold_date_sk#33] +Right keys [1]: [d_date_sk#34] Join condition: None (37) Project [codegen id : 16] -Output [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] -Input [5]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32, d_date_sk#33, d_date#34] +Output [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] +Input [5]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33, d_date_sk#34, d_date#35] (38) HashAggregate [codegen id : 16] -Input [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] -Keys [2]: [ss_item_sk#30, d_date#34] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum#35] -Results [3]: [ss_item_sk#30, d_date#34, sum#36] +Input [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] +Keys [2]: [ss_item_sk#31, d_date#35] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#32))] +Aggregate Attributes [1]: [sum#36] +Results [3]: [ss_item_sk#31, d_date#35, sum#37] (39) Exchange -Input [3]: [ss_item_sk#30, d_date#34, sum#36] -Arguments: hashpartitioning(ss_item_sk#30, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [ss_item_sk#31, d_date#35, sum#37] +Arguments: hashpartitioning(ss_item_sk#31, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#38] (40) HashAggregate [codegen id : 17] -Input [3]: [ss_item_sk#30, d_date#34, sum#36] -Keys [2]: [ss_item_sk#30, d_date#34] -Functions [1]: [sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#31))#38] -Results [4]: [ss_item_sk#30 AS item_sk#39, d_date#34, MakeDecimal(sum(UnscaledValue(ss_sales_price#31))#38,17,2) AS sumss#40, ss_item_sk#30] +Input [3]: [ss_item_sk#31, d_date#35, sum#37] +Keys [2]: [ss_item_sk#31, d_date#35] +Functions [1]: [sum(UnscaledValue(ss_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#32))#39] +Results [4]: [ss_item_sk#31 AS item_sk#40, d_date#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#32))#39,17,2) AS sumss#41, ss_item_sk#31] (41) Exchange -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: hashpartitioning(ss_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#42] (42) Sort [codegen id : 18] -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 (43) Window -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#42], [ss_item_sk#30], [d_date#34 ASC NULLS FIRST] +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: [row_number() windowspecdefinition(ss_item_sk#31, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [ss_item_sk#31], [d_date#35 ASC NULLS FIRST] (44) Project [codegen id : 19] -Output [4]: [item_sk#39, d_date#34, sumss#40, rk#42] -Input [5]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30, rk#42] +Output [4]: [item_sk#40, d_date#35, sumss#41, rk#43] +Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] (45) Exchange -Input [4]: [item_sk#39, d_date#34, sumss#40, rk#42] -Arguments: hashpartitioning(item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#43] +Input [4]: [item_sk#40, d_date#35, sumss#41, rk#43] +Arguments: hashpartitioning(item_sk#40, 5), ENSURE_REQUIREMENTS, [id=#44] (46) Sort [codegen id : 20] -Input [4]: [item_sk#39, d_date#34, sumss#40, rk#42] -Arguments: [item_sk#39 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#40, d_date#35, sumss#41, rk#43] +Arguments: [item_sk#40 ASC NULLS FIRST], false, 0 (47) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30] +Output [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] (48) Sort [codegen id : 24] -Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30] -Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] +Arguments: [ss_item_sk#48 ASC NULLS FIRST, d_date#46 ASC NULLS FIRST], false, 0 (49) Window -Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30] -Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#47], [ss_item_sk#30], [d_date#45 ASC NULLS FIRST] +Input [4]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48] +Arguments: [row_number() windowspecdefinition(ss_item_sk#48, d_date#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [ss_item_sk#48], [d_date#46 ASC NULLS FIRST] (50) Project [codegen id : 25] -Output [3]: [item_sk#44, sumss#46, rk#47] -Input [5]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#30, rk#47] +Output [3]: [item_sk#45, sumss#47, rk#49] +Input [5]: [item_sk#45, d_date#46, sumss#47, ss_item_sk#48, rk#49] (51) Exchange -Input [3]: [item_sk#44, sumss#46, rk#47] -Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [3]: [item_sk#45, sumss#47, rk#49] +Arguments: hashpartitioning(item_sk#45, 5), ENSURE_REQUIREMENTS, [id=#50] (52) Sort [codegen id : 26] -Input [3]: [item_sk#44, sumss#46, rk#47] -Arguments: [item_sk#44 ASC NULLS FIRST], false, 0 +Input [3]: [item_sk#45, sumss#47, rk#49] +Arguments: [item_sk#45 ASC NULLS FIRST], false, 0 (53) SortMergeJoin [codegen id : 27] -Left keys [1]: [item_sk#39] -Right keys [1]: [item_sk#44] -Join condition: (rk#42 >= rk#47) +Left keys [1]: [item_sk#40] +Right keys [1]: [item_sk#45] +Join condition: (rk#43 >= rk#49) (54) Project [codegen id : 27] -Output [4]: [item_sk#39, d_date#34, sumss#40, sumss#46] -Input [7]: [item_sk#39, d_date#34, sumss#40, rk#42, item_sk#44, sumss#46, rk#47] +Output [4]: [item_sk#40, d_date#35, sumss#41, sumss#47] +Input [7]: [item_sk#40, d_date#35, sumss#41, rk#43, item_sk#45, sumss#47, rk#49] (55) HashAggregate [codegen id : 27] -Input [4]: [item_sk#39, d_date#34, sumss#40, sumss#46] -Keys [3]: [item_sk#39, d_date#34, sumss#40] -Functions [1]: [partial_sum(sumss#46)] -Aggregate Attributes [2]: [sum#49, isEmpty#50] -Results [5]: [item_sk#39, d_date#34, sumss#40, sum#51, isEmpty#52] +Input [4]: [item_sk#40, d_date#35, sumss#41, sumss#47] +Keys [3]: [item_sk#40, d_date#35, sumss#41] +Functions [1]: [partial_sum(sumss#47)] +Aggregate Attributes [2]: [sum#51, isEmpty#52] +Results [5]: [item_sk#40, d_date#35, sumss#41, sum#53, isEmpty#54] (56) HashAggregate [codegen id : 27] -Input [5]: [item_sk#39, d_date#34, sumss#40, sum#51, isEmpty#52] -Keys [3]: [item_sk#39, d_date#34, sumss#40] -Functions [1]: [sum(sumss#46)] -Aggregate Attributes [1]: [sum(sumss#46)#53] -Results [3]: [item_sk#39, d_date#34, sum(sumss#46)#53 AS cume_sales#54] +Input [5]: [item_sk#40, d_date#35, sumss#41, sum#53, isEmpty#54] +Keys [3]: [item_sk#40, d_date#35, sumss#41] +Functions [1]: [sum(sumss#47)] +Aggregate Attributes [1]: [sum(sumss#47)#55] +Results [3]: [item_sk#40, d_date#35, sum(sumss#47)#55 AS cume_sales#56] (57) Exchange -Input [3]: [item_sk#39, d_date#34, cume_sales#54] -Arguments: hashpartitioning(item_sk#39, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [3]: [item_sk#40, d_date#35, cume_sales#56] +Arguments: hashpartitioning(item_sk#40, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#57] (58) Sort [codegen id : 28] -Input [3]: [item_sk#39, d_date#34, cume_sales#54] -Arguments: [item_sk#39 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 +Input [3]: [item_sk#40, d_date#35, cume_sales#56] +Arguments: [item_sk#40 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 (59) SortMergeJoin Left keys [2]: [item_sk#13, d_date#6] -Right keys [2]: [item_sk#39, d_date#34] +Right keys [2]: [item_sk#40, d_date#35] Join condition: None (60) Filter [codegen id : 29] -Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END) +Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END) (61) Project [codegen id : 29] -Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END AS item_sk#56, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#34 END AS d_date#57, cume_sales#28 AS web_sales#58, cume_sales#54 AS store_sales#59] -Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] +Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END AS item_sk#58, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#35 END AS d_date#59, cume_sales#29 AS web_sales#60, cume_sales#56 AS store_sales#61] +Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] (62) Exchange -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: hashpartitioning(item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: hashpartitioning(item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#62] (63) Sort [codegen id : 30] -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], false, 0 (64) Window -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: [row_number() windowspecdefinition(item_sk#56, d_date#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#61], [item_sk#56], [d_date#57 ASC NULLS FIRST] +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: [row_number() windowspecdefinition(item_sk#58, d_date#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#63], [item_sk#58], [d_date#59 ASC NULLS FIRST] (65) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#62, d_date#63, web_sales#64, store_sales#65] +Output [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] (66) 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#64, d_date#65, web_sales#66, store_sales#67] +Arguments: [item_sk#64 ASC NULLS FIRST, d_date#65 ASC NULLS FIRST], false, 0 (67) 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#64, d_date#65, web_sales#66, store_sales#67] +Arguments: [row_number() windowspecdefinition(item_sk#64, d_date#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#68], [item_sk#64], [d_date#65 ASC NULLS FIRST] (68) 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#64, web_sales#66, store_sales#67, rk#68] +Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] (69) SortMergeJoin [codegen id : 62] -Left keys [1]: [item_sk#56] -Right keys [1]: [item_sk#62] -Join condition: (rk#61 >= rk#66) +Left keys [1]: [item_sk#58] +Right keys [1]: [item_sk#64] +Join condition: (rk#63 >= rk#68) (70) 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] +Output [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] +Input [9]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63, item_sk#64, web_sales#66, store_sales#67, rk#68] (71) 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] +Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] +Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Functions [2]: [partial_max(web_sales#66), partial_max(store_sales#67)] +Aggregate Attributes [2]: [max#69, max#70] +Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#71, max#72] (72) HashAggregate [codegen id : 62] -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)#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] +Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#71, max#72] +Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Functions [2]: [max(web_sales#66), max(store_sales#67)] +Aggregate Attributes [2]: [max(web_sales#66)#73, max(store_sales#67)#74] +Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max(web_sales#66)#73 AS web_cumulative#75, max(store_sales#67)#74 AS store_cumulative#76] (73) 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#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] +Condition : ((isnotnull(web_cumulative#75) AND isnotnull(store_cumulative#76)) AND (web_cumulative#75 > store_cumulative#76)) (74) 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#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] +Arguments: 100, [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#75, store_cumulative#76] ===== Subqueries ===== @@ -420,6 +420,6 @@ ReusedExchange (75) (75) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#5, d_date#6] -Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#33 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 8aaaf48a8162c..b7d3461ce34ea 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 @@ -153,248 +153,248 @@ Output [4]: [item_sk#13, d_date#6, sumws#14, rk#16] Input [5]: [item_sk#13, d_date#6, sumws#14, ws_item_sk#1, rk#16] (18) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] +Output [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] (19) Sort [codegen id : 8] -Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] -Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] +Arguments: [ws_item_sk#20 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 (20) Window -Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#20], [ws_item_sk#1], [d_date#18 ASC NULLS FIRST] +Input [4]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20] +Arguments: [row_number() windowspecdefinition(ws_item_sk#20, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#21], [ws_item_sk#20], [d_date#18 ASC NULLS FIRST] (21) Project [codegen id : 9] -Output [3]: [item_sk#17, sumws#19, rk#20] -Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#1, rk#20] +Output [3]: [item_sk#17, sumws#19, rk#21] +Input [5]: [item_sk#17, d_date#18, sumws#19, ws_item_sk#20, rk#21] (22) BroadcastExchange -Input [3]: [item_sk#17, sumws#19, rk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [3]: [item_sk#17, sumws#19, rk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] (23) BroadcastHashJoin [codegen id : 10] Left keys [1]: [item_sk#13] Right keys [1]: [item_sk#17] -Join condition: (rk#16 >= rk#20) +Join condition: (rk#16 >= rk#21) (24) Project [codegen id : 10] Output [4]: [item_sk#13, d_date#6, sumws#14, sumws#19] -Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#17, sumws#19, rk#20] +Input [7]: [item_sk#13, d_date#6, sumws#14, rk#16, item_sk#17, sumws#19, rk#21] (25) HashAggregate [codegen id : 10] Input [4]: [item_sk#13, d_date#6, sumws#14, sumws#19] Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [partial_sum(sumws#19)] -Aggregate Attributes [2]: [sum#22, isEmpty#23] -Results [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] +Aggregate Attributes [2]: [sum#23, isEmpty#24] +Results [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] (26) Exchange -Input [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] -Arguments: hashpartitioning(item_sk#13, d_date#6, sumws#14, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] +Arguments: hashpartitioning(item_sk#13, d_date#6, sumws#14, 5), ENSURE_REQUIREMENTS, [id=#27] (27) HashAggregate [codegen id : 11] -Input [5]: [item_sk#13, d_date#6, sumws#14, sum#24, isEmpty#25] +Input [5]: [item_sk#13, d_date#6, sumws#14, sum#25, isEmpty#26] Keys [3]: [item_sk#13, d_date#6, sumws#14] Functions [1]: [sum(sumws#19)] -Aggregate Attributes [1]: [sum(sumws#19)#27] -Results [3]: [item_sk#13, d_date#6, sum(sumws#19)#27 AS cume_sales#28] +Aggregate Attributes [1]: [sum(sumws#19)#28] +Results [3]: [item_sk#13, d_date#6, sum(sumws#19)#28 AS cume_sales#29] (28) Exchange -Input [3]: [item_sk#13, d_date#6, cume_sales#28] -Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [3]: [item_sk#13, d_date#6, cume_sales#29] +Arguments: hashpartitioning(item_sk#13, d_date#6, 5), ENSURE_REQUIREMENTS, [id=#30] (29) Sort [codegen id : 12] -Input [3]: [item_sk#13, d_date#6, cume_sales#28] +Input [3]: [item_sk#13, d_date#6, cume_sales#29] Arguments: [item_sk#13 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 (30) Scan parquet default.store_sales -Output [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +Output [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#33), dynamicpruningexpression(ss_sold_date_sk#33 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (31) ColumnarToRow [codegen id : 14] -Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] +Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] (32) Filter [codegen id : 14] -Input [3]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32] -Condition : isnotnull(ss_item_sk#30) +Input [3]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33] +Condition : isnotnull(ss_item_sk#31) (33) ReusedExchange [Reuses operator id: 8] -Output [2]: [d_date_sk#33, d_date#34] +Output [2]: [d_date_sk#34, d_date#35] (34) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ss_sold_date_sk#32] -Right keys [1]: [d_date_sk#33] +Left keys [1]: [ss_sold_date_sk#33] +Right keys [1]: [d_date_sk#34] Join condition: None (35) Project [codegen id : 14] -Output [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] -Input [5]: [ss_item_sk#30, ss_sales_price#31, ss_sold_date_sk#32, d_date_sk#33, d_date#34] +Output [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] +Input [5]: [ss_item_sk#31, ss_sales_price#32, ss_sold_date_sk#33, d_date_sk#34, d_date#35] (36) HashAggregate [codegen id : 14] -Input [3]: [ss_item_sk#30, ss_sales_price#31, d_date#34] -Keys [2]: [ss_item_sk#30, d_date#34] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum#35] -Results [3]: [ss_item_sk#30, d_date#34, sum#36] +Input [3]: [ss_item_sk#31, ss_sales_price#32, d_date#35] +Keys [2]: [ss_item_sk#31, d_date#35] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#32))] +Aggregate Attributes [1]: [sum#36] +Results [3]: [ss_item_sk#31, d_date#35, sum#37] (37) Exchange -Input [3]: [ss_item_sk#30, d_date#34, sum#36] -Arguments: hashpartitioning(ss_item_sk#30, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [ss_item_sk#31, d_date#35, sum#37] +Arguments: hashpartitioning(ss_item_sk#31, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#38] (38) HashAggregate [codegen id : 15] -Input [3]: [ss_item_sk#30, d_date#34, sum#36] -Keys [2]: [ss_item_sk#30, d_date#34] -Functions [1]: [sum(UnscaledValue(ss_sales_price#31))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#31))#38] -Results [4]: [ss_item_sk#30 AS item_sk#39, d_date#34, MakeDecimal(sum(UnscaledValue(ss_sales_price#31))#38,17,2) AS sumss#40, ss_item_sk#30] +Input [3]: [ss_item_sk#31, d_date#35, sum#37] +Keys [2]: [ss_item_sk#31, d_date#35] +Functions [1]: [sum(UnscaledValue(ss_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#32))#39] +Results [4]: [ss_item_sk#31 AS item_sk#40, d_date#35, MakeDecimal(sum(UnscaledValue(ss_sales_price#32))#39,17,2) AS sumss#41, ss_item_sk#31] (39) Exchange -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: hashpartitioning(ss_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#41] +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: hashpartitioning(ss_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#42] (40) Sort [codegen id : 16] -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: [ss_item_sk#31 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 (41) Window -Input [4]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30] -Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#42], [ss_item_sk#30], [d_date#34 ASC NULLS FIRST] +Input [4]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31] +Arguments: [row_number() windowspecdefinition(ss_item_sk#31, d_date#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [ss_item_sk#31], [d_date#35 ASC NULLS FIRST] (42) Project [codegen id : 22] -Output [4]: [item_sk#39, d_date#34, sumss#40, rk#42] -Input [5]: [item_sk#39, d_date#34, sumss#40, ss_item_sk#30, rk#42] +Output [4]: [item_sk#40, d_date#35, sumss#41, rk#43] +Input [5]: [item_sk#40, d_date#35, sumss#41, ss_item_sk#31, rk#43] (43) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] +Output [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] (44) Sort [codegen id : 20] -Input [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] -Arguments: [ss_item_sk#30 ASC NULLS FIRST, d_date#44 ASC NULLS FIRST], false, 0 +Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] +Arguments: [ss_item_sk#47 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], false, 0 (45) Window -Input [4]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30] -Arguments: [row_number() windowspecdefinition(ss_item_sk#30, d_date#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#46], [ss_item_sk#30], [d_date#44 ASC NULLS FIRST] +Input [4]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47] +Arguments: [row_number() windowspecdefinition(ss_item_sk#47, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [ss_item_sk#47], [d_date#45 ASC NULLS FIRST] (46) Project [codegen id : 21] -Output [3]: [item_sk#43, sumss#45, rk#46] -Input [5]: [item_sk#43, d_date#44, sumss#45, ss_item_sk#30, rk#46] +Output [3]: [item_sk#44, sumss#46, rk#48] +Input [5]: [item_sk#44, d_date#45, sumss#46, ss_item_sk#47, rk#48] (47) BroadcastExchange -Input [3]: [item_sk#43, sumss#45, rk#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#47] +Input [3]: [item_sk#44, sumss#46, rk#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#49] (48) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [item_sk#39] -Right keys [1]: [item_sk#43] -Join condition: (rk#42 >= rk#46) +Left keys [1]: [item_sk#40] +Right keys [1]: [item_sk#44] +Join condition: (rk#43 >= rk#48) (49) Project [codegen id : 22] -Output [4]: [item_sk#39, d_date#34, sumss#40, sumss#45] -Input [7]: [item_sk#39, d_date#34, sumss#40, rk#42, item_sk#43, sumss#45, rk#46] +Output [4]: [item_sk#40, d_date#35, sumss#41, sumss#46] +Input [7]: [item_sk#40, d_date#35, sumss#41, rk#43, item_sk#44, sumss#46, rk#48] (50) HashAggregate [codegen id : 22] -Input [4]: [item_sk#39, d_date#34, sumss#40, sumss#45] -Keys [3]: [item_sk#39, d_date#34, sumss#40] -Functions [1]: [partial_sum(sumss#45)] -Aggregate Attributes [2]: [sum#48, isEmpty#49] -Results [5]: [item_sk#39, d_date#34, sumss#40, sum#50, isEmpty#51] +Input [4]: [item_sk#40, d_date#35, sumss#41, sumss#46] +Keys [3]: [item_sk#40, d_date#35, sumss#41] +Functions [1]: [partial_sum(sumss#46)] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] (51) Exchange -Input [5]: [item_sk#39, d_date#34, sumss#40, sum#50, isEmpty#51] -Arguments: hashpartitioning(item_sk#39, d_date#34, sumss#40, 5), ENSURE_REQUIREMENTS, [id=#52] +Input [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] +Arguments: hashpartitioning(item_sk#40, d_date#35, sumss#41, 5), ENSURE_REQUIREMENTS, [id=#54] (52) HashAggregate [codegen id : 23] -Input [5]: [item_sk#39, d_date#34, sumss#40, sum#50, isEmpty#51] -Keys [3]: [item_sk#39, d_date#34, sumss#40] -Functions [1]: [sum(sumss#45)] -Aggregate Attributes [1]: [sum(sumss#45)#53] -Results [3]: [item_sk#39, d_date#34, sum(sumss#45)#53 AS cume_sales#54] +Input [5]: [item_sk#40, d_date#35, sumss#41, sum#52, isEmpty#53] +Keys [3]: [item_sk#40, d_date#35, sumss#41] +Functions [1]: [sum(sumss#46)] +Aggregate Attributes [1]: [sum(sumss#46)#55] +Results [3]: [item_sk#40, d_date#35, sum(sumss#46)#55 AS cume_sales#56] (53) Exchange -Input [3]: [item_sk#39, d_date#34, cume_sales#54] -Arguments: hashpartitioning(item_sk#39, d_date#34, 5), ENSURE_REQUIREMENTS, [id=#55] +Input [3]: [item_sk#40, d_date#35, cume_sales#56] +Arguments: hashpartitioning(item_sk#40, d_date#35, 5), ENSURE_REQUIREMENTS, [id=#57] (54) Sort [codegen id : 24] -Input [3]: [item_sk#39, d_date#34, cume_sales#54] -Arguments: [item_sk#39 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST], false, 0 +Input [3]: [item_sk#40, d_date#35, cume_sales#56] +Arguments: [item_sk#40 ASC NULLS FIRST, d_date#35 ASC NULLS FIRST], false, 0 (55) SortMergeJoin Left keys [2]: [item_sk#13, d_date#6] -Right keys [2]: [item_sk#39, d_date#34] +Right keys [2]: [item_sk#40, d_date#35] Join condition: None (56) Filter [codegen id : 25] -Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END) +Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END) (57) Project [codegen id : 25] -Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#39 END AS item_sk#56, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#34 END AS d_date#57, cume_sales#28 AS web_sales#58, cume_sales#54 AS store_sales#59] -Input [6]: [item_sk#13, d_date#6, cume_sales#28, item_sk#39, d_date#34, cume_sales#54] +Output [4]: [CASE WHEN isnotnull(item_sk#13) THEN item_sk#13 ELSE item_sk#40 END AS item_sk#58, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#35 END AS d_date#59, cume_sales#29 AS web_sales#60, cume_sales#56 AS store_sales#61] +Input [6]: [item_sk#13, d_date#6, cume_sales#29, item_sk#40, d_date#35, cume_sales#56] (58) Exchange -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: hashpartitioning(item_sk#56, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: hashpartitioning(item_sk#58, 5), ENSURE_REQUIREMENTS, [id=#62] (59) Sort [codegen id : 26] -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 +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], false, 0 (60) Window -Input [4]: [item_sk#56, d_date#57, web_sales#58, store_sales#59] -Arguments: [row_number() windowspecdefinition(item_sk#56, d_date#57 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#61], [item_sk#56], [d_date#57 ASC NULLS FIRST] +Input [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Arguments: [row_number() windowspecdefinition(item_sk#58, d_date#59 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#63], [item_sk#58], [d_date#59 ASC NULLS FIRST] (61) ReusedExchange [Reuses operator id: unknown] -Output [4]: [item_sk#62, d_date#63, web_sales#64, store_sales#65] +Output [4]: [item_sk#64, d_date#65, web_sales#66, store_sales#67] (62) 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#64, d_date#65, web_sales#66, store_sales#67] +Arguments: [item_sk#64 ASC NULLS FIRST, d_date#65 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#64, d_date#65, web_sales#66, store_sales#67] +Arguments: [row_number() windowspecdefinition(item_sk#64, d_date#65 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#68], [item_sk#64], [d_date#65 ASC NULLS FIRST] (64) 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#64, web_sales#66, store_sales#67, rk#68] +Input [5]: [item_sk#64, d_date#65, web_sales#66, store_sales#67, rk#68] (65) 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#64, web_sales#66, store_sales#67, rk#68] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] (66) BroadcastHashJoin [codegen id : 54] -Left keys [1]: [item_sk#56] -Right keys [1]: [item_sk#62] -Join condition: (rk#61 >= rk#66) +Left keys [1]: [item_sk#58] +Right keys [1]: [item_sk#64] +Join condition: (rk#63 >= rk#68) (67) 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] +Output [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] +Input [9]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, rk#63, item_sk#64, web_sales#66, store_sales#67, rk#68] (68) 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] +Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_sales#66, store_sales#67] +Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Functions [2]: [partial_max(web_sales#66), partial_max(store_sales#67)] +Aggregate Attributes [2]: [max#70, max#71] +Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#72, max#73] (69) HashAggregate [codegen id : 54] -Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, max#70, max#71] -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] +Input [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max#72, max#73] +Keys [4]: [item_sk#58, d_date#59, web_sales#60, store_sales#61] +Functions [2]: [max(web_sales#66), max(store_sales#67)] +Aggregate Attributes [2]: [max(web_sales#66)#74, max(store_sales#67)#75] +Results [6]: [item_sk#58, d_date#59, web_sales#60, store_sales#61, max(web_sales#66)#74 AS web_cumulative#76, max(store_sales#67)#75 AS store_cumulative#77] (70) 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#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] +Condition : ((isnotnull(web_cumulative#76) AND isnotnull(store_cumulative#77)) AND (web_cumulative#76 > store_cumulative#77)) (71) 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#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] +Arguments: 100, [item_sk#58 ASC NULLS FIRST, d_date#59 ASC NULLS FIRST], [item_sk#58, d_date#59, web_sales#60, store_sales#61, web_cumulative#76, store_cumulative#77] ===== Subqueries ===== @@ -405,6 +405,6 @@ ReusedExchange (72) (72) ReusedExchange [Reuses operator id: 8] Output [2]: [d_date_sk#5, d_date#6] -Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#33 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 ed0399519f579..e3de4a7a053b1 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 @@ -229,78 +229,78 @@ Output [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] (39) HashAggregate [codegen id : 21] Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum#34] Keys [5]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33] -Functions [1]: [sum(UnscaledValue(cs_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#3))#35] -Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#3))#35,17,2) AS sum_sales#36] +Functions [1]: [sum(UnscaledValue(cs_sales_price#35))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#35))#36] +Results [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, MakeDecimal(sum(UnscaledValue(cs_sales_price#35))#36,17,2) AS sum_sales#37] (40) Exchange -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, 5), ENSURE_REQUIREMENTS, [id=#38] (41) Sort [codegen id : 22] -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST], false, 0 (42) Window -Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36] -Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#38], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] +Input [6]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37] +Arguments: [rank(d_year#32, d_moy#33) windowspecdefinition(i_category#29, i_brand#30, cc_name#31, d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#29, i_brand#30, cc_name#31], [d_year#32 ASC NULLS FIRST, d_moy#33 ASC NULLS FIRST] (43) Project [codegen id : 23] -Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#36, rn#38] +Output [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Input [7]: [i_category#29, i_brand#30, cc_name#31, d_year#32, d_moy#33, sum_sales#37, rn#39] (44) Exchange -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#38 + 1), 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Arguments: hashpartitioning(i_category#29, i_brand#30, cc_name#31, (rn#39 + 1), 5), ENSURE_REQUIREMENTS, [id=#40] (45) Sort [codegen id : 24] -Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] -Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#38 + 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] +Arguments: [i_category#29 ASC NULLS FIRST, i_brand#30 ASC NULLS FIRST, cc_name#31 ASC NULLS FIRST, (rn#39 + 1) ASC NULLS FIRST], false, 0 (46) SortMergeJoin [codegen id : 25] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#38 + 1)] +Right keys [4]: [i_category#29, i_brand#30, cc_name#31, (rn#39 + 1)] Join condition: None (47) Project [codegen id : 25] -Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36] -Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#36, rn#38] +Output [9]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37] +Input [13]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, i_category#29, i_brand#30, cc_name#31, sum_sales#37, rn#39] (48) ReusedExchange [Reuses operator id: 40] -Output [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] +Output [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] (49) Sort [codegen id : 34] -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST], false, 0 +Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST], false, 0 (50) Window -Input [6]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45] -Arguments: [rank(d_year#43, d_moy#44) windowspecdefinition(i_category#40, i_brand#41, cc_name#42, d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#46], [i_category#40, i_brand#41, cc_name#42], [d_year#43 ASC NULLS FIRST, d_moy#44 ASC NULLS FIRST] +Input [6]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46] +Arguments: [rank(d_year#44, d_moy#45) windowspecdefinition(i_category#41, i_brand#42, cc_name#43, d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#47], [i_category#41, i_brand#42, cc_name#43], [d_year#44 ASC NULLS FIRST, d_moy#45 ASC NULLS FIRST] (51) Project [codegen id : 35] -Output [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Input [7]: [i_category#40, i_brand#41, cc_name#42, d_year#43, d_moy#44, sum_sales#45, rn#46] +Output [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Input [7]: [i_category#41, i_brand#42, cc_name#43, d_year#44, d_moy#45, sum_sales#46, rn#47] (52) Exchange -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Arguments: hashpartitioning(i_category#40, i_brand#41, cc_name#42, (rn#46 - 1), 5), ENSURE_REQUIREMENTS, [id=#47] +Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Arguments: hashpartitioning(i_category#41, i_brand#42, cc_name#43, (rn#47 - 1), 5), ENSURE_REQUIREMENTS, [id=#48] (53) Sort [codegen id : 36] -Input [5]: [i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] -Arguments: [i_category#40 ASC NULLS FIRST, i_brand#41 ASC NULLS FIRST, cc_name#42 ASC NULLS FIRST, (rn#46 - 1) ASC NULLS FIRST], false, 0 +Input [5]: [i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] +Arguments: [i_category#41 ASC NULLS FIRST, i_brand#42 ASC NULLS FIRST, cc_name#43 ASC NULLS FIRST, (rn#47 - 1) ASC NULLS FIRST], false, 0 (54) SortMergeJoin [codegen id : 37] Left keys [4]: [i_category#16, i_brand#15, cc_name#11, rn#27] -Right keys [4]: [i_category#40, i_brand#41, cc_name#42, (rn#46 - 1)] +Right keys [4]: [i_category#41, i_brand#42, cc_name#43, (rn#47 - 1)] Join condition: None (55) Project [codegen id : 37] -Output [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#36 AS psum#48, sum_sales#45 AS nsum#49] -Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#36, i_category#40, i_brand#41, cc_name#42, sum_sales#45, rn#46] +Output [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, sum_sales#37 AS psum#49, sum_sales#46 AS nsum#50] +Input [14]: [i_category#16, i_brand#15, cc_name#11, d_year#7, d_moy#8, sum_sales#22, avg_monthly_sales#25, rn#27, sum_sales#37, i_category#41, i_brand#42, cc_name#43, sum_sales#46, rn#47] (56) TakeOrderedAndProject -Input [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] -Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#22 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#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#48, nsum#49] +Input [8]: [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, 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#25 as decimal(22,6)))), DecimalType(22,6), true) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#16, i_brand#15, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#22, psum#49, nsum#50] ===== Subqueries ===== 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 c347bbbe3b4ad..a6742cf4ab1cf 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 @@ -202,70 +202,70 @@ Output [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum#32] (34) HashAggregate [codegen id : 13] 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#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#33] -Results [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#33,17,2) AS sum_sales#34] +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] (35) Exchange -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] -Arguments: hashpartitioning(i_category#27, i_brand#28, cc_name#29, 5), ENSURE_REQUIREMENTS, [id=#35] +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] (36) Sort [codegen id : 14] -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] +Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#35] 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 (37) Window -Input [6]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34] -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#36], [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#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] (38) Project [codegen id : 15] -Output [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] -Input [7]: [i_category#27, i_brand#28, cc_name#29, d_year#30, d_moy#31, sum_sales#34, rn#36] +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] (39) BroadcastExchange -Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#37] +Input [5]: [i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [id=#38] (40) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#36 + 1)] +Right keys [4]: [i_category#27, i_brand#28, cc_name#29, (rn#37 + 1)] Join condition: None (41) Project [codegen id : 23] -Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34] -Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#34, rn#36] +Output [9]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35] +Input [13]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, i_category#27, i_brand#28, cc_name#29, sum_sales#35, rn#37] (42) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] +Output [6]: [i_category#39, i_brand#40, cc_name#41, d_year#42, d_moy#43, sum_sales#44] (43) Sort [codegen id : 21] -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] -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 +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 (44) Window -Input [6]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43] -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#44], [i_category#38, i_brand#39, cc_name#40], [d_year#41 ASC NULLS FIRST, d_moy#42 ASC NULLS FIRST] +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] (45) Project [codegen id : 22] -Output [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] -Input [7]: [i_category#38, i_brand#39, cc_name#40, d_year#41, d_moy#42, sum_sales#43, rn#44] +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] (46) BroadcastExchange -Input [5]: [i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#45] +Input [5]: [i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [id=#46] (47) BroadcastHashJoin [codegen id : 23] Left keys [4]: [i_category#3, i_brand#2, cc_name#15, rn#26] -Right keys [4]: [i_category#38, i_brand#39, cc_name#40, (rn#44 - 1)] +Right keys [4]: [i_category#39, i_brand#40, cc_name#41, (rn#45 - 1)] Join condition: None (48) Project [codegen id : 23] -Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#34 AS psum#46, sum_sales#43 AS nsum#47] -Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#34, i_category#38, i_brand#39, cc_name#40, sum_sales#43, rn#44] +Output [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#21, sum_sales#35 AS psum#47, sum_sales#44 AS nsum#48] +Input [14]: [i_category#3, i_brand#2, cc_name#15, d_year#11, d_moy#12, sum_sales#21, avg_monthly_sales#24, rn#26, sum_sales#35, i_category#39, i_brand#40, cc_name#41, sum_sales#44, rn#45] (49) TakeOrderedAndProject -Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, 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#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#21, psum#46, nsum#47] +Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, 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#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#21, psum#47, nsum#48] ===== Subqueries ===== 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 5374a708a0295..ef8088e415cc5 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 @@ -278,165 +278,165 @@ Output [6]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, Input [8]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] (41) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#70] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#25] +Right keys [1]: [d_date_sk#70] Join condition: None (43) Project [codegen id : 11] Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#25] +Input [7]: [date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68, d_date_sk#70] (44) HashAggregate [codegen id : 11] Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] -Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] +Results [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] (45) Exchange -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] +Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#79] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Input [5]: [cp_catalog_page_id#68, sum#75, sum#76, sum#77, sum#78] Keys [1]: [cp_catalog_page_id#68] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] -Results [5]: [catalog channel AS channel#83, concat(catalog_page, cp_catalog_page_id#68) AS id#84, MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#85, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS returns#86, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#87] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] +Results [5]: [catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#68) AS id#85, MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS returns#87, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#88] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] -Condition : isnotnull(ws_web_site_sk#88) +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : isnotnull(ws_web_site_sk#89) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] (53) Exchange -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: hashpartitioning(wr_item_sk#98, wr_order_number#99, 5), ENSURE_REQUIREMENTS, [id=#103] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: hashpartitioning(wr_item_sk#99, wr_order_number#100, 5), ENSURE_REQUIREMENTS, [id=#104] (54) Sort [codegen id : 15] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: [wr_item_sk#98 ASC NULLS FIRST, wr_order_number#99 ASC NULLS FIRST], false, 0 +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: [wr_item_sk#99 ASC NULLS FIRST, wr_order_number#100 ASC NULLS FIRST], false, 0 (55) Scan parquet default.web_sales -Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (56) ColumnarToRow [codegen id : 16] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (57) Filter [codegen id : 16] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) (58) Project [codegen id : 16] -Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (59) Exchange -Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Arguments: hashpartitioning(cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint), 5), ENSURE_REQUIREMENTS, [id=#106] +Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Arguments: hashpartitioning(cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint), 5), ENSURE_REQUIREMENTS, [id=#109] (60) Sort [codegen id : 17] -Input [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Arguments: [cast(ws_item_sk#104 as bigint) ASC NULLS FIRST, cast(ws_order_number#105 as bigint) ASC NULLS FIRST], false, 0 +Input [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Arguments: [cast(ws_item_sk#105 as bigint) ASC NULLS FIRST, cast(ws_order_number#107 as bigint) ASC NULLS FIRST], false, 0 (61) SortMergeJoin [codegen id : 18] -Left keys [2]: [wr_item_sk#98, wr_order_number#99] -Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] +Left keys [2]: [wr_item_sk#99, wr_order_number#100] +Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] Join condition: None (62) Project [codegen id : 18] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#107, wr_returned_date_sk#102 AS date_sk#108, 0.00 AS sales_price#109, 0.00 AS profit#110, wr_return_amt#100 AS return_amt#111, wr_net_loss#101 AS net_loss#112] -Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#110, wr_returned_date_sk#103 AS date_sk#111, 0.00 AS sales_price#112, 0.00 AS profit#113, wr_return_amt#101 AS return_amt#114, wr_net_loss#102 AS net_loss#115] +Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] (63) Union (64) Scan parquet default.web_site -Output [2]: [web_site_sk#113, web_site_id#114] +Output [2]: [web_site_sk#116, web_site_id#117] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 19] -Input [2]: [web_site_sk#113, web_site_id#114] +Input [2]: [web_site_sk#116, web_site_id#117] (66) Filter [codegen id : 19] -Input [2]: [web_site_sk#113, web_site_id#114] -Condition : isnotnull(web_site_sk#113) +Input [2]: [web_site_sk#116, web_site_id#117] +Condition : isnotnull(web_site_sk#116) (67) BroadcastExchange -Input [2]: [web_site_sk#113, web_site_id#114] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#115] +Input [2]: [web_site_sk#116, web_site_id#117] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] (68) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [wsr_web_site_sk#92] -Right keys [1]: [web_site_sk#113] +Left keys [1]: [wsr_web_site_sk#93] +Right keys [1]: [web_site_sk#116] Join condition: None (69) Project [codegen id : 21] -Output [6]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Input [8]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#113, web_site_id#114] +Output [6]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [8]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] (70) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#119] (71) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [date_sk#93] -Right keys [1]: [cast(d_date_sk#25 as bigint)] +Left keys [1]: [date_sk#94] +Right keys [1]: [cast(d_date_sk#119 as bigint)] Join condition: None (72) Project [codegen id : 21] -Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Input [7]: [date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114, d_date_sk#25] +Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [7]: [date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117, d_date_sk#119] (73) HashAggregate [codegen id : 21] -Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#114] -Keys [1]: [web_site_id#114] -Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum#116, sum#117, sum#118, sum#119] -Results [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] +Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Keys [1]: [web_site_id#117] +Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum#120, sum#121, sum#122, sum#123] +Results [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] (74) Exchange -Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -Arguments: hashpartitioning(web_site_id#114, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] +Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#128] (75) HashAggregate [codegen id : 22] -Input [5]: [web_site_id#114, sum#120, sum#121, sum#122, sum#123] -Keys [1]: [web_site_id#114] -Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#125, sum(UnscaledValue(return_amt#96))#126, sum(UnscaledValue(profit#95))#127, sum(UnscaledValue(net_loss#97))#128] -Results [5]: [web channel AS channel#129, concat(web_site, web_site_id#114) AS id#130, MakeDecimal(sum(UnscaledValue(sales_price#94))#125,17,2) AS sales#131, MakeDecimal(sum(UnscaledValue(return_amt#96))#126,17,2) AS returns#132, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#127,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#128,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#133] +Input [5]: [web_site_id#117, sum#124, sum#125, sum#126, sum#127] +Keys [1]: [web_site_id#117] +Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#129, sum(UnscaledValue(return_amt#97))#130, sum(UnscaledValue(profit#96))#131, sum(UnscaledValue(net_loss#98))#132] +Results [5]: [web channel AS channel#133, concat(web_site, web_site_id#117) AS id#134, MakeDecimal(sum(UnscaledValue(sales_price#95))#129,17,2) AS sales#135, MakeDecimal(sum(UnscaledValue(return_amt#97))#130,17,2) AS returns#136, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#131,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#132,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#137] (76) Union @@ -444,99 +444,99 @@ Results [5]: [web channel AS channel#129, concat(web_site, web_site_id#114) AS i Input [5]: [channel#41, id#42, sales#43, returns#44, profit#45] Keys [2]: [channel#41, id#42] Functions [3]: [partial_sum(sales#43), partial_sum(returns#44), partial_sum(profit#45)] -Aggregate Attributes [6]: [sum#134, isEmpty#135, sum#136, isEmpty#137, sum#138, isEmpty#139] -Results [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Aggregate Attributes [6]: [sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Results [8]: [channel#41, id#42, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] (78) Exchange -Input [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#146] +Input [8]: [channel#41, id#42, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] +Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#150] (79) HashAggregate [codegen id : 24] -Input [8]: [channel#41, id#42, sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Input [8]: [channel#41, id#42, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#45)] -Aggregate Attributes [3]: [sum(sales#43)#147, sum(returns#44)#148, sum(profit#45)#149] -Results [5]: [channel#41, id#42, cast(sum(sales#43)#147 as decimal(37,2)) AS sales#150, cast(sum(returns#44)#148 as decimal(37,2)) AS returns#151, cast(sum(profit#45)#149 as decimal(38,2)) AS profit#152] +Aggregate Attributes [3]: [sum(sales#43)#151, sum(returns#44)#152, sum(profit#45)#153] +Results [5]: [channel#41, id#42, cast(sum(sales#43)#151 as decimal(37,2)) AS sales#154, cast(sum(returns#44)#152 as decimal(37,2)) AS returns#155, cast(sum(profit#45)#153 as decimal(38,2)) AS profit#156] (80) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Output [8]: [channel#41, id#42, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] (81) HashAggregate [codegen id : 48] -Input [8]: [channel#41, id#42, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] +Input [8]: [channel#41, id#42, sum#157, isEmpty#158, sum#159, isEmpty#160, sum#161, isEmpty#162] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#159)] -Aggregate Attributes [3]: [sum(sales#43)#160, sum(returns#44)#161, sum(profit#159)#162] -Results [4]: [channel#41, sum(sales#43)#160 AS sales#163, sum(returns#44)#161 AS returns#164, sum(profit#159)#162 AS profit#165] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#163)] +Aggregate Attributes [3]: [sum(sales#43)#164, sum(returns#44)#165, sum(profit#163)#166] +Results [4]: [channel#41, sum(sales#43)#164 AS sales#167, sum(returns#44)#165 AS returns#168, sum(profit#163)#166 AS profit#169] (82) HashAggregate [codegen id : 48] -Input [4]: [channel#41, sales#163, returns#164, profit#165] +Input [4]: [channel#41, sales#167, returns#168, profit#169] Keys [1]: [channel#41] -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#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] +Functions [3]: [partial_sum(sales#167), partial_sum(returns#168), partial_sum(profit#169)] +Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Results [7]: [channel#41, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] (83) Exchange -Input [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] -Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#178] +Input [7]: [channel#41, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#182] (84) HashAggregate [codegen id : 49] -Input [7]: [channel#41, sum#172, isEmpty#173, sum#174, isEmpty#175, sum#176, isEmpty#177] +Input [7]: [channel#41, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Keys [1]: [channel#41] -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#41, 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#167), sum(returns#168), sum(profit#169)] +Aggregate Attributes [3]: [sum(sales#167)#183, sum(returns#168)#184, sum(profit#169)#185] +Results [5]: [channel#41, null AS id#186, sum(sales#167)#183 AS sum(sales)#187, sum(returns#168)#184 AS sum(returns)#188, sum(profit#169)#185 AS sum(profit)#189] (85) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] +Output [8]: [channel#41, id#42, sum#190, isEmpty#191, sum#192, isEmpty#193, sum#194, isEmpty#195] (86) HashAggregate [codegen id : 73] -Input [8]: [channel#41, id#42, sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] +Input [8]: [channel#41, id#42, sum#190, isEmpty#191, sum#192, isEmpty#193, sum#194, isEmpty#195] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#192)] -Aggregate Attributes [3]: [sum(sales#43)#193, sum(returns#44)#194, sum(profit#192)#195] -Results [3]: [sum(sales#43)#193 AS sales#163, sum(returns#44)#194 AS returns#164, sum(profit#192)#195 AS profit#165] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#196)] +Aggregate Attributes [3]: [sum(sales#43)#197, sum(returns#44)#198, sum(profit#196)#199] +Results [3]: [sum(sales#43)#197 AS sales#167, sum(returns#44)#198 AS returns#168, sum(profit#196)#199 AS profit#169] (87) HashAggregate [codegen id : 73] -Input [3]: [sales#163, returns#164, profit#165] +Input [3]: [sales#167, returns#168, profit#169] Keys: [] -Functions [3]: [partial_sum(sales#163), partial_sum(returns#164), partial_sum(profit#165)] -Aggregate Attributes [6]: [sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201] -Results [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] +Functions [3]: [partial_sum(sales#167), partial_sum(returns#168), partial_sum(profit#169)] +Aggregate Attributes [6]: [sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205] +Results [6]: [sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211] (88) Exchange -Input [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#208] +Input [6]: [sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#212] (89) HashAggregate [codegen id : 74] -Input [6]: [sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207] +Input [6]: [sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211] Keys: [] -Functions [3]: [sum(sales#163), sum(returns#164), sum(profit#165)] -Aggregate Attributes [3]: [sum(sales#163)#209, sum(returns#164)#210, sum(profit#165)#211] -Results [5]: [null AS channel#212, null AS id#213, sum(sales#163)#209 AS sum(sales)#214, sum(returns#164)#210 AS sum(returns)#215, sum(profit#165)#211 AS sum(profit)#216] +Functions [3]: [sum(sales#167), sum(returns#168), sum(profit#169)] +Aggregate Attributes [3]: [sum(sales#167)#213, sum(returns#168)#214, sum(profit#169)#215] +Results [5]: [null AS channel#216, null AS id#217, sum(sales#167)#213 AS sum(sales)#218, sum(returns#168)#214 AS sum(returns)#219, sum(profit#169)#215 AS sum(profit)#220] (90) Union (91) HashAggregate [codegen id : 75] -Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -Keys [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Keys [5]: [channel#41, id#42, sales#154, returns#155, profit#156] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Results [5]: [channel#41, id#42, sales#154, returns#155, profit#156] (92) Exchange -Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -Arguments: hashpartitioning(channel#41, id#42, sales#150, returns#151, profit#152, 5), ENSURE_REQUIREMENTS, [id=#217] +Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Arguments: hashpartitioning(channel#41, id#42, sales#154, returns#155, profit#156, 5), ENSURE_REQUIREMENTS, [id=#221] (93) HashAggregate [codegen id : 76] -Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -Keys [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Keys [5]: [channel#41, id#42, sales#154, returns#155, profit#156] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#150, returns#151, profit#152] +Results [5]: [channel#41, id#42, sales#154, returns#155, profit#156] (94) TakeOrderedAndProject -Input [5]: [channel#41, id#42, sales#150, returns#151, profit#152] -Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#150, returns#151, profit#152] +Input [5]: [channel#41, id#42, sales#154, returns#155, profit#156] +Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#154, returns#155, profit#156] ===== Subqueries ===== @@ -554,12 +554,12 @@ ReusedExchange (96) (96) ReusedExchange [Reuses operator id: 20] -Output [1]: [d_date_sk#25] +Output [1]: [d_date_sk#70] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 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 89362fb4f0efe..c6b64d59abf82 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 @@ -248,180 +248,180 @@ Input [4]: [cr_catalog_page_sk#57, cr_return_amount#58, cr_net_loss#59, cr_retur (34) Union (35) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#67] (36) BroadcastHashJoin [codegen id : 11] Left keys [1]: [date_sk#52] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#67] Join condition: None (37) Project [codegen id : 11] Output [5]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56] -Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#22] +Input [7]: [page_sk#51, date_sk#52, sales_price#53, profit#54, return_amt#55, net_loss#56, d_date_sk#67] (38) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Output [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 10] -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] (40) Filter [codegen id : 10] -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] -Condition : isnotnull(cp_catalog_page_sk#67) +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Condition : isnotnull(cp_catalog_page_sk#68) (41) BroadcastExchange -Input [2]: [cp_catalog_page_sk#67, cp_catalog_page_id#68] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#69] +Input [2]: [cp_catalog_page_sk#68, cp_catalog_page_id#69] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#70] (42) BroadcastHashJoin [codegen id : 11] Left keys [1]: [page_sk#51] -Right keys [1]: [cp_catalog_page_sk#67] +Right keys [1]: [cp_catalog_page_sk#68] Join condition: None (43) Project [codegen id : 11] -Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#67, cp_catalog_page_id#68] +Output [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] +Input [7]: [page_sk#51, sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_sk#68, cp_catalog_page_id#69] (44) HashAggregate [codegen id : 11] -Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#68] -Keys [1]: [cp_catalog_page_id#68] +Input [5]: [sales_price#53, profit#54, return_amt#55, net_loss#56, cp_catalog_page_id#69] +Keys [1]: [cp_catalog_page_id#69] Functions [4]: [partial_sum(UnscaledValue(sales_price#53)), partial_sum(UnscaledValue(return_amt#55)), partial_sum(UnscaledValue(profit#54)), partial_sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum#70, sum#71, sum#72, sum#73] -Results [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] +Aggregate Attributes [4]: [sum#71, sum#72, sum#73, sum#74] +Results [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] (45) Exchange -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Arguments: hashpartitioning(cp_catalog_page_id#68, 5), ENSURE_REQUIREMENTS, [id=#78] +Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] +Arguments: hashpartitioning(cp_catalog_page_id#69, 5), ENSURE_REQUIREMENTS, [id=#79] (46) HashAggregate [codegen id : 12] -Input [5]: [cp_catalog_page_id#68, sum#74, sum#75, sum#76, sum#77] -Keys [1]: [cp_catalog_page_id#68] +Input [5]: [cp_catalog_page_id#69, sum#75, sum#76, sum#77, sum#78] +Keys [1]: [cp_catalog_page_id#69] Functions [4]: [sum(UnscaledValue(sales_price#53)), sum(UnscaledValue(return_amt#55)), sum(UnscaledValue(profit#54)), sum(UnscaledValue(net_loss#56))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#79, sum(UnscaledValue(return_amt#55))#80, sum(UnscaledValue(profit#54))#81, sum(UnscaledValue(net_loss#56))#82] -Results [5]: [catalog channel AS channel#83, concat(catalog_page, cp_catalog_page_id#68) AS id#84, MakeDecimal(sum(UnscaledValue(sales_price#53))#79,17,2) AS sales#85, MakeDecimal(sum(UnscaledValue(return_amt#55))#80,17,2) AS returns#86, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#81,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#82,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#87] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#53))#80, sum(UnscaledValue(return_amt#55))#81, sum(UnscaledValue(profit#54))#82, sum(UnscaledValue(net_loss#56))#83] +Results [5]: [catalog channel AS channel#84, concat(catalog_page, cp_catalog_page_id#69) AS id#85, MakeDecimal(sum(UnscaledValue(sales_price#53))#80,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(return_amt#55))#81,17,2) AS returns#87, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#54))#82,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#56))#83,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#88] (47) Scan parquet default.web_sales -Output [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cast(ws_sold_date_sk#91 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cast(ws_sold_date_sk#92 as bigint)), dynamicpruningexpression(cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct (48) ColumnarToRow [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (49) Filter [codegen id : 13] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] -Condition : isnotnull(ws_web_site_sk#88) +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : isnotnull(ws_web_site_sk#89) (50) Project [codegen id : 13] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#92, cast(ws_sold_date_sk#91 as bigint) AS date_sk#93, ws_ext_sales_price#89 AS sales_price#94, ws_net_profit#90 AS profit#95, 0.00 AS return_amt#96, 0.00 AS net_loss#97] -Input [4]: [ws_web_site_sk#88, ws_ext_sales_price#89, ws_net_profit#90, ws_sold_date_sk#91] +Output [6]: [ws_web_site_sk#89 AS wsr_web_site_sk#93, cast(ws_sold_date_sk#92 as bigint) AS date_sk#94, ws_ext_sales_price#90 AS sales_price#95, ws_net_profit#91 AS profit#96, 0.00 AS return_amt#97, 0.00 AS net_loss#98] +Input [4]: [ws_web_site_sk#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (51) Scan parquet default.web_returns -Output [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Output [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#102), dynamicpruningexpression(wr_returned_date_sk#102 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(wr_returned_date_sk#103), dynamicpruningexpression(wr_returned_date_sk#103 IN dynamicpruning#5)] ReadSchema: struct (52) ColumnarToRow [codegen id : 14] -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] (53) BroadcastExchange -Input [5]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#103] +Input [5]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true], input[1, bigint, true]),false), [id=#104] (54) Scan parquet default.web_sales -Output [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct (55) ColumnarToRow -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (56) Filter -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] -Condition : ((isnotnull(ws_item_sk#104) AND isnotnull(ws_order_number#105)) AND isnotnull(ws_web_site_sk#88)) +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] +Condition : ((isnotnull(ws_item_sk#105) AND isnotnull(ws_order_number#107)) AND isnotnull(ws_web_site_sk#106)) (57) Project -Output [3]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] -Input [4]: [ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105, ws_sold_date_sk#91] +Output [3]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] +Input [4]: [ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107, ws_sold_date_sk#108] (58) BroadcastHashJoin [codegen id : 15] -Left keys [2]: [wr_item_sk#98, wr_order_number#99] -Right keys [2]: [cast(ws_item_sk#104 as bigint), cast(ws_order_number#105 as bigint)] +Left keys [2]: [wr_item_sk#99, wr_order_number#100] +Right keys [2]: [cast(ws_item_sk#105 as bigint), cast(ws_order_number#107 as bigint)] Join condition: None (59) Project [codegen id : 15] -Output [6]: [ws_web_site_sk#88 AS wsr_web_site_sk#106, wr_returned_date_sk#102 AS date_sk#107, 0.00 AS sales_price#108, 0.00 AS profit#109, wr_return_amt#100 AS return_amt#110, wr_net_loss#101 AS net_loss#111] -Input [8]: [wr_item_sk#98, wr_order_number#99, wr_return_amt#100, wr_net_loss#101, wr_returned_date_sk#102, ws_item_sk#104, ws_web_site_sk#88, ws_order_number#105] +Output [6]: [ws_web_site_sk#106 AS wsr_web_site_sk#109, wr_returned_date_sk#103 AS date_sk#110, 0.00 AS sales_price#111, 0.00 AS profit#112, wr_return_amt#101 AS return_amt#113, wr_net_loss#102 AS net_loss#114] +Input [8]: [wr_item_sk#99, wr_order_number#100, wr_return_amt#101, wr_net_loss#102, wr_returned_date_sk#103, ws_item_sk#105, ws_web_site_sk#106, ws_order_number#107] (60) Union (61) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#115] (62) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [date_sk#93] -Right keys [1]: [cast(d_date_sk#22 as bigint)] +Left keys [1]: [date_sk#94] +Right keys [1]: [cast(d_date_sk#115 as bigint)] Join condition: None (63) Project [codegen id : 18] -Output [5]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97] -Input [7]: [wsr_web_site_sk#92, date_sk#93, sales_price#94, profit#95, return_amt#96, net_loss#97, d_date_sk#22] +Output [5]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98] +Input [7]: [wsr_web_site_sk#93, date_sk#94, sales_price#95, profit#96, return_amt#97, net_loss#98, d_date_sk#115] (64) Scan parquet default.web_site -Output [2]: [web_site_sk#112, web_site_id#113] +Output [2]: [web_site_sk#116, web_site_id#117] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (65) ColumnarToRow [codegen id : 17] -Input [2]: [web_site_sk#112, web_site_id#113] +Input [2]: [web_site_sk#116, web_site_id#117] (66) Filter [codegen id : 17] -Input [2]: [web_site_sk#112, web_site_id#113] -Condition : isnotnull(web_site_sk#112) +Input [2]: [web_site_sk#116, web_site_id#117] +Condition : isnotnull(web_site_sk#116) (67) BroadcastExchange -Input [2]: [web_site_sk#112, web_site_id#113] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#114] +Input [2]: [web_site_sk#116, web_site_id#117] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#118] (68) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [wsr_web_site_sk#92] -Right keys [1]: [web_site_sk#112] +Left keys [1]: [wsr_web_site_sk#93] +Right keys [1]: [web_site_sk#116] Join condition: None (69) Project [codegen id : 18] -Output [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] -Input [7]: [wsr_web_site_sk#92, sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_sk#112, web_site_id#113] +Output [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Input [7]: [wsr_web_site_sk#93, sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_sk#116, web_site_id#117] (70) HashAggregate [codegen id : 18] -Input [5]: [sales_price#94, profit#95, return_amt#96, net_loss#97, web_site_id#113] -Keys [1]: [web_site_id#113] -Functions [4]: [partial_sum(UnscaledValue(sales_price#94)), partial_sum(UnscaledValue(return_amt#96)), partial_sum(UnscaledValue(profit#95)), partial_sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum#115, sum#116, sum#117, sum#118] -Results [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] +Input [5]: [sales_price#95, profit#96, return_amt#97, net_loss#98, web_site_id#117] +Keys [1]: [web_site_id#117] +Functions [4]: [partial_sum(UnscaledValue(sales_price#95)), partial_sum(UnscaledValue(return_amt#97)), partial_sum(UnscaledValue(profit#96)), partial_sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum#119, sum#120, sum#121, sum#122] +Results [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] (71) Exchange -Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -Arguments: hashpartitioning(web_site_id#113, 5), ENSURE_REQUIREMENTS, [id=#123] +Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] +Arguments: hashpartitioning(web_site_id#117, 5), ENSURE_REQUIREMENTS, [id=#127] (72) HashAggregate [codegen id : 19] -Input [5]: [web_site_id#113, sum#119, sum#120, sum#121, sum#122] -Keys [1]: [web_site_id#113] -Functions [4]: [sum(UnscaledValue(sales_price#94)), sum(UnscaledValue(return_amt#96)), sum(UnscaledValue(profit#95)), sum(UnscaledValue(net_loss#97))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#94))#124, sum(UnscaledValue(return_amt#96))#125, sum(UnscaledValue(profit#95))#126, sum(UnscaledValue(net_loss#97))#127] -Results [5]: [web channel AS channel#128, concat(web_site, web_site_id#113) AS id#129, MakeDecimal(sum(UnscaledValue(sales_price#94))#124,17,2) AS sales#130, MakeDecimal(sum(UnscaledValue(return_amt#96))#125,17,2) AS returns#131, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#95))#126,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#97))#127,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#132] +Input [5]: [web_site_id#117, sum#123, sum#124, sum#125, sum#126] +Keys [1]: [web_site_id#117] +Functions [4]: [sum(UnscaledValue(sales_price#95)), sum(UnscaledValue(return_amt#97)), sum(UnscaledValue(profit#96)), sum(UnscaledValue(net_loss#98))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#95))#128, sum(UnscaledValue(return_amt#97))#129, sum(UnscaledValue(profit#96))#130, sum(UnscaledValue(net_loss#98))#131] +Results [5]: [web channel AS channel#132, concat(web_site, web_site_id#117) AS id#133, MakeDecimal(sum(UnscaledValue(sales_price#95))#128,17,2) AS sales#134, MakeDecimal(sum(UnscaledValue(return_amt#97))#129,17,2) AS returns#135, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#96))#130,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#98))#131,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#136] (73) Union @@ -429,99 +429,99 @@ Results [5]: [web channel AS channel#128, concat(web_site, web_site_id#113) AS i Input [5]: [channel#41, id#42, sales#43, returns#44, profit#45] Keys [2]: [channel#41, id#42] Functions [3]: [partial_sum(sales#43), partial_sum(returns#44), partial_sum(profit#45)] -Aggregate Attributes [6]: [sum#133, isEmpty#134, sum#135, isEmpty#136, sum#137, isEmpty#138] -Results [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Aggregate Attributes [6]: [sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] +Results [8]: [channel#41, id#42, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] (75) Exchange -Input [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] -Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#145] +Input [8]: [channel#41, id#42, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Arguments: hashpartitioning(channel#41, id#42, 5), ENSURE_REQUIREMENTS, [id=#149] (76) HashAggregate [codegen id : 21] -Input [8]: [channel#41, id#42, sum#139, isEmpty#140, sum#141, isEmpty#142, sum#143, isEmpty#144] +Input [8]: [channel#41, id#42, sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] Keys [2]: [channel#41, id#42] Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#45)] -Aggregate Attributes [3]: [sum(sales#43)#146, sum(returns#44)#147, sum(profit#45)#148] -Results [5]: [channel#41, id#42, cast(sum(sales#43)#146 as decimal(37,2)) AS sales#149, cast(sum(returns#44)#147 as decimal(37,2)) AS returns#150, cast(sum(profit#45)#148 as decimal(38,2)) AS profit#151] +Aggregate Attributes [3]: [sum(sales#43)#150, sum(returns#44)#151, sum(profit#45)#152] +Results [5]: [channel#41, id#42, cast(sum(sales#43)#150 as decimal(37,2)) AS sales#153, cast(sum(returns#44)#151 as decimal(37,2)) AS returns#154, cast(sum(profit#45)#152 as decimal(38,2)) AS profit#155] (77) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] +Output [8]: [channel#41, id#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] (78) HashAggregate [codegen id : 42] -Input [8]: [channel#41, id#42, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] +Input [8]: [channel#41, id#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#158)] -Aggregate Attributes [3]: [sum(sales#43)#159, sum(returns#44)#160, sum(profit#158)#161] -Results [4]: [channel#41, sum(sales#43)#159 AS sales#162, sum(returns#44)#160 AS returns#163, sum(profit#158)#161 AS profit#164] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#162)] +Aggregate Attributes [3]: [sum(sales#43)#163, sum(returns#44)#164, sum(profit#162)#165] +Results [4]: [channel#41, sum(sales#43)#163 AS sales#166, sum(returns#44)#164 AS returns#167, sum(profit#162)#165 AS profit#168] (79) HashAggregate [codegen id : 42] -Input [4]: [channel#41, sales#162, returns#163, profit#164] +Input [4]: [channel#41, sales#166, returns#167, profit#168] Keys [1]: [channel#41] -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#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Functions [3]: [partial_sum(sales#166), partial_sum(returns#167), partial_sum(profit#168)] +Aggregate Attributes [6]: [sum#169, isEmpty#170, sum#171, isEmpty#172, sum#173, isEmpty#174] +Results [7]: [channel#41, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] (80) Exchange -Input [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#177] +Input [7]: [channel#41, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] +Arguments: hashpartitioning(channel#41, 5), ENSURE_REQUIREMENTS, [id=#181] (81) HashAggregate [codegen id : 43] -Input [7]: [channel#41, sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Input [7]: [channel#41, sum#175, isEmpty#176, sum#177, isEmpty#178, sum#179, isEmpty#180] Keys [1]: [channel#41] -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#41, 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#166), sum(returns#167), sum(profit#168)] +Aggregate Attributes [3]: [sum(sales#166)#182, sum(returns#167)#183, sum(profit#168)#184] +Results [5]: [channel#41, null AS id#185, sum(sales#166)#182 AS sum(sales)#186, sum(returns#167)#183 AS sum(returns)#187, sum(profit#168)#184 AS sum(profit)#188] (82) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#41, id#42, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Output [8]: [channel#41, id#42, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194] (83) HashAggregate [codegen id : 64] -Input [8]: [channel#41, id#42, sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Input [8]: [channel#41, id#42, sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194] Keys [2]: [channel#41, id#42] -Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#191)] -Aggregate Attributes [3]: [sum(sales#43)#192, sum(returns#44)#193, sum(profit#191)#194] -Results [3]: [sum(sales#43)#192 AS sales#162, sum(returns#44)#193 AS returns#163, sum(profit#191)#194 AS profit#164] +Functions [3]: [sum(sales#43), sum(returns#44), sum(profit#195)] +Aggregate Attributes [3]: [sum(sales#43)#196, sum(returns#44)#197, sum(profit#195)#198] +Results [3]: [sum(sales#43)#196 AS sales#166, sum(returns#44)#197 AS returns#167, sum(profit#195)#198 AS profit#168] (84) HashAggregate [codegen id : 64] -Input [3]: [sales#162, returns#163, profit#164] +Input [3]: [sales#166, returns#167, profit#168] Keys: [] -Functions [3]: [partial_sum(sales#162), partial_sum(returns#163), partial_sum(profit#164)] -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#166), partial_sum(returns#167), partial_sum(profit#168)] +Aggregate Attributes [6]: [sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204] +Results [6]: [sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210] (85) Exchange -Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#207] +Input [6]: [sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#211] (86) HashAggregate [codegen id : 65] -Input [6]: [sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206] +Input [6]: [sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210] Keys: [] -Functions [3]: [sum(sales#162), sum(returns#163), sum(profit#164)] -Aggregate Attributes [3]: [sum(sales#162)#208, sum(returns#163)#209, sum(profit#164)#210] -Results [5]: [null AS channel#211, null AS id#212, sum(sales#162)#208 AS sum(sales)#213, sum(returns#163)#209 AS sum(returns)#214, sum(profit#164)#210 AS sum(profit)#215] +Functions [3]: [sum(sales#166), sum(returns#167), sum(profit#168)] +Aggregate Attributes [3]: [sum(sales#166)#212, sum(returns#167)#213, sum(profit#168)#214] +Results [5]: [null AS channel#215, null AS id#216, sum(sales#166)#212 AS sum(sales)#217, sum(returns#167)#213 AS sum(returns)#218, sum(profit#168)#214 AS sum(profit)#219] (87) Union (88) HashAggregate [codegen id : 66] -Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -Keys [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Keys [5]: [channel#41, id#42, sales#153, returns#154, profit#155] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Results [5]: [channel#41, id#42, sales#153, returns#154, profit#155] (89) Exchange -Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -Arguments: hashpartitioning(channel#41, id#42, sales#149, returns#150, profit#151, 5), ENSURE_REQUIREMENTS, [id=#216] +Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Arguments: hashpartitioning(channel#41, id#42, sales#153, returns#154, profit#155, 5), ENSURE_REQUIREMENTS, [id=#220] (90) HashAggregate [codegen id : 67] -Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -Keys [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Keys [5]: [channel#41, id#42, sales#153, returns#154, profit#155] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#41, id#42, sales#149, returns#150, profit#151] +Results [5]: [channel#41, id#42, sales#153, returns#154, profit#155] (91) TakeOrderedAndProject -Input [5]: [channel#41, id#42, sales#149, returns#150, profit#151] -Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#149, returns#150, profit#151] +Input [5]: [channel#41, id#42, sales#153, returns#154, profit#155] +Arguments: 100, [channel#41 ASC NULLS FIRST, id#42 ASC NULLS FIRST], [channel#41, id#42, sales#153, returns#154, profit#155] ===== Subqueries ===== @@ -539,12 +539,12 @@ ReusedExchange (93) (93) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#67] Subquery:4 Hosting operator id = 30 Hosting Expression = cr_returned_date_sk#60 IN dynamicpruning#50 -Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#91 as bigint) IN dynamicpruning#5 +Subquery:5 Hosting operator id = 47 Hosting Expression = cast(ws_sold_date_sk#92 as bigint) IN dynamicpruning#5 -Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#102 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 51 Hosting Expression = wr_returned_date_sk#103 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 2a9e1ad7c715b..3847146fa8fa8 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 @@ -795,353 +795,353 @@ Input [17]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_str Arguments: [item_sk#112 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, store_zip#114 ASC NULLS FIRST], false, 0 (132) Scan parquet default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#129)] +PartitionFilters: [isnotnull(ss_sold_date_sk#140), dynamicpruningexpression(ss_sold_date_sk#140 IN dynamicpruning#141)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (133) ColumnarToRow [codegen id : 44] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] (134) Filter [codegen id : 44] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Condition : (((((((isnotnull(ss_item_sk#129) AND isnotnull(ss_ticket_number#136)) AND isnotnull(ss_store_sk#134)) AND isnotnull(ss_customer_sk#130)) AND isnotnull(ss_cdemo_sk#131)) AND isnotnull(ss_promo_sk#135)) AND isnotnull(ss_hdemo_sk#132)) AND isnotnull(ss_addr_sk#133)) (135) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint), 5), ENSURE_REQUIREMENTS, [id=#130] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Arguments: hashpartitioning(cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint), 5), ENSURE_REQUIREMENTS, [id=#142] (136) Sort [codegen id : 45] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [cast(ss_item_sk#1 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#8 as bigint) ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Arguments: [cast(ss_item_sk#129 as bigint) ASC NULLS FIRST, cast(ss_ticket_number#136 as bigint) ASC NULLS FIRST], false, 0 (137) ReusedExchange [Reuses operator id: 10] -Output [2]: [sr_item_sk#15, sr_ticket_number#16] +Output [2]: [sr_item_sk#143, sr_ticket_number#144] (138) Sort [codegen id : 47] -Input [2]: [sr_item_sk#15, sr_ticket_number#16] -Arguments: [sr_item_sk#15 ASC NULLS FIRST, sr_ticket_number#16 ASC NULLS FIRST], false, 0 +Input [2]: [sr_item_sk#143, sr_ticket_number#144] +Arguments: [sr_item_sk#143 ASC NULLS FIRST, sr_ticket_number#144 ASC NULLS FIRST], false, 0 (139) SortMergeJoin [codegen id : 56] -Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] -Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] +Left keys [2]: [cast(ss_item_sk#129 as bigint), cast(ss_ticket_number#136 as bigint)] +Right keys [2]: [sr_item_sk#143, sr_ticket_number#144] Join condition: None (140) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_ticket_number#136, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, sr_item_sk#143, sr_ticket_number#144] (141) ReusedExchange [Reuses operator id: 33] -Output [1]: [cs_item_sk#19] +Output [1]: [cs_item_sk#145] (142) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#19] +Left keys [1]: [ss_item_sk#129] +Right keys [1]: [cs_item_sk#145] Join condition: None (143) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, cs_item_sk#145] (144) Scan parquet default.date_dim -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#146, d_year#147] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (145) ColumnarToRow [codegen id : 54] -Input [2]: [d_date_sk#43, d_year#44] +Input [2]: [d_date_sk#146, d_year#147] (146) Filter [codegen id : 54] -Input [2]: [d_date_sk#43, d_year#44] -Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#43)) +Input [2]: [d_date_sk#146, d_year#147] +Condition : ((isnotnull(d_year#147) AND (d_year#147 = 2000)) AND isnotnull(d_date_sk#146)) (147) BroadcastExchange -Input [2]: [d_date_sk#43, d_year#44] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#131] +Input [2]: [d_date_sk#146, d_year#147] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#148] (148) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#43] +Left keys [1]: [ss_sold_date_sk#140] +Right keys [1]: [d_date_sk#146] Join condition: None (149) Project [codegen id : 56] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#43, d_year#44] +Output [11]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147] +Input [13]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, ss_sold_date_sk#140, d_date_sk#146, d_year#147] (150) ReusedExchange [Reuses operator id: 45] -Output [3]: [s_store_sk#46, s_store_name#47, s_zip#48] +Output [3]: [s_store_sk#149, s_store_name#150, s_zip#151] (151) BroadcastHashJoin [codegen id : 56] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#46] +Left keys [1]: [ss_store_sk#134] +Right keys [1]: [s_store_sk#149] Join condition: None (152) Project [codegen id : 56] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_sk#46, s_store_name#47, s_zip#48] +Output [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Input [14]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_store_sk#134, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_sk#149, s_store_name#150, s_zip#151] (153) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#132] +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Arguments: hashpartitioning(ss_customer_sk#130, 5), ENSURE_REQUIREMENTS, [id=#152] (154) Sort [codegen id : 57] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [12]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151] +Arguments: [ss_customer_sk#130 ASC NULLS FIRST], false, 0 (155) ReusedExchange [Reuses operator id: 53] -Output [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Output [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] (156) Sort [codegen id : 59] -Input [6]: [c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] -Arguments: [c_customer_sk#51 ASC NULLS FIRST], false, 0 +Input [6]: [c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Arguments: [c_customer_sk#153 ASC NULLS FIRST], false, 0 (157) SortMergeJoin [codegen id : 62] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#51] +Left keys [1]: [ss_customer_sk#130] +Right keys [1]: [c_customer_sk#153] Join condition: None (158) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_customer_sk#51, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] +Input [18]: [ss_item_sk#129, ss_customer_sk#130, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_customer_sk#153, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158] (159) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#133, d_year#134] +Output [2]: [d_date_sk#159, d_year#160] (160) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_sales_date_sk#56] -Right keys [1]: [d_date_sk#133] +Left keys [1]: [c_first_sales_date_sk#158] +Right keys [1]: [d_date_sk#159] Join condition: None (161) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, c_first_sales_date_sk#56, d_date_sk#133, d_year#134] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, c_first_sales_date_sk#158, d_date_sk#159, d_year#160] (162) ReusedExchange [Reuses operator id: 60] -Output [2]: [d_date_sk#135, d_year#136] +Output [2]: [d_date_sk#161, d_year#162] (163) BroadcastHashJoin [codegen id : 62] -Left keys [1]: [c_first_shipto_date_sk#55] -Right keys [1]: [d_date_sk#135] +Left keys [1]: [c_first_shipto_date_sk#157] +Right keys [1]: [d_date_sk#161] Join condition: None (164) Project [codegen id : 62] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, c_first_shipto_date_sk#55, d_year#134, d_date_sk#135, d_year#136] +Output [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, c_first_shipto_date_sk#157, d_year#160, d_date_sk#161, d_year#162] (165) Exchange -Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Arguments: hashpartitioning(ss_cdemo_sk#3, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Arguments: hashpartitioning(ss_cdemo_sk#131, 5), ENSURE_REQUIREMENTS, [id=#163] (166) Sort [codegen id : 63] -Input [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Arguments: [ss_cdemo_sk#3 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Arguments: [ss_cdemo_sk#131 ASC NULLS FIRST], false, 0 (167) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#64, cd_marital_status#65] +Output [2]: [cd_demo_sk#164, cd_marital_status#165] (168) Sort [codegen id : 65] -Input [2]: [cd_demo_sk#64, cd_marital_status#65] -Arguments: [cd_demo_sk#64 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#164, cd_marital_status#165] +Arguments: [cd_demo_sk#164 ASC NULLS FIRST], false, 0 (169) SortMergeJoin [codegen id : 66] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#64] +Left keys [1]: [ss_cdemo_sk#131] +Right keys [1]: [cd_demo_sk#164] Join condition: None (170) Project [codegen id : 66] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_demo_sk#64, cd_marital_status#65] +Output [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Input [18]: [ss_item_sk#129, ss_cdemo_sk#131, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_demo_sk#164, cd_marital_status#165] (171) Exchange -Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Arguments: hashpartitioning(c_current_cdemo_sk#52, 5), ENSURE_REQUIREMENTS, [id=#138] +Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Arguments: hashpartitioning(c_current_cdemo_sk#154, 5), ENSURE_REQUIREMENTS, [id=#166] (172) Sort [codegen id : 67] -Input [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65] -Arguments: [c_current_cdemo_sk#52 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165] +Arguments: [c_current_cdemo_sk#154 ASC NULLS FIRST], false, 0 (173) ReusedExchange [Reuses operator id: 71] -Output [2]: [cd_demo_sk#139, cd_marital_status#140] +Output [2]: [cd_demo_sk#167, cd_marital_status#168] (174) Sort [codegen id : 69] -Input [2]: [cd_demo_sk#139, cd_marital_status#140] -Arguments: [cd_demo_sk#139 ASC NULLS FIRST], false, 0 +Input [2]: [cd_demo_sk#167, cd_marital_status#168] +Arguments: [cd_demo_sk#167 ASC NULLS FIRST], false, 0 (175) SortMergeJoin [codegen id : 73] -Left keys [1]: [c_current_cdemo_sk#52] -Right keys [1]: [cd_demo_sk#139] -Join condition: NOT (cd_marital_status#65 = cd_marital_status#140) +Left keys [1]: [c_current_cdemo_sk#154] +Right keys [1]: [cd_demo_sk#167] +Join condition: NOT (cd_marital_status#165 = cd_marital_status#168) (176) Project [codegen id : 73] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_cdemo_sk#52, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, cd_marital_status#65, cd_demo_sk#139, cd_marital_status#140] +Output [14]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [18]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_cdemo_sk#154, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, cd_marital_status#165, cd_demo_sk#167, cd_marital_status#168] (177) ReusedExchange [Reuses operator id: 84] -Output [1]: [p_promo_sk#70] +Output [1]: [p_promo_sk#169] (178) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#70] +Left keys [1]: [ss_promo_sk#135] +Right keys [1]: [p_promo_sk#169] Join condition: None (179) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, p_promo_sk#70] +Output [13]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162] +Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_promo_sk#135, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, p_promo_sk#169] (180) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#72, hd_income_band_sk#73] +Output [2]: [hd_demo_sk#170, hd_income_band_sk#171] (181) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#72] +Left keys [1]: [ss_hdemo_sk#132] +Right keys [1]: [hd_demo_sk#170] Join condition: None (182) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_demo_sk#72, hd_income_band_sk#73] +Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171] +Input [15]: [ss_item_sk#129, ss_hdemo_sk#132, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_demo_sk#170, hd_income_band_sk#171] (183) ReusedExchange [Reuses operator id: 90] -Output [2]: [hd_demo_sk#141, hd_income_band_sk#142] +Output [2]: [hd_demo_sk#172, hd_income_band_sk#173] (184) BroadcastHashJoin [codegen id : 73] -Left keys [1]: [c_current_hdemo_sk#53] -Right keys [1]: [hd_demo_sk#141] +Left keys [1]: [c_current_hdemo_sk#155] +Right keys [1]: [hd_demo_sk#172] Join condition: None (185) Project [codegen id : 73] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_hdemo_sk#53, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_demo_sk#141, hd_income_band_sk#142] +Output [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Input [15]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_hdemo_sk#155, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_demo_sk#172, hd_income_band_sk#173] (186) Exchange -Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Arguments: hashpartitioning(ss_addr_sk#5, 5), ENSURE_REQUIREMENTS, [id=#143] +Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Arguments: hashpartitioning(ss_addr_sk#133, 5), ENSURE_REQUIREMENTS, [id=#174] (187) Sort [codegen id : 74] -Input [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142] -Arguments: [ss_addr_sk#5 ASC NULLS FIRST], false, 0 +Input [13]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173] +Arguments: [ss_addr_sk#133 ASC NULLS FIRST], false, 0 (188) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Output [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] (189) Sort [codegen id : 76] -Input [5]: [ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: [ca_address_sk#78 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: [ca_address_sk#175 ASC NULLS FIRST], false, 0 (190) SortMergeJoin [codegen id : 77] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#78] +Left keys [1]: [ss_addr_sk#133] +Right keys [1]: [ca_address_sk#175] Join condition: None (191) Project [codegen id : 77] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_address_sk#78, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] +Output [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Input [18]: [ss_item_sk#129, ss_addr_sk#133, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_address_sk#175, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] (192) Exchange -Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: hashpartitioning(c_current_addr_sk#54, 5), ENSURE_REQUIREMENTS, [id=#144] +Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: hashpartitioning(c_current_addr_sk#156, 5), ENSURE_REQUIREMENTS, [id=#180] (193) Sort [codegen id : 78] -Input [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82] -Arguments: [c_current_addr_sk#54 ASC NULLS FIRST], false, 0 +Input [16]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179] +Arguments: [c_current_addr_sk#156 ASC NULLS FIRST], false, 0 (194) ReusedExchange [Reuses operator id: 101] -Output [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Output [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] (195) Sort [codegen id : 80] -Input [5]: [ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Arguments: [ca_address_sk#145 ASC NULLS FIRST], false, 0 +Input [5]: [ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Arguments: [ca_address_sk#181 ASC NULLS FIRST], false, 0 (196) SortMergeJoin [codegen id : 84] -Left keys [1]: [c_current_addr_sk#54] -Right keys [1]: [ca_address_sk#145] +Left keys [1]: [c_current_addr_sk#156] +Right keys [1]: [ca_address_sk#181] Join condition: None (197) Project [codegen id : 84] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, c_current_addr_sk#54, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_address_sk#145, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] +Output [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [21]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, c_current_addr_sk#156, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_address_sk#181, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] (198) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#90] +Output [1]: [ib_income_band_sk#186] (199) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#73] -Right keys [1]: [ib_income_band_sk#90] +Left keys [1]: [hd_income_band_sk#171] +Right keys [1]: [ib_income_band_sk#186] Join condition: None (200) Project [codegen id : 84] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#73, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#90] +Output [18]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [20]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#171, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#186] (201) ReusedExchange [Reuses operator id: 114] -Output [1]: [ib_income_band_sk#150] +Output [1]: [ib_income_band_sk#187] (202) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [hd_income_band_sk#142] -Right keys [1]: [ib_income_band_sk#150] +Left keys [1]: [hd_income_band_sk#173] +Right keys [1]: [ib_income_band_sk#187] Join condition: None (203) Project [codegen id : 84] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, hd_income_band_sk#142, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, ib_income_band_sk#150] +Output [17]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185] +Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, hd_income_band_sk#173, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, ib_income_band_sk#187] (204) ReusedExchange [Reuses operator id: 124] -Output [2]: [i_item_sk#93, i_product_name#96] +Output [2]: [i_item_sk#188, i_product_name#189] (205) BroadcastHashJoin [codegen id : 84] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#93] +Left keys [1]: [ss_item_sk#129] +Right keys [1]: [i_item_sk#188] Join condition: None (206) Project [codegen id : 84] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, s_store_name#47, s_zip#48, d_year#134, d_year#136, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] +Output [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] +Input [19]: [ss_item_sk#129, ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, s_store_name#150, s_zip#151, d_year#160, d_year#162, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] (207) HashAggregate [codegen id : 84] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#44, d_year#134, d_year#136, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, i_item_sk#93, i_product_name#96] -Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#151, sum#152, sum#153, sum#154] -Results [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] +Input [18]: [ss_wholesale_cost#137, ss_list_price#138, ss_coupon_amt#139, d_year#147, d_year#160, d_year#162, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, i_item_sk#188, i_product_name#189] +Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#137)), partial_sum(UnscaledValue(ss_list_price#138)), partial_sum(UnscaledValue(ss_coupon_amt#139))] +Aggregate Attributes [4]: [count#190, sum#191, sum#192, sum#193] +Results [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] (208) Exchange -Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] -Arguments: hashpartitioning(i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, 5), ENSURE_REQUIREMENTS, [id=#159] +Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] +Arguments: hashpartitioning(i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, 5), ENSURE_REQUIREMENTS, [id=#198] (209) HashAggregate [codegen id : 85] -Input [19]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136, count#155, sum#156, sum#157, sum#158] -Keys [15]: [i_product_name#96, i_item_sk#93, s_store_name#47, s_zip#48, ca_street_number#79, ca_street_name#80, ca_city#81, ca_zip#82, ca_street_number#146, ca_street_name#147, ca_city#148, ca_zip#149, d_year#44, d_year#134, d_year#136] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#160, sum(UnscaledValue(ss_wholesale_cost#9))#161, sum(UnscaledValue(ss_list_price#10))#162, sum(UnscaledValue(ss_coupon_amt#11))#163] -Results [8]: [i_item_sk#93 AS item_sk#164, s_store_name#47 AS store_name#165, s_zip#48 AS store_zip#166, d_year#44 AS syear#167, count(1)#160 AS cnt#168, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#161,17,2) AS s1#169, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#162,17,2) AS s2#170, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#163,17,2) AS s3#171] +Input [19]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162, count#194, sum#195, sum#196, sum#197] +Keys [15]: [i_product_name#189, i_item_sk#188, s_store_name#150, s_zip#151, ca_street_number#176, ca_street_name#177, ca_city#178, ca_zip#179, ca_street_number#182, ca_street_name#183, ca_city#184, ca_zip#185, d_year#147, d_year#160, d_year#162] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#137)), sum(UnscaledValue(ss_list_price#138)), sum(UnscaledValue(ss_coupon_amt#139))] +Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#137))#200, sum(UnscaledValue(ss_list_price#138))#201, sum(UnscaledValue(ss_coupon_amt#139))#202] +Results [8]: [i_item_sk#188 AS item_sk#203, s_store_name#150 AS store_name#204, s_zip#151 AS store_zip#205, d_year#147 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#137))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#138))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#139))#202,17,2) AS s3#210] (210) Exchange -Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] -Arguments: hashpartitioning(item_sk#164, store_name#165, store_zip#166, 5), ENSURE_REQUIREMENTS, [id=#172] +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] (211) Sort [codegen id : 86] -Input [8]: [item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] -Arguments: [item_sk#164 ASC NULLS FIRST, store_name#165 ASC NULLS FIRST, store_zip#166 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 (212) SortMergeJoin [codegen id : 87] Left keys [3]: [item_sk#112, store_name#113, store_zip#114] -Right keys [3]: [item_sk#164, store_name#165, store_zip#166] -Join condition: (cnt#168 <= cnt#124) +Right keys [3]: [item_sk#203, store_name#204, store_zip#205] +Join condition: (cnt#207 <= cnt#124) (213) Project [codegen id : 87] -Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#164, store_name#165, store_zip#166, syear#167, cnt#168, s1#169, s2#170, s3#171] +Output [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Input [25]: [product_name#111, item_sk#112, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] (214) Exchange -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#169 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#173] +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: rangepartitioning(product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#208 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] (215) Sort [codegen id : 88] -Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#169, s2#170, s3#171, syear#167, cnt#168] -Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#168 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#169 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#111, store_name#113, store_zip#114, b_street_number#115, b_streen_name#116, b_city#117, b_zip#118, c_street_number#119, c_street_name#120, c_city#121, c_zip#122, syear#123, cnt#124, s1#125, s2#126, s3#127, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: [product_name#111 ASC NULLS FIRST, store_name#113 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#125 ASC NULLS FIRST, s1#208 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1152,11 +1152,11 @@ ReusedExchange (216) (216) ReusedExchange [Reuses operator id: 39] Output [2]: [d_date_sk#43, d_year#44] -Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#129 +Subquery:2 Hosting operator id = 132 Hosting Expression = ss_sold_date_sk#140 IN dynamicpruning#141 ReusedExchange (217) (217) ReusedExchange [Reuses operator id: 147] -Output [2]: [d_date_sk#43, d_year#44] +Output [2]: [d_date_sk#146, d_year#147] 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 5c2ac1abaaa7e..0115f4ab8a63a 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 @@ -701,323 +701,323 @@ Input [17]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_str Arguments: [item_sk#105 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, store_zip#107 ASC NULLS FIRST], false, 0 (115) Scan parquet default.store_sales -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#122)] +PartitionFilters: [isnotnull(ss_sold_date_sk#133), dynamicpruningexpression(ss_sold_date_sk#133 IN dynamicpruning#134)] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] ReadSchema: struct (116) ColumnarToRow [codegen id : 27] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] (117) Filter [codegen id : 27] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Condition : (((((((isnotnull(ss_item_sk#122) AND isnotnull(ss_ticket_number#129)) AND isnotnull(ss_store_sk#127)) AND isnotnull(ss_customer_sk#123)) AND isnotnull(ss_cdemo_sk#124)) AND isnotnull(ss_promo_sk#128)) AND isnotnull(ss_hdemo_sk#125)) AND isnotnull(ss_addr_sk#126)) (118) BroadcastExchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#123] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint), cast(input[7, int, false] as bigint)),false), [id=#135] (119) Scan parquet default.store_returns -Output [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Output [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] ReadSchema: struct (120) ColumnarToRow -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] (121) Filter -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] -Condition : (isnotnull(sr_item_sk#15) AND isnotnull(sr_ticket_number#16)) +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] +Condition : (isnotnull(sr_item_sk#136) AND isnotnull(sr_ticket_number#137)) (122) Project -Output [2]: [sr_item_sk#15, sr_ticket_number#16] -Input [3]: [sr_item_sk#15, sr_ticket_number#16, sr_returned_date_sk#17] +Output [2]: [sr_item_sk#136, sr_ticket_number#137] +Input [3]: [sr_item_sk#136, sr_ticket_number#137, sr_returned_date_sk#138] (123) BroadcastHashJoin [codegen id : 28] -Left keys [2]: [cast(ss_item_sk#1 as bigint), cast(ss_ticket_number#8 as bigint)] -Right keys [2]: [sr_item_sk#15, sr_ticket_number#16] +Left keys [2]: [cast(ss_item_sk#122 as bigint), cast(ss_ticket_number#129 as bigint)] +Right keys [2]: [sr_item_sk#136, sr_ticket_number#137] Join condition: None (124) Project [codegen id : 28] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#15, sr_ticket_number#16] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_ticket_number#129, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, sr_item_sk#136, sr_ticket_number#137] (125) Exchange -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#124] +Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: hashpartitioning(ss_item_sk#122, 5), ENSURE_REQUIREMENTS, [id=#139] (126) Sort [codegen id : 29] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Input [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Arguments: [ss_item_sk#122 ASC NULLS FIRST], false, 0 (127) ReusedExchange [Reuses operator id: 28] -Output [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] +Output [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] (128) HashAggregate [codegen id : 35] -Input [4]: [cs_item_sk#19, sum#125, sum#126, isEmpty#127] -Keys [1]: [cs_item_sk#19] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#21)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#21))#128, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129] -Results [3]: [cs_item_sk#19, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#21))#128,17,2) AS sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#129 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] +Input [4]: [cs_item_sk#140, sum#141, sum#142, isEmpty#143] +Keys [1]: [cs_item_sk#140] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#144)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#144))#148, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149] +Results [3]: [cs_item_sk#140, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#144))#148,17,2) AS sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#149 AS sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] (129) Filter [codegen id : 35] -Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] -Condition : (isnotnull(sum(cs_ext_list_price#21)#130) AND (cast(sum(cs_ext_list_price#21)#130 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131)), DecimalType(21,2), true))) +Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] +Condition : (isnotnull(sum(cs_ext_list_price#144)#150) AND (cast(sum(cs_ext_list_price#144)#150 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151)), DecimalType(21,2), true))) (130) Project [codegen id : 35] -Output [1]: [cs_item_sk#19] -Input [3]: [cs_item_sk#19, sum(cs_ext_list_price#21)#130, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#26 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#27 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#28 as decimal(9,2)))), DecimalType(9,2), true))#131] +Output [1]: [cs_item_sk#140] +Input [3]: [cs_item_sk#140, sum(cs_ext_list_price#144)#150, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#145 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#146 as decimal(8,2)))), DecimalType(8,2), true) as decimal(9,2))) + promote_precision(cast(cr_store_credit#147 as decimal(9,2)))), DecimalType(9,2), true))#151] (131) Sort [codegen id : 35] -Input [1]: [cs_item_sk#19] -Arguments: [cs_item_sk#19 ASC NULLS FIRST], false, 0 +Input [1]: [cs_item_sk#140] +Arguments: [cs_item_sk#140 ASC NULLS FIRST], false, 0 (132) SortMergeJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#19] +Left keys [1]: [ss_item_sk#122] +Right keys [1]: [cs_item_sk#140] Join condition: None (133) Project [codegen id : 51] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#19] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133] +Input [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, cs_item_sk#140] (134) Scan parquet default.date_dim -Output [2]: [d_date_sk#42, d_year#43] +Output [2]: [d_date_sk#152, d_year#153] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct (135) ColumnarToRow [codegen id : 36] -Input [2]: [d_date_sk#42, d_year#43] +Input [2]: [d_date_sk#152, d_year#153] (136) Filter [codegen id : 36] -Input [2]: [d_date_sk#42, d_year#43] -Condition : ((isnotnull(d_year#43) AND (d_year#43 = 2000)) AND isnotnull(d_date_sk#42)) +Input [2]: [d_date_sk#152, d_year#153] +Condition : ((isnotnull(d_year#153) AND (d_year#153 = 2000)) AND isnotnull(d_date_sk#152)) (137) BroadcastExchange -Input [2]: [d_date_sk#42, d_year#43] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#132] +Input [2]: [d_date_sk#152, d_year#153] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#154] (138) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#42] +Left keys [1]: [ss_sold_date_sk#133] +Right keys [1]: [d_date_sk#152] Join condition: None (139) Project [codegen id : 51] -Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#42, d_year#43] +Output [11]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153] +Input [13]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, ss_sold_date_sk#133, d_date_sk#152, d_year#153] (140) ReusedExchange [Reuses operator id: 44] -Output [3]: [s_store_sk#45, s_store_name#46, s_zip#47] +Output [3]: [s_store_sk#155, s_store_name#156, s_zip#157] (141) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#45] +Left keys [1]: [ss_store_sk#127] +Right keys [1]: [s_store_sk#155] Join condition: None (142) Project [codegen id : 51] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47] -Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_sk#45, s_store_name#46, s_zip#47] +Output [12]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157] +Input [14]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_store_sk#127, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_sk#155, s_store_name#156, s_zip#157] (143) ReusedExchange [Reuses operator id: 50] -Output [6]: [c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] +Output [6]: [c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] (144) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#49] +Left keys [1]: [ss_customer_sk#123] +Right keys [1]: [c_customer_sk#158] Join condition: None (145) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] -Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_customer_sk#49, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] +Input [18]: [ss_item_sk#122, ss_customer_sk#123, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_customer_sk#158, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163] (146) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#133, d_year#134] +Output [2]: [d_date_sk#164, d_year#165] (147) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_sales_date_sk#54] -Right keys [1]: [d_date_sk#133] +Left keys [1]: [c_first_sales_date_sk#163] +Right keys [1]: [d_date_sk#164] Join condition: None (148) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, c_first_sales_date_sk#54, d_date_sk#133, d_year#134] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, c_first_sales_date_sk#163, d_date_sk#164, d_year#165] (149) ReusedExchange [Reuses operator id: 56] -Output [2]: [d_date_sk#135, d_year#136] +Output [2]: [d_date_sk#166, d_year#167] (150) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_first_shipto_date_sk#53] -Right keys [1]: [d_date_sk#135] +Left keys [1]: [c_first_shipto_date_sk#162] +Right keys [1]: [d_date_sk#166] Join condition: None (151) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, c_first_shipto_date_sk#53, d_year#134, d_date_sk#135, d_year#136] +Output [16]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, c_first_shipto_date_sk#162, d_year#165, d_date_sk#166, d_year#167] (152) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#61, cd_marital_status#62] +Output [2]: [cd_demo_sk#168, cd_marital_status#169] (153) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#61] +Left keys [1]: [ss_cdemo_sk#124] +Right keys [1]: [cd_demo_sk#168] Join condition: None (154) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62] -Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_demo_sk#61, cd_marital_status#62] +Output [16]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169] +Input [18]: [ss_item_sk#122, ss_cdemo_sk#124, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_demo_sk#168, cd_marital_status#169] (155) ReusedExchange [Reuses operator id: 65] -Output [2]: [cd_demo_sk#137, cd_marital_status#138] +Output [2]: [cd_demo_sk#170, cd_marital_status#171] (156) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_cdemo_sk#50] -Right keys [1]: [cd_demo_sk#137] -Join condition: NOT (cd_marital_status#62 = cd_marital_status#138) +Left keys [1]: [c_current_cdemo_sk#159] +Right keys [1]: [cd_demo_sk#170] +Join condition: NOT (cd_marital_status#169 = cd_marital_status#171) (157) Project [codegen id : 51] -Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_cdemo_sk#50, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, cd_marital_status#62, cd_demo_sk#137, cd_marital_status#138] +Output [14]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [18]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_cdemo_sk#159, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, cd_marital_status#169, cd_demo_sk#170, cd_marital_status#171] (158) ReusedExchange [Reuses operator id: 74] -Output [1]: [p_promo_sk#66] +Output [1]: [p_promo_sk#172] (159) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#66] +Left keys [1]: [ss_promo_sk#128] +Right keys [1]: [p_promo_sk#172] Join condition: None (160) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, p_promo_sk#66] +Output [13]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167] +Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_promo_sk#128, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, p_promo_sk#172] (161) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#68, hd_income_band_sk#69] +Output [2]: [hd_demo_sk#173, hd_income_band_sk#174] (162) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#68] +Left keys [1]: [ss_hdemo_sk#125] +Right keys [1]: [hd_demo_sk#173] Join condition: None (163) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69] -Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_demo_sk#68, hd_income_band_sk#69] +Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174] +Input [15]: [ss_item_sk#122, ss_hdemo_sk#125, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_demo_sk#173, hd_income_band_sk#174] (164) ReusedExchange [Reuses operator id: 80] -Output [2]: [hd_demo_sk#139, hd_income_band_sk#140] +Output [2]: [hd_demo_sk#175, hd_income_band_sk#176] (165) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_hdemo_sk#51] -Right keys [1]: [hd_demo_sk#139] +Left keys [1]: [c_current_hdemo_sk#160] +Right keys [1]: [hd_demo_sk#175] Join condition: None (166) Project [codegen id : 51] -Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_hdemo_sk#51, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_demo_sk#139, hd_income_band_sk#140] +Output [13]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176] +Input [15]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_hdemo_sk#160, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_demo_sk#175, hd_income_band_sk#176] (167) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] +Output [5]: [ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] (168) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#73] +Left keys [1]: [ss_addr_sk#126] +Right keys [1]: [ca_address_sk#177] Join condition: None (169) Project [codegen id : 51] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_address_sk#73, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77] +Output [16]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] +Input [18]: [ss_item_sk#122, ss_addr_sk#126, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_address_sk#177, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181] (170) ReusedExchange [Reuses operator id: 89] -Output [5]: [ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Output [5]: [ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] (171) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [c_current_addr_sk#52] -Right keys [1]: [ca_address_sk#141] +Left keys [1]: [c_current_addr_sk#161] +Right keys [1]: [ca_address_sk#182] Join condition: None (172) Project [codegen id : 51] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, c_current_addr_sk#52, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_address_sk#141, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] +Output [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [21]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, c_current_addr_sk#161, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_address_sk#182, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] (173) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#84] +Output [1]: [ib_income_band_sk#187] (174) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#69] -Right keys [1]: [ib_income_band_sk#84] +Left keys [1]: [hd_income_band_sk#174] +Right keys [1]: [ib_income_band_sk#187] Join condition: None (175) Project [codegen id : 51] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#69, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#84] +Output [18]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [20]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#174, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#187] (176) ReusedExchange [Reuses operator id: 98] -Output [1]: [ib_income_band_sk#146] +Output [1]: [ib_income_band_sk#188] (177) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [hd_income_band_sk#140] -Right keys [1]: [ib_income_band_sk#146] +Left keys [1]: [hd_income_band_sk#176] +Right keys [1]: [ib_income_band_sk#188] Join condition: None (178) Project [codegen id : 51] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, hd_income_band_sk#140, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, ib_income_band_sk#146] +Output [17]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186] +Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, hd_income_band_sk#176, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, ib_income_band_sk#188] (179) ReusedExchange [Reuses operator id: 108] -Output [2]: [i_item_sk#87, i_product_name#90] +Output [2]: [i_item_sk#189, i_product_name#190] (180) BroadcastHashJoin [codegen id : 51] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#87] +Left keys [1]: [ss_item_sk#122] +Right keys [1]: [i_item_sk#189] Join condition: None (181) Project [codegen id : 51] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, s_store_name#46, s_zip#47, d_year#134, d_year#136, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] +Output [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Input [19]: [ss_item_sk#122, ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, s_store_name#156, s_zip#157, d_year#165, d_year#167, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] (182) HashAggregate [codegen id : 51] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#43, d_year#134, d_year#136, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, i_item_sk#87, i_product_name#90] -Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count#147, sum#148, sum#149, sum#150] -Results [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] +Input [18]: [ss_wholesale_cost#130, ss_list_price#131, ss_coupon_amt#132, d_year#153, d_year#165, d_year#167, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, i_item_sk#189, i_product_name#190] +Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#130)), partial_sum(UnscaledValue(ss_list_price#131)), partial_sum(UnscaledValue(ss_coupon_amt#132))] +Aggregate Attributes [4]: [count#191, sum#192, sum#193, sum#194] +Results [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] (183) HashAggregate [codegen id : 51] -Input [19]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136, count#151, sum#152, sum#153, sum#154] -Keys [15]: [i_product_name#90, i_item_sk#87, s_store_name#46, s_zip#47, ca_street_number#74, ca_street_name#75, ca_city#76, ca_zip#77, ca_street_number#142, ca_street_name#143, ca_city#144, ca_zip#145, d_year#43, d_year#134, d_year#136] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] -Aggregate Attributes [4]: [count(1)#155, sum(UnscaledValue(ss_wholesale_cost#9))#156, sum(UnscaledValue(ss_list_price#10))#157, sum(UnscaledValue(ss_coupon_amt#11))#158] -Results [8]: [i_item_sk#87 AS item_sk#159, s_store_name#46 AS store_name#160, s_zip#47 AS store_zip#161, d_year#43 AS syear#162, count(1)#155 AS cnt#163, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#156,17,2) AS s1#164, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#157,17,2) AS s2#165, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#158,17,2) AS s3#166] +Input [19]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167, count#195, sum#196, sum#197, sum#198] +Keys [15]: [i_product_name#190, i_item_sk#189, s_store_name#156, s_zip#157, ca_street_number#178, ca_street_name#179, ca_city#180, ca_zip#181, ca_street_number#183, ca_street_name#184, ca_city#185, ca_zip#186, d_year#153, d_year#165, d_year#167] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#130)), sum(UnscaledValue(ss_list_price#131)), sum(UnscaledValue(ss_coupon_amt#132))] +Aggregate Attributes [4]: [count(1)#199, sum(UnscaledValue(ss_wholesale_cost#130))#200, sum(UnscaledValue(ss_list_price#131))#201, sum(UnscaledValue(ss_coupon_amt#132))#202] +Results [8]: [i_item_sk#189 AS item_sk#203, s_store_name#156 AS store_name#204, s_zip#157 AS store_zip#205, d_year#153 AS syear#206, count(1)#199 AS cnt#207, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#130))#200,17,2) AS s1#208, MakeDecimal(sum(UnscaledValue(ss_list_price#131))#201,17,2) AS s2#209, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#132))#202,17,2) AS s3#210] (184) Exchange -Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] -Arguments: hashpartitioning(item_sk#159, store_name#160, store_zip#161, 5), ENSURE_REQUIREMENTS, [id=#167] +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: hashpartitioning(item_sk#203, store_name#204, store_zip#205, 5), ENSURE_REQUIREMENTS, [id=#211] (185) Sort [codegen id : 52] -Input [8]: [item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] -Arguments: [item_sk#159 ASC NULLS FIRST, store_name#160 ASC NULLS FIRST, store_zip#161 ASC NULLS FIRST], false, 0 +Input [8]: [item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] +Arguments: [item_sk#203 ASC NULLS FIRST, store_name#204 ASC NULLS FIRST, store_zip#205 ASC NULLS FIRST], false, 0 (186) SortMergeJoin [codegen id : 53] Left keys [3]: [item_sk#105, store_name#106, store_zip#107] -Right keys [3]: [item_sk#159, store_name#160, store_zip#161] -Join condition: (cnt#163 <= cnt#117) +Right keys [3]: [item_sk#203, store_name#204, store_zip#205] +Join condition: (cnt#207 <= cnt#117) (187) Project [codegen id : 53] -Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#159, store_name#160, store_zip#161, syear#162, cnt#163, s1#164, s2#165, s3#166] +Output [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Input [25]: [product_name#104, item_sk#105, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, item_sk#203, store_name#204, store_zip#205, syear#206, cnt#207, s1#208, s2#209, s3#210] (188) Exchange -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#164 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#168] +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: rangepartitioning(product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#208 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#212] (189) Sort [codegen id : 54] -Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#164, s2#165, s3#166, syear#162, cnt#163] -Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#163 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#164 ASC NULLS FIRST], true, 0 +Input [21]: [product_name#104, store_name#106, store_zip#107, b_street_number#108, b_streen_name#109, b_city#110, b_zip#111, c_street_number#112, c_street_name#113, c_city#114, c_zip#115, syear#116, cnt#117, s1#118, s2#119, s3#120, s1#208, s2#209, s3#210, syear#206, cnt#207] +Arguments: [product_name#104 ASC NULLS FIRST, store_name#106 ASC NULLS FIRST, cnt#207 ASC NULLS FIRST, s1#118 ASC NULLS FIRST, s1#208 ASC NULLS FIRST], true, 0 ===== Subqueries ===== @@ -1028,11 +1028,11 @@ ReusedExchange (190) (190) ReusedExchange [Reuses operator id: 38] Output [2]: [d_date_sk#42, d_year#43] -Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#122 +Subquery:2 Hosting operator id = 115 Hosting Expression = ss_sold_date_sk#133 IN dynamicpruning#134 ReusedExchange (191) (191) ReusedExchange [Reuses operator id: 137] -Output [2]: [d_date_sk#42, d_year#43] +Output [2]: [d_date_sk#152, d_year#153] 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 413a918da6a0f..af8035e67b061 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 @@ -206,234 +206,234 @@ Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 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))#28 as decimal(38,2)) AS sumsales#29] (29) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#30, isEmpty#31] +Output [10]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, s_store_id#37, sum#38, isEmpty#39] (30) HashAggregate [codegen id : 16] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#30, isEmpty#31] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#32] -Results [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#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))#32 AS sumsales#33] +Input [10]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, s_store_id#37, sum#38, isEmpty#39] +Keys [8]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, s_store_id#37] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#40 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#41 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#40 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#41 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#42] +Results [8]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#40 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#41 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#42 AS sumsales#43] (31) HashAggregate [codegen id : 16] -Input [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, sumsales#33] -Keys [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#34, isEmpty#35] -Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, sum#36, isEmpty#37] +Input [8]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sumsales#43] +Keys [7]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#44, isEmpty#45] +Results [9]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sum#46, isEmpty#47] (32) Exchange -Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, sum#36, isEmpty#37] -Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [9]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sum#46, isEmpty#47] +Arguments: hashpartitioning(i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, 5), ENSURE_REQUIREMENTS, [id=#48] (33) HashAggregate [codegen id : 17] -Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, sum#36, isEmpty#37] -Keys [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#39] -Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, null AS s_store_id#40, sum(sumsales#33)#39 AS sumsales#41] +Input [9]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, sum#46, isEmpty#47] +Keys [7]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#49] +Results [9]: [i_category#30, i_class#31, i_brand#32, i_product_name#33, d_year#34, d_qoy#35, d_moy#36, null AS s_store_id#50, sum(sumsales#43)#49 AS sumsales#51] (34) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#42, isEmpty#43] +Output [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] (35) HashAggregate [codegen id : 25] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#42, isEmpty#43] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#44] -Results [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, 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))#44 AS sumsales#33] +Input [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] +Keys [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#62 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#63 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#62 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#63 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#64] +Results [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#62 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#63 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#64 AS sumsales#43] (36) HashAggregate [codegen id : 25] -Input [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, sumsales#33] -Keys [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#45, isEmpty#46] -Results [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, sum#47, isEmpty#48] +Input [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sumsales#43] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] (37) Exchange -Input [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, sum#47, isEmpty#48] -Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Arguments: hashpartitioning(i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, 5), ENSURE_REQUIREMENTS, [id=#69] (38) HashAggregate [codegen id : 26] -Input [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, sum#47, isEmpty#48] -Keys [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#50] -Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, null AS d_moy#51, null AS s_store_id#52, sum(sumsales#33)#50 AS sumsales#53] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#70] +Results [9]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, null AS d_moy#71, null AS s_store_id#72, sum(sumsales#43)#70 AS sumsales#73] (39) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#54, isEmpty#55] +Output [10]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81, sum#82, isEmpty#83] (40) HashAggregate [codegen id : 34] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#54, isEmpty#55] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#56] -Results [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#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))#56 AS sumsales#33] +Input [10]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81, sum#82, isEmpty#83] +Keys [8]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, d_qoy#79, d_moy#80, s_store_id#81] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#84 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#85 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#84 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#86] +Results [6]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#84 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#85 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#86 AS sumsales#43] (41) HashAggregate [codegen id : 34] -Input [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, sumsales#33] -Keys [5]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#57, isEmpty#58] -Results [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, sum#59, isEmpty#60] +Input [6]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sumsales#43] +Keys [5]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#87, isEmpty#88] +Results [7]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum#89, isEmpty#90] (42) Exchange -Input [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, sum#59, isEmpty#60] -Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#61] +Input [7]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum#89, isEmpty#90] +Arguments: hashpartitioning(i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, 5), ENSURE_REQUIREMENTS, [id=#91] (43) HashAggregate [codegen id : 35] -Input [7]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, sum#59, isEmpty#60] -Keys [5]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#62] -Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, null AS d_qoy#63, null AS d_moy#64, null AS s_store_id#65, sum(sumsales#33)#62 AS sumsales#66] +Input [7]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, sum#89, isEmpty#90] +Keys [5]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#92] +Results [9]: [i_category#74, i_class#75, i_brand#76, i_product_name#77, d_year#78, null AS d_qoy#93, null AS d_moy#94, null AS s_store_id#95, sum(sumsales#43)#92 AS sumsales#96] (44) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#67, isEmpty#68] +Output [10]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, d_year#101, d_qoy#102, d_moy#103, s_store_id#104, sum#105, isEmpty#106] (45) HashAggregate [codegen id : 43] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#67, isEmpty#68] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#69] -Results [5]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, 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))#69 AS sumsales#33] +Input [10]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, d_year#101, d_qoy#102, d_moy#103, s_store_id#104, sum#105, isEmpty#106] +Keys [8]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, d_year#101, d_qoy#102, d_moy#103, s_store_id#104] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#107 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#108 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#107 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#108 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#109] +Results [5]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#107 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#108 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#109 AS sumsales#43] (46) HashAggregate [codegen id : 43] -Input [5]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, sumsales#33] -Keys [4]: [i_category#20, i_class#19, i_brand#18, i_product_name#21] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#70, isEmpty#71] -Results [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, sum#72, isEmpty#73] +Input [5]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sumsales#43] +Keys [4]: [i_category#97, i_class#98, i_brand#99, i_product_name#100] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#110, isEmpty#111] +Results [6]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sum#112, isEmpty#113] (47) Exchange -Input [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, sum#72, isEmpty#73] -Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, i_product_name#21, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [6]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sum#112, isEmpty#113] +Arguments: hashpartitioning(i_category#97, i_class#98, i_brand#99, i_product_name#100, 5), ENSURE_REQUIREMENTS, [id=#114] (48) HashAggregate [codegen id : 44] -Input [6]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, sum#72, isEmpty#73] -Keys [4]: [i_category#20, i_class#19, i_brand#18, i_product_name#21] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#75] -Results [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, null AS d_year#76, null AS d_qoy#77, null AS d_moy#78, null AS s_store_id#79, sum(sumsales#33)#75 AS sumsales#80] +Input [6]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, sum#112, isEmpty#113] +Keys [4]: [i_category#97, i_class#98, i_brand#99, i_product_name#100] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#115] +Results [9]: [i_category#97, i_class#98, i_brand#99, i_product_name#100, null AS d_year#116, null AS d_qoy#117, null AS d_moy#118, null AS s_store_id#119, sum(sumsales#43)#115 AS sumsales#120] (49) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#81, isEmpty#82] +Output [10]: [i_category#121, i_class#122, i_brand#123, i_product_name#124, d_year#125, d_qoy#126, d_moy#127, s_store_id#128, sum#129, isEmpty#130] (50) HashAggregate [codegen id : 52] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#81, isEmpty#82] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#83] -Results [4]: [i_category#20, i_class#19, i_brand#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))#83 AS sumsales#33] +Input [10]: [i_category#121, i_class#122, i_brand#123, i_product_name#124, d_year#125, d_qoy#126, d_moy#127, s_store_id#128, sum#129, isEmpty#130] +Keys [8]: [i_category#121, i_class#122, i_brand#123, i_product_name#124, d_year#125, d_qoy#126, d_moy#127, s_store_id#128] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#131 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#132 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#131 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#132 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#133] +Results [4]: [i_category#121, i_class#122, i_brand#123, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#131 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#132 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#133 AS sumsales#43] (51) HashAggregate [codegen id : 52] -Input [4]: [i_category#20, i_class#19, i_brand#18, sumsales#33] -Keys [3]: [i_category#20, i_class#19, i_brand#18] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#84, isEmpty#85] -Results [5]: [i_category#20, i_class#19, i_brand#18, sum#86, isEmpty#87] +Input [4]: [i_category#121, i_class#122, i_brand#123, sumsales#43] +Keys [3]: [i_category#121, i_class#122, i_brand#123] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#134, isEmpty#135] +Results [5]: [i_category#121, i_class#122, i_brand#123, sum#136, isEmpty#137] (52) Exchange -Input [5]: [i_category#20, i_class#19, i_brand#18, sum#86, isEmpty#87] -Arguments: hashpartitioning(i_category#20, i_class#19, i_brand#18, 5), ENSURE_REQUIREMENTS, [id=#88] +Input [5]: [i_category#121, i_class#122, i_brand#123, sum#136, isEmpty#137] +Arguments: hashpartitioning(i_category#121, i_class#122, i_brand#123, 5), ENSURE_REQUIREMENTS, [id=#138] (53) HashAggregate [codegen id : 53] -Input [5]: [i_category#20, i_class#19, i_brand#18, sum#86, isEmpty#87] -Keys [3]: [i_category#20, i_class#19, i_brand#18] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#89] -Results [9]: [i_category#20, i_class#19, i_brand#18, null AS i_product_name#90, null AS d_year#91, null AS d_qoy#92, null AS d_moy#93, null AS s_store_id#94, sum(sumsales#33)#89 AS sumsales#95] +Input [5]: [i_category#121, i_class#122, i_brand#123, sum#136, isEmpty#137] +Keys [3]: [i_category#121, i_class#122, i_brand#123] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#139] +Results [9]: [i_category#121, i_class#122, i_brand#123, null AS i_product_name#140, null AS d_year#141, null AS d_qoy#142, null AS d_moy#143, null AS s_store_id#144, sum(sumsales#43)#139 AS sumsales#145] (54) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#96, isEmpty#97] +Output [10]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sum#154, isEmpty#155] (55) HashAggregate [codegen id : 61] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#96, isEmpty#97] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#98] -Results [3]: [i_category#20, i_class#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))#98 AS sumsales#33] +Input [10]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153, sum#154, isEmpty#155] +Keys [8]: [i_category#146, i_class#147, i_brand#148, i_product_name#149, d_year#150, d_qoy#151, d_moy#152, s_store_id#153] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#156 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#157 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#156 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#157 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#158] +Results [3]: [i_category#146, i_class#147, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#156 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#157 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#158 AS sumsales#43] (56) HashAggregate [codegen id : 61] -Input [3]: [i_category#20, i_class#19, sumsales#33] -Keys [2]: [i_category#20, i_class#19] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#99, isEmpty#100] -Results [4]: [i_category#20, i_class#19, sum#101, isEmpty#102] +Input [3]: [i_category#146, i_class#147, sumsales#43] +Keys [2]: [i_category#146, i_class#147] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#159, isEmpty#160] +Results [4]: [i_category#146, i_class#147, sum#161, isEmpty#162] (57) Exchange -Input [4]: [i_category#20, i_class#19, sum#101, isEmpty#102] -Arguments: hashpartitioning(i_category#20, i_class#19, 5), ENSURE_REQUIREMENTS, [id=#103] +Input [4]: [i_category#146, i_class#147, sum#161, isEmpty#162] +Arguments: hashpartitioning(i_category#146, i_class#147, 5), ENSURE_REQUIREMENTS, [id=#163] (58) HashAggregate [codegen id : 62] -Input [4]: [i_category#20, i_class#19, sum#101, isEmpty#102] -Keys [2]: [i_category#20, i_class#19] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#104] -Results [9]: [i_category#20, i_class#19, null AS i_brand#105, null AS i_product_name#106, null AS d_year#107, null AS d_qoy#108, null AS d_moy#109, null AS s_store_id#110, sum(sumsales#33)#104 AS sumsales#111] +Input [4]: [i_category#146, i_class#147, sum#161, isEmpty#162] +Keys [2]: [i_category#146, i_class#147] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#164] +Results [9]: [i_category#146, i_class#147, null AS i_brand#165, null AS i_product_name#166, null AS d_year#167, null AS d_qoy#168, null AS d_moy#169, null AS s_store_id#170, sum(sumsales#43)#164 AS sumsales#171] (59) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#112, isEmpty#113] +Output [10]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179, sum#180, isEmpty#181] (60) HashAggregate [codegen id : 70] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#112, isEmpty#113] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#114] -Results [2]: [i_category#20, 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))#114 AS sumsales#33] +Input [10]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179, sum#180, isEmpty#181] +Keys [8]: [i_category#172, i_class#173, i_brand#174, i_product_name#175, d_year#176, d_qoy#177, d_moy#178, s_store_id#179] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#182 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#183 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#182 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#183 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#184] +Results [2]: [i_category#172, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#182 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#183 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#184 AS sumsales#43] (61) HashAggregate [codegen id : 70] -Input [2]: [i_category#20, sumsales#33] -Keys [1]: [i_category#20] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#115, isEmpty#116] -Results [3]: [i_category#20, sum#117, isEmpty#118] +Input [2]: [i_category#172, sumsales#43] +Keys [1]: [i_category#172] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#185, isEmpty#186] +Results [3]: [i_category#172, sum#187, isEmpty#188] (62) Exchange -Input [3]: [i_category#20, sum#117, isEmpty#118] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#119] +Input [3]: [i_category#172, sum#187, isEmpty#188] +Arguments: hashpartitioning(i_category#172, 5), ENSURE_REQUIREMENTS, [id=#189] (63) HashAggregate [codegen id : 71] -Input [3]: [i_category#20, sum#117, isEmpty#118] -Keys [1]: [i_category#20] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#120] -Results [9]: [i_category#20, null AS i_class#121, null AS i_brand#122, null AS i_product_name#123, null AS d_year#124, null AS d_qoy#125, null AS d_moy#126, null AS s_store_id#127, sum(sumsales#33)#120 AS sumsales#128] +Input [3]: [i_category#172, sum#187, isEmpty#188] +Keys [1]: [i_category#172] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#190] +Results [9]: [i_category#172, null AS i_class#191, null AS i_brand#192, null AS i_product_name#193, null AS d_year#194, null AS d_qoy#195, null AS d_moy#196, null AS s_store_id#197, sum(sumsales#43)#190 AS sumsales#198] (64) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#129, isEmpty#130] +Output [10]: [i_category#199, i_class#200, i_brand#201, i_product_name#202, d_year#203, d_qoy#204, d_moy#205, s_store_id#206, sum#207, isEmpty#208] (65) HashAggregate [codegen id : 79] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#129, isEmpty#130] -Keys [8]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#131] -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))#131 AS sumsales#33] +Input [10]: [i_category#199, i_class#200, i_brand#201, i_product_name#202, d_year#203, d_qoy#204, d_moy#205, s_store_id#206, sum#207, isEmpty#208] +Keys [8]: [i_category#199, i_class#200, i_brand#201, i_product_name#202, d_year#203, d_qoy#204, d_moy#205, s_store_id#206] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#209 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#210 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#209 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#210 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#211] +Results [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#209 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#210 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#211 AS sumsales#43] (66) HashAggregate [codegen id : 79] -Input [1]: [sumsales#33] +Input [1]: [sumsales#43] Keys: [] -Functions [1]: [partial_sum(sumsales#33)] -Aggregate Attributes [2]: [sum#132, isEmpty#133] -Results [2]: [sum#134, isEmpty#135] +Functions [1]: [partial_sum(sumsales#43)] +Aggregate Attributes [2]: [sum#212, isEmpty#213] +Results [2]: [sum#214, isEmpty#215] (67) Exchange -Input [2]: [sum#134, isEmpty#135] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#136] +Input [2]: [sum#214, isEmpty#215] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#216] (68) HashAggregate [codegen id : 80] -Input [2]: [sum#134, isEmpty#135] +Input [2]: [sum#214, isEmpty#215] Keys: [] -Functions [1]: [sum(sumsales#33)] -Aggregate Attributes [1]: [sum(sumsales#33)#137] -Results [9]: [null AS i_category#138, null AS i_class#139, null AS i_brand#140, null AS i_product_name#141, null AS d_year#142, null AS d_qoy#143, null AS d_moy#144, null AS s_store_id#145, sum(sumsales#33)#137 AS sumsales#146] +Functions [1]: [sum(sumsales#43)] +Aggregate Attributes [1]: [sum(sumsales#43)#217] +Results [9]: [null AS i_category#218, null AS i_class#219, null AS i_brand#220, null AS i_product_name#221, null AS d_year#222, null AS d_qoy#223, null AS d_moy#224, null AS s_store_id#225, sum(sumsales#43)#217 AS sumsales#226] (69) Union (70) Exchange Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#147] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#227] (71) Sort [codegen id : 81] Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29] @@ -441,15 +441,15 @@ Arguments: [i_category#20 ASC NULLS FIRST, sumsales#29 DESC NULLS LAST], false, (72) Window Input [9]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29] -Arguments: [rank(sumsales#29) windowspecdefinition(i_category#20, sumsales#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#148], [i_category#20], [sumsales#29 DESC NULLS LAST] +Arguments: [rank(sumsales#29) windowspecdefinition(i_category#20, sumsales#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#228], [i_category#20], [sumsales#29 DESC NULLS LAST] (73) Filter [codegen id : 82] -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] -Condition : (rk#148 <= 100) +Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#228] +Condition : (rk#228 <= 100) (74) TakeOrderedAndProject -Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] -Arguments: 100, [i_category#20 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#18 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#14 ASC NULLS FIRST, sumsales#29 ASC NULLS FIRST, rk#148 ASC NULLS FIRST], [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#148] +Input [10]: [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#228] +Arguments: 100, [i_category#20 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#18 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#14 ASC NULLS FIRST, sumsales#29 ASC NULLS FIRST, rk#228 ASC NULLS FIRST], [i_category#20, i_class#19, i_brand#18, i_product_name#21, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#29, rk#228] ===== Subqueries ===== 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 fa822f3ac9ed7..e10a5ea0aff0b 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 @@ -191,234 +191,234 @@ Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, 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))#27 as decimal(38,2)) AS sumsales#28] (26) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#29, isEmpty#30] +Output [10]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, s_store_id#36, sum#37, isEmpty#38] (27) HashAggregate [codegen id : 10] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#29, isEmpty#30] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#31] -Results [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#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))#31 AS sumsales#32] +Input [10]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, s_store_id#36, sum#37, isEmpty#38] +Keys [8]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, s_store_id#36] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#39 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#40 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#39 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#40 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#41] +Results [8]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#39 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#40 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#41 AS sumsales#42] (28) HashAggregate [codegen id : 10] -Input [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, sumsales#32] -Keys [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, sum#35, isEmpty#36] +Input [8]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sumsales#42] +Keys [7]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#43, isEmpty#44] +Results [9]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sum#45, isEmpty#46] (29) Exchange -Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, sum#35, isEmpty#36] -Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [9]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sum#45, isEmpty#46] +Arguments: hashpartitioning(i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, 5), ENSURE_REQUIREMENTS, [id=#47] (30) HashAggregate [codegen id : 11] -Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, sum#35, isEmpty#36] -Keys [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#38] -Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, null AS s_store_id#39, sum(sumsales#32)#38 AS sumsales#40] +Input [9]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, sum#45, isEmpty#46] +Keys [7]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#48] +Results [9]: [i_category#29, i_class#30, i_brand#31, i_product_name#32, d_year#33, d_qoy#34, d_moy#35, null AS s_store_id#49, sum(sumsales#42)#48 AS sumsales#50] (31) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#41, isEmpty#42] +Output [10]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, d_moy#57, s_store_id#58, sum#59, isEmpty#60] (32) HashAggregate [codegen id : 16] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#41, isEmpty#42] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#43] -Results [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, 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))#43 AS sumsales#32] +Input [10]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, d_moy#57, s_store_id#58, sum#59, isEmpty#60] +Keys [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, d_moy#57, s_store_id#58] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#61 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#62 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#61 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#62 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#63] +Results [7]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#61 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#62 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#63 AS sumsales#42] (33) HashAggregate [codegen id : 16] -Input [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, sumsales#32] -Keys [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#44, isEmpty#45] -Results [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, sum#46, isEmpty#47] +Input [7]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sumsales#42] +Keys [6]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#64, isEmpty#65] +Results [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sum#66, isEmpty#67] (34) Exchange -Input [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, sum#46, isEmpty#47] -Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sum#66, isEmpty#67] +Arguments: hashpartitioning(i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, 5), ENSURE_REQUIREMENTS, [id=#68] (35) HashAggregate [codegen id : 17] -Input [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, sum#46, isEmpty#47] -Keys [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#49] -Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, null AS d_moy#50, null AS s_store_id#51, sum(sumsales#32)#49 AS sumsales#52] +Input [8]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, sum#66, isEmpty#67] +Keys [6]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#69] +Results [9]: [i_category#51, i_class#52, i_brand#53, i_product_name#54, d_year#55, d_qoy#56, null AS d_moy#70, null AS s_store_id#71, sum(sumsales#42)#69 AS sumsales#72] (36) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#53, isEmpty#54] +Output [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] (37) HashAggregate [codegen id : 22] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#53, isEmpty#54] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#55] -Results [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#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))#55 AS sumsales#32] +Input [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] +Keys [8]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#83 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#84 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#83 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#85] +Results [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#83 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#84 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#85 AS sumsales#42] (38) HashAggregate [codegen id : 22] -Input [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, sumsales#32] -Keys [5]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#56, isEmpty#57] -Results [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, sum#58, isEmpty#59] +Input [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sumsales#42] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#86, isEmpty#87] +Results [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] (39) Exchange -Input [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, sum#58, isEmpty#59] -Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Arguments: hashpartitioning(i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, 5), ENSURE_REQUIREMENTS, [id=#90] (40) HashAggregate [codegen id : 23] -Input [7]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, sum#58, isEmpty#59] -Keys [5]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#61] -Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, null AS d_qoy#62, null AS d_moy#63, null AS s_store_id#64, sum(sumsales#32)#61 AS sumsales#65] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#91] +Results [9]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, null AS d_qoy#92, null AS d_moy#93, null AS s_store_id#94, sum(sumsales#42)#91 AS sumsales#95] (41) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#66, isEmpty#67] +Output [10]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, d_year#100, d_qoy#101, d_moy#102, s_store_id#103, sum#104, isEmpty#105] (42) HashAggregate [codegen id : 28] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#66, isEmpty#67] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#68] -Results [5]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, 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))#68 AS sumsales#32] +Input [10]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, d_year#100, d_qoy#101, d_moy#102, s_store_id#103, sum#104, isEmpty#105] +Keys [8]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, d_year#100, d_qoy#101, d_moy#102, s_store_id#103] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#106 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#107 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#106 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#107 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#108] +Results [5]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#106 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#107 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#108 AS sumsales#42] (43) HashAggregate [codegen id : 28] -Input [5]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, sumsales#32] -Keys [4]: [i_category#19, i_class#18, i_brand#17, i_product_name#20] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#69, isEmpty#70] -Results [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, sum#71, isEmpty#72] +Input [5]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sumsales#42] +Keys [4]: [i_category#96, i_class#97, i_brand#98, i_product_name#99] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#109, isEmpty#110] +Results [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] (44) Exchange -Input [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, sum#71, isEmpty#72] -Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, i_product_name#20, 5), ENSURE_REQUIREMENTS, [id=#73] +Input [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] +Arguments: hashpartitioning(i_category#96, i_class#97, i_brand#98, i_product_name#99, 5), ENSURE_REQUIREMENTS, [id=#113] (45) HashAggregate [codegen id : 29] -Input [6]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, sum#71, isEmpty#72] -Keys [4]: [i_category#19, i_class#18, i_brand#17, i_product_name#20] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#74] -Results [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, null AS d_year#75, null AS d_qoy#76, null AS d_moy#77, null AS s_store_id#78, sum(sumsales#32)#74 AS sumsales#79] +Input [6]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, sum#111, isEmpty#112] +Keys [4]: [i_category#96, i_class#97, i_brand#98, i_product_name#99] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#114] +Results [9]: [i_category#96, i_class#97, i_brand#98, i_product_name#99, null AS d_year#115, null AS d_qoy#116, null AS d_moy#117, null AS s_store_id#118, sum(sumsales#42)#114 AS sumsales#119] (46) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#80, isEmpty#81] +Output [10]: [i_category#120, i_class#121, i_brand#122, i_product_name#123, d_year#124, d_qoy#125, d_moy#126, s_store_id#127, sum#128, isEmpty#129] (47) HashAggregate [codegen id : 34] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#80, isEmpty#81] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#82] -Results [4]: [i_category#19, i_class#18, i_brand#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))#82 AS sumsales#32] +Input [10]: [i_category#120, i_class#121, i_brand#122, i_product_name#123, d_year#124, d_qoy#125, d_moy#126, s_store_id#127, sum#128, isEmpty#129] +Keys [8]: [i_category#120, i_class#121, i_brand#122, i_product_name#123, d_year#124, d_qoy#125, d_moy#126, s_store_id#127] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#130 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#131 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#130 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#131 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#132] +Results [4]: [i_category#120, i_class#121, i_brand#122, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#130 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#131 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#132 AS sumsales#42] (48) HashAggregate [codegen id : 34] -Input [4]: [i_category#19, i_class#18, i_brand#17, sumsales#32] -Keys [3]: [i_category#19, i_class#18, i_brand#17] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#83, isEmpty#84] -Results [5]: [i_category#19, i_class#18, i_brand#17, sum#85, isEmpty#86] +Input [4]: [i_category#120, i_class#121, i_brand#122, sumsales#42] +Keys [3]: [i_category#120, i_class#121, i_brand#122] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#133, isEmpty#134] +Results [5]: [i_category#120, i_class#121, i_brand#122, sum#135, isEmpty#136] (49) Exchange -Input [5]: [i_category#19, i_class#18, i_brand#17, sum#85, isEmpty#86] -Arguments: hashpartitioning(i_category#19, i_class#18, i_brand#17, 5), ENSURE_REQUIREMENTS, [id=#87] +Input [5]: [i_category#120, i_class#121, i_brand#122, sum#135, isEmpty#136] +Arguments: hashpartitioning(i_category#120, i_class#121, i_brand#122, 5), ENSURE_REQUIREMENTS, [id=#137] (50) HashAggregate [codegen id : 35] -Input [5]: [i_category#19, i_class#18, i_brand#17, sum#85, isEmpty#86] -Keys [3]: [i_category#19, i_class#18, i_brand#17] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#88] -Results [9]: [i_category#19, i_class#18, i_brand#17, null AS i_product_name#89, null AS d_year#90, null AS d_qoy#91, null AS d_moy#92, null AS s_store_id#93, sum(sumsales#32)#88 AS sumsales#94] +Input [5]: [i_category#120, i_class#121, i_brand#122, sum#135, isEmpty#136] +Keys [3]: [i_category#120, i_class#121, i_brand#122] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#138] +Results [9]: [i_category#120, i_class#121, i_brand#122, null AS i_product_name#139, null AS d_year#140, null AS d_qoy#141, null AS d_moy#142, null AS s_store_id#143, sum(sumsales#42)#138 AS sumsales#144] (51) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#95, isEmpty#96] +Output [10]: [i_category#145, i_class#146, i_brand#147, i_product_name#148, d_year#149, d_qoy#150, d_moy#151, s_store_id#152, sum#153, isEmpty#154] (52) HashAggregate [codegen id : 40] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#95, isEmpty#96] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#97] -Results [3]: [i_category#19, i_class#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))#97 AS sumsales#32] +Input [10]: [i_category#145, i_class#146, i_brand#147, i_product_name#148, d_year#149, d_qoy#150, d_moy#151, s_store_id#152, sum#153, isEmpty#154] +Keys [8]: [i_category#145, i_class#146, i_brand#147, i_product_name#148, d_year#149, d_qoy#150, d_moy#151, s_store_id#152] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#155 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#156 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#155 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#156 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#157] +Results [3]: [i_category#145, i_class#146, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#155 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#156 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#157 AS sumsales#42] (53) HashAggregate [codegen id : 40] -Input [3]: [i_category#19, i_class#18, sumsales#32] -Keys [2]: [i_category#19, i_class#18] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#98, isEmpty#99] -Results [4]: [i_category#19, i_class#18, sum#100, isEmpty#101] +Input [3]: [i_category#145, i_class#146, sumsales#42] +Keys [2]: [i_category#145, i_class#146] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#158, isEmpty#159] +Results [4]: [i_category#145, i_class#146, sum#160, isEmpty#161] (54) Exchange -Input [4]: [i_category#19, i_class#18, sum#100, isEmpty#101] -Arguments: hashpartitioning(i_category#19, i_class#18, 5), ENSURE_REQUIREMENTS, [id=#102] +Input [4]: [i_category#145, i_class#146, sum#160, isEmpty#161] +Arguments: hashpartitioning(i_category#145, i_class#146, 5), ENSURE_REQUIREMENTS, [id=#162] (55) HashAggregate [codegen id : 41] -Input [4]: [i_category#19, i_class#18, sum#100, isEmpty#101] -Keys [2]: [i_category#19, i_class#18] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#103] -Results [9]: [i_category#19, i_class#18, null AS i_brand#104, null AS i_product_name#105, null AS d_year#106, null AS d_qoy#107, null AS d_moy#108, null AS s_store_id#109, sum(sumsales#32)#103 AS sumsales#110] +Input [4]: [i_category#145, i_class#146, sum#160, isEmpty#161] +Keys [2]: [i_category#145, i_class#146] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#163] +Results [9]: [i_category#145, i_class#146, null AS i_brand#164, null AS i_product_name#165, null AS d_year#166, null AS d_qoy#167, null AS d_moy#168, null AS s_store_id#169, sum(sumsales#42)#163 AS sumsales#170] (56) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#111, isEmpty#112] +Output [10]: [i_category#171, i_class#172, i_brand#173, i_product_name#174, d_year#175, d_qoy#176, d_moy#177, s_store_id#178, sum#179, isEmpty#180] (57) HashAggregate [codegen id : 46] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#111, isEmpty#112] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#113] -Results [2]: [i_category#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))#113 AS sumsales#32] +Input [10]: [i_category#171, i_class#172, i_brand#173, i_product_name#174, d_year#175, d_qoy#176, d_moy#177, s_store_id#178, sum#179, isEmpty#180] +Keys [8]: [i_category#171, i_class#172, i_brand#173, i_product_name#174, d_year#175, d_qoy#176, d_moy#177, s_store_id#178] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#181 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#182 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#181 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#182 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#183] +Results [2]: [i_category#171, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#181 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#182 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#183 AS sumsales#42] (58) HashAggregate [codegen id : 46] -Input [2]: [i_category#19, sumsales#32] -Keys [1]: [i_category#19] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#114, isEmpty#115] -Results [3]: [i_category#19, sum#116, isEmpty#117] +Input [2]: [i_category#171, sumsales#42] +Keys [1]: [i_category#171] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#184, isEmpty#185] +Results [3]: [i_category#171, sum#186, isEmpty#187] (59) Exchange -Input [3]: [i_category#19, sum#116, isEmpty#117] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, [id=#118] +Input [3]: [i_category#171, sum#186, isEmpty#187] +Arguments: hashpartitioning(i_category#171, 5), ENSURE_REQUIREMENTS, [id=#188] (60) HashAggregate [codegen id : 47] -Input [3]: [i_category#19, sum#116, isEmpty#117] -Keys [1]: [i_category#19] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#119] -Results [9]: [i_category#19, null AS i_class#120, null AS i_brand#121, null AS i_product_name#122, null AS d_year#123, null AS d_qoy#124, null AS d_moy#125, null AS s_store_id#126, sum(sumsales#32)#119 AS sumsales#127] +Input [3]: [i_category#171, sum#186, isEmpty#187] +Keys [1]: [i_category#171] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#189] +Results [9]: [i_category#171, null AS i_class#190, null AS i_brand#191, null AS i_product_name#192, null AS d_year#193, null AS d_qoy#194, null AS d_moy#195, null AS s_store_id#196, sum(sumsales#42)#189 AS sumsales#197] (61) ReusedExchange [Reuses operator id: unknown] -Output [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#128, isEmpty#129] +Output [10]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205, sum#206, isEmpty#207] (62) HashAggregate [codegen id : 52] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sum#128, isEmpty#129] -Keys [8]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14] -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))#130] -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))#130 AS sumsales#32] +Input [10]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205, sum#206, isEmpty#207] +Keys [8]: [i_category#198, i_class#199, i_brand#200, i_product_name#201, d_year#202, d_qoy#203, d_moy#204, s_store_id#205] +Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#208 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#209 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#208 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#209 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#210] +Results [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#208 as decimal(12,2))) * promote_precision(cast(cast(ss_quantity#209 as decimal(10,0)) as decimal(12,2)))), DecimalType(18,2), true), 0.00))#210 AS sumsales#42] (63) HashAggregate [codegen id : 52] -Input [1]: [sumsales#32] +Input [1]: [sumsales#42] Keys: [] -Functions [1]: [partial_sum(sumsales#32)] -Aggregate Attributes [2]: [sum#131, isEmpty#132] -Results [2]: [sum#133, isEmpty#134] +Functions [1]: [partial_sum(sumsales#42)] +Aggregate Attributes [2]: [sum#211, isEmpty#212] +Results [2]: [sum#213, isEmpty#214] (64) Exchange -Input [2]: [sum#133, isEmpty#134] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#135] +Input [2]: [sum#213, isEmpty#214] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#215] (65) HashAggregate [codegen id : 53] -Input [2]: [sum#133, isEmpty#134] +Input [2]: [sum#213, isEmpty#214] Keys: [] -Functions [1]: [sum(sumsales#32)] -Aggregate Attributes [1]: [sum(sumsales#32)#136] -Results [9]: [null AS i_category#137, null AS i_class#138, null AS i_brand#139, null AS i_product_name#140, null AS d_year#141, null AS d_qoy#142, null AS d_moy#143, null AS s_store_id#144, sum(sumsales#32)#136 AS sumsales#145] +Functions [1]: [sum(sumsales#42)] +Aggregate Attributes [1]: [sum(sumsales#42)#216] +Results [9]: [null AS i_category#217, null AS i_class#218, null AS i_brand#219, null AS i_product_name#220, null AS d_year#221, null AS d_qoy#222, null AS d_moy#223, null AS s_store_id#224, sum(sumsales#42)#216 AS sumsales#225] (66) Union (67) Exchange Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28] -Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, [id=#146] +Arguments: hashpartitioning(i_category#19, 5), ENSURE_REQUIREMENTS, [id=#226] (68) Sort [codegen id : 54] Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28] @@ -426,15 +426,15 @@ Arguments: [i_category#19 ASC NULLS FIRST, sumsales#28 DESC NULLS LAST], false, (69) Window Input [9]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28] -Arguments: [rank(sumsales#28) windowspecdefinition(i_category#19, sumsales#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#147], [i_category#19], [sumsales#28 DESC NULLS LAST] +Arguments: [rank(sumsales#28) windowspecdefinition(i_category#19, sumsales#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#227], [i_category#19], [sumsales#28 DESC NULLS LAST] (70) Filter [codegen id : 55] -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] -Condition : (rk#147 <= 100) +Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#227] +Condition : (rk#227 <= 100) (71) TakeOrderedAndProject -Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] -Arguments: 100, [i_category#19 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_product_name#20 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#14 ASC NULLS FIRST, sumsales#28 ASC NULLS FIRST, rk#147 ASC NULLS FIRST], [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#147] +Input [10]: [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#227] +Arguments: 100, [i_category#19 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_product_name#20 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#11 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, s_store_id#14 ASC NULLS FIRST, sumsales#28 ASC NULLS FIRST, rk#227 ASC NULLS FIRST], [i_category#19, i_class#18, i_brand#17, i_product_name#20, d_year#9, d_qoy#11, d_moy#10, s_store_id#14, sumsales#28, rk#227] ===== Subqueries ===== 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 e024d06c710a7..35ed13e84c742 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 @@ -122,109 +122,109 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] (16) Filter [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_store_sk#11) (17) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#14] (18) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#13] +Right keys [1]: [d_date_sk#14] Join condition: None (19) Project [codegen id : 4] -Output [2]: [ss_store_sk#1, ss_net_profit#2] -Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] +Output [2]: [ss_store_sk#11, ss_net_profit#12] +Input [4]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13, d_date_sk#14] (20) Scan parquet default.store -Output [2]: [s_store_sk#8, s_state#10] +Output [2]: [s_store_sk#15, s_state#16] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (21) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#8, s_state#10] +Input [2]: [s_store_sk#15, s_state#16] (22) Filter [codegen id : 3] -Input [2]: [s_store_sk#8, s_state#10] -Condition : isnotnull(s_store_sk#8) +Input [2]: [s_store_sk#15, s_state#16] +Condition : isnotnull(s_store_sk#15) (23) BroadcastExchange -Input [2]: [s_store_sk#8, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +Input [2]: [s_store_sk#15, s_state#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#8] +Left keys [1]: [ss_store_sk#11] +Right keys [1]: [s_store_sk#15] Join condition: None (25) Project [codegen id : 4] -Output [2]: [ss_net_profit#2, s_state#10] -Input [4]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_state#10] +Output [2]: [ss_net_profit#12, s_state#16] +Input [4]: [ss_store_sk#11, ss_net_profit#12, s_store_sk#15, s_state#16] (26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#2, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#12] -Results [2]: [s_state#10, sum#13] +Input [2]: [ss_net_profit#12, s_state#16] +Keys [1]: [s_state#16] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum#18] +Results [2]: [s_state#16, sum#19] (27) Exchange -Input [2]: [s_state#10, sum#13] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [2]: [s_state#16, sum#19] +Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, [id=#20] (28) HashAggregate [codegen id : 5] -Input [2]: [s_state#10, sum#13] -Keys [1]: [s_state#10] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#15] -Results [3]: [s_state#10 AS s_state#16, s_state#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#15,17,2) AS _w2#17] +Input [2]: [s_state#16, sum#19] +Keys [1]: [s_state#16] +Functions [1]: [sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#12))#21] +Results [3]: [s_state#16 AS s_state#22, s_state#16, MakeDecimal(sum(UnscaledValue(ss_net_profit#12))#21,17,2) AS _w2#23] (29) Exchange -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, [id=#24] (30) Sort [codegen id : 6] -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [s_state#10 ASC NULLS FIRST, _w2#17 DESC NULLS LAST], false, 0 +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: [s_state#16 ASC NULLS FIRST, _w2#23 DESC NULLS LAST], false, 0 (31) Window -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#10], [_w2#17 DESC NULLS LAST] +Input [3]: [s_state#22, s_state#16, _w2#23] +Arguments: [rank(_w2#23) windowspecdefinition(s_state#16, _w2#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#25], [s_state#16], [_w2#23 DESC NULLS LAST] (32) Filter [codegen id : 7] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (ranking#19 <= 5) +Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] +Condition : (ranking#25 <= 5) (33) Project [codegen id : 7] -Output [1]: [s_state#16] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] +Output [1]: [s_state#22] +Input [4]: [s_state#22, s_state#16, _w2#23, ranking#25] (34) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#20] +Input [1]: [s_state#22] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#26] (35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#10] -Right keys [1]: [s_state#16] +Right keys [1]: [s_state#22] Join condition: None (36) BroadcastExchange Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] @@ -239,115 +239,115 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_county#9, s_state#10 Input [3]: [ss_net_profit#2, s_county#9, s_state#10] Keys [2]: [s_state#10, s_county#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#10, s_county#9, sum#23] +Aggregate Attributes [1]: [sum#28] +Results [3]: [s_state#10, s_county#9, sum#29] (40) Exchange -Input [3]: [s_state#10, s_county#9, sum#23] -Arguments: hashpartitioning(s_state#10, s_county#9, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [s_state#10, s_county#9, sum#29] +Arguments: hashpartitioning(s_state#10, s_county#9, 5), ENSURE_REQUIREMENTS, [id=#30] (41) HashAggregate [codegen id : 10] -Input [3]: [s_state#10, s_county#9, sum#23] +Input [3]: [s_state#10, s_county#9, sum#29] Keys [2]: [s_state#10, s_county#9] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) as decimal(27,2)) AS total_sum#26, s_state#10, s_county#9, 0 AS g_state#27, 0 AS g_county#28, 0 AS lochierarchy#29] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) as decimal(27,2)) AS total_sum#32, s_state#10, s_county#9, 0 AS g_state#33, 0 AS g_county#34, 0 AS lochierarchy#35] (42) ReusedExchange [Reuses operator id: unknown] -Output [3]: [s_state#10, s_county#9, sum#30] +Output [3]: [s_state#36, s_county#37, sum#38] (43) HashAggregate [codegen id : 20] -Input [3]: [s_state#10, s_county#9, sum#30] -Keys [2]: [s_state#10, s_county#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS total_sum#32, s_state#10] +Input [3]: [s_state#36, s_county#37, sum#38] +Keys [2]: [s_state#36, s_county#37] +Functions [1]: [sum(UnscaledValue(ss_net_profit#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#39))#40] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#39))#40,17,2) AS total_sum#41, s_state#36] (44) HashAggregate [codegen id : 20] -Input [2]: [total_sum#32, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(total_sum#32)] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [3]: [s_state#10, sum#35, isEmpty#36] +Input [2]: [total_sum#41, s_state#36] +Keys [1]: [s_state#36] +Functions [1]: [partial_sum(total_sum#41)] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [s_state#36, sum#44, isEmpty#45] (45) Exchange -Input [3]: [s_state#10, sum#35, isEmpty#36] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [s_state#36, sum#44, isEmpty#45] +Arguments: hashpartitioning(s_state#36, 5), ENSURE_REQUIREMENTS, [id=#46] (46) HashAggregate [codegen id : 21] -Input [3]: [s_state#10, sum#35, isEmpty#36] -Keys [1]: [s_state#10] -Functions [1]: [sum(total_sum#32)] -Aggregate Attributes [1]: [sum(total_sum#32)#38] -Results [6]: [sum(total_sum#32)#38 AS total_sum#39, s_state#10, null AS s_county#40, 0 AS g_state#41, 1 AS g_county#42, 1 AS lochierarchy#43] +Input [3]: [s_state#36, sum#44, isEmpty#45] +Keys [1]: [s_state#36] +Functions [1]: [sum(total_sum#41)] +Aggregate Attributes [1]: [sum(total_sum#41)#47] +Results [6]: [sum(total_sum#41)#47 AS total_sum#48, s_state#36, null AS s_county#49, 0 AS g_state#50, 1 AS g_county#51, 1 AS lochierarchy#52] (47) ReusedExchange [Reuses operator id: unknown] -Output [3]: [s_state#10, s_county#9, sum#44] +Output [3]: [s_state#53, s_county#54, sum#55] (48) HashAggregate [codegen id : 31] -Input [3]: [s_state#10, s_county#9, sum#44] -Keys [2]: [s_state#10, s_county#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#45] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#45,17,2) AS total_sum#32] +Input [3]: [s_state#53, s_county#54, sum#55] +Keys [2]: [s_state#53, s_county#54] +Functions [1]: [sum(UnscaledValue(ss_net_profit#56))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#56))#57] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#56))#57,17,2) AS total_sum#41] (49) HashAggregate [codegen id : 31] -Input [1]: [total_sum#32] +Input [1]: [total_sum#41] Keys: [] -Functions [1]: [partial_sum(total_sum#32)] -Aggregate Attributes [2]: [sum#46, isEmpty#47] -Results [2]: [sum#48, isEmpty#49] +Functions [1]: [partial_sum(total_sum#41)] +Aggregate Attributes [2]: [sum#58, isEmpty#59] +Results [2]: [sum#60, isEmpty#61] (50) Exchange -Input [2]: [sum#48, isEmpty#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [sum#60, isEmpty#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] (51) HashAggregate [codegen id : 32] -Input [2]: [sum#48, isEmpty#49] +Input [2]: [sum#60, isEmpty#61] Keys: [] -Functions [1]: [sum(total_sum#32)] -Aggregate Attributes [1]: [sum(total_sum#32)#51] -Results [6]: [sum(total_sum#32)#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] +Functions [1]: [sum(total_sum#41)] +Aggregate Attributes [1]: [sum(total_sum#41)#63] +Results [6]: [sum(total_sum#41)#63 AS total_sum#64, null AS s_state#65, null AS s_county#66, 1 AS g_state#67, 1 AS g_county#68, 2 AS lochierarchy#69] (52) Union (53) HashAggregate [codegen id : 33] -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Keys [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Keys [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Results [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] (54) Exchange -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Arguments: hashpartitioning(total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Arguments: hashpartitioning(total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35, 5), ENSURE_REQUIREMENTS, [id=#70] (55) HashAggregate [codegen id : 34] -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Keys [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Keys [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, CASE WHEN (g_county#28 = 0) THEN s_state#10 END AS _w0#59] +Results [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, CASE WHEN (g_county#34 = 0) THEN s_state#10 END AS _w0#71] (56) Exchange -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: hashpartitioning(lochierarchy#29, _w0#59, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: hashpartitioning(lochierarchy#35, _w0#71, 5), ENSURE_REQUIREMENTS, [id=#72] (57) Sort [codegen id : 35] -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: [lochierarchy#29 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: [lochierarchy#35 ASC NULLS FIRST, _w0#71 ASC NULLS FIRST, total_sum#32 DESC NULLS LAST], false, 0 (58) Window -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#29, _w0#59, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#29, _w0#59], [total_sum#26 DESC NULLS LAST] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: [rank(total_sum#32) windowspecdefinition(lochierarchy#35, _w0#71, total_sum#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#73], [lochierarchy#35, _w0#71], [total_sum#32 DESC NULLS LAST] (59) Project [codegen id : 36] -Output [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] -Input [6]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59, rank_within_parent#61] +Output [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] +Input [6]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71, rank_within_parent#73] (60) TakeOrderedAndProject -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] -Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#10 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] +Arguments: 100, [lochierarchy#35 DESC NULLS LAST, CASE WHEN (lochierarchy#35 = 0) THEN s_state#10 END ASC NULLS FIRST, rank_within_parent#73 ASC NULLS FIRST], [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] ===== Subqueries ===== @@ -358,6 +358,6 @@ ReusedExchange (61) (61) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#5] -Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#13 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 5ae5fd82839cc..df0ee68806062 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 @@ -122,109 +122,109 @@ Input [3]: [s_store_sk#8, s_county#9, s_state#10] Condition : isnotnull(s_store_sk#8) (14) Scan parquet default.store_sales -Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Output [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct (15) ColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] (16) Filter [codegen id : 4] -Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_store_sk#1) +Input [3]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_store_sk#11) (17) Scan parquet default.store -Output [2]: [s_store_sk#8, s_state#10] +Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct (18) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#8, s_state#10] +Input [2]: [s_store_sk#14, s_state#15] (19) Filter [codegen id : 2] -Input [2]: [s_store_sk#8, s_state#10] -Condition : isnotnull(s_store_sk#8) +Input [2]: [s_store_sk#14, s_state#15] +Condition : isnotnull(s_store_sk#14) (20) BroadcastExchange -Input [2]: [s_store_sk#8, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#11] +Input [2]: [s_store_sk#14, s_state#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#16] (21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#8] +Left keys [1]: [ss_store_sk#11] +Right keys [1]: [s_store_sk#14] Join condition: None (22) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, ss_sold_date_sk#3, s_state#10] -Input [5]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, s_store_sk#8, s_state#10] +Output [3]: [ss_net_profit#12, ss_sold_date_sk#13, s_state#15] +Input [5]: [ss_store_sk#11, ss_net_profit#12, ss_sold_date_sk#13, s_store_sk#14, s_state#15] (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#17] (24) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#13] +Right keys [1]: [d_date_sk#17] Join condition: None (25) Project [codegen id : 4] -Output [2]: [ss_net_profit#2, s_state#10] -Input [4]: [ss_net_profit#2, ss_sold_date_sk#3, s_state#10, d_date_sk#5] +Output [2]: [ss_net_profit#12, s_state#15] +Input [4]: [ss_net_profit#12, ss_sold_date_sk#13, s_state#15, d_date_sk#17] (26) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#2, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#12] -Results [2]: [s_state#10, sum#13] +Input [2]: [ss_net_profit#12, s_state#15] +Keys [1]: [s_state#15] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum#18] +Results [2]: [s_state#15, sum#19] (27) Exchange -Input [2]: [s_state#10, sum#13] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] +Input [2]: [s_state#15, sum#19] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [id=#20] (28) HashAggregate [codegen id : 5] -Input [2]: [s_state#10, sum#13] -Keys [1]: [s_state#10] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#15] -Results [3]: [s_state#10 AS s_state#16, s_state#10, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#15,17,2) AS _w2#17] +Input [2]: [s_state#15, sum#19] +Keys [1]: [s_state#15] +Functions [1]: [sum(UnscaledValue(ss_net_profit#12))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#12))#21] +Results [3]: [s_state#15 AS s_state#22, s_state#15, MakeDecimal(sum(UnscaledValue(ss_net_profit#12))#21,17,2) AS _w2#23] (29) Exchange -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#18] +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: hashpartitioning(s_state#15, 5), ENSURE_REQUIREMENTS, [id=#24] (30) Sort [codegen id : 6] -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [s_state#10 ASC NULLS FIRST, _w2#17 DESC NULLS LAST], false, 0 +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: [s_state#15 ASC NULLS FIRST, _w2#23 DESC NULLS LAST], false, 0 (31) Window -Input [3]: [s_state#16, s_state#10, _w2#17] -Arguments: [rank(_w2#17) windowspecdefinition(s_state#10, _w2#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#10], [_w2#17 DESC NULLS LAST] +Input [3]: [s_state#22, s_state#15, _w2#23] +Arguments: [rank(_w2#23) windowspecdefinition(s_state#15, _w2#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#25], [s_state#15], [_w2#23 DESC NULLS LAST] (32) Filter [codegen id : 7] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] -Condition : (ranking#19 <= 5) +Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] +Condition : (ranking#25 <= 5) (33) Project [codegen id : 7] -Output [1]: [s_state#16] -Input [4]: [s_state#16, s_state#10, _w2#17, ranking#19] +Output [1]: [s_state#22] +Input [4]: [s_state#22, s_state#15, _w2#23, ranking#25] (34) BroadcastExchange -Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#20] +Input [1]: [s_state#22] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#26] (35) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_state#10] -Right keys [1]: [s_state#16] +Right keys [1]: [s_state#22] Join condition: None (36) BroadcastExchange Input [3]: [s_store_sk#8, s_county#9, s_state#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (37) BroadcastHashJoin [codegen id : 9] Left keys [1]: [ss_store_sk#1] @@ -239,115 +239,115 @@ Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#8, s_county#9, s_state#10 Input [3]: [ss_net_profit#2, s_county#9, s_state#10] Keys [2]: [s_state#10, s_county#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#10, s_county#9, sum#23] +Aggregate Attributes [1]: [sum#28] +Results [3]: [s_state#10, s_county#9, sum#29] (40) Exchange -Input [3]: [s_state#10, s_county#9, sum#23] -Arguments: hashpartitioning(s_state#10, s_county#9, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [3]: [s_state#10, s_county#9, sum#29] +Arguments: hashpartitioning(s_state#10, s_county#9, 5), ENSURE_REQUIREMENTS, [id=#30] (41) HashAggregate [codegen id : 10] -Input [3]: [s_state#10, s_county#9, sum#23] +Input [3]: [s_state#10, s_county#9, sum#29] Keys [2]: [s_state#10, s_county#9] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) as decimal(27,2)) AS total_sum#26, s_state#10, s_county#9, 0 AS g_state#27, 0 AS g_county#28, 0 AS lochierarchy#29] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) as decimal(27,2)) AS total_sum#32, s_state#10, s_county#9, 0 AS g_state#33, 0 AS g_county#34, 0 AS lochierarchy#35] (42) ReusedExchange [Reuses operator id: unknown] -Output [3]: [s_state#10, s_county#9, sum#30] +Output [3]: [s_state#36, s_county#37, sum#38] (43) HashAggregate [codegen id : 20] -Input [3]: [s_state#10, s_county#9, sum#30] -Keys [2]: [s_state#10, s_county#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#31] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31,17,2) AS total_sum#32, s_state#10] +Input [3]: [s_state#36, s_county#37, sum#38] +Keys [2]: [s_state#36, s_county#37] +Functions [1]: [sum(UnscaledValue(ss_net_profit#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#39))#40] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#39))#40,17,2) AS total_sum#41, s_state#36] (44) HashAggregate [codegen id : 20] -Input [2]: [total_sum#32, s_state#10] -Keys [1]: [s_state#10] -Functions [1]: [partial_sum(total_sum#32)] -Aggregate Attributes [2]: [sum#33, isEmpty#34] -Results [3]: [s_state#10, sum#35, isEmpty#36] +Input [2]: [total_sum#41, s_state#36] +Keys [1]: [s_state#36] +Functions [1]: [partial_sum(total_sum#41)] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [3]: [s_state#36, sum#44, isEmpty#45] (45) Exchange -Input [3]: [s_state#10, sum#35, isEmpty#36] -Arguments: hashpartitioning(s_state#10, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [3]: [s_state#36, sum#44, isEmpty#45] +Arguments: hashpartitioning(s_state#36, 5), ENSURE_REQUIREMENTS, [id=#46] (46) HashAggregate [codegen id : 21] -Input [3]: [s_state#10, sum#35, isEmpty#36] -Keys [1]: [s_state#10] -Functions [1]: [sum(total_sum#32)] -Aggregate Attributes [1]: [sum(total_sum#32)#38] -Results [6]: [sum(total_sum#32)#38 AS total_sum#39, s_state#10, null AS s_county#40, 0 AS g_state#41, 1 AS g_county#42, 1 AS lochierarchy#43] +Input [3]: [s_state#36, sum#44, isEmpty#45] +Keys [1]: [s_state#36] +Functions [1]: [sum(total_sum#41)] +Aggregate Attributes [1]: [sum(total_sum#41)#47] +Results [6]: [sum(total_sum#41)#47 AS total_sum#48, s_state#36, null AS s_county#49, 0 AS g_state#50, 1 AS g_county#51, 1 AS lochierarchy#52] (47) ReusedExchange [Reuses operator id: unknown] -Output [3]: [s_state#10, s_county#9, sum#44] +Output [3]: [s_state#53, s_county#54, sum#55] (48) HashAggregate [codegen id : 31] -Input [3]: [s_state#10, s_county#9, sum#44] -Keys [2]: [s_state#10, s_county#9] -Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#45] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#45,17,2) AS total_sum#32] +Input [3]: [s_state#53, s_county#54, sum#55] +Keys [2]: [s_state#53, s_county#54] +Functions [1]: [sum(UnscaledValue(ss_net_profit#56))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#56))#57] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#56))#57,17,2) AS total_sum#41] (49) HashAggregate [codegen id : 31] -Input [1]: [total_sum#32] +Input [1]: [total_sum#41] Keys: [] -Functions [1]: [partial_sum(total_sum#32)] -Aggregate Attributes [2]: [sum#46, isEmpty#47] -Results [2]: [sum#48, isEmpty#49] +Functions [1]: [partial_sum(total_sum#41)] +Aggregate Attributes [2]: [sum#58, isEmpty#59] +Results [2]: [sum#60, isEmpty#61] (50) Exchange -Input [2]: [sum#48, isEmpty#49] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#50] +Input [2]: [sum#60, isEmpty#61] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#62] (51) HashAggregate [codegen id : 32] -Input [2]: [sum#48, isEmpty#49] +Input [2]: [sum#60, isEmpty#61] Keys: [] -Functions [1]: [sum(total_sum#32)] -Aggregate Attributes [1]: [sum(total_sum#32)#51] -Results [6]: [sum(total_sum#32)#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] +Functions [1]: [sum(total_sum#41)] +Aggregate Attributes [1]: [sum(total_sum#41)#63] +Results [6]: [sum(total_sum#41)#63 AS total_sum#64, null AS s_state#65, null AS s_county#66, 1 AS g_state#67, 1 AS g_county#68, 2 AS lochierarchy#69] (52) Union (53) HashAggregate [codegen id : 33] -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Keys [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Keys [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] Functions: [] Aggregate Attributes: [] -Results [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Results [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] (54) Exchange -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Arguments: hashpartitioning(total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29, 5), ENSURE_REQUIREMENTS, [id=#58] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Arguments: hashpartitioning(total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35, 5), ENSURE_REQUIREMENTS, [id=#70] (55) HashAggregate [codegen id : 34] -Input [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] -Keys [6]: [total_sum#26, s_state#10, s_county#9, g_state#27, g_county#28, lochierarchy#29] +Input [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] +Keys [6]: [total_sum#32, s_state#10, s_county#9, g_state#33, g_county#34, lochierarchy#35] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, CASE WHEN (g_county#28 = 0) THEN s_state#10 END AS _w0#59] +Results [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, CASE WHEN (g_county#34 = 0) THEN s_state#10 END AS _w0#71] (56) Exchange -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: hashpartitioning(lochierarchy#29, _w0#59, 5), ENSURE_REQUIREMENTS, [id=#60] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: hashpartitioning(lochierarchy#35, _w0#71, 5), ENSURE_REQUIREMENTS, [id=#72] (57) Sort [codegen id : 35] -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: [lochierarchy#29 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: [lochierarchy#35 ASC NULLS FIRST, _w0#71 ASC NULLS FIRST, total_sum#32 DESC NULLS LAST], false, 0 (58) Window -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59] -Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#29, _w0#59, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#61], [lochierarchy#29, _w0#59], [total_sum#26 DESC NULLS LAST] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71] +Arguments: [rank(total_sum#32) windowspecdefinition(lochierarchy#35, _w0#71, total_sum#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#73], [lochierarchy#35, _w0#71], [total_sum#32 DESC NULLS LAST] (59) Project [codegen id : 36] -Output [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] -Input [6]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, _w0#59, rank_within_parent#61] +Output [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] +Input [6]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, _w0#71, rank_within_parent#73] (60) TakeOrderedAndProject -Input [5]: [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] -Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#10 END ASC NULLS FIRST, rank_within_parent#61 ASC NULLS FIRST], [total_sum#26, s_state#10, s_county#9, lochierarchy#29, rank_within_parent#61] +Input [5]: [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] +Arguments: 100, [lochierarchy#35 DESC NULLS LAST, CASE WHEN (lochierarchy#35 = 0) THEN s_state#10 END ASC NULLS FIRST, rank_within_parent#73 ASC NULLS FIRST], [total_sum#32, s_state#10, s_county#9, lochierarchy#35, rank_within_parent#73] ===== Subqueries ===== @@ -358,6 +358,6 @@ ReusedExchange (61) (61) ReusedExchange [Reuses operator id: 8] Output [1]: [d_date_sk#5] -Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 14 Hosting Expression = ss_sold_date_sk#13 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 035f89d3a04f7..7eea8040043d6 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 @@ -199,285 +199,285 @@ Input [2]: [customer_id#18, year_total#19] Arguments: [customer_id#18 ASC NULLS FIRST], false, 0 (25) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] +Output [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (26) ColumnarToRow [codegen id : 10] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] +Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] (27) Filter [codegen id : 10] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3] -Condition : isnotnull(ss_customer_sk#1) +Input [3]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23] +Condition : isnotnull(ss_customer_sk#21) (28) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#25, d_year#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (29) ColumnarToRow [codegen id : 9] -Input [2]: [d_date_sk#5, d_year#6] +Input [2]: [d_date_sk#25, d_year#26] (30) Filter [codegen id : 9] -Input [2]: [d_date_sk#5, d_year#6] -Condition : (((isnotnull(d_year#6) AND (d_year#6 = 2002)) AND d_year#6 IN (2001,2002)) AND isnotnull(d_date_sk#5)) +Input [2]: [d_date_sk#25, d_year#26] +Condition : (((isnotnull(d_year#26) AND (d_year#26 = 2002)) AND d_year#26 IN (2001,2002)) AND isnotnull(d_date_sk#25)) (31) BroadcastExchange -Input [2]: [d_date_sk#5, d_year#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +Input [2]: [d_date_sk#25, d_year#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (32) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ss_sold_date_sk#3] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#23] +Right keys [1]: [d_date_sk#25] Join condition: None (33) Project [codegen id : 10] -Output [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Input [5]: [ss_customer_sk#1, ss_net_paid#2, ss_sold_date_sk#3, d_date_sk#5, d_year#6] +Output [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Input [5]: [ss_customer_sk#21, ss_net_paid#22, ss_sold_date_sk#23, d_date_sk#25, d_year#26] (34) Exchange -Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Arguments: hashpartitioning(ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#23] +Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Arguments: hashpartitioning(ss_customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#28] (35) Sort [codegen id : 11] -Input [3]: [ss_customer_sk#1, ss_net_paid#2, d_year#6] -Arguments: [ss_customer_sk#1 ASC NULLS FIRST], false, 0 +Input [3]: [ss_customer_sk#21, ss_net_paid#22, d_year#26] +Arguments: [ss_customer_sk#21 ASC NULLS FIRST], false, 0 (36) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] (37) Sort [codegen id : 13] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 (38) SortMergeJoin [codegen id : 14] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ss_customer_sk#21] +Right keys [1]: [c_customer_sk#29] Join condition: None (39) Project [codegen id : 14] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] -Input [7]: [ss_customer_sk#1, ss_net_paid#2, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] +Input [7]: [ss_customer_sk#21, ss_net_paid#22, d_year#26, c_customer_sk#29, c_customer_id#30, c_first_name#31, c_last_name#32] (40) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ss_net_paid#2, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#2))] -Aggregate Attributes [1]: [sum#24] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, ss_net_paid#22, d_year#26] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#22))] +Aggregate Attributes [1]: [sum#33] +Results [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] (41) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#26] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] +Arguments: hashpartitioning(c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, 5), ENSURE_REQUIREMENTS, [id=#35] (42) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#25] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ss_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#2))#27] -Results [4]: [c_customer_id#10 AS customer_id#28, c_first_name#11 AS customer_first_name#29, c_last_name#12 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#2))#27,17,2) AS year_total#31] +Input [5]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26, sum#34] +Keys [4]: [c_customer_id#30, c_first_name#31, c_last_name#32, d_year#26] +Functions [1]: [sum(UnscaledValue(ss_net_paid#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#22))#36] +Results [4]: [c_customer_id#30 AS customer_id#37, c_first_name#31 AS customer_first_name#38, c_last_name#32 AS customer_last_name#39, MakeDecimal(sum(UnscaledValue(ss_net_paid#22))#36,17,2) AS year_total#40] (43) Exchange -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: hashpartitioning(customer_id#28, 5), ENSURE_REQUIREMENTS, [id=#32] +Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] +Arguments: hashpartitioning(customer_id#37, 5), ENSURE_REQUIREMENTS, [id=#41] (44) Sort [codegen id : 16] -Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] -Arguments: [customer_id#28 ASC NULLS FIRST], false, 0 +Input [4]: [customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40] +Arguments: [customer_id#37 ASC NULLS FIRST], false, 0 (45) SortMergeJoin [codegen id : 17] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#28] +Right keys [1]: [customer_id#37] Join condition: None (46) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#4)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#4)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 19] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] (48) Filter [codegen id : 19] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_bill_customer_sk#33) +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_bill_customer_sk#42) (49) ReusedExchange [Reuses operator id: 7] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#45, d_year#46] (50) BroadcastHashJoin [codegen id : 19] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (51) Project [codegen id : 19] -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] +Output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Input [5]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44, d_date_sk#45, d_year#46] (52) Exchange -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#36] +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Arguments: hashpartitioning(ws_bill_customer_sk#42, 5), ENSURE_REQUIREMENTS, [id=#47] (53) Sort [codegen id : 20] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46] +Arguments: [ws_bill_customer_sk#42 ASC NULLS FIRST], false, 0 (54) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] (55) Sort [codegen id : 22] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] +Arguments: [c_customer_sk#48 ASC NULLS FIRST], false, 0 (56) SortMergeJoin [codegen id : 23] -Left keys [1]: [ws_bill_customer_sk#33] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ws_bill_customer_sk#42] +Right keys [1]: [c_customer_sk#48] Join condition: None (57) Project [codegen id : 23] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] +Input [7]: [ws_bill_customer_sk#42, ws_net_paid#43, d_year#46, c_customer_sk#48, c_customer_id#49, c_first_name#50, c_last_name#51] (58) HashAggregate [codegen id : 23] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum#37] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, ws_net_paid#43, d_year#46] +Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#43))] +Aggregate Attributes [1]: [sum#52] +Results [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] (59) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#39] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] +Arguments: hashpartitioning(c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, 5), ENSURE_REQUIREMENTS, [id=#54] (60) HashAggregate [codegen id : 24] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#38] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#40] -Results [2]: [c_customer_id#10 AS customer_id#41, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#40,17,2) AS year_total#42] +Input [5]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46, sum#53] +Keys [4]: [c_customer_id#49, c_first_name#50, c_last_name#51, d_year#46] +Functions [1]: [sum(UnscaledValue(ws_net_paid#43))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#43))#55] +Results [2]: [c_customer_id#49 AS customer_id#56, MakeDecimal(sum(UnscaledValue(ws_net_paid#43))#55,17,2) AS year_total#57] (61) Filter [codegen id : 24] -Input [2]: [customer_id#41, year_total#42] -Condition : (isnotnull(year_total#42) AND (year_total#42 > 0.00)) +Input [2]: [customer_id#56, year_total#57] +Condition : (isnotnull(year_total#57) AND (year_total#57 > 0.00)) (62) Project [codegen id : 24] -Output [2]: [customer_id#41 AS customer_id#43, year_total#42 AS year_total#44] -Input [2]: [customer_id#41, year_total#42] +Output [2]: [customer_id#56 AS customer_id#58, year_total#57 AS year_total#59] +Input [2]: [customer_id#56, year_total#57] (63) Exchange -Input [2]: [customer_id#43, year_total#44] -Arguments: hashpartitioning(customer_id#43, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [2]: [customer_id#58, year_total#59] +Arguments: hashpartitioning(customer_id#58, 5), ENSURE_REQUIREMENTS, [id=#60] (64) Sort [codegen id : 25] -Input [2]: [customer_id#43, year_total#44] -Arguments: [customer_id#43 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#58, year_total#59] +Arguments: [customer_id#58 ASC NULLS FIRST], false, 0 (65) SortMergeJoin [codegen id : 26] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#43] +Right keys [1]: [customer_id#58] Join condition: None (66) Project [codegen id : 26] -Output [7]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44] -Input [8]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#43, year_total#44] +Output [7]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59] +Input [8]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, customer_id#58, year_total#59] (67) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#35), dynamicpruningexpression(ws_sold_date_sk#35 IN dynamicpruning#21)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (68) ColumnarToRow [codegen id : 28] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] (69) Filter [codegen id : 28] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35] -Condition : isnotnull(ws_bill_customer_sk#33) +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_bill_customer_sk#61) (70) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#64, d_year#65] (71) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [ws_sold_date_sk#35] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#63] +Right keys [1]: [d_date_sk#64] Join condition: None (72) Project [codegen id : 28] -Output [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Input [5]: [ws_bill_customer_sk#33, ws_net_paid#34, ws_sold_date_sk#35, d_date_sk#5, d_year#6] +Output [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Input [5]: [ws_bill_customer_sk#61, ws_net_paid#62, ws_sold_date_sk#63, d_date_sk#64, d_year#65] (73) Exchange -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: hashpartitioning(ws_bill_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#46] +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Arguments: hashpartitioning(ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#66] (74) Sort [codegen id : 29] -Input [3]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6] -Arguments: [ws_bill_customer_sk#33 ASC NULLS FIRST], false, 0 +Input [3]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65] +Arguments: [ws_bill_customer_sk#61 ASC NULLS FIRST], false, 0 (75) ReusedExchange [Reuses operator id: 15] -Output [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] (76) Sort [codegen id : 31] -Input [4]: [c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] -Arguments: [c_customer_sk#9 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] +Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0 (77) SortMergeJoin [codegen id : 32] -Left keys [1]: [ws_bill_customer_sk#33] -Right keys [1]: [c_customer_sk#9] +Left keys [1]: [ws_bill_customer_sk#61] +Right keys [1]: [c_customer_sk#67] Join condition: None (78) Project [codegen id : 32] -Output [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Input [7]: [ws_bill_customer_sk#33, ws_net_paid#34, d_year#6, c_customer_sk#9, c_customer_id#10, c_first_name#11, c_last_name#12] +Output [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] +Input [7]: [ws_bill_customer_sk#61, ws_net_paid#62, d_year#65, c_customer_sk#67, c_customer_id#68, c_first_name#69, c_last_name#70] (79) HashAggregate [codegen id : 32] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, ws_net_paid#34, d_year#6] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum#47] -Results [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, ws_net_paid#62, d_year#65] +Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#62))] +Aggregate Attributes [1]: [sum#71] +Results [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] (80) Exchange -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] -Arguments: hashpartitioning(c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] +Arguments: hashpartitioning(c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, 5), ENSURE_REQUIREMENTS, [id=#73] (81) HashAggregate [codegen id : 33] -Input [5]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6, sum#48] -Keys [4]: [c_customer_id#10, c_first_name#11, c_last_name#12, d_year#6] -Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#50] -Results [2]: [c_customer_id#10 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#50,17,2) AS year_total#52] +Input [5]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65, sum#72] +Keys [4]: [c_customer_id#68, c_first_name#69, c_last_name#70, d_year#65] +Functions [1]: [sum(UnscaledValue(ws_net_paid#62))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#62))#74] +Results [2]: [c_customer_id#68 AS customer_id#75, MakeDecimal(sum(UnscaledValue(ws_net_paid#62))#74,17,2) AS year_total#76] (82) Exchange -Input [2]: [customer_id#51, year_total#52] -Arguments: hashpartitioning(customer_id#51, 5), ENSURE_REQUIREMENTS, [id=#53] +Input [2]: [customer_id#75, year_total#76] +Arguments: hashpartitioning(customer_id#75, 5), ENSURE_REQUIREMENTS, [id=#77] (83) Sort [codegen id : 34] -Input [2]: [customer_id#51, year_total#52] -Arguments: [customer_id#51 ASC NULLS FIRST], false, 0 +Input [2]: [customer_id#75, year_total#76] +Arguments: [customer_id#75 ASC NULLS FIRST], false, 0 (84) SortMergeJoin [codegen id : 35] Left keys [1]: [customer_id#18] -Right keys [1]: [customer_id#51] -Join condition: (CASE WHEN (year_total#44 > 0.00) THEN CheckOverflow((promote_precision(year_total#52) / promote_precision(year_total#44)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#31) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#75] +Join condition: (CASE WHEN (year_total#59 > 0.00) THEN CheckOverflow((promote_precision(year_total#76) / promote_precision(year_total#59)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#19 > 0.00) THEN CheckOverflow((promote_precision(year_total#40) / promote_precision(year_total#19)), DecimalType(37,20), true) ELSE null END) (85) Project [codegen id : 35] -Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Input [9]: [customer_id#18, year_total#19, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#44, customer_id#51, year_total#52] +Output [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] +Input [9]: [customer_id#18, year_total#19, customer_id#37, customer_first_name#38, customer_last_name#39, year_total#40, year_total#59, customer_id#75, year_total#76] (86) TakeOrderedAndProject -Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] -Arguments: 100, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] +Input [3]: [customer_id#37, customer_first_name#38, customer_last_name#39] +Arguments: 100, [customer_first_name#38 ASC NULLS FIRST, customer_id#37 ASC NULLS FIRST, customer_last_name#39 ASC NULLS FIRST], [customer_id#37, customer_first_name#38, customer_last_name#39] ===== Subqueries ===== @@ -488,15 +488,15 @@ ReusedExchange (87) (87) ReusedExchange [Reuses operator id: 7] Output [2]: [d_date_sk#5, d_year#6] -Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#21 +Subquery:2 Hosting operator id = 25 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#24 ReusedExchange (88) (88) ReusedExchange [Reuses operator id: 31] -Output [2]: [d_date_sk#5, d_year#6] +Output [2]: [d_date_sk#25, d_year#26] -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#4 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#4 -Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#35 IN dynamicpruning#21 +Subquery:4 Hosting operator id = 67 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#24 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 44a902cf6da07..3ee6b56189888 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 @@ -171,282 +171,282 @@ Input [2]: [customer_id#17, year_total#18] Condition : (isnotnull(year_total#18) AND (year_total#18 > 0.00)) (20) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (21) ColumnarToRow [codegen id : 6] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] (22) Filter [codegen id : 6] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22] +Condition : (isnotnull(c_customer_sk#19) AND isnotnull(c_customer_id#20)) (23) Scan parquet default.store_sales -Output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Output [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] (25) Filter [codegen id : 4] -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -Condition : isnotnull(ss_customer_sk#5) +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_customer_sk#23) (26) BroadcastExchange -Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [3]: [ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#27] (27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ss_customer_sk#5] +Left keys [1]: [c_customer_sk#19] +Right keys [1]: [ss_customer_sk#23] Join condition: None (28) Project [codegen id : 6] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25] +Input [7]: [c_customer_sk#19, c_customer_id#20, c_first_name#21, c_last_name#22, ss_customer_sk#23, ss_net_paid#24, ss_sold_date_sk#25] (29) Scan parquet default.date_dim -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#28, d_year#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] ReadSchema: struct (30) ColumnarToRow [codegen id : 5] -Input [2]: [d_date_sk#10, d_year#11] +Input [2]: [d_date_sk#28, d_year#29] (31) Filter [codegen id : 5] -Input [2]: [d_date_sk#10, d_year#11] -Condition : (((isnotnull(d_year#11) AND (d_year#11 = 2002)) AND d_year#11 IN (2001,2002)) AND isnotnull(d_date_sk#10)) +Input [2]: [d_date_sk#28, d_year#29] +Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) (32) BroadcastExchange -Input [2]: [d_date_sk#10, d_year#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] +Input [2]: [d_date_sk#28, d_year#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#30] (33) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#7] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ss_sold_date_sk#25] +Right keys [1]: [d_date_sk#28] Join condition: None (34) Project [codegen id : 6] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] +Input [7]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, ss_sold_date_sk#25, d_date_sk#28, d_year#29] (35) HashAggregate [codegen id : 6] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum#22] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, ss_net_paid#24, d_year#29] +Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#24))] +Aggregate Attributes [1]: [sum#31] +Results [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] (36) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#24] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] +Arguments: hashpartitioning(c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, 5), ENSURE_REQUIREMENTS, [id=#33] (37) HashAggregate [codegen id : 7] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#23] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#25] -Results [4]: [c_customer_id#2 AS customer_id#26, c_first_name#3 AS customer_first_name#27, c_last_name#4 AS customer_last_name#28, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#25,17,2) AS year_total#29] +Input [5]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29, sum#32] +Keys [4]: [c_customer_id#20, c_first_name#21, c_last_name#22, d_year#29] +Functions [1]: [sum(UnscaledValue(ss_net_paid#24))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#24))#34] +Results [4]: [c_customer_id#20 AS customer_id#35, c_first_name#21 AS customer_first_name#36, c_last_name#22 AS customer_last_name#37, MakeDecimal(sum(UnscaledValue(ss_net_paid#24))#34,17,2) AS year_total#38] (38) BroadcastExchange -Input [4]: [customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#30] +Input [4]: [customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#39] (39) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#26] +Right keys [1]: [customer_id#35] Join condition: None (40) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (41) ColumnarToRow [codegen id : 10] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] (42) Filter [codegen id : 10] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43] +Condition : (isnotnull(c_customer_sk#40) AND isnotnull(c_customer_id#41)) (43) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (44) ColumnarToRow [codegen id : 8] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] (45) Filter [codegen id : 8] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_bill_customer_sk#31) +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Condition : isnotnull(ws_bill_customer_sk#44) (46) BroadcastExchange -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] +Input [3]: [ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#47] (47) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#31] +Left keys [1]: [c_customer_sk#40] +Right keys [1]: [ws_bill_customer_sk#44] Join condition: None (48) Project [codegen id : 10] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46] +Input [7]: [c_customer_sk#40, c_customer_id#41, c_first_name#42, c_last_name#43, ws_bill_customer_sk#44, ws_net_paid#45, ws_sold_date_sk#46] (49) ReusedExchange [Reuses operator id: 13] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#48, d_year#49] (50) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#46] +Right keys [1]: [d_date_sk#48] Join condition: None (51) Project [codegen id : 10] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] +Input [7]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, ws_sold_date_sk#46, d_date_sk#48, d_year#49] (52) HashAggregate [codegen id : 10] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum#35] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, ws_net_paid#45, d_year#49] +Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#45))] +Aggregate Attributes [1]: [sum#50] +Results [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] (53) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] +Arguments: hashpartitioning(c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, 5), ENSURE_REQUIREMENTS, [id=#52] (54) HashAggregate [codegen id : 11] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#36] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#38] -Results [2]: [c_customer_id#2 AS customer_id#39, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#38,17,2) AS year_total#40] +Input [5]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49, sum#51] +Keys [4]: [c_customer_id#41, c_first_name#42, c_last_name#43, d_year#49] +Functions [1]: [sum(UnscaledValue(ws_net_paid#45))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#45))#53] +Results [2]: [c_customer_id#41 AS customer_id#54, MakeDecimal(sum(UnscaledValue(ws_net_paid#45))#53,17,2) AS year_total#55] (55) Filter [codegen id : 11] -Input [2]: [customer_id#39, year_total#40] -Condition : (isnotnull(year_total#40) AND (year_total#40 > 0.00)) +Input [2]: [customer_id#54, year_total#55] +Condition : (isnotnull(year_total#55) AND (year_total#55 > 0.00)) (56) Project [codegen id : 11] -Output [2]: [customer_id#39 AS customer_id#41, year_total#40 AS year_total#42] -Input [2]: [customer_id#39, year_total#40] +Output [2]: [customer_id#54 AS customer_id#56, year_total#55 AS year_total#57] +Input [2]: [customer_id#54, year_total#55] (57) BroadcastExchange -Input [2]: [customer_id#41, year_total#42] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#43] +Input [2]: [customer_id#56, year_total#57] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#58] (58) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#41] +Right keys [1]: [customer_id#56] Join condition: None (59) Project [codegen id : 16] -Output [7]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42] -Input [8]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, customer_id#41, year_total#42] +Output [7]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, year_total#57] +Input [8]: [customer_id#17, year_total#18, customer_id#35, customer_first_name#36, customer_last_name#37, year_total#38, customer_id#56, year_total#57] (60) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Output [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] ReadSchema: struct (61) ColumnarToRow [codegen id : 14] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] (62) Filter [codegen id : 14] -Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) +Input [4]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62] +Condition : (isnotnull(c_customer_sk#59) AND isnotnull(c_customer_id#60)) (63) Scan parquet default.web_sales -Output [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (64) ColumnarToRow [codegen id : 12] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] (65) Filter [codegen id : 12] -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Condition : isnotnull(ws_bill_customer_sk#31) +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Condition : isnotnull(ws_bill_customer_sk#63) (66) BroadcastExchange -Input [3]: [ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#44] +Input [3]: [ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] (67) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ws_bill_customer_sk#31] +Left keys [1]: [c_customer_sk#59] +Right keys [1]: [ws_bill_customer_sk#63] Join condition: None (68) Project [codegen id : 14] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33] -Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ws_bill_customer_sk#31, ws_net_paid#32, ws_sold_date_sk#33] +Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65] +Input [7]: [c_customer_sk#59, c_customer_id#60, c_first_name#61, c_last_name#62, ws_bill_customer_sk#63, ws_net_paid#64, ws_sold_date_sk#65] (69) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#67, d_year#68] (70) BroadcastHashJoin [codegen id : 14] -Left keys [1]: [ws_sold_date_sk#33] -Right keys [1]: [d_date_sk#10] +Left keys [1]: [ws_sold_date_sk#65] +Right keys [1]: [d_date_sk#67] Join condition: None (71) Project [codegen id : 14] -Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, ws_sold_date_sk#33, d_date_sk#10, d_year#11] +Output [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] +Input [7]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, ws_sold_date_sk#65, d_date_sk#67, d_year#68] (72) HashAggregate [codegen id : 14] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ws_net_paid#32, d_year#11] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum#45] -Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, ws_net_paid#64, d_year#68] +Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#64))] +Aggregate Attributes [1]: [sum#69] +Results [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] (73) Exchange -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] -Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, 5), ENSURE_REQUIREMENTS, [id=#47] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] +Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, 5), ENSURE_REQUIREMENTS, [id=#71] (74) HashAggregate [codegen id : 15] -Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11, sum#46] -Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#11] -Functions [1]: [sum(UnscaledValue(ws_net_paid#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#32))#48] -Results [2]: [c_customer_id#2 AS customer_id#49, MakeDecimal(sum(UnscaledValue(ws_net_paid#32))#48,17,2) AS year_total#50] +Input [5]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68, sum#70] +Keys [4]: [c_customer_id#60, c_first_name#61, c_last_name#62, d_year#68] +Functions [1]: [sum(UnscaledValue(ws_net_paid#64))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#64))#72] +Results [2]: [c_customer_id#60 AS customer_id#73, MakeDecimal(sum(UnscaledValue(ws_net_paid#64))#72,17,2) AS year_total#74] (75) BroadcastExchange -Input [2]: [customer_id#49, year_total#50] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#51] +Input [2]: [customer_id#73, year_total#74] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [id=#75] (76) BroadcastHashJoin [codegen id : 16] Left keys [1]: [customer_id#17] -Right keys [1]: [customer_id#49] -Join condition: (CASE WHEN (year_total#42 > 0.00) THEN CheckOverflow((promote_precision(year_total#50) / promote_precision(year_total#42)), DecimalType(37,20), true) ELSE null END > CASE WHEN (year_total#18 > 0.00) THEN CheckOverflow((promote_precision(year_total#29) / promote_precision(year_total#18)), DecimalType(37,20), true) ELSE null END) +Right keys [1]: [customer_id#73] +Join condition: (CASE WHEN (year_total#57 > 0.00) THEN CheckOverflow((promote_precision(year_total#74) / promote_precision(year_total#57)), 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) (77) Project [codegen id : 16] -Output [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] -Input [9]: [customer_id#17, year_total#18, customer_id#26, customer_first_name#27, customer_last_name#28, year_total#29, year_total#42, customer_id#49, year_total#50] +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#57, customer_id#73, year_total#74] (78) TakeOrderedAndProject -Input [3]: [customer_id#26, customer_first_name#27, customer_last_name#28] -Arguments: 100, [customer_first_name#27 ASC NULLS FIRST, customer_id#26 ASC NULLS FIRST, customer_last_name#28 ASC NULLS FIRST], [customer_id#26, customer_first_name#27, customer_last_name#28] +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] ===== Subqueries ===== @@ -457,15 +457,15 @@ ReusedExchange (79) (79) ReusedExchange [Reuses operator id: 13] Output [2]: [d_date_sk#10, d_year#11] -Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#19 +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 ReusedExchange (80) (80) ReusedExchange [Reuses operator id: 32] -Output [2]: [d_date_sk#10, d_year#11] +Output [2]: [d_date_sk#28, d_year#29] -Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#8 -Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#19 +Subquery:4 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#26 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 7ecbe2e51c9b4..56998d9197835 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 @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#31] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#36, d_year#37] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#36] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] -Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (42) Exchange -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_item_sk#47) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ws_item_sk#47] +Right keys [1]: [i_item_sk#52] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#57, d_year#58] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#57] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] (55) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] -Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (61) Exchange -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] +Right keys [2]: [wr_order_number#61, wr_item_sk#60] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] +Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#55, sum#56] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Aggregate Attributes [2]: [sum#69, sum#70] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] +Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Condition : isnotnull(cs_item_sk#79) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [cs_item_sk#79] +Right keys [1]: [i_item_sk#85] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] +Input [2]: [d_date_sk#90, d_year#91] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] -Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) +Input [2]: [d_date_sk#90, d_year#91] +Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) (83) BroadcastExchange -Input [2]: [d_date_sk#71, d_year#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [d_date_sk#90, d_year#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [cs_sold_date_sk#83] +Right keys [1]: [d_date_sk#90] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] (86) Exchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] -Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#2, cs_item_sk#1] -Right keys [2]: [cr_order_number#19, cr_item_sk#18] +Left keys [2]: [cs_order_number#80, cs_item_sk#79] +Right keys [2]: [cr_order_number#95, cr_item_sk#94] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Condition : isnotnull(ss_item_sk#26) +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Condition : isnotnull(ss_item_sk#98) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#98] +Right keys [1]: [i_item_sk#103] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#108, d_year#109] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ss_sold_date_sk#102] +Right keys [1]: [d_date_sk#108] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] (101) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] +Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] +Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Condition : isnotnull(ws_item_sk#116) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ws_item_sk#116] +Right keys [1]: [i_item_sk#121] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#126, d_year#127] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ws_sold_date_sk#120] +Right keys [1]: [d_date_sk#126] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] (116) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] +Right keys [2]: [wr_order_number#130, wr_item_sk#129] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] +Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Aggregate Attributes [2]: [sum#135, sum#136] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] (127) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] +Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] (129) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] (130) Sort [codegen id : 50] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 (131) 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#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] (133) TakeOrderedAndProject -Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] -Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST, sales_amt_diff#97 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST, sales_amt_diff#150 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 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 7ecbe2e51c9b4..56998d9197835 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 @@ -266,153 +266,153 @@ Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_pri Condition : isnotnull(ss_item_sk#26) (30) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (31) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#7] +Right keys [1]: [i_item_sk#31] Join condition: None (32) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] +Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#31, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35] (33) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#36, d_year#37] (34) BroadcastHashJoin [codegen id : 10] Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#36] Join condition: None (35) Project [codegen id : 10] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_date_sk#36, d_year#37] (36) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#31] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] +Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#38] (37) Sort [codegen id : 11] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] +Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37] Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 (38) Scan parquet default.store_returns -Output [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct (39) ColumnarToRow [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (40) Filter [codegen id : 12] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] -Condition : (isnotnull(sr_ticket_number#33) AND isnotnull(sr_item_sk#32)) +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] +Condition : (isnotnull(sr_ticket_number#40) AND isnotnull(sr_item_sk#39)) (41) Project [codegen id : 12] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Input [5]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35, sr_returned_date_sk#36] +Output [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Input [5]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42, sr_returned_date_sk#43] (42) Exchange -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: hashpartitioning(sr_ticket_number#33, sr_item_sk#32, 5), ENSURE_REQUIREMENTS, [id=#37] +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: hashpartitioning(sr_ticket_number#40, sr_item_sk#39, 5), ENSURE_REQUIREMENTS, [id=#44] (43) Sort [codegen id : 13] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] +Arguments: [sr_ticket_number#40 ASC NULLS FIRST, sr_item_sk#39 ASC NULLS FIRST], false, 0 (44) SortMergeJoin Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Right keys [2]: [sr_ticket_number#40, sr_item_sk#39] Join condition: None (45) Project [codegen id : 14] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#38, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#39] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#37, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, (ss_quantity#28 - coalesce(sr_return_quantity#41, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#42, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#46] +Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#32, i_class_id#33, i_category_id#34, i_manufact_id#35, d_year#37, sr_item_sk#39, sr_ticket_number#40, sr_return_quantity#41, sr_return_amt#42] (46) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#6)] +PartitionFilters: [isnotnull(ws_sold_date_sk#51), dynamicpruningexpression(ws_sold_date_sk#51 IN dynamicpruning#6)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] (48) Filter [codegen id : 17] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51] +Condition : isnotnull(ws_item_sk#47) (49) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [5]: [i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (50) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#7] +Left keys [1]: [ws_item_sk#47] +Right keys [1]: [i_item_sk#52] Join condition: None (51) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] +Input [10]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_item_sk#52, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56] (52) ReusedExchange [Reuses operator id: 14] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#57, d_year#58] (53) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#51] +Right keys [1]: [d_date_sk#57] Join condition: None (54) Project [codegen id : 17] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_date_sk#14, d_year#15] +Output [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Input [11]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, ws_sold_date_sk#51, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_date_sk#57, d_year#58] (55) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#45] +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: hashpartitioning(cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint), 5), ENSURE_REQUIREMENTS, [id=#59] (56) Sort [codegen id : 18] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58] +Arguments: [cast(ws_order_number#48 as bigint) ASC NULLS FIRST, cast(ws_item_sk#47 as bigint) ASC NULLS FIRST], false, 0 (57) Scan parquet default.web_returns -Output [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (58) ColumnarToRow [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (59) Filter [codegen id : 19] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] -Condition : (isnotnull(wr_order_number#47) AND isnotnull(wr_item_sk#46)) +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_order_number#61) AND isnotnull(wr_item_sk#60)) (60) Project [codegen id : 19] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Input [5]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49, wr_returned_date_sk#50] +Output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63, wr_returned_date_sk#64] (61) Exchange -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: hashpartitioning(wr_order_number#47, wr_item_sk#46, 5), ENSURE_REQUIREMENTS, [id=#51] +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: hashpartitioning(wr_order_number#61, wr_item_sk#60, 5), ENSURE_REQUIREMENTS, [id=#65] (62) Sort [codegen id : 20] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] +Arguments: [wr_order_number#61 ASC NULLS FIRST, wr_item_sk#60 ASC NULLS FIRST], false, 0 (63) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#48 as bigint), cast(ws_item_sk#47 as bigint)] +Right keys [2]: [wr_order_number#61, wr_item_sk#60] Join condition: None (64) Project [codegen id : 21] -Output [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#52, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#53] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#15, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#58, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, (ws_quantity#49 - coalesce(wr_return_quantity#62, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#50 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#63, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#67] +Input [13]: [ws_item_sk#47, ws_order_number#48, ws_quantity#49, ws_ext_sales_price#50, i_brand_id#53, i_class_id#54, i_category_id#55, i_manufact_id#56, d_year#58, wr_item_sk#60, wr_order_number#61, wr_return_quantity#62, wr_return_amt#63] (65) Union @@ -425,7 +425,7 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac (67) Exchange Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#54] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#68] (68) HashAggregate [codegen id : 23] Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] @@ -438,290 +438,290 @@ Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufac Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#24, sales_amt#25] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#55, sum#56] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Aggregate Attributes [2]: [sum#69, sum#70] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] (70) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] -Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#59] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] +Arguments: hashpartitioning(d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#73] (71) HashAggregate [codegen id : 24] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#57, sum#58] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#71, sum#72] Keys [5]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#60, sum(UnscaledValue(sales_amt#25))#61] -Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#60 AS sales_cnt#62, MakeDecimal(sum(UnscaledValue(sales_amt#25))#61,18,2) AS sales_amt#63] +Aggregate Attributes [2]: [sum(sales_cnt#24)#74, sum(UnscaledValue(sales_amt#25))#75] +Results [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#24)#74 AS sales_cnt#76, MakeDecimal(sum(UnscaledValue(sales_amt#25))#75,18,2) AS sales_amt#77] (72) Exchange -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#64] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [id=#78] (73) Sort [codegen id : 25] -Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63] +Input [7]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77] Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 (74) Scan parquet default.catalog_sales -Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Output [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(cs_sold_date_sk#83), dynamicpruningexpression(cs_sold_date_sk#83 IN dynamicpruning#84)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] (76) Filter [codegen id : 28] -Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] -Condition : isnotnull(cs_item_sk#1) +Input [5]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83] +Condition : isnotnull(cs_item_sk#79) (77) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (78) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_item_sk#1] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [cs_item_sk#79] +Right keys [1]: [i_item_sk#85] Join condition: None (79) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Input [10]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_item_sk#85, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] (80) Scan parquet default.date_dim -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] 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 (81) ColumnarToRow [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] +Input [2]: [d_date_sk#90, d_year#91] (82) Filter [codegen id : 27] -Input [2]: [d_date_sk#71, d_year#72] -Condition : ((isnotnull(d_year#72) AND (d_year#72 = 2001)) AND isnotnull(d_date_sk#71)) +Input [2]: [d_date_sk#90, d_year#91] +Condition : ((isnotnull(d_year#91) AND (d_year#91 = 2001)) AND isnotnull(d_date_sk#90)) (83) BroadcastExchange -Input [2]: [d_date_sk#71, d_year#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#73] +Input [2]: [d_date_sk#90, d_year#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#92] (84) BroadcastHashJoin [codegen id : 28] -Left keys [1]: [cs_sold_date_sk#5] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [cs_sold_date_sk#83] +Right keys [1]: [d_date_sk#90] Join condition: None (85) Project [codegen id : 28] -Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Input [11]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, cs_sold_date_sk#83, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_date_sk#90, d_year#91] (86) Exchange -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: hashpartitioning(cs_order_number#80, cs_item_sk#79, 5), ENSURE_REQUIREMENTS, [id=#93] (87) Sort [codegen id : 29] -Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 +Input [9]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91] +Arguments: [cs_order_number#80 ASC NULLS FIRST, cs_item_sk#79 ASC NULLS FIRST], false, 0 (88) ReusedExchange [Reuses operator id: 23] -Output [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (89) Sort [codegen id : 31] -Input [4]: [cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] -Arguments: [cr_order_number#19 ASC NULLS FIRST, cr_item_sk#18 ASC NULLS FIRST], false, 0 +Input [4]: [cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] +Arguments: [cr_order_number#95 ASC NULLS FIRST, cr_item_sk#94 ASC NULLS FIRST], false, 0 (90) SortMergeJoin -Left keys [2]: [cs_order_number#2, cs_item_sk#1] -Right keys [2]: [cr_order_number#19, cr_item_sk#18] +Left keys [2]: [cs_order_number#80, cs_item_sk#79] +Right keys [2]: [cr_order_number#95, cr_item_sk#94] Join condition: None (91) Project [codegen id : 32] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (cs_quantity#3 - coalesce(cr_return_quantity#20, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#21, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] -Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, cr_item_sk#18, cr_order_number#19, cr_return_quantity#20, cr_return_amount#21] +Output [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, (cs_quantity#81 - coalesce(cr_return_quantity#96, 0)) AS sales_cnt#24, CheckOverflow((promote_precision(cast(cs_ext_sales_price#82 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#97, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#25] +Input [13]: [cs_item_sk#79, cs_order_number#80, cs_quantity#81, cs_ext_sales_price#82, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, d_year#91, cr_item_sk#94, cr_order_number#95, cr_return_quantity#96, cr_return_amount#97] (92) Scan parquet default.store_sales -Output [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Output [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ss_sold_date_sk#102), dynamicpruningexpression(ss_sold_date_sk#102 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct (93) ColumnarToRow [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] (94) Filter [codegen id : 35] -Input [5]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Condition : isnotnull(ss_item_sk#26) +Input [5]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102] +Condition : isnotnull(ss_item_sk#98) (95) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (96) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_item_sk#26] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ss_item_sk#98] +Right keys [1]: [i_item_sk#103] Join condition: None (97) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] +Input [10]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_item_sk#103, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107] (98) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#108, d_year#109] (99) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [ss_sold_date_sk#30] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ss_sold_date_sk#102] +Right keys [1]: [d_date_sk#108] Join condition: None (100) Project [codegen id : 35] -Output [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, ss_sold_date_sk#30, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Input [11]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, ss_sold_date_sk#102, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_date_sk#108, d_year#109] (101) Exchange -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint), 5), ENSURE_REQUIREMENTS, [id=#75] +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: hashpartitioning(cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint), 5), ENSURE_REQUIREMENTS, [id=#110] (102) Sort [codegen id : 36] -Input [9]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ss_ticket_number#27 as bigint) ASC NULLS FIRST, cast(ss_item_sk#26 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109] +Arguments: [cast(ss_ticket_number#99 as bigint) ASC NULLS FIRST, cast(ss_item_sk#98 as bigint) ASC NULLS FIRST], false, 0 (103) ReusedExchange [Reuses operator id: 42] -Output [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (104) Sort [codegen id : 38] -Input [4]: [sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] -Arguments: [sr_ticket_number#33 ASC NULLS FIRST, sr_item_sk#32 ASC NULLS FIRST], false, 0 +Input [4]: [sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] +Arguments: [sr_ticket_number#112 ASC NULLS FIRST, sr_item_sk#111 ASC NULLS FIRST], false, 0 (105) SortMergeJoin -Left keys [2]: [cast(ss_ticket_number#27 as bigint), cast(ss_item_sk#26 as bigint)] -Right keys [2]: [sr_ticket_number#33, sr_item_sk#32] +Left keys [2]: [cast(ss_ticket_number#99 as bigint), cast(ss_item_sk#98 as bigint)] +Right keys [2]: [sr_ticket_number#112, sr_item_sk#111] Join condition: None (106) Project [codegen id : 39] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ss_quantity#28 - coalesce(sr_return_quantity#34, 0)) AS sales_cnt#76, CheckOverflow((promote_precision(cast(ss_ext_sales_price#29 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#35, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#77] -Input [13]: [ss_item_sk#26, ss_ticket_number#27, ss_quantity#28, ss_ext_sales_price#29, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, sr_item_sk#32, sr_ticket_number#33, sr_return_quantity#34, sr_return_amt#35] +Output [7]: [d_year#109, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, (ss_quantity#100 - coalesce(sr_return_quantity#113, 0)) AS sales_cnt#45, CheckOverflow((promote_precision(cast(ss_ext_sales_price#101 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#114, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#115] +Input [13]: [ss_item_sk#98, ss_ticket_number#99, ss_quantity#100, ss_ext_sales_price#101, i_brand_id#104, i_class_id#105, i_category_id#106, i_manufact_id#107, d_year#109, sr_item_sk#111, sr_ticket_number#112, sr_return_quantity#113, sr_return_amt#114] (107) Scan parquet default.web_sales -Output [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Output [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#65)] +PartitionFilters: [isnotnull(ws_sold_date_sk#120), dynamicpruningexpression(ws_sold_date_sk#120 IN dynamicpruning#84)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct (108) ColumnarToRow [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] (109) Filter [codegen id : 42] -Input [5]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44] -Condition : isnotnull(ws_item_sk#40) +Input [5]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120] +Condition : isnotnull(ws_item_sk#116) (110) ReusedExchange [Reuses operator id: 8] -Output [5]: [i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [5]: [i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (111) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_item_sk#40] -Right keys [1]: [i_item_sk#66] +Left keys [1]: [ws_item_sk#116] +Right keys [1]: [i_item_sk#121] Join condition: None (112) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Input [10]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_item_sk#66, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] +Input [10]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_item_sk#121, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125] (113) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#126, d_year#127] (114) BroadcastHashJoin [codegen id : 42] -Left keys [1]: [ws_sold_date_sk#44] -Right keys [1]: [d_date_sk#71] +Left keys [1]: [ws_sold_date_sk#120] +Right keys [1]: [d_date_sk#126] Join condition: None (115) Project [codegen id : 42] -Output [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Input [11]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, ws_sold_date_sk#44, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_date_sk#71, d_year#72] +Output [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Input [11]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, ws_sold_date_sk#120, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_date_sk#126, d_year#127] (116) Exchange -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: hashpartitioning(cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint), 5), ENSURE_REQUIREMENTS, [id=#78] +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: hashpartitioning(cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint), 5), ENSURE_REQUIREMENTS, [id=#128] (117) Sort [codegen id : 43] -Input [9]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72] -Arguments: [cast(ws_order_number#41 as bigint) ASC NULLS FIRST, cast(ws_item_sk#40 as bigint) ASC NULLS FIRST], false, 0 +Input [9]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127] +Arguments: [cast(ws_order_number#117 as bigint) ASC NULLS FIRST, cast(ws_item_sk#116 as bigint) ASC NULLS FIRST], false, 0 (118) ReusedExchange [Reuses operator id: 61] -Output [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (119) Sort [codegen id : 45] -Input [4]: [wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] -Arguments: [wr_order_number#47 ASC NULLS FIRST, wr_item_sk#46 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] +Arguments: [wr_order_number#130 ASC NULLS FIRST, wr_item_sk#129 ASC NULLS FIRST], false, 0 (120) SortMergeJoin -Left keys [2]: [cast(ws_order_number#41 as bigint), cast(ws_item_sk#40 as bigint)] -Right keys [2]: [wr_order_number#47, wr_item_sk#46] +Left keys [2]: [cast(ws_order_number#117 as bigint), cast(ws_item_sk#116 as bigint)] +Right keys [2]: [wr_order_number#130, wr_item_sk#129] Join condition: None (121) Project [codegen id : 46] -Output [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, (ws_quantity#42 - coalesce(wr_return_quantity#48, 0)) AS sales_cnt#79, CheckOverflow((promote_precision(cast(ws_ext_sales_price#43 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#49, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#80] -Input [13]: [ws_item_sk#40, ws_order_number#41, ws_quantity#42, ws_ext_sales_price#43, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, d_year#72, wr_item_sk#46, wr_order_number#47, wr_return_quantity#48, wr_return_amt#49] +Output [7]: [d_year#127, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, (ws_quantity#118 - coalesce(wr_return_quantity#131, 0)) AS sales_cnt#66, CheckOverflow((promote_precision(cast(ws_ext_sales_price#119 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#132, 0.00) as decimal(8,2)))), DecimalType(8,2), true) AS sales_amt#133] +Input [13]: [ws_item_sk#116, ws_order_number#117, ws_quantity#118, ws_ext_sales_price#119, i_brand_id#122, i_class_id#123, i_category_id#124, i_manufact_id#125, d_year#127, wr_item_sk#129, wr_order_number#130, wr_return_quantity#131, wr_return_amt#132] (122) Union (123) HashAggregate [codegen id : 47] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (124) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#81] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25, 5), ENSURE_REQUIREMENTS, [id=#134] (125) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] Functions: [] Aggregate Attributes: [] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] (126) HashAggregate [codegen id : 48] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#24, sales_amt#25] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#24, sales_amt#25] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [partial_sum(sales_cnt#24), partial_sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum#82, sum#83] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] +Aggregate Attributes [2]: [sum#135, sum#136] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] (127) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Arguments: hashpartitioning(d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#86] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Arguments: hashpartitioning(d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#139] (128) HashAggregate [codegen id : 49] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum#84, sum#85] -Keys [5]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum#137, sum#138] +Keys [5]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] Functions [2]: [sum(sales_cnt#24), sum(UnscaledValue(sales_amt#25))] -Aggregate Attributes [2]: [sum(sales_cnt#24)#87, sum(UnscaledValue(sales_amt#25))#88] -Results [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sum(sales_cnt#24)#87 AS sales_cnt#89, MakeDecimal(sum(UnscaledValue(sales_amt#25))#88,18,2) AS sales_amt#90] +Aggregate Attributes [2]: [sum(sales_cnt#24)#140, sum(UnscaledValue(sales_amt#25))#141] +Results [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sum(sales_cnt#24)#140 AS sales_cnt#142, MakeDecimal(sum(UnscaledValue(sales_amt#25))#141,18,2) AS sales_amt#143] (129) Exchange -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: hashpartitioning(i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, 5), ENSURE_REQUIREMENTS, [id=#91] +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: hashpartitioning(i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, 5), ENSURE_REQUIREMENTS, [id=#144] (130) Sort [codegen id : 50] -Input [7]: [d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] -Arguments: [i_brand_id#67 ASC NULLS FIRST, i_class_id#68 ASC NULLS FIRST, i_category_id#69 ASC NULLS FIRST, i_manufact_id#70 ASC NULLS FIRST], false, 0 +Input [7]: [d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] +Arguments: [i_brand_id#86 ASC NULLS FIRST, i_class_id#87 ASC NULLS FIRST, i_category_id#88 ASC NULLS FIRST, i_manufact_id#89 ASC NULLS FIRST], false, 0 (131) 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#67, i_class_id#68, i_category_id#69, i_manufact_id#70] -Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#62 as decimal(17,2))) / promote_precision(cast(sales_cnt#89 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) +Right keys [4]: [i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89] +Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#76 as decimal(17,2))) / promote_precision(cast(sales_cnt#142 as decimal(17,2)))), DecimalType(37,20), true) < 0.90000000000000000000) (132) Project [codegen id : 51] -Output [10]: [d_year#72 AS prev_year#92, d_year#15 AS year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#89 AS prev_yr_cnt#94, sales_cnt#62 AS curr_yr_cnt#95, (sales_cnt#62 - sales_cnt#89) AS sales_cnt_diff#96, CheckOverflow((promote_precision(cast(sales_amt#63 as decimal(19,2))) - promote_precision(cast(sales_amt#90 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#97] -Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#62, sales_amt#63, d_year#72, i_brand_id#67, i_class_id#68, i_category_id#69, i_manufact_id#70, sales_cnt#89, sales_amt#90] +Output [10]: [d_year#91 AS prev_year#145, d_year#15 AS year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#142 AS prev_yr_cnt#147, sales_cnt#76 AS curr_yr_cnt#148, (sales_cnt#76 - sales_cnt#142) AS sales_cnt_diff#149, CheckOverflow((promote_precision(cast(sales_amt#77 as decimal(19,2))) - promote_precision(cast(sales_amt#143 as decimal(19,2)))), DecimalType(19,2), true) AS sales_amt_diff#150] +Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#76, sales_amt#77, d_year#91, i_brand_id#86, i_class_id#87, i_category_id#88, i_manufact_id#89, sales_cnt#142, sales_amt#143] (133) TakeOrderedAndProject -Input [10]: [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] -Arguments: 100, [sales_cnt_diff#96 ASC NULLS FIRST, sales_amt_diff#97 ASC NULLS FIRST], [prev_year#92, year#93, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#94, curr_yr_cnt#95, sales_cnt_diff#96, sales_amt_diff#97] +Input [10]: [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] +Arguments: 100, [sales_cnt_diff#149 ASC NULLS FIRST, sales_amt_diff#150 ASC NULLS FIRST], [prev_year#145, year#146, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#147, curr_yr_cnt#148, sales_cnt_diff#149, sales_amt_diff#150] ===== Subqueries ===== @@ -734,17 +734,17 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#6 -Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#6 +Subquery:3 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#51 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#65 +Subquery:4 Hosting operator id = 74 Hosting Expression = cs_sold_date_sk#83 IN dynamicpruning#84 ReusedExchange (135) (135) ReusedExchange [Reuses operator id: 83] -Output [2]: [d_date_sk#71, d_year#72] +Output [2]: [d_date_sk#90, d_year#91] -Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#65 +Subquery:5 Hosting operator id = 92 Hosting Expression = ss_sold_date_sk#102 IN dynamicpruning#84 -Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#65 +Subquery:6 Hosting operator id = 107 Hosting Expression = ws_sold_date_sk#120 IN dynamicpruning#84 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 e6ade3880e2d4..422443509b417 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 @@ -222,38 +222,38 @@ Output [4]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_s Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25] (26) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#26] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Right keys [1]: [cast(d_date_sk#26 as bigint)] Join condition: None (28) Project [codegen id : 6] Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#6] +Input [5]: [sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, s_store_sk#25, d_date_sk#26] (29) HashAggregate [codegen id : 6] Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] Keys [1]: [s_store_sk#25] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#26, sum#27] -Results [3]: [s_store_sk#25, sum#28, sum#29] +Aggregate Attributes [2]: [sum#27, sum#28] +Results [3]: [s_store_sk#25, sum#29, sum#30] (30) Exchange -Input [3]: [s_store_sk#25, sum#28, sum#29] -Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [s_store_sk#25, sum#29, sum#30] +Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#31] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#25, sum#28, sum#29] +Input [3]: [s_store_sk#25, sum#29, sum#30] Keys [1]: [s_store_sk#25] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] -Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] +Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] (32) BroadcastExchange -Input [3]: [s_store_sk#25, returns#33, profit_loss#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [3]: [s_store_sk#25, returns#34, profit_loss#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] @@ -261,341 +261,341 @@ Right keys [1]: [s_store_sk#25] Join condition: None (34) Project [codegen id : 8] -Output [5]: [store channel AS channel#36, s_store_sk#9 AS id#37, sales#18, coalesce(returns#33, 0.00) AS returns#38, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#39] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] +Output [5]: [store channel AS channel#37, s_store_sk#9 AS id#38, sales#18, coalesce(returns#34, 0.00) AS returns#39, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#40] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#34, profit_loss#35] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#45] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#43] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cs_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] +Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum#44, sum#45] -Results [3]: [cs_call_center_sk#40, sum#46, sum#47] +Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum#46, sum#47] +Results [3]: [cs_call_center_sk#41, sum#48, sum#49] (41) Exchange -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] (42) HashAggregate [codegen id : 14] -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] -Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] +Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] (43) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +Output [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#55), dynamicpruningexpression(cr_returned_date_sk#55 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#57), dynamicpruningexpression(cr_returned_date_sk#57 IN dynamicpruning#5)] ReadSchema: struct (44) ColumnarToRow [codegen id : 12] -Input [3]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55] +Input [3]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57] (45) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#58] (46) BroadcastHashJoin [codegen id : 12] -Left keys [1]: [cr_returned_date_sk#55] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#57] +Right keys [1]: [d_date_sk#58] Join condition: None (47) Project [codegen id : 12] -Output [2]: [cr_return_amount#53, cr_net_loss#54] -Input [4]: [cr_return_amount#53, cr_net_loss#54, cr_returned_date_sk#55, d_date_sk#6] +Output [2]: [cr_return_amount#55, cr_net_loss#56] +Input [4]: [cr_return_amount#55, cr_net_loss#56, cr_returned_date_sk#57, d_date_sk#58] (48) HashAggregate [codegen id : 12] -Input [2]: [cr_return_amount#53, cr_net_loss#54] +Input [2]: [cr_return_amount#55, cr_net_loss#56] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#53)), partial_sum(UnscaledValue(cr_net_loss#54))] -Aggregate Attributes [2]: [sum#56, sum#57] -Results [2]: [sum#58, sum#59] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#55)), partial_sum(UnscaledValue(cr_net_loss#56))] +Aggregate Attributes [2]: [sum#59, sum#60] +Results [2]: [sum#61, sum#62] (49) Exchange -Input [2]: [sum#58, sum#59] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#60] +Input [2]: [sum#61, sum#62] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#63] (50) HashAggregate [codegen id : 13] -Input [2]: [sum#58, sum#59] +Input [2]: [sum#61, sum#62] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#53)), sum(UnscaledValue(cr_net_loss#54))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#53))#61, sum(UnscaledValue(cr_net_loss#54))#62] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#53))#61,17,2) AS returns#63, MakeDecimal(sum(UnscaledValue(cr_net_loss#54))#62,17,2) AS profit_loss#64] +Functions [2]: [sum(UnscaledValue(cr_return_amount#55)), sum(UnscaledValue(cr_net_loss#56))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#55))#64, sum(UnscaledValue(cr_net_loss#56))#65] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#55))#64,17,2) AS returns#66, MakeDecimal(sum(UnscaledValue(cr_net_loss#56))#65,17,2) AS profit_loss#67] (51) BroadcastExchange -Input [2]: [returns#63, profit_loss#64] -Arguments: IdentityBroadcastMode, [id=#65] +Input [2]: [returns#66, profit_loss#67] +Arguments: IdentityBroadcastMode, [id=#68] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#66, cs_call_center_sk#40 AS id#67, sales#51, returns#63, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#64 as decimal(18,2)))), DecimalType(18,2), true) AS profit#68] -Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#63, profit_loss#64] +Output [5]: [catalog channel AS channel#69, cs_call_center_sk#41 AS id#70, sales#53, returns#66, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#67 as decimal(18,2)))), DecimalType(18,2), true) AS profit#71] +Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#66, profit_loss#67] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] -Condition : isnotnull(ws_web_page_sk#69) +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Condition : isnotnull(ws_web_page_sk#72) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#76] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#72] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#75] +Right keys [1]: [d_date_sk#76] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] -Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] +Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] +Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#73] +Output [1]: [wp_web_page_sk#77] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#73] +Input [1]: [wp_web_page_sk#77] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#73] -Condition : isnotnull(wp_web_page_sk#73) +Input [1]: [wp_web_page_sk#77] +Condition : isnotnull(wp_web_page_sk#77) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [1]: [wp_web_page_sk#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#69] -Right keys [1]: [wp_web_page_sk#73] +Left keys [1]: [ws_web_page_sk#72] +Right keys [1]: [wp_web_page_sk#77] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum#75, sum#76] -Results [3]: [wp_web_page_sk#73, sum#77, sum#78] +Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum#79, sum#80] +Results [3]: [wp_web_page_sk#77, sum#81, sum#82] (67) Exchange -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] -Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] +Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : isnotnull(wr_web_page_sk#84) +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Condition : isnotnull(wr_web_page_sk#88) (72) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#88] +Output [1]: [wp_web_page_sk#92] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#84] -Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] +Left keys [1]: [wr_web_page_sk#88] +Right keys [1]: [cast(wp_web_page_sk#92 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [4]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] -Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88] +Output [4]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] +Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92] (75) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#93] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#87] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#91] +Right keys [1]: [cast(d_date_sk#93 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Input [5]: [wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, wp_web_page_sk#88, d_date_sk#6] +Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] +Input [5]: [wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, wp_web_page_sk#92, d_date_sk#93] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum#89, sum#90] -Results [3]: [wp_web_page_sk#88, sum#91, sum#92] +Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#92] +Keys [1]: [wp_web_page_sk#92] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum#94, sum#95] +Results [3]: [wp_web_page_sk#92, sum#96, sum#97] (79) Exchange -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [3]: [wp_web_page_sk#92, sum#96, sum#97] +Arguments: hashpartitioning(wp_web_page_sk#92, 5), ENSURE_REQUIREMENTS, [id=#98] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] -Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] +Input [3]: [wp_web_page_sk#92, sum#96, sum#97] +Keys [1]: [wp_web_page_sk#92] +Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] +Results [3]: [wp_web_page_sk#92, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] +Input [3]: [wp_web_page_sk#92, returns#101, profit_loss#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#73] -Right keys [1]: [wp_web_page_sk#88] +Left keys [1]: [wp_web_page_sk#77] +Right keys [1]: [wp_web_page_sk#92] Join condition: None (83) Project [codegen id : 22] -Output [5]: [web channel AS channel#99, wp_web_page_sk#73 AS id#100, sales#82, coalesce(returns#96, 0.00) AS returns#101, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#102] -Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] +Output [5]: [web channel AS channel#104, wp_web_page_sk#77 AS id#105, sales#86, coalesce(returns#101, 0.00) AS returns#106, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#107] +Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#92, returns#101, profit_loss#102] (84) Union (85) HashAggregate [codegen id : 23] -Input [5]: [channel#36, id#37, sales#18, returns#38, profit#39] -Keys [2]: [channel#36, id#37] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#38), partial_sum(profit#39)] -Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Results [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Input [5]: [channel#37, id#38, sales#18, returns#39, profit#40] +Keys [2]: [channel#37, id#38] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#39), partial_sum(profit#40)] +Aggregate Attributes [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +Results [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] (86) Exchange -Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#36, id#37, 5), ENSURE_REQUIREMENTS, [id=#115] +Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [id=#120] (87) HashAggregate [codegen id : 24] -Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#18)#116, sum(returns#38)#117, sum(profit#39)#118] -Results [5]: [channel#36, id#37, cast(sum(sales#18)#116 as decimal(37,2)) AS sales#119, cast(sum(returns#38)#117 as decimal(37,2)) AS returns#120, cast(sum(profit#39)#118 as decimal(38,2)) AS profit#121] +Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#40)] +Aggregate Attributes [3]: [sum(sales#18)#121, sum(returns#39)#122, sum(profit#40)#123] +Results [5]: [channel#37, id#38, cast(sum(sales#18)#121 as decimal(37,2)) AS sales#124, cast(sum(returns#39)#122 as decimal(37,2)) AS returns#125, cast(sum(profit#40)#123 as decimal(38,2)) AS profit#126] (88) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] +Output [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] (89) HashAggregate [codegen id : 48] -Input [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#128)] -Aggregate Attributes [3]: [sum(sales#18)#129, sum(returns#38)#130, sum(profit#128)#131] -Results [4]: [channel#36, sum(sales#18)#129 AS sales#132, sum(returns#38)#130 AS returns#133, sum(profit#128)#131 AS profit#134] +Input [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#133)] +Aggregate Attributes [3]: [sum(sales#18)#134, sum(returns#39)#135, sum(profit#133)#136] +Results [4]: [channel#37, sum(sales#18)#134 AS sales#137, sum(returns#39)#135 AS returns#138, sum(profit#133)#136 AS profit#139] (90) HashAggregate [codegen id : 48] -Input [4]: [channel#36, sales#132, returns#133, profit#134] -Keys [1]: [channel#36] -Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] -Aggregate Attributes [6]: [sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Results [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Input [4]: [channel#37, sales#137, returns#138, profit#139] +Keys [1]: [channel#37] +Functions [3]: [partial_sum(sales#137), partial_sum(returns#138), partial_sum(profit#139)] +Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Results [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] (91) Exchange -Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Arguments: hashpartitioning(channel#36, 5), ENSURE_REQUIREMENTS, [id=#147] +Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [id=#152] (92) HashAggregate [codegen id : 49] -Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Keys [1]: [channel#36] -Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] -Aggregate Attributes [3]: [sum(sales#132)#148, sum(returns#133)#149, sum(profit#134)#150] -Results [5]: [channel#36, null AS id#151, sum(sales#132)#148 AS sales#152, sum(returns#133)#149 AS returns#153, sum(profit#134)#150 AS profit#154] +Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Keys [1]: [channel#37] +Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] +Aggregate Attributes [3]: [sum(sales#137)#153, sum(returns#138)#154, sum(profit#139)#155] +Results [5]: [channel#37, null AS id#156, sum(sales#137)#153 AS sales#157, sum(returns#138)#154 AS returns#158, sum(profit#139)#155 AS profit#159] (93) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] +Output [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] (94) HashAggregate [codegen id : 73] -Input [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#161)] -Aggregate Attributes [3]: [sum(sales#18)#162, sum(returns#38)#163, sum(profit#161)#164] -Results [3]: [sum(sales#18)#162 AS sales#132, sum(returns#38)#163 AS returns#133, sum(profit#161)#164 AS profit#134] +Input [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#166)] +Aggregate Attributes [3]: [sum(sales#18)#167, sum(returns#39)#168, sum(profit#166)#169] +Results [3]: [sum(sales#18)#167 AS sales#137, sum(returns#39)#168 AS returns#138, sum(profit#166)#169 AS profit#139] (95) HashAggregate [codegen id : 73] -Input [3]: [sales#132, returns#133, profit#134] +Input [3]: [sales#137, returns#138, profit#139] Keys: [] -Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] -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#137), partial_sum(returns#138), partial_sum(profit#139)] +Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Results [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] (96) Exchange -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#177] +Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#182] (97) HashAggregate [codegen id : 74] -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Keys: [] -Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] -Aggregate Attributes [3]: [sum(sales#132)#178, sum(returns#133)#179, sum(profit#134)#180] -Results [5]: [null AS channel#181, null AS id#182, sum(sales#132)#178 AS sales#183, sum(returns#133)#179 AS returns#184, sum(profit#134)#180 AS profit#185] +Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] +Aggregate Attributes [3]: [sum(sales#137)#183, sum(returns#138)#184, sum(profit#139)#185] +Results [5]: [null AS channel#186, null AS id#187, sum(sales#137)#183 AS sales#188, sum(returns#138)#184 AS returns#189, sum(profit#139)#185 AS profit#190] (98) Union (99) HashAggregate [codegen id : 75] -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] (100) Exchange -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Arguments: hashpartitioning(channel#36, id#37, sales#119, returns#120, profit#121, 5), ENSURE_REQUIREMENTS, [id=#186] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Arguments: hashpartitioning(channel#37, id#38, sales#124, returns#125, profit#126, 5), ENSURE_REQUIREMENTS, [id=#191] (101) HashAggregate [codegen id : 76] -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] (102) TakeOrderedAndProject -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Arguments: 100, [channel#36 ASC NULLS FIRST, id#37 ASC NULLS FIRST], [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#124, returns#125, profit#126] ===== Subqueries ===== @@ -611,14 +611,14 @@ ReusedExchange (104) (104) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#26] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#55 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 43 Hosting Expression = cr_returned_date_sk#57 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 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 0ec2ed7cca356..705277b65ff0a 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 @@ -210,392 +210,392 @@ Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_s Condition : isnotnull(sr_store_sk#20) (23) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#25] (24) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_returned_date_sk#23] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Right keys [1]: [cast(d_date_sk#25 as bigint)] Join condition: None (25) Project [codegen id : 6] Output [3]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22] -Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#6] +Input [5]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, sr_returned_date_sk#23, d_date_sk#25] (26) ReusedExchange [Reuses operator id: 14] -Output [1]: [s_store_sk#25] +Output [1]: [s_store_sk#26] (27) BroadcastHashJoin [codegen id : 6] Left keys [1]: [sr_store_sk#20] -Right keys [1]: [cast(s_store_sk#25 as bigint)] +Right keys [1]: [cast(s_store_sk#26 as bigint)] Join condition: None (28) Project [codegen id : 6] -Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#25] +Output [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] +Input [4]: [sr_store_sk#20, sr_return_amt#21, sr_net_loss#22, s_store_sk#26] (29) HashAggregate [codegen id : 6] -Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#25] -Keys [1]: [s_store_sk#25] +Input [3]: [sr_return_amt#21, sr_net_loss#22, s_store_sk#26] +Keys [1]: [s_store_sk#26] Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#21)), partial_sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum#26, sum#27] -Results [3]: [s_store_sk#25, sum#28, sum#29] +Aggregate Attributes [2]: [sum#27, sum#28] +Results [3]: [s_store_sk#26, sum#29, sum#30] (30) Exchange -Input [3]: [s_store_sk#25, sum#28, sum#29] -Arguments: hashpartitioning(s_store_sk#25, 5), ENSURE_REQUIREMENTS, [id=#30] +Input [3]: [s_store_sk#26, sum#29, sum#30] +Arguments: hashpartitioning(s_store_sk#26, 5), ENSURE_REQUIREMENTS, [id=#31] (31) HashAggregate [codegen id : 7] -Input [3]: [s_store_sk#25, sum#28, sum#29] -Keys [1]: [s_store_sk#25] +Input [3]: [s_store_sk#26, sum#29, sum#30] +Keys [1]: [s_store_sk#26] Functions [2]: [sum(UnscaledValue(sr_return_amt#21)), sum(UnscaledValue(sr_net_loss#22))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#31, sum(UnscaledValue(sr_net_loss#22))#32] -Results [3]: [s_store_sk#25, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#31,17,2) AS returns#33, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#32,17,2) AS profit_loss#34] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#21))#32, sum(UnscaledValue(sr_net_loss#22))#33] +Results [3]: [s_store_sk#26, MakeDecimal(sum(UnscaledValue(sr_return_amt#21))#32,17,2) AS returns#34, MakeDecimal(sum(UnscaledValue(sr_net_loss#22))#33,17,2) AS profit_loss#35] (32) BroadcastExchange -Input [3]: [s_store_sk#25, returns#33, profit_loss#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#35] +Input [3]: [s_store_sk#26, returns#34, profit_loss#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#36] (33) BroadcastHashJoin [codegen id : 8] Left keys [1]: [s_store_sk#9] -Right keys [1]: [s_store_sk#25] +Right keys [1]: [s_store_sk#26] Join condition: None (34) Project [codegen id : 8] -Output [5]: [store channel AS channel#36, s_store_sk#9 AS id#37, sales#18, coalesce(returns#33, 0.00) AS returns#38, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#34, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#39] -Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#25, returns#33, profit_loss#34] +Output [5]: [store channel AS channel#37, s_store_sk#9 AS id#38, sales#18, coalesce(returns#34, 0.00) AS returns#39, CheckOverflow((promote_precision(cast(profit#19 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#35, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#40] +Input [6]: [s_store_sk#9, sales#18, profit#19, s_store_sk#26, returns#34, profit_loss#35] (35) Scan parquet default.catalog_sales -Output [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Output [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cs_sold_date_sk#44), dynamicpruningexpression(cs_sold_date_sk#44 IN dynamicpruning#5)] ReadSchema: struct (36) ColumnarToRow [codegen id : 10] -Input [4]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +Input [4]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44] (37) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#45] (38) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#43] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cs_sold_date_sk#44] +Right keys [1]: [d_date_sk#45] Join condition: None (39) Project [codegen id : 10] -Output [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Input [5]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43, d_date_sk#6] +Output [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Input [5]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43, cs_sold_date_sk#44, d_date_sk#45] (40) HashAggregate [codegen id : 10] -Input [3]: [cs_call_center_sk#40, cs_ext_sales_price#41, cs_net_profit#42] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#41)), partial_sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum#44, sum#45] -Results [3]: [cs_call_center_sk#40, sum#46, sum#47] +Input [3]: [cs_call_center_sk#41, cs_ext_sales_price#42, cs_net_profit#43] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#42)), partial_sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum#46, sum#47] +Results [3]: [cs_call_center_sk#41, sum#48, sum#49] (41) Exchange -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Arguments: hashpartitioning(cs_call_center_sk#40, 5), ENSURE_REQUIREMENTS, [id=#48] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Arguments: hashpartitioning(cs_call_center_sk#41, 5), ENSURE_REQUIREMENTS, [id=#50] (42) HashAggregate [codegen id : 11] -Input [3]: [cs_call_center_sk#40, sum#46, sum#47] -Keys [1]: [cs_call_center_sk#40] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#41)), sum(UnscaledValue(cs_net_profit#42))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#41))#49, sum(UnscaledValue(cs_net_profit#42))#50] -Results [3]: [cs_call_center_sk#40, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#41))#49,17,2) AS sales#51, MakeDecimal(sum(UnscaledValue(cs_net_profit#42))#50,17,2) AS profit#52] +Input [3]: [cs_call_center_sk#41, sum#48, sum#49] +Keys [1]: [cs_call_center_sk#41] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#42)), sum(UnscaledValue(cs_net_profit#43))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#42))#51, sum(UnscaledValue(cs_net_profit#43))#52] +Results [3]: [cs_call_center_sk#41, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#42))#51,17,2) AS sales#53, MakeDecimal(sum(UnscaledValue(cs_net_profit#43))#52,17,2) AS profit#54] (43) BroadcastExchange -Input [3]: [cs_call_center_sk#40, sales#51, profit#52] -Arguments: IdentityBroadcastMode, [id=#53] +Input [3]: [cs_call_center_sk#41, sales#53, profit#54] +Arguments: IdentityBroadcastMode, [id=#55] (44) Scan parquet default.catalog_returns -Output [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Output [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(cr_returned_date_sk#58), dynamicpruningexpression(cr_returned_date_sk#58 IN dynamicpruning#5)] ReadSchema: struct (45) ColumnarToRow [codegen id : 13] -Input [3]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Input [3]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58] (46) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#59] (47) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#56] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [cr_returned_date_sk#58] +Right keys [1]: [d_date_sk#59] Join condition: None (48) Project [codegen id : 13] -Output [2]: [cr_return_amount#54, cr_net_loss#55] -Input [4]: [cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56, d_date_sk#6] +Output [2]: [cr_return_amount#56, cr_net_loss#57] +Input [4]: [cr_return_amount#56, cr_net_loss#57, cr_returned_date_sk#58, d_date_sk#59] (49) HashAggregate [codegen id : 13] -Input [2]: [cr_return_amount#54, cr_net_loss#55] +Input [2]: [cr_return_amount#56, cr_net_loss#57] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#54)), partial_sum(UnscaledValue(cr_net_loss#55))] -Aggregate Attributes [2]: [sum#57, sum#58] -Results [2]: [sum#59, sum#60] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#56)), partial_sum(UnscaledValue(cr_net_loss#57))] +Aggregate Attributes [2]: [sum#60, sum#61] +Results [2]: [sum#62, sum#63] (50) Exchange -Input [2]: [sum#59, sum#60] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#61] +Input [2]: [sum#62, sum#63] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#64] (51) HashAggregate -Input [2]: [sum#59, sum#60] +Input [2]: [sum#62, sum#63] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#54)), sum(UnscaledValue(cr_net_loss#55))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#54))#62, sum(UnscaledValue(cr_net_loss#55))#63] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#54))#62,17,2) AS returns#64, MakeDecimal(sum(UnscaledValue(cr_net_loss#55))#63,17,2) AS profit_loss#65] +Functions [2]: [sum(UnscaledValue(cr_return_amount#56)), sum(UnscaledValue(cr_net_loss#57))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#56))#65, sum(UnscaledValue(cr_net_loss#57))#66] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#56))#65,17,2) AS returns#67, MakeDecimal(sum(UnscaledValue(cr_net_loss#57))#66,17,2) AS profit_loss#68] (52) BroadcastNestedLoopJoin [codegen id : 14] Join condition: None (53) Project [codegen id : 14] -Output [5]: [catalog channel AS channel#66, cs_call_center_sk#40 AS id#67, sales#51, returns#64, CheckOverflow((promote_precision(cast(profit#52 as decimal(18,2))) - promote_precision(cast(profit_loss#65 as decimal(18,2)))), DecimalType(18,2), true) AS profit#68] -Input [5]: [cs_call_center_sk#40, sales#51, profit#52, returns#64, profit_loss#65] +Output [5]: [catalog channel AS channel#69, cs_call_center_sk#41 AS id#70, sales#53, returns#67, CheckOverflow((promote_precision(cast(profit#54 as decimal(18,2))) - promote_precision(cast(profit_loss#68 as decimal(18,2)))), DecimalType(18,2), true) AS profit#71] +Input [5]: [cs_call_center_sk#41, sales#53, profit#54, returns#67, profit_loss#68] (54) Scan parquet default.web_sales -Output [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Output [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#5)] +PartitionFilters: [isnotnull(ws_sold_date_sk#75), dynamicpruningexpression(ws_sold_date_sk#75 IN dynamicpruning#5)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] (56) Filter [codegen id : 17] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72] -Condition : isnotnull(ws_web_page_sk#69) +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75] +Condition : isnotnull(ws_web_page_sk#72) (57) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#76] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#72] -Right keys [1]: [d_date_sk#6] +Left keys [1]: [ws_sold_date_sk#75] +Right keys [1]: [d_date_sk#76] Join condition: None (59) Project [codegen id : 17] -Output [3]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71] -Input [5]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, ws_sold_date_sk#72, d_date_sk#6] +Output [3]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74] +Input [5]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, ws_sold_date_sk#75, d_date_sk#76] (60) Scan parquet default.web_page -Output [1]: [wp_web_page_sk#73] +Output [1]: [wp_web_page_sk#77] Batched: true Location [not included in comparison]/{warehouse_dir}/web_page] PushedFilters: [IsNotNull(wp_web_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [1]: [wp_web_page_sk#73] +Input [1]: [wp_web_page_sk#77] (62) Filter [codegen id : 16] -Input [1]: [wp_web_page_sk#73] -Condition : isnotnull(wp_web_page_sk#73) +Input [1]: [wp_web_page_sk#77] +Condition : isnotnull(wp_web_page_sk#77) (63) BroadcastExchange -Input [1]: [wp_web_page_sk#73] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#74] +Input [1]: [wp_web_page_sk#77] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#78] (64) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_web_page_sk#69] -Right keys [1]: [wp_web_page_sk#73] +Left keys [1]: [ws_web_page_sk#72] +Right keys [1]: [wp_web_page_sk#77] Join condition: None (65) Project [codegen id : 17] -Output [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Input [4]: [ws_web_page_sk#69, ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] +Output [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Input [4]: [ws_web_page_sk#72, ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] (66) HashAggregate [codegen id : 17] -Input [3]: [ws_ext_sales_price#70, ws_net_profit#71, wp_web_page_sk#73] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#70)), partial_sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum#75, sum#76] -Results [3]: [wp_web_page_sk#73, sum#77, sum#78] +Input [3]: [ws_ext_sales_price#73, ws_net_profit#74, wp_web_page_sk#77] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#73)), partial_sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum#79, sum#80] +Results [3]: [wp_web_page_sk#77, sum#81, sum#82] (67) Exchange -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Arguments: hashpartitioning(wp_web_page_sk#73, 5), ENSURE_REQUIREMENTS, [id=#79] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, [id=#83] (68) HashAggregate [codegen id : 22] -Input [3]: [wp_web_page_sk#73, sum#77, sum#78] -Keys [1]: [wp_web_page_sk#73] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#70)), sum(UnscaledValue(ws_net_profit#71))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#70))#80, sum(UnscaledValue(ws_net_profit#71))#81] -Results [3]: [wp_web_page_sk#73, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#70))#80,17,2) AS sales#82, MakeDecimal(sum(UnscaledValue(ws_net_profit#71))#81,17,2) AS profit#83] +Input [3]: [wp_web_page_sk#77, sum#81, sum#82] +Keys [1]: [wp_web_page_sk#77] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#73)), sum(UnscaledValue(ws_net_profit#74))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#73))#84, sum(UnscaledValue(ws_net_profit#74))#85] +Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#73))#84,17,2) AS sales#86, MakeDecimal(sum(UnscaledValue(ws_net_profit#74))#85,17,2) AS profit#87] (69) Scan parquet default.web_returns -Output [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Output [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#87), dynamicpruningexpression(wr_returned_date_sk#87 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(wr_returned_date_sk#91), dynamicpruningexpression(wr_returned_date_sk#91 IN dynamicpruning#24)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct (70) ColumnarToRow [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] (71) Filter [codegen id : 20] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] -Condition : isnotnull(wr_web_page_sk#84) +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91] +Condition : isnotnull(wr_web_page_sk#88) (72) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#92] (73) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_returned_date_sk#87] -Right keys [1]: [cast(d_date_sk#6 as bigint)] +Left keys [1]: [wr_returned_date_sk#91] +Right keys [1]: [cast(d_date_sk#92 as bigint)] Join condition: None (74) Project [codegen id : 20] -Output [3]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86] -Input [5]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87, d_date_sk#6] +Output [3]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90] +Input [5]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wr_returned_date_sk#91, d_date_sk#92] (75) ReusedExchange [Reuses operator id: 63] -Output [1]: [wp_web_page_sk#88] +Output [1]: [wp_web_page_sk#93] (76) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [wr_web_page_sk#84] -Right keys [1]: [cast(wp_web_page_sk#88 as bigint)] +Left keys [1]: [wr_web_page_sk#88] +Right keys [1]: [cast(wp_web_page_sk#93 as bigint)] Join condition: None (77) Project [codegen id : 20] -Output [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Input [4]: [wr_web_page_sk#84, wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] +Output [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] +Input [4]: [wr_web_page_sk#88, wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] (78) HashAggregate [codegen id : 20] -Input [3]: [wr_return_amt#85, wr_net_loss#86, wp_web_page_sk#88] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#85)), partial_sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum#89, sum#90] -Results [3]: [wp_web_page_sk#88, sum#91, sum#92] +Input [3]: [wr_return_amt#89, wr_net_loss#90, wp_web_page_sk#93] +Keys [1]: [wp_web_page_sk#93] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#89)), partial_sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum#94, sum#95] +Results [3]: [wp_web_page_sk#93, sum#96, sum#97] (79) Exchange -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Arguments: hashpartitioning(wp_web_page_sk#88, 5), ENSURE_REQUIREMENTS, [id=#93] +Input [3]: [wp_web_page_sk#93, sum#96, sum#97] +Arguments: hashpartitioning(wp_web_page_sk#93, 5), ENSURE_REQUIREMENTS, [id=#98] (80) HashAggregate [codegen id : 21] -Input [3]: [wp_web_page_sk#88, sum#91, sum#92] -Keys [1]: [wp_web_page_sk#88] -Functions [2]: [sum(UnscaledValue(wr_return_amt#85)), sum(UnscaledValue(wr_net_loss#86))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#85))#94, sum(UnscaledValue(wr_net_loss#86))#95] -Results [3]: [wp_web_page_sk#88, MakeDecimal(sum(UnscaledValue(wr_return_amt#85))#94,17,2) AS returns#96, MakeDecimal(sum(UnscaledValue(wr_net_loss#86))#95,17,2) AS profit_loss#97] +Input [3]: [wp_web_page_sk#93, sum#96, sum#97] +Keys [1]: [wp_web_page_sk#93] +Functions [2]: [sum(UnscaledValue(wr_return_amt#89)), sum(UnscaledValue(wr_net_loss#90))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#89))#99, sum(UnscaledValue(wr_net_loss#90))#100] +Results [3]: [wp_web_page_sk#93, MakeDecimal(sum(UnscaledValue(wr_return_amt#89))#99,17,2) AS returns#101, MakeDecimal(sum(UnscaledValue(wr_net_loss#90))#100,17,2) AS profit_loss#102] (81) BroadcastExchange -Input [3]: [wp_web_page_sk#88, returns#96, profit_loss#97] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#98] +Input [3]: [wp_web_page_sk#93, returns#101, profit_loss#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#103] (82) BroadcastHashJoin [codegen id : 22] -Left keys [1]: [wp_web_page_sk#73] -Right keys [1]: [wp_web_page_sk#88] +Left keys [1]: [wp_web_page_sk#77] +Right keys [1]: [wp_web_page_sk#93] Join condition: None (83) Project [codegen id : 22] -Output [5]: [web channel AS channel#99, wp_web_page_sk#73 AS id#100, sales#82, coalesce(returns#96, 0.00) AS returns#101, CheckOverflow((promote_precision(cast(profit#83 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#97, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#102] -Input [6]: [wp_web_page_sk#73, sales#82, profit#83, wp_web_page_sk#88, returns#96, profit_loss#97] +Output [5]: [web channel AS channel#104, wp_web_page_sk#77 AS id#105, sales#86, coalesce(returns#101, 0.00) AS returns#106, CheckOverflow((promote_precision(cast(profit#87 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#102, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS profit#107] +Input [6]: [wp_web_page_sk#77, sales#86, profit#87, wp_web_page_sk#93, returns#101, profit_loss#102] (84) Union (85) HashAggregate [codegen id : 23] -Input [5]: [channel#36, id#37, sales#18, returns#38, profit#39] -Keys [2]: [channel#36, id#37] -Functions [3]: [partial_sum(sales#18), partial_sum(returns#38), partial_sum(profit#39)] -Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Results [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Input [5]: [channel#37, id#38, sales#18, returns#39, profit#40] +Keys [2]: [channel#37, id#38] +Functions [3]: [partial_sum(sales#18), partial_sum(returns#39), partial_sum(profit#40)] +Aggregate Attributes [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +Results [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] (86) Exchange -Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#36, id#37, 5), ENSURE_REQUIREMENTS, [id=#115] +Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [id=#120] (87) HashAggregate [codegen id : 24] -Input [8]: [channel#36, id#37, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#18)#116, sum(returns#38)#117, sum(profit#39)#118] -Results [5]: [channel#36, id#37, cast(sum(sales#18)#116 as decimal(37,2)) AS sales#119, cast(sum(returns#38)#117 as decimal(37,2)) AS returns#120, cast(sum(profit#39)#118 as decimal(38,2)) AS profit#121] +Input [8]: [channel#37, id#38, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#40)] +Aggregate Attributes [3]: [sum(sales#18)#121, sum(returns#39)#122, sum(profit#40)#123] +Results [5]: [channel#37, id#38, cast(sum(sales#18)#121 as decimal(37,2)) AS sales#124, cast(sum(returns#39)#122 as decimal(37,2)) AS returns#125, cast(sum(profit#40)#123 as decimal(38,2)) AS profit#126] (88) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] +Output [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] (89) HashAggregate [codegen id : 48] -Input [8]: [channel#36, id#37, sum#122, isEmpty#123, sum#124, isEmpty#125, sum#126, isEmpty#127] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#128)] -Aggregate Attributes [3]: [sum(sales#18)#129, sum(returns#38)#130, sum(profit#128)#131] -Results [4]: [channel#36, sum(sales#18)#129 AS sales#132, sum(returns#38)#130 AS returns#133, sum(profit#128)#131 AS profit#134] +Input [8]: [channel#37, id#38, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#133)] +Aggregate Attributes [3]: [sum(sales#18)#134, sum(returns#39)#135, sum(profit#133)#136] +Results [4]: [channel#37, sum(sales#18)#134 AS sales#137, sum(returns#39)#135 AS returns#138, sum(profit#133)#136 AS profit#139] (90) HashAggregate [codegen id : 48] -Input [4]: [channel#36, sales#132, returns#133, profit#134] -Keys [1]: [channel#36] -Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] -Aggregate Attributes [6]: [sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] -Results [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Input [4]: [channel#37, sales#137, returns#138, profit#139] +Keys [1]: [channel#37] +Functions [3]: [partial_sum(sales#137), partial_sum(returns#138), partial_sum(profit#139)] +Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] +Results [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] (91) Exchange -Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Arguments: hashpartitioning(channel#36, 5), ENSURE_REQUIREMENTS, [id=#147] +Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Arguments: hashpartitioning(channel#37, 5), ENSURE_REQUIREMENTS, [id=#152] (92) HashAggregate [codegen id : 49] -Input [7]: [channel#36, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Keys [1]: [channel#36] -Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] -Aggregate Attributes [3]: [sum(sales#132)#148, sum(returns#133)#149, sum(profit#134)#150] -Results [5]: [channel#36, null AS id#151, sum(sales#132)#148 AS sales#152, sum(returns#133)#149 AS returns#153, sum(profit#134)#150 AS profit#154] +Input [7]: [channel#37, sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +Keys [1]: [channel#37] +Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] +Aggregate Attributes [3]: [sum(sales#137)#153, sum(returns#138)#154, sum(profit#139)#155] +Results [5]: [channel#37, null AS id#156, sum(sales#137)#153 AS sales#157, sum(returns#138)#154 AS returns#158, sum(profit#139)#155 AS profit#159] (93) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] +Output [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] (94) HashAggregate [codegen id : 73] -Input [8]: [channel#36, id#37, sum#155, isEmpty#156, sum#157, isEmpty#158, sum#159, isEmpty#160] -Keys [2]: [channel#36, id#37] -Functions [3]: [sum(sales#18), sum(returns#38), sum(profit#161)] -Aggregate Attributes [3]: [sum(sales#18)#162, sum(returns#38)#163, sum(profit#161)#164] -Results [3]: [sum(sales#18)#162 AS sales#132, sum(returns#38)#163 AS returns#133, sum(profit#161)#164 AS profit#134] +Input [8]: [channel#37, id#38, sum#160, isEmpty#161, sum#162, isEmpty#163, sum#164, isEmpty#165] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#18), sum(returns#39), sum(profit#166)] +Aggregate Attributes [3]: [sum(sales#18)#167, sum(returns#39)#168, sum(profit#166)#169] +Results [3]: [sum(sales#18)#167 AS sales#137, sum(returns#39)#168 AS returns#138, sum(profit#166)#169 AS profit#139] (95) HashAggregate [codegen id : 73] -Input [3]: [sales#132, returns#133, profit#134] +Input [3]: [sales#137, returns#138, profit#139] Keys: [] -Functions [3]: [partial_sum(sales#132), partial_sum(returns#133), partial_sum(profit#134)] -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#137), partial_sum(returns#138), partial_sum(profit#139)] +Aggregate Attributes [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Results [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] (96) Exchange -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#177] +Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#182] (97) HashAggregate [codegen id : 74] -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +Input [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Keys: [] -Functions [3]: [sum(sales#132), sum(returns#133), sum(profit#134)] -Aggregate Attributes [3]: [sum(sales#132)#178, sum(returns#133)#179, sum(profit#134)#180] -Results [5]: [null AS channel#181, null AS id#182, sum(sales#132)#178 AS sales#183, sum(returns#133)#179 AS returns#184, sum(profit#134)#180 AS profit#185] +Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] +Aggregate Attributes [3]: [sum(sales#137)#183, sum(returns#138)#184, sum(profit#139)#185] +Results [5]: [null AS channel#186, null AS id#187, sum(sales#137)#183 AS sales#188, sum(returns#138)#184 AS returns#189, sum(profit#139)#185 AS profit#190] (98) Union (99) HashAggregate [codegen id : 75] -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] (100) Exchange -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Arguments: hashpartitioning(channel#36, id#37, sales#119, returns#120, profit#121, 5), ENSURE_REQUIREMENTS, [id=#186] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Arguments: hashpartitioning(channel#37, id#38, sales#124, returns#125, profit#126, 5), ENSURE_REQUIREMENTS, [id=#191] (101) HashAggregate [codegen id : 76] -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Keys [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Keys [5]: [channel#37, id#38, sales#124, returns#125, profit#126] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#36, id#37, sales#119, returns#120, profit#121] +Results [5]: [channel#37, id#38, sales#124, returns#125, profit#126] (102) TakeOrderedAndProject -Input [5]: [channel#36, id#37, sales#119, returns#120, profit#121] -Arguments: 100, [channel#36 ASC NULLS FIRST, id#37 ASC NULLS FIRST], [channel#36, id#37, sales#119, returns#120, profit#121] +Input [5]: [channel#37, id#38, sales#124, returns#125, profit#126] +Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#124, returns#125, profit#126] ===== Subqueries ===== @@ -611,14 +611,14 @@ ReusedExchange (104) (104) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#6] +Output [1]: [d_date_sk#25] -Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 35 Hosting Expression = cs_sold_date_sk#44 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 44 Hosting Expression = cr_returned_date_sk#58 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 54 Hosting Expression = ws_sold_date_sk#75 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#87 IN dynamicpruning#24 +Subquery:6 Hosting operator id = 69 Hosting Expression = wr_returned_date_sk#91 IN dynamicpruning#24 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt index 0bd6422fffa5a..a12ab50a89f43 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt @@ -248,164 +248,164 @@ Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale Input [9]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_order_number#33, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, cr_item_sk#39, cr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#43, d_year#44] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [cs_sold_date_sk#37] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#43] Join condition: None (41) Project [codegen id : 13] -Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] -Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#14, d_year#15] +Output [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] +Input [8]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, cs_sold_date_sk#37, d_date_sk#43, d_year#44] (42) HashAggregate [codegen id : 13] -Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#15] -Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [cs_bill_customer_sk#31, cs_item_sk#32, cs_quantity#34, cs_wholesale_cost#35, cs_sales_price#36, d_year#44] +Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [partial_sum(cs_quantity#34), partial_sum(UnscaledValue(cs_wholesale_cost#35)), partial_sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum#43, sum#44, sum#45] -Results [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] +Aggregate Attributes [3]: [sum#45, sum#46, sum#47] +Results [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] (43) Exchange -Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] -Arguments: hashpartitioning(d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] +Arguments: hashpartitioning(d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, 5), ENSURE_REQUIREMENTS, [id=#51] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31, sum#46, sum#47, sum#48] -Keys [3]: [d_year#15, cs_item_sk#32, cs_bill_customer_sk#31] +Input [6]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31, sum#48, sum#49, sum#50] +Keys [3]: [d_year#44, cs_item_sk#32, cs_bill_customer_sk#31] Functions [3]: [sum(cs_quantity#34), sum(UnscaledValue(cs_wholesale_cost#35)), sum(UnscaledValue(cs_sales_price#36))] -Aggregate Attributes [3]: [sum(cs_quantity#34)#50, sum(UnscaledValue(cs_wholesale_cost#35))#51, sum(UnscaledValue(cs_sales_price#36))#52] -Results [6]: [d_year#15 AS cs_sold_year#53, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#54, sum(cs_quantity#34)#50 AS cs_qty#55, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#51,17,2) AS cs_wc#56, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#52,17,2) AS cs_sp#57] +Aggregate Attributes [3]: [sum(cs_quantity#34)#52, sum(UnscaledValue(cs_wholesale_cost#35))#53, sum(UnscaledValue(cs_sales_price#36))#54] +Results [6]: [d_year#44 AS cs_sold_year#55, cs_item_sk#32, cs_bill_customer_sk#31 AS cs_customer_sk#56, sum(cs_quantity#34)#52 AS cs_qty#57, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#35))#53,17,2) AS cs_wc#58, MakeDecimal(sum(UnscaledValue(cs_sales_price#36))#54,17,2) AS cs_sp#59] (45) Filter [codegen id : 14] -Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] -Condition : (coalesce(cs_qty#55, 0) > 0) +Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] +Condition : (coalesce(cs_qty#57, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] -Arguments: [cs_sold_year#53 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] +Arguments: [cs_sold_year#55 ASC NULLS FIRST, cs_item_sk#32 ASC NULLS FIRST, cs_customer_sk#56 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54] +Right keys [3]: [cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#53, cs_item_sk#32, cs_customer_sk#54, cs_qty#55, cs_wc#56, cs_sp#57] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_sold_year#55, cs_item_sk#32, cs_customer_sk#56, cs_qty#57, cs_wc#58, cs_sp#59] (49) Scan parquet default.web_sales -Output [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Output [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#64), dynamicpruningexpression(ws_sold_date_sk#64 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#66), dynamicpruningexpression(ws_sold_date_sk#66 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] (51) Filter [codegen id : 16] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Condition : (isnotnull(ws_item_sk#58) AND isnotnull(ws_bill_customer_sk#59)) +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Condition : (isnotnull(ws_item_sk#60) AND isnotnull(ws_bill_customer_sk#61)) (52) Exchange -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Arguments: hashpartitioning(cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint), 5), ENSURE_REQUIREMENTS, [id=#65] +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Arguments: hashpartitioning(cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint), 5), ENSURE_REQUIREMENTS, [id=#67] (53) Sort [codegen id : 17] -Input [7]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Arguments: [cast(ws_order_number#60 as bigint) ASC NULLS FIRST, cast(ws_item_sk#58 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Arguments: [cast(ws_order_number#62 as bigint) ASC NULLS FIRST, cast(ws_item_sk#60 as bigint) ASC NULLS FIRST], false, 0 (54) Scan parquet default.web_returns -Output [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Output [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] (56) Filter [codegen id : 18] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] -Condition : (isnotnull(wr_order_number#67) AND isnotnull(wr_item_sk#66)) +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] +Condition : (isnotnull(wr_order_number#69) AND isnotnull(wr_item_sk#68)) (57) Project [codegen id : 18] -Output [2]: [wr_item_sk#66, wr_order_number#67] -Input [3]: [wr_item_sk#66, wr_order_number#67, wr_returned_date_sk#68] +Output [2]: [wr_item_sk#68, wr_order_number#69] +Input [3]: [wr_item_sk#68, wr_order_number#69, wr_returned_date_sk#70] (58) Exchange -Input [2]: [wr_item_sk#66, wr_order_number#67] -Arguments: hashpartitioning(wr_order_number#67, wr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [wr_item_sk#68, wr_order_number#69] +Arguments: hashpartitioning(wr_order_number#69, wr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] (59) Sort [codegen id : 19] -Input [2]: [wr_item_sk#66, wr_order_number#67] -Arguments: [wr_order_number#67 ASC NULLS FIRST, wr_item_sk#66 ASC NULLS FIRST], false, 0 +Input [2]: [wr_item_sk#68, wr_order_number#69] +Arguments: [wr_order_number#69 ASC NULLS FIRST, wr_item_sk#68 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cast(ws_order_number#60 as bigint), cast(ws_item_sk#58 as bigint)] -Right keys [2]: [wr_order_number#67, wr_item_sk#66] +Left keys [2]: [cast(ws_order_number#62 as bigint), cast(ws_item_sk#60 as bigint)] +Right keys [2]: [wr_order_number#69, wr_item_sk#68] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] -Condition : isnull(wr_order_number#67) +Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] +Condition : isnull(wr_order_number#69) (62) Project [codegen id : 21] -Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64] -Input [9]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_order_number#60, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, wr_item_sk#66, wr_order_number#67] +Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66] +Input [9]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_order_number#62, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, wr_item_sk#68, wr_order_number#69] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#72, d_year#73] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [ws_sold_date_sk#64] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [ws_sold_date_sk#66] +Right keys [1]: [d_date_sk#72] Join condition: None (65) Project [codegen id : 21] -Output [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] -Input [8]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, ws_sold_date_sk#64, d_date_sk#14, d_year#15] +Output [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] +Input [8]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, ws_sold_date_sk#66, d_date_sk#72, d_year#73] (66) HashAggregate [codegen id : 21] -Input [6]: [ws_item_sk#58, ws_bill_customer_sk#59, ws_quantity#61, ws_wholesale_cost#62, ws_sales_price#63, d_year#15] -Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] -Functions [3]: [partial_sum(ws_quantity#61), partial_sum(UnscaledValue(ws_wholesale_cost#62)), partial_sum(UnscaledValue(ws_sales_price#63))] -Aggregate Attributes [3]: [sum#70, sum#71, sum#72] -Results [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] +Input [6]: [ws_item_sk#60, ws_bill_customer_sk#61, ws_quantity#63, ws_wholesale_cost#64, ws_sales_price#65, d_year#73] +Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] +Functions [3]: [partial_sum(ws_quantity#63), partial_sum(UnscaledValue(ws_wholesale_cost#64)), partial_sum(UnscaledValue(ws_sales_price#65))] +Aggregate Attributes [3]: [sum#74, sum#75, sum#76] +Results [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] (67) Exchange -Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] +Arguments: hashpartitioning(d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, 5), ENSURE_REQUIREMENTS, [id=#80] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59, sum#73, sum#74, sum#75] -Keys [3]: [d_year#15, ws_item_sk#58, ws_bill_customer_sk#59] -Functions [3]: [sum(ws_quantity#61), sum(UnscaledValue(ws_wholesale_cost#62)), sum(UnscaledValue(ws_sales_price#63))] -Aggregate Attributes [3]: [sum(ws_quantity#61)#77, sum(UnscaledValue(ws_wholesale_cost#62))#78, sum(UnscaledValue(ws_sales_price#63))#79] -Results [6]: [d_year#15 AS ws_sold_year#80, ws_item_sk#58, ws_bill_customer_sk#59 AS ws_customer_sk#81, sum(ws_quantity#61)#77 AS ws_qty#82, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#62))#78,17,2) AS ws_wc#83, MakeDecimal(sum(UnscaledValue(ws_sales_price#63))#79,17,2) AS ws_sp#84] +Input [6]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61, sum#77, sum#78, sum#79] +Keys [3]: [d_year#73, ws_item_sk#60, ws_bill_customer_sk#61] +Functions [3]: [sum(ws_quantity#63), sum(UnscaledValue(ws_wholesale_cost#64)), sum(UnscaledValue(ws_sales_price#65))] +Aggregate Attributes [3]: [sum(ws_quantity#63)#81, sum(UnscaledValue(ws_wholesale_cost#64))#82, sum(UnscaledValue(ws_sales_price#65))#83] +Results [6]: [d_year#73 AS ws_sold_year#84, ws_item_sk#60, ws_bill_customer_sk#61 AS ws_customer_sk#85, sum(ws_quantity#63)#81 AS ws_qty#86, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#64))#82,17,2) AS ws_wc#87, MakeDecimal(sum(UnscaledValue(ws_sales_price#65))#83,17,2) AS ws_sp#88] (69) Filter [codegen id : 22] -Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] -Condition : (coalesce(ws_qty#82, 0) > 0) +Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] +Condition : (coalesce(ws_qty#86, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] -Arguments: [ws_sold_year#80 ASC NULLS FIRST, ws_item_sk#58 ASC NULLS FIRST, ws_customer_sk#81 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] +Arguments: [ws_sold_year#84 ASC NULLS FIRST, ws_item_sk#60 ASC NULLS FIRST, ws_customer_sk#85 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81] +Right keys [3]: [ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85] Join condition: None (72) Project [codegen id : 23] -Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#82 + cs_qty#55), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#82, 0) + coalesce(cs_qty#55, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#83, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#56, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#84, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#57, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#55, cs_wc#56, cs_sp#57, ws_sold_year#80, ws_item_sk#58, ws_customer_sk#81, ws_qty#82, ws_wc#83, ws_sp#84] +Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#86 + cs_qty#57), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#86, 0) + coalesce(cs_qty#57, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#87, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#58, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#88, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#59, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, cs_qty#57, cs_wc#58, cs_sp#59, ws_sold_year#84, ws_item_sk#60, ws_customer_sk#85, ws_qty#86, ws_wc#87, ws_sp#88] (73) TakeOrderedAndProject -Input [13]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, ratio#85 ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] +Input [13]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, ratio#89 ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = cs_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#64 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = ws_sold_date_sk#66 IN dynamicpruning#8 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt index 2d84cacd5d09c..b14ab47d28afe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt @@ -248,164 +248,164 @@ Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale Input [9]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_order_number#33, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, wr_item_sk#39, wr_order_number#40] (39) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#43, d_year#44] (40) BroadcastHashJoin [codegen id : 13] Left keys [1]: [ws_sold_date_sk#37] -Right keys [1]: [d_date_sk#14] +Right keys [1]: [d_date_sk#43] Join condition: None (41) Project [codegen id : 13] -Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] -Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#14, d_year#15] +Output [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] +Input [8]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, ws_sold_date_sk#37, d_date_sk#43, d_year#44] (42) HashAggregate [codegen id : 13] -Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#15] -Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [ws_item_sk#31, ws_bill_customer_sk#32, ws_quantity#34, ws_wholesale_cost#35, ws_sales_price#36, d_year#44] +Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [partial_sum(ws_quantity#34), partial_sum(UnscaledValue(ws_wholesale_cost#35)), partial_sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum#43, sum#44, sum#45] -Results [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] +Aggregate Attributes [3]: [sum#45, sum#46, sum#47] +Results [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] (43) Exchange -Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] -Arguments: hashpartitioning(d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#49] +Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] +Arguments: hashpartitioning(d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, 5), ENSURE_REQUIREMENTS, [id=#51] (44) HashAggregate [codegen id : 14] -Input [6]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32, sum#46, sum#47, sum#48] -Keys [3]: [d_year#15, ws_item_sk#31, ws_bill_customer_sk#32] +Input [6]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32, sum#48, sum#49, sum#50] +Keys [3]: [d_year#44, ws_item_sk#31, ws_bill_customer_sk#32] Functions [3]: [sum(ws_quantity#34), sum(UnscaledValue(ws_wholesale_cost#35)), sum(UnscaledValue(ws_sales_price#36))] -Aggregate Attributes [3]: [sum(ws_quantity#34)#50, sum(UnscaledValue(ws_wholesale_cost#35))#51, sum(UnscaledValue(ws_sales_price#36))#52] -Results [6]: [d_year#15 AS ws_sold_year#53, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#54, sum(ws_quantity#34)#50 AS ws_qty#55, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#51,17,2) AS ws_wc#56, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#52,17,2) AS ws_sp#57] +Aggregate Attributes [3]: [sum(ws_quantity#34)#52, sum(UnscaledValue(ws_wholesale_cost#35))#53, sum(UnscaledValue(ws_sales_price#36))#54] +Results [6]: [d_year#44 AS ws_sold_year#55, ws_item_sk#31, ws_bill_customer_sk#32 AS ws_customer_sk#56, sum(ws_quantity#34)#52 AS ws_qty#57, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#35))#53,17,2) AS ws_wc#58, MakeDecimal(sum(UnscaledValue(ws_sales_price#36))#54,17,2) AS ws_sp#59] (45) Filter [codegen id : 14] -Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] -Condition : (coalesce(ws_qty#55, 0) > 0) +Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] +Condition : (coalesce(ws_qty#57, 0) > 0) (46) Sort [codegen id : 14] -Input [6]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] -Arguments: [ws_sold_year#53 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#54 ASC NULLS FIRST], false, 0 +Input [6]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] +Arguments: [ws_sold_year#55 ASC NULLS FIRST, ws_item_sk#31 ASC NULLS FIRST, ws_customer_sk#56 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 15] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54] +Right keys [3]: [ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56] Join condition: None (48) Project [codegen id : 15] -Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57] -Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#53, ws_item_sk#31, ws_customer_sk#54, ws_qty#55, ws_wc#56, ws_sp#57] +Output [9]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59] +Input [12]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_sold_year#55, ws_item_sk#31, ws_customer_sk#56, ws_qty#57, ws_wc#58, ws_sp#59] (49) Scan parquet default.catalog_sales -Output [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Output [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#64), dynamicpruningexpression(cs_sold_date_sk#64 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(cs_sold_date_sk#66), dynamicpruningexpression(cs_sold_date_sk#66 IN dynamicpruning#8)] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 16] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] (51) Filter [codegen id : 16] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Condition : (isnotnull(cs_item_sk#59) AND isnotnull(cs_bill_customer_sk#58)) +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Condition : (isnotnull(cs_item_sk#61) AND isnotnull(cs_bill_customer_sk#60)) (52) Exchange -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Arguments: hashpartitioning(cs_order_number#60, cs_item_sk#59, 5), ENSURE_REQUIREMENTS, [id=#65] +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Arguments: hashpartitioning(cs_order_number#62, cs_item_sk#61, 5), ENSURE_REQUIREMENTS, [id=#67] (53) Sort [codegen id : 17] -Input [7]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Arguments: [cs_order_number#60 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST], false, 0 +Input [7]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Arguments: [cs_order_number#62 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST], false, 0 (54) Scan parquet default.catalog_returns -Output [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Output [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct (55) ColumnarToRow [codegen id : 18] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] (56) Filter [codegen id : 18] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] -Condition : (isnotnull(cr_order_number#67) AND isnotnull(cr_item_sk#66)) +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] +Condition : (isnotnull(cr_order_number#69) AND isnotnull(cr_item_sk#68)) (57) Project [codegen id : 18] -Output [2]: [cr_item_sk#66, cr_order_number#67] -Input [3]: [cr_item_sk#66, cr_order_number#67, cr_returned_date_sk#68] +Output [2]: [cr_item_sk#68, cr_order_number#69] +Input [3]: [cr_item_sk#68, cr_order_number#69, cr_returned_date_sk#70] (58) Exchange -Input [2]: [cr_item_sk#66, cr_order_number#67] -Arguments: hashpartitioning(cr_order_number#67, cr_item_sk#66, 5), ENSURE_REQUIREMENTS, [id=#69] +Input [2]: [cr_item_sk#68, cr_order_number#69] +Arguments: hashpartitioning(cr_order_number#69, cr_item_sk#68, 5), ENSURE_REQUIREMENTS, [id=#71] (59) Sort [codegen id : 19] -Input [2]: [cr_item_sk#66, cr_order_number#67] -Arguments: [cr_order_number#67 ASC NULLS FIRST, cr_item_sk#66 ASC NULLS FIRST], false, 0 +Input [2]: [cr_item_sk#68, cr_order_number#69] +Arguments: [cr_order_number#69 ASC NULLS FIRST, cr_item_sk#68 ASC NULLS FIRST], false, 0 (60) SortMergeJoin -Left keys [2]: [cs_order_number#60, cs_item_sk#59] -Right keys [2]: [cr_order_number#67, cr_item_sk#66] +Left keys [2]: [cs_order_number#62, cs_item_sk#61] +Right keys [2]: [cr_order_number#69, cr_item_sk#68] Join condition: None (61) Filter [codegen id : 21] -Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] -Condition : isnull(cr_order_number#67) +Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] +Condition : isnull(cr_order_number#69) (62) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64] -Input [9]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_order_number#60, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, cr_item_sk#66, cr_order_number#67] +Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66] +Input [9]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_order_number#62, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, cr_item_sk#68, cr_order_number#69] (63) ReusedExchange [Reuses operator id: 18] -Output [2]: [d_date_sk#14, d_year#15] +Output [2]: [d_date_sk#72, d_year#73] (64) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [cs_sold_date_sk#64] -Right keys [1]: [d_date_sk#14] +Left keys [1]: [cs_sold_date_sk#66] +Right keys [1]: [d_date_sk#72] Join condition: None (65) Project [codegen id : 21] -Output [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] -Input [8]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, cs_sold_date_sk#64, d_date_sk#14, d_year#15] +Output [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] +Input [8]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, cs_sold_date_sk#66, d_date_sk#72, d_year#73] (66) HashAggregate [codegen id : 21] -Input [6]: [cs_bill_customer_sk#58, cs_item_sk#59, cs_quantity#61, cs_wholesale_cost#62, cs_sales_price#63, d_year#15] -Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] -Functions [3]: [partial_sum(cs_quantity#61), partial_sum(UnscaledValue(cs_wholesale_cost#62)), partial_sum(UnscaledValue(cs_sales_price#63))] -Aggregate Attributes [3]: [sum#70, sum#71, sum#72] -Results [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] +Input [6]: [cs_bill_customer_sk#60, cs_item_sk#61, cs_quantity#63, cs_wholesale_cost#64, cs_sales_price#65, d_year#73] +Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] +Functions [3]: [partial_sum(cs_quantity#63), partial_sum(UnscaledValue(cs_wholesale_cost#64)), partial_sum(UnscaledValue(cs_sales_price#65))] +Aggregate Attributes [3]: [sum#74, sum#75, sum#76] +Results [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] (67) Exchange -Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, 5), ENSURE_REQUIREMENTS, [id=#76] +Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] +Arguments: hashpartitioning(d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, 5), ENSURE_REQUIREMENTS, [id=#80] (68) HashAggregate [codegen id : 22] -Input [6]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58, sum#73, sum#74, sum#75] -Keys [3]: [d_year#15, cs_item_sk#59, cs_bill_customer_sk#58] -Functions [3]: [sum(cs_quantity#61), sum(UnscaledValue(cs_wholesale_cost#62)), sum(UnscaledValue(cs_sales_price#63))] -Aggregate Attributes [3]: [sum(cs_quantity#61)#77, sum(UnscaledValue(cs_wholesale_cost#62))#78, sum(UnscaledValue(cs_sales_price#63))#79] -Results [6]: [d_year#15 AS cs_sold_year#80, cs_item_sk#59, cs_bill_customer_sk#58 AS cs_customer_sk#81, sum(cs_quantity#61)#77 AS cs_qty#82, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#62))#78,17,2) AS cs_wc#83, MakeDecimal(sum(UnscaledValue(cs_sales_price#63))#79,17,2) AS cs_sp#84] +Input [6]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60, sum#77, sum#78, sum#79] +Keys [3]: [d_year#73, cs_item_sk#61, cs_bill_customer_sk#60] +Functions [3]: [sum(cs_quantity#63), sum(UnscaledValue(cs_wholesale_cost#64)), sum(UnscaledValue(cs_sales_price#65))] +Aggregate Attributes [3]: [sum(cs_quantity#63)#81, sum(UnscaledValue(cs_wholesale_cost#64))#82, sum(UnscaledValue(cs_sales_price#65))#83] +Results [6]: [d_year#73 AS cs_sold_year#84, cs_item_sk#61, cs_bill_customer_sk#60 AS cs_customer_sk#85, sum(cs_quantity#63)#81 AS cs_qty#86, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#64))#82,17,2) AS cs_wc#87, MakeDecimal(sum(UnscaledValue(cs_sales_price#65))#83,17,2) AS cs_sp#88] (69) Filter [codegen id : 22] -Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] -Condition : (coalesce(cs_qty#82, 0) > 0) +Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] +Condition : (coalesce(cs_qty#86, 0) > 0) (70) Sort [codegen id : 22] -Input [6]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] -Arguments: [cs_sold_year#80 ASC NULLS FIRST, cs_item_sk#59 ASC NULLS FIRST, cs_customer_sk#81 ASC NULLS FIRST], false, 0 +Input [6]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] +Arguments: [cs_sold_year#84 ASC NULLS FIRST, cs_item_sk#61 ASC NULLS FIRST, cs_customer_sk#85 ASC NULLS FIRST], false, 0 (71) SortMergeJoin [codegen id : 23] Left keys [3]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81] +Right keys [3]: [cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85] Join condition: None (72) Project [codegen id : 23] -Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#55 + cs_qty#82), 1) as double)), 2) AS ratio#85, ss_qty#28 AS store_qty#86, ss_wc#29 AS store_wholesale_cost#87, ss_sp#30 AS store_sales_price#88, (coalesce(ws_qty#55, 0) + coalesce(cs_qty#82, 0)) AS other_chan_qty#89, CheckOverflow((promote_precision(cast(coalesce(ws_wc#56, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#83, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#90, CheckOverflow((promote_precision(cast(coalesce(ws_sp#57, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#84, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#55, ws_wc#56, ws_sp#57, cs_sold_year#80, cs_item_sk#59, cs_customer_sk#81, cs_qty#82, cs_wc#83, cs_sp#84] +Output [13]: [round((cast(ss_qty#28 as double) / cast(coalesce((ws_qty#57 + cs_qty#86), 1) as double)), 2) AS ratio#89, ss_qty#28 AS store_qty#90, ss_wc#29 AS store_wholesale_cost#91, ss_sp#30 AS store_sales_price#92, (coalesce(ws_qty#57, 0) + coalesce(cs_qty#86, 0)) AS other_chan_qty#93, CheckOverflow((promote_precision(cast(coalesce(ws_wc#58, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_wc#87, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_wholesale_cost#94, CheckOverflow((promote_precision(cast(coalesce(ws_sp#59, 0.00) as decimal(18,2))) + promote_precision(cast(coalesce(cs_sp#88, 0.00) as decimal(18,2)))), DecimalType(18,2), true) AS other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Input [15]: [ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30, ws_qty#57, ws_wc#58, ws_sp#59, cs_sold_year#84, cs_item_sk#61, cs_customer_sk#85, cs_qty#86, cs_wc#87, cs_sp#88] (73) TakeOrderedAndProject -Input [13]: [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] -Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#89 ASC NULLS FIRST, other_chan_wholesale_cost#90 ASC NULLS FIRST, other_chan_sales_price#91 ASC NULLS FIRST, ratio#85 ASC NULLS FIRST], [ratio#85, store_qty#86, store_wholesale_cost#87, store_sales_price#88, other_chan_qty#89, other_chan_wholesale_cost#90, other_chan_sales_price#91] +Input [13]: [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95, ss_sold_year#27, ss_item_sk#1, ss_customer_sk#2, ss_qty#28, ss_wc#29, ss_sp#30] +Arguments: 100, [ss_sold_year#27 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#28 DESC NULLS LAST, ss_wc#29 DESC NULLS LAST, ss_sp#30 DESC NULLS LAST, other_chan_qty#93 ASC NULLS FIRST, other_chan_wholesale_cost#94 ASC NULLS FIRST, other_chan_sales_price#95 ASC NULLS FIRST, ratio#89 ASC NULLS FIRST], [ratio#89, store_qty#90, store_wholesale_cost#91, store_sales_price#92, other_chan_qty#93, other_chan_wholesale_cost#94, other_chan_sales_price#95] ===== Subqueries ===== @@ -418,6 +418,6 @@ Output [2]: [d_date_sk#14, d_year#15] Subquery:2 Hosting operator id = 25 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#64 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 49 Hosting Expression = cs_sold_date_sk#66 IN dynamicpruning#8 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 8a384ec4b1795..a5d47ede475ca 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 @@ -380,224 +380,224 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#16] +Output [1]: [i_item_sk#61] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#16] +Right keys [1]: [i_item_sk#61] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#16] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, i_item_sk#61] (60) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#19] +Output [1]: [p_promo_sk#62] (61) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#19] +Right keys [1]: [p_promo_sk#62] Join condition: None (62) Project [codegen id : 19] Output [6]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58] -Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#19] +Input [8]: [cs_catalog_page_sk#47, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, p_promo_sk#62] (63) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#63] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#22] +Right keys [1]: [d_date_sk#63] Join condition: None (65) Project [codegen id : 19] Output [5]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#22] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#63] (66) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (67) ColumnarToRow [codegen id : 18] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] (68) Filter [codegen id : 18] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Condition : isnotnull(cp_catalog_page_sk#61) +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Condition : isnotnull(cp_catalog_page_sk#64) (69) BroadcastExchange -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#66] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#61] +Right keys [1]: [cp_catalog_page_sk#64] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] +Input [7]: [cs_catalog_page_sk#47, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#64, cp_catalog_page_id#65] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Keys [1]: [cp_catalog_page_id#62] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#65] +Keys [1]: [cp_catalog_page_id#65] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Results [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] (73) Exchange -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [id=#77] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [cp_catalog_page_id#62] +Input [6]: [cp_catalog_page_id#65, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Keys [1]: [cp_catalog_page_id#65] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] -Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_page_id#62) AS id#79, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#80, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#81, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#82] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] +Results [5]: [catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#65) AS id#82, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#83, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#84, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#85] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) (78) Exchange -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] -Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (84) Exchange -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] -Right keys [2]: [wr_item_sk#91, wr_order_number#92] +Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] +Right keys [2]: [wr_item_sk#94, wr_order_number#95] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [i_item_sk#16] +Output [1]: [i_item_sk#100] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#83] -Right keys [1]: [i_item_sk#16] +Left keys [1]: [ws_item_sk#86] +Right keys [1]: [i_item_sk#100] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, i_item_sk#16] +Output [7]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, i_item_sk#100] (91) ReusedExchange [Reuses operator id: 25] -Output [1]: [p_promo_sk#19] +Output [1]: [p_promo_sk#101] (92) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#85] -Right keys [1]: [p_promo_sk#19] +Left keys [1]: [ws_promo_sk#88] +Right keys [1]: [p_promo_sk#101] Join condition: None (93) Project [codegen id : 29] -Output [6]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [8]: [ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, p_promo_sk#19] +Output [6]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [8]: [ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, p_promo_sk#101] (94) ReusedExchange [Reuses operator id: 32] -Output [1]: [d_date_sk#22] +Output [1]: [d_date_sk#102] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#89] -Right keys [1]: [d_date_sk#22] +Left keys [1]: [ws_sold_date_sk#92] +Right keys [1]: [d_date_sk#102] Join condition: None (96) Project [codegen id : 29] -Output [5]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] -Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#22] +Output [5]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] +Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#102] (97) Scan parquet default.web_site -Output [2]: [web_site_sk#97, web_site_id#98] +Output [2]: [web_site_sk#103, web_site_id#104] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (98) ColumnarToRow [codegen id : 28] -Input [2]: [web_site_sk#97, web_site_id#98] +Input [2]: [web_site_sk#103, web_site_id#104] (99) Filter [codegen id : 28] -Input [2]: [web_site_sk#97, web_site_id#98] -Condition : isnotnull(web_site_sk#97) +Input [2]: [web_site_sk#103, web_site_id#104] +Condition : isnotnull(web_site_sk#103) (100) BroadcastExchange -Input [2]: [web_site_sk#97, web_site_id#98] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Input [2]: [web_site_sk#103, web_site_id#104] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#105] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#84] -Right keys [1]: [web_site_sk#97] +Left keys [1]: [ws_web_site_sk#87] +Right keys [1]: [web_site_sk#103] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [7]: [ws_web_site_sk#84, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] +Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] +Input [7]: [ws_web_site_sk#87, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#103, web_site_id#104] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Keys [1]: [web_site_id#98] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#104] +Keys [1]: [web_site_id#104] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Results [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] (104) Exchange -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] +Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Arguments: hashpartitioning(web_site_id#104, 5), ENSURE_REQUIREMENTS, [id=#116] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Keys [1]: [web_site_id#98] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] -Results [5]: [web channel AS channel#114, concat(web_site, web_site_id#98) AS id#115, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#116, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#117, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#118] +Input [6]: [web_site_id#104, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Keys [1]: [web_site_id#104] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] +Results [5]: [web channel AS channel#120, concat(web_site, web_site_id#104) AS id#121, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#122, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#123, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#124] (106) Union @@ -605,99 +605,99 @@ Results [5]: [web channel AS channel#114, concat(web_site, web_site_id#98) AS id Input [5]: [channel#42, id#43, sales#44, returns#45, profit#46] Keys [2]: [channel#42, id#43] Functions [3]: [partial_sum(sales#44), partial_sum(returns#45), partial_sum(profit#46)] -Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Results [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Aggregate Attributes [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Results [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] (108) Exchange -Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#131] +Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#137] (109) HashAggregate [codegen id : 32] -Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#132, sum(returns#45)#133, sum(profit#46)#134] -Results [5]: [channel#42, id#43, cast(sum(sales#44)#132 as decimal(37,2)) AS sales#135, cast(sum(returns#45)#133 as decimal(38,2)) AS returns#136, cast(sum(profit#46)#134 as decimal(38,2)) AS profit#137] +Aggregate Attributes [3]: [sum(sales#44)#138, sum(returns#45)#139, sum(profit#46)#140] +Results [5]: [channel#42, id#43, cast(sum(sales#44)#138 as decimal(37,2)) AS sales#141, cast(sum(returns#45)#139 as decimal(38,2)) AS returns#142, cast(sum(profit#46)#140 as decimal(38,2)) AS profit#143] (110) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Output [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] (111) HashAggregate [codegen id : 64] -Input [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Input [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#144, sum(returns#45)#145, sum(profit#46)#146] -Results [4]: [channel#42, sum(sales#44)#144 AS sales#147, sum(returns#45)#145 AS returns#148, sum(profit#46)#146 AS profit#149] +Aggregate Attributes [3]: [sum(sales#44)#150, sum(returns#45)#151, sum(profit#46)#152] +Results [4]: [channel#42, sum(sales#44)#150 AS sales#153, sum(returns#45)#151 AS returns#154, sum(profit#46)#152 AS profit#155] (112) HashAggregate [codegen id : 64] -Input [4]: [channel#42, sales#147, returns#148, profit#149] +Input [4]: [channel#42, sales#153, returns#154, profit#155] Keys [1]: [channel#42] -Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] -Aggregate Attributes [6]: [sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] -Results [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +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#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] (113) Exchange -Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] -Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#162] +Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#168] (114) HashAggregate [codegen id : 65] -Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] Keys [1]: [channel#42] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#163, sum(returns#148)#164, sum(profit#149)#165] -Results [5]: [channel#42, null AS id#166, sum(sales#147)#163 AS sales#167, sum(returns#148)#164 AS returns#168, sum(profit#149)#165 AS profit#169] +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#42, null AS id#172, sum(sales#153)#169 AS sales#173, sum(returns#154)#170 AS returns#174, sum(profit#155)#171 AS profit#175] (115) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Output [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] (116) HashAggregate [codegen id : 97] -Input [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Input [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#176, sum(returns#45)#177, sum(profit#46)#178] -Results [3]: [sum(sales#44)#176 AS sales#147, sum(returns#45)#177 AS returns#148, sum(profit#46)#178 AS profit#149] +Aggregate Attributes [3]: [sum(sales#44)#182, sum(returns#45)#183, sum(profit#46)#184] +Results [3]: [sum(sales#44)#182 AS sales#153, sum(returns#45)#183 AS returns#154, sum(profit#46)#184 AS profit#155] (117) HashAggregate [codegen id : 97] -Input [3]: [sales#147, returns#148, profit#149] +Input [3]: [sales#153, returns#154, profit#155] Keys: [] -Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] -Aggregate Attributes [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] -Results [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Functions [3]: [partial_sum(sales#153), partial_sum(returns#154), partial_sum(profit#155)] +Aggregate Attributes [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Results [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] (118) Exchange -Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#191] +Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#197] (119) HashAggregate [codegen id : 98] -Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] Keys: [] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#192, sum(returns#148)#193, sum(profit#149)#194] -Results [5]: [null AS channel#195, null AS id#196, sum(sales#147)#192 AS sales#197, sum(returns#148)#193 AS returns#198, sum(profit#149)#194 AS profit#199] +Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] +Aggregate Attributes [3]: [sum(sales#153)#198, sum(returns#154)#199, sum(profit#155)#200] +Results [5]: [null AS channel#201, null AS id#202, sum(sales#153)#198 AS sales#203, sum(returns#154)#199 AS returns#204, sum(profit#155)#200 AS profit#205] (120) Union (121) HashAggregate [codegen id : 99] -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] (122) Exchange -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Arguments: hashpartitioning(channel#42, id#43, sales#135, returns#136, profit#137, 5), ENSURE_REQUIREMENTS, [id=#200] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Arguments: hashpartitioning(channel#42, id#43, sales#141, returns#142, profit#143, 5), ENSURE_REQUIREMENTS, [id=#206] (123) HashAggregate [codegen id : 100] -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] (124) TakeOrderedAndProject -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#141, returns#142, profit#143] ===== Subqueries ===== @@ -710,6 +710,6 @@ Output [1]: [d_date_sk#22] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 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 596f9497b5905..9e5bd3aeaa1d1 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 @@ -380,224 +380,224 @@ Output [8]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_ Input [11]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_order_number#50, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_item_sk#55, cr_order_number#56, cr_return_amount#57, cr_net_loss#58] (57) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#61] (58) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_sold_date_sk#53] -Right keys [1]: [d_date_sk#16] +Right keys [1]: [d_date_sk#61] Join condition: None (59) Project [codegen id : 19] Output [7]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#16] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cs_sold_date_sk#53, cr_return_amount#57, cr_net_loss#58, d_date_sk#61] (60) Scan parquet default.catalog_page -Output [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct (61) ColumnarToRow [codegen id : 16] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] (62) Filter [codegen id : 16] -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Condition : isnotnull(cp_catalog_page_sk#61) +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Condition : isnotnull(cp_catalog_page_sk#62) (63) BroadcastExchange -Input [2]: [cp_catalog_page_sk#61, cp_catalog_page_id#62] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#63] +Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#64] (64) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_catalog_page_sk#47] -Right keys [1]: [cp_catalog_page_sk#61] +Right keys [1]: [cp_catalog_page_sk#62] Join condition: None (65) Project [codegen id : 19] -Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#61, cp_catalog_page_id#62] +Output [7]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [9]: [cs_catalog_page_sk#47, cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_sk#62, cp_catalog_page_id#63] (66) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#22] +Output [1]: [i_item_sk#65] (67) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_item_sk#48] -Right keys [1]: [i_item_sk#22] +Right keys [1]: [i_item_sk#65] Join condition: None (68) Project [codegen id : 19] -Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, i_item_sk#22] +Output [6]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [8]: [cs_item_sk#48, cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, i_item_sk#65] (69) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#25] +Output [1]: [p_promo_sk#66] (70) BroadcastHashJoin [codegen id : 19] Left keys [1]: [cs_promo_sk#49] -Right keys [1]: [p_promo_sk#25] +Right keys [1]: [p_promo_sk#66] Join condition: None (71) Project [codegen id : 19] -Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62, p_promo_sk#25] +Output [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Input [7]: [cs_promo_sk#49, cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63, p_promo_sk#66] (72) HashAggregate [codegen id : 19] -Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#62] -Keys [1]: [cp_catalog_page_id#62] +Input [5]: [cs_ext_sales_price#51, cs_net_profit#52, cr_return_amount#57, cr_net_loss#58, cp_catalog_page_id#63] +Keys [1]: [cp_catalog_page_id#63] Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#51)), partial_sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#64, sum#65, isEmpty#66, sum#67, isEmpty#68] -Results [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] +Aggregate Attributes [5]: [sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] +Results [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] (73) Exchange -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Arguments: hashpartitioning(cp_catalog_page_id#62, 5), ENSURE_REQUIREMENTS, [id=#74] +Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Arguments: hashpartitioning(cp_catalog_page_id#63, 5), ENSURE_REQUIREMENTS, [id=#77] (74) HashAggregate [codegen id : 20] -Input [6]: [cp_catalog_page_id#62, sum#69, sum#70, isEmpty#71, sum#72, isEmpty#73] -Keys [1]: [cp_catalog_page_id#62] +Input [6]: [cp_catalog_page_id#63, sum#72, sum#73, isEmpty#74, sum#75, isEmpty#76] +Keys [1]: [cp_catalog_page_id#63] Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#51)), sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#75, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77] -Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_page_id#62) AS id#79, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#75,17,2) AS sales#80, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#76 AS returns#81, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#77 AS profit#82] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#51))#78, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80] +Results [5]: [catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#63) AS id#82, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#51))#78,17,2) AS sales#83, sum(coalesce(cast(cr_return_amount#57 as decimal(12,2)), 0.00))#79 AS returns#84, sum(CheckOverflow((promote_precision(cast(cs_net_profit#52 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#58 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#80 AS profit#85] (75) Scan parquet default.web_sales -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#89), dynamicpruningexpression(ws_sold_date_sk#89 IN dynamicpruning#8)] +PartitionFilters: [isnotnull(ws_sold_date_sk#92), dynamicpruningexpression(ws_sold_date_sk#92 IN dynamicpruning#8)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct (76) ColumnarToRow [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] (77) Filter [codegen id : 21] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Condition : ((isnotnull(ws_web_site_sk#84) AND isnotnull(ws_item_sk#83)) AND isnotnull(ws_promo_sk#85)) +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Condition : ((isnotnull(ws_web_site_sk#87) AND isnotnull(ws_item_sk#86)) AND isnotnull(ws_promo_sk#88)) (78) Exchange -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: hashpartitioning(cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint), 5), ENSURE_REQUIREMENTS, [id=#90] +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: hashpartitioning(cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint), 5), ENSURE_REQUIREMENTS, [id=#93] (79) Sort [codegen id : 22] -Input [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89] -Arguments: [cast(ws_item_sk#83 as bigint) ASC NULLS FIRST, cast(ws_order_number#86 as bigint) ASC NULLS FIRST], false, 0 +Input [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92] +Arguments: [cast(ws_item_sk#86 as bigint) ASC NULLS FIRST, cast(ws_order_number#89 as bigint) ASC NULLS FIRST], false, 0 (80) Scan parquet default.web_returns -Output [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct (81) ColumnarToRow [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (82) Filter [codegen id : 23] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] -Condition : (isnotnull(wr_item_sk#91) AND isnotnull(wr_order_number#92)) +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Condition : (isnotnull(wr_item_sk#94) AND isnotnull(wr_order_number#95)) (83) Project [codegen id : 23] -Output [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Input [5]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94, wr_returned_date_sk#95] +Output [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] (84) Exchange -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: hashpartitioning(wr_item_sk#91, wr_order_number#92, 5), ENSURE_REQUIREMENTS, [id=#96] +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: hashpartitioning(wr_item_sk#94, wr_order_number#95, 5), ENSURE_REQUIREMENTS, [id=#99] (85) Sort [codegen id : 24] -Input [4]: [wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] -Arguments: [wr_item_sk#91 ASC NULLS FIRST, wr_order_number#92 ASC NULLS FIRST], false, 0 +Input [4]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] +Arguments: [wr_item_sk#94 ASC NULLS FIRST, wr_order_number#95 ASC NULLS FIRST], false, 0 (86) SortMergeJoin -Left keys [2]: [cast(ws_item_sk#83 as bigint), cast(ws_order_number#86 as bigint)] -Right keys [2]: [wr_item_sk#91, wr_order_number#92] +Left keys [2]: [cast(ws_item_sk#86 as bigint), cast(ws_order_number#89 as bigint)] +Right keys [2]: [wr_item_sk#94, wr_order_number#95] Join condition: None (87) Project [codegen id : 29] -Output [8]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94] -Input [11]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_order_number#86, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_item_sk#91, wr_order_number#92, wr_return_amt#93, wr_net_loss#94] +Output [8]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97] +Input [11]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_order_number#89, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97] (88) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#16] +Output [1]: [d_date_sk#100] (89) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_sold_date_sk#89] -Right keys [1]: [d_date_sk#16] +Left keys [1]: [ws_sold_date_sk#92] +Right keys [1]: [d_date_sk#100] Join condition: None (90) Project [codegen id : 29] -Output [7]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, ws_sold_date_sk#89, wr_return_amt#93, wr_net_loss#94, d_date_sk#16] +Output [7]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, ws_sold_date_sk#92, wr_return_amt#96, wr_net_loss#97, d_date_sk#100] (91) Scan parquet default.web_site -Output [2]: [web_site_sk#97, web_site_id#98] +Output [2]: [web_site_sk#101, web_site_id#102] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct (92) ColumnarToRow [codegen id : 26] -Input [2]: [web_site_sk#97, web_site_id#98] +Input [2]: [web_site_sk#101, web_site_id#102] (93) Filter [codegen id : 26] -Input [2]: [web_site_sk#97, web_site_id#98] -Condition : isnotnull(web_site_sk#97) +Input [2]: [web_site_sk#101, web_site_id#102] +Condition : isnotnull(web_site_sk#101) (94) BroadcastExchange -Input [2]: [web_site_sk#97, web_site_id#98] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#99] +Input [2]: [web_site_sk#101, web_site_id#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] (95) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_web_site_sk#84] -Right keys [1]: [web_site_sk#97] +Left keys [1]: [ws_web_site_sk#87] +Right keys [1]: [web_site_sk#101] Join condition: None (96) Project [codegen id : 29] -Output [7]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [9]: [ws_item_sk#83, ws_web_site_sk#84, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_sk#97, web_site_id#98] +Output [7]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [9]: [ws_item_sk#86, ws_web_site_sk#87, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_sk#101, web_site_id#102] (97) ReusedExchange [Reuses operator id: 31] -Output [1]: [i_item_sk#22] +Output [1]: [i_item_sk#104] (98) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_item_sk#83] -Right keys [1]: [i_item_sk#22] +Left keys [1]: [ws_item_sk#86] +Right keys [1]: [i_item_sk#104] Join condition: None (99) Project [codegen id : 29] -Output [6]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [8]: [ws_item_sk#83, ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, i_item_sk#22] +Output [6]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [8]: [ws_item_sk#86, ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, i_item_sk#104] (100) ReusedExchange [Reuses operator id: 38] -Output [1]: [p_promo_sk#25] +Output [1]: [p_promo_sk#105] (101) BroadcastHashJoin [codegen id : 29] -Left keys [1]: [ws_promo_sk#85] -Right keys [1]: [p_promo_sk#25] +Left keys [1]: [ws_promo_sk#88] +Right keys [1]: [p_promo_sk#105] Join condition: None (102) Project [codegen id : 29] -Output [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Input [7]: [ws_promo_sk#85, ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98, p_promo_sk#25] +Output [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Input [7]: [ws_promo_sk#88, ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102, p_promo_sk#105] (103) HashAggregate [codegen id : 29] -Input [5]: [ws_ext_sales_price#87, ws_net_profit#88, wr_return_amt#93, wr_net_loss#94, web_site_id#98] -Keys [1]: [web_site_id#98] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#87)), partial_sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [5]: [sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104] -Results [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] +Input [5]: [ws_ext_sales_price#90, ws_net_profit#91, wr_return_amt#96, wr_net_loss#97, web_site_id#102] +Keys [1]: [web_site_id#102] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#90)), partial_sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [5]: [sum#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Results [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] (104) Exchange -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Arguments: hashpartitioning(web_site_id#98, 5), ENSURE_REQUIREMENTS, [id=#110] +Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Arguments: hashpartitioning(web_site_id#102, 5), ENSURE_REQUIREMENTS, [id=#116] (105) HashAggregate [codegen id : 30] -Input [6]: [web_site_id#98, sum#105, sum#106, isEmpty#107, sum#108, isEmpty#109] -Keys [1]: [web_site_id#98] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#87)), sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#87))#111, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113] -Results [5]: [web channel AS channel#114, concat(web_site, web_site_id#98) AS id#115, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#87))#111,17,2) AS sales#116, sum(coalesce(cast(wr_return_amt#93 as decimal(12,2)), 0.00))#112 AS returns#117, sum(CheckOverflow((promote_precision(cast(ws_net_profit#88 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#94 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#113 AS profit#118] +Input [6]: [web_site_id#102, sum#111, sum#112, isEmpty#113, sum#114, isEmpty#115] +Keys [1]: [web_site_id#102] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#90)), sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#90))#117, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119] +Results [5]: [web channel AS channel#120, concat(web_site, web_site_id#102) AS id#121, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#90))#117,17,2) AS sales#122, sum(coalesce(cast(wr_return_amt#96 as decimal(12,2)), 0.00))#118 AS returns#123, sum(CheckOverflow((promote_precision(cast(ws_net_profit#91 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#97 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2), true))#119 AS profit#124] (106) Union @@ -605,99 +605,99 @@ Results [5]: [web channel AS channel#114, concat(web_site, web_site_id#98) AS id Input [5]: [channel#42, id#43, sales#44, returns#45, profit#46] Keys [2]: [channel#42, id#43] Functions [3]: [partial_sum(sales#44), partial_sum(returns#45), partial_sum(profit#46)] -Aggregate Attributes [6]: [sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] -Results [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Aggregate Attributes [6]: [sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Results [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] (108) Exchange -Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] -Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#131] +Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Arguments: hashpartitioning(channel#42, id#43, 5), ENSURE_REQUIREMENTS, [id=#137] (109) HashAggregate [codegen id : 32] -Input [8]: [channel#42, id#43, sum#125, isEmpty#126, sum#127, isEmpty#128, sum#129, isEmpty#130] +Input [8]: [channel#42, id#43, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#132, sum(returns#45)#133, sum(profit#46)#134] -Results [5]: [channel#42, id#43, cast(sum(sales#44)#132 as decimal(37,2)) AS sales#135, cast(sum(returns#45)#133 as decimal(38,2)) AS returns#136, cast(sum(profit#46)#134 as decimal(38,2)) AS profit#137] +Aggregate Attributes [3]: [sum(sales#44)#138, sum(returns#45)#139, sum(profit#46)#140] +Results [5]: [channel#42, id#43, cast(sum(sales#44)#138 as decimal(37,2)) AS sales#141, cast(sum(returns#45)#139 as decimal(38,2)) AS returns#142, cast(sum(profit#46)#140 as decimal(38,2)) AS profit#143] (110) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Output [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] (111) HashAggregate [codegen id : 64] -Input [8]: [channel#42, id#43, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Input [8]: [channel#42, id#43, sum#144, isEmpty#145, sum#146, isEmpty#147, sum#148, isEmpty#149] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#144, sum(returns#45)#145, sum(profit#46)#146] -Results [4]: [channel#42, sum(sales#44)#144 AS sales#147, sum(returns#45)#145 AS returns#148, sum(profit#46)#146 AS profit#149] +Aggregate Attributes [3]: [sum(sales#44)#150, sum(returns#45)#151, sum(profit#46)#152] +Results [4]: [channel#42, sum(sales#44)#150 AS sales#153, sum(returns#45)#151 AS returns#154, sum(profit#46)#152 AS profit#155] (112) HashAggregate [codegen id : 64] -Input [4]: [channel#42, sales#147, returns#148, profit#149] +Input [4]: [channel#42, sales#153, returns#154, profit#155] Keys [1]: [channel#42] -Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] -Aggregate Attributes [6]: [sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155] -Results [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +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#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] (113) Exchange -Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] -Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#162] +Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] +Arguments: hashpartitioning(channel#42, 5), ENSURE_REQUIREMENTS, [id=#168] (114) HashAggregate [codegen id : 65] -Input [7]: [channel#42, sum#156, isEmpty#157, sum#158, isEmpty#159, sum#160, isEmpty#161] +Input [7]: [channel#42, sum#162, isEmpty#163, sum#164, isEmpty#165, sum#166, isEmpty#167] Keys [1]: [channel#42] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#163, sum(returns#148)#164, sum(profit#149)#165] -Results [5]: [channel#42, null AS id#166, sum(sales#147)#163 AS sales#167, sum(returns#148)#164 AS returns#168, sum(profit#149)#165 AS profit#169] +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#42, null AS id#172, sum(sales#153)#169 AS sales#173, sum(returns#154)#170 AS returns#174, sum(profit#155)#171 AS profit#175] (115) ReusedExchange [Reuses operator id: unknown] -Output [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Output [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] (116) HashAggregate [codegen id : 97] -Input [8]: [channel#42, id#43, sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Input [8]: [channel#42, id#43, sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] Keys [2]: [channel#42, id#43] Functions [3]: [sum(sales#44), sum(returns#45), sum(profit#46)] -Aggregate Attributes [3]: [sum(sales#44)#176, sum(returns#45)#177, sum(profit#46)#178] -Results [3]: [sum(sales#44)#176 AS sales#147, sum(returns#45)#177 AS returns#148, sum(profit#46)#178 AS profit#149] +Aggregate Attributes [3]: [sum(sales#44)#182, sum(returns#45)#183, sum(profit#46)#184] +Results [3]: [sum(sales#44)#182 AS sales#153, sum(returns#45)#183 AS returns#154, sum(profit#46)#184 AS profit#155] (117) HashAggregate [codegen id : 97] -Input [3]: [sales#147, returns#148, profit#149] +Input [3]: [sales#153, returns#154, profit#155] Keys: [] -Functions [3]: [partial_sum(sales#147), partial_sum(returns#148), partial_sum(profit#149)] -Aggregate Attributes [6]: [sum#179, isEmpty#180, sum#181, isEmpty#182, sum#183, isEmpty#184] -Results [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Functions [3]: [partial_sum(sales#153), partial_sum(returns#154), partial_sum(profit#155)] +Aggregate Attributes [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Results [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] (118) Exchange -Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#191] +Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#197] (119) HashAggregate [codegen id : 98] -Input [6]: [sum#185, isEmpty#186, sum#187, isEmpty#188, sum#189, isEmpty#190] +Input [6]: [sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196] Keys: [] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#192, sum(returns#148)#193, sum(profit#149)#194] -Results [5]: [null AS channel#195, null AS id#196, sum(sales#147)#192 AS sales#197, sum(returns#148)#193 AS returns#198, sum(profit#149)#194 AS profit#199] +Functions [3]: [sum(sales#153), sum(returns#154), sum(profit#155)] +Aggregate Attributes [3]: [sum(sales#153)#198, sum(returns#154)#199, sum(profit#155)#200] +Results [5]: [null AS channel#201, null AS id#202, sum(sales#153)#198 AS sales#203, sum(returns#154)#199 AS returns#204, sum(profit#155)#200 AS profit#205] (120) Union (121) HashAggregate [codegen id : 99] -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] (122) Exchange -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Arguments: hashpartitioning(channel#42, id#43, sales#135, returns#136, profit#137, 5), ENSURE_REQUIREMENTS, [id=#200] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Arguments: hashpartitioning(channel#42, id#43, sales#141, returns#142, profit#143, 5), ENSURE_REQUIREMENTS, [id=#206] (123) HashAggregate [codegen id : 100] -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Keys [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Keys [5]: [channel#42, id#43, sales#141, returns#142, profit#143] Functions: [] Aggregate Attributes: [] -Results [5]: [channel#42, id#43, sales#135, returns#136, profit#137] +Results [5]: [channel#42, id#43, sales#141, returns#142, profit#143] (124) TakeOrderedAndProject -Input [5]: [channel#42, id#43, sales#135, returns#136, profit#137] -Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#135, returns#136, profit#137] +Input [5]: [channel#42, id#43, sales#141, returns#142, profit#143] +Arguments: 100, [channel#42 ASC NULLS FIRST, id#43 ASC NULLS FIRST], [channel#42, id#43, sales#141, returns#142, profit#143] ===== Subqueries ===== @@ -710,6 +710,6 @@ Output [1]: [d_date_sk#16] Subquery:2 Hosting operator id = 44 Hosting Expression = cs_sold_date_sk#53 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#89 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 75 Hosting Expression = ws_sold_date_sk#92 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 3333d5d1aed52..64f6270e81183 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 @@ -131,60 +131,60 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#15] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#15,17,2) as decimal(27,2)) AS total_sum#16, i_category#10, i_class#9, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] (20) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#10, i_class#9, sum#20] +Output [3]: [i_category#20, i_class#21, sum#22] (21) HashAggregate [codegen id : 8] -Input [3]: [i_category#10, i_class#9, sum#20] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#21] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#21,17,2) AS total_sum#22, i_category#10] +Input [3]: [i_category#20, i_class#21, sum#22] +Keys [2]: [i_category#20, i_class#21] +Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#23))#24] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#23))#24,17,2) AS total_sum#25, i_category#20] (22) HashAggregate [codegen id : 8] -Input [2]: [total_sum#22, i_category#10] -Keys [1]: [i_category#10] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [3]: [i_category#10, sum#25, isEmpty#26] +Input [2]: [total_sum#25, i_category#20] +Keys [1]: [i_category#20] +Functions [1]: [partial_sum(total_sum#25)] +Aggregate Attributes [2]: [sum#26, isEmpty#27] +Results [3]: [i_category#20, sum#28, isEmpty#29] (23) Exchange -Input [3]: [i_category#10, sum#25, isEmpty#26] -Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [i_category#20, sum#28, isEmpty#29] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#30] (24) HashAggregate [codegen id : 9] -Input [3]: [i_category#10, sum#25, isEmpty#26] -Keys [1]: [i_category#10] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#28] -Results [6]: [sum(total_sum#22)#28 AS total_sum#29, i_category#10, null AS i_class#30, 0 AS g_category#31, 1 AS g_class#32, 1 AS lochierarchy#33] +Input [3]: [i_category#20, sum#28, isEmpty#29] +Keys [1]: [i_category#20] +Functions [1]: [sum(total_sum#25)] +Aggregate Attributes [1]: [sum(total_sum#25)#31] +Results [6]: [sum(total_sum#25)#31 AS total_sum#32, i_category#20, null AS i_class#33, 0 AS g_category#34, 1 AS g_class#35, 1 AS lochierarchy#36] (25) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#10, i_class#9, sum#34] +Output [3]: [i_category#37, i_class#38, sum#39] (26) HashAggregate [codegen id : 13] -Input [3]: [i_category#10, i_class#9, sum#34] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#35] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#35,17,2) AS total_sum#22] +Input [3]: [i_category#37, i_class#38, sum#39] +Keys [2]: [i_category#37, i_class#38] +Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#40))#41] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#40))#41,17,2) AS total_sum#25] (27) HashAggregate [codegen id : 13] -Input [1]: [total_sum#22] +Input [1]: [total_sum#25] Keys: [] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [2]: [sum#38, isEmpty#39] +Functions [1]: [partial_sum(total_sum#25)] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [2]: [sum#44, isEmpty#45] (28) Exchange -Input [2]: [sum#38, isEmpty#39] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] +Input [2]: [sum#44, isEmpty#45] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#46] (29) HashAggregate [codegen id : 14] -Input [2]: [sum#38, isEmpty#39] +Input [2]: [sum#44, isEmpty#45] Keys: [] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#41] -Results [6]: [sum(total_sum#22)#41 AS total_sum#42, null AS i_category#43, null AS i_class#44, 1 AS g_category#45, 1 AS g_class#46, 2 AS lochierarchy#47] +Functions [1]: [sum(total_sum#25)] +Aggregate Attributes [1]: [sum(total_sum#25)#47] +Results [6]: [sum(total_sum#25)#47 AS total_sum#48, null AS i_category#49, null AS i_class#50, 1 AS g_category#51, 1 AS g_class#52, 2 AS lochierarchy#53] (30) Union @@ -197,34 +197,34 @@ Results [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, (32) Exchange Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] -Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#48] +Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#54] (33) HashAggregate [codegen id : 16] Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Keys [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#49] +Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#55] (34) Exchange -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: hashpartitioning(lochierarchy#19, _w0#49, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: hashpartitioning(lochierarchy#19, _w0#55, 5), ENSURE_REQUIREMENTS, [id=#56] (35) Sort [codegen id : 17] -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#49 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 (36) Window -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#49, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#51], [lochierarchy#19, _w0#49], [total_sum#16 DESC NULLS LAST] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#55, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#57], [lochierarchy#19, _w0#55], [total_sum#16 DESC NULLS LAST] (37) Project [codegen id : 18] -Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] -Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49, rank_within_parent#51] +Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] +Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55, rank_within_parent#57] (38) TakeOrderedAndProject -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#51 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] +Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#57 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] ===== Subqueries ===== 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 3333d5d1aed52..64f6270e81183 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 @@ -131,60 +131,60 @@ Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#15] Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#15,17,2) as decimal(27,2)) AS total_sum#16, i_category#10, i_class#9, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] (20) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#10, i_class#9, sum#20] +Output [3]: [i_category#20, i_class#21, sum#22] (21) HashAggregate [codegen id : 8] -Input [3]: [i_category#10, i_class#9, sum#20] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#21] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#21,17,2) AS total_sum#22, i_category#10] +Input [3]: [i_category#20, i_class#21, sum#22] +Keys [2]: [i_category#20, i_class#21] +Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#23))#24] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#23))#24,17,2) AS total_sum#25, i_category#20] (22) HashAggregate [codegen id : 8] -Input [2]: [total_sum#22, i_category#10] -Keys [1]: [i_category#10] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#23, isEmpty#24] -Results [3]: [i_category#10, sum#25, isEmpty#26] +Input [2]: [total_sum#25, i_category#20] +Keys [1]: [i_category#20] +Functions [1]: [partial_sum(total_sum#25)] +Aggregate Attributes [2]: [sum#26, isEmpty#27] +Results [3]: [i_category#20, sum#28, isEmpty#29] (23) Exchange -Input [3]: [i_category#10, sum#25, isEmpty#26] -Arguments: hashpartitioning(i_category#10, 5), ENSURE_REQUIREMENTS, [id=#27] +Input [3]: [i_category#20, sum#28, isEmpty#29] +Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, [id=#30] (24) HashAggregate [codegen id : 9] -Input [3]: [i_category#10, sum#25, isEmpty#26] -Keys [1]: [i_category#10] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#28] -Results [6]: [sum(total_sum#22)#28 AS total_sum#29, i_category#10, null AS i_class#30, 0 AS g_category#31, 1 AS g_class#32, 1 AS lochierarchy#33] +Input [3]: [i_category#20, sum#28, isEmpty#29] +Keys [1]: [i_category#20] +Functions [1]: [sum(total_sum#25)] +Aggregate Attributes [1]: [sum(total_sum#25)#31] +Results [6]: [sum(total_sum#25)#31 AS total_sum#32, i_category#20, null AS i_class#33, 0 AS g_category#34, 1 AS g_class#35, 1 AS lochierarchy#36] (25) ReusedExchange [Reuses operator id: unknown] -Output [3]: [i_category#10, i_class#9, sum#34] +Output [3]: [i_category#37, i_class#38, sum#39] (26) HashAggregate [codegen id : 13] -Input [3]: [i_category#10, i_class#9, sum#34] -Keys [2]: [i_category#10, i_class#9] -Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#35] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#35,17,2) AS total_sum#22] +Input [3]: [i_category#37, i_class#38, sum#39] +Keys [2]: [i_category#37, i_class#38] +Functions [1]: [sum(UnscaledValue(ws_net_paid#40))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#40))#41] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#40))#41,17,2) AS total_sum#25] (27) HashAggregate [codegen id : 13] -Input [1]: [total_sum#22] +Input [1]: [total_sum#25] Keys: [] -Functions [1]: [partial_sum(total_sum#22)] -Aggregate Attributes [2]: [sum#36, isEmpty#37] -Results [2]: [sum#38, isEmpty#39] +Functions [1]: [partial_sum(total_sum#25)] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [2]: [sum#44, isEmpty#45] (28) Exchange -Input [2]: [sum#38, isEmpty#39] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#40] +Input [2]: [sum#44, isEmpty#45] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#46] (29) HashAggregate [codegen id : 14] -Input [2]: [sum#38, isEmpty#39] +Input [2]: [sum#44, isEmpty#45] Keys: [] -Functions [1]: [sum(total_sum#22)] -Aggregate Attributes [1]: [sum(total_sum#22)#41] -Results [6]: [sum(total_sum#22)#41 AS total_sum#42, null AS i_category#43, null AS i_class#44, 1 AS g_category#45, 1 AS g_class#46, 2 AS lochierarchy#47] +Functions [1]: [sum(total_sum#25)] +Aggregate Attributes [1]: [sum(total_sum#25)#47] +Results [6]: [sum(total_sum#25)#47 AS total_sum#48, null AS i_category#49, null AS i_class#50, 1 AS g_category#51, 1 AS g_class#52, 2 AS lochierarchy#53] (30) Union @@ -197,34 +197,34 @@ Results [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, (32) Exchange Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] -Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#48] +Arguments: hashpartitioning(total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19, 5), ENSURE_REQUIREMENTS, [id=#54] (33) HashAggregate [codegen id : 16] Input [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Keys [6]: [total_sum#16, i_category#10, i_class#9, g_category#17, g_class#18, lochierarchy#19] Functions: [] Aggregate Attributes: [] -Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#49] +Results [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, CASE WHEN (g_class#18 = 0) THEN i_category#10 END AS _w0#55] (34) Exchange -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: hashpartitioning(lochierarchy#19, _w0#49, 5), ENSURE_REQUIREMENTS, [id=#50] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: hashpartitioning(lochierarchy#19, _w0#55, 5), ENSURE_REQUIREMENTS, [id=#56] (35) Sort [codegen id : 17] -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#49 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: [lochierarchy#19 ASC NULLS FIRST, _w0#55 ASC NULLS FIRST, total_sum#16 DESC NULLS LAST], false, 0 (36) Window -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49] -Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#49, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#51], [lochierarchy#19, _w0#49], [total_sum#16 DESC NULLS LAST] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55] +Arguments: [rank(total_sum#16) windowspecdefinition(lochierarchy#19, _w0#55, total_sum#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#57], [lochierarchy#19, _w0#55], [total_sum#16 DESC NULLS LAST] (37) Project [codegen id : 18] -Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] -Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#49, rank_within_parent#51] +Output [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] +Input [6]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, _w0#55, rank_within_parent#57] (38) TakeOrderedAndProject -Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#51 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#51] +Input [5]: [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] +Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#57 ASC NULLS FIRST], [total_sum#16, i_category#10, i_class#9, lochierarchy#19, rank_within_parent#57] ===== Subqueries ===== From 5d85df36589160aa779f95cf194b2eec51f5703c Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 29 Mar 2021 20:32:21 +0800 Subject: [PATCH 37/38] match SubqueryExpression --- .../spark/sql/catalyst/analysis/DeduplicateRelations.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index e4a709c1cdf21..140bc15bb6dbb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable.ArrayBuffer -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, AttributeSet, NamedExpression, PlanExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, AttributeSet, NamedExpression, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule @@ -104,7 +104,7 @@ object DeduplicateRelations extends Rule[LogicalPlan] { } val planWithNewSubquery = newPlan.transformExpressions { - case subquery: PlanExpression[LogicalPlan @unchecked] => + case subquery: SubqueryExpression => val (renewed, collected) = renewDuplicatedRelations( existingRelations ++ relations, subquery.plan) relations ++= collected From f0c7ce423009e9465ec614c9e4c64781229e1f19 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 29 Mar 2021 20:33:57 +0800 Subject: [PATCH 38/38] fix scala2.13 error --- .../spark/sql/catalyst/analysis/DeduplicateRelations.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index 140bc15bb6dbb..fdd9df061b5fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -95,9 +95,9 @@ object DeduplicateRelations extends Rule[LogicalPlan] { .flatMap(_.output).zip(newChildren.flatMap(_.output)) .filter { case (a1, a2) => a1.exprId != a2.exprId } ) - plan.withNewChildren(newChildren).rewriteAttrs(attrMap) + plan.withNewChildren(newChildren.toSeq).rewriteAttrs(attrMap) } else { - plan.withNewChildren(newChildren) + plan.withNewChildren(newChildren.toSeq) } } else { plan @@ -110,7 +110,7 @@ object DeduplicateRelations extends Rule[LogicalPlan] { relations ++= collected subquery.withNewPlan(renewed) } - (planWithNewSubquery, relations) + (planWithNewSubquery, relations.toSeq) } private def isDuplicated(